]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
osd: Build ceph-osd without using namespace declarations in headers 34409/head
authorAdam C. Emerson <aemerson@redhat.com>
Thu, 2 Apr 2020 23:32:14 +0000 (19:32 -0400)
committerAdam C. Emerson <aemerson@redhat.com>
Mon, 6 Apr 2020 15:15:06 +0000 (11:15 -0400)
This is part of a series of commits to clean up using namespace at top
level in headers.

Signed-off-by: Adam C. Emerson <aemerson@redhat.com>
42 files changed:
src/ceph_osd.cc
src/mgr/OSDPerfMetricTypes.cc
src/objclass/class_api.cc
src/osd/ECBackend.cc
src/osd/ECBackend.h
src/osd/ECTransaction.cc
src/osd/ECTransaction.h
src/osd/ECUtil.cc
src/osd/ECUtil.h
src/osd/ExtentCache.cc
src/osd/ExtentCache.h
src/osd/MissingLoc.cc
src/osd/MissingLoc.h
src/osd/OSD.cc
src/osd/OSD.h
src/osd/PG.cc
src/osd/PG.h
src/osd/PGBackend.cc
src/osd/PGBackend.h
src/osd/PGLog.cc
src/osd/PGLog.h
src/osd/PGStateUtils.cc
src/osd/PGStateUtils.h
src/osd/PGTransaction.h
src/osd/PeeringState.cc
src/osd/PeeringState.h
src/osd/PrimaryLogPG.cc
src/osd/PrimaryLogPG.h
src/osd/ReplicatedBackend.cc
src/osd/ReplicatedBackend.h
src/osd/ScrubStore.cc
src/osd/ScrubStore.h
src/osd/Session.cc
src/osd/Session.h
src/osd/SnapMapper.cc
src/osd/SnapMapper.h
src/osd/TierAgentState.h
src/osd/Watch.cc
src/osd/objclass.cc
src/osd/object_state.h
src/osd/osd_internal_types.h
src/osd/scheduler/OpSchedulerItem.h

index 1e9516aa30d6250215c4a67eecd883fca33b5fe3..9fde44000288ff98f6964f7f8dfbfca9b3e92670 100644 (file)
 #define dout_context g_ceph_context
 #define dout_subsys ceph_subsys_osd
 
+using std::cerr;
+using std::cout;
+using std::map;
+using std::ostringstream;
+using std::string;
+using std::vector;
+
+using ceph::bufferlist;
+
 namespace {
 
 TracepointProvider::Traits osd_tracepoint_traits("libosd_tp.so",
@@ -241,7 +250,7 @@ int main(int argc, const char **argv)
        try {
          decode(e, p);
        }
-       catch (const buffer::error &e) {
+       catch (const ceph::buffer::error &e) {
          derr << "failed to decode LogEntry at offset " << pos << dendl;
          forker.exit(1);
        }
index 06f7081d9aa42c5ec5bd6e2d4ab0f98a793636e7..bce95e0ae3889730ca6c1ab9d5d5c9dbbc5d175a 100644 (file)
@@ -5,6 +5,8 @@
 
 #include <ostream>
 
+using ceph::bufferlist;
+
 std::ostream& operator<<(std::ostream& os,
                          const OSDPerfMetricSubKeyDescriptor &d) {
   switch(d.type) {
index 2ff595f08a67f085059ed850006f830f5ceb15b2..b50b6caf23f92c1948c0203920c5355c4390de11 100644 (file)
@@ -92,12 +92,14 @@ void cls_unregister_filter(cls_filter_handle_t handle)
   filter->unregister();
 }
 
-int cls_cxx_read(cls_method_context_t hctx, int ofs, int len, bufferlist *outbl)
+int cls_cxx_read(cls_method_context_t hctx, int ofs, int len,
+                ceph::buffer::list *outbl)
 {
   return cls_cxx_read2(hctx, ofs, len, outbl, 0);
 }
 
-int cls_cxx_write(cls_method_context_t hctx, int ofs, int len, bufferlist *inbl)
+int cls_cxx_write(cls_method_context_t hctx, int ofs, int len,
+                 ceph::buffer::list *inbl)
 {
   return cls_cxx_write2(hctx, ofs, len, inbl, 0);
 }
@@ -133,7 +135,7 @@ int cls_gen_rand_base64(char *dest, int size) /* size should be the required str
   return 0;
 }
 
-void cls_cxx_subop_version(cls_method_context_t hctx, string *s)
+void cls_cxx_subop_version(cls_method_context_t hctx, std::string *s)
 {
   if (!s)
     return;
index 68ab48b3fdbd7e1eb0f0f2494eb1a565c420f78e..6c33e6c405248aadecdbc29a6c3f85bf066faaf0 100644 (file)
 #define DOUT_PREFIX_ARGS this
 #undef dout_prefix
 #define dout_prefix _prefix(_dout, this)
+
+using std::dec;
+using std::hex;
+using std::list;
+using std::make_pair;
+using std::map;
+using std::pair;
+using std::ostream;
+using std::set;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+
+using ceph::bufferhash;
+using ceph::bufferlist;
+using ceph::bufferptr;
+using ceph::ErasureCodeInterfaceRef;
+using ceph::Formatter;
+
 static ostream& _prefix(std::ostream *_dout, ECBackend *pgb) {
   return pgb->get_parent()->gen_dbg_prefix(*_dout);
 }
index 9d553bb68ff74e318633463417f74d4888a04b4f..d659d64dff614bae6b7ef5a7e7d0582ae4adf36b 100644 (file)
@@ -92,7 +92,7 @@ public:
   void on_change() override;
   void clear_recovery_state() override;
 
-  void dump_recovery_info(Formatter *f) const override;
+  void dump_recovery_info(ceph::Formatter *f) const override;
 
   void call_write_ordered(std::function<void(void)> &&cb) override;
 
@@ -103,7 +103,7 @@ public:
     PGTransactionUPtr &&t,
     const eversion_t &trim_to,
     const eversion_t &min_last_complete_ondisk,
-    vector<pg_log_entry_t>&& log_entries,
+    std::vector<pg_log_entry_t>&& log_entries,
     std::optional<pg_hit_set_history_t> &hset_history,
     Context *on_all_commit,
     ceph_tid_t tid,
@@ -116,7 +116,7 @@ public:
     uint64_t off,
     uint64_t len,
     uint32_t op_flags,
-    bufferlist *bl) override;
+    ceph::buffer::list *bl) override;
 
   /**
    * Async read mechanism
@@ -126,30 +126,30 @@ public:
    * buffer as well as for calling the callbacks.
    *
    * One tricky bit is that two reads may possibly not read from the same
-   * set of replicas.  This could result in two reads completing in the
+   * std::set of replicas.  This could result in two reads completing in the
    * wrong (from the interface user's point of view) order.  Thus, we
    * maintain a queue of in progress reads (@see in_progress_client_reads)
    * to ensure that we always call the completion callback in order.
    *
    * Another subtly is that while we may read a degraded object, we will
-   * still only perform a client read from shards in the acting set.  This
+   * still only perform a client read from shards in the acting std::set.  This
    * ensures that we won't ever have to restart a client initiated read in
    * check_recovery_sources.
    */
   void objects_read_and_reconstruct(
-    const map<hobject_t, std::list<boost::tuple<uint64_t, uint64_t, uint32_t> >
+    const std::map<hobject_t, std::list<boost::tuple<uint64_t, uint64_t, uint32_t> >
     > &reads,
     bool fast_read,
-    GenContextURef<map<hobject_t,pair<int, extent_map> > &&> &&func);
+    GenContextURef<std::map<hobject_t,std::pair<int, extent_map> > &&> &&func);
 
   friend struct CallClientContexts;
   struct ClientAsyncReadStatus {
     unsigned objects_to_read;
-    GenContextURef<map<hobject_t,pair<int, extent_map> > &&> func;
-    map<hobject_t,pair<int, extent_map> > results;
+    GenContextURef<std::map<hobject_t,std::pair<int, extent_map> > &&> func;
+    std::map<hobject_t,std::pair<int, extent_map> > results;
     explicit ClientAsyncReadStatus(
       unsigned objects_to_read,
-      GenContextURef<map<hobject_t,pair<int, extent_map> > &&> &&func)
+      GenContextURef<std::map<hobject_t,std::pair<int, extent_map> > &&> &&func)
       : objects_to_read(objects_to_read), func(std::move(func)) {}
     void complete_object(
       const hobject_t &hoid,
@@ -158,7 +158,7 @@ public:
       ceph_assert(objects_to_read);
       --objects_to_read;
       ceph_assert(!results.count(hoid));
-      results.emplace(hoid, make_pair(err, std::move(buffers)));
+      results.emplace(hoid, std::make_pair(err, std::move(buffers)));
     }
     bool is_complete() const {
       return objects_to_read == 0;
@@ -167,19 +167,19 @@ public:
       func.release()->complete(std::move(results));
     }
   };
-  list<ClientAsyncReadStatus> in_progress_client_reads;
+  std::list<ClientAsyncReadStatus> in_progress_client_reads;
   void objects_read_async(
     const hobject_t &hoid,
-    const list<pair<boost::tuple<uint64_t, uint64_t, uint32_t>,
-                   pair<bufferlist*, Context*> > > &to_read,
+    const std::list<std::pair<boost::tuple<uint64_t, uint64_t, uint32_t>,
+                   std::pair<ceph::buffer::list*, Context*> > > &to_read,
     Context *on_complete,
     bool fast_read = false) override;
 
   template <typename Func>
   void objects_read_async_no_cache(
-    const map<hobject_t,extent_set> &to_read,
+    const std::map<hobject_t,extent_set> &to_read,
     Func &&on_complete) {
-    map<hobject_t,std::list<boost::tuple<uint64_t, uint64_t, uint32_t> > > _to_read;
+    std::map<hobject_t,std::list<boost::tuple<uint64_t, uint64_t, uint32_t> > > _to_read;
     for (auto &&hpair: to_read) {
       auto &l = _to_read[hpair.first];
       for (auto extent: hpair.second) {
@@ -190,7 +190,7 @@ public:
       _to_read,
       false,
       make_gen_lambda_context<
-      map<hobject_t,pair<int, extent_map> > &&, Func>(
+      std::map<hobject_t,std::pair<int, extent_map> > &&, Func>(
          std::forward<Func>(on_complete)));
   }
   void kick_reads() {
@@ -208,8 +208,8 @@ private:
                        sinfo.get_stripe_width());
   }
 
-  void get_want_to_read_shards(set<int> *want_to_read) const {
-    const vector<int> &chunk_mapping = ec_impl->get_chunk_mapping();
+  void get_want_to_read_shards(std::set<int> *want_to_read) const {
+    const std::vector<int> &chunk_mapping = ec_impl->get_chunk_mapping();
     for (int i = 0; i < (int)ec_impl->get_data_chunk_count(); ++i) {
       int chunk = (int)chunk_mapping.size() > i ? chunk_mapping[i] : i;
       want_to_read->insert(chunk);
@@ -249,8 +249,8 @@ private:
   struct RecoveryOp {
     hobject_t hoid;
     eversion_t v;
-    set<pg_shard_t> missing_on;
-    set<shard_id_t> missing_on_shards;
+    std::set<pg_shard_t> missing_on;
+    std::set<shard_id_t> missing_on_shards;
 
     ObjectRecoveryInfo recovery_info;
     ObjectRecoveryProgress recovery_progress;
@@ -278,21 +278,21 @@ private:
     }
 
     // must be filled if state == WRITING
-    map<int, bufferlist> returned_data;
-    map<string, bufferlist> xattrs;
+    std::map<int, ceph::buffer::list> returned_data;
+    std::map<std::string, ceph::buffer::list> xattrs;
     ECUtil::HashInfoRef hinfo;
     ObjectContextRef obc;
-    set<pg_shard_t> waiting_on_pushes;
+    std::set<pg_shard_t> waiting_on_pushes;
 
     // valid in state READING
-    pair<uint64_t, uint64_t> extent_requested;
+    std::pair<uint64_t, uint64_t> extent_requested;
 
-    void dump(Formatter *f) const;
+    void dump(ceph::Formatter *f) const;
 
     RecoveryOp() : state(IDLE) {}
   };
   friend ostream &operator<<(ostream &lhs, const RecoveryOp &rhs);
-  map<hobject_t, RecoveryOp> recovery_ops;
+  std::map<hobject_t, RecoveryOp> recovery_ops;
 
   void continue_recovery_op(
     RecoveryOp &op,
@@ -301,8 +301,8 @@ private:
   friend struct OnRecoveryReadComplete;
   void handle_recovery_read_complete(
     const hobject_t &hoid,
-    boost::tuple<uint64_t, uint64_t, map<pg_shard_t, bufferlist> > &to_read,
-    std::optional<map<string, bufferlist> > attrs,
+    boost::tuple<uint64_t, uint64_t, std::map<pg_shard_t, ceph::buffer::list> > &to_read,
+    std::optional<std::map<std::string, ceph::buffer::list> > attrs,
     RecoveryMessages *m);
   void handle_recovery_push(
     const PushOp &op,
@@ -314,9 +314,9 @@ private:
     RecoveryMessages *m);
   void get_all_avail_shards(
     const hobject_t &hoid,
-    const set<pg_shard_t> &error_shards,
-    set<int> &have,
-    map<shard_id_t, pg_shard_t> &shards,
+    const std::set<pg_shard_t> &error_shards,
+    std::set<int> &have,
+    std::map<shard_id_t, pg_shard_t> &shards,
     bool for_recovery);
 
 public:
@@ -325,7 +325,7 @@ public:
    *
    * To avoid duplicating the logic for requesting and waiting for
    * multiple object shards, there is a common async read mechanism
-   * taking a map of hobject_t->read_request_t which defines callbacks
+   * taking a std::map of hobject_t->read_request_t which defines callbacks
    * taking read_result_ts as arguments.
    *
    * tid_to_read_map gives open read ops.  check_recovery_sources uses
@@ -343,23 +343,23 @@ public:
    */
   struct read_result_t {
     int r;
-    map<pg_shard_t, int> errors;
-    std::optional<map<string, bufferlist> > attrs;
-    list<
+    std::map<pg_shard_t, int> errors;
+    std::optional<std::map<std::string, ceph::buffer::list> > attrs;
+    std::list<
       boost::tuple<
-       uint64_t, uint64_t, map<pg_shard_t, bufferlist> > > returned;
+       uint64_t, uint64_t, std::map<pg_shard_t, ceph::buffer::list> > > returned;
     read_result_t() : r(0) {}
   };
   struct read_request_t {
-    const list<boost::tuple<uint64_t, uint64_t, uint32_t> > to_read;
-    const map<pg_shard_t, vector<pair<int, int>>> need;
+    const std::list<boost::tuple<uint64_t, uint64_t, uint32_t> > to_read;
+    const std::map<pg_shard_t, std::vector<std::pair<int, int>>> need;
     const bool want_attrs;
-    GenContext<pair<RecoveryMessages *, read_result_t& > &> *cb;
+    GenContext<std::pair<RecoveryMessages *, read_result_t& > &> *cb;
     read_request_t(
-      const list<boost::tuple<uint64_t, uint64_t, uint32_t> > &to_read,
-      const map<pg_shard_t, vector<pair<int, int>>> &need,
+      const std::list<boost::tuple<uint64_t, uint64_t, uint32_t> > &to_read,
+      const std::map<pg_shard_t, std::vector<std::pair<int, int>>> &need,
       bool want_attrs,
-      GenContext<pair<RecoveryMessages *, read_result_t& > &> *cb)
+      GenContext<std::pair<RecoveryMessages *, read_result_t& > &> *cb)
       : to_read(to_read), need(need), want_attrs(want_attrs),
        cb(cb) {}
   };
@@ -379,16 +379,16 @@ public:
 
     ZTracer::Trace trace;
 
-    map<hobject_t, set<int>> want_to_read;
-    map<hobject_t, read_request_t> to_read;
-    map<hobject_t, read_result_t> complete;
+    std::map<hobject_t, std::set<int>> want_to_read;
+    std::map<hobject_t, read_request_t> to_read;
+    std::map<hobject_t, read_result_t> complete;
 
-    map<hobject_t, set<pg_shard_t>> obj_to_source;
-    map<pg_shard_t, set<hobject_t> > source_to_obj;
+    std::map<hobject_t, std::set<pg_shard_t>> obj_to_source;
+    std::map<pg_shard_t, std::set<hobject_t> > source_to_obj;
 
-    void dump(Formatter *f) const;
+    void dump(ceph::Formatter *f) const;
 
-    set<pg_shard_t> in_progress;
+    std::set<pg_shard_t> in_progress;
 
     ReadOp(
       int priority,
@@ -396,8 +396,8 @@ public:
       bool do_redundant_reads,
       bool for_recovery,
       OpRequestRef op,
-      map<hobject_t, set<int>> &&_want_to_read,
-      map<hobject_t, read_request_t> &&_to_read)
+      std::map<hobject_t, std::set<int>> &&_want_to_read,
+      std::map<hobject_t, read_request_t> &&_to_read)
       : priority(priority), tid(tid), op(op), do_redundant_reads(do_redundant_reads),
        for_recovery(for_recovery), want_to_read(std::move(_want_to_read)),
        to_read(std::move(_to_read)) {
@@ -408,7 +408,7 @@ public:
            boost::make_tuple(
              extent.get<0>(),
              extent.get<1>(),
-             map<pg_shard_t, bufferlist>()));
+             std::map<pg_shard_t, ceph::buffer::list>()));
        }
       }
     }
@@ -422,12 +422,12 @@ public:
     ReadOp &op);
   void complete_read_op(ReadOp &rop, RecoveryMessages *m);
   friend ostream &operator<<(ostream &lhs, const ReadOp &rhs);
-  map<ceph_tid_t, ReadOp> tid_to_read_map;
-  map<pg_shard_t, set<ceph_tid_t> > shard_to_read_map;
+  std::map<ceph_tid_t, ReadOp> tid_to_read_map;
+  std::map<pg_shard_t, std::set<ceph_tid_t> > shard_to_read_map;
   void start_read_op(
     int priority,
-    map<hobject_t, set<int>> &want_to_read,
-    map<hobject_t, read_request_t> &to_read,
+    std::map<hobject_t, std::set<int>> &want_to_read,
+    std::map<hobject_t, read_request_t> &to_read,
     OpRequestRef op,
     bool do_redundant_reads, bool for_recovery);
 
@@ -448,7 +448,7 @@ public:
    *
    * As with client reads, there is a possibility of out-of-order
    * completions. Thus, callbacks and completion are called in order
-   * on the writing list.
+   * on the writing std::list.
    */
   struct Op : boost::intrusive::list_base_hook<> {
     /// From submit_transaction caller, describes operation
@@ -457,7 +457,7 @@ public:
     eversion_t version;
     eversion_t trim_to;
     std::optional<pg_hit_set_history_t> updated_hit_set_history;
-    vector<pg_log_entry_t> log_entries;
+    std::vector<pg_log_entry_t> log_entries;
     ceph_tid_t tid;
     osd_reqid_t reqid;
     ZTracer::Trace trace;
@@ -465,14 +465,14 @@ public:
     eversion_t roll_forward_to; /// Soon to be generated internally
 
     /// Ancillary also provided from submit_transaction caller
-    map<hobject_t, ObjectContextRef> obc_map;
+    std::map<hobject_t, ObjectContextRef> obc_map;
 
     /// see call_write_ordered
     std::list<std::function<void(void)> > on_write;
 
     /// Generated internally
-    set<hobject_t> temp_added;
-    set<hobject_t> temp_cleared;
+    std::set<hobject_t> temp_added;
+    std::set<hobject_t> temp_cleared;
 
     ECTransaction::WritePlan plan;
     bool requires_rmw() const { return !plan.to_read.empty(); }
@@ -482,19 +482,19 @@ public:
     bool using_cache = true;
 
     /// In progress read state;
-    map<hobject_t,extent_set> pending_read; // subset already being read
-    map<hobject_t,extent_set> remote_read;  // subset we must read
-    map<hobject_t,extent_map> remote_read_result;
+    std::map<hobject_t,extent_set> pending_read; // subset already being read
+    std::map<hobject_t,extent_set> remote_read;  // subset we must read
+    std::map<hobject_t,extent_map> remote_read_result;
     bool read_in_progress() const {
       return !remote_read.empty() && remote_read_result.empty();
     }
 
     /// In progress write state.
-    set<pg_shard_t> pending_commit;
+    std::set<pg_shard_t> pending_commit;
     // we need pending_apply for pre-mimic peers so that we don't issue a
     // read on a remote shard before it has applied a previous write.  We can
     // remove this after nautilus.
-    set<pg_shard_t> pending_apply;
+    std::set<pg_shard_t> pending_apply;
     bool write_in_progress() const {
       return !pending_commit.empty() || !pending_apply.empty();
     }
@@ -515,10 +515,10 @@ public:
   friend ostream &operator<<(ostream &lhs, const Op &rhs);
 
   ExtentCache cache;
-  map<ceph_tid_t, Op> tid_to_op_map; /// Owns Op structure
+  std::map<ceph_tid_t, Op> tid_to_op_map; /// Owns Op structure
 
   /**
-   * We model the possible rmw states as a set of waitlists.
+   * We model the possible rmw states as a std::set of waitlists.
    * All writes at this time complete in order, so a write blocked
    * at waiting_state blocks all writes behind it as well (same for
    * other states).
@@ -570,7 +570,7 @@ public:
   bool try_finish_rmw();
   void check_ops();
 
-  ErasureCodeInterfaceRef ec_impl;
+  ceph::ErasureCodeInterfaceRef ec_impl;
 
 
   /**
@@ -579,22 +579,22 @@ public:
    * Determines the whether _have is sufficient to recover an object
    */
   class ECRecPred : public IsPGRecoverablePredicate {
-    set<int> want;
-    ErasureCodeInterfaceRef ec_impl;
+    std::set<int> want;
+    ceph::ErasureCodeInterfaceRef ec_impl;
   public:
-    explicit ECRecPred(ErasureCodeInterfaceRef ec_impl) : ec_impl(ec_impl) {
+    explicit ECRecPred(ceph::ErasureCodeInterfaceRef ec_impl) : ec_impl(ec_impl) {
       for (unsigned i = 0; i < ec_impl->get_chunk_count(); ++i) {
        want.insert(i);
       }
     }
-    bool operator()(const set<pg_shard_t> &_have) const override {
-      set<int> have;
-      for (set<pg_shard_t>::const_iterator i = _have.begin();
+    bool operator()(const std::set<pg_shard_t> &_have) const override {
+      std::set<int> have;
+      for (std::set<pg_shard_t>::const_iterator i = _have.begin();
           i != _have.end();
           ++i) {
        have.insert(i->shard);
       }
-      map<int, vector<pair<int, int>>> min;
+      std::map<int, std::vector<std::pair<int, int>>> min;
       return ec_impl->minimum_to_decode(want, have, &min) == 0;
     }
   };
@@ -620,8 +620,8 @@ public:
   public:
     ECReadPred(
       pg_shard_t whoami,
-      ErasureCodeInterfaceRef ec_impl) : whoami(whoami), rec_pred(ec_impl) {}
-    bool operator()(const set<pg_shard_t> &_have) const override {
+      ceph::ErasureCodeInterfaceRef ec_impl) : whoami(whoami), rec_pred(ec_impl) {}
+    bool operator()(const std::set<pg_shard_t> &_have) const override {
       return _have.count(whoami) && rec_pred(_have);
     }
   };
@@ -634,7 +634,7 @@ public:
   /// If modified, ensure that the ref is held until the update is applied
   SharedPtrRegistry<hobject_t, ECUtil::HashInfo> unstable_hashinfo_registry;
   ECUtil::HashInfoRef get_hash_info(const hobject_t &hoid, bool checks = true,
-                                   const map<string,bufferptr> *attr = NULL);
+                                   const std::map<std::string, ceph::buffer::ptr> *attr = NULL);
 
 public:
   ECBackend(
@@ -643,29 +643,29 @@ public:
     ObjectStore::CollectionHandle &ch,
     ObjectStore *store,
     CephContext *cct,
-    ErasureCodeInterfaceRef ec_impl,
+    ceph::ErasureCodeInterfaceRef ec_impl,
     uint64_t stripe_width);
 
   /// Returns to_read replicas sufficient to reconstruct want
   int get_min_avail_to_read_shards(
     const hobject_t &hoid,     ///< [in] object
-    const set<int> &want,      ///< [in] desired shards
+    const std::set<int> &want,      ///< [in] desired shards
     bool for_recovery,         ///< [in] true if we may use non-acting replicas
     bool do_redundant_reads,   ///< [in] true if we want to issue redundant reads to reduce latency
-    map<pg_shard_t, vector<pair<int, int>>> *to_read   ///< [out] shards, corresponding subchunks to read
+    std::map<pg_shard_t, std::vector<std::pair<int, int>>> *to_read   ///< [out] shards, corresponding subchunks to read
     ); ///< @return error code, 0 on success
 
   int get_remaining_shards(
     const hobject_t &hoid,
-    const set<int> &avail,
-    const set<int> &want,
+    const std::set<int> &avail,
+    const std::set<int> &want,
     const read_result_t &result,
-    map<pg_shard_t, vector<pair<int, int>>> *to_read,
+    std::map<pg_shard_t, std::vector<std::pair<int, int>>> *to_read,
     bool for_recovery);
 
   int objects_get_attrs(
     const hobject_t &hoid,
-    map<string, bufferlist> *out) override;
+    std::map<std::string, ceph::buffer::list> *out) override;
 
   void rollback_append(
     const hobject_t &hoid,
@@ -683,7 +683,7 @@ public:
     return sinfo.logical_to_next_chunk_offset(logical_size);
   }
   void _failed_push(const hobject_t &hoid,
-    pair<RecoveryMessages *, ECBackend::read_result_t &> &in);
+    std::pair<RecoveryMessages *, ECBackend::read_result_t &> &in);
 };
 ostream &operator<<(ostream &lhs, const ECBackend::pipeline_state_t &rhs);
 
index 1eec367c976c54941d47c3fadec69288d996191e..603f9af0ea329f6c43d454055de21fdb0e68bf96 100644 (file)
 #include "os/ObjectStore.h"
 #include "common/inline_variant.h"
 
+using std::make_pair;
+using std::map;
+using std::pair;
+using std::set;
+using std::string;
+using std::vector;
+
+using ceph::bufferlist;
+using ceph::decode;
+using ceph::encode;
+using ceph::ErasureCodeInterfaceRef;
 
 void encode_and_write(
   pg_t pgid,
@@ -190,14 +201,14 @@ void ECTransaction::generate_transactions(
       bufferlist old_hinfo;
       encode(*hinfo, old_hinfo);
       xattr_rollback[ECUtil::get_hinfo_key()] = old_hinfo;
-      
+
       if (op.is_none() && op.truncate && op.truncate->first == 0) {
        ceph_assert(op.truncate->first == 0);
        ceph_assert(op.truncate->first ==
               op.truncate->second);
        ceph_assert(entry);
        ceph_assert(obc);
-       
+
        if (op.truncate->first != op.truncate->second) {
          op.truncate->first = op.truncate->second;
        } else {
index 704be1f08269c1073bed062897607a1a6f53cc33..5cb16261a7d4b708278875bfa9700b98be0f962b 100644 (file)
@@ -26,10 +26,10 @@ namespace ECTransaction {
   struct WritePlan {
     PGTransactionUPtr t;
     bool invalidates_cache = false; // Yes, both are possible
-    map<hobject_t,extent_set> to_read;
-    map<hobject_t,extent_set> will_write; // superset of to_read
+    std::map<hobject_t,extent_set> to_read;
+    std::map<hobject_t,extent_set> will_write; // superset of to_read
 
-    map<hobject_t,ECUtil::HashInfoRef> hash_infos;
+    std::map<hobject_t,ECUtil::HashInfoRef> hash_infos;
   };
 
   bool requires_overwrite(
@@ -44,7 +44,7 @@ namespace ECTransaction {
     DoutPrefixProvider *dpp) {
     WritePlan plan;
     t->safe_create_traverse(
-      [&](pair<const hobject_t, PGTransaction::ObjectOperation> &i) {
+      [&](std::pair<const hobject_t, PGTransaction::ObjectOperation> &i) {
        ECUtil::HashInfoRef hinfo = get_hinfo(i.first);
        plan.hash_infos[i.first] = hinfo;
 
@@ -184,15 +184,15 @@ namespace ECTransaction {
 
   void generate_transactions(
     WritePlan &plan,
-    ErasureCodeInterfaceRef &ecimpl,
+    ceph::ErasureCodeInterfaceRef &ecimpl,
     pg_t pgid,
     const ECUtil::stripe_info_t &sinfo,
-    const map<hobject_t,extent_map> &partial_extents,
-    vector<pg_log_entry_t> &entries,
-    map<hobject_t,extent_map> *written,
-    map<shard_id_t, ObjectStore::Transaction> *transactions,
-    set<hobject_t> *temp_added,
-    set<hobject_t> *temp_removed,
+    const std::map<hobject_t,extent_map> &partial_extents,
+    std::vector<pg_log_entry_t> &entries,
+    std::map<hobject_t,extent_map> *written,
+    std::map<shard_id_t, ObjectStore::Transaction> *transactions,
+    std::set<hobject_t> *temp_added,
+    std::set<hobject_t> *temp_removed,
     DoutPrefixProvider *dpp,
     const ceph_release_t require_osd_release = ceph_release_t::unknown);
 };
index 311e8526a769da2e2a4d9ed7e76502317251d93f..94b32845847a65edc9344e3564265e32247b3b54 100644 (file)
@@ -5,6 +5,9 @@
 #include "ECUtil.h"
 
 using namespace std;
+using ceph::bufferlist;
+using ceph::ErasureCodeInterfaceRef;
+using ceph::Formatter;
 
 int ECUtil::decode(
   const stripe_info_t &sinfo,
index 8e980e27ebcdc141feae994b97cfa4b39301ab8d..dce78b8a8683eacd93f2fcbfbfeac094a2e790cf 100644 (file)
@@ -81,22 +81,22 @@ public:
 
 int decode(
   const stripe_info_t &sinfo,
-  ErasureCodeInterfaceRef &ec_impl,
-  std::map<int, bufferlist> &to_decode,
-  bufferlist *out);
+  ceph::ErasureCodeInterfaceRef &ec_impl,
+  std::map<int, ceph::buffer::list> &to_decode,
+  ceph::buffer::list *out);
 
 int decode(
   const stripe_info_t &sinfo,
-  ErasureCodeInterfaceRef &ec_impl,
-  std::map<int, bufferlist> &to_decode,
-  std::map<int, bufferlist*> &out);
+  ceph::ErasureCodeInterfaceRef &ec_impl,
+  std::map<int, ceph::buffer::list> &to_decode,
+  std::map<int, ceph::buffer::list*> &out);
 
 int encode(
   const stripe_info_t &sinfo,
-  ErasureCodeInterfaceRef &ec_impl,
-  bufferlist &in,
+  ceph::ErasureCodeInterfaceRef &ec_impl,
+  ceph::buffer::list &in,
   const std::set<int> &want,
-  std::map<int, bufferlist> *out);
+  std::map<int, ceph::buffer::list> *out);
 
 class HashInfo {
   uint64_t total_chunk_size = 0;
@@ -108,16 +108,16 @@ public:
   HashInfo() {}
   explicit HashInfo(unsigned num_chunks) :
     cumulative_shard_hashes(num_chunks, -1) {}
-  void append(uint64_t old_size, std::map<int, bufferlist> &to_append);
+  void append(uint64_t old_size, std::map<int, ceph::buffer::list> &to_append);
   void clear() {
     total_chunk_size = 0;
     cumulative_shard_hashes = std::vector<uint32_t>(
       cumulative_shard_hashes.size(),
       -1);
   }
-  void encode(bufferlist &bl) const;
-  void decode(bufferlist::const_iterator &bl);
-  void dump(Formatter *f) const;
+  void encode(ceph::buffer::list &bl) const;
+  void decode(ceph::buffer::list::const_iterator &bl);
+  void dump(ceph::Formatter *f) const;
   static void generate_test_instances(std::list<HashInfo*>& o);
   uint32_t get_chunk_hash(int shard) const {
     ceph_assert((unsigned)shard < cumulative_shard_hashes.size());
index a09fc86efa94df575e17f231bb5e3de87a0fce71..3a8bbf11bb4cff785d5c259cb14e827114c3102f 100644 (file)
 
 #include "ExtentCache.h"
 
+using std::ostream;
+
+using ceph::bufferlist;
+
 void ExtentCache::extent::_link_pin_state(pin_state &pin_state)
 {
   ceph_assert(parent_extent_set);
index bf5a60dc1b6f270446eb97221655f6be23324745..9d387879216cb3b82acfaae2c069d77b7c5b40fa 100644 (file)
 
 /// If someone wants these types, but not ExtentCache, move to another file
 struct bl_split_merge {
-  bufferlist split(
+  ceph::buffer::list split(
     uint64_t offset,
     uint64_t length,
-    bufferlist &bl) const {
-    bufferlist out;
+    ceph::buffer::list &bl) const {
+    ceph::buffer::list out;
     out.substr_of(bl, offset, length);
     return out;
   }
-  bool can_merge(const bufferlist &left, const bufferlist &right) const {
+  bool can_merge(const ceph::buffer::list &left, const ceph::buffer::list &right) const {
     return true;
   }
-  bufferlist merge(bufferlist &&left, bufferlist &&right) const {
-    bufferlist bl;
+  ceph::buffer::list merge(ceph::buffer::list &&left, ceph::buffer::list &&right) const {
+    ceph::buffer::list bl;
     bl.claim(left);
     bl.claim_append(right);
     return bl;
   }
-  uint64_t length(const bufferlist &b) const { return b.length(); }
+  uint64_t length(const ceph::buffer::list &b) const { return b.length(); }
 };
 using extent_set = interval_set<uint64_t>;
-using extent_map = interval_map<uint64_t, bufferlist, bl_split_merge>;
+using extent_map = interval_map<uint64_t, ceph::buffer::list, bl_split_merge>;
 
 class ExtentCache {
   struct object_extent_set;
@@ -131,7 +131,7 @@ private:
 
     uint64_t offset;
     uint64_t length;
-    std::optional<bufferlist> bl;
+    std::optional<ceph::buffer::list> bl;
 
     uint64_t get_length() const {
       return length;
@@ -151,7 +151,7 @@ private:
       return parent_pin_state->tid;
     }
 
-    extent(uint64_t offset, bufferlist _bl)
+    extent(uint64_t offset, ceph::buffer::list _bl)
       : offset(offset), length(_bl.length()), bl(_bl) {}
 
     extent(uint64_t offset, uint64_t length)
@@ -204,7 +204,7 @@ private:
        UPDATE_PIN
       };
       type action = NONE;
-      std::optional<bufferlist> bl;
+      std::optional<ceph::buffer::list> bl;
     };
     template <typename F>
     void traverse_update(
@@ -253,7 +253,7 @@ private:
              (ext->offset + ext->get_length() > offset)) {
            extent *head = nullptr;
            if (ext->bl) {
-             bufferlist bl;
+             ceph::buffer::list bl;
              bl.substr_of(
                *(ext->bl),
                0,
@@ -271,7 +271,7 @@ private:
              (ext->offset + ext->get_length()) - (offset + length);
            extent *tail = nullptr;
            if (ext->bl) {
-             bufferlist bl;
+             ceph::buffer::list bl;
              bl.substr_of(
                *(ext->bl),
                ext->get_length() - nlen,
@@ -284,7 +284,7 @@ private:
          }
          if (action.action == update_action::UPDATE_PIN) {
            if (ext->bl) {
-             bufferlist bl;
+             ceph::buffer::list bl;
              bl.substr_of(
                *(ext->bl),
                extoff - ext->offset,
@@ -381,7 +381,7 @@ private:
 
   void release_pin(pin_state &p) {
     for (auto iter = p.pin_list.begin(); iter != p.pin_list.end(); ) {
-      unique_ptr<extent> extent(&*iter); // we now own this
+      std::unique_ptr<extent> extent(&*iter); // we now own this
       iter++; // unlink will invalidate
       ceph_assert(extent->parent_extent_set);
       auto &eset = *(extent->parent_extent_set);
@@ -482,10 +482,9 @@ public:
     release_pin(pin);
   }
 
-  ostream &print(
-    ostream &out) const;
+  std::ostream &print(std::ostream &out) const;
 };
 
-ostream &operator<<(ostream &lhs, const ExtentCache &cache);
+std::ostream &operator <<(std::ostream &lhs, const ExtentCache &cache);
 
 #endif
index 8e96f0fb8be33fbe795ca4bc9e476e59e1c89084..2682e57c26faf622e9905a64387bec4c97e0e27b 100644 (file)
@@ -8,6 +8,8 @@
 #define dout_prefix (gen_prefix(*_dout))
 #define dout_subsys ceph_subsys_osd
 
+using std::set;
+
 bool MissingLoc::readable_with_acting(
   const hobject_t &hoid,
   const set<pg_shard_t> &acting) const {
@@ -21,9 +23,7 @@ bool MissingLoc::readable_with_acting(
   const set<pg_shard_t> &locs = missing_loc_entry->second;
   ldout(cct, 10) << __func__ << ": locs:" << locs << dendl;
   set<pg_shard_t> have_acting;
-  for (set<pg_shard_t>::const_iterator i = locs.begin();
-       i != locs.end();
-       ++i) {
+  for (auto i = locs.begin(); i != locs.end(); ++i) {
     if (acting.count(*i))
       have_acting.insert(*i);
   }
@@ -38,7 +38,7 @@ void MissingLoc::add_batch_sources_info(
                     << sources.size() << dendl;
   unsigned loop = 0;
   bool sources_updated = false;
-  for (map<hobject_t, pg_missing_item>::const_iterator i = needs_recovery_map.begin();
+  for (auto i = needs_recovery_map.begin();
       i != needs_recovery_map.end();
       ++i) {
     if (handle && ++loop >= cct->_conf->osd_loop_before_reset_tphandle) {
@@ -74,7 +74,7 @@ bool MissingLoc::add_source_info(
   unsigned loop = 0;
   bool sources_updated = false;
   // found items?
-  for (map<hobject_t,pg_missing_item>::const_iterator p = needs_recovery_map.begin();
+  for (auto p = needs_recovery_map.begin();
        p != needs_recovery_map.end();
        ++p) {
     const hobject_t &soid(p->first);
@@ -139,7 +139,7 @@ bool MissingLoc::add_source_info(
 void MissingLoc::check_recovery_sources(const OSDMapRef& osdmap)
 {
   set<pg_shard_t> now_down;
-  for (set<pg_shard_t>::iterator p = missing_loc_sources.begin();
+  for (auto p = missing_loc_sources.begin();
        p != missing_loc_sources.end();
        ) {
     if (osdmap->is_up(p->osd)) {
@@ -158,9 +158,9 @@ void MissingLoc::check_recovery_sources(const OSDMapRef& osdmap)
                       << missing_loc_sources << dendl;
 
     // filter missing_loc
-    map<hobject_t, set<pg_shard_t>>::iterator p = missing_loc.begin();
+    auto p = missing_loc.begin();
     while (p != missing_loc.end()) {
-      set<pg_shard_t>::iterator q = p->second.begin();
+      auto q = p->second.begin();
       bool changed = false;
       while (q != p->second.end()) {
        if (now_down.count(*q)) {
@@ -189,9 +189,9 @@ void MissingLoc::remove_stray_recovery_sources(pg_shard_t stray)
 {
   ldout(cct, 10) << __func__ << " remove osd " << stray << " from missing_loc" << dendl;
   // filter missing_loc
-  map<hobject_t, set<pg_shard_t>>::iterator p = missing_loc.begin();
+  auto p = missing_loc.begin();
   while (p != missing_loc.end()) {
-    set<pg_shard_t>::iterator q = p->second.begin();
+    auto q = p->second.begin();
     bool changed = false;
     while (q != p->second.end()) {
       if (*q == stray) {
@@ -214,9 +214,7 @@ void MissingLoc::remove_stray_recovery_sources(pg_shard_t stray)
     }
   }
   // filter missing_loc_sources
-  for (set<pg_shard_t>::iterator p = missing_loc_sources.begin();
-       p != missing_loc_sources.end();
-       ) {
+  for (auto p = missing_loc_sources.begin(); p != missing_loc_sources.end();) {
     if (*p != stray) {
       ++p;
       continue;
index f0cbadbb44ad3d0743b60cd17fd5d30acebd1f96..ed091bb21132053c366058857728a88c5521caef 100644 (file)
@@ -17,7 +17,7 @@ class MissingLoc {
 
   class MappingInfo {
   public:
-    virtual const set<pg_shard_t> &get_upset() const = 0;
+    virtual const std::set<pg_shard_t> &get_upset() const = 0;
     virtual bool is_ec_pg() const = 0;
     virtual int get_pg_size() const = 0;
     virtual ~MappingInfo() {}
@@ -35,7 +35,7 @@ class MissingLoc {
              (l.up == r.up &&
               (l.other < r.other)));
     }
-    friend ostream& operator<<(ostream& out, const loc_count_t& l) {
+    friend std::ostream& operator<<(std::ostream& out, const loc_count_t& l) {
       ceph_assert(l.up >= 0);
       ceph_assert(l.other >= 0);
       return out << "(" << l.up << "+" << l.other << ")";
@@ -43,9 +43,9 @@ class MissingLoc {
   };
 
 
-  using missing_by_count_t = map < shard_id_t, map<loc_count_t,int> >;
+  using missing_by_count_t = std::map<shard_id_t, std::map<loc_count_t,int>>;
  private:
-  loc_count_t _get_count(const set<pg_shard_t> &shards) {
+  loc_count_t _get_count(const std::set<pg_shard_t> &shards) {
     loc_count_t r;
     for (auto s : shards) {
       if (mapping_info->get_upset().count(s)) {
@@ -57,21 +57,21 @@ class MissingLoc {
     return r;
   }
 
-  map<hobject_t, pg_missing_item> needs_recovery_map;
-  map<hobject_t, set<pg_shard_t> > missing_loc;
-  set<pg_shard_t> missing_loc_sources;
+  std::map<hobject_t, pg_missing_item> needs_recovery_map;
+  std::map<hobject_t, std::set<pg_shard_t> > missing_loc;
+  std::set<pg_shard_t> missing_loc_sources;
 
   // for every entry in missing_loc, we count how many of each type of shard we have,
-  // and maintain totals here.  The sum of the values for this map will always equal
+  // and maintain totals here.  The sum of the values for this std::map will always equal
   // missing_loc.size().
   missing_by_count_t missing_by_count;
 
   void pgs_by_shard_id(
-    const set<pg_shard_t>& s,
-    map< shard_id_t, set<pg_shard_t> >& pgsbs) {
+    const std::set<pg_shard_t>& s,
+    std::map<shard_id_t, std::set<pg_shard_t> >& pgsbs) {
     if (mapping_info->is_ec_pg()) {
       int num_shards = mapping_info->get_pg_size();
-      // For completely missing shards initialize with empty set<pg_shard_t>
+      // For completely missing shards initialize with empty std::set<pg_shard_t>
       for (int i = 0 ; i < num_shards ; ++i) {
        shard_id_t shard(i);
        pgsbs[shard];
@@ -83,14 +83,14 @@ class MissingLoc {
     }
   }
 
-  void _inc_count(const set<pg_shard_t>& s) {
-    map< shard_id_t, set<pg_shard_t> > pgsbs;
+  void _inc_count(const std::set<pg_shard_t>& s) {
+    std::map< shard_id_t, std::set<pg_shard_t> > pgsbs;
     pgs_by_shard_id(s, pgsbs);
     for (auto shard: pgsbs)
       ++missing_by_count[shard.first][_get_count(shard.second)];
   }
-  void _dec_count(const set<pg_shard_t>& s) {
-    map< shard_id_t, set<pg_shard_t> > pgsbs;
+  void _dec_count(const std::set<pg_shard_t>& s) {
+    std::map< shard_id_t, std::set<pg_shard_t> > pgsbs;
     pgs_by_shard_id(s, pgsbs);
     for (auto shard: pgsbs) {
       auto p = missing_by_count[shard.first].find(_get_count(shard.second));
@@ -105,7 +105,7 @@ class MissingLoc {
   MappingInfo *mapping_info;
   DoutPrefixProvider *dpp;
   CephContext *cct;
-  set<pg_shard_t> empty_set;
+  std::set<pg_shard_t> empty_set;
  public:
   boost::scoped_ptr<IsPGReadablePredicate> is_readable;
   boost::scoped_ptr<IsPGRecoverablePredicate> is_recoverable;
@@ -130,7 +130,7 @@ class MissingLoc {
   bool needs_recovery(
     const hobject_t &hoid,
     eversion_t *v = 0) const {
-    map<hobject_t, pg_missing_item>::const_iterator i =
+    std::map<hobject_t, pg_missing_item>::const_iterator i =
       needs_recovery_map.find(hoid);
     if (i == needs_recovery_map.end())
       return false;
@@ -157,10 +157,10 @@ class MissingLoc {
   }
   bool readable_with_acting(
     const hobject_t &hoid,
-    const set<pg_shard_t> &acting) const;
+    const std::set<pg_shard_t> &acting) const;
   uint64_t num_unfound() const {
     uint64_t ret = 0;
-    for (map<hobject_t, pg_missing_item>::const_iterator i =
+    for (std::map<hobject_t, pg_missing_item>::const_iterator i =
           needs_recovery_map.begin();
         i != needs_recovery_map.end();
         ++i) {
@@ -174,7 +174,7 @@ class MissingLoc {
   }
 
   bool have_unfound() const {
-    for (map<hobject_t, pg_missing_item>::const_iterator i =
+    for (std::map<hobject_t, pg_missing_item>::const_iterator i =
           needs_recovery_map.begin();
         i != needs_recovery_map.end();
         ++i) {
@@ -196,7 +196,7 @@ class MissingLoc {
   void add_location(const hobject_t &hoid, pg_shard_t location) {
     auto p = missing_loc.find(hoid);
     if (p == missing_loc.end()) {
-      p = missing_loc.emplace(hoid, set<pg_shard_t>()).first;
+      p = missing_loc.emplace(hoid, std::set<pg_shard_t>()).first;
     } else {
       _dec_count(p->second);
     }
@@ -225,11 +225,11 @@ class MissingLoc {
   }
 
   void add_active_missing(const pg_missing_t &missing) {
-    for (map<hobject_t, pg_missing_item>::const_iterator i =
+    for (std::map<hobject_t, pg_missing_item>::const_iterator i =
           missing.get_items().begin();
         i != missing.get_items().end();
         ++i) {
-      map<hobject_t, pg_missing_item>::const_iterator j =
+      std::map<hobject_t, pg_missing_item>::const_iterator j =
        needs_recovery_map.find(i->first);
       if (j == needs_recovery_map.end()) {
        needs_recovery_map.insert(*i);
@@ -263,7 +263,7 @@ class MissingLoc {
 
   /// Adds recovery sources in batch
   void add_batch_sources_info(
-    const set<pg_shard_t> &sources,  ///< [in] a set of resources which can be used for all objects
+    const std::set<pg_shard_t> &sources,  ///< [in] a std::set of resources which can be used for all objects
     HBHandle *handle  ///< [in] ThreadPool handle
     );
 
@@ -273,7 +273,7 @@ class MissingLoc {
   /// Remove stray from recovery sources
   void remove_stray_recovery_sources(pg_shard_t stray);
 
-  /// Call when hoid is no longer missing in acting set
+  /// Call when hoid is no longer missing in acting std::set
   void recovered(const hobject_t &hoid) {
     needs_recovery_map.erase(hoid);
     auto p = missing_loc.find(hoid);
@@ -287,11 +287,11 @@ class MissingLoc {
   void rebuild(
     const hobject_t &hoid,
     pg_shard_t self,
-    const set<pg_shard_t> &to_recover,
+    const std::set<pg_shard_t> &to_recover,
     const pg_info_t &info,
     const pg_missing_t &missing,
-    const map<pg_shard_t, pg_missing_t> &pmissing,
-    const map<pg_shard_t, pg_info_t> &pinfo) {
+    const std::map<pg_shard_t, pg_missing_t> &pmissing,
+    const std::map<pg_shard_t, pg_info_t> &pinfo) {
     recovered(hoid);
     std::optional<pg_missing_item> item;
     auto miter = missing.get_items().find(hoid);
@@ -317,7 +317,7 @@ class MissingLoc {
     if (item->is_delete())
       return;
     auto mliter =
-      missing_loc.emplace(hoid, set<pg_shard_t>()).first;
+      missing_loc.emplace(hoid, std::set<pg_shard_t>()).first;
     ceph_assert(info.last_backfill.is_max());
     ceph_assert(info.last_update >= item->need);
     if (!missing.is_missing(hoid))
@@ -335,14 +335,14 @@ class MissingLoc {
     _inc_count(mliter->second);
   }
 
-  const set<pg_shard_t> &get_locations(const hobject_t &hoid) const {
+  const std::set<pg_shard_t> &get_locations(const hobject_t &hoid) const {
     auto it = missing_loc.find(hoid);
     return it == missing_loc.end() ? empty_set : it->second;
   }
-  const map<hobject_t, set<pg_shard_t>> &get_missing_locs() const {
+  const std::map<hobject_t, std::set<pg_shard_t>> &get_missing_locs() const {
     return missing_loc;
   }
-  const map<hobject_t, pg_missing_item> &get_needs_recovery() const {
+  const std::map<hobject_t, pg_missing_item> &get_needs_recovery() const {
     return needs_recovery_map;
   }
 
index 7ff0ee94e9bfb2225da217e6a6cfd536d7e340a2..2308563cbbddc08ea1feed9af3e112457d67f509 100644 (file)
 #undef dout_prefix
 #define dout_prefix _prefix(_dout, whoami, get_osdmap_epoch())
 
+using std::deque;
+using std::list;
+using std::lock_guard;
+using std::make_pair;
+using std::make_tuple;
+using std::make_unique;
+using std::map;
+using std::ostream;
+using std::ostringstream;
+using std::pair;
+using std::set;
+using std::string;
+using std::stringstream;
+using std::to_string;
+using std::unique_ptr;
+using std::vector;
+
+using ceph::bufferlist;
+using ceph::bufferptr;
+using ceph::decode;
+using ceph::encode;
+using ceph::fixed_u_to_string;
+using ceph::Formatter;
+using ceph::heartbeat_handle_d;
+using ceph::make_mutex;
+
 using namespace ceph::osd::scheduler;
 using TOPNSPC::common::cmd_getval;
 
@@ -5834,9 +5860,9 @@ void OSD::tick_without_osd_lock()
   ceph_assert(ceph_mutex_is_locked(tick_timer_lock));
   dout(10) << "tick_without_osd_lock" << dendl;
 
-  logger->set(l_osd_cached_crc, buffer::get_cached_crc());
-  logger->set(l_osd_cached_crc_adjusted, buffer::get_cached_crc_adjusted());
-  logger->set(l_osd_missed_crc, buffer::get_missed_crc());
+  logger->set(l_osd_cached_crc, ceph::buffer::get_cached_crc());
+  logger->set(l_osd_cached_crc_adjusted, ceph::buffer::get_cached_crc_adjusted());
+  logger->set(l_osd_missed_crc, ceph::buffer::get_missed_crc());
 
   // refresh osd stats
   struct store_statfs_t stbuf;
@@ -5889,7 +5915,7 @@ void OSD::tick_without_osd_lock()
       // borrow lec lock to pretect last_sent_beacon from changing
       std::lock_guard l{min_last_epoch_clean_lock};
       const auto elapsed = now - last_sent_beacon;
-      if (chrono::duration_cast<chrono::seconds>(elapsed).count() >
+      if (std::chrono::duration_cast<std::chrono::seconds>(elapsed).count() >
         cct->_conf->osd_beacon_report_interval) {
         need_send_beacon = true;
       }
@@ -7110,7 +7136,7 @@ int OSD::ms_handle_authentication(Connection *con)
     try {
       decode(str, p);
     }
-    catch (buffer::error& e) {
+    catch (ceph::buffer::error& e) {
       dout(10) << __func__ << " session " << s << " " << s->entity_name
               << " failed to decode caps string" << dendl;
       ret = -EACCES;
@@ -7542,7 +7568,7 @@ MPGStats* OSD::collect_pg_stats()
     }
     pg->get_pg_stats([&](const pg_stat_t& s, epoch_t lec) {
        m->pg_stat[pg->pg_id.pgid] = s;
-       min_last_epoch_clean = min(min_last_epoch_clean, lec);
+       min_last_epoch_clean = std::min(min_last_epoch_clean, lec);
        min_last_epoch_clean_pgs.push_back(pg->pg_id.pgid);
       });
   }
index 59f84ed0bf3c7cfb2abaab7b6a246f3f2b4afd2d..4eab442a9eb3a3f8734d5c1d4cc9290f948130fc 100644 (file)
@@ -1,4 +1,4 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*- 
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
 // vim: ts=8 sw=2 smarttab
 /*
  * Ceph - scalable distributed file system
@@ -7,9 +7,9 @@
  *
  * This is free software; you can redistribute it and/or
  * modify it under the terms of the GNU Lesser General Public
- * License version 2.1, as published by the Free Software 
+ * License version 2.1, as published by the Free Software
  * Foundation.  See file COPYING.
- * 
+ *
  */
 
 #ifndef CEPH_OSD_H
@@ -30,8 +30,8 @@
 #include "mgr/MgrClient.h"
 
 #include "os/ObjectStore.h"
-#include "OSDCap.h" 
+#include "OSDCap.h"
+
 #include "auth/KeyRing.h"
 
 #include "osd/ClassHandler.h"
@@ -175,11 +175,11 @@ public:
   }
 
   /*
-   * osdmap - current published map
-   * next_osdmap - pre_published map that is about to be published.
+   * osdmap - current published std::map
+   * next_osdmap - pre_published std::map that is about to be published.
    *
    * We use the next_osdmap to send messages and initiate connections,
-   * but only if the target is the same instance as the one in the map
+   * but only if the target is the same instance as the one in the std::map
    * epoch the current user is working from (i.e., the result is
    * equivalent to what is in next_osdmap).
    *
@@ -201,7 +201,7 @@ public:
 
   void activate_map();
   /// map epochs reserved below
-  map<epoch_t, unsigned> map_reservations;
+  std::map<epoch_t, unsigned> map_reservations;
 
   /// gets ref to next_osdmap and registers the epoch as reserved
   OSDMapRef get_nextmap_reserved() {
@@ -209,15 +209,15 @@ public:
     if (!next_osdmap)
       return OSDMapRef();
     epoch_t e = next_osdmap->get_epoch();
-    map<epoch_t, unsigned>::iterator i =
-      map_reservations.insert(make_pair(e, 0)).first;
+    std::map<epoch_t, unsigned>::iterator i =
+      map_reservations.insert(std::make_pair(e, 0)).first;
     i->second++;
     return next_osdmap;
   }
   /// releases reservation on map
   void release_map(OSDMapRef osdmap) {
     std::lock_guard l(pre_publish_lock);
-    map<epoch_t, unsigned>::iterator i =
+    std::map<epoch_t, unsigned>::iterator i =
       map_reservations.find(osdmap->get_epoch());
     ceph_assert(i != map_reservations.end());
     ceph_assert(i->second > 0);
@@ -258,7 +258,7 @@ public:
                                        OSDSuperblock& superblock);
 
   ConnectionRef get_con_osd_cluster(int peer, epoch_t from_epoch);
-  pair<ConnectionRef,ConnectionRef> get_con_osd_hb(int peer, epoch_t from_epoch);  // (back, front)
+  std::pair<ConnectionRef,ConnectionRef> get_con_osd_hb(int peer, epoch_t from_epoch);  // (back, front)
   void send_message_osd_cluster(int peer, Message *m, epoch_t from_epoch);
   void send_message_osd_cluster(std::vector<std::pair<int, Message*>>& messages, epoch_t from_epoch);
   void send_message_osd_cluster(Message *m, Connection *con) {
@@ -296,7 +296,7 @@ public:
     /// order the jobs by sched_time
     bool operator<(const ScrubJob& rhs) const;
   };
-  set<ScrubJob> sched_scrub_pg;
+  std::set<ScrubJob> sched_scrub_pg;
 
   /// @returns the scrub_reg_stamp used for unregister the scrub job
   utime_t reg_pg_scrub(spg_t pgid, utime_t t, double pool_scrub_min_interval,
@@ -316,7 +316,7 @@ public:
     std::lock_guard l(sched_scrub_lock);
     if (sched_scrub_pg.empty())
       return false;
-    set<ScrubJob>::iterator iter = sched_scrub_pg.begin();
+    std::set<ScrubJob>::iterator iter = sched_scrub_pg.begin();
     *out = *iter;
     return true;
   }
@@ -325,7 +325,7 @@ public:
     std::lock_guard l(sched_scrub_lock);
     if (sched_scrub_pg.empty())
       return false;
-    set<ScrubJob>::const_iterator iter = sched_scrub_pg.lower_bound(next);
+    std::set<ScrubJob>::const_iterator iter = sched_scrub_pg.lower_bound(next);
     if (iter == sched_scrub_pg.cend())
       return false;
     ++iter;
@@ -335,7 +335,7 @@ public:
     return true;
   }
 
-  void dumps_scrub(Formatter *f) {
+  void dumps_scrub(ceph::Formatter *f) {
     ceph_assert(f != nullptr);
     std::lock_guard l(sched_scrub_lock);
 
@@ -356,11 +356,11 @@ public:
   void dec_scrubs_local();
   bool inc_scrubs_remote();
   void dec_scrubs_remote();
-  void dump_scrub_reservations(Formatter *f);
+  void dump_scrub_reservations(ceph::Formatter *f);
 
   void reply_op_error(OpRequestRef op, int err);
   void reply_op_error(OpRequestRef op, int err, eversion_t v, version_t uv,
-                     vector<pg_log_op_return_item_t> op_returns);
+                     std::vector<pg_log_op_return_item_t> op_returns);
   void handle_misdirected_op(PG *pg, OpRequestRef op);
 
 
@@ -368,12 +368,12 @@ private:
   // -- agent shared state --
   ceph::mutex agent_lock = ceph::make_mutex("OSDService::agent_lock");
   ceph::condition_variable agent_cond;
-  map<uint64_t, set<PGRef> > agent_queue;
-  set<PGRef>::iterator agent_queue_pos;
+  std::map<uint64_t, std::set<PGRef> > agent_queue;
+  std::set<PGRef>::iterator agent_queue_pos;
   bool agent_valid_iterator;
   int agent_ops;
   int flush_mode_high_count; //once have one pg with FLUSH_MODE_HIGH then flush objects with high speed
-  set<hobject_t> agent_oids;
+  std::set<hobject_t> agent_oids;
   bool agent_active;
   struct AgentThread : public Thread {
     OSDService *osd;
@@ -395,15 +395,15 @@ public:
     if (!agent_queue.empty() &&
        agent_queue.rbegin()->first < priority)
       agent_valid_iterator = false;  // inserting higher-priority queue
-    set<PGRef>& nq = agent_queue[priority];
+    std::set<PGRef>& nq = agent_queue[priority];
     if (nq.empty())
       agent_cond.notify_all();
     nq.insert(pg);
   }
 
   void _dequeue(PG *pg, uint64_t old_priority) {
-    set<PGRef>& oq = agent_queue[old_priority];
-    set<PGRef>::iterator p = oq.find(pg);
+    std::set<PGRef>& oq = agent_queue[old_priority];
+    std::set<PGRef>::iterator p = oq.find(pg);
     ceph_assert(p != oq.end());
     if (p == agent_queue_pos)
       ++agent_queue_pos;
@@ -561,11 +561,11 @@ public:
 
   // -- pg merge --
   ceph::mutex merge_lock = ceph::make_mutex("OSD::merge_lock");
-  map<pg_t,eversion_t> ready_to_merge_source;   // pg -> version
-  map<pg_t,std::tuple<eversion_t,epoch_t,epoch_t>> ready_to_merge_target;  // pg -> (version,les,lec)
-  set<pg_t> not_ready_to_merge_source;
-  map<pg_t,pg_t> not_ready_to_merge_target;
-  set<pg_t> sent_ready_to_merge_source;
+  std::map<pg_t,eversion_t> ready_to_merge_source;   // pg -> version
+  std::map<pg_t,std::tuple<eversion_t,epoch_t,epoch_t>> ready_to_merge_target;  // pg -> (version,les,lec)
+  std::set<pg_t> not_ready_to_merge_source;
+  std::map<pg_t,pg_t> not_ready_to_merge_target;
+  std::set<pg_t> sent_ready_to_merge_source;
 
   void set_ready_to_merge_source(PG *pg,
                                 eversion_t version);
@@ -585,22 +585,22 @@ public:
 private:
   ceph::mutex pg_temp_lock = ceph::make_mutex("OSDService::pg_temp_lock");
   struct pg_temp_t {
-    vector<int> acting;
+    std::vector<int> acting;
     bool forced = false;
   };
-  map<pg_t, pg_temp_t> pg_temp_wanted;
-  map<pg_t, pg_temp_t> pg_temp_pending;
+  std::map<pg_t, pg_temp_t> pg_temp_wanted;
+  std::map<pg_t, pg_temp_t> pg_temp_pending;
   void _sent_pg_temp();
   friend std::ostream& operator<<(std::ostream&, const pg_temp_t&);
 public:
-  void queue_want_pg_temp(pg_t pgid, const vector<int>& want,
+  void queue_want_pg_temp(pg_t pgid, const std::vector<int>& want,
                          bool forced = false);
   void remove_want_pg_temp(pg_t pgid);
   void requeue_pg_temp();
   void send_pg_temp();
 
   ceph::mutex pg_created_lock = ceph::make_mutex("OSDService::pg_created_lock");
-  set<pg_t> pg_created;
+  std::set<pg_t> pg_created;
   void send_pg_created(pg_t pgid);
   void prune_pg_created();
   void send_pg_created();
@@ -615,19 +615,19 @@ public:
 private:
   // -- pg recovery and associated throttling --
   ceph::mutex recovery_lock = ceph::make_mutex("OSDService::recovery_lock");
-  list<pair<epoch_t, PGRef> > awaiting_throttle;
+  std::list<std::pair<epoch_t, PGRef> > awaiting_throttle;
 
   utime_t defer_recovery_until;
   uint64_t recovery_ops_active;
   uint64_t recovery_ops_reserved;
   bool recovery_paused;
 #ifdef DEBUG_RECOVERY_OIDS
-  map<spg_t, set<hobject_t> > recovery_oids;
+  std::map<spg_t, std::set<hobject_t> > recovery_oids;
 #endif
   bool _recover_now(uint64_t *available_pushes);
   void _maybe_queue_recovery();
   void _queue_for_recovery(
-    pair<epoch_t, PGRef> p, uint64_t reserved_pushes);
+    std::pair<epoch_t, PGRef> p, uint64_t reserved_pushes);
 public:
   void start_recovery_op(PG *pg, const hobject_t& soid);
   void finish_recovery_op(PG *pg, const hobject_t& soid, bool dequeue);
@@ -669,15 +669,15 @@ public:
     std::lock_guard l(recovery_lock);
 
     if (pg->is_forced_recovery_or_backfill()) {
-      awaiting_throttle.push_front(make_pair(pg->get_osdmap()->get_epoch(), pg));
+      awaiting_throttle.push_front(std::make_pair(pg->get_osdmap()->get_epoch(), pg));
     } else {
-      awaiting_throttle.push_back(make_pair(pg->get_osdmap()->get_epoch(), pg));
+      awaiting_throttle.push_back(std::make_pair(pg->get_osdmap()->get_epoch(), pg));
     }
     _maybe_queue_recovery();
   }
   void queue_recovery_after_sleep(PG *pg, epoch_t queued, uint64_t reserved_pushes) {
     std::lock_guard l(recovery_lock);
-    _queue_for_recovery(make_pair(queued, pg), reserved_pushes);
+    _queue_for_recovery(std::make_pair(queued, pg), reserved_pushes);
   }
 
   void queue_check_readable(spg_t spgid,
@@ -687,8 +687,8 @@ public:
   // osd map cache (past osd maps)
   ceph::mutex map_cache_lock = ceph::make_mutex("OSDService::map_cache_lock");
   SharedLRU<epoch_t, const OSDMap> map_cache;
-  SimpleLRU<epoch_t, bufferlist> map_bl_cache;
-  SimpleLRU<epoch_t, bufferlist> map_bl_inc_cache;
+  SimpleLRU<epoch_t, ceph::buffer::list> map_bl_cache;
+  SimpleLRU<epoch_t, ceph::buffer::list> map_bl_inc_cache;
 
   OSDMapRef try_get_map(epoch_t e);
   OSDMapRef get_map(epoch_t e) {
@@ -702,23 +702,23 @@ public:
   }
   OSDMapRef _add_map(OSDMap *o);
 
-  void _add_map_bl(epoch_t e, bufferlist& bl);
-  bool get_map_bl(epoch_t e, bufferlist& bl) {
+  void _add_map_bl(epoch_t e, ceph::buffer::list& bl);
+  bool get_map_bl(epoch_t e, ceph::buffer::list& bl) {
     std::lock_guard l(map_cache_lock);
     return _get_map_bl(e, bl);
   }
-  bool _get_map_bl(epoch_t e, bufferlist& bl);
+  bool _get_map_bl(epoch_t e, ceph::buffer::list& bl);
 
-  void _add_map_inc_bl(epoch_t e, bufferlist& bl);
-  bool get_inc_map_bl(epoch_t e, bufferlist& bl);
+  void _add_map_inc_bl(epoch_t e, ceph::buffer::list& bl);
+  bool get_inc_map_bl(epoch_t e, ceph::buffer::list& bl);
 
   /// identify split child pgids over a osdmap interval
   void identify_splits_and_merges(
     OSDMapRef old_map,
     OSDMapRef new_map,
     spg_t pgid,
-    set<pair<spg_t,epoch_t>> *new_children,
-    set<pair<spg_t,epoch_t>> *merge_pgs);
+    std::set<std::pair<spg_t,epoch_t>> *new_children,
+    std::set<std::pair<spg_t,epoch_t>> *merge_pgs);
 
   void need_heartbeat_peer_update();
 
@@ -735,7 +735,7 @@ public:
 
   void set_statfs(const struct store_statfs_t &stbuf,
     osd_alert_list_t& alerts);
-  osd_stat_t set_osd_stat(vector<int>& hb_peers, int num_pgs);
+  osd_stat_t set_osd_stat(std::vector<int>& hb_peers, int num_pgs);
   void inc_osd_stat_repaired(void);
   float compute_adjusted_ratio(osd_stat_t new_stat, float *pratio, uint64_t adjust_used = 0);
   osd_stat_t get_osd_stat() {
@@ -749,7 +749,7 @@ public:
     std::lock_guard l(stat_lock);
     return osd_stat.seq;
   }
-  void get_hb_pingtime(map<int, osd_stat_t::Interfaces> *pp)
+  void get_hb_pingtime(std::map<int, osd_stat_t::Interfaces> *pp)
   {
     std::lock_guard l(stat_lock);
     *pp = osd_stat.hb_pingtime;
@@ -771,7 +771,7 @@ private:
     default: return "???";
     }
   }
-  s_names get_full_state(string type) const {
+  s_names get_full_state(std::string type) const {
     if (type == "none")
       return NONE;
     else if (type == "failsafe")
@@ -793,7 +793,7 @@ private:
   bool _check_full(DoutPrefixProvider *dpp, s_names type) const;
 public:
   void check_full_status(float ratio, float pratio);
-  s_names recalc_full_state(float ratio, float pratio, string &inject);
+  s_names recalc_full_state(float ratio, float pratio, std::string &inject);
   bool _tentative_full(DoutPrefixProvider *dpp, s_names type, uint64_t adjust_used, osd_stat_t);
   bool check_failsafe_full(DoutPrefixProvider *dpp) const;
   bool check_full(DoutPrefixProvider *dpp) const;
@@ -817,13 +817,13 @@ private:
   epoch_t bind_epoch;  // epoch we last did a bind to new ip:ports
 public:
   /**
-   * Retrieve the boot_, up_, and bind_ epochs the OSD has set. The params
+   * Retrieve the boot_, up_, and bind_ epochs the OSD has std::set. The params
    * can be NULL if you don't care about them.
    */
   void retrieve_epochs(epoch_t *_boot_epoch, epoch_t *_up_epoch,
                        epoch_t *_bind_epoch) const;
   /**
-   * Set the boot, up, and bind epochs. Any NULL params will not be set.
+   * Std::set the boot, up, and bind epochs. Any NULL params will not be std::set.
    */
   void set_epochs(const epoch_t *_boot_epoch, const epoch_t *_up_epoch,
                   const epoch_t *_bind_epoch);
@@ -849,7 +849,7 @@ public:
   ceph::mutex hb_stamp_lock = ceph::make_mutex("OSDServce::hb_stamp_lock");
 
   /// osd -> heartbeat stamps
-  vector<HeartbeatStampsRef> hb_stamps;
+  std::vector<HeartbeatStampsRef> hb_stamps;
 
   /// get or create a ref for a peer's HeartbeatStamps
   HeartbeatStampsRef get_hb_stamps(unsigned osd);
@@ -886,8 +886,8 @@ public:
 
 #ifdef PG_DEBUG_REFS
   ceph::mutex pgid_lock = ceph::make_mutex("OSDService::pgid_lock");
-  map<spg_t, int> pgid_tracker;
-  map<spg_t, PG*> live_pgs;
+  std::map<spg_t, int> pgid_tracker;
+  std::map<spg_t, PG*> live_pgs;
   void add_pgid(spg_t pgid, PG *pg);
   void remove_pgid(spg_t pgid, PG *pg);
   void dump_live_pgids();
@@ -904,7 +904,7 @@ public:
 
   These are the constraints:
 
-  - client ops must remained ordered by client, regardless of map epoch
+  - client ops must remained ordered by client, regardless of std::map epoch
   - peering messages/events from peers must remain ordered by peer
   - peering messages and client ops need not be ordered relative to each other
 
@@ -943,27 +943,27 @@ public:
 
   - when we advance the osdmap on the OSDShard, we scan pg slots and
     discard any slots with no pg (and not waiting_for_split) that no
-    longer map to the current host.
+    longer std::map to the current host.
 
   */
 
 struct OSDShardPGSlot {
   using OpSchedulerItem = ceph::osd::scheduler::OpSchedulerItem;
   PGRef pg;                      ///< pg reference
-  deque<OpSchedulerItem> to_process; ///< order items for this slot
+  std::deque<OpSchedulerItem> to_process; ///< order items for this slot
   int num_running = 0;          ///< _process threads doing pg lookup/lock
 
-  deque<OpSchedulerItem> waiting;   ///< waiting for pg (or map + pg)
+  std::deque<OpSchedulerItem> waiting;   ///< waiting for pg (or map + pg)
 
   /// waiting for map (peering evt)
-  map<epoch_t,deque<OpSchedulerItem>> waiting_peering;
+  std::map<epoch_t,std::deque<OpSchedulerItem>> waiting_peering;
 
   /// incremented by wake_pg_waiters; indicates racing _process threads
   /// should bail out (their op has been requeued)
   uint64_t requeue_seq = 0;
 
   /// waiting for split child to materialize in these epoch(s)
-  set<epoch_t> waiting_for_split;
+  std::set<epoch_t> waiting_for_split;
 
   epoch_t epoch = 0;
   boost::intrusive::set_member_hook<> pg_epoch_item;
@@ -977,9 +977,9 @@ struct OSDShard {
   CephContext *cct;
   OSD *osd;
 
-  string shard_name;
+  std::string shard_name;
 
-  string sdata_wait_lock_name;
+  std::string sdata_wait_lock_name;
   ceph::mutex sdata_wait_lock;
   ceph::condition_variable sdata_cond;
 
@@ -991,13 +991,13 @@ struct OSDShard {
     return shard_osdmap;
   }
 
-  string shard_lock_name;
+  std::string shard_lock_name;
   ceph::mutex shard_lock;   ///< protects remaining members below
 
   /// map of slots for each spg_t.  maintains ordering of items dequeued
   /// from scheduler while _process thread drops shard lock to acquire the
   /// pg lock.  stale slots are removed by consume_map.
-  unordered_map<spg_t,unique_ptr<OSDShardPGSlot>> pg_slots;
+  std::unordered_map<spg_t,std::unique_ptr<OSDShardPGSlot>> pg_slots;
 
   struct pg_slot_compare_by_epoch {
     bool operator()(const OSDShardPGSlot& l, const OSDShardPGSlot& r) const {
@@ -1042,13 +1042,13 @@ struct OSDShard {
 
   void identify_splits_and_merges(
     const OSDMapRef& as_of_osdmap,
-    set<pair<spg_t,epoch_t>> *split_children,
-    set<pair<spg_t,epoch_t>> *merge_pgs);
-  void _prime_splits(set<pair<spg_t,epoch_t>> *pgids);
+    std::set<std::pair<spg_t,epoch_t>> *split_children,
+    std::set<std::pair<spg_t,epoch_t>> *merge_pgs);
+  void _prime_splits(std::set<std::pair<spg_t,epoch_t>> *pgids);
   void prime_splits(const OSDMapRef& as_of_osdmap,
-                   set<pair<spg_t,epoch_t>> *pgids);
+                   std::set<std::pair<spg_t,epoch_t>> *pgids);
   void prime_merges(const OSDMapRef& as_of_osdmap,
-                   set<pair<spg_t,epoch_t>> *merge_pgs);
+                   std::set<std::pair<spg_t,epoch_t>> *merge_pgs);
   void register_and_wake_split_child(PG *pg);
   void unprime_split_children(spg_t parent, unsigned old_pg_num);
 
@@ -1127,9 +1127,9 @@ protected:
   void asok_command(
     std::string_view prefix,
     const cmdmap_t& cmdmap,
-    Formatter *f,
-    const bufferlist& inbl,
-    std::function<void(int,const std::string&,bufferlist&)> on_finish);
+    ceph::Formatter *f,
+    const ceph::buffer::list& inbl,
+    std::function<void(int,const std::string&,ceph::buffer::list&)> on_finish);
 
 public:
   int get_nodeid() { return whoami; }
@@ -1159,17 +1159,17 @@ public:
   }
 
   static ghobject_t make_pg_log_oid(spg_t pg) {
-    stringstream ss;
+    std::stringstream ss;
     ss << "pglog_" << pg;
-    string s;
+    std::string s;
     getline(ss, s);
     return ghobject_t(hobject_t(sobject_t(object_t(s.c_str()), 0)));
   }
   
   static ghobject_t make_pg_biginfo_oid(spg_t pg) {
-    stringstream ss;
+    std::stringstream ss;
     ss << "pginfo_" << pg;
-    string s;
+    std::string s;
     getline(ss, s);
     return ghobject_t(hobject_t(sobject_t(object_t(s.c_str()), 0)));
   }
@@ -1182,7 +1182,7 @@ public:
     return ghobject_t(
       hobject_t(
        sobject_t(
-         object_t(string("final_pool_") + stringify(pool)),
+         object_t(std::string("final_pool_") + stringify(pool)),
          CEPH_NOSNAP)));
   }
 
@@ -1198,7 +1198,7 @@ public:
   /**
    * get_osd_initial_compat_set()
    *
-   * Get the initial feature set for this OSD.  Features
+   * Get the initial feature std::set for this OSD.  Features
    * here are automatically upgraded.
    *
    * Return value: Initial osd CompatSet
@@ -1296,10 +1296,10 @@ private:
   void dispatch_session_waiting(const ceph::ref_t<Session>& session, OSDMapRef osdmap);
 
   ceph::mutex session_waiting_lock = ceph::make_mutex("OSD::session_waiting_lock");
-  set<ceph::ref_t<Session>> session_waiting_for_map;
+  std::set<ceph::ref_t<Session>> session_waiting_for_map;
 
   /// Caller assumes refs for included Sessions
-  void get_sessions_waiting_for_map(set<ceph::ref_t<Session>> *out) {
+  void get_sessions_waiting_for_map(std::set<ceph::ref_t<Session>> *out) {
     std::lock_guard l(session_waiting_lock);
     out->swap(session_waiting_for_map);
   }
@@ -1312,7 +1312,7 @@ private:
     session_waiting_for_map.erase(session);
   }
   void dispatch_sessions_waiting_on_map() {
-    set<ceph::ref_t<Session>> sessions_to_check;
+    std::set<ceph::ref_t<Session>> sessions_to_check;
     get_sessions_waiting_for_map(&sessions_to_check);
     for (auto i = sessions_to_check.begin();
         i != sessions_to_check.end();
@@ -1371,7 +1371,7 @@ private:
     static constexpr int HEARTBEAT_MAX_CONN = 2;
     /// history of inflight pings, arranging by timestamp we sent
     /// send time -> deadline -> remaining replies
-    map<utime_t, pair<utime_t, int>> ping_history;
+    std::map<utime_t, std::pair<utime_t, int>> ping_history;
 
     utime_t hb_interval_start;
     uint32_t hb_average_count = 0;
@@ -1380,16 +1380,16 @@ private:
     uint32_t hb_total_back = 0;
     uint32_t hb_min_back = UINT_MAX;
     uint32_t hb_max_back = 0;
-    vector<uint32_t> hb_back_pingtime;
-    vector<uint32_t> hb_back_min;
-    vector<uint32_t> hb_back_max;
+    std::vector<uint32_t> hb_back_pingtime;
+    std::vector<uint32_t> hb_back_min;
+    std::vector<uint32_t> hb_back_max;
 
     uint32_t hb_total_front = 0;
     uint32_t hb_min_front = UINT_MAX;
     uint32_t hb_max_front = 0;
-    vector<uint32_t> hb_front_pingtime;
-    vector<uint32_t> hb_front_min;
-    vector<uint32_t> hb_front_max;
+    std::vector<uint32_t> hb_front_pingtime;
+    std::vector<uint32_t> hb_front_min;
+    std::vector<uint32_t> hb_front_max;
 
     bool is_stale(utime_t stale) {
       if (ping_history.empty()) {
@@ -1434,11 +1434,11 @@ private:
   };
 
   ceph::mutex heartbeat_lock = ceph::make_mutex("OSD::heartbeat_lock");
-  map<int, int> debug_heartbeat_drops_remaining;
+  std::map<int, int> debug_heartbeat_drops_remaining;
   ceph::condition_variable heartbeat_cond;
   bool heartbeat_stop;
   std::atomic<bool> heartbeat_need_update;   
-  map<int,HeartbeatInfo> heartbeat_peers;  ///< map of osd id to HeartbeatInfo
+  std::map<int,HeartbeatInfo> heartbeat_peers;  ///< map of osd id to HeartbeatInfo
   utime_t last_mon_heartbeat;
   Messenger *hb_front_client_messenger;
   Messenger *hb_back_client_messenger;
@@ -1522,9 +1522,9 @@ public:
 
 private:
   // -- waiters --
-  list<OpRequestRef> finished;
+  std::list<OpRequestRef> finished;
   
-  void take_waiters(list<OpRequestRef>& ls) {
+  void take_waiters(std::list<OpRequestRef>& ls) {
     ceph_assert(ceph_mutex_is_locked(osd_lock));
     finished.splice(finished.end(), ls);
   }
@@ -1532,7 +1532,7 @@ private:
   
   // -- op tracking --
   OpTracker op_tracker;
-  void test_ops(std::string command, std::string args, ostream& ss);
+  void test_ops(std::string command, std::string args, std::ostream& ss);
   friend class TestOpsSocketHook;
   TestOpsSocketHook *test_ops_hook;
   friend struct C_FinishSplits;
@@ -1585,14 +1585,14 @@ protected:
       OpSchedulerItem&& qi);
 
     /// try to do some work
-    void _process(uint32_t thread_index, heartbeat_handle_d *hb) override;
+    void _process(uint32_t thread_index, ceph::heartbeat_handle_d *hb) override;
 
     /// enqueue a new item
     void _enqueue(OpSchedulerItem&& item) override;
 
     /// requeue an old item (at the front of the line)
     void _enqueue_front(OpSchedulerItem&& item) override;
-      
+
     void return_waiting_threads() override {
       for(uint32_t i = 0; i < osd->num_shards; i++) {
        OSDShard* sdata = osd->shards[i];
@@ -1612,7 +1612,7 @@ protected:
       }
     }
 
-    void dump(Formatter *f) {
+    void dump(ceph::Formatter *f) {
       for(uint32_t i = 0; i < osd->num_shards; i++) {
        auto &&sdata = osd->shards[i];
 
@@ -1639,7 +1639,7 @@ protected:
       }
     }
 
-    void handle_oncommits(list<Context*>& oncommits) {
+    void handle_oncommits(std::list<Context*>& oncommits) {
       for (auto p : oncommits) {
        p->complete(0);
       }
@@ -1668,7 +1668,7 @@ protected:
     ThreadPool::TPHandle& handle);
 
   friend class PG;
-  friend class OSDShard;
+  friend struct OSDShard;
   friend class PrimaryLogPG;
 
 
@@ -1692,8 +1692,8 @@ protected:
   pool_pg_num_history_t pg_num_history;
 
   ceph::shared_mutex map_lock = ceph::make_shared_mutex("OSD::map_lock");
-  list<OpRequestRef>  waiting_for_osdmap;
-  deque<utime_t> osd_markdown_log;
+  std::list<OpRequestRef>  waiting_for_osdmap;
+  std::deque<utime_t> osd_markdown_log;
 
   friend struct send_map_on_destruct;
 
@@ -1703,7 +1703,7 @@ protected:
   void trim_maps(epoch_t oldest, int nreceived, bool skip_maps);
   void note_down_osd(int osd);
   void note_up_osd(int osd);
-  friend class C_OnMapCommit;
+  friend struct C_OnMapCommit;
 
   bool advance_pg(
     epoch_t advance_to,
@@ -1720,13 +1720,13 @@ protected:
   OSDMapRef add_map(OSDMap *o) {
     return service.add_map(o);
   }
-  bool get_map_bl(epoch_t e, bufferlist& bl) {
+  bool get_map_bl(epoch_t e, ceph::buffer::list& bl) {
     return service.get_map_bl(e, bl);
   }
 
 public:
   // -- shards --
-  vector<OSDShard*> shards;
+  std::vector<OSDShard*> shards;
   uint32_t num_shards = 0;
 
   void inc_num_pgs() {
@@ -1742,7 +1742,7 @@ public:
 protected:
   ceph::mutex merge_lock = ceph::make_mutex("OSD::merge_lock");
   /// merge epoch -> target pgid -> source pgid -> pg
-  map<epoch_t,map<spg_t,map<spg_t,PGRef>>> merge_waiters;
+  std::map<epoch_t,std::map<spg_t,std::map<spg_t,PGRef>>> merge_waiters;
 
   bool add_merge_waiter(OSDMapRef nextmap, spg_t target, PGRef source,
                        unsigned need);
@@ -1762,8 +1762,8 @@ protected:
   void register_pg(PGRef pg);
   bool try_finish_pg_delete(PG *pg, unsigned old_pg_num);
 
-  void _get_pgs(vector<PGRef> *v, bool clear_too=false);
-  void _get_pgids(vector<spg_t> *v);
+  void _get_pgs(std::vector<PGRef> *v, bool clear_too=false);
+  void _get_pgids(std::vector<spg_t> *v);
 
 public:
   PGRef lookup_lock_pg(spg_t pgid);
@@ -1793,11 +1793,11 @@ protected:
 
   void split_pgs(
     PG *parent,
-    const set<spg_t> &childpgids, set<PGRef> *out_pgs,
+    const std::set<spg_t> &childpgids, std::set<PGRef> *out_pgs,
     OSDMapRef curmap,
     OSDMapRef nextmap,
     PeeringCtx &rctx);
-  void _finish_splits(set<PGRef>& pgs);
+  void _finish_splits(std::set<PGRef>& pgs);
 
   // == monitor interaction ==
   ceph::mutex mon_report_lock = ceph::make_mutex("OSD::mon_report_lock");
@@ -1809,7 +1809,7 @@ protected:
   void _got_mon_epochs(epoch_t oldest, epoch_t newest);
   void _preboot(epoch_t oldest, epoch_t newest);
   void _send_boot();
-  void _collect_metadata(map<string,string> *pmeta);
+  void _collect_metadata(std::map<std::string,std::string> *pmeta);
   void _get_purged_snaps();
   void handle_get_purged_snaps_reply(MMonGetPurgedSnapsReply *r);
 
@@ -1840,8 +1840,8 @@ protected:
   void got_full_map(epoch_t e);
 
   // -- failures --
-  map<int,utime_t> failure_queue;
-  map<int,pair<utime_t,entity_addrvec_t> > failure_pending;
+  std::map<int,utime_t> failure_queue;
+  std::map<int,std::pair<utime_t,entity_addrvec_t> > failure_pending;
 
   void requeue_failures();
   void send_failures();
@@ -1876,7 +1876,7 @@ protected:
   bool require_self_aliveness(const Message *m, epoch_t alive_since);
   /**
    * Verifies that the OSD who sent the given op has the same
-   * address as in the given map.
+   * address as in the given std::map.
    * @pre op was sent by an OSD using the cluster messenger
    */
   bool require_same_peer_instance(const Message *m, const OSDMapRef& map,
@@ -1998,9 +1998,9 @@ private:
   // static bits
   static int mkfs(CephContext *cct, ObjectStore *store, uuid_d fsid, int whoami);
 
-  /* remove any non-user xattrs from a map of them */
-  void filter_xattrs(map<string, bufferptr>& attrs) {
-    for (map<string, bufferptr>::iterator iter = attrs.begin();
+  /* remove any non-user xattrs from a std::map of them */
+  void filter_xattrs(std::map<std::string, ceph::buffer::ptr>& attrs) {
+    for (std::map<std::string, ceph::buffer::ptr>::iterator iter = attrs.begin();
         iter != attrs.end();
         ) {
       if (('_' != iter->first.at(0)) || (iter->first.size() == 1))
@@ -2010,7 +2010,7 @@ private:
   }
 
 private:
-  int mon_cmd_maybe_osd_create(string &cmd);
+  int mon_cmd_maybe_osd_create(std::string &cmd);
   int update_crush_device_class();
   int update_crush_location();
 
@@ -2018,8 +2018,8 @@ private:
                        ObjectStore *store,
                        uuid_d& cluster_fsid, uuid_d& osd_fsid, int whoami);
 
-  void handle_scrub(struct MOSDScrub *m);
-  void handle_fast_scrub(struct MOSDScrub2 *m);
+  void handle_scrub(class MOSDScrub *m);
+  void handle_fast_scrub(class MOSDScrub2 *m);
   void handle_osd_ping(class MOSDPing *m);
 
   size_t get_num_cache_shards();
@@ -2033,11 +2033,11 @@ private:
   int get_recovery_max_active();
 
   void scrub_purged_snaps();
-  void probe_smart(const string& devid, ostream& ss);
+  void probe_smart(const std::string& devid, std::ostream& ss);
 
 public:
   static int peek_meta(ObjectStore *store,
-                      string *magic,
+                      std::string *magic,
                       uuid_d *cluster_fsid,
                       uuid_d *osd_fsid,
                       int *whoami,
index 0649e1c9e770d3e94a1ffab4203dcdf2da0b7cd3..cd3081d202ea4f39a4603fc32f2db975db5fbdd2 100644 (file)
 #undef dout_prefix
 #define dout_prefix _prefix(_dout, this)
 
+using std::list;
+using std::map;
+using std::ostringstream;
+using std::pair;
+using std::set;
+using std::string;
+using std::stringstream;
+using std::unique_ptr;
+using std::vector;
+
+using ceph::bufferlist;
+using ceph::bufferptr;
+using ceph::decode;
+using ceph::encode;
+using ceph::Formatter;
+
 using namespace ceph::osd::scheduler;
 
 template <class T>
@@ -1262,9 +1278,7 @@ void PG::filter_snapc(vector<snapid_t> &snaps)
 
 void PG::requeue_object_waiters(map<hobject_t, list<OpRequestRef>>& m)
 {
-  for (map<hobject_t, list<OpRequestRef>>::iterator it = m.begin();
-       it != m.end();
-       ++it)
+  for (auto it = m.begin(); it != m.end(); ++it)
     requeue_ops(it->second);
   m.clear();
 }
index d187c3a630c20e740045bbd488c02e68738d307d..2f384f582aa36b5db36a5e4cd69c9d6d05949225 100644 (file)
@@ -55,7 +55,7 @@
 #include <string>
 #include <tuple>
 
-//#define DEBUG_RECOVERY_OIDS   // track set of recovering oids explicitly, to find counting bugs
+//#define DEBUG_RECOVERY_OIDS   // track std::set of recovering oids explicitly, to find counting bugs
 //#define PG_DEBUG_REFS    // track provenance of pg refs, helpful for finding leaks
 
 class OSD;
@@ -97,7 +97,7 @@ class PGRecoveryStats {
     // cppcheck-suppress unreachableCode
     per_state_info() : enter(0), exit(0), events(0) {}
   };
-  map<const char *,per_state_info> info;
+  std::map<const char *,per_state_info> info;
   ceph::mutex lock = ceph::make_mutex("PGRecoverStats::lock");
 
   public:
@@ -109,7 +109,7 @@ class PGRecoveryStats {
   }
   void dump(ostream& out) {
     std::lock_guard l(lock);
-    for (map<const char *,per_state_info>::iterator p = info.begin(); p != info.end(); ++p) {
+    for (std::map<const char *,per_state_info>::iterator p = info.begin(); p != info.end(); ++p) {
       per_state_info& i = p->second;
       out << i.enter << "\t" << i.exit << "\t"
          << i.events << "\t" << i.event_time << "\t"
@@ -119,10 +119,10 @@ class PGRecoveryStats {
     }
   }
 
-  void dump_formatted(Formatter *f) {
+  void dump_formatted(ceph::Formatter *f) {
     std::lock_guard l(lock);
     f->open_array_section("pg_recovery_stats");
-    for (map<const char *,per_state_info>::iterator p = info.begin();
+    for (std::map<const char *,per_state_info>::iterator p = info.begin();
         p != info.end(); ++p) {
       per_state_info& i = p->second;
       f->open_object_section("recovery_state");
@@ -133,10 +133,10 @@ class PGRecoveryStats {
       f->dump_stream("total_time") << i.total_time;
       f->dump_stream("min_time") << i.min_time;
       f->dump_stream("max_time") << i.max_time;
-      vector<string> states;
+      std::vector<std::string> states;
       get_str_vec(p->first, "/", states);
       f->open_array_section("nested_states");
-      for (vector<string>::iterator st = states.begin();
+      for (std::vector<std::string>::iterator st = states.begin();
           st != states.end(); ++st) {
        f->dump_string("state", *st);
       }
@@ -169,7 +169,7 @@ class PGRecoveryStats {
  */
 
 class PG : public DoutPrefixProvider, public PeeringState::PeeringListener {
-  friend class NamedState;
+  friend struct NamedState;
   friend class PeeringState;
 
 public:
@@ -291,10 +291,10 @@ public:
   int get_role() const {
     return recovery_state.get_role();
   }
-  const vector<int> get_acting() const {
+  const std::vector<int> get_acting() const {
     return recovery_state.get_acting();
   }
-  const set<pg_shard_t> &get_actingset() const {
+  const std::set<pg_shard_t> &get_actingset() const {
     return recovery_state.get_actingset();
   }
   int get_acting_primary() const {
@@ -303,7 +303,7 @@ public:
   pg_shard_t get_primary() const {
     return recovery_state.get_primary();
   }
-  const vector<int> get_up() const {
+  const std::vector<int> get_up() const {
     return recovery_state.get_up();
   }
   int get_up_primary() const {
@@ -315,7 +315,7 @@ public:
   bool is_acting_recovery_backfill(pg_shard_t osd) const {
     return recovery_state.is_acting_recovery_backfill(osd);
   }
-  const set<pg_shard_t> &get_acting_recovery_backfill() const {
+  const std::set<pg_shard_t> &get_acting_recovery_backfill() const {
     return recovery_state.get_acting_recovery_backfill();
   }
   bool is_acting(pg_shard_t osd) const {
@@ -324,16 +324,16 @@ public:
   bool is_up(pg_shard_t osd) const {
     return recovery_state.is_up(osd);
   }
-  static bool has_shard(bool ec, const vector<int>& v, pg_shard_t osd) {
+  static bool has_shard(bool ec, const std::vector<int>& v, pg_shard_t osd) {
     return PeeringState::has_shard(ec, v, osd);
   }
 
   /// initialize created PG
   void init(
     int role,
-    const vector<int>& up,
+    const std::vector<int>& up,
     int up_primary,
-    const vector<int>& acting,
+    const std::vector<int>& acting,
     int acting_primary,
     const pg_history_t& history,
     const PastIntervals& pim,
@@ -361,7 +361,7 @@ public:
   void update_snap_mapper_bits(uint32_t bits) {
     snap_mapper.update_bits(bits);
   }
-  void start_split_stats(const set<spg_t>& childpgs, vector<object_stat_sum_t> *v);
+  void start_split_stats(const std::set<spg_t>& childpgs, std::vector<object_stat_sum_t> *v);
   virtual void split_colls(
     spg_t child,
     int split_bits,
@@ -369,7 +369,7 @@ public:
     const pg_pool_t *pool,
     ObjectStore::Transaction &t) = 0;
   void split_into(pg_t child_pgid, PG *child, unsigned split_bits);
-  void merge_from(map<spg_t,PGRef>& sources, PeeringCtx &rctx,
+  void merge_from(std::map<spg_t,PGRef>& sources, PeeringCtx &rctx,
                  unsigned split_bits,
                  const pg_merge_meta_t& last_pg_merge_meta);
   void finish_split_stats(const object_stat_sum_t& stats,
@@ -381,7 +381,7 @@ public:
   void reg_next_scrub();
   void unreg_next_scrub();
 
-  void queue_want_pg_temp(const vector<int> &wanted) override;
+  void queue_want_pg_temp(const std::vector<int> &wanted) override;
   void clear_want_pg_temp() override;
 
   void on_new_interval() override;
@@ -492,12 +492,12 @@ public:
   void queue_flushed(epoch_t started_at);
   void handle_advance_map(
     OSDMapRef osdmap, OSDMapRef lastmap,
-    vector<int>& newup, int up_primary,
-    vector<int>& newacting, int acting_primary,
+    std::vector<int>& newup, int up_primary,
+    std::vector<int>& newacting, int acting_primary,
     PeeringCtx &rctx);
   void handle_activate_map(PeeringCtx &rctx);
   void handle_initialize(PeeringCtx &rxcx);
-  void handle_query_state(Formatter *f);
+  void handle_query_state(ceph::Formatter *f);
 
   /**
    * @param ops_begun returns how many recovery ops the function started
@@ -513,8 +513,8 @@ public:
 
   virtual void get_watchers(std::list<obj_watch_item_t> *ls) = 0;
 
-  void dump_pgstate_history(Formatter *f);
-  void dump_missing(Formatter *f);
+  void dump_pgstate_history(ceph::Formatter *f);
+  void dump_missing(ceph::Formatter *f);
 
   void get_pg_stats(std::function<void(const pg_stat_t&, epoch_t lec)> f);
   void with_heartbeat_peers(std::function<void(int)> f);
@@ -536,10 +536,10 @@ public:
 
   virtual void snap_trimmer(epoch_t epoch_queued) = 0;
   virtual void do_command(
-    const string_view& prefix,
+    const std::string_view& prefix,
     const cmdmap_t& cmdmap,
-    const bufferlist& idata,
-    std::function<void(int,const std::string&,bufferlist&)> on_finish) = 0;
+    const ceph::buffer::list& idata,
+    std::function<void(int,const std::string&,ceph::buffer::list&)> on_finish) = 0;
 
   virtual bool agent_work(int max) = 0;
   virtual bool agent_work(int max, int agent_flush_quota) = 0;
@@ -613,8 +613,8 @@ protected:
 
 #ifdef PG_DEBUG_REFS
   ceph::mutex _ref_id_lock = ceph::make_mutex("PG::_ref_id_lock");
-  map<uint64_t, string> _live_ids;
-  map<string, uint64_t> _tag_counts;
+  std::map<uint64_t, std::string> _live_ids;
+  std::map<std::string, uint64_t> _tag_counts;
   uint64_t _ref_id = 0;
 
   friend uint64_t get_with_id(PG *pg) { return pg->get_with_id(); }
@@ -657,7 +657,7 @@ protected:
 
   // ------------------
   interval_set<snapid_t> snap_trimq;
-  set<snapid_t> snap_trimq_repeat;
+  std::set<snapid_t> snap_trimq_repeat;
 
   /* You should not use these items without taking their respective queue locks
    * (if they have one) */
@@ -666,7 +666,7 @@ protected:
   bool recovery_queued;
 
   int recovery_ops_active;
-  set<pg_shard_t> waiting_on_backfill;
+  std::set<pg_shard_t> waiting_on_backfill;
 #ifdef DEBUG_RECOVERY_OIDS
   multiset<hobject_t> recovering_oids;
 #endif
@@ -683,13 +683,13 @@ protected:
   }
 
   /* heartbeat peers */
-  void set_probe_targets(const set<pg_shard_t> &probe_set) override;
+  void set_probe_targets(const std::set<pg_shard_t> &probe_set) override;
   void clear_probe_targets() override;
 
   ceph::mutex heartbeat_peer_lock =
     ceph::make_mutex("PG::heartbeat_peer_lock");
-  set<int> heartbeat_peers;
-  set<int> probe_targets;
+  std::set<int> heartbeat_peers;
+  std::set<int> probe_targets;
 
 public:
   /**
@@ -704,7 +704,7 @@ public:
   struct BackfillInterval {
     // info about a backfill interval on a peer
     eversion_t version; /// version at which the scan occurred
-    map<hobject_t,eversion_t> objects;
+    std::map<hobject_t,eversion_t> objects;
     hobject_t begin;
     hobject_t end;
 
@@ -713,7 +713,7 @@ public:
       *this = BackfillInterval();
     }
 
-    /// clear objects list only
+    /// clear objects std::list only
     void clear_objects() {
       objects.clear();
     }
@@ -759,11 +759,11 @@ public:
     }
 
     /// dump
-    void dump(Formatter *f) const {
+    void dump(ceph::Formatter *f) const {
       f->dump_stream("begin") << begin;
       f->dump_stream("end") << end;
       f->open_array_section("objects");
-      for (map<hobject_t, eversion_t>::const_iterator i =
+      for (std::map<hobject_t, eversion_t>::const_iterator i =
             objects.begin();
           i != objects.end();
           ++i) {
@@ -778,7 +778,7 @@ public:
 
 protected:
   BackfillInterval backfill_info;
-  map<pg_shard_t, BackfillInterval> peer_backfill_info;
+  std::map<pg_shard_t, BackfillInterval> peer_backfill_info;
   bool backfill_reserving;
 
   // The primary's num_bytes and local num_bytes for this pg, only valid
@@ -934,7 +934,7 @@ protected:
    *     queues because we assume they cannot apply at that time (this is
    *     probably mostly true).
    *
-   *  3. The requeue_ops helper will push ops onto the waiting_for_map list if
+   *  3. The requeue_ops helper will push ops onto the waiting_for_map std::list if
    *     it is non-empty.
    *
    * These three behaviors are generally sufficient to maintain ordering, with
@@ -946,40 +946,40 @@ protected:
   // ops with newer maps than our (or blocked behind them)
   // track these by client, since inter-request ordering doesn't otherwise
   // matter.
-  unordered_map<entity_name_t,list<OpRequestRef>> waiting_for_map;
+  std::unordered_map<entity_name_t,std::list<OpRequestRef>> waiting_for_map;
 
   // ops waiting on peered
-  list<OpRequestRef>            waiting_for_peered;
+  std::list<OpRequestRef>            waiting_for_peered;
 
   /// ops waiting on readble
-  list<OpRequestRef>            waiting_for_readable;
+  std::list<OpRequestRef>            waiting_for_readable;
 
   // ops waiting on active (require peered as well)
-  list<OpRequestRef>            waiting_for_active;
-  list<OpRequestRef>            waiting_for_flush;
-  list<OpRequestRef>            waiting_for_scrub;
+  std::list<OpRequestRef>            waiting_for_active;
+  std::list<OpRequestRef>            waiting_for_flush;
+  std::list<OpRequestRef>            waiting_for_scrub;
 
-  list<OpRequestRef>            waiting_for_cache_not_full;
-  list<OpRequestRef>            waiting_for_clean_to_primary_repair;
-  map<hobject_t, list<OpRequestRef>> waiting_for_unreadable_object,
+  std::list<OpRequestRef>            waiting_for_cache_not_full;
+  std::list<OpRequestRef>            waiting_for_clean_to_primary_repair;
+  std::map<hobject_t, std::list<OpRequestRef>> waiting_for_unreadable_object,
                             waiting_for_degraded_object,
                             waiting_for_blocked_object;
 
-  set<hobject_t> objects_blocked_on_cache_full;
-  map<hobject_t,snapid_t> objects_blocked_on_degraded_snap;
-  map<hobject_t,ObjectContextRef> objects_blocked_on_snap_promotion;
+  std::set<hobject_t> objects_blocked_on_cache_full;
+  std::map<hobject_t,snapid_t> objects_blocked_on_degraded_snap;
+  std::map<hobject_t,ObjectContextRef> objects_blocked_on_snap_promotion;
 
   // Callbacks should assume pg (and nothing else) is locked
-  map<hobject_t, list<Context*>> callbacks_for_degraded_object;
+  std::map<hobject_t, std::list<Context*>> callbacks_for_degraded_object;
 
-  map<eversion_t,
-      list<
-       tuple<OpRequestRef, version_t, int,
-             vector<pg_log_op_return_item_t>>>> waiting_for_ondisk;
+  std::map<eversion_t,
+      std::list<
+       std::tuple<OpRequestRef, version_t, int,
+                  std::vector<pg_log_op_return_item_t>>>> waiting_for_ondisk;
 
-  void requeue_object_waiters(map<hobject_t, list<OpRequestRef>>& m);
+  void requeue_object_waiters(std::map<hobject_t, std::list<OpRequestRef>>& m);
   void requeue_op(OpRequestRef op);
-  void requeue_ops(list<OpRequestRef> &l);
+  void requeue_ops(std::list<OpRequestRef> &l);
 
   // stats that persist lazily
   object_stat_collection_t unstable_stats;
@@ -1028,7 +1028,7 @@ protected:
   
   void update_object_snap_mapping(
     ObjectStore::Transaction *t, const hobject_t &soid,
-    const set<snapid_t> &snaps);
+    const std::set<snapid_t> &snaps);
   void clear_object_snap_mapping(
     ObjectStore::Transaction *t, const hobject_t &soid);
   void remove_snap_mapped_object(
@@ -1045,7 +1045,7 @@ protected:
 
   void purge_strays();
 
-  void update_heartbeat_peers(set<int> peers) override;
+  void update_heartbeat_peers(std::set<int> peers) override;
 
   Context *finish_sync_event;
 
@@ -1067,12 +1067,12 @@ protected:
   virtual void _split_into(pg_t child_pgid, PG *child, unsigned split_bits) = 0;
 
   friend class C_OSD_RepModify_Commit;
-  friend class C_DeleteMore;
+  friend struct C_DeleteMore;
 
   // -- backoff --
   ceph::mutex backoff_lock = // orders inside Backoff::lock
     ceph::make_mutex("PG::backoff_lock");
-  map<hobject_t,set<ceph::ref_t<Backoff>>> backoffs;
+  std::map<hobject_t,std::set<ceph::ref_t<Backoff>>> backoffs;
 
   void add_backoff(const ceph::ref_t<Session>& s, const hobject_t& begin, const hobject_t& end);
   void release_backoffs(const hobject_t& begin, const hobject_t& end);
@@ -1101,13 +1101,13 @@ public:
     ~Scrubber();
 
     // metadata
-    set<pg_shard_t> reserved_peers;
+    std::set<pg_shard_t> reserved_peers;
     bool local_reserved, remote_reserved, reserve_failed;
     epoch_t epoch_start;
 
     // common to both scrubs
     bool active;
-    set<pg_shard_t> waiting_on_whom;
+    std::set<pg_shard_t> waiting_on_whom;
     int shallow_errors;
     int deep_errors;
     int fixed;
@@ -1116,7 +1116,7 @@ public:
     epoch_t replica_scrub_start = 0;
     ScrubMap replica_scrubmap;
     ScrubMapBuilder replica_scrubmap_pos;
-    map<pg_shard_t, ScrubMap> received_maps;
+    std::map<pg_shard_t, ScrubMap> received_maps;
     OpRequestRef active_rep_scrub;
     utime_t scrub_reg_stamp;  // stamp we registered for
 
@@ -1145,13 +1145,13 @@ public:
     bool deep_scrub_on_error;
 
     // Maps from objects with errors to missing/inconsistent peers
-    map<hobject_t, set<pg_shard_t>> missing;
-    map<hobject_t, set<pg_shard_t>> inconsistent;
+    std::map<hobject_t, std::set<pg_shard_t>> missing;
+    std::map<hobject_t, std::set<pg_shard_t>> inconsistent;
 
-    // Map from object with errors to good peers
-    map<hobject_t, list<pair<ScrubMap::object, pg_shard_t> >> authoritative;
+    // Std::map from object with errors to good peers
+    std::map<hobject_t, std::list<std::pair<ScrubMap::object, pg_shard_t> >> authoritative;
 
-    // Cleaned map pending snap metadata scrub
+    // Cleaned std::map pending snap metadata scrub
     ScrubMap cleaned_meta_map;
 
     void clean_meta_map(ScrubMap &for_meta_scrub) {
@@ -1207,14 +1207,14 @@ public:
     int preempt_left;
     int preempt_divisor;
 
-    list<Context*> callbacks;
+    std::list<Context*> callbacks;
     void add_callback(Context *context) {
       callbacks.push_back(context);
     }
     void run_callbacks() {
-      list<Context*> to_run;
+      std::list<Context*> to_run;
       to_run.swap(callbacks);
-      for (list<Context*>::iterator i = to_run.begin();
+      for (std::list<Context*>::iterator i = to_run.begin();
           i != to_run.end();
           ++i) {
        (*i)->complete(0);
@@ -1307,8 +1307,8 @@ protected:
 
   void repair_object(
     const hobject_t &soid,
-    const list<pair<ScrubMap::object, pg_shard_t> > &ok_peers,
-    const set<pg_shard_t> &bad_peers);
+    const std::list<std::pair<ScrubMap::object, pg_shard_t> > &ok_peers,
+    const std::set<pg_shard_t> &bad_peers);
 
   void chunky_scrub(ThreadPool::TPHandle &handle);
   void scrub_compare_maps();
@@ -1321,7 +1321,7 @@ protected:
   void scrub_clear_state(bool keep_repair = false);
   void _scan_snaps(ScrubMap &map);
   void _repair_oinfo_oid(ScrubMap &map);
-  void _scan_rollback_obs(const vector<ghobject_t> &rollback_obs);
+  void _scan_rollback_obs(const std::vector<ghobject_t> &rollback_obs);
   void _request_scrub_map(pg_shard_t replica, eversion_t version,
                           hobject_t start, hobject_t end, bool deep,
                          bool allow_preemption);
@@ -1340,7 +1340,7 @@ protected:
   virtual void scrub_snapshot_metadata(
     ScrubMap &map,
     const std::map<hobject_t,
-                   pair<std::optional<uint32_t>,
+                   std::pair<std::optional<uint32_t>,
                         std::optional<uint32_t>>> &missing_digest) { }
   virtual void _scrub_clear_state() { }
   virtual void _scrub_finish() { }
@@ -1446,7 +1446,7 @@ protected:
     eversion_t *version,
     version_t *user_version,
     int *return_code,
-    vector<pg_log_op_return_item_t> *op_returns) const;
+    std::vector<pg_log_op_return_item_t> *op_returns) const;
   eversion_t projected_last_update;
   eversion_t get_next_version() const {
     eversion_t at_version(
@@ -1463,10 +1463,10 @@ protected:
   std::string get_corrupt_pg_log_name() const;
 
   void update_snap_map(
-    const vector<pg_log_entry_t> &log_entries,
+    const std::vector<pg_log_entry_t> &log_entries,
     ObjectStore::Transaction& t);
 
-  void filter_snapc(vector<snapid_t> &snaps);
+  void filter_snapc(std::vector<snapid_t> &snaps);
 
   virtual void kick_snap_trim() = 0;
   virtual void snap_trimmer_scrub_complete() = 0;
@@ -1503,7 +1503,7 @@ protected:
   bool op_has_sufficient_caps(OpRequestRef& op);
 
   // abstract bits
-  friend class FlushState;
+  friend struct FlushState;
 
   friend ostream& operator<<(ostream& out, const PG& pg);
 
index 06bb0f423f37f16efede853f3bf0804150775bae..630df7a57f493b236865755a6f6344c124c2e41d 100644 (file)
 #include "messages/MOSDPGRecoveryDelete.h"
 #include "messages/MOSDPGRecoveryDeleteReply.h"
 
+using std::list;
+using std::make_pair;
+using std::map;
+using std::ostream;
+using std::ostringstream;
+using std::pair;
+using std::set;
+using std::string;
+using std::stringstream;
+using std::vector;
+
+using ceph::bufferlist;
+using ceph::bufferptr;
+using ceph::ErasureCodeProfile;
+using ceph::ErasureCodeInterfaceRef;
+
 #define dout_context cct
 #define dout_subsys ceph_subsys_osd
 #define DOUT_PREFIX_ARGS this
index 223dce56786ba0d83376ec7cde160fecce8aefbe..de60db344fbbe55cc9f0c429d6a601dc4a273f88 100644 (file)
@@ -57,7 +57,7 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
    ObjectStore *store;
    const coll_t coll;
    ObjectStore::CollectionHandle &ch;
- public:       
+ public:
    /**
     * Provides interfaces for PGBackend callbacks
     *
@@ -111,10 +111,10 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
        const object_stat_sum_t &delta_stats) = 0;
 
      /**
-      * Called when a read from a set of replicas/primary fails
+      * Called when a read from a std::set of replicas/primary fails
       */
      virtual void on_failed_pull(
-       const set<pg_shard_t> &from,
+       const std::set<pg_shard_t> &from,
        const hobject_t &soid,
        const eversion_t &v
        ) = 0;
@@ -152,31 +152,31 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
        OpRequestRef op = OpRequestRef()
        ) = 0;
      virtual void queue_transactions(
-       vector<ObjectStore::Transaction>& tls,
+       std::vector<ObjectStore::Transaction>& tls,
        OpRequestRef op = OpRequestRef()
        ) = 0;
      virtual epoch_t get_interval_start_epoch() const = 0;
      virtual epoch_t get_last_peering_reset_epoch() const = 0;
 
-     virtual const set<pg_shard_t> &get_acting_recovery_backfill_shards() const = 0;
-     virtual const set<pg_shard_t> &get_acting_shards() const = 0;
-     virtual const set<pg_shard_t> &get_backfill_shards() const = 0;
+     virtual const std::set<pg_shard_t> &get_acting_recovery_backfill_shards() const = 0;
+     virtual const std::set<pg_shard_t> &get_acting_shards() const = 0;
+     virtual const std::set<pg_shard_t> &get_backfill_shards() const = 0;
 
      virtual std::ostream& gen_dbg_prefix(std::ostream& out) const = 0;
 
-     virtual const map<hobject_t, set<pg_shard_t>> &get_missing_loc_shards()
+     virtual const std::map<hobject_t, std::set<pg_shard_t>> &get_missing_loc_shards()
        const = 0;
 
      virtual const pg_missing_tracker_t &get_local_missing() const = 0;
      virtual void add_local_next_event(const pg_log_entry_t& e) = 0;
-     virtual const map<pg_shard_t, pg_missing_t> &get_shard_missing()
+     virtual const std::map<pg_shard_t, pg_missing_t> &get_shard_missing()
        const = 0;
      virtual const pg_missing_const_i * maybe_get_shard_missing(
        pg_shard_t peer) const {
        if (peer == primary_shard()) {
         return &get_local_missing();
        } else {
-        map<pg_shard_t, pg_missing_t>::const_iterator i =
+        std::map<pg_shard_t, pg_missing_t>::const_iterator i =
           get_shard_missing().find(peer);
         if (i == get_shard_missing().end()) {
           return nullptr;
@@ -191,12 +191,12 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
        return *m;
      }
 
-     virtual const map<pg_shard_t, pg_info_t> &get_shard_info() const = 0;
+     virtual const std::map<pg_shard_t, pg_info_t> &get_shard_info() const = 0;
      virtual const pg_info_t &get_shard_info(pg_shard_t peer) const {
        if (peer == primary_shard()) {
         return get_info();
        } else {
-        map<pg_shard_t, pg_info_t>::const_iterator i =
+        std::map<pg_shard_t, pg_info_t>::const_iterator i =
           get_shard_info().find(peer);
         ceph_assert(i != get_shard_info().end());
         return i->second;
@@ -212,7 +212,7 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
 
      virtual ObjectContextRef get_obc(
        const hobject_t &hoid,
-       const map<string, bufferlist> &attrs) = 0;
+       const std::map<std::string, ceph::buffer::list> &attrs) = 0;
 
      virtual bool try_lock_for_read(
        const hobject_t &hoid,
@@ -231,7 +231,7 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
      virtual bool pg_is_repair() const = 0;
 
      virtual void log_operation(
-       vector<pg_log_entry_t>&& logv,
+       std::vector<pg_log_entry_t>&& logv,
        const std::optional<pg_hit_set_history_t> &hset_history,
        const eversion_t &trim_to,
        const eversion_t &roll_forward_to,
@@ -242,7 +242,7 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
 
      virtual void pgb_set_object_snap_mapping(
        const hobject_t &soid,
-       const set<snapid_t> &snaps,
+       const std::set<snapid_t> &snaps,
        ObjectStore::Transaction *t) = 0;
 
      virtual void pgb_clear_object_snap_mapping(
@@ -327,14 +327,14 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
    /**
     * RecoveryHandle
     *
-    * We may want to recover multiple objects in the same set of
+    * We may want to recover multiple objects in the same std::set of
     * messages.  RecoveryHandle is an interface for the opaque
     * object used by the implementation to store the details of
     * the pending recovery operations.
     */
    struct RecoveryHandle {
      bool cache_dont_need;
-     map<pg_shard_t, vector<pair<hobject_t, eversion_t> > > deletes;
+     std::map<pg_shard_t, std::vector<std::pair<hobject_t, eversion_t> > > deletes;
 
      RecoveryHandle(): cache_dont_need(false) {}
      virtual ~RecoveryHandle() {}
@@ -352,7 +352,7 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
    void recover_delete_object(const hobject_t &oid, eversion_t v,
                              RecoveryHandle *h);
    void send_recovery_deletes(int prio,
-                             const map<pg_shard_t, vector<pair<hobject_t, eversion_t> > > &deletes);
+                             const std::map<pg_shard_t, std::vector<std::pair<hobject_t, eversion_t> > > &deletes);
 
    /**
     * recover_object
@@ -372,7 +372,7 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
     *
     * head may be NULL only if the head/snapdir is missing
     *
-    * @param missing [in] set of info, missing pairs for queried nodes
+    * @param missing [in] std::set of info, missing pairs for queried nodes
     * @param overlaps [in] mapping of object to file offset overlaps
     */
    virtual int recover_object(
@@ -417,23 +417,23 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
    virtual int get_ec_data_chunk_count() const { return 0; };
    virtual int get_ec_stripe_chunk_size() const { return 0; };
 
-   virtual void dump_recovery_info(Formatter *f) const = 0;
+   virtual void dump_recovery_info(ceph::Formatter *f) const = 0;
 
  private:
-   set<hobject_t> temp_contents;
+   std::set<hobject_t> temp_contents;
  public:
    // Track contents of temp collection, clear on reset
    void add_temp_obj(const hobject_t &oid) {
      temp_contents.insert(oid);
    }
-   void add_temp_objs(const set<hobject_t> &oids) {
+   void add_temp_objs(const std::set<hobject_t> &oids) {
      temp_contents.insert(oids.begin(), oids.end());
    }
    void clear_temp_obj(const hobject_t &oid) {
      temp_contents.erase(oid);
    }
-   void clear_temp_objs(const set<hobject_t> &oids) {
-     for (set<hobject_t>::const_iterator i = oids.begin();
+   void clear_temp_objs(const std::set<hobject_t> &oids) {
+     for (std::set<hobject_t>::const_iterator i = oids.begin();
          i != oids.end();
          ++i) {
        temp_contents.erase(*i);
@@ -451,7 +451,7 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
      const eversion_t &trim_to,           ///< [in] trim log to here
      const eversion_t &min_last_complete_ondisk, ///< [in] lower bound on
                                                  ///  committed version
-     vector<pg_log_entry_t>&& log_entries, ///< [in] log entries for t
+     std::vector<pg_log_entry_t>&& log_entries, ///< [in] log entries for t
      /// [in] hitset history (if updated with this transaction)
      std::optional<pg_hit_set_history_t> &hset_history,
      Context *on_all_commit,              ///< [in] called when all commit
@@ -493,7 +493,7 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
    /// Reapply old attributes
    void rollback_setattrs(
      const hobject_t &hoid,
-     map<string, std::optional<bufferlist> > &old_attrs,
+     std::map<std::string, std::optional<ceph::buffer::list> > &old_attrs,
      ObjectStore::Transaction *t);
 
    /// Truncate object to rollback append
@@ -524,7 +524,7 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
    /// Clone the extents back into place
    void rollback_extents(
      version_t gen,
-     const vector<pair<uint64_t, uint64_t> > &extents,
+     const std::vector<std::pair<uint64_t, uint64_t> > &extents,
      const hobject_t &hoid,
      ObjectStore::Transaction *t);
  public:
@@ -535,48 +535,48 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
      version_t gen,
      ObjectStore::Transaction *t);
 
-   /// List objects in collection
+   /// Std::list objects in collection
    int objects_list_partial(
      const hobject_t &begin,
      int min,
      int max,
-     vector<hobject_t> *ls,
+     std::vector<hobject_t> *ls,
      hobject_t *next);
 
    int objects_list_range(
      const hobject_t &start,
      const hobject_t &end,
-     vector<hobject_t> *ls,
-     vector<ghobject_t> *gen_obs=0);
+     std::vector<hobject_t> *ls,
+     std::vector<ghobject_t> *gen_obs=0);
 
    int objects_get_attr(
      const hobject_t &hoid,
-     const string &attr,
-     bufferlist *out);
+     const std::string &attr,
+     ceph::buffer::list *out);
 
    virtual int objects_get_attrs(
      const hobject_t &hoid,
-     map<string, bufferlist> *out);
+     std::map<std::string, ceph::buffer::list> *out);
 
    virtual int objects_read_sync(
      const hobject_t &hoid,
      uint64_t off,
      uint64_t len,
      uint32_t op_flags,
-     bufferlist *bl) = 0;
+     ceph::buffer::list *bl) = 0;
 
    virtual int objects_readv_sync(
      const hobject_t &hoid,
-     map<uint64_t, uint64_t>&& m,
+     std::map<uint64_t, uint64_t>&& m,
      uint32_t op_flags,
-     bufferlist *bl) {
+     ceph::buffer::list *bl) {
      return -EOPNOTSUPP;
    }
 
    virtual void objects_read_async(
      const hobject_t &hoid,
-     const list<pair<boost::tuple<uint64_t, uint64_t, uint32_t>,
-               pair<bufferlist*, Context*> > > &to_read,
+     const std::list<std::pair<boost::tuple<uint64_t, uint64_t, uint32_t>,
+               std::pair<ceph::buffer::list*, Context*> > > &to_read,
      Context *on_complete, bool fast_read = false) = 0;
 
    virtual bool auto_repair_supported() const = 0;
@@ -590,30 +590,30 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
      const ScrubMap::object &candidate,
      shard_info_wrapper& shard_error,
      inconsistent_obj_wrapper &result,
-     ostream &errorstream,
+     std::ostream &errorstream,
      bool has_snapset);
-   map<pg_shard_t, ScrubMap *>::const_iterator be_select_auth_object(
+   std::map<pg_shard_t, ScrubMap *>::const_iterator be_select_auth_object(
      const hobject_t &obj,
-     const map<pg_shard_t,ScrubMap*> &maps,
+     const std::map<pg_shard_t,ScrubMap*> &maps,
      object_info_t *auth_oi,
-     map<pg_shard_t, shard_info_wrapper> &shard_map,
+     std::map<pg_shard_t, shard_info_wrapper> &shard_map,
      bool &digest_match,
      spg_t pgid,
-     ostream &errorstream);
+     std::ostream &errorstream);
    void be_compare_scrubmaps(
-     const map<pg_shard_t,ScrubMap*> &maps,
-     const set<hobject_t> &master_set,
+     const std::map<pg_shard_t,ScrubMap*> &maps,
+     const std::set<hobject_t> &master_set,
      bool repair,
-     map<hobject_t, set<pg_shard_t>> &missing,
-     map<hobject_t, set<pg_shard_t>> &inconsistent,
-     map<hobject_t, list<pg_shard_t>> &authoritative,
-     map<hobject_t, pair<std::optional<uint32_t>,
+     std::map<hobject_t, std::set<pg_shard_t>> &missing,
+     std::map<hobject_t, std::set<pg_shard_t>> &inconsistent,
+     std::map<hobject_t, std::list<pg_shard_t>> &authoritative,
+     std::map<hobject_t, std::pair<std::optional<uint32_t>,
                          std::optional<uint32_t>>> &missing_digest,
      int &shallow_errors, int &deep_errors,
      Scrub::Store *store,
      const spg_t& pgid,
-     const vector<int> &acting,
-     ostream &errorstream);
+     const std::vector<int> &acting,
+     std::ostream &errorstream);
    virtual uint64_t be_get_ondisk_size(
      uint64_t logical_size) = 0;
    virtual int be_deep_scrub(
@@ -622,14 +622,14 @@ typedef std::shared_ptr<const OSDMap> OSDMapRef;
      ScrubMapBuilder &pos,
      ScrubMap::object &o) = 0;
    void be_omap_checks(
-     const map<pg_shard_t,ScrubMap*> &maps,
-     const set<hobject_t> &master_set,
+     const std::map<pg_shard_t,ScrubMap*> &maps,
+     const std::set<hobject_t> &master_set,
      omap_stat_t& omap_stats,
-     ostream &warnstream) const;
+     std::ostream &warnstream) const;
 
    static PGBackend *build_pg_backend(
      const pg_pool_t &pool,
-     const map<string,string>& profile,
+     const std::map<std::string,std::string>& profile,
      Listener *l,
      coll_t coll,
      ObjectStore::CollectionHandle &ch,
index 9e81f9f5f3b13d8abf867dcf9b296d2fe5aaa3ad..19f2d21440ac168c681d9bbb1381c0c4de3691d6 100644 (file)
 #include "include/unordered_map.h"
 #include "common/ceph_context.h"
 
+using std::make_pair;
+using std::map;
+using std::ostream;
+using std::set;
+using std::string;
+
+using ceph::bufferlist;
+using ceph::decode;
+using ceph::encode;
+
 #define dout_context cct
 #define dout_subsys ceph_subsys_osd
 #undef dout_prefix
@@ -140,18 +150,14 @@ void PGLog::IndexedLog::trim(
 ostream& PGLog::IndexedLog::print(ostream& out) const
 {
   out << *this << std::endl;
-  for (list<pg_log_entry_t>::const_iterator p = log.begin();
-       p != log.end();
-       ++p) {
+  for (auto p = log.begin(); p != log.end(); ++p) {
     out << *p << " " <<
       (logged_object(p->soid) ? "indexed" : "NOT INDEXED") <<
       std::endl;
     ceph_assert(!p->reqid_is_indexed() || logged_req(p->reqid));
   }
 
-  for (list<pg_log_dup_t>::const_iterator p = dups.begin();
-       p != dups.end();
-       ++p) {
+  for (auto p = dups.begin(); p != dups.end(); ++p) {
     out << *p << std::endl;
   }
 
@@ -232,15 +238,14 @@ void PGLog::proc_replica_log(
     we will send the peer enough log to arrive at the same state.
   */
 
-  for (map<hobject_t, pg_missing_item>::const_iterator i = omissing.get_items().begin();
+  for (auto i = omissing.get_items().begin();
        i != omissing.get_items().end();
        ++i) {
     dout(20) << " before missing " << i->first << " need " << i->second.need
             << " have " << i->second.have << dendl;
   }
 
-  list<pg_log_entry_t>::const_reverse_iterator first_non_divergent =
-    log.log.rbegin();
+  auto first_non_divergent = log.log.rbegin();
   while (1) {
     if (first_non_divergent == log.log.rend())
       break;
@@ -295,10 +300,7 @@ void PGLog::proc_replica_log(
     eversion_t first_missing =
       omissing.get_items().at(omissing.get_rmissing().begin()->second).need;
     oinfo.last_complete = eversion_t();
-    list<pg_log_entry_t>::const_iterator i = olog.log.begin();
-    for (;
-        i != olog.log.end();
-        ++i) {
+    for (auto i = olog.log.begin(); i != olog.log.end(); ++i) {
       if (i->version < first_missing)
        oinfo.last_complete = i->version;
       else
@@ -368,7 +370,7 @@ void PGLog::merge_log(pg_info_t &oinfo, pg_log_t &olog, pg_shard_t fromosd,
   // The logs must overlap.
   ceph_assert(log.head >= olog.tail && olog.head >= log.tail);
 
-  for (map<hobject_t, pg_missing_item>::const_iterator i = missing.get_items().begin();
+  for (auto i = missing.get_items().begin();
        i != missing.get_items().end();
        ++i) {
     dout(20) << "pg_missing_t sobject: " << i->first << dendl;
@@ -383,12 +385,10 @@ void PGLog::merge_log(pg_info_t &oinfo, pg_log_t &olog, pg_shard_t fromosd,
   eversion_t orig_tail = log.tail;
   if (olog.tail < log.tail) {
     dout(10) << "merge_log extending tail to " << olog.tail << dendl;
-    list<pg_log_entry_t>::iterator from = olog.log.begin();
-    list<pg_log_entry_t>::iterator to;
+    auto from = olog.log.begin();
+    auto to = from;
     eversion_t last;
-    for (to = from;
-        to != olog.log.end();
-        ++to) {
+    for (; to != olog.log.end(); ++to) {
       if (to->version > log.tail)
        break;
       log.index(*to);
@@ -425,8 +425,8 @@ void PGLog::merge_log(pg_info_t &oinfo, pg_log_t &olog, pg_shard_t fromosd,
     dout(10) << "merge_log extending head to " << olog.head << dendl;
 
     // find start point in olog
-    list<pg_log_entry_t>::iterator to = olog.log.end();
-    list<pg_log_entry_t>::iterator from = olog.log.end();
+    auto to = olog.log.end();
+    auto from = olog.log.end();
     eversion_t lower_bound = std::max(olog.tail, orig_tail);
     while (1) {
       if (from == olog.log.begin())
@@ -593,22 +593,18 @@ void PGLog::check() {
   if (log.log.size() != log_keys_debug.size()) {
     derr << "log.log.size() != log_keys_debug.size()" << dendl;
     derr << "actual log:" << dendl;
-    for (list<pg_log_entry_t>::iterator i = log.log.begin();
-        i != log.log.end();
-        ++i) {
+    for (auto i = log.log.begin(); i != log.log.end(); ++i) {
       derr << "    " << *i << dendl;
     }
     derr << "log_keys_debug:" << dendl;
-    for (set<string>::const_iterator i = log_keys_debug.begin();
+    for (auto i = log_keys_debug.begin();
         i != log_keys_debug.end();
         ++i) {
       derr << "    " << *i << dendl;
     }
   }
   ceph_assert(log.log.size() == log_keys_debug.size());
-  for (list<pg_log_entry_t>::iterator i = log.log.begin();
-       i != log.log.end();
-       ++i) {
+  for (auto i = log.log.begin(); i != log.log.end(); ++i) {
     ceph_assert(log_keys_debug.count(i->get_key_name()));
   }
 }
@@ -731,7 +727,7 @@ void PGLog::_write_log_and_missing_wo_missing(
     clear_after(log_keys_debug, dirty_from.get_key_name());
   }
 
-  for (list<pg_log_entry_t>::iterator p = log.log.begin();
+  for (auto p = log.log.begin();
        p != log.log.end() && p->version <= dirty_to;
        ++p) {
     bufferlist bl(sizeof(*p) * 2);
@@ -739,7 +735,7 @@ void PGLog::_write_log_and_missing_wo_missing(
     (*km)[p->get_key_name()].claim(bl);
   }
 
-  for (list<pg_log_entry_t>::reverse_iterator p = log.log.rbegin();
+  for (auto p = log.log.rbegin();
        p != log.log.rend() &&
         (p->version >= dirty_from || p->version >= writeout_from) &&
         p->version >= dirty_to;
@@ -750,7 +746,7 @@ void PGLog::_write_log_and_missing_wo_missing(
   }
 
   if (log_keys_debug) {
-    for (map<string, bufferlist>::iterator i = (*km).begin();
+    for (auto i = (*km).begin();
         i != (*km).end();
         ++i) {
       if (i->first[0] == '_')
@@ -786,7 +782,7 @@ void PGLog::_write_log_and_missing_wo_missing(
     (*km)[entry.get_key_name()].claim(bl);
   }
 
-  for (list<pg_log_dup_t>::reverse_iterator p = log.dups.rbegin();
+  for (auto p = log.dups.rbegin();
        p != log.dups.rend() &&
         (p->version >= dirty_from_dups || p->version >= write_from_dups) &&
         p->version >= dirty_to_dups;
@@ -860,7 +856,7 @@ void PGLog::_write_log_and_missing(
     clear_after(log_keys_debug, dirty_from.get_key_name());
   }
 
-  for (list<pg_log_entry_t>::iterator p = log.log.begin();
+  for (auto p = log.log.begin();
        p != log.log.end() && p->version <= dirty_to;
        ++p) {
     bufferlist bl(sizeof(*p) * 2);
@@ -868,7 +864,7 @@ void PGLog::_write_log_and_missing(
     (*km)[p->get_key_name()].claim(bl);
   }
 
-  for (list<pg_log_entry_t>::reverse_iterator p = log.log.rbegin();
+  for (auto p = log.log.rbegin();
        p != log.log.rend() &&
         (p->version >= dirty_from || p->version >= writeout_from) &&
         p->version >= dirty_to;
@@ -879,7 +875,7 @@ void PGLog::_write_log_and_missing(
   }
 
   if (log_keys_debug) {
-    for (map<string, bufferlist>::iterator i = (*km).begin();
+    for (auto i = (*km).begin();
         i != (*km).end();
         ++i) {
       if (i->first[0] == '_')
@@ -915,7 +911,7 @@ void PGLog::_write_log_and_missing(
     (*km)[entry.get_key_name()].claim(bl);
   }
 
-  for (list<pg_log_dup_t>::reverse_iterator p = log.dups.rbegin();
+  for (auto p = log.dups.rbegin();
        p != log.dups.rend() &&
         (p->version >= dirty_from_dups || p->version >= write_from_dups) &&
         p->version >= dirty_to_dups;
@@ -980,7 +976,7 @@ void PGLog::rebuild_missing_set_with_deletes(
   // versions on disk, just as if we were reading the log + metadata
   // off disk originally
   set<hobject_t> did;
-  for (list<pg_log_entry_t>::reverse_iterator i = log.log.rbegin();
+  for (auto i = log.log.rbegin();
        i != log.log.rend();
        ++i) {
     if (i->version <= info.last_complete)
index 401e114ac59d7e931135d668998942fc954fadd1..9d3a3be6ac8385147055eb1593e0af5cadefbd4a 100644 (file)
@@ -64,7 +64,7 @@ struct PGLog : DoutPrefixProvider {
       version_t v) = 0;
     virtual ~LogEntryHandler() {}
   };
-  using LogEntryHandlerRef = unique_ptr<LogEntryHandler>;
+  using LogEntryHandlerRef = std::unique_ptr<LogEntryHandler>;
 
 public:
   /**
@@ -78,7 +78,7 @@ public:
     mutable ceph::unordered_map<osd_reqid_t,pg_log_dup_t*> dup_index;
 
     // recovery pointers
-    list<pg_log_entry_t>::iterator complete_to; // not inclusive of referenced item
+    std::list<pg_log_entry_t>::iterator complete_to; // not inclusive of referenced item
     version_t last_requested = 0;               // last object requested by primary
 
     //
@@ -267,16 +267,15 @@ public:
       eversion_t *version,
       version_t *user_version,
       int *return_code,
-      vector<pg_log_op_return_item_t> *op_returns) const
+      std::vector<pg_log_op_return_item_t> *op_returns) const
     {
       ceph_assert(version);
       ceph_assert(user_version);
       ceph_assert(return_code);
-      ceph::unordered_map<osd_reqid_t,pg_log_entry_t*>::const_iterator p;
       if (!(indexed_data & PGLOG_INDEXED_CALLER_OPS)) {
         index_caller_ops();
       }
-      p = caller_ops.find(r);
+      auto p = caller_ops.find(r);
       if (p != caller_ops.end()) {
        *version = p->second->version;
        *user_version = p->second->user_version;
@@ -338,9 +337,9 @@ public:
       return false;
     }
 
-    /// get a (bounded) list of recent reqids for the given object
+    /// get a (bounded) std::list of recent reqids for the given object
     void get_object_reqids(const hobject_t& oid, unsigned max,
-                          mempool::osd_pglog::vector<pair<osd_reqid_t, version_t> > *pls,
+                          mempool::osd_pglog::vector<std::pair<osd_reqid_t, version_t> > *pls,
                           mempool::osd_pglog::map<uint32_t, int> *return_codes) const {
        // make sure object is present at least once before we do an
        // O(n) search.
@@ -350,16 +349,14 @@ public:
       if (objects.count(oid) == 0)
        return;
 
-      for (list<pg_log_entry_t>::const_reverse_iterator i = log.rbegin();
-           i != log.rend();
-           ++i) {
+      for (auto i = log.rbegin(); i != log.rend(); ++i) {
        if (i->soid == oid) {
          if (i->reqid_is_indexed()) {
            if (i->op == pg_log_entry_t::ERROR) {
              // propagate op errors to the cache tier's PG log
              return_codes->emplace(pls->size(), i->return_code);
            }
-           pls->push_back(make_pair(i->reqid, i->user_version));
+           pls->push_back(std::make_pair(i->reqid, i->user_version));
          }
 
          pls->insert(pls->end(), i->extra_reqids.begin(), i->extra_reqids.end());
@@ -398,9 +395,7 @@ public:
        PGLOG_INDEXED_EXTRA_CALLER_OPS;
 
       if (to_index & any_log_entry_index) {
-       for (list<pg_log_entry_t>::const_iterator i = log.begin();
-            i != log.end();
-            ++i) {
+       for (auto i = log.begin(); i != log.end(); ++i) {
          if (to_index & PGLOG_INDEXED_OBJECTS) {
            if (i->object_is_indexed()) {
              objects[i->soid] = const_cast<pg_log_entry_t*>(&(*i));
@@ -418,7 +413,7 @@ public:
                 j != i->extra_reqids.end();
                 ++j) {
              extra_caller_ops.insert(
-               make_pair(j->first, const_cast<pg_log_entry_t*>(&(*i))));
+               std::make_pair(j->first, const_cast<pg_log_entry_t*>(&(*i))));
            }
          }
        }
@@ -459,7 +454,7 @@ public:
         for (auto j = e.extra_reqids.begin();
             j != e.extra_reqids.end();
             ++j) {
-         extra_caller_ops.insert(make_pair(j->first, &e));
+         extra_caller_ops.insert(std::make_pair(j->first, &e));
         }
       }
     }
@@ -491,8 +486,7 @@ public:
         for (auto j = e.extra_reqids.begin();
              j != e.extra_reqids.end();
              ++j) {
-          for (ceph::unordered_multimap<osd_reqid_t,pg_log_entry_t*>::iterator k =
-                extra_caller_ops.find(j->first);
+          for (auto k = extra_caller_ops.find(j->first);
                k != extra_caller_ops.end() && k->first == j->first;
                ++k) {
             if (k->second == &e) {
@@ -553,7 +547,7 @@ public:
         for (auto j = e.extra_reqids.begin();
             j != e.extra_reqids.end();
             ++j) {
-         extra_caller_ops.insert(make_pair(j->first, &(log.back())));
+         extra_caller_ops.insert(std::make_pair(j->first, &(log.back())));
         }
       }
 
@@ -565,11 +559,11 @@ public:
     void trim(
       CephContext* cct,
       eversion_t s,
-      set<eversion_t> *trimmed,
-      set<string>* trimmed_dups,
+      std::set<eversion_t> *trimmed,
+      std::set<std::string>* trimmed_dups,
       eversion_t *write_from_dups);
 
-    ostream& print(ostream& out) const;
+    std::ostream& print(std::ostream& out) const;
   }; // IndexedLog
 
 
@@ -582,11 +576,11 @@ protected:
   eversion_t dirty_to;         ///< must clear/writeout all keys <= dirty_to
   eversion_t dirty_from;       ///< must clear/writeout all keys >= dirty_from
   eversion_t writeout_from;    ///< must writout keys >= writeout_from
-  set<eversion_t> trimmed;     ///< must clear keys in trimmed
+  std::set<eversion_t> trimmed;     ///< must clear keys in trimmed
   eversion_t dirty_to_dups;    ///< must clear/writeout all dups <= dirty_to_dups
   eversion_t dirty_from_dups;  ///< must clear/writeout all dups >= dirty_from_dups
   eversion_t write_from_dups;  ///< must write keys >= write_from_dups
-  set<string> trimmed_dups;    ///< must clear keys in trimmed_dups
+  std::set<std::string> trimmed_dups;    ///< must clear keys in trimmed_dups
   CephContext *cct;
   bool pg_log_debug;
   /// Log is clean on [dirty_to, dirty_from)
@@ -647,18 +641,18 @@ public:
 protected:
 
   /// DEBUG
-  set<string> log_keys_debug;
-  static void clear_after(set<string> *log_keys_debug, const string &lb) {
+  std::set<std::string> log_keys_debug;
+  static void clear_after(std::set<std::string> *log_keys_debug, const std::string &lb) {
     if (!log_keys_debug)
       return;
-    for (set<string>::iterator i = log_keys_debug->lower_bound(lb);
+    for (auto i = log_keys_debug->lower_bound(lb);
         i != log_keys_debug->end();
         log_keys_debug->erase(i++));
   }
-  static void clear_up_to(set<string> *log_keys_debug, const string &ub) {
+  static void clear_up_to(std::set<std::string> *log_keys_debug, const std::string &ub) {
     if (!log_keys_debug)
       return;
-    for (set<string>::iterator i = log_keys_debug->begin();
+    for (auto i = log_keys_debug->begin();
         i != log_keys_debug->end() && *i < ub;
         log_keys_debug->erase(i++));
   }
@@ -697,7 +691,7 @@ public:
 
   void clear();
 
-  //////////////////// get or set missing ////////////////////
+  //////////////////// get or std::set missing ////////////////////
 
   const pg_missing_tracker_t& get_missing() const { return missing; }
 
@@ -709,7 +703,7 @@ public:
     missing.add_next_event(e);
   }
 
-  //////////////////// get or set log ////////////////////
+  //////////////////// get or std::set log ////////////////////
 
   const IndexedLog &get_log() const { return log; }
 
@@ -769,7 +763,7 @@ public:
     log.skip_can_rollback_to_to_head();
   }
 
-  //////////////////// get or set log & missing ////////////////////
+  //////////////////// get or std::set log & missing ////////////////////
 
   void reset_backfill_claim_log(const pg_log_t &o, LogEntryHandler *h) {
     log.trim_rollback_info_to(log.head, h);
@@ -795,12 +789,12 @@ public:
   }
 
   void merge_from(
-    const vector<PGLog*>& sources,
+    const std::vector<PGLog*>& sources,
     eversion_t last_update) {
     unindex();
     missing.clear();
 
-    vector<pg_log_t*> slogs;
+    std::vector<pg_log_t*> slogs;
     for (auto s : sources) {
       slogs.push_back(&s->log);
     }
@@ -878,7 +872,7 @@ public:
 protected:
   static void split_by_object(
     mempool::osd_pglog::list<pg_log_entry_t> &entries,
-    map<hobject_t, mempool::osd_pglog::list<pg_log_entry_t>> *out_entries) {
+    std::map<hobject_t, mempool::osd_pglog::list<pg_log_entry_t>> *out_entries) {
     while (!entries.empty()) {
       auto &out_list = (*out_entries)[entries.front().soid];
       out_list.splice(out_list.end(), entries, entries.begin());
@@ -931,7 +925,7 @@ protected:
     mempool::osd_pglog::list<pg_log_entry_t> entries;
     eversion_t last;
     bool seen_non_error = false;
-    for (list<pg_log_entry_t>::const_iterator i = orig_entries.begin();
+    for (auto i = orig_entries.begin();
         i != orig_entries.end();
         ++i) {
       // all entries are on hoid
@@ -945,7 +939,7 @@ protected:
       }
       
       // No need to check the first entry since it prior_version is unavailable
-      // in the list
+      // in the std::list
       // No need to check if the prior_version is the minimal version
       // No need to check the first non-error entry since the leading error
       // entries are not its prior version
@@ -983,8 +977,7 @@ protected:
                       << " last_divergent_update: " << last_divergent_update
                       << dendl;
 
-    ceph::unordered_map<hobject_t, pg_log_entry_t*>::const_iterator objiter =
-      log.objects.find(hoid);
+    auto objiter = log.objects.find(hoid);
     if (objiter != log.objects.end() &&
        objiter->second->version >= first_divergent_update) {
       /// Case 1)
@@ -1077,9 +1070,7 @@ protected:
                        << " olog_can_rollback_to: "
                        << olog_can_rollback_to << dendl;
     /// Distinguish between 4) and 5)
-    for (list<pg_log_entry_t>::const_reverse_iterator i = entries.rbegin();
-        i != entries.rend();
-        ++i) {
+    for (auto i = entries.rbegin(); i != entries.rend(); ++i) {
       if (!i->can_rollback() || i->version <= olog_can_rollback_to) {
        ldpp_dout(dpp, 10) << __func__ << ": hoid " << hoid << " cannot rollback "
                           << *i << dendl;
@@ -1090,9 +1081,7 @@ protected:
 
     if (can_rollback) {
       /// Case 4)
-      for (list<pg_log_entry_t>::const_reverse_iterator i = entries.rbegin();
-          i != entries.rend();
-          ++i) {
+      for (auto i = entries.rbegin(); i != entries.rend(); ++i) {
        ceph_assert(i->can_rollback() && i->version > olog_can_rollback_to);
        ldpp_dout(dpp, 10) << __func__ << ": hoid " << hoid
                           << " rolling back " << *i << dendl;
@@ -1134,11 +1123,9 @@ protected:
     LogEntryHandler *rollbacker,         ///< [in] optional rollbacker object
     const DoutPrefixProvider *dpp        ///< [in] logging provider
     ) {
-    map<hobject_t, mempool::osd_pglog::list<pg_log_entry_t> > split;
+    std::map<hobject_t, mempool::osd_pglog::list<pg_log_entry_t> > split;
     split_by_object(entries, &split);
-    for (map<hobject_t, mempool::osd_pglog::list<pg_log_entry_t>>::iterator i = split.begin();
-        i != split.end();
-        ++i) {
+    for (auto i = split.begin(); i != split.end(); ++i) {
       _merge_object_divergent_entries(
        log,
        i->first,
@@ -1202,9 +1189,7 @@ public:
     if (log && !entries.empty()) {
       ceph_assert(log->head < entries.begin()->version);
     }
-    for (list<pg_log_entry_t>::const_iterator p = entries.begin();
-        p != entries.end();
-        ++p) {
+    for (auto p = entries.begin(); p != entries.end(); ++p) {
       invalidate_stats = invalidate_stats || !p->is_error();
       if (log) {
        ldpp_dout(dpp, 20) << "update missing, append " << *p << dendl;
@@ -1253,7 +1238,7 @@ public:
        // iterator may still point at log.end(). Reset it to point
        // before these new lost_delete entries.  This only occurs
        // when lost+delete entries are initially added, which is
-       // always in a list of solely lost_delete entries, so it is
+       // always in a std::list of solely lost_delete entries, so it is
        // sufficient to check whether the first entry is a
        // lost_delete
        reset_complete_to(nullptr);
@@ -1264,22 +1249,22 @@ public:
 
   void write_log_and_missing(
     ObjectStore::Transaction& t,
-    map<string,bufferlist> *km,
+    std::map<std::string,ceph::buffer::list> *km,
     const coll_t& coll,
     const ghobject_t &log_oid,
     bool require_rollback);
 
   static void write_log_and_missing_wo_missing(
     ObjectStore::Transaction& t,
-    map<string,bufferlist>* km,
+    std::map<std::string,ceph::buffer::list>* km,
     pg_log_t &log,
     const coll_t& coll,
-    const ghobject_t &log_oid, map<eversion_t, hobject_t> &divergent_priors,
+    const ghobject_t &log_oid, std::map<eversion_t, hobject_t> &divergent_priors,
     bool require_rollback);
 
   static void write_log_and_missing(
     ObjectStore::Transaction& t,
-    map<string,bufferlist>* km,
+    std::map<std::string,ceph::buffer::list>* km,
     pg_log_t &log,
     const coll_t& coll,
     const ghobject_t &log_oid,
@@ -1289,10 +1274,10 @@ public:
 
   static void _write_log_and_missing_wo_missing(
     ObjectStore::Transaction& t,
-    map<string,bufferlist>* km,
+    std::map<std::string,ceph::buffer::list>* km,
     pg_log_t &log,
     const coll_t& coll, const ghobject_t &log_oid,
-    map<eversion_t, hobject_t> &divergent_priors,
+    std::map<eversion_t, hobject_t> &divergent_priors,
     eversion_t dirty_to,
     eversion_t dirty_from,
     eversion_t writeout_from,
@@ -1302,19 +1287,19 @@ public:
     eversion_t dirty_to_dups,
     eversion_t dirty_from_dups,
     eversion_t write_from_dups,
-    set<string> *log_keys_debug
+    std::set<std::string> *log_keys_debug
     );
 
   static void _write_log_and_missing(
     ObjectStore::Transaction& t,
-    map<string,bufferlist>* km,
+    std::map<std::string,ceph::buffer::list>* km,
     pg_log_t &log,
     const coll_t& coll, const ghobject_t &log_oid,
     eversion_t dirty_to,
     eversion_t dirty_from,
     eversion_t writeout_from,
-    set<eversion_t> &&trimmed,
-    set<string> &&trimmed_dups,
+    std::set<eversion_t> &&trimmed,
+    std::set<std::string> &&trimmed_dups,
     const pg_missing_tracker_t &missing,
     bool touch_log,
     bool require_rollback,
@@ -1323,7 +1308,7 @@ public:
     eversion_t dirty_from_dups,
     eversion_t write_from_dups,
     bool *may_include_deletes_in_missing_dirty,
-    set<string> *log_keys_debug
+    std::set<std::string> *log_keys_debug
     );
 
   void read_log_and_missing(
@@ -1331,7 +1316,7 @@ public:
     ObjectStore::CollectionHandle& ch,
     ghobject_t pgmeta_oid,
     const pg_info_t &info,
-    ostringstream &oss,
+    std::ostringstream &oss,
     bool tolerate_divergent_missing_log,
     bool debug_verify_stored_missing = false
     ) {
@@ -1353,11 +1338,11 @@ public:
     const pg_info_t &info,
     IndexedLog &log,
     missing_type &missing,
-    ostringstream &oss,
+    std::ostringstream &oss,
     bool tolerate_divergent_missing_log,
     bool *clear_divergent_priors = nullptr,
     const DoutPrefixProvider *dpp = nullptr,
-    set<string> *log_keys_debug = nullptr,
+    std::set<std::string> *log_keys_debug = nullptr,
     bool debug_verify_stored_missing = false
     ) {
     ldpp_dout(dpp, 20) << "read_log_and_missing coll " << ch->cid
@@ -1374,17 +1359,18 @@ public:
     eversion_t on_disk_rollback_info_trimmed_to = eversion_t();
     ObjectMap::ObjectMapIterator p = store->get_omap_iterator(ch,
                                                              pgmeta_oid);
-    map<eversion_t, hobject_t> divergent_priors;
+    std::map<eversion_t, hobject_t> divergent_priors;
     bool must_rebuild = false;
     missing.may_include_deletes = false;
-    list<pg_log_entry_t> entries;
-    list<pg_log_dup_t> dups;
+    std::list<pg_log_entry_t> entries;
+    std::list<pg_log_dup_t> dups;
     if (p) {
+      using ceph::decode;
       for (p->seek_to_first(); p->valid() ; p->next()) {
        // non-log pgmeta_oid keys are prefixed with _; skip those
        if (p->key()[0] == '_')
          continue;
-       bufferlist bl = p->value();//Copy bufferlist before creating iterator
+       auto bl = p->value();//Copy ceph::buffer::list before creating iterator
        auto bp = bl.cbegin();
        if (p->key() == "divergent_priors") {
          decode(divergent_priors, bp);
@@ -1398,7 +1384,7 @@ public:
          decode(on_disk_rollback_info_trimmed_to, bp);
        } else if (p->key() == "may_include_deletes_in_missing") {
          missing.may_include_deletes = true;
-       } else if (p->key().substr(0, 7) == string("missing")) {
+       } else if (p->key().substr(0, 7) == std::string("missing")) {
          hobject_t oid;
          pg_missing_item item;
          decode(oid, bp);
@@ -1408,7 +1394,7 @@ public:
            ceph_assert(missing.may_include_deletes);
          }
          missing.add(oid, std::move(item));
-       } else if (p->key().substr(0, 4) == string("dup_")) {
+       } else if (p->key().substr(0, 4) == std::string("dup_")) {
          pg_log_dup_t dup;
          decode(dup, bp);
          if (!dups.empty()) {
@@ -1446,12 +1432,10 @@ public:
          << info.last_complete
          << "," << info.last_update << "]" << dendl;
 
-       set<hobject_t> did;
-       set<hobject_t> checked;
-       set<hobject_t> skipped;
-       for (list<pg_log_entry_t>::reverse_iterator i = log.log.rbegin();
-            i != log.log.rend();
-            ++i) {
+       std::set<hobject_t> did;
+       std::set<hobject_t> checked;
+       std::set<hobject_t> skipped;
+       for (auto i = log.log.rbegin(); i != log.log.rend(); ++i) {
          if (i->soid > info.last_backfill)
            continue;
          if (i->is_error())
@@ -1462,7 +1446,7 @@ public:
          if (!missing.may_include_deletes && i->is_delete())
            continue;
 
-         bufferlist bv;
+         ceph::buffer::list bv;
          int r = store->getattr(
            ch,
            ghobject_t(i->soid, ghobject_t::NO_GEN, info.pgid.shard),
@@ -1511,14 +1495,14 @@ public:
            if (checked.count(i.first))
              continue;
            if (i.first > info.last_backfill) {
-             ldpp_dout(dpp, -1) << __func__ << ": invalid missing set entry "
+             ldpp_dout(dpp, -1) << __func__ << ": invalid missing std::set entry "
                                << "found before last_backfill: "
                                << i.first << " " << i.second
                                << " last_backfill = " << info.last_backfill
                                << dendl;
-             ceph_abort_msg("invalid missing set entry found");
+             ceph_abort_msg("invalid missing std::set entry found");
            }
-           bufferlist bv;
+           ceph::buffer::list bv;
            int r = store->getattr(
              ch,
              ghobject_t(i.first, ghobject_t::NO_GEN, info.pgid.shard),
@@ -1533,8 +1517,7 @@ public:
          }
        } else {
          ceph_assert(must_rebuild);
-         for (map<eversion_t, hobject_t>::reverse_iterator i =
-                divergent_priors.rbegin();
+         for (auto i = divergent_priors.rbegin();
               i != divergent_priors.rend();
               ++i) {
            if (i->first <= info.last_complete) break;
@@ -1542,7 +1525,7 @@ public:
              continue;
            if (did.count(i->second)) continue;
            did.insert(i->second);
-           bufferlist bv;
+           ceph::buffer::list bv;
            int r = store->getattr(
              ch,
              ghobject_t(i->second, ghobject_t::NO_GEN, info.pgid.shard),
@@ -1561,10 +1544,10 @@ public:
                 * version would not have been recovered, and a newer version
                 * would show up in the log above.
                 */
-               /**
+             /**
                 * Unfortunately the assessment above is incorrect because of
                 * http://tracker.ceph.com/issues/17916 (we were incorrectly
-                * not removing the divergent_priors set from disk state!),
+                * not removing the divergent_priors std::set from disk state!),
                 * so let's check that.
                 */
              if (oi.version > i->first && tolerate_divergent_missing_log) {
@@ -1630,8 +1613,8 @@ public:
     void process_entry(crimson::os::FuturizedStore::OmapIteratorRef &p) {
       if (p->key()[0] == '_')
        return;
-      //Copy bufferlist before creating iterator
-      ceph::bufferlist bl = p->value();
+      //Copy ceph::buffer::list before creating iterator
+      auto bl = p->value();
       auto bp = bl.cbegin();
       if (p->key() == "divergent_priors") {
        decode(divergent_priors, bp);
@@ -1644,7 +1627,7 @@ public:
        decode(on_disk_rollback_info_trimmed_to, bp);
       } else if (p->key() == "may_include_deletes_in_missing") {
        missing.may_include_deletes = true;
-      } else if (p->key().substr(0, 7) == string("missing")) {
+      } else if (p->key().substr(0, 7) == std::string("missing")) {
        hobject_t oid;
        pg_missing_item item;
        decode(oid, bp);
@@ -1653,7 +1636,7 @@ public:
          ceph_assert(missing.may_include_deletes);
        }
        missing.add(oid, std::move(item));
-      } else if (p->key().substr(0, 4) == string("dup_")) {
+      } else if (p->key().substr(0, 4) == std::string("dup_")) {
        pg_log_dup_t dup;
        decode(dup, bp);
        if (!dups.empty()) {
index fb5e1976ed70d5f802886f3c1d8f3b9ee31265cb..5dbe78eb7c0234eb0052aaa123bb2f5b7ffeb0c3 100644 (file)
@@ -4,6 +4,8 @@
 #include "PGStateUtils.h"
 #include "common/Clock.h"
 
+using ceph::Formatter;
+
 /*------NamedState----*/
 NamedState::NamedState(PGStateHistory *pgsh, const char *state_name)
   : pgsh(pgsh), state_name(state_name), enter_time(ceph_clock_now()) {
index 46458d60251a7dcc7ab9cebf9f9df3edd00c6e3d..50f6f624106dbe2bc4db5391d58f42cc96033844 100644 (file)
@@ -72,7 +72,7 @@ public:
     pi = nullptr;
   }
 
-  void dump(Formatter* f) const;
+  void dump(ceph::Formatter* f) const;
 
   const char *get_current_state() const {
     if (pi == nullptr) return "unknown";
index 51657c706ca4b2c73d85e85d125922b601a536f5..3b5b9e72cfa076bd29b7a8624ce8ddbbdbaebc2b 100644 (file)
@@ -37,7 +37,7 @@
  */
 class PGTransaction {
 public:
-  map<hobject_t, ObjectContextRef> obc_map;
+  std::map<hobject_t, ObjectContextRef> obc_map;
 
   class ObjectOperation {
   public:
@@ -127,17 +127,17 @@ public:
      * remember the lowest truncate and the final object size
      * (the last truncate).  We also adjust the buffers map
      * to account for truncates overriding previous writes */
-    std::optional<pair<uint64_t, uint64_t> > truncate = std::nullopt;
+    std::optional<std::pair<uint64_t, uint64_t> > truncate = std::nullopt;
 
-    std::map<string, std::optional<bufferlist> > attr_updates;
+    std::map<std::string, std::optional<ceph::buffer::list> > attr_updates;
 
     enum class OmapUpdateType {Remove, Insert, RemoveRange};
-    std::vector<std::pair<OmapUpdateType, bufferlist> > omap_updates;
+    std::vector<std::pair<OmapUpdateType, ceph::buffer::list> > omap_updates;
 
-    std::optional<bufferlist> omap_header;
+    std::optional<ceph::buffer::list> omap_header;
 
     /// (old, new) -- only valid with no truncate or buffer updates
-    std::optional<pair<set<snapid_t>, set<snapid_t> > > updated_snaps;
+    std::optional<std::pair<std::set<snapid_t>, std::set<snapid_t>>> updated_snaps;
 
     struct alloc_hint_t {
       uint64_t expected_object_size;
@@ -148,7 +148,7 @@ public:
 
     struct BufferUpdate {
       struct Write {
-       bufferlist buffer;
+       ceph::buffer::list buffer;
        uint32_t fadvise_flags;
       };
       struct Zero {
@@ -174,7 +174,7 @@ public:
        return match(
          bu,
          [&](const BufferUpdate::Write &w) -> BufferUpdateType {
-           bufferlist bl;
+           ceph::buffer::list bl;
            bl.substr_of(w.buffer, offset, len);
            return BufferUpdate::Write{bl, w.fadvise_flags};
          },
@@ -224,7 +224,7 @@ public:
          [&](const BufferUpdate::Write &w) -> BufferUpdateType {
            auto r = boost::get<BufferUpdate::Write>(&right);
            ceph_assert(r && w.fadvise_flags == r->fadvise_flags);
-           bufferlist bl = w.buffer;
+           ceph::buffer::list bl = w.buffer;
            bl.append(r->buffer);
            return BufferUpdate::Write{bl, w.fadvise_flags};
          },
@@ -246,7 +246,7 @@ public:
 
     friend class PGTransaction;
   };
-  map<hobject_t, ObjectOperation> op_map;
+  std::map<hobject_t, ObjectOperation> op_map;
 private:
   ObjectOperation &get_object_op_for_modify(const hobject_t &hoid) {
     auto &op = op_map[hoid];
@@ -319,8 +319,8 @@ public:
 
   void update_snaps(
     const hobject_t &hoid,         ///< [in] object for snaps
-    const set<snapid_t> &old_snaps,///< [in] old snaps value
-    const set<snapid_t> &new_snaps ///< [in] new snaps value
+    const std::set<snapid_t> &old_snaps,///< [in] old snaps value
+    const std::set<snapid_t> &new_snaps ///< [in] new snaps value
     ) {
     auto &op = get_object_op(hoid);
     ceph_assert(!op.updated_snaps);
@@ -359,7 +359,7 @@ public:
   /// Attr ops
   void setattrs(
     const hobject_t &hoid,         ///< [in] object to write
-    map<string, bufferlist> &attrs ///< [in] attrs, may be cleared
+    std::map<std::string, ceph::buffer::list> &attrs ///< [in] attrs, may be cleared
     ) {
     auto &op = get_object_op_for_modify(hoid);
     for (auto &&i: attrs) {
@@ -370,8 +370,8 @@ public:
   }
   void setattr(
     const hobject_t &hoid,         ///< [in] object to write
-    const string &attrname,        ///< [in] attr to write
-    bufferlist &bl                 ///< [in] val to write, may be claimed
+    const std::string &attrname,        ///< [in] attr to write
+    ceph::buffer::list &bl                 ///< [in] val to write, may be claimed
     ) {
     auto &op = get_object_op_for_modify(hoid);
     auto& d = op.attr_updates[attrname];
@@ -380,7 +380,7 @@ public:
   }
   void rmattr(
     const hobject_t &hoid,         ///< [in] object to write
-    const string &attrname         ///< [in] attr to remove
+    const std::string &attrname         ///< [in] attr to remove
     ) {
     auto &op = get_object_op_for_modify(hoid);
     op.attr_updates[attrname] = std::nullopt;
@@ -403,7 +403,7 @@ public:
     const hobject_t &hoid,         ///< [in] object to write
     uint64_t off,                  ///< [in] off at which to write
     uint64_t len,                  ///< [in] len to write from bl
-    bufferlist &bl,                ///< [in] bl to write will be claimed to len
+    ceph::buffer::list &bl,                ///< [in] bl to write will be claimed to len
     uint32_t fadvise_flags = 0     ///< [in] fadvise hint
     ) {
     auto &op = get_object_op_for_modify(hoid);
@@ -445,47 +445,49 @@ public:
   /// Omap updates
   void omap_setkeys(
     const hobject_t &hoid,         ///< [in] object to write
-    bufferlist &keys_bl            ///< [in] encoded map<string, bufferlist>
+    ceph::buffer::list &keys_bl            ///< [in] encoded map<string, ceph::buffer::list>
     ) {
     auto &op = get_object_op_for_modify(hoid);
     op.omap_updates.emplace_back(
-      make_pair(
+      std::make_pair(
        ObjectOperation::OmapUpdateType::Insert,
        keys_bl));
   }
   void omap_setkeys(
     const hobject_t &hoid,         ///< [in] object to write
-    map<string, bufferlist> &keys  ///< [in] omap keys, may be cleared
+    std::map<std::string, ceph::buffer::list> &keys  ///< [in] omap keys, may be cleared
     ) {
-    bufferlist bl;
+    using ceph::encode;
+    ceph::buffer::list bl;
     encode(keys, bl);
     omap_setkeys(hoid, bl);
   }
   void omap_rmkeys(
     const hobject_t &hoid,         ///< [in] object to write
-    bufferlist &keys_bl            ///< [in] encode set<string>
+    ceph::buffer::list &keys_bl            ///< [in] encode set<string>
     ) {
     auto &op = get_object_op_for_modify(hoid);
     op.omap_updates.emplace_back(
-      make_pair(
+      std::make_pair(
        ObjectOperation::OmapUpdateType::Remove,
        keys_bl));
   }
   void omap_rmkeys(
     const hobject_t &hoid,         ///< [in] object to write
-    set<string> &keys              ///< [in] omap keys, may be cleared
+    std::set<std::string> &keys              ///< [in] omap keys, may be cleared
     ) {
-    bufferlist bl;
+    using ceph::encode;
+    ceph::buffer::list bl;
     encode(keys, bl);
     omap_rmkeys(hoid, bl);
   }
   void omap_rmkeyrange(
     const hobject_t &hoid,         ///< [in] object to write
-    bufferlist &range_bl           ///< [in] encode string[2]
+    ceph::buffer::list &range_bl           ///< [in] encode string[2]
     ) {
     auto &op = get_object_op_for_modify(hoid);
     op.omap_updates.emplace_back(
-      make_pair(
+      std::make_pair(
        ObjectOperation::OmapUpdateType::RemoveRange,
        range_bl));
   }
@@ -494,14 +496,14 @@ public:
     std::string& key_begin,        ///< [in] first key in range
     std::string& key_end           ///< [in] first key past range, range is [first,last)
     ) {
-    bufferlist bl;
+    ceph::buffer::list bl;
     ::encode(key_begin, bl);
     ::encode(key_end, bl);
     omap_rmkeyrange(hoid, bl);
   }
   void omap_setheader(
     const hobject_t &hoid,         ///< [in] object to write
-    bufferlist &header             ///< [in] header
+    ceph::buffer::list &header             ///< [in] header
     ) {
     auto &op = get_object_op_for_modify(hoid);
     op.omap_header = header;
@@ -548,8 +550,8 @@ public:
    */
   template <typename T>
   void safe_create_traverse(T &&t) {
-    map<hobject_t, list<hobject_t>> dgraph;
-    list<hobject_t> stack;
+    std::map<hobject_t, std::list<hobject_t>> dgraph;
+    std::list<hobject_t> stack;
 
     // Populate stack with roots, dgraph with edges
     for (auto &&opair: op_map) {
index 704a6a2236f326a73b9777ee9a9a829f1c735636..98484cc14fabe5439fcf848b5afd954630dbcb7d 100644 (file)
 #define dout_context cct
 #define dout_subsys ceph_subsys_osd
 
+using std::dec;
+using std::hex;
+using std::make_pair;
+using std::map;
+using std::ostream;
+using std::pair;
+using std::set;
+using std::stringstream;
+using std::vector;
+
+using ceph::Formatter;
+using ceph::make_message;
+
 BufferedRecoveryMessages::BufferedRecoveryMessages(
   ceph_release_t r,
   PeeringCtx &ctx)
@@ -202,9 +215,7 @@ void PeeringState::check_recovery_sources(const OSDMapRef& osdmap)
   missing_loc.check_recovery_sources(osdmap);
   pl->check_recovery_sources(osdmap);
 
-  for (set<pg_shard_t>::iterator i = peer_log_requested.begin();
-       i != peer_log_requested.end();
-       ) {
+  for (auto i = peer_log_requested.begin(); i != peer_log_requested.end();) {
     if (!osdmap->is_up(i->osd)) {
       psdout(10) << "peer_log_requested removing " << *i << dendl;
       peer_log_requested.erase(i++);
@@ -213,9 +224,8 @@ void PeeringState::check_recovery_sources(const OSDMapRef& osdmap)
     }
   }
 
-  for (set<pg_shard_t>::iterator i = peer_missing_requested.begin();
-       i != peer_missing_requested.end();
-       ) {
+  for (auto i = peer_missing_requested.begin();
+       i != peer_missing_requested.end();) {
     if (!osdmap->is_up(i->osd)) {
       psdout(10) << "peer_missing_requested removing " << *i << dendl;
       peer_missing_requested.erase(i++);
@@ -261,9 +271,7 @@ void PeeringState::purge_strays()
   psdout(10) << "purge_strays " << stray_set << dendl;
 
   bool removed = false;
-  for (set<pg_shard_t>::iterator p = stray_set.begin();
-       p != stray_set.end();
-       ++p) {
+  for (auto p = stray_set.begin(); p != stray_set.end(); ++p) {
     ceph_assert(!is_acting_recovery_backfill(*p));
     if (get_osdmap()->is_up(p->osd)) {
       psdout(10) << "sending PGRemove to osd." << *p << dendl;
@@ -299,7 +307,7 @@ void PeeringState::purge_strays()
 bool PeeringState::proc_replica_info(
   pg_shard_t from, const pg_info_t &oinfo, epoch_t send_epoch)
 {
-  map<pg_shard_t, pg_info_t>::iterator p = peer_info.find(from);
+  auto p = peer_info.find(from);
   if (p != peer_info.end() && p->second.last_update == oinfo.last_update) {
     psdout(10) << " got dup osd." << from << " info "
               << oinfo << ", identical to ours" << dendl;
@@ -340,7 +348,7 @@ void PeeringState::remove_down_peer_info(const OSDMapRef &osdmap)
 {
   // Remove any downed osds from peer_info
   bool removed = false;
-  map<pg_shard_t, pg_info_t>::iterator p = peer_info.begin();
+  auto p = peer_info.begin();
   while (p != peer_info.end()) {
     if (!osdmap->is_up(p->first.osd)) {
       psdout(10) << " dropping down osd." << p->first << " info " << p->second << dendl;
@@ -375,9 +383,7 @@ void PeeringState::update_heartbeat_peers()
     if (up[i] != CRUSH_ITEM_NONE)
       new_peers.insert(up[i]);
   }
-  for (map<pg_shard_t,pg_info_t>::iterator p = peer_info.begin();
-       p != peer_info.end();
-       ++p) {
+  for (auto p = peer_info.begin(); p != peer_info.end(); ++p) {
     new_peers.insert(p->first.osd);
   }
   pl->update_heartbeat_peers(std::move(new_peers));
@@ -721,14 +727,14 @@ void PeeringState::on_new_interval()
   // initialize features
   acting_features = CEPH_FEATURES_SUPPORTED_DEFAULT;
   upacting_features = CEPH_FEATURES_SUPPORTED_DEFAULT;
-  for (vector<int>::iterator p = acting.begin(); p != acting.end(); ++p) {
+  for (auto p = acting.begin(); p != acting.end(); ++p) {
     if (*p == CRUSH_ITEM_NONE)
       continue;
     uint64_t f = osdmap->get_xinfo(*p).features;
     acting_features &= f;
     upacting_features &= f;
   }
-  for (vector<int>::iterator p = up.begin(); p != up.end(); ++p) {
+  for (auto p = up.begin(); p != up.end(); ++p) {
     if (*p == CRUSH_ITEM_NONE)
       continue;
     upacting_features &= osdmap->get_xinfo(*p).features;
@@ -1273,9 +1279,7 @@ PastIntervals::PriorSet PeeringState::build_prior()
 {
   if (1) {
     // sanity check
-    for (map<pg_shard_t,pg_info_t>::iterator it = peer_info.begin();
-        it != peer_info.end();
-        ++it) {
+    for (auto it = peer_info.begin(); it != peer_info.end(); ++it) {
       ceph_assert(info.history.last_epoch_started >=
                  it->second.history.last_epoch_started);
     }
@@ -1341,12 +1345,12 @@ bool PeeringState::needs_recovery() const
   }
 
   ceph_assert(!acting_recovery_backfill.empty());
-  set<pg_shard_t>::const_iterator end = acting_recovery_backfill.end();
-  set<pg_shard_t>::const_iterator a = acting_recovery_backfill.begin();
+  auto end = acting_recovery_backfill.end();
+  auto a = acting_recovery_backfill.begin();
   for (; a != end; ++a) {
     if (*a == get_primary()) continue;
     pg_shard_t peer = *a;
-    map<pg_shard_t, pg_missing_t>::const_iterator pm = peer_missing.find(peer);
+    auto pm = peer_missing.find(peer);
     if (pm == peer_missing.end()) {
       psdout(10) << __func__ << " osd." << peer << " doesn't have missing set"
                 << dendl;
@@ -1369,11 +1373,11 @@ bool PeeringState::needs_backfill() const
 
   // We can assume that only possible osds that need backfill
   // are on the backfill_targets vector nodes.
-  set<pg_shard_t>::const_iterator end = backfill_targets.end();
-  set<pg_shard_t>::const_iterator a = backfill_targets.begin();
+  auto end = backfill_targets.end();
+  auto a = backfill_targets.begin();
   for (; a != end; ++a) {
     pg_shard_t peer = *a;
-    map<pg_shard_t, pg_info_t>::const_iterator pi = peer_info.find(peer);
+    auto pi = peer_info.find(peer);
     if (!pi->second.last_backfill.is_max()) {
       psdout(10) << __func__ << " osd." << peer
                 << " has last_backfill " << pi->second.last_backfill << dendl;
@@ -1393,12 +1397,12 @@ bool PeeringState::all_unfound_are_queried_or_lost(
 {
   ceph_assert(is_primary());
 
-  set<pg_shard_t>::const_iterator peer = might_have_unfound.begin();
-  set<pg_shard_t>::const_iterator mend = might_have_unfound.end();
+  auto peer = might_have_unfound.begin();
+  auto mend = might_have_unfound.end();
   for (; peer != mend; ++peer) {
     if (peer_missing.count(*peer))
       continue;
-    map<pg_shard_t, pg_info_t>::const_iterator iter = peer_info.find(*peer);
+    auto iter = peer_info.find(*peer);
     if (iter != peer_info.end() &&
         (iter->second.is_empty() || iter->second.dne()))
       continue;
@@ -1449,9 +1453,7 @@ map<pg_shard_t, pg_info_t>::const_iterator PeeringState::find_best_info(
    * when you find bugs! */
   eversion_t min_last_update_acceptable = eversion_t::max();
   epoch_t max_last_epoch_started_found = 0;
-  for (map<pg_shard_t, pg_info_t>::const_iterator i = infos.begin();
-       i != infos.end();
-       ++i) {
+  for (auto i = infos.begin(); i != infos.end(); ++i) {
     if (!cct->_conf->osd_find_best_info_ignore_history_les &&
        max_last_epoch_started_found < i->second.history.last_epoch_started) {
       *history_les_bound = true;
@@ -1463,9 +1465,7 @@ map<pg_shard_t, pg_info_t>::const_iterator PeeringState::find_best_info(
       max_last_epoch_started_found = i->second.last_epoch_started;
     }
   }
-  for (map<pg_shard_t, pg_info_t>::const_iterator i = infos.begin();
-       i != infos.end();
-       ++i) {
+  for (auto i = infos.begin(); i != infos.end(); ++i) {
     if (max_last_epoch_started_found <= i->second.last_epoch_started) {
       if (min_last_update_acceptable > i->second.last_update)
        min_last_update_acceptable = i->second.last_update;
@@ -1474,14 +1474,12 @@ map<pg_shard_t, pg_info_t>::const_iterator PeeringState::find_best_info(
   if (min_last_update_acceptable == eversion_t::max())
     return infos.end();
 
-  map<pg_shard_t, pg_info_t>::const_iterator best = infos.end();
+  auto best = infos.end();
   // find osd with newest last_update (oldest for ec_pool).
   // if there are multiples, prefer
   //  - a longer tail, if it brings another peer into log contiguity
   //  - the current primary
-  for (map<pg_shard_t, pg_info_t>::const_iterator p = infos.begin();
-       p != infos.end();
-       ++p) {
+  for (auto p = infos.begin(); p != infos.end(); ++p) {
     if (restrict_to_up_acting && !is_up(p->first) &&
        !is_acting(p->first))
       continue;
@@ -1564,7 +1562,7 @@ void PeeringState::calc_ec_acting(
 {
   vector<int> want(size, CRUSH_ITEM_NONE);
   map<shard_id_t, set<pg_shard_t> > all_info_by_shard;
-  for (map<pg_shard_t, pg_info_t>::const_iterator i = all_info.begin();
+  for (auto i = all_info.begin();
        i != all_info.end();
        ++i) {
     all_info_by_shard[i->first.shard].insert(i->first);
@@ -1592,7 +1590,7 @@ void PeeringState::calc_ec_acting(
       ss << " selecting acting[i]: " << pg_shard_t(acting[i], shard_id_t(i)) << std::endl;
       want[i] = acting[i];
     } else if (!restrict_to_up_acting) {
-      for (set<pg_shard_t>::iterator j = all_info_by_shard[shard_id_t(i)].begin();
+      for (auto j = all_info_by_shard[shard_id_t(i)].begin();
           j != all_info_by_shard[shard_id_t(i)].end();
           ++j) {
        ceph_assert(j->shard == i);
@@ -1729,7 +1727,7 @@ void PeeringState::calc_replicated_acting(
     // skip up osds we already considered above
     if (acting_cand == primary->first)
       continue;
-    vector<int>::const_iterator up_it = find(up.begin(), up.end(), i);
+    auto up_it = find(up.begin(), up.end(), i);
     if (up_it != up.end())
       continue;
 
@@ -1772,10 +1770,10 @@ void PeeringState::calc_replicated_acting(
     // skip up osds we already considered above
     if (i.first == primary->first)
       continue;
-    vector<int>::const_iterator up_it = find(up.begin(), up.end(), i.first.osd);
+    auto up_it = find(up.begin(), up.end(), i.first.osd);
     if (up_it != up.end())
       continue;
-    vector<int>::const_iterator acting_it = find(
+    auto acting_it = find(
       acting.begin(), acting.end(), i.first.osd);
     if (acting_it != acting.end())
       continue;
@@ -2012,16 +2010,14 @@ bool PeeringState::choose_acting(pg_shard_t &auth_log_shard_id,
   all_info[pg_whoami] = info;
 
   if (cct->_conf->subsys.should_gather<dout_subsys, 10>()) {
-    for (map<pg_shard_t, pg_info_t>::iterator p = all_info.begin();
-         p != all_info.end();
-         ++p) {
+    for (auto p = all_info.begin(); p != all_info.end(); ++p) {
       psdout(10) << __func__ << " all_info osd." << p->first << " "
                 << p->second << dendl;
     }
   }
 
-  map<pg_shard_t, pg_info_t>::const_iterator auth_log_shard =
-    find_best_info(all_info, restrict_to_up_acting, history_les_bound);
+  auto auth_log_shard = find_best_info(all_info, restrict_to_up_acting,
+                                      history_les_bound);
 
   if (auth_log_shard == all_info.end()) {
     if (up != acting) {
@@ -2138,9 +2134,7 @@ bool PeeringState::choose_acting(pg_shard_t &auth_log_shard_id,
   }
   // Will not change if already set because up would have had to change
   // Verify that nothing in backfill is in stray_set
-  for (set<pg_shard_t>::iterator i = want_backfill.begin();
-      i != want_backfill.end();
-      ++i) {
+  for (auto i = want_backfill.begin(); i != want_backfill.end(); ++i) {
     ceph_assert(stray_set.find(*i) == stray_set.end());
   }
   psdout(10) << "choose_acting want=" << want << " backfill_targets="
@@ -2227,8 +2221,8 @@ bool PeeringState::discover_all_missing(
             << unfound << " unfound"
             << dendl;
 
-  std::set<pg_shard_t>::const_iterator m = might_have_unfound.begin();
-  std::set<pg_shard_t>::const_iterator mend = might_have_unfound.end();
+  auto m = might_have_unfound.begin();
+  auto mend = might_have_unfound.end();
   for (; m != mend; ++m) {
     pg_shard_t peer(*m);
 
@@ -2242,7 +2236,7 @@ bool PeeringState::discover_all_missing(
       continue;
     }
 
-    map<pg_shard_t, pg_info_t>::const_iterator iter = peer_info.find(peer);
+    auto iter = peer_info.find(peer);
     if (iter != peer_info.end() &&
         (iter->second.is_empty() || iter->second.dne())) {
       // ignore empty peers
@@ -2306,9 +2300,7 @@ void PeeringState::build_might_have_unfound()
     pool.info.is_erasure());
 
   // include any (stray) peers
-  for (map<pg_shard_t, pg_info_t>::iterator p = peer_info.begin();
-       p != peer_info.end();
-       ++p)
+  for (auto p = peer_info.begin(); p != peer_info.end(); ++p)
     might_have_unfound.insert(p->first);
 
   psdout(15) << __func__ << ": built " << might_have_unfound << dendl;
@@ -2416,7 +2408,7 @@ void PeeringState::activate(
                                                 prior_readable_until_ub);
 
     ceph_assert(!acting_recovery_backfill.empty());
-    for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+    for (auto i = acting_recovery_backfill.begin();
         i != acting_recovery_backfill.end();
         ++i) {
       if (*i == pg_whoami) continue;
@@ -2521,9 +2513,7 @@ void PeeringState::activate(
 
       // update local version of peer's missing list!
       if (m && pi.last_backfill != hobject_t()) {
-        for (list<pg_log_entry_t>::iterator p = m->log.log.begin();
-             p != m->log.log.end();
-             ++p) {
+        for (auto p = m->log.log.begin(); p != m->log.log.end(); ++p) {
          if (p->soid <= pi.last_backfill &&
              !p->is_error()) {
            if (perform_deletes_during_peering() && p->is_delete()) {
@@ -2558,7 +2548,7 @@ void PeeringState::activate(
 
     // Set up missing_loc
     set<pg_shard_t> complete_shards;
-    for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+    for (auto i = acting_recovery_backfill.begin();
         i != acting_recovery_backfill.end();
         ++i) {
       psdout(20) << __func__ << " setting up missing_loc from shard " << *i
@@ -2591,7 +2581,7 @@ void PeeringState::activate(
       } else {
         missing_loc.add_source_info(pg_whoami, info, pg_log.get_missing(),
                                    ctx.handle);
-        for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+        for (auto i = acting_recovery_backfill.begin();
             i != acting_recovery_backfill.end();
             ++i) {
          if (*i == pg_whoami) continue;
@@ -2605,9 +2595,7 @@ void PeeringState::activate(
             ctx.handle);
         }
       }
-      for (map<pg_shard_t, pg_missing_t>::iterator i = peer_missing.begin();
-          i != peer_missing.end();
-          ++i) {
+      for (auto i = peer_missing.begin(); i != peer_missing.end(); ++i) {
        if (is_acting_recovery_backfill(i->first))
          continue;
        ceph_assert(peer_info.count(i->first));
@@ -2767,8 +2755,7 @@ void PeeringState::proc_replica_log(
             << oinfo << " " << omissing << dendl;
   might_have_unfound.insert(from);
 
-  for (map<hobject_t, pg_missing_item>::const_iterator i =
-        omissing.get_items().begin();
+  for (auto i = omissing.get_items().begin();
        i != omissing.get_items().end();
        ++i) {
     psdout(20) << " after missing " << i->first
@@ -3152,9 +3139,7 @@ void PeeringState::update_blocked_by()
   info.stats.blocked_by.clear();
   info.stats.blocked_by.resize(keep);
   unsigned pos = 0;
-  for (set<int>::iterator p = blocked_by.begin();
-       p != blocked_by.end() && keep > 0;
-       ++p) {
+  for (auto p = blocked_by.begin(); p != blocked_by.end() && keep > 0; ++p) {
     if (skip > 0 && (rand() % (skip + keep) < skip)) {
       --skip;
     } else {
@@ -3620,24 +3605,22 @@ void PeeringState::dump_peering_state(Formatter *f)
   f->dump_string("state", get_pg_state_string());
   f->dump_unsigned("epoch", get_osdmap_epoch());
   f->open_array_section("up");
-  for (vector<int>::const_iterator p = up.begin(); p != up.end(); ++p)
+  for (auto p = up.begin(); p != up.end(); ++p)
     f->dump_unsigned("osd", *p);
   f->close_section();
   f->open_array_section("acting");
-  for (vector<int>::const_iterator p = acting.begin(); p != acting.end(); ++p)
+  for (auto p = acting.begin(); p != acting.end(); ++p)
     f->dump_unsigned("osd", *p);
   f->close_section();
   if (!backfill_targets.empty()) {
     f->open_array_section("backfill_targets");
-    for (set<pg_shard_t>::iterator p = backfill_targets.begin();
-        p != backfill_targets.end();
-        ++p)
+    for (auto p = backfill_targets.begin(); p != backfill_targets.end(); ++p)
       f->dump_stream("shard") << *p;
     f->close_section();
   }
   if (!async_recovery_targets.empty()) {
     f->open_array_section("async_recovery_targets");
-    for (set<pg_shard_t>::iterator p = async_recovery_targets.begin();
+    for (auto p = async_recovery_targets.begin();
         p != async_recovery_targets.end();
         ++p)
       f->dump_stream("shard") << *p;
@@ -3645,7 +3628,7 @@ void PeeringState::dump_peering_state(Formatter *f)
   }
   if (!acting_recovery_backfill.empty()) {
     f->open_array_section("acting_recovery_backfill");
-    for (set<pg_shard_t>::iterator p = acting_recovery_backfill.begin();
+    for (auto p = acting_recovery_backfill.begin();
         p != acting_recovery_backfill.end();
         ++p)
       f->dump_stream("shard") << *p;
@@ -3657,9 +3640,7 @@ void PeeringState::dump_peering_state(Formatter *f)
   f->close_section();
 
   f->open_array_section("peer_info");
-  for (map<pg_shard_t, pg_info_t>::const_iterator p = peer_info.begin();
-       p != peer_info.end();
-       ++p) {
+  for (auto p = peer_info.begin(); p != peer_info.end(); ++p) {
     f->open_object_section("info");
     f->dump_stream("peer") << p->first;
     p->second.dump(f);
@@ -3731,7 +3712,7 @@ void PeeringState::merge_new_log_entries(
   ceph_assert(is_primary());
 
   bool rebuild_missing = append_log_entries_update_missing(entries, t, trim_to, roll_forward_to);
-  for (set<pg_shard_t>::const_iterator i = acting_recovery_backfill.begin();
+  for (auto i = acting_recovery_backfill.begin();
        i != acting_recovery_backfill.end();
        ++i) {
     pg_shard_t peer(*i);
@@ -3828,9 +3809,7 @@ void PeeringState::append_log(
     pg_log.skip_rollforward();
   }
 
-  for (vector<pg_log_entry_t>::const_iterator p = logv.begin();
-       p != logv.end();
-       ++p) {
+  for (auto p = logv.begin(); p != logv.end(); ++p) {
     add_log_entry(*p, transaction_applied);
 
     /* We don't want to leave the rollforward artifacts around
@@ -4070,7 +4049,7 @@ void PeeringState::calc_trim_to()
         cct->_conf->osd_pg_log_trim_max >= cct->_conf->osd_pg_log_trim_min) {
       return;
     }
-    list<pg_log_entry_t>::const_iterator it = pg_log.get_log().log.begin();
+    auto it = pg_log.get_log().log.begin();
     eversion_t new_trim_to;
     for (size_t i = 0; i < num_to_trim; ++i) {
       new_trim_to = it->version;
@@ -4146,7 +4125,7 @@ void PeeringState::apply_op_stats(
   info.stats.stats.add(delta_stats);
   info.stats.stats.floor(0);
 
-  for (set<pg_shard_t>::const_iterator i = get_backfill_targets().begin();
+  for (auto i = get_backfill_targets().begin();
        i != get_backfill_targets().end();
        ++i) {
     pg_shard_t bt = *i;
@@ -4581,9 +4560,7 @@ boost::statechart::result PeeringState::Peering::react(const QueryState& q)
   q.f->close_section();
 
   q.f->open_array_section("probing_osds");
-  for (set<pg_shard_t>::iterator p = prior_set.probe.begin();
-       p != prior_set.probe.end();
-       ++p)
+  for (auto p = prior_set.probe.begin(); p != prior_set.probe.end(); ++p)
     q.f->dump_stream("osd") << *p;
   q.f->close_section();
 
@@ -4591,14 +4568,12 @@ boost::statechart::result PeeringState::Peering::react(const QueryState& q)
     q.f->dump_string("blocked", "peering is blocked due to down osds");
 
   q.f->open_array_section("down_osds_we_would_probe");
-  for (set<int>::iterator p = prior_set.down.begin();
-       p != prior_set.down.end();
-       ++p)
+  for (auto p = prior_set.down.begin(); p != prior_set.down.end(); ++p)
     q.f->dump_int("osd", *p);
   q.f->close_section();
 
   q.f->open_array_section("peering_blocked_by");
-  for (map<int,epoch_t>::iterator p = prior_set.blocked_by.begin();
+  for (auto p = prior_set.blocked_by.begin();
        p != prior_set.blocked_by.end();
        ++p) {
     q.f->open_object_section("osd");
@@ -4656,7 +4631,7 @@ void PeeringState::Backfilling::backfill_release_reservations()
 {
   DECLARE_LOCALS;
   pl->cancel_local_background_io_reservation();
-  for (set<pg_shard_t>::iterator it = ps->backfill_targets.begin();
+  for (auto it = ps->backfill_targets.begin();
        it != ps->backfill_targets.end();
        ++it) {
     ceph_assert(*it != ps->pg_whoami);
@@ -5329,10 +5304,9 @@ void PeeringState::Recovering::release_reservations(bool cancel)
   ceph_assert(cancel || !ps->pg_log.get_missing().have_missing());
 
   // release remote reservations
-  for (set<pg_shard_t>::const_iterator i =
-        context< Active >().remote_shards_to_reserve_recovery.begin();
-        i != context< Active >().remote_shards_to_reserve_recovery.end();
-        ++i) {
+  for (auto i = context< Active >().remote_shards_to_reserve_recovery.begin();
+       i != context< Active >().remote_shards_to_reserve_recovery.end();
+       ++i) {
     if (*i == ps->pg_whoami) // skip myself
       continue;
     pl->send_cluster_message(
@@ -5498,9 +5472,7 @@ set<pg_shard_t> unique_osd_shard_set(const pg_shard_t & skip, const T &in)
 {
   set<int> osds_found;
   set<pg_shard_t> out;
-  for (typename T::const_iterator i = in.begin();
-       i != in.end();
-       ++i) {
+  for (auto i = in.begin(); i != in.end(); ++i) {
     if (*i != skip && !osds_found.count(i->osd)) {
       osds_found.insert(i->osd);
       out.insert(*i);
@@ -5538,7 +5510,7 @@ PeeringState::Active::Active(my_context ctx)
 
   // everyone has to commit/ack before we are truly active
   ps->blocked_by.clear();
-  for (set<pg_shard_t>::iterator p = ps->acting_recovery_backfill.begin();
+  for (auto p = ps->acting_recovery_backfill.begin();
        p != ps->acting_recovery_backfill.end();
        ++p) {
     if (p->shard != ps->pg_whoami.shard) {
@@ -5768,7 +5740,7 @@ boost::statechart::result PeeringState::Active::react(const QueryState& q)
 
   {
     q.f->open_array_section("might_have_unfound");
-    for (set<pg_shard_t>::iterator p = ps->might_have_unfound.begin();
+    for (auto p = ps->might_have_unfound.begin();
         p != ps->might_have_unfound.end();
         ++p) {
       q.f->open_object_section("osd");
@@ -5789,7 +5761,7 @@ boost::statechart::result PeeringState::Active::react(const QueryState& q)
   {
     q.f->open_object_section("recovery_progress");
     q.f->open_array_section("backfill_targets");
-    for (set<pg_shard_t>::const_iterator p = ps->backfill_targets.begin();
+    for (auto p = ps->backfill_targets.begin();
         p != ps->backfill_targets.end(); ++p)
       q.f->dump_stream("replica") << *p;
     q.f->close_section();
@@ -6360,9 +6332,7 @@ void PeeringState::GetInfo::get_infos()
   PastIntervals::PriorSet &prior_set = context< Peering >().prior_set;
 
   ps->blocked_by.clear();
-  for (set<pg_shard_t>::const_iterator it = prior_set.probe.begin();
-       it != prior_set.probe.end();
-       ++it) {
+  for (auto it = prior_set.probe.begin(); it != prior_set.probe.end(); ++it) {
     pg_shard_t peer = *it;
     if (peer == ps->pg_whoami) {
       continue;
@@ -6399,7 +6369,7 @@ boost::statechart::result PeeringState::GetInfo::react(const MNotifyRec& infoevt
 
   DECLARE_LOCALS;
 
-  set<pg_shard_t>::iterator p = peer_info_requested.find(infoevt.from);
+  auto p = peer_info_requested.find(infoevt.from);
   if (p != peer_info_requested.end()) {
     peer_info_requested.erase(p);
     ps->blocked_by.erase(infoevt.from.osd);
@@ -6418,7 +6388,7 @@ boost::statechart::result PeeringState::GetInfo::react(const MNotifyRec& infoevt
       // filter out any osds that got dropped from the probe set from
       // peer_info_requested.  this is less expensive than restarting
       // peering (which would re-probe everyone).
-      set<pg_shard_t>::iterator p = peer_info_requested.begin();
+      auto p = peer_info_requested.begin();
       while (p != peer_info_requested.end()) {
        if (prior_set.probe.count(*p) == 0) {
          psdout(20) << " dropping osd." << *p << " from info_requested, no longer in probe set" << dendl;
@@ -6452,7 +6422,7 @@ boost::statechart::result PeeringState::GetInfo::react(const QueryState& q)
   q.f->dump_stream("enter_time") << enter_time;
 
   q.f->open_array_section("requested_info_from");
-  for (set<pg_shard_t>::iterator p = peer_info_requested.begin();
+  for (auto p = peer_info_requested.begin();
        p != peer_info_requested.end();
        ++p) {
     q.f->open_object_section("osd");
@@ -6523,7 +6493,7 @@ PeeringState::GetLog::GetLog(my_context ctx)
   // how much log to request?
   eversion_t request_log_from = ps->info.last_update;
   ceph_assert(!ps->acting_recovery_backfill.empty());
-  for (set<pg_shard_t>::iterator p = ps->acting_recovery_backfill.begin();
+  for (auto p = ps->acting_recovery_backfill.begin();
        p != ps->acting_recovery_backfill.end();
        ++p) {
     if (*p == ps->pg_whoami) continue;
@@ -6628,7 +6598,7 @@ boost::statechart::result PeeringState::WaitActingChange::react(const AdvMap& ad
   OSDMapRef osdmap = advmap.osdmap;
 
   psdout(10) << "verifying no want_acting " << ps->want_acting << " targets didn't go down" << dendl;
-  for (vector<int>::iterator p = ps->want_acting.begin(); p != ps->want_acting.end(); ++p) {
+  for (auto p = ps->want_acting.begin(); p != ps->want_acting.end(); ++p) {
     if (!osdmap->is_up(*p)) {
       psdout(10) << " want_acting target osd." << *p << " went down, resetting" << dendl;
       post_event(advmap);
@@ -6816,7 +6786,7 @@ PeeringState::GetMissing::GetMissing(my_context ctx)
   ps->log_weirdness();
   ceph_assert(!ps->acting_recovery_backfill.empty());
   eversion_t since;
-  for (set<pg_shard_t>::iterator i = ps->acting_recovery_backfill.begin();
+  for (auto i = ps->acting_recovery_backfill.begin();
        i != ps->acting_recovery_backfill.end();
        ++i) {
     if (*i == ps->get_primary()) continue;
@@ -6923,7 +6893,7 @@ boost::statechart::result PeeringState::GetMissing::react(const QueryState& q)
   q.f->dump_stream("enter_time") << enter_time;
 
   q.f->open_array_section("peer_missing_requested");
-  for (set<pg_shard_t>::iterator p = peer_missing_requested.begin();
+  for (auto p = peer_missing_requested.begin();
        p != peer_missing_requested.end();
        ++p) {
     q.f->open_object_section("osd");
index a88d1e0f2b8efe40869ea609a8d1c113488eae7b..fa2ca6ec767829e21b9bd028906ad635edcea1d6 100644 (file)
@@ -30,13 +30,13 @@ struct PGPool {
   CephContext* cct;
   epoch_t cached_epoch;
   int64_t id;
-  string name;
+  std::string name;
 
   pg_pool_t info;
   SnapContext snapc;   // the default pool snapc, ready to go.
 
   PGPool(CephContext* cct, OSDMapRef map, int64_t i, const pg_pool_t& info,
-        const string& name)
+        const std::string& name)
     : cct(cct),
       cached_epoch(map->get_epoch()),
       id(i),
@@ -59,12 +59,12 @@ struct PGPool {
   }
 };
 
-class PeeringCtx;
+struct PeeringCtx;
 
 // [primary only] content recovery state
 struct BufferedRecoveryMessages {
   ceph_release_t require_osd_release;
-  map<int, vector<MessageRef>> message_map;
+  std::map<int, std::vector<MessageRef>> message_map;
 
   BufferedRecoveryMessages(ceph_release_t r)
     : require_osd_release(r) {
@@ -123,7 +123,7 @@ struct HeartbeatStamps : public RefCountedObject {
   /// highest up_from we've seen from this rank
   epoch_t up_from = 0;
 
-  void print(ostream& out) const {
+  void print(std::ostream& out) const {
     std::lock_guard l(lock);
     out << "hbstamp(osd." << osd << " up_from " << up_from
        << " peer_clock_delta [";
@@ -181,7 +181,7 @@ private:
 };
 using HeartbeatStampsRef = ceph::ref_t<HeartbeatStamps>;
 
-inline ostream& operator<<(ostream& out, const HeartbeatStamps& hb)
+inline std::ostream& operator<<(std::ostream& out, const HeartbeatStamps& hb)
 {
   hb.print(out);
   return out;
@@ -340,15 +340,15 @@ public:
 
     //============================ HB =============================
     /// Update hb set to peers
-    virtual void update_heartbeat_peers(set<int> peers) = 0;
+    virtual void update_heartbeat_peers(std::set<int> peers) = 0;
 
-    /// Set targets being probed in this interval
-    virtual void set_probe_targets(const set<pg_shard_t> &probe_set) = 0;
+    /// Std::set targets being probed in this interval
+    virtual void set_probe_targets(const std::set<pg_shard_t> &probe_set) = 0;
     /// Clear targets being probed in this interval
     virtual void clear_probe_targets() = 0;
 
     /// Queue for a pg_temp of wanted
-    virtual void queue_want_pg_temp(const vector<int> &wanted) = 0;
+    virtual void queue_want_pg_temp(const std::vector<int> &wanted) = 0;
     /// Clear queue for a pg_temp of wanted
     virtual void clear_want_pg_temp() = 0;
 
@@ -388,7 +388,7 @@ public:
     virtual void set_ready_to_merge_target(eversion_t lu, epoch_t les, epoch_t lec) = 0;
     virtual void set_ready_to_merge_source(eversion_t lu) = 0;
 
-    // ==================== Map notifications ===================
+    // ==================== Std::map notifications ===================
     virtual void on_active_actmap() = 0;
     virtual void on_active_advmap(const OSDMapRef &osdmap) = 0;
     virtual epoch_t oldest_stored_osdmap() = 0;
@@ -418,7 +418,7 @@ public:
     virtual void log_state_exit(
       const char *state_name, utime_t enter_time,
       uint64_t events, utime_t event_dur) = 0;
-    virtual void dump_recovery_info(Formatter *f) const = 0;
+    virtual void dump_recovery_info(ceph::Formatter *f) const = 0;
 
     virtual OstreamTemp get_clog_info() = 0;
     virtual OstreamTemp get_clog_error() = 0;
@@ -428,8 +428,8 @@ public:
   };
 
   struct QueryState : boost::statechart::event< QueryState > {
-    Formatter *f;
-    explicit QueryState(Formatter *f) : f(f) {}
+    ceph::Formatter *f;
+    explicit QueryState(ceph::Formatter *f) : f(f) {}
     void print(std::ostream *out) const {
       *out << "Query";
     }
@@ -438,12 +438,12 @@ public:
   struct AdvMap : boost::statechart::event< AdvMap > {
     OSDMapRef osdmap;
     OSDMapRef lastmap;
-    vector<int> newup, newacting;
+    std::vector<int> newup, newacting;
     int up_primary, acting_primary;
     AdvMap(
       OSDMapRef osdmap, OSDMapRef lastmap,
-      vector<int>& newup, int up_primary,
-      vector<int>& newacting, int acting_primary):
+      std::vector<int>& newup, int up_primary,
+      std::vector<int>& newacting, int acting_primary):
       osdmap(osdmap), lastmap(lastmap),
       newup(newup),
       newacting(newacting),
@@ -804,8 +804,8 @@ public:
     explicit Active(my_context ctx);
     void exit();
 
-    const set<pg_shard_t> remote_shards_to_reserve_recovery;
-    const set<pg_shard_t> remote_shards_to_reserve_backfill;
+    const std::set<pg_shard_t> remote_shards_to_reserve_recovery;
+    const std::set<pg_shard_t> remote_shards_to_reserve_backfill;
     bool all_replicas_activated;
 
     typedef boost::mpl::list <
@@ -928,7 +928,7 @@ public:
       boost::statechart::custom_reaction< RemoteReservationRevoked >,
       boost::statechart::transition< AllBackfillsReserved, Backfilling >
       > reactions;
-    set<pg_shard_t>::const_iterator backfill_osd_it;
+    std::set<pg_shard_t>::const_iterator backfill_osd_it;
     explicit WaitRemoteBackfillReserved(my_context ctx);
     void retry();
     void exit();
@@ -1134,7 +1134,7 @@ public:
       boost::statechart::custom_reaction< RemoteRecoveryReserved >,
       boost::statechart::transition< AllRemotesReserved, Recovering >
       > reactions;
-    set<pg_shard_t>::const_iterator remote_recovery_reservation_it;
+    std::set<pg_shard_t>::const_iterator remote_recovery_reservation_it;
     explicit WaitRemoteRecoveryReserved(my_context ctx);
     boost::statechart::result react(const RemoteRecoveryReserved &evt);
     void exit();
@@ -1228,7 +1228,7 @@ public:
   struct GetLog;
 
   struct GetInfo : boost::statechart::state< GetInfo, Peering >, NamedState {
-    set<pg_shard_t> peer_info_requested;
+    std::set<pg_shard_t> peer_info_requested;
 
     explicit GetInfo(my_context ctx);
     void exit();
@@ -1272,7 +1272,7 @@ public:
   struct WaitUpThru;
 
   struct GetMissing : boost::statechart::state< GetMissing, Peering >, NamedState {
-    set<pg_shard_t> peer_missing_requested;
+    std::set<pg_shard_t> peer_missing_requested;
 
     explicit GetMissing(my_context ctx);
     void exit();
@@ -1360,15 +1360,15 @@ public:
   pg_shard_t primary;        ///< id/shard of primary
   pg_shard_t pg_whoami;      ///< my id/shard
   pg_shard_t up_primary;     ///< id/shard of primary of up set
-  vector<int> up;            ///< crush mapping without temp pgs
-  set<pg_shard_t> upset;     ///< up in set form
-  vector<int> acting;        ///< actual acting set for the current interval
-  set<pg_shard_t> actingset; ///< acting in set form
+  std::vector<int> up;            ///< crush mapping without temp pgs
+  std::set<pg_shard_t> upset;     ///< up in set form
+  std::vector<int> acting;        ///< actual acting set for the current interval
+  std::set<pg_shard_t> actingset; ///< acting in set form
 
   /// union of acting, recovery, and backfill targets
-  set<pg_shard_t> acting_recovery_backfill;
+  std::set<pg_shard_t> acting_recovery_backfill;
 
-  vector<HeartbeatStampsRef> hb_stamps;
+  std::vector<HeartbeatStampsRef> hb_stamps;
 
   ceph::signedspan readable_interval = ceph::signedspan::zero();
 
@@ -1382,7 +1382,7 @@ public:
   ceph::signedspan prior_readable_until_ub = ceph::signedspan::zero();
 
   /// pg instances from prior interval(s) that may still be readable
-  set<int> prior_readable_down_osds;
+  std::set<int> prior_readable_down_osds;
 
   /// [replica] upper bound we got from the primary (primary's clock)
   ceph::signedspan readable_until_ub_from_primary = ceph::signedspan::zero();
@@ -1391,7 +1391,7 @@ public:
   ceph::signedspan readable_until_ub_sent = ceph::signedspan::zero();
 
   /// [primary] readable ub acked by acting set members
-  vector<ceph::signedspan> acting_readable_until_ub;
+  std::vector<ceph::signedspan> acting_readable_until_ub;
 
   bool send_notify = false; ///< True if a notify needs to be sent to the primary
 
@@ -1418,13 +1418,13 @@ public:
   /**
    * Primary state
    */
-  set<pg_shard_t>    stray_set; ///< non-acting osds that have PG data.
-  map<pg_shard_t, pg_info_t>    peer_info; ///< info from peers (stray or prior)
-  map<pg_shard_t, int64_t>    peer_bytes; ///< Peer's num_bytes from peer_info
-  set<pg_shard_t> peer_purged; ///< peers purged
-  map<pg_shard_t, pg_missing_t> peer_missing; ///< peer missing sets
-  set<pg_shard_t> peer_log_requested; ///< logs i've requested (and start stamps)
-  set<pg_shard_t> peer_missing_requested; ///< missing sets requested
+  std::set<pg_shard_t>    stray_set; ///< non-acting osds that have PG data.
+  std::map<pg_shard_t, pg_info_t>    peer_info; ///< info from peers (stray or prior)
+  std::map<pg_shard_t, int64_t>    peer_bytes; ///< Peer's num_bytes from peer_info
+  std::set<pg_shard_t> peer_purged; ///< peers purged
+  std::map<pg_shard_t, pg_missing_t> peer_missing; ///< peer missing sets
+  std::set<pg_shard_t> peer_log_requested; ///< logs i've requested (and start stamps)
+  std::set<pg_shard_t> peer_missing_requested; ///< missing sets requested
 
   /// features supported by all peers
   uint64_t peer_features = CEPH_FEATURES_SUPPORTED_DEFAULT;
@@ -1436,29 +1436,29 @@ public:
   /// most recently consumed osdmap's require_osd_version
   ceph_release_t last_require_osd_release = ceph_release_t::unknown;
 
-  vector<int> want_acting; ///< non-empty while peering needs a new acting set
+  std::vector<int> want_acting; ///< non-empty while peering needs a new acting set
 
   // acting_recovery_backfill contains shards that are acting,
   // async recovery targets, or backfill targets.
-  map<pg_shard_t,eversion_t> peer_last_complete_ondisk;
+  std::map<pg_shard_t,eversion_t> peer_last_complete_ondisk;
 
   /// up: min over last_complete_ondisk, peer_last_complete_ondisk
   eversion_t  min_last_complete_ondisk;
   /// point to which the log should be trimmed
   eversion_t  pg_trim_to;
 
-  set<int> blocked_by; ///< osds we are blocked by (for pg stats)
+  std::set<int> blocked_by; ///< osds we are blocked by (for pg stats)
 
   bool need_up_thru = false; ///< true if osdmap with updated up_thru needed
 
   /// I deleted these strays; ignore racing PGInfo from them
-  set<pg_shard_t> peer_activated;
+  std::set<pg_shard_t> peer_activated;
 
-  set<pg_shard_t> backfill_targets;       ///< osds to be backfilled
-  set<pg_shard_t> async_recovery_targets; ///< osds to be async recovered
+  std::set<pg_shard_t> backfill_targets;       ///< osds to be backfilled
+  std::set<pg_shard_t> async_recovery_targets; ///< osds to be async recovered
 
   /// osds which might have objects on them which are unfound on the primary
-  set<pg_shard_t> might_have_unfound;
+  std::set<pg_shard_t> might_have_unfound;
 
   bool deleting = false;  /// true while in removing or OSD is shutting down
   std::atomic<bool> deleted = {false}; /// true once deletion complete
@@ -1484,14 +1484,14 @@ public:
   bool should_restart_peering(
     int newupprimary,
     int newactingprimary,
-    const vector<int>& newup,
-    const vector<int>& newacting,
+    const std::vector<int>& newup,
+    const std::vector<int>& newacting,
     OSDMapRef lastmap,
     OSDMapRef osdmap);
   void start_peering_interval(
     const OSDMapRef lastmap,
-    const vector<int>& newup, int up_primary,
-    const vector<int>& newacting, int acting_primary,
+    const std::vector<int>& newup, int up_primary,
+    const std::vector<int>& newacting, int acting_primary,
     ObjectStore::Transaction &t);
   void on_new_interval();
   void clear_recovery_state();
@@ -1516,50 +1516,50 @@ public:
 
   void reject_reservation();
 
-  // acting set
-  map<pg_shard_t, pg_info_t>::const_iterator find_best_info(
-    const map<pg_shard_t, pg_info_t> &infos,
+  // acting std::set
+  std::map<pg_shard_t, pg_info_t>::const_iterator find_best_info(
+    const std::map<pg_shard_t, pg_info_t> &infos,
     bool restrict_to_up_acting,
     bool *history_les_bound) const;
   static void calc_ec_acting(
-    map<pg_shard_t, pg_info_t>::const_iterator auth_log_shard,
+    std::map<pg_shard_t, pg_info_t>::const_iterator auth_log_shard,
     unsigned size,
-    const vector<int> &acting,
-    const vector<int> &up,
-    const map<pg_shard_t, pg_info_t> &all_info,
+    const std::vector<int> &acting,
+    const std::vector<int> &up,
+    const std::map<pg_shard_t, pg_info_t> &all_info,
     bool restrict_to_up_acting,
-    vector<int> *want,
-    set<pg_shard_t> *backfill,
-    set<pg_shard_t> *acting_backfill,
-    ostream &ss);
+    std::vector<int> *want,
+    std::set<pg_shard_t> *backfill,
+    std::set<pg_shard_t> *acting_backfill,
+    std::ostream &ss);
   static void calc_replicated_acting(
-    map<pg_shard_t, pg_info_t>::const_iterator auth_log_shard,
+    std::map<pg_shard_t, pg_info_t>::const_iterator auth_log_shard,
     uint64_t force_auth_primary_missing_objects,
     unsigned size,
-    const vector<int> &acting,
-    const vector<int> &up,
+    const std::vector<int> &acting,
+    const std::vector<int> &up,
     pg_shard_t up_primary,
-    const map<pg_shard_t, pg_info_t> &all_info,
+    const std::map<pg_shard_t, pg_info_t> &all_info,
     bool restrict_to_up_acting,
-    vector<int> *want,
-    set<pg_shard_t> *backfill,
-    set<pg_shard_t> *acting_backfill,
+    std::vector<int> *want,
+    std::set<pg_shard_t> *backfill,
+    std::set<pg_shard_t> *acting_backfill,
     const OSDMapRef osdmap,
-    ostream &ss);
+    std::ostream &ss);
   void choose_async_recovery_ec(
-    const map<pg_shard_t, pg_info_t> &all_info,
+    const std::map<pg_shard_t, pg_info_t> &all_info,
     const pg_info_t &auth_info,
-    vector<int> *want,
-    set<pg_shard_t> *async_recovery,
+    std::vector<int> *want,
+    std::set<pg_shard_t> *async_recovery,
     const OSDMapRef osdmap) const;
   void choose_async_recovery_replicated(
-    const map<pg_shard_t, pg_info_t> &all_info,
+    const std::map<pg_shard_t, pg_info_t> &all_info,
     const pg_info_t &auth_info,
-    vector<int> *want,
-    set<pg_shard_t> *async_recovery,
+    std::vector<int> *want,
+    std::set<pg_shard_t> *async_recovery,
     const OSDMapRef osdmap) const;
 
-  bool recoverable(const vector<int> &want) const;
+  bool recoverable(const std::vector<int> &want) const;
   bool choose_acting(pg_shard_t &auth_log_shard,
                     bool restrict_to_up_acting,
                     bool *history_les_bound,
@@ -1591,7 +1591,7 @@ public:
   void calc_min_last_complete_ondisk() {
     eversion_t min = last_complete_ondisk;
     ceph_assert(!acting_recovery_backfill.empty());
-    for (set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
+    for (std::set<pg_shard_t>::iterator i = acting_recovery_backfill.begin();
         i != acting_recovery_backfill.end();
         ++i) {
       if (*i == get_primary()) continue;
@@ -1609,7 +1609,7 @@ public:
 
   void fulfill_info(
     pg_shard_t from, const pg_query_t &query,
-    pair<pg_shard_t, pg_info_t> &notify_info);
+    std::pair<pg_shard_t, pg_info_t> &notify_info);
   void fulfill_log(
     pg_shard_t from, const pg_query_t &query, epoch_t query_epoch);
   void fulfill_query(const MQuery& q, PeeringCtxWrapper &rctx);
@@ -1653,8 +1653,8 @@ public:
   /// Init fresh instance of PG
   void init(
     int role,
-    const vector<int>& newup, int new_up_primary,
-    const vector<int>& newacting, int new_acting_primary,
+    const std::vector<int>& newup, int new_up_primary,
+    const std::vector<int>& newacting, int new_acting_primary,
     const pg_history_t& history,
     const PastIntervals& pi,
     bool backfill,
@@ -1674,20 +1674,20 @@ public:
     return ret;
   }
 
-  /// Set initial primary/acting
+  /// Std::set initial primary/acting
   void init_primary_up_acting(
-    const vector<int> &newup,
-    const vector<int> &newacting,
+    const std::vector<int> &newup,
+    const std::vector<int> &newacting,
     int new_up_primary,
     int new_acting_primary);
   void init_hb_stamps();
 
-  /// Set initial role
+  /// Std::set initial role
   void set_role(int r) {
     role = r;
   }
 
-  /// Set predicates used for determining readable and recoverable
+  /// Std::set predicates used for determining readable and recoverable
   void set_backend_predicates(
     IsPGReadablePredicate *is_readable,
     IsPGRecoverablePredicate *is_recoverable) {
@@ -1699,7 +1699,7 @@ public:
 
   /// Get stats for child pgs
   void start_split_stats(
-    const set<spg_t>& childpgs, vector<object_stat_sum_t> *out);
+    const std::set<spg_t>& childpgs, std::vector<object_stat_sum_t> *out);
 
   /// Update new child with stats
   void finish_split_stats(
@@ -1711,7 +1711,7 @@ public:
 
   /// Merge state from sources
   void merge_from(
-    map<spg_t,PeeringState *>& sources,
+    std::map<spg_t,PeeringState *>& sources,
     PeeringCtx &rctx,
     unsigned split_bits,
     const pg_merge_meta_t& last_pg_merge_meta);
@@ -1787,7 +1787,7 @@ public:
    * Updates local log to reflect new write from primary.
    */
   void append_log(
-    vector<pg_log_entry_t>&& logv,
+    std::vector<pg_log_entry_t>&& logv,
     eversion_t trim_to,
     eversion_t roll_forward_to,
     eversion_t min_last_complete_ondisk,
@@ -1821,7 +1821,7 @@ public:
   /// Pre-process pending update on hoid represented by logv
   void pre_submit_op(
     const hobject_t &hoid,
-    const vector<pg_log_entry_t>& logv,
+    const std::vector<pg_log_entry_t>& logv,
     eversion_t at_version);
 
   /// Signal that oid has been locally recovered to version v
@@ -1880,16 +1880,16 @@ public:
    *
    * Force oid on peer to be missing at version.  If the object does not
    * currently need recovery, either candidates if provided or the remainder
-   * of the acting set will be deemed to have the object.
+   * of the acting std::set will be deemed to have the object.
    */
   void force_object_missing(
     const pg_shard_t &peer,
     const hobject_t &oid,
     eversion_t version) {
-    force_object_missing(set<pg_shard_t>{peer}, oid, version);
+    force_object_missing(std::set<pg_shard_t>{peer}, oid, version);
   }
   void force_object_missing(
-    const set<pg_shard_t> &peer,
+    const std::set<pg_shard_t> &peer,
     const hobject_t &oid,
     eversion_t version);
 
@@ -1897,12 +1897,12 @@ public:
   void prepare_backfill_for_missing(
     const hobject_t &soid,
     const eversion_t &version,
-    const vector<pg_shard_t> &targets);
+    const std::vector<pg_shard_t> &targets);
 
-  /// Set targets with the right version for revert (see recover_primary)
+  /// Std::set targets with the right version for revert (see recover_primary)
   void set_revert_with_targets(
     const hobject_t &soid,
-    const set<pg_shard_t> &good_peers);
+    const std::set<pg_shard_t> &good_peers);
 
   /// Update lcod for fromosd
   void update_peer_last_complete_ondisk(
@@ -1945,9 +1945,9 @@ public:
   void advance_map(
     OSDMapRef osdmap,       ///< [in] new osdmap
     OSDMapRef lastmap,      ///< [in] prev osdmap
-    vector<int>& newup,     ///< [in] new up set
+    std::vector<int>& newup,     ///< [in] new up set
     int up_primary,         ///< [in] new up primary
-    vector<int>& newacting, ///< [in] new acting
+    std::vector<int>& newacting, ///< [in] new acting
     int acting_primary,     ///< [in] new acting primary
     PeeringCtx &rctx        ///< [out] recovery context
     );
@@ -1992,7 +1992,7 @@ public:
   }
 
   /// Get prior intervals' readable_until down OSDs of note
-  const set<int>& get_prior_readable_down_osds() const {
+  const std::set<int>& get_prior_readable_down_osds() const {
     return prior_readable_down_osds;
   }
 
@@ -2064,7 +2064,7 @@ public:
   bool is_deleted() const {
     return deleted;
   }
-  const set<pg_shard_t> &get_upset() const override {
+  const std::set<pg_shard_t> &get_upset() const override {
     return upset;
   }
   bool is_acting_recovery_backfill(pg_shard_t osd) const {
@@ -2076,7 +2076,7 @@ public:
   bool is_up(pg_shard_t osd) const {
     return has_shard(pool.info.is_erasure(), up, osd);
   }
-  static bool has_shard(bool ec, const vector<int>& v, pg_shard_t osd) {
+  static bool has_shard(bool ec, const std::vector<int>& v, pg_shard_t osd) {
     if (ec) {
       return v.size() > (unsigned)osd.shard && v[osd.shard] == osd.osd;
     } else {
@@ -2101,10 +2101,10 @@ public:
   int get_role() const {
     return role;
   }
-  const vector<int> &get_acting() const {
+  const std::vector<int> &get_acting() const {
     return acting;
   }
-  const set<pg_shard_t> &get_actingset() const {
+  const std::set<pg_shard_t> &get_actingset() const {
     return actingset;
   }
   int get_acting_primary() const {
@@ -2113,7 +2113,7 @@ public:
   pg_shard_t get_primary() const {
     return primary;
   }
-  const vector<int> &get_up() const {
+  const std::vector<int> &get_up() const {
     return up;
   }
   int get_up_primary() const {
@@ -2123,16 +2123,16 @@ public:
   bool is_backfill_target(pg_shard_t osd) const {
     return backfill_targets.count(osd);
   }
-  const set<pg_shard_t> &get_backfill_targets() const {
+  const std::set<pg_shard_t> &get_backfill_targets() const {
     return backfill_targets;
   }
   bool is_async_recovery_target(pg_shard_t peer) const {
     return async_recovery_targets.count(peer);
   }
-  const set<pg_shard_t> &get_async_recovery_targets() const {
+  const std::set<pg_shard_t> &get_async_recovery_targets() const {
     return async_recovery_targets;
   }
-  const set<pg_shard_t> &get_acting_recovery_backfill() const {
+  const std::set<pg_shard_t> &get_acting_recovery_backfill() const {
     return acting_recovery_backfill;
   }
 
@@ -2297,17 +2297,17 @@ public:
   }
 
   /// Dump representation of past_intervals to out
-  void print_past_intervals(ostream &out) const {
+  void print_past_intervals(std::ostream &out) const {
     out << "[" << past_intervals.get_bounds()
        << ")/" << past_intervals.size();
   }
 
-  void dump_history(Formatter *f) const {
+  void dump_history(ceph::Formatter *f) const {
     state_history.dump(f);
   }
 
   /// Dump formatted peering status
-  void dump_peering_state(Formatter *f);
+  void dump_peering_state(ceph::Formatter *f);
 
 private:
   /// Mask feature vector with feature set from new peer
@@ -2347,7 +2347,7 @@ public:
   /// Must be called once per start_flush
   void complete_flush();
 
-  friend ostream &operator<<(ostream &out, const PeeringState &ps);
+  friend std::ostream &operator<<(std::ostream &out, const PeeringState &ps);
 };
 
-ostream &operator<<(ostream &out, const PeeringState &ps);
+std::ostream &operator<<(std::ostream &out, const PeeringState &ps);
index 873dd906e5131779d9dc0dafd6a7920dadab2d7a..d1c5ec46277b45f274631853b170c0fc86d4e977 100644 (file)
 #define DOUT_PREFIX_ARGS this, osd->whoami, get_osdmap()
 #undef dout_prefix
 #define dout_prefix _prefix(_dout, this)
-using TOPNSPC::common::cmd_getval;
-
-template <typename T>
-static ostream& _prefix(std::ostream *_dout, T *pg) {
-  return pg->gen_prefix(*_dout);
-}
-
 
 #include <sstream>
 #include <utility>
@@ -78,7 +71,34 @@ static ostream& _prefix(std::ostream *_dout, T *pg) {
 
 MEMPOOL_DEFINE_OBJECT_FACTORY(PrimaryLogPG, replicatedpg, osd);
 
+using std::list;
+using std::ostream;
+using std::pair;
+using std::make_pair;
+using std::map;
+using std::ostringstream;
+using std::set;
+using std::string;
+using std::string_view;
+using std::stringstream;
+using std::unique_ptr;
+using std::vector;
+
+using ceph::bufferlist;
+using ceph::bufferptr;
+using ceph::Formatter;
+using ceph::decode;
+using ceph::decode_noclear;
+using ceph::encode;
+using ceph::encode_destructively;
+
 using namespace ceph::osd::scheduler;
+using TOPNSPC::common::cmd_getval;
+
+template <typename T>
+static ostream& _prefix(std::ostream *_dout, T *pg) {
+  return pg->gen_prefix(*_dout);
+}
 
 /**
  * The CopyCallback class defines an interface for completions to the
@@ -173,7 +193,7 @@ public:
     else
       c.release()->complete(r);
   }
-  bool sync_finish(int r) {
+  bool sync_finish(int r) override {
     // we assume here all blessed/wrapped Contexts can complete synchronously.
     c.release()->complete(r);
     return true;
@@ -455,7 +475,7 @@ void PrimaryLogPG::on_global_recover(
   recovery_state.object_recovered(soid, stat_diff);
   publish_stats_to_osd();
   dout(10) << "pushed " << soid << " to all replicas" << dendl;
-  map<hobject_t, ObjectContextRef>::iterator i = recovering.find(soid);
+  auto i = recovering.find(soid);
   ceph_assert(i != recovering.end());
 
   if (i->second && i->second->rwstate.recovery_read_marker) {
@@ -894,7 +914,7 @@ PrimaryLogPG::get_pgls_filter(bufferlist::const_iterator& iter)
   try {
     decode(type, iter);
   }
-  catch (buffer::error& e) {
+  catch (ceph::buffer::error& e) {
     return { -EINVAL, nullptr };
   }
 
@@ -1181,7 +1201,7 @@ void PrimaryLogPG::do_pg_op(OpRequestRef op)
        decode(cname, bp);
        decode(mname, bp);
       }
-      catch (const buffer::error& e) {
+      catch (const ceph::buffer::error& e) {
        dout(0) << "unable to decode PGLS_FILTER description in " << *m << dendl;
        result = -EINVAL;
        break;
@@ -1216,7 +1236,7 @@ void PrimaryLogPG::do_pg_op(OpRequestRef op)
        try {
          decode(response.handle, bp);
        }
-       catch (const buffer::error& e) {
+       catch (const ceph::buffer::error& e) {
          dout(0) << "unable to decode PGNLS handle in " << *m << dendl;
          result = -EINVAL;
          break;
@@ -1347,7 +1367,7 @@ void PrimaryLogPG::do_pg_op(OpRequestRef op)
        decode(cname, bp);
        decode(mname, bp);
       }
-      catch (const buffer::error& e) {
+      catch (const ceph::buffer::error& e) {
        dout(0) << "unable to decode PGLS_FILTER description in " << *m << dendl;
        result = -EINVAL;
        break;
@@ -1381,7 +1401,7 @@ void PrimaryLogPG::do_pg_op(OpRequestRef op)
        try {
          decode(response.handle, bp);
        }
-       catch (const buffer::error& e) {
+       catch (const ceph::buffer::error& e) {
          dout(0) << "unable to decode PGLS handle in " << *m << dendl;
          result = -EINVAL;
          break;
@@ -1557,7 +1577,7 @@ int PrimaryLogPG::do_scrub_ls(const MOSDOp *m, OSDOp *osd_op)
   scrub_ls_arg_t arg;
   try {
     arg.decode(bp);
-  } catch (buffer::error&) {
+  } catch (ceph::buffer::error&) {
     dout(10) << " corrupted scrub_ls_arg_t" << dendl;
     return -EINVAL;
   }
@@ -2565,11 +2585,11 @@ int PrimaryLogPG::do_manifest_flush(OpRequestRef op, ObjectContextRef obc, Flush
       object_t fp_oid = [fp_algo, &chunk_data]() -> string {
         switch (fp_algo) {
        case pg_pool_t::TYPE_FINGERPRINT_SHA1:
-         return crypto::digest<crypto::SHA1>(chunk_data).to_str();
+         return ceph::crypto::digest<ceph::crypto::SHA1>(chunk_data).to_str();
        case pg_pool_t::TYPE_FINGERPRINT_SHA256:
-         return crypto::digest<crypto::SHA256>(chunk_data).to_str();
+         return ceph::crypto::digest<ceph::crypto::SHA256>(chunk_data).to_str();
        case pg_pool_t::TYPE_FINGERPRINT_SHA512:
-         return crypto::digest<crypto::SHA512>(chunk_data).to_str();
+         return ceph::crypto::digest<ceph::crypto::SHA512>(chunk_data).to_str();
        default:
          assert(0 == "unrecognized fingerprint type");
          return {};
@@ -2577,12 +2597,12 @@ int PrimaryLogPG::do_manifest_flush(OpRequestRef op, ObjectContextRef obc, Flush
       }();
       bufferlist in;
       if (fp_oid != tgt_soid.oid) {
-       // decrement old chunk's reference count 
+       // decrement old chunk's reference count
        ObjectOperation dec_op;
        cls_chunk_refcount_put_op put_call;
        put_call.source = soid;
-       ::encode(put_call, in);                             
-       dec_op.call("cas", "chunk_put", in);         
+       ::encode(put_call, in);
+       dec_op.call("cas", "chunk_put", in);
        // we don't care dec_op's completion. scrub for dedup will fix this.
        tid = osd->objecter->mutate(
          tgt_soid.oid, oloc, dec_op, snapc,
@@ -4185,7 +4205,7 @@ void PrimaryLogPG::do_scan(
 
       // take care to preserve ordering!
       bi.clear_objects();
-      ::decode_noclear(bi.objects, p);
+      decode_noclear(bi.objects, p);
 
       if (waiting_on_backfill.erase(from)) {
        if (waiting_on_backfill.empty()) {
@@ -4870,7 +4890,7 @@ int PrimaryLogPG::do_tmapup(OpContext *ctx, bufferlist::const_iterator& bp, OSDO
        decode(op, bp);
        decode(key, bp);
       }
-      catch (buffer::error& e) {
+      catch (ceph::buffer::error& e) {
        return -EINVAL;
       }
       if (key < last_in_key) {
@@ -4913,7 +4933,7 @@ int PrimaryLogPG::do_tmapup(OpContext *ctx, bufferlist::const_iterator& bp, OSDO
        try {
          decode(val, bp);
        }
-       catch (buffer::error& e) {
+       catch (ceph::buffer::error& e) {
          return -EINVAL;
        }
        encode(key, newkeydata);
@@ -4928,7 +4948,7 @@ int PrimaryLogPG::do_tmapup(OpContext *ctx, bufferlist::const_iterator& bp, OSDO
        try {
          decode(val, bp);
        }
-       catch (buffer::error& e) {
+       catch (ceph::buffer::error& e) {
          return -EINVAL;
        }
        encode(key, newkeydata);
@@ -5066,7 +5086,7 @@ struct ToSparseReadResult : public Context {
     bufferlist outdata;
     map<uint64_t, uint64_t> extents = {{data_offset, r}};
     encode(extents, outdata);
-    ::encode_destructively(*data_bl, outdata);
+    encode_destructively(*data_bl, outdata);
     data_bl->swap(outdata);
   }
 };
@@ -5298,7 +5318,7 @@ int PrimaryLogPG::finish_checksum(OSDOp& osd_op,
   bufferptr csum_data;
   if (csum_count > 0) {
     size_t csum_value_size = Checksummer::get_csum_value_size(csum_type);
-    csum_data = buffer::create(csum_value_size * csum_count);
+    csum_data = ceph::buffer::create(csum_value_size * csum_count);
     csum_data.zero();
     csum.append(csum_data);
 
@@ -5823,7 +5843,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
          bp.copy(op.cls.class_len, cname);
          bp.copy(op.cls.method_len, mname);
          bp.copy(op.cls.indata_len, indata);
-       } catch (buffer::error& e) {
+       } catch (ceph::buffer::error& e) {
          dout(10) << "call unable to decode class + method + indata" << dendl;
          dout(30) << "in dump: ";
          osd_op.indata.hexdump(*_dout);
@@ -6110,7 +6130,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
            try {
              decode(u64val, bp);
            }
-           catch (buffer::error& e) {
+           catch (ceph::buffer::error& e) {
              result = -EINVAL;
              goto fail;
            }
@@ -6281,7 +6301,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
           decode(ver, bp);
          decode(timeout, bp);
           decode(bl, bp);
-       } catch (const buffer::error &e) {
+       } catch (const ceph::buffer::error &e) {
          timeout = 0;
        }
        tracepoint(osd, do_osd_op_pre_notify, soid.oid.name.c_str(), soid.snap.val, timeout);
@@ -6315,7 +6335,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
          tracepoint(osd, do_osd_op_pre_notify_ack, soid.oid.name.c_str(), soid.snap.val, notify_id, watch_cookie, "Y");
          OpContext::NotifyAck ack(notify_id, watch_cookie, reply_bl);
          ctx->notify_acks.push_back(ack);
-       } catch (const buffer::error &e) {
+       } catch (const ceph::buffer::error &e) {
          tracepoint(osd, do_osd_op_pre_notify_ack, soid.oid.name.c_str(), soid.snap.val, op.watch.cookie, 0, "N");
          OpContext::NotifyAck ack(
            // op.watch.cookie is actually the notify_id for historical reasons
@@ -6550,7 +6570,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
            try {
              decode(category, p);
            }
-           catch (buffer::error& e) {
+           catch (ceph::buffer::error& e) {
              result = -EINVAL;
              goto fail;
            }
@@ -6810,7 +6830,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
          decode(target_name, bp);
          decode(target_oloc, bp);
        }
-       catch (buffer::error& e) {
+       catch (ceph::buffer::error& e) {
          result = -EINVAL;
          goto fail;
        }
@@ -6919,7 +6939,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
          decode(tgt_name, bp);
          decode(tgt_offset, bp);
        }
-       catch (buffer::error& e) {
+       catch (ceph::buffer::error& e) {
          result = -EINVAL;
          goto fail;
        }
@@ -7328,7 +7348,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
          decode(start_after, bp);
          decode(max_return, bp);
        }
-       catch (buffer::error& e) {
+       catch (ceph::buffer::error& e) {
          result = -EINVAL;
          tracepoint(osd, do_osd_op_pre_omapgetkeys, soid.oid.name.c_str(), soid.snap.val, "???", 0);
          goto fail;
@@ -7375,7 +7395,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
          decode(max_return, bp);
          decode(filter_prefix, bp);
        }
-       catch (buffer::error& e) {
+       catch (ceph::buffer::error& e) {
          result = -EINVAL;
          tracepoint(osd, do_osd_op_pre_omapgetvals, soid.oid.name.c_str(), soid.snap.val, "???", 0, "???");
          goto fail;
@@ -7441,7 +7461,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
        try {
          decode(keys_to_get, bp);
        }
-       catch (buffer::error& e) {
+       catch (ceph::buffer::error& e) {
          result = -EINVAL;
          tracepoint(osd, do_osd_op_pre_omapgetvalsbykeys, soid.oid.name.c_str(), soid.snap.val, "???");
          goto fail;
@@ -7469,7 +7489,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
        try {
          decode(assertions, bp);
        }
-       catch (buffer::error& e) {
+       catch (ceph::buffer::error& e) {
          result = -EINVAL;
          tracepoint(osd, do_osd_op_pre_omap_cmp, soid.oid.name.c_str(), soid.snap.val, "???");
          goto fail;
@@ -7547,7 +7567,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
        try {
          decode_str_str_map_to_bl(bp, &to_set_bl);
        }
-       catch (buffer::error& e) {
+       catch (ceph::buffer::error& e) {
          result = -EINVAL;
          tracepoint(osd, do_osd_op_pre_omapsetvals, soid.oid.name.c_str(), soid.snap.val);
          goto fail;
@@ -7632,7 +7652,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
        try {
          decode_str_set_to_bl(bp, &to_rm_bl);
        }
-       catch (buffer::error& e) {
+       catch (ceph::buffer::error& e) {
          result = -EINVAL;
          tracepoint(osd, do_osd_op_pre_omaprmkeys, soid.oid.name.c_str(), soid.snap.val);
          goto fail;
@@ -7662,7 +7682,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
        try {
          decode(key_begin, bp);
          decode(key_end, bp);
-       } catch (buffer::error& e) {
+       } catch (ceph::buffer::error& e) {
          result = -EINVAL;
          goto fail;
        }
@@ -7714,7 +7734,7 @@ int PrimaryLogPG::do_osd_ops(OpContext *ctx, vector<OSDOp>& ops)
            have_truncate = true;
          }
        }
-       catch (buffer::error& e) {
+       catch (ceph::buffer::error& e) {
          result = -EINVAL;
          tracepoint(osd,
                     do_osd_op_pre_copy_from,
@@ -8779,7 +8799,7 @@ int PrimaryLogPG::do_copy_get(OpContext *ctx, bufferlist::const_iterator& bp,
     decode(cursor, bp);
     decode(out_max, bp);
   }
-  catch (buffer::error& e) {
+  catch (ceph::buffer::error& e) {
     result = -EINVAL;
     return result;
   }
@@ -11433,7 +11453,7 @@ SnapSetContext *PrimaryLogPG::get_snapset_context(
       bufferlist::const_iterator bvp = bv.begin();
       try {
        ssc->snapset.decode(bvp);
-      } catch (buffer::error& e) {
+      } catch (ceph::buffer::error& e) {
         dout(0) << __func__ << " Can't decode snapset: " << e << dendl;
        return NULL;
       }
@@ -14761,7 +14781,7 @@ void PrimaryLogPG::scrub_snapshot_metadata(
       try {
        oi = object_info_t(); // Initialize optional<> before decode into it
        oi->decode(bv);
-      } catch (buffer::error& e) {
+      } catch (ceph::buffer::error& e) {
        oi = std::nullopt;
        osd->clog->error() << mode << " " << info.pgid << " " << soid
                << " : can't decode '" << OI_ATTR << "' attr " << e.what();
@@ -14889,7 +14909,7 @@ void PrimaryLogPG::scrub_snapshot_metadata(
          snapset = SnapSet(); // Initialize optional<> before decoding into it
          decode(*snapset, blp);
           head_error.ss_bl.push_back(p->second.attrs[SS_ATTR]);
-        } catch (buffer::error& e) {
+        } catch (ceph::buffer::error& e) {
          snapset = std::nullopt;
           osd->clog->error() << mode << " " << info.pgid << " " << soid
                << " : can't decode '" << SS_ATTR << "' attr " << e.what();
index 7fc9f17e7b3a577afc63abcdab497a1610c22d2c..f7b2e32fc3ffc0c3a5f4e0c7ac7d9409cfa34a26 100644 (file)
@@ -88,7 +88,7 @@ public:
 
     version_t user_version; ///< The copy source's user version
     bool should_requeue;  ///< op should be requeued on cancel
-    vector<snapid_t> snaps;  ///< src's snaps (if clone)
+    std::vector<snapid_t> snaps;  ///< src's snaps (if clone)
     snapid_t snap_seq;       ///< src's snap_seq (if head)
     librados::snap_set_t snapset; ///< src snapset (if head)
     bool mirror_snapset;
@@ -96,9 +96,9 @@ public:
     uint32_t flags;    // object_copy_data_t::FLAG_*
     uint32_t source_data_digest, source_omap_digest;
     uint32_t data_digest, omap_digest;
-    mempool::osd_pglog::vector<pair<osd_reqid_t, version_t> > reqids; // [(reqid, user_version)]
-    mempool::osd_pglog::map<uint32_t, int> reqid_return_codes; // map reqids by index to error code
-    map<string, bufferlist> attrs; // xattrs
+    mempool::osd_pglog::vector<std::pair<osd_reqid_t, version_t> > reqids; // [(reqid, user_version)]
+    mempool::osd_pglog::map<uint32_t, int> reqid_return_codes; // std::map reqids by index to error code
+    std::map<std::string, ceph::buffer::list> attrs; // xattrs
     uint64_t truncate_seq;
     uint64_t truncate_size;
     bool is_data_digest() {
@@ -136,10 +136,10 @@ public:
     ceph_tid_t objecter_tid2;
 
     object_copy_cursor_t cursor;
-    map<string,bufferlist> attrs;
-    bufferlist data;
-    bufferlist omap_header;
-    bufferlist omap_data;
+    std::map<std::string,ceph::buffer::list> attrs;
+    ceph::buffer::list data;
+    ceph::buffer::list omap_header;
+    ceph::buffer::list omap_data;
     int rval;
 
     object_copy_cursor_t temp_cursor;
@@ -154,12 +154,12 @@ public:
     unsigned src_obj_fadvise_flags;
     unsigned dest_obj_fadvise_flags;
 
-    map<uint64_t, CopyOpRef> chunk_cops;
+    std::map<uint64_t, CopyOpRef> chunk_cops;
     int num_chunk;
     bool failed;
     uint64_t start_offset = 0;
     uint64_t last_offset = 0;
-    vector<OSDOp> chunk_ops;
+    std::vector<OSDOp> chunk_ops;
   
     CopyOp(CopyCallback *cb_, ObjectContextRef _obc, hobject_t s,
           object_locator_t l,
@@ -194,20 +194,20 @@ public:
   typedef boost::tuple<int, CopyResults*> CopyCallbackResults;
 
   friend class CopyFromCallback;
-  friend class CopyFromFinisher;
+  friend struct CopyFromFinisher;
   friend class PromoteCallback;
-  friend class PromoteFinisher;
+  friend struct PromoteFinisher;
 
   struct ProxyReadOp {
     OpRequestRef op;
     hobject_t soid;
     ceph_tid_t objecter_tid;
-    vector<OSDOp> &ops;
+    std::vector<OSDOp> &ops;
     version_t user_version;
     int data_offset;
     bool canceled;              ///< true if canceled
 
-    ProxyReadOp(OpRequestRef _op, hobject_t oid, vector<OSDOp>& _ops)
+    ProxyReadOp(OpRequestRef _op, hobject_t oid, std::vector<OSDOp>& _ops)
       : op(_op), soid(oid),
         objecter_tid(0), ops(_ops),
        user_version(0), data_offset(0),
@@ -220,14 +220,14 @@ public:
     OpRequestRef op;
     hobject_t soid;
     ceph_tid_t objecter_tid;
-    vector<OSDOp> &ops;
+    std::vector<OSDOp> &ops;
     version_t user_version;
     bool sent_reply;
     utime_t mtime;
     bool canceled;
     osd_reqid_t reqid;
 
-    ProxyWriteOp(OpRequestRef _op, hobject_t oid, vector<OSDOp>& _ops, osd_reqid_t _reqid)
+    ProxyWriteOp(OpRequestRef _op, hobject_t oid, std::vector<OSDOp>& _ops, osd_reqid_t _reqid)
       : ctx(NULL), op(_op), soid(oid),
         objecter_tid(0), ops(_ops),
        user_version(0), sent_reply(false),
@@ -239,7 +239,7 @@ public:
   struct FlushOp {
     ObjectContextRef obc;       ///< obc we are flushing
     OpRequestRef op;            ///< initiating op
-    list<OpRequestRef> dup_ops; ///< bandwagon jumpers
+    std::list<OpRequestRef> dup_ops; ///< bandwagon jumpers
     version_t flushed_version;  ///< user version we are flushing
     ceph_tid_t objecter_tid;    ///< copy-from request tid
     int rval;                   ///< copy-from result
@@ -247,8 +247,8 @@ public:
     bool removal;               ///< we are removing the backend object
     std::optional<std::function<void()>> on_flush; ///< callback, may be null
     // for chunked object
-    map<uint64_t, int> io_results; 
-    map<uint64_t, ceph_tid_t> io_tids; 
+    std::map<uint64_t, int> io_results; 
+    std::map<uint64_t, ceph_tid_t> io_tids; 
     uint64_t chunks;
 
     FlushOp()
@@ -267,7 +267,7 @@ public:
       : cb(cb), objecter_tid(tid) {}
   };
   typedef std::shared_ptr<ManifestOp> ManifestOpRef;
-  map<hobject_t, ManifestOpRef> manifest_ops;
+  std::map<hobject_t, ManifestOpRef> manifest_ops;
 
   boost::scoped_ptr<PGBackend> pgbackend;
   PGBackend *get_pgbackend() override {
@@ -307,7 +307,7 @@ public:
     const object_stat_sum_t &stat_diff,
     bool is_delete) override;
   void on_failed_pull(
-    const set<pg_shard_t> &from,
+    const std::set<pg_shard_t> &from,
     const hobject_t &soid,
     const eversion_t &version) override;
   void cancel_pull(const hobject_t &soid) override;
@@ -338,7 +338,7 @@ public:
                         OpRequestRef op) override {
     osd->store->queue_transaction(ch, std::move(t), op);
   }
-  void queue_transactions(vector<ObjectStore::Transaction>& tls,
+  void queue_transactions(std::vector<ObjectStore::Transaction>& tls,
                          OpRequestRef op) override {
     osd->store->queue_transactions(ch, tls, op, NULL);
   }
@@ -348,13 +348,13 @@ public:
   epoch_t get_last_peering_reset_epoch() const override {
     return get_last_peering_reset();
   }
-  const set<pg_shard_t> &get_acting_recovery_backfill_shards() const override {
+  const std::set<pg_shard_t> &get_acting_recovery_backfill_shards() const override {
     return get_acting_recovery_backfill();
   }
-  const set<pg_shard_t> &get_acting_shards() const override {
+  const std::set<pg_shard_t> &get_acting_shards() const override {
     return recovery_state.get_actingset();
   }
-  const set<pg_shard_t> &get_backfill_shards() const override {
+  const std::set<pg_shard_t> &get_backfill_shards() const override {
     return get_backfill_targets();
   }
 
@@ -362,15 +362,15 @@ public:
     return gen_prefix(out);
   }
 
-  const map<hobject_t, set<pg_shard_t>>
+  const std::map<hobject_t, std::set<pg_shard_t>>
     &get_missing_loc_shards() const override {
     return recovery_state.get_missing_loc().get_missing_locs();
   }
-  const map<pg_shard_t, pg_missing_t> &get_shard_missing() const override {
+  const std::map<pg_shard_t, pg_missing_t> &get_shard_missing() const override {
     return recovery_state.get_peer_missing();
   }
   using PGBackend::Listener::get_shard_missing;
-  const map<pg_shard_t, pg_info_t> &get_shard_info() const override {
+  const std::map<pg_shard_t, pg_info_t> &get_shard_info() const override {
     return recovery_state.get_peer_info();
   }
   using PGBackend::Listener::get_shard_info;  
@@ -401,7 +401,7 @@ public:
 
   ObjectContextRef get_obc(
     const hobject_t &hoid,
-    const map<string, bufferlist> &attrs) override {
+    const std::map<std::string, ceph::buffer::list> &attrs) override {
     return get_object_context(hoid, true, &attrs);
   }
 
@@ -444,7 +444,7 @@ public:
 
   void pgb_set_object_snap_mapping(
     const hobject_t &soid,
-    const set<snapid_t> &snaps,
+    const std::set<snapid_t> &snaps,
     ObjectStore::Transaction *t) override {
     return update_object_snap_mapping(t, soid, snaps);
   }
@@ -455,7 +455,7 @@ public:
   }
 
   void log_operation(
-    vector<pg_log_entry_t>&& logv,
+    std::vector<pg_log_entry_t>&& logv,
     const std::optional<pg_hit_set_history_t> &hset_history,
     const eversion_t &trim_to,
     const eversion_t &roll_forward_to,
@@ -486,7 +486,7 @@ public:
   }
 
   void replica_clear_repop_obc(
-    const vector<pg_log_entry_t> &logv,
+    const std::vector<pg_log_entry_t> &logv,
     ObjectStore::Transaction &t);
 
   void op_applied(const eversion_t &applied_version) override;
@@ -575,7 +575,7 @@ public:
   };
   void complete_disconnect_watches(
     ObjectContextRef obc,
-    const list<watch_disconnect_t> &to_disconnect);
+    const std::list<watch_disconnect_t> &to_disconnect);
 
   struct OpFinisher {
     virtual ~OpFinisher() {
@@ -590,7 +590,7 @@ public:
   struct OpContext {
     OpRequestRef op;
     osd_reqid_t reqid;
-    vector<OSDOp> *ops;
+    std::vector<OSDOp> *ops;
 
     const ObjectState *obs; // Old objectstate
     const SnapSet *snapset; // Old snapset
@@ -604,26 +604,26 @@ public:
     bool user_modify;     // user-visible modification
     bool undirty;         // user explicitly un-dirtying this object
     bool cache_evict;     ///< true if this is a cache eviction
-    bool ignore_cache;    ///< true if IGNORE_CACHE flag is set
+    bool ignore_cache;    ///< true if IGNORE_CACHE flag is std::set
     bool ignore_log_op_stats;  // don't log op stats
     bool update_log_only; ///< this is a write that returned an error - just record in pg log for dup detection
     ObjectCleanRegions clean_regions;
 
     // side effects
-    list<pair<watch_info_t,bool> > watch_connects; ///< new watch + will_ping flag
-    list<watch_disconnect_t> watch_disconnects; ///< old watch + send_discon
-    list<notify_info_t> notifies;
+    std::list<std::pair<watch_info_t,bool> > watch_connects; ///< new watch + will_ping flag
+    std::list<watch_disconnect_t> watch_disconnects; ///< old watch + send_discon
+    std::list<notify_info_t> notifies;
     struct NotifyAck {
       std::optional<uint64_t> watch_cookie;
       uint64_t notify_id;
-      bufferlist reply_bl;
+      ceph::buffer::list reply_bl;
       explicit NotifyAck(uint64_t notify_id) : notify_id(notify_id) {}
-      NotifyAck(uint64_t notify_id, uint64_t cookie, bufferlist& rbl)
+      NotifyAck(uint64_t notify_id, uint64_t cookie, ceph::buffer::list& rbl)
        : watch_cookie(cookie), notify_id(notify_id) {
        reply_bl.claim(rbl);
       }
     };
-    list<NotifyAck> notify_acks;
+    std::list<NotifyAck> notify_acks;
 
     uint64_t bytes_written, bytes_read;
 
@@ -638,7 +638,7 @@ public:
     int processed_subop_count = 0;
 
     PGTransactionUPtr op_t;
-    vector<pg_log_entry_t> log;
+    std::vector<pg_log_entry_t> log;
     std::optional<pg_hit_set_history_t> updated_hset_history;
 
     interval_set<uint64_t> modified_ranges;
@@ -656,15 +656,15 @@ public:
     int num_read;    ///< count read ops
     int num_write;   ///< count update ops
 
-    mempool::osd_pglog::vector<pair<osd_reqid_t, version_t> > extra_reqids;
+    mempool::osd_pglog::vector<std::pair<osd_reqid_t, version_t> > extra_reqids;
     mempool::osd_pglog::map<uint32_t, int> extra_reqid_return_codes;
 
     hobject_t new_temp_oid, discard_temp_oid;  ///< temp objects we should start/stop tracking
 
-    list<std::function<void()>> on_applied;
-    list<std::function<void()>> on_committed;
-    list<std::function<void()>> on_finish;
-    list<std::function<void()>> on_success;
+    std::list<std::function<void()>> on_applied;
+    std::list<std::function<void()>> on_committed;
+    std::list<std::function<void()>> on_finish;
+    std::list<std::function<void()>> on_success;
     template <typename F>
     void register_on_finish(F &&f) {
       on_finish.emplace_back(std::forward<F>(f));
@@ -685,8 +685,8 @@ public:
     bool sent_reply = false;
 
     // pending async reads <off, len, op_flags> -> <outbl, outr>
-    list<pair<boost::tuple<uint64_t, uint64_t, unsigned>,
-             pair<bufferlist*, Context*> > > pending_async_reads;
+    std::list<std::pair<boost::tuple<uint64_t, uint64_t, unsigned>,
+             std::pair<ceph::buffer::list*, Context*> > > pending_async_reads;
     int inflightreads;
     friend struct OnReadComplete;
     void start_async_reads(PrimaryLogPG *pg);
@@ -703,7 +703,7 @@ public:
     OpContext(const OpContext& other);
     const OpContext& operator=(const OpContext& other);
 
-    OpContext(OpRequestRef _op, osd_reqid_t _reqid, vector<OSDOp>* _ops,
+    OpContext(OpRequestRef _op, osd_reqid_t _reqid, std::vector<OSDOp>* _ops,
              ObjectContextRef& obc,
              PrimaryLogPG *_pg) :
       op(_op), reqid(_reqid), ops(_ops),
@@ -727,7 +727,7 @@ public:
       }
     }
     OpContext(OpRequestRef _op, osd_reqid_t _reqid,
-              vector<OSDOp>* _ops, PrimaryLogPG *_pg) :
+              std::vector<OSDOp>* _ops, PrimaryLogPG *_pg) :
       op(_op), reqid(_reqid), ops(_ops), obs(NULL), snapset(0),
       modify(false), user_modify(false), undirty(false), cache_evict(false),
       ignore_cache(false), ignore_log_op_stats(false), update_log_only(false),
@@ -749,8 +749,8 @@ public:
       ceph_assert(!op_t);
       if (reply)
        reply->put();
-      for (list<pair<boost::tuple<uint64_t, uint64_t, unsigned>,
-                    pair<bufferlist*, Context*> > >::iterator i =
+      for (std::list<std::pair<boost::tuple<uint64_t, uint64_t, unsigned>,
+                    std::pair<ceph::buffer::list*, Context*> > >::iterator i =
             pending_async_reads.begin();
           i != pending_async_reads.end();
           pending_async_reads.erase(i++)) {
@@ -791,9 +791,9 @@ public:
 
     ObcLockManager lock_manager;
 
-    list<std::function<void()>> on_committed;
-    list<std::function<void()>> on_success;
-    list<std::function<void()>> on_finish;
+    std::list<std::function<void()>> on_committed;
+    std::list<std::function<void()>> on_success;
+    std::list<std::function<void()>> on_finish;
     
     RepGather(
       OpContext *c, ceph_tid_t rt,
@@ -907,7 +907,7 @@ protected:
    */
   void release_object_locks(
     ObcLockManager &lock_manager) {
-    list<pair<ObjectContextRef, list<OpRequestRef> > > to_req;
+    std::list<std::pair<ObjectContextRef, std::list<OpRequestRef> > > to_req;
     bool requeue_recovery = false;
     bool requeue_snaptrim = false;
     lock_manager.put_locks(
@@ -985,10 +985,10 @@ protected:
     int r = 0);
   struct LogUpdateCtx {
     boost::intrusive_ptr<RepGather> repop;
-    set<pg_shard_t> waiting_on;
+    std::set<pg_shard_t> waiting_on;
   };
   void cancel_log_updates();
-  map<ceph_tid_t, LogUpdateCtx> log_entry_update_waiting_on;
+  std::map<ceph_tid_t, LogUpdateCtx> log_entry_update_waiting_on;
 
 
   // hot/cold tracking
@@ -1046,19 +1046,19 @@ protected:
 
   // projected object info
   SharedLRU<hobject_t, ObjectContext> object_contexts;
-  // map from oid.snapdir() to SnapSetContext *
-  map<hobject_t, SnapSetContext*> snapset_contexts;
+  // std::map from oid.snapdir() to SnapSetContext *
+  std::map<hobject_t, SnapSetContext*> snapset_contexts;
   ceph::mutex snapset_contexts_lock =
     ceph::make_mutex("PrimaryLogPG::snapset_contexts_lock");
 
   // debug order that client ops are applied
-  map<hobject_t, map<client_t, ceph_tid_t>> debug_op_order;
+  std::map<hobject_t, std::map<client_t, ceph_tid_t>> debug_op_order;
 
   void populate_obc_watchers(ObjectContextRef obc);
   void check_blacklisted_obc_watchers(ObjectContextRef obc);
   void check_blacklisted_watchers() override;
-  void get_watchers(list<obj_watch_item_t> *ls) override;
-  void get_obc_watchers(ObjectContextRef obc, list<obj_watch_item_t> &pg_watchers);
+  void get_watchers(std::list<obj_watch_item_t> *ls) override;
+  void get_obc_watchers(ObjectContextRef obc, std::list<obj_watch_item_t> &pg_watchers);
 public:
   void handle_watch_timeout(WatchRef watch);
 protected:
@@ -1067,7 +1067,7 @@ protected:
   ObjectContextRef get_object_context(
     const hobject_t& soid,
     bool can_create,
-    const map<string, bufferlist> *attrs = 0
+    const std::map<std::string, ceph::buffer::list> *attrs = 0
     );
 
   void context_registry_on_change();
@@ -1087,7 +1087,7 @@ protected:
   SnapSetContext *get_snapset_context(
     const hobject_t& oid,
     bool can_create,
-    const map<string, bufferlist> *attrs = 0,
+    const std::map<std::string, ceph::buffer::list> *attrs = 0,
     bool oid_existed = true //indicate this oid whether exsited in backend
     );
   void register_snapset_context(SnapSetContext *ssc) {
@@ -1104,7 +1104,7 @@ protected:
   }
   void put_snapset_context(SnapSetContext *ssc);
 
-  map<hobject_t, ObjectContextRef> recovering;
+  std::map<hobject_t, ObjectContextRef> recovering;
 
   /*
    * Backfill
@@ -1120,12 +1120,12 @@ protected:
    *   - are not included in pg stats (yet)
    *   - have their stats in pending_backfill_updates on the primary
    */
-  set<hobject_t> backfills_in_flight;
-  map<hobject_t, pg_stat_t> pending_backfill_updates;
+  std::set<hobject_t> backfills_in_flight;
+  std::map<hobject_t, pg_stat_t> pending_backfill_updates;
 
-  void dump_recovery_info(Formatter *f) const override {
+  void dump_recovery_info(ceph::Formatter *f) const override {
     f->open_array_section("waiting_on_backfill");
-    for (set<pg_shard_t>::const_iterator p = waiting_on_backfill.begin();
+    for (std::set<pg_shard_t>::const_iterator p = waiting_on_backfill.begin();
         p != waiting_on_backfill.end(); ++p)
       f->dump_stream("osd") << *p;
     f->close_section();
@@ -1137,7 +1137,7 @@ protected:
     }
     {
       f->open_array_section("peer_backfill_info");
-      for (map<pg_shard_t, BackfillInterval>::const_iterator pbi =
+      for (std::map<pg_shard_t, BackfillInterval>::const_iterator pbi =
             peer_backfill_info.begin();
           pbi != peer_backfill_info.end(); ++pbi) {
         f->dump_stream("osd") << pbi->first;
@@ -1149,7 +1149,7 @@ protected:
     }
     {
       f->open_array_section("backfills_in_flight");
-      for (set<hobject_t>::const_iterator i = backfills_in_flight.begin();
+      for (std::set<hobject_t>::const_iterator i = backfills_in_flight.begin();
           i != backfills_in_flight.end();
           ++i) {
        f->dump_stream("object") << *i;
@@ -1158,7 +1158,7 @@ protected:
     }
     {
       f->open_array_section("recovering");
-      for (map<hobject_t, ObjectContextRef>::const_iterator i = recovering.begin();
+      for (std::map<hobject_t, ObjectContextRef>::const_iterator i = recovering.begin();
           i != recovering.end();
           ++i) {
        f->dump_stream("object") << i->first;
@@ -1281,7 +1281,7 @@ protected:
   void do_cache_redirect(OpRequestRef op);
   /**
    * This function attempts to start a promote.  Either it succeeds,
-   * or places op on a wait list.  If op is null, failure means that
+   * or places op on a wait std::list.  If op is null, failure means that
    * this is a noop.  If a future user wants to be able to distinguish
    * these cases, a return value should be added.
    */
@@ -1294,7 +1294,7 @@ protected:
     );
 
   int prepare_transaction(OpContext *ctx);
-  list<pair<OpRequestRef, OpContext*> > in_progress_async_reads;
+  std::list<std::pair<OpRequestRef, OpContext*> > in_progress_async_reads;
   void complete_read_ctx(int result, OpContext *ctx);
   
   // pg on-disk content
@@ -1312,7 +1312,7 @@ protected:
   hobject_t earliest_peer_backfill() const;
   bool all_peer_done() const;
   /**
-   * @param work_started will be set to true if recover_backfill got anywhere
+   * @param work_started will be std::set to true if recover_backfill got anywhere
    * @returns the number of operations started
    */
   uint64_t recover_backfill(uint64_t max, ThreadPool::TPHandle &handle,
@@ -1324,7 +1324,7 @@ protected:
    * @min return at least this many items, unless we are done
    * @max return no more than this many items
    * @bi.begin first item should be >= this value
-   * @bi [out] resulting map of objects to eversion_t's
+   * @bi [out] resulting std::map of objects to eversion_t's
    */
   void scan_range(
     int min, int max, BackfillInterval *bi,
@@ -1339,7 +1339,7 @@ protected:
 
   int prep_backfill_object_push(
     hobject_t oid, eversion_t v, ObjectContextRef obc,
-    vector<pg_shard_t> peers,
+    std::vector<pg_shard_t> peers,
     PGBackend::RecoveryHandle *h);
   void send_remove_op(const hobject_t& oid, eversion_t v, pg_shard_t peer);
 
@@ -1354,9 +1354,9 @@ protected:
   void recover_got(hobject_t oid, eversion_t v);
 
   // -- copyfrom --
-  map<hobject_t, CopyOpRef> copy_ops;
+  std::map<hobject_t, CopyOpRef> copy_ops;
 
-  int do_copy_get(OpContext *ctx, bufferlist::const_iterator& bp, OSDOp& op,
+  int do_copy_get(OpContext *ctx, ceph::buffer::list::const_iterator& bp, OSDOp& op,
                  ObjectContextRef& obc);
   int finish_copy_get();
 
@@ -1391,13 +1391,13 @@ protected:
   void _copy_some(ObjectContextRef obc, CopyOpRef cop);
   void finish_copyfrom(CopyFromCallback *cb);
   void finish_promote(int r, CopyResults *results, ObjectContextRef obc);
-  void cancel_copy(CopyOpRef cop, bool requeue, vector<ceph_tid_t> *tids);
-  void cancel_copy_ops(bool requeue, vector<ceph_tid_t> *tids);
+  void cancel_copy(CopyOpRef cop, bool requeue, std::vector<ceph_tid_t> *tids);
+  void cancel_copy_ops(bool requeue, std::vector<ceph_tid_t> *tids);
 
   friend struct C_Copyfrom;
 
   // -- flush --
-  map<hobject_t, FlushOpRef> flush_ops;
+  std::map<hobject_t, FlushOpRef> flush_ops;
 
   /// start_flush takes ownership of on_flush iff ret == -EINPROGRESS
   int start_flush(
@@ -1406,8 +1406,8 @@ protected:
     std::optional<std::function<void()>> &&on_flush);
   void finish_flush(hobject_t oid, ceph_tid_t tid, int r);
   int try_flush_mark_clean(FlushOpRef fop);
-  void cancel_flush(FlushOpRef fop, bool requeue, vector<ceph_tid_t> *tids);
-  void cancel_flush_ops(bool requeue, vector<ceph_tid_t> *tids);
+  void cancel_flush(FlushOpRef fop, bool requeue, std::vector<ceph_tid_t> *tids);
+  void cancel_flush_ops(bool requeue, std::vector<ceph_tid_t> *tids);
 
   /// @return false if clone is has been evicted
   bool is_present_clone(hobject_t coid);
@@ -1420,7 +1420,7 @@ protected:
   void scrub_snapshot_metadata(
     ScrubMap &map,
     const std::map<hobject_t,
-                   pair<std::optional<uint32_t>,
+                   std::pair<std::optional<uint32_t>,
                         std::optional<uint32_t>>> &missing_digest) override;
   void _scrub_clear_state() override;
   void _scrub_finish() override;
@@ -1430,21 +1430,21 @@ protected:
                    unsigned split_bits) override;
   void apply_and_flush_repops(bool requeue);
 
-  int do_xattr_cmp_u64(int op, __u64 v1, bufferlist& xattr);
-  int do_xattr_cmp_str(int op, string& v1s, bufferlist& xattr);
+  int do_xattr_cmp_u64(int op, __u64 v1, ceph::buffer::list& xattr);
+  int do_xattr_cmp_str(int op, std::string& v1s, ceph::buffer::list& xattr);
 
   // -- checksum --
-  int do_checksum(OpContext *ctx, OSDOp& osd_op, bufferlist::const_iterator *bl_it);
+  int do_checksum(OpContext *ctx, OSDOp& osd_op, ceph::buffer::list::const_iterator *bl_it);
   int finish_checksum(OSDOp& osd_op, Checksummer::CSumType csum_type,
-                      bufferlist::const_iterator *init_value_bl_it,
-                      const bufferlist &read_bl);
+                      ceph::buffer::list::const_iterator *init_value_bl_it,
+                      const ceph::buffer::list &read_bl);
 
-  friend class C_ChecksumRead;
+  friend struct C_ChecksumRead;
 
   int do_extent_cmp(OpContext *ctx, OSDOp& osd_op);
-  int finish_extent_cmp(OSDOp& osd_op, const bufferlist &read_bl);
+  int finish_extent_cmp(OSDOp& osd_op, const ceph::buffer::list &read_bl);
 
-  friend class C_ExtentCmpRead;
+  friend struct C_ExtentCmpRead;
 
   int do_read(OpContext *ctx, OSDOp& osd_op);
   int do_sparse_read(OpContext *ctx, OSDOp& osd_op);
@@ -1453,27 +1453,27 @@ protected:
   bool pgls_filter(const PGLSFilter& filter, const hobject_t& sobj);
 
   std::pair<int, std::unique_ptr<const PGLSFilter>> get_pgls_filter(
-    bufferlist::const_iterator& iter);
+    ceph::buffer::list::const_iterator& iter);
 
-  map<hobject_t, list<OpRequestRef>> in_progress_proxy_ops;
+  std::map<hobject_t, std::list<OpRequestRef>> in_progress_proxy_ops;
   void kick_proxy_ops_blocked(hobject_t& soid);
-  void cancel_proxy_ops(bool requeue, vector<ceph_tid_t> *tids);
+  void cancel_proxy_ops(bool requeue, std::vector<ceph_tid_t> *tids);
 
   // -- proxyread --
-  map<ceph_tid_t, ProxyReadOpRef> proxyread_ops;
+  std::map<ceph_tid_t, ProxyReadOpRef> proxyread_ops;
 
   void do_proxy_read(OpRequestRef op, ObjectContextRef obc = NULL);
   void finish_proxy_read(hobject_t oid, ceph_tid_t tid, int r);
-  void cancel_proxy_read(ProxyReadOpRef prdop, vector<ceph_tid_t> *tids);
+  void cancel_proxy_read(ProxyReadOpRef prdop, std::vector<ceph_tid_t> *tids);
 
   friend struct C_ProxyRead;
 
   // -- proxywrite --
-  map<ceph_tid_t, ProxyWriteOpRef> proxywrite_ops;
+  std::map<ceph_tid_t, ProxyWriteOpRef> proxywrite_ops;
 
   void do_proxy_write(OpRequestRef op, ObjectContextRef obc = NULL);
   void finish_proxy_write(hobject_t oid, ceph_tid_t tid, int r);
-  void cancel_proxy_write(ProxyWriteOpRef pwop, vector<ceph_tid_t> *tids);
+  void cancel_proxy_write(ProxyWriteOpRef pwop, std::vector<ceph_tid_t> *tids);
 
   friend struct C_ProxyWrite_Commit;
 
@@ -1496,31 +1496,31 @@ protected:
                             uint64_t last_offset);
   void handle_manifest_flush(hobject_t oid, ceph_tid_t tid, int r,
                             uint64_t offset, uint64_t last_offset, epoch_t lpr);
-  void cancel_manifest_ops(bool requeue, vector<ceph_tid_t> *tids);
+  void cancel_manifest_ops(bool requeue, std::vector<ceph_tid_t> *tids);
   void refcount_manifest(ObjectContextRef obc, object_locator_t oloc, hobject_t soid,
                          SnapContext snapc, bool get, RefCountCallback *cb, uint64_t offset);
 
   friend struct C_ProxyChunkRead;
   friend class PromoteManifestCallback;
-  friend class C_CopyChunk;
+  friend struct C_CopyChunk;
   friend struct C_ManifestFlush;
   friend struct RefCountCallback;
 
 public:
   PrimaryLogPG(OSDService *o, OSDMapRef curmap,
               const PGPool &_pool,
-              const map<string,string>& ec_profile,
+              const std::map<std::string,std::string>& ec_profile,
               spg_t p);
   ~PrimaryLogPG() override {}
 
   void do_command(
-    const string_view& prefix,
+    const std::string_view& prefix,
     const cmdmap_t& cmdmap,
-    const bufferlist& idata,
-    std::function<void(int,const std::string&,bufferlist&)> on_finish) override;
+    const ceph::buffer::list& idata,
+    std::function<void(int,const std::string&,ceph::buffer::list&)> on_finish) override;
 
-  void clear_cache();
-  int get_cache_obj_count() {
+  void clear_cache() override;
+  int get_cache_obj_count() override {
     return object_contexts.get_count();
   }
   unsigned get_pg_shard() const {
@@ -1547,12 +1547,12 @@ public:
   void snap_trimmer(epoch_t e) override;
   void kick_snap_trim() override;
   void snap_trimmer_scrub_complete() override;
-  int do_osd_ops(OpContext *ctx, vector<OSDOp>& ops);
+  int do_osd_ops(OpContext *ctx, std::vector<OSDOp>& ops);
 
-  int _get_tmap(OpContext *ctx, bufferlist *header, bufferlist *vals);
+  int _get_tmap(OpContext *ctx, ceph::buffer::list *header, ceph::buffer::list *vals);
   int do_tmap2omap(OpContext *ctx, unsigned flags);
-  int do_tmapup(OpContext *ctx, bufferlist::const_iterator& bp, OSDOp& osd_op);
-  int do_tmapup_slow(OpContext *ctx, bufferlist::const_iterator& bp, OSDOp& osd_op, bufferlist& bl);
+  int do_tmapup(OpContext *ctx, ceph::buffer::list::const_iterator& bp, OSDOp& osd_op);
+  int do_tmapup_slow(OpContext *ctx, ceph::buffer::list::const_iterator& bp, OSDOp& osd_op, ceph::buffer::list& bl);
 
   void do_osd_op_effects(OpContext *ctx, const ConnectionRef& conn);
 private:
@@ -1585,7 +1585,7 @@ private:
     const char *mode,
     bool allow_incomplete_clones,
     std::optional<snapid_t> target,
-    vector<snapid_t>::reverse_iterator *curclone,
+    std::vector<snapid_t>::reverse_iterator *curclone,
     inconsistent_snapset_wrapper &snap_error);
 
 public:
@@ -1660,7 +1660,7 @@ private:
       boost::statechart::transition< Reset, NotTrimming >
       > reactions;
 
-    set<hobject_t> in_flight;
+    std::set<hobject_t> in_flight;
     snapid_t snap_to_trim;
 
     explicit Trimming(my_context ctx)
@@ -1901,13 +1901,13 @@ public:
   bool is_backfill_target(pg_shard_t osd) const {
     return recovery_state.is_backfill_target(osd);
   }
-  const set<pg_shard_t> &get_backfill_targets() const {
+  const std::set<pg_shard_t> &get_backfill_targets() const {
     return recovery_state.get_backfill_targets();
   }
   bool is_async_recovery_target(pg_shard_t peer) const {
     return recovery_state.is_async_recovery_target(peer);
   }
-  const set<pg_shard_t> &get_async_recovery_targets() const {
+  const std::set<pg_shard_t> &get_async_recovery_targets() const {
     return recovery_state.get_async_recovery_targets();
   }
   bool is_degraded_or_backfilling_object(const hobject_t& oid);
@@ -1930,7 +1930,7 @@ public:
 
   void mark_all_unfound_lost(
     int what,
-    std::function<void(int,const std::string&,bufferlist&)> on_finish);
+    std::function<void(int,const std::string&,ceph::buffer::list&)> on_finish);
   eversion_t pick_newest_available(const hobject_t& oid);
 
   void do_update_log_missing(
@@ -1957,23 +1957,23 @@ public:
   void setattr_maybe_cache(
     ObjectContextRef obc,
     PGTransaction *t,
-    const string &key,
-    bufferlist &val);
+    const std::string &key,
+    ceph::buffer::list &val);
   void setattrs_maybe_cache(
     ObjectContextRef obc,
     PGTransaction *t,
-    map<string, bufferlist> &attrs);
+    std::map<std::string, ceph::buffer::list> &attrs);
   void rmattr_maybe_cache(
     ObjectContextRef obc,
     PGTransaction *t,
-    const string &key);
+    const std::string &key);
   int getattr_maybe_cache(
     ObjectContextRef obc,
-    const string &key,
-    bufferlist *val);
+    const std::string &key,
+    ceph::buffer::list *val);
   int getattrs_maybe_cache(
     ObjectContextRef obc,
-    map<string, bufferlist> *out);
+    std::map<std::string, ceph::buffer::list> *out);
 
 public:
   void set_dynamic_perf_stats_queries(
index dfdadfa5202a388499b48e7ebdb07b11e3de8bfa..e831101aa87ad164991b6d45580519138c59ac04 100644 (file)
@@ -33,6 +33,21 @@ static ostream& _prefix(std::ostream *_dout, ReplicatedBackend *pgb) {
   return pgb->get_parent()->gen_dbg_prefix(*_dout);
 }
 
+using std::list;
+using std::make_pair;
+using std::map;
+using std::ostringstream;
+using std::set;
+using std::pair;
+using std::string;
+using std::unique_ptr;
+using std::vector;
+
+using ceph::bufferhash;
+using ceph::bufferlist;
+using ceph::decode;
+using ceph::encode;
+
 namespace {
 class PG_SendMessageOnConn: public Context {
   PGBackend::Listener *pg;
@@ -1360,8 +1375,8 @@ void ReplicatedBackend::prepare_pull(
     // probably because user feed a wrong pullee
     p = q->second.begin();
     std::advance(p,
-                 util::generate_random_number<int>(0,
-                                                   q->second.size() - 1));
+                 ceph::util::generate_random_number<int>(0,
+                                                        q->second.size() - 1));
   }
   ceph_assert(get_osdmap()->is_up(p->osd));
   pg_shard_t fromshard = *p;
index 83ba03d7f9e19c8c3d357e5c1815c3d85f25afe9..f4b5063579a9e9237250774a1b545a14c8198a0f 100644 (file)
@@ -20,8 +20,8 @@
 struct C_ReplicatedBackend_OnPullComplete;
 class ReplicatedBackend : public PGBackend {
   struct RPGHandle : public PGBackend::RecoveryHandle {
-    map<pg_shard_t, vector<PushOp> > pushes;
-    map<pg_shard_t, vector<PullOp> > pulls;
+    std::map<pg_shard_t, std::vector<PushOp> > pushes;
+    std::map<pg_shard_t, std::vector<PullOp> > pulls;
   };
   friend struct C_ReplicatedBackend_OnPullComplete;
 public:
@@ -68,7 +68,7 @@ public:
 
   class RPCRecPred : public IsPGRecoverablePredicate {
   public:
-    bool operator()(const set<pg_shard_t> &have) const override {
+    bool operator()(const std::set<pg_shard_t> &have) const override {
       return !have.empty();
     }
   };
@@ -80,7 +80,7 @@ public:
     pg_shard_t whoami;
   public:
     explicit RPCReadPred(pg_shard_t whoami) : whoami(whoami) {}
-    bool operator()(const set<pg_shard_t> &have) const override {
+    bool operator()(const std::set<pg_shard_t> &have) const override {
       return have.count(whoami);
     }
   };
@@ -88,17 +88,17 @@ public:
     return new RPCReadPred(get_parent()->whoami_shard());
   }
 
-  void dump_recovery_info(Formatter *f) const override {
+  void dump_recovery_info(ceph::Formatter *f) const override {
     {
       f->open_array_section("pull_from_peer");
-      for (map<pg_shard_t, set<hobject_t> >::const_iterator i = pull_from_peer.begin();
+      for (std::map<pg_shard_t, std::set<hobject_t> >::const_iterator i = pull_from_peer.begin();
           i != pull_from_peer.end();
           ++i) {
        f->open_object_section("pulling_from");
        f->dump_stream("pull_from") << i->first;
        {
          f->open_array_section("pulls");
-         for (set<hobject_t>::const_iterator j = i->second.begin();
+         for (std::set<hobject_t>::const_iterator j = i->second.begin();
               j != i->second.end();
               ++j) {
            f->open_object_section("pull_info");
@@ -114,7 +114,7 @@ public:
     }
     {
       f->open_array_section("pushing");
-      for (map<hobject_t, map<pg_shard_t, PushInfo>>::const_iterator i =
+      for (std::map<hobject_t, std::map<pg_shard_t, PushInfo>>::const_iterator i =
             pushing.begin();
           i != pushing.end();
           ++i) {
@@ -122,7 +122,7 @@ public:
        f->dump_stream("pushing") << i->first;
        {
          f->open_array_section("pushing_to");
-         for (map<pg_shard_t, PushInfo>::const_iterator j = i->second.begin();
+         for (std::map<pg_shard_t, PushInfo>::const_iterator j = i->second.begin();
               j != i->second.end();
               ++j) {
            f->open_object_section("push_progress");
@@ -147,18 +147,18 @@ public:
     uint64_t off,
     uint64_t len,
     uint32_t op_flags,
-    bufferlist *bl) override;
+    ceph::buffer::list *bl) override;
 
   int objects_readv_sync(
     const hobject_t &hoid,
-    map<uint64_t, uint64_t>&& m,
+    std::map<uint64_t, uint64_t>&& m,
     uint32_t op_flags,
-    bufferlist *bl) override;
+    ceph::buffer::list *bl) override;
 
   void objects_read_async(
     const hobject_t &hoid,
-    const list<pair<boost::tuple<uint64_t, uint64_t, uint32_t>,
-              pair<bufferlist*, Context*> > > &to_read,
+    const std::list<std::pair<boost::tuple<uint64_t, uint64_t, uint32_t>,
+              std::pair<ceph::buffer::list*, Context*> > > &to_read,
                Context *on_complete,
                bool fast_read = false) override;
 
@@ -171,7 +171,7 @@ private:
     object_stat_sum_t stat;
     ObcLockManager lock_manager;
 
-    void dump(Formatter *f) const {
+    void dump(ceph::Formatter *f) const {
       {
        f->open_object_section("recovery_progress");
        recovery_progress.dump(f);
@@ -184,7 +184,7 @@ private:
       }
     }
   };
-  map<hobject_t, map<pg_shard_t, PushInfo>> pushing;
+  std::map<hobject_t, std::map<pg_shard_t, PushInfo>> pushing;
 
   // pull
   struct PullInfo {
@@ -198,7 +198,7 @@ private:
     bool cache_dont_need;
     ObcLockManager lock_manager;
 
-    void dump(Formatter *f) const {
+    void dump(ceph::Formatter *f) const {
       {
        f->open_object_section("recovery_progress");
        recovery_progress.dump(f);
@@ -216,15 +216,15 @@ private:
     }
   };
 
-  map<hobject_t, PullInfo> pulling;
+  std::map<hobject_t, PullInfo> pulling;
 
   // Reverse mapping from osd peer to objects being pulled from that peer
-  map<pg_shard_t, set<hobject_t> > pull_from_peer;
+  std::map<pg_shard_t, std::set<hobject_t> > pull_from_peer;
   void clear_pull(
-    map<hobject_t, PullInfo>::iterator piter,
+    std::map<hobject_t, PullInfo>::iterator piter,
     bool clear_pull_from_peer = true);
   void clear_pull_from(
-    map<hobject_t, PullInfo>::iterator piter);
+    std::map<hobject_t, PullInfo>::iterator piter);
 
   void _do_push(OpRequestRef op);
   void _do_pull_response(OpRequestRef op);
@@ -247,23 +247,23 @@ private:
   };
   bool handle_pull_response(
     pg_shard_t from, const PushOp &op, PullOp *response,
-    list<pull_complete_info> *to_continue,
+    std::list<pull_complete_info> *to_continue,
     ObjectStore::Transaction *t);
   void handle_push(pg_shard_t from, const PushOp &op, PushReplyOp *response,
                   ObjectStore::Transaction *t, bool is_repair);
 
   static void trim_pushed_data(const interval_set<uint64_t> &copy_subset,
                               const interval_set<uint64_t> &intervals_received,
-                              bufferlist data_received,
+                              ceph::buffer::list data_received,
                               interval_set<uint64_t> *intervals_usable,
-                              bufferlist *data_usable);
+                              ceph::buffer::list *data_usable);
   void _failed_pull(pg_shard_t from, const hobject_t &soid);
 
-  void send_pushes(int prio, map<pg_shard_t, vector<PushOp> > &pushes);
+  void send_pushes(int prio, std::map<pg_shard_t, std::vector<PushOp> > &pushes);
   void prep_push_op_blank(const hobject_t& soid, PushOp *op);
   void send_pulls(
     int priority,
-    map<pg_shard_t, vector<PullOp> > &pulls);
+    std::map<pg_shard_t, std::vector<PullOp> > &pulls);
 
   int build_push_op(const ObjectRecoveryInfo &recovery_info,
                    const ObjectRecoveryProgress &progress,
@@ -278,10 +278,10 @@ private:
                        bool cache_dont_need,
                        interval_set<uint64_t> &data_zeros,
                        const interval_set<uint64_t> &intervals_included,
-                       bufferlist data_included,
-                       bufferlist omap_header,
-                       const map<string, bufferlist> &attrs,
-                       const map<string, bufferlist> &omap_entries,
+                       ceph::buffer::list data_included,
+                       ceph::buffer::list omap_header,
+                       const std::map<std::string, ceph::buffer::list> &attrs,
+                       const std::map<std::string, ceph::buffer::list> &omap_entries,
                        ObjectStore::Transaction *t);
   void submit_push_complete(const ObjectRecoveryInfo &recovery_info,
                            ObjectStore::Transaction *t);
@@ -290,7 +290,7 @@ private:
     SnapSet& snapset, const hobject_t& poid, const pg_missing_t& missing,
     const hobject_t &last_backfill,
     interval_set<uint64_t>& data_subset,
-    map<hobject_t, interval_set<uint64_t>>& clone_subsets,
+    std::map<hobject_t, interval_set<uint64_t>>& clone_subsets,
     ObcLockManager &lock_manager);
   void prepare_pull(
     eversion_t v,
@@ -314,7 +314,7 @@ private:
     const hobject_t& soid, pg_shard_t peer,
     eversion_t version,
     interval_set<uint64_t> &data_subset,
-    map<hobject_t, interval_set<uint64_t>>& clone_subsets,
+    std::map<hobject_t, interval_set<uint64_t>>& clone_subsets,
     PushOp *op,
     bool cache,
     ObcLockManager &&lock_manager);
@@ -323,7 +323,7 @@ private:
     const pg_missing_t& missing,
     const hobject_t &last_backfill,
     interval_set<uint64_t>& data_subset,
-    map<hobject_t, interval_set<uint64_t>>& clone_subsets,
+    std::map<hobject_t, interval_set<uint64_t>>& clone_subsets,
     ObcLockManager &lock_manager);
   ObjectRecoveryInfo recalc_subsets(
     const ObjectRecoveryInfo& recovery_info,
@@ -335,7 +335,7 @@ private:
    */
   struct InProgressOp : public RefCountedObject {
     ceph_tid_t tid;
-    set<pg_shard_t> waiting_for_commit;
+    std::set<pg_shard_t> waiting_for_commit;
     Context *on_commit;
     OpRequestRef op;
     eversion_t v;
@@ -349,7 +349,7 @@ private:
        tid(tid), on_commit(on_commit),
        op(op), v(v) {}
   };
-  map<ceph_tid_t, ceph::ref_t<InProgressOp>> in_progress_ops;
+  std::map<ceph_tid_t, ceph::ref_t<InProgressOp>> in_progress_ops;
 public:
   friend class C_OSD_OnOpCommit;
 
@@ -366,7 +366,7 @@ public:
     PGTransactionUPtr &&t,
     const eversion_t &trim_to,
     const eversion_t &min_last_complete_ondisk,
-    vector<pg_log_entry_t>&& log_entries,
+    std::vector<pg_log_entry_t>&& log_entries,
     std::optional<pg_hit_set_history_t> &hset_history,
     Context *on_all_commit,
     ceph_tid_t tid,
@@ -384,7 +384,7 @@ private:
     eversion_t min_last_complete_ondisk,
     hobject_t new_temp_oid,
     hobject_t discard_temp_oid,
-    const bufferlist &log_entries,
+    const ceph::buffer::list &log_entries,
     std::optional<pg_hit_set_history_t> &hset_history,
     ObjectStore::Transaction &op_t,
     pg_shard_t peer,
@@ -398,7 +398,7 @@ private:
     eversion_t min_last_complete_ondisk,
     hobject_t new_temp_oid,
     hobject_t discard_temp_oid,
-    const vector<pg_log_entry_t> &log_entries,
+    const std::vector<pg_log_entry_t> &log_entries,
     std::optional<pg_hit_set_history_t> &hset_history,
     InProgressOp *op,
     ObjectStore::Transaction &op_t);
index 036f4a88e7393a70e49c56955805a8fdf6807483..75f834fd47f68cb4b8df3d33cbca4e8ffc40395f 100644 (file)
@@ -6,6 +6,12 @@
 #include "common/scrub_types.h"
 #include "include/rados/rados_types.hpp"
 
+using std::ostringstream;
+using std::string;
+using std::vector;
+
+using ceph::bufferlist;
+
 namespace {
 ghobject_t make_scrub_object(const spg_t& pgid)
 {
index 39c7da67cff4e33e49240a6b79b1cab9af0c06fb..7722a66c35fb7a3352f4838c3ac2c2e2b78e626e 100644 (file)
@@ -28,18 +28,18 @@ public:
   bool empty() const;
   void flush(ObjectStore::Transaction *);
   void cleanup(ObjectStore::Transaction *);
-  std::vector<bufferlist> get_snap_errors(ObjectStore* store,
+  std::vector<ceph::buffer::list> get_snap_errors(ObjectStore* store,
                                          int64_t pool,
                                          const librados::object_id_t& start,
                                          uint64_t max_return);
-  std::vector<bufferlist> get_object_errors(ObjectStore* store,
+  std::vector<ceph::buffer::list> get_object_errors(ObjectStore* store,
                                            int64_t pool,
                                            const librados::object_id_t& start,
                                            uint64_t max_return);
 private:
   Store(const coll_t& coll, const ghobject_t& oid, ObjectStore* store);
-  std::vector<bufferlist> get_errors(ObjectStore* store,
-                                    const string& start, const string& end,
+  std::vector<ceph::buffer::list> get_errors(ObjectStore* store,
+                                    const std::string& start, const std::string& end,
                                     uint64_t max_return);
 private:
   const coll_t coll;
@@ -47,8 +47,8 @@ private:
   // a temp object holding mappings from seq-id to inconsistencies found in
   // scrubbing
   OSDriver driver;
-  MapCacher::MapCacher<std::string, bufferlist> backend;
-  map<string, bufferlist> results;
+  MapCacher::MapCacher<std::string, ceph::buffer::list> backend;
+  std::map<std::string, ceph::buffer::list> results;
 };
 }
 
index c3699593e5aa5fb8e1ac928502c7f0ad5edb108e..454e1b85768069e5ec365c206da3f1bb21e5b34f 100644 (file)
@@ -9,6 +9,9 @@
 #define dout_context cct
 #define dout_subsys ceph_subsys_osd
 
+using std::map;
+using std::set;
+
 void Session::clear_backoffs()
 {
   map<spg_t,map<hobject_t,set<ceph::ref_t<Backoff>>>> ls;
index 27000a95a562358fa3bb9741a8b1515c4c6e08f0..a42d37bfecc092da64557e2e5575e104ecceb147 100644 (file)
@@ -99,7 +99,7 @@ struct Backoff : public RefCountedObject {
   //   - both null (teardown), or
   //   - only session is set (and state == DELETING)
   PGRef pg;             ///< owning pg
-  ceph::ref_t<class Session> session;   ///< owning session
+  ceph::ref_t<struct Session> session;   ///< owning session
   hobject_t begin, end; ///< [) range to block, unless ==, then single obj
 
   friend ostream& operator<<(ostream& out, const Backoff& b) {
@@ -143,7 +143,7 @@ struct Session : public RefCountedObject {
   /// protects backoffs; orders inside Backoff::lock *and* PG::backoff_lock
   ceph::mutex backoff_lock = ceph::make_mutex("Session::backoff_lock");
   std::atomic<int> backoff_count= {0};  ///< simple count of backoffs
-  map<spg_t,map<hobject_t,set<ceph::ref_t<Backoff>>>> backoffs;
+  std::map<spg_t, std::map<hobject_t, std::set<ceph::ref_t<Backoff>>>> backoffs;
 
   std::atomic<uint64_t> backoff_seq = {0};
 
index 582bf2191f5e61b81432b7d7b795fd0bdf9e08c4..912c539b8bcf886dd4416264d2f937498918ce93 100644 (file)
 #undef dout_prefix
 #define dout_prefix *_dout << "snap_mapper."
 
+using std::make_pair;
+using std::map;
+using std::pair;
+using std::set;
 using std::string;
+using std::vector;
+
+using ceph::bufferlist;
+using ceph::decode;
+using ceph::encode;
+using ceph::timespan_str;
 
 const string SnapMapper::LEGACY_MAPPING_PREFIX = "MAP_";
 const string SnapMapper::MAPPING_PREFIX = "SNA_";
index 3fbc2261a9df05038033fc6a99b0a04bf4ebc010..f8c2aff1b20f420826ac6b0e0593fd06f6444bfb 100644 (file)
@@ -18,7 +18,7 @@
 #include <string>
 #include <set>
 #include <utility>
-#include <string.h>
+#include <cstring>
 
 #include "common/map_cacher.hpp"
 #include "common/hobject.h"
 #include "os/ObjectStore.h"
 #include "osd/OSDMap.h"
 
-class OSDriver : public MapCacher::StoreDriver<std::string, bufferlist> {
+class OSDriver : public MapCacher::StoreDriver<std::string, ceph::buffer::list> {
   ObjectStore *os;
   ObjectStore::CollectionHandle ch;
   ghobject_t hoid;
 
 public:
-  class OSTransaction : public MapCacher::Transaction<std::string, bufferlist> {
+  class OSTransaction : public MapCacher::Transaction<std::string, ceph::buffer::list> {
     friend class OSDriver;
     coll_t cid;
     ghobject_t hoid;
@@ -46,7 +46,7 @@ public:
       : cid(cid), hoid(hoid), t(t) {}
   public:
     void set_keys(
-      const std::map<std::string, bufferlist> &to_set) override {
+      const std::map<std::string, ceph::buffer::list> &to_set) override {
       t->omap_setkeys(cid, hoid, to_set);
     }
     void remove_keys(
@@ -71,10 +71,10 @@ public:
   }
   int get_keys(
     const std::set<std::string> &keys,
-    std::map<std::string, bufferlist> *out) override;
+    std::map<std::string, ceph::buffer::list> *out) override;
   int get_next(
     const std::string &key,
-    pair<std::string, bufferlist> *next) override;
+    std::pair<std::string, ceph::buffer::list> *next) override;
 };
 
 /**
@@ -86,7 +86,7 @@ public:
  *
  * We accomplish this using two sets of keys:
  *  1) OBJECT_PREFIX + obj.str() -> encoding of object_snaps
- *  2) MAPPING_PREFIX + poolid + snapid_t + obj.str() -> encoding of pair<snapid_t, obj>
+ *  2) MAPPING_PREFIX + poolid + snapid_t + obj.str() -> encoding of std::pair<snapid_t, obj>
  *
  * The on disk strings and encodings are implemented in to_raw, to_raw_key,
  * from_raw, to_object_key.
@@ -108,23 +108,23 @@ public:
     object_snaps(hobject_t oid, const std::set<snapid_t> &snaps)
       : oid(oid), snaps(snaps) {}
     object_snaps() {}
-    void encode(bufferlist &bl) const;
-    void decode(bufferlist::const_iterator &bp);
+    void encode(ceph::buffer::list &bl) const;
+    void decode(ceph::buffer::list::const_iterator &bp);
   };
 
   struct Mapping {
     snapid_t snap;
     hobject_t hoid;
-    explicit Mapping(const pair<snapid_t, hobject_t> &in)
+    explicit Mapping(const std::pair<snapid_t, hobject_t> &in)
       : snap(in.first), hoid(in.second) {}
     Mapping() : snap(0) {}
-    void encode(bufferlist &bl) const {
+    void encode(ceph::buffer::list &bl) const {
       ENCODE_START(1, 1, bl);
       encode(snap, bl);
       encode(hoid, bl);
       ENCODE_FINISH(bl);
     }
-    void decode(bufferlist::const_iterator &bl) {
+    void decode(ceph::buffer::list::const_iterator &bl) {
       DECODE_START(1, bl);
       decode(snap, bl);
       decode(hoid, bl);
@@ -157,7 +157,7 @@ public:
 
     bool _parse_m();   ///< advance the (object) mapper pointer
 
-    vector<std::tuple<int64_t, snapid_t, uint32_t, shard_id_t>> stray;
+    std::vector<std::tuple<int64_t, snapid_t, uint32_t, shard_id_t>> stray;
 
     Scrubber(
       CephContext *cct,
@@ -187,7 +187,7 @@ public:
     ObjectStore::CollectionHandle& ch,
     ghobject_t hoid,
     ObjectStore::Transaction *t,
-    map<epoch_t,mempool::osdmap::map<int64_t,snap_interval_set_t>> purged_snaps);
+    std::map<epoch_t,mempool::osdmap::map<int64_t,snap_interval_set_t>> purged_snaps);
   static void scrub_purged_snaps(
     CephContext *cct,
     ObjectStore *store,
@@ -205,11 +205,11 @@ private:
     snapid_t *begin, snapid_t *end);
   static void make_purged_snap_key_value(
     int64_t pool, snapid_t begin,
-    snapid_t end, map<string,bufferlist> *m);
-  static string make_purged_snap_key(int64_t pool, snapid_t last);
+    snapid_t end, std::map<std::string,ceph::buffer::list> *m);
+  static std::string make_purged_snap_key(int64_t pool, snapid_t last);
 
 
-  MapCacher::MapCacher<std::string, bufferlist> backend;
+  MapCacher::MapCacher<std::string, ceph::buffer::list> backend;
 
   static std::string get_legacy_prefix(snapid_t snap);
   std::string to_legacy_raw_key(
@@ -220,13 +220,13 @@ private:
   std::string to_raw_key(
     const std::pair<snapid_t, hobject_t> &to_map);
 
-  std::pair<std::string, bufferlist> to_raw(
+  std::pair<std::string, ceph::buffer::list> to_raw(
     const std::pair<snapid_t, hobject_t> &to_map);
 
   static bool is_mapping(const std::string &to_test);
 
   static std::pair<snapid_t, hobject_t> from_raw(
-    const std::pair<std::string, bufferlist> &image);
+    const std::pair<std::string, ceph::buffer::list> &image);
 
   std::string to_object_key(const hobject_t &hoid);
 
@@ -235,38 +235,38 @@ private:
   void set_snaps(
     const hobject_t &oid,
     const object_snaps &out,
-    MapCacher::Transaction<std::string, bufferlist> *t);
+    MapCacher::Transaction<std::string, ceph::buffer::list> *t);
 
   void clear_snaps(
     const hobject_t &oid,
-    MapCacher::Transaction<std::string, bufferlist> *t);
+    MapCacher::Transaction<std::string, ceph::buffer::list> *t);
 
   // True if hoid belongs in this mapping based on mask_bits and match
   bool check(const hobject_t &hoid) const;
 
   int _remove_oid(
     const hobject_t &oid,    ///< [in] oid to remove
-    MapCacher::Transaction<std::string, bufferlist> *t ///< [out] transaction
+    MapCacher::Transaction<std::string, ceph::buffer::list> *t ///< [out] transaction
     );
 
 public:
-  static string make_shard_prefix(shard_id_t shard) {
+  static std::string make_shard_prefix(shard_id_t shard) {
     if (shard == shard_id_t::NO_SHARD)
-      return string();
+      return std::string();
     char buf[20];
     int r = snprintf(buf, sizeof(buf), ".%x", (int)shard);
     ceph_assert(r < (int)sizeof(buf));
-    return string(buf, r) + '_';
+    return std::string(buf, r) + '_';
   }
   uint32_t mask_bits;
   const uint32_t match;
-  string last_key_checked;
+  std::string last_key_checked;
   const int64_t pool;
   const shard_id_t shard;
-  const string shard_prefix;
+  const std::string shard_prefix;
   SnapMapper(
     CephContext* cct,
-    MapCacher::StoreDriver<std::string, bufferlist> *driver,
+    MapCacher::StoreDriver<std::string, ceph::buffer::list> *driver,
     uint32_t match,  ///< [in] pgid
     uint32_t bits,   ///< [in] current split bits
     int64_t pool,    ///< [in] pool
@@ -277,20 +277,18 @@ public:
     update_bits(mask_bits);
   }
 
-  set<string> prefixes;
+  std::set<std::string> prefixes;
   /// Update bits in case of pg split or merge
   void update_bits(
     uint32_t new_bits  ///< [in] new split bits
     ) {
     mask_bits = new_bits;
-    set<string> _prefixes = hobject_t::get_prefixes(
+    std::set<std::string> _prefixes = hobject_t::get_prefixes(
       mask_bits,
       match,
       pool);
     prefixes.clear();
-    for (set<string>::iterator i = _prefixes.begin();
-        i != _prefixes.end();
-        ++i) {
+    for (auto i = _prefixes.begin(); i != _prefixes.end(); ++i) {
       prefixes.insert(shard_prefix + *i);
     }
   }
@@ -298,29 +296,29 @@ public:
   /// Update snaps for oid, empty new_snaps removes the mapping
   int update_snaps(
     const hobject_t &oid,       ///< [in] oid to update
-    const std::set<snapid_t> &new_snaps, ///< [in] new snap set
+    const std::set<snapid_t> &new_snaps, ///< [in] new snap std::set
     const std::set<snapid_t> *old_snaps, ///< [in] old snaps (for debugging)
-    MapCacher::Transaction<std::string, bufferlist> *t ///< [out] transaction
+    MapCacher::Transaction<std::string, ceph::buffer::list> *t ///< [out] transaction
     ); ///@ return error, 0 on success
 
   /// Add mapping for oid, must not already be mapped
   void add_oid(
     const hobject_t &oid,       ///< [in] oid to add
     const std::set<snapid_t>& new_snaps, ///< [in] snaps
-    MapCacher::Transaction<std::string, bufferlist> *t ///< [out] transaction
+    MapCacher::Transaction<std::string, ceph::buffer::list> *t ///< [out] transaction
     );
 
   /// Returns first object with snap as a snap
   int get_next_objects_to_trim(
     snapid_t snap,              ///< [in] snap to check
     unsigned max,               ///< [in] max to get
-    vector<hobject_t> *out      ///< [out] next objects to trim (must be empty)
+    std::vector<hobject_t> *out      ///< [out] next objects to trim (must be empty)
     );  ///< @return error, -ENOENT if no more objects
 
   /// Remove mapping for oid
   int remove_oid(
     const hobject_t &oid,    ///< [in] oid to remove
-    MapCacher::Transaction<std::string, bufferlist> *t ///< [out] transaction
+    MapCacher::Transaction<std::string, ceph::buffer::list> *t ///< [out] transaction
     ); ///< @return error, -ENOENT if the object is not mapped
 
   /// Get snaps for oid
index 2c58534b1c2da6a2c7cb2dd12a4fde9201b9a7d4..28e1598a95962a04a1b83e90503428beb70314f0 100644 (file)
 #ifndef CEPH_OSD_TIERAGENT_H
 #define CEPH_OSD_TIERAGENT_H
 
+#include <ctime>
+#include <list>
+#include <map>
+#include <utility>
+
+#include "common/Formatter.h"
+#include "common/histogram.h"
+#include "common/hobject.h"
+
+#include "osd/HitSet.h"
+
 struct TierAgentState {
   /// current position iterating across pool
   hobject_t position;
@@ -27,10 +38,10 @@ struct TierAgentState {
   int hist_age;
 
   /// past HitSet(s) (not current)
-  map<time_t,HitSetRef> hit_set_map;
+  std::map<time_t,HitSetRef> hit_set_map;
 
   /// a few recent things we've seen that are clean
-  list<hobject_t> recent_clean;
+  std::list<hobject_t> recent_clean;
 
   enum flush_mode_t {
     FLUSH_MODE_IDLE,   // nothing to flush
@@ -89,7 +100,7 @@ struct TierAgentState {
 
   /// add archived HitSet
   void add_hit_set(time_t start, HitSetRef hs) {
-    hit_set_map.insert(make_pair(start, hs));
+    hit_set_map.insert(std::make_pair(start, hs));
   }
 
   /// remove old/trimmed HitSet
@@ -103,7 +114,7 @@ struct TierAgentState {
     hit_set_map.clear();
   }
 
-  void dump(Formatter *f) const {
+  void dump(ceph::Formatter *f) const {
     f->dump_string("flush_mode", get_flush_mode_name());
     f->dump_string("evict_mode", get_evict_mode_name());
     f->dump_unsigned("evict_effort", evict_effort);
index c1e26f230c74820f9e1e03e971da3c4a82cfa59b..78aae6e2d9629b98bd6fb4abcdc8b950a68959ad 100644 (file)
 
 #include "common/config.h"
 
-struct CancelableContext : public Context {
-  virtual void cancel() = 0;
-};
-
 #define dout_context osd->cct
 #define dout_subsys ceph_subsys_osd
 #undef dout_prefix
 #define dout_prefix _prefix(_dout, this)
 
+using std::list;
+using std::make_pair;
+using std::pair;
+using std::ostream;
+using std::set;
+
+using ceph::bufferlist;
+using ceph::decode;
+using ceph::encode;
+
+struct CancelableContext : public Context {
+  virtual void cancel() = 0;
+};
+
+
 static ostream& _prefix(
   std::ostream* _dout,
   Notify *notify) {
@@ -106,9 +117,7 @@ void Notify::do_timeout()
   _watchers.swap(watchers);
   lock.unlock();
 
-  for (set<WatchRef>::iterator i = _watchers.begin();
-       i != _watchers.end();
-       ++i) {
+  for (auto i = _watchers.begin(); i != _watchers.end(); ++i) {
     boost::intrusive_ptr<PrimaryLogPG> pg((*i)->get_pg());
     pg->lock();
     if (!(*i)->is_discarded()) {
@@ -185,7 +194,7 @@ void Notify::maybe_complete_notify()
     bufferlist bl;
     encode(notify_replies, bl);
     list<pair<uint64_t,uint64_t> > missed;
-    for (set<WatchRef>::iterator p = watchers.begin(); p != watchers.end(); ++p) {
+    for (auto p = watchers.begin(); p != watchers.end(); ++p) {
       missed.push_back(make_pair((*p)->get_watcher_gid(),
                                 (*p)->get_cookie()));
     }
@@ -372,7 +381,7 @@ void Watch::connect(ConnectionRef con, bool _will_ping)
     auto sessionref = static_cast<Session*>(priv.get());
     sessionref->wstate.addWatch(self.lock());
     priv.reset();
-    for (map<uint64_t, NotifyRef>::iterator i = in_progress_notifies.begin();
+    for (auto i = in_progress_notifies.begin();
         i != in_progress_notifies.end();
         ++i) {
       send_notify(i->second);
@@ -397,7 +406,7 @@ void Watch::disconnect()
 void Watch::discard()
 {
   dout(10) << "discard" << dendl;
-  for (map<uint64_t, NotifyRef>::iterator i = in_progress_notifies.begin();
+  for (auto i = in_progress_notifies.begin();
        i != in_progress_notifies.end();
        ++i) {
     i->second->discard();
@@ -437,7 +446,7 @@ void Watch::remove(bool send_disconnect)
                                         CEPH_WATCH_EVENT_DISCONNECT, empty));
     conn->send_message(reply);
   }
-  for (map<uint64_t, NotifyRef>::iterator i = in_progress_notifies.begin();
+  for (auto i = in_progress_notifies.begin();
        i != in_progress_notifies.end();
        ++i) {
     i->second->complete_watcher_remove(self.lock());
@@ -489,7 +498,7 @@ void Watch::send_notify(NotifyRef notif)
 void Watch::notify_ack(uint64_t notify_id, bufferlist& reply_bl)
 {
   dout(10) << "notify_ack" << dendl;
-  map<uint64_t, NotifyRef>::iterator i = in_progress_notifies.find(notify_id);
+  auto i = in_progress_notifies.find(notify_id);
   if (i != in_progress_notifies.end()) {
     i->second->complete_watcher(self.lock(), reply_bl);
     in_progress_notifies.erase(i);
index 1cc63fbb1451d9292cadc9d4ed44c54a6e35a946..470706afeaf66ac81bd56161ca8e805742abb950 100644 (file)
 
 #define dout_context ClassHandler::get_instance().cct
 
+using std::map;
+using std::set;
+using std::string;
+using std::vector;
+
+using ceph::bufferlist;
+using ceph::decode;
+using ceph::encode;
+using ceph::real_time;
+
 
 int cls_call(cls_method_context_t hctx, const char *cls, const char *method,
-                                 char *indata, int datalen,
-                                 char **outdata, int *outdatalen)
+            char *indata, int datalen, char **outdata, int *outdatalen)
 {
   PrimaryLogPG::OpContext **pctx = (PrimaryLogPG::OpContext **)hctx;
   bufferlist idata;
@@ -49,7 +58,7 @@ int cls_call(cls_method_context_t hctx, const char *cls, const char *method,
 }
 
 int cls_getxattr(cls_method_context_t hctx, const char *name,
-                                 char **outdata, int *outdatalen)
+                char **outdata, int *outdatalen)
 {
   PrimaryLogPG::OpContext **pctx = (PrimaryLogPG::OpContext **)hctx;
   vector<OSDOp> nops(1);
@@ -73,7 +82,7 @@ int cls_getxattr(cls_method_context_t hctx, const char *name,
 }
 
 int cls_setxattr(cls_method_context_t hctx, const char *name,
-                                 const char *value, int val_len)
+                const char *value, int val_len)
 {
   PrimaryLogPG::OpContext **pctx = (PrimaryLogPG::OpContext **)hctx;
   vector<OSDOp> nops(1);
@@ -91,7 +100,7 @@ int cls_setxattr(cls_method_context_t hctx, const char *name,
 }
 
 int cls_read(cls_method_context_t hctx, int ofs, int len,
-                                 char **outdata, int *outdatalen)
+            char **outdata, int *outdatalen)
 {
   PrimaryLogPG::OpContext **pctx = (PrimaryLogPG::OpContext **)hctx;
   vector<OSDOp> ops(1);
@@ -150,7 +159,7 @@ int cls_cxx_stat(cls_method_context_t hctx, uint64_t *size, time_t *mtime)
   try {
     decode(s, iter);
     decode(ut, iter);
-  } catch (buffer::error& err) {
+  } catch (ceph::buffer::error& err) {
     return -EIO;
   }
   if (size)
@@ -175,7 +184,7 @@ int cls_cxx_stat2(cls_method_context_t hctx, uint64_t *size, ceph::real_time *mt
   try {
     decode(s, iter);
     decode(ut, iter);
-  } catch (buffer::error& err) {
+  } catch (ceph::buffer::error& err) {
     return -EIO;
   }
   if (size)
@@ -294,7 +303,7 @@ int cls_cxx_getxattrs(cls_method_context_t hctx, map<string, bufferlist> *attrse
   auto iter = op.outdata.cbegin();
   try {
     decode(*attrset, iter);
-  } catch (buffer::error& err) {
+  } catch (ceph::buffer::error& err) {
     return -EIO;
   }
   return 0;
@@ -353,7 +362,7 @@ int cls_cxx_map_get_all_vals(cls_method_context_t hctx, map<string, bufferlist>*
   try {
     decode(*vals, iter);
     decode(*more, iter);
-  } catch (buffer::error& err) {
+  } catch (ceph::buffer::error& err) {
     return -EIO;
   }
   return vals->size();
@@ -381,7 +390,7 @@ int cls_cxx_map_get_keys(cls_method_context_t hctx, const string &start_obj,
   try {
     decode(*keys, iter);
     decode(*more, iter);
-  } catch (buffer::error& err) {
+  } catch (ceph::buffer::error& err) {
     return -EIO;
   }
   return keys->size();
@@ -410,7 +419,7 @@ int cls_cxx_map_get_vals(cls_method_context_t hctx, const string &start_obj,
   try {
     decode(*vals, iter);
     decode(*more, iter);
-  } catch (buffer::error& err) {
+  } catch (ceph::buffer::error& err) {
     return -EIO;
   }
   return vals->size();
@@ -459,7 +468,7 @@ int cls_cxx_map_get_val(cls_method_context_t hctx, const string &key,
       return -ENOENT;
 
     *outbl = iter->second;
-  } catch (buffer::error& e) {
+  } catch (ceph::buffer::error& e) {
     return -EIO;
   }
   return 0;
@@ -567,7 +576,7 @@ int cls_cxx_list_watchers(cls_method_context_t hctx,
   auto iter = op.outdata.cbegin();
   try {
     decode(*watchers, iter);
-  } catch (buffer::error& err) {
+  } catch (ceph::buffer::error& err) {
     return -EIO;
   }
   return 0;
@@ -623,8 +632,8 @@ int cls_get_snapset_seq(cls_method_context_t hctx, uint64_t *snap_seq) {
 }
 
 int cls_cxx_chunk_write_and_set(cls_method_context_t hctx, int ofs, int len,
-                   bufferlist *write_inbl, uint32_t op_flags, bufferlist *set_inbl,
-                  int set_len)
+                               bufferlist *write_inbl, uint32_t op_flags,
+                               bufferlist *set_inbl, int set_len)
 {
   PrimaryLogPG::OpContext **pctx = (PrimaryLogPG::OpContext **)hctx;
   char cname[] = "cas";
index 9ef708a664cc4bd7d472f0ab1ed741f76b99605a..31987d2a4f2f6e78ada4242cbf2b808c8b009787 100644 (file)
@@ -181,7 +181,7 @@ struct RWState {
   }
 };
 
-inline ostream& operator<<(ostream& out, const RWState& rw)
+inline std::ostream& operator<<(std::ostream& out, const RWState& rw)
 {
   return out << "rwstate(" << rw.get_state_name()
             << " n=" << rw.count
index 1373470d06f8a6123f4fb5944330e4df26fa46d1..17f4f314643415f9953d68a697ca198139ec34ab 100644 (file)
@@ -25,7 +25,7 @@ struct SnapSetContext {
   explicit SnapSetContext(const hobject_t& o) :
     oid(o), ref(0), registered(false), exists(true) { }
 };
-class ObjectContext;
+struct ObjectContext;
 typedef std::shared_ptr<ObjectContext> ObjectContextRef;
 
 struct ObjectContext {
@@ -38,10 +38,10 @@ struct ObjectContext {
 public:
 
   // any entity in obs.oi.watchers MUST be in either watchers or unconnected_watchers.
-  map<pair<uint64_t, entity_name_t>, WatchRef> watchers;
+  std::map<std::pair<uint64_t, entity_name_t>, WatchRef> watchers;
 
   // attr cache
-  map<string, bufferlist> attr_cache;
+  std::map<std::string, ceph::buffer::list> attr_cache;
 
   RWState rwstate;
   std::list<OpRequestRef> waiters;  ///< ops waiting on state change
@@ -74,21 +74,21 @@ public:
     }
     return false;
   }
-  void wake(list<OpRequestRef> *requeue) {
+  void wake(std::list<OpRequestRef> *requeue) {
     rwstate.release_waiters();
     requeue->splice(requeue->end(), waiters);
   }
-  void put_read(list<OpRequestRef> *requeue) {
+  void put_read(std::list<OpRequestRef> *requeue) {
     if (rwstate.put_read()) {
       wake(requeue);
     }
   }
-  void put_write(list<OpRequestRef> *requeue) {
+  void put_write(std::list<OpRequestRef> *requeue) {
     if (rwstate.put_write()) {
       wake(requeue);
     }
   }
-  void put_excl(list<OpRequestRef> *requeue) {
+  void put_excl(std::list<OpRequestRef> *requeue) {
     if (rwstate.put_excl()) {
       wake(requeue);
     }
@@ -120,14 +120,14 @@ public:
   bool try_get_read_lock() {
     return rwstate.get_read_lock();
   }
-  void drop_recovery_read(list<OpRequestRef> *ls) {
+  void drop_recovery_read(std::list<OpRequestRef> *ls) {
     ceph_assert(rwstate.recovery_read_marker);
     put_read(ls);
     rwstate.recovery_read_marker = false;
   }
   void put_lock_type(
     RWState::State type,
-    list<OpRequestRef> *to_wake,
+    std::list<OpRequestRef> *to_wake,
     bool *requeue_recovery,
     bool *requeue_snaptrimmer) {
     switch (type) {
@@ -185,7 +185,7 @@ public:
 
 };
 
-inline ostream& operator<<(ostream& out, const ObjectState& obs)
+inline std::ostream& operator<<(std::ostream& out, const ObjectState& obs)
 {
   out << obs.oi.soid;
   if (!obs.exists)
@@ -193,7 +193,7 @@ inline ostream& operator<<(ostream& out, const ObjectState& obs)
   return out;
 }
 
-inline ostream& operator<<(ostream& out, const ObjectContext& obc)
+inline std::ostream& operator<<(std::ostream& out, const ObjectContext& obc)
 {
   return out << "obc(" << obc.obs << " " << obc.rwstate << ")";
 }
@@ -207,7 +207,7 @@ class ObcLockManager {
       RWState::State type)
       : obc(std::move(obc)), type(type) {}
   };
-  map<hobject_t, ObjectLockState> locks;
+  std::map<hobject_t, ObjectLockState> locks;
 public:
   ObcLockManager() = default;
   ObcLockManager(ObcLockManager &&) = default;
@@ -223,7 +223,7 @@ public:
     OpRequestRef& op) {
     ceph_assert(locks.find(hoid) == locks.end());
     if (obc->get_lock_type(op, type)) {
-      locks.insert(make_pair(hoid, ObjectLockState(obc, type)));
+      locks.insert(std::make_pair(hoid, ObjectLockState(obc, type)));
       return true;
     } else {
       return false;
@@ -236,7 +236,7 @@ public:
     ceph_assert(locks.find(hoid) == locks.end());
     if (obc->rwstate.take_write_lock()) {
       locks.insert(
-       make_pair(
+       std::make_pair(
          hoid, ObjectLockState(obc, RWState::RWWRITE)));
       return true;
     } else {
@@ -251,7 +251,7 @@ public:
     ceph_assert(locks.find(hoid) == locks.end());
     if (obc->get_snaptrimmer_write(mark_if_unsuccessful)) {
       locks.insert(
-       make_pair(
+       std::make_pair(
          hoid, ObjectLockState(obc, RWState::RWWRITE)));
       return true;
     } else {
@@ -266,7 +266,7 @@ public:
     ceph_assert(locks.find(hoid) == locks.end());
     if (obc->get_write_greedy(op)) {
       locks.insert(
-       make_pair(
+       std::make_pair(
          hoid, ObjectLockState(obc, RWState::RWWRITE)));
       return true;
     } else {
@@ -281,7 +281,7 @@ public:
     ceph_assert(locks.find(hoid) == locks.end());
     if (obc->try_get_read_lock()) {
       locks.insert(
-       make_pair(
+       std::make_pair(
          hoid,
          ObjectLockState(obc, RWState::RWREAD)));
       return true;
@@ -291,11 +291,11 @@ public:
   }
 
   void put_locks(
-    list<pair<ObjectContextRef, list<OpRequestRef> > > *to_requeue,
+    std::list<std::pair<ObjectContextRef, std::list<OpRequestRef> > > *to_requeue,
     bool *requeue_recovery,
     bool *requeue_snaptrimmer) {
     for (auto& p: locks) {
-      list<OpRequestRef> _to_requeue;
+      std::list<OpRequestRef> _to_requeue;
       p.second.obc->put_lock_type(
        p.second.type,
        &_to_requeue,
index 08b888c6d55a1a5ac8caac32fd99e0dea0ede2a9..7282ad30d7363c1a0b86dfec4a3cc823fb776267 100644 (file)
@@ -40,7 +40,7 @@ class OpSchedulerItem {
 public:
   class OrderLocker {
   public:
-    using Ref = unique_ptr<OrderLocker>;
+    using Ref = std::unique_ptr<OrderLocker>;
     virtual void lock() = 0;
     virtual void unlock() = 0;
     virtual ~OrderLocker() {}
@@ -85,13 +85,13 @@ public:
       return nullptr;
     }
 
-    virtual ostream &print(ostream &rhs) const = 0;
+    virtual std::ostream &print(std::ostream &rhs) const = 0;
 
     virtual void run(OSD *osd, OSDShard *sdata, PGRef& pg, ThreadPool::TPHandle &handle) = 0;
     virtual op_scheduler_class get_scheduler_class() const = 0;
 
     virtual ~OpQueueable() {}
-    friend ostream& operator<<(ostream& out, const OpQueueable& q) {
+    friend std::ostream& operator<<(std::ostream& out, const OpQueueable& q) {
       return q.print(out);
     }
 
@@ -169,7 +169,7 @@ public:
     return qitem->get_scheduler_class();
   }
 
-  friend ostream& operator<<(ostream& out, const OpSchedulerItem& item) {
+  friend std::ostream& operator<<(std::ostream& out, const OpSchedulerItem& item) {
      out << "OpSchedulerItem("
         << item.get_ordering_token() << " " << *item.qitem
         << " prio " << item.get_priority()
@@ -235,7 +235,7 @@ public:
     return op_type_t::client_op;
   }
 
-  ostream &print(ostream &rhs) const final {
+  std::ostream &print(std::ostream &rhs) const final {
     return rhs << "PGOpItem(op=" << *(op->get_req()) << ")";
   }
 
@@ -263,7 +263,7 @@ public:
   op_type_t get_op_type() const final {
     return op_type_t::peering_event;
   }
-  ostream &print(ostream &rhs) const final {
+  std::ostream &print(std::ostream &rhs) const final {
     return rhs << "PGPeeringEvent(" << evt->get_desc() << ")";
   }
   void run(OSD *osd, OSDShard *sdata, PGRef& pg, ThreadPool::TPHandle &handle) final;
@@ -291,7 +291,7 @@ public:
   op_type_t get_op_type() const final {
     return op_type_t::bg_snaptrim;
   }
-  ostream &print(ostream &rhs) const final {
+  std::ostream &print(std::ostream &rhs) const final {
     return rhs << "PGSnapTrim(pgid=" << get_pgid()
               << "epoch_queued=" << epoch_queued
               << ")";
@@ -313,7 +313,7 @@ public:
   op_type_t get_op_type() const final {
     return op_type_t::bg_scrub;
   }
-  ostream &print(ostream &rhs) const final {
+  std::ostream &print(std::ostream &rhs) const final {
     return rhs << "PGScrub(pgid=" << get_pgid()
               << "epoch_queued=" << epoch_queued
               << ")";
@@ -339,7 +339,7 @@ public:
   op_type_t get_op_type() const final {
     return op_type_t::bg_recovery;
   }
-  ostream &print(ostream &rhs) const final {
+  std::ostream &print(std::ostream &rhs) const final {
     return rhs << "PGRecovery(pgid=" << get_pgid()
               << "epoch_queued=" << epoch_queued
               << "reserved_pushes=" << reserved_pushes
@@ -356,7 +356,7 @@ public:
 };
 
 class PGRecoveryContext : public PGOpQueueable {
-  unique_ptr<GenContext<ThreadPool::TPHandle&>> c;
+  std::unique_ptr<GenContext<ThreadPool::TPHandle&>> c;
   epoch_t epoch;
 public:
   PGRecoveryContext(spg_t pgid,
@@ -366,7 +366,7 @@ public:
   op_type_t get_op_type() const final {
     return op_type_t::bg_recovery;
   }
-  ostream &print(ostream &rhs) const final {
+  std::ostream &print(std::ostream &rhs) const final {
     return rhs << "PGRecoveryContext(pgid=" << get_pgid()
               << " c=" << c.get() << " epoch=" << epoch
               << ")";
@@ -389,7 +389,7 @@ public:
   op_type_t get_op_type() const final {
     return op_type_t::bg_pg_delete;
   }
-  ostream &print(ostream &rhs) const final {
+  std::ostream &print(std::ostream &rhs) const final {
     return rhs << "PGDelete(" << get_pgid()
               << " e" << epoch_queued
               << ")";