]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
OSD: s/PGQueueable/OpQueueItem/g
authorKefu Chai <kchai@redhat.com>
Fri, 30 Jun 2017 07:13:48 +0000 (15:13 +0800)
committermyoungwon oh <omwmw@sk.com>
Mon, 9 Oct 2017 16:31:51 +0000 (01:31 +0900)
PGQueueable was always a pretty bad name, and going forward, we won't
necessarily be scheduling things ordered only by the PG lock, so let's
change it.

Signed-off-by: Samuel Just <sjust@redhat.com>
Signed-off-by: Kefu Chai <kchai@redhat.com>
Signed-off-by: Myoungwon Oh <omwmw@sk.com>
13 files changed:
src/osd/CMakeLists.txt
src/osd/OSD.cc
src/osd/OSD.h
src/osd/OpQueueItem.cc [new file with mode: 0644]
src/osd/OpQueueItem.h [new file with mode: 0644]
src/osd/PG.cc
src/osd/PGQueueable.cc [deleted file]
src/osd/PGQueueable.h [deleted file]
src/osd/PrimaryLogPG.cc
src/osd/mClockClientQueue.h
src/osd/mClockOpClassQueue.h
src/test/osd/TestMClockClientQueue.cc
src/test/osd/TestMClockOpClassQueue.cc

index 3ec6f31a604815dd29e8c43f52d62bc434073f15..8c7b2e8f9f6f41808f076cbbad9d9fc0f0622b1f 100644 (file)
@@ -31,7 +31,7 @@ set(osd_srcs
   ExtentCache.cc
   mClockOpClassQueue.cc
   mClockClientQueue.cc
-  PGQueueable.cc
+  OpQueueItem.cc
   ${CMAKE_SOURCE_DIR}/src/common/TrackedOp.cc
   ${osd_cyg_functions_src}
   ${osdc_osd_srcs})
index 9bd7566401fc42a99b5b3552e055e78e8e348db6..6a69997d6a06dab6fbdc3fdef0272465db40709a 100644 (file)
@@ -1674,12 +1674,12 @@ void OSDService::handle_misdirected_op(PG *pg, OpRequestRef op)
               << " in e" << m->get_map_epoch() << "/" << osdmap->get_epoch();
 }
 
-void OSDService::enqueue_back(spg_t pgid, PGQueueable qi)
+void OSDService::enqueue_back(spg_t pgid, OpQueueItem qi)
 {
   osd->op_shardedwq.queue(make_pair(pgid, qi));
 }
 
-void OSDService::enqueue_front(spg_t pgid, PGQueueable qi)
+void OSDService::enqueue_front(spg_t pgid, OpQueueItem qi)
 {
   osd->op_shardedwq.queue_front(make_pair(pgid, qi));
 }
@@ -1695,7 +1695,7 @@ void OSDService::queue_for_snap_trim(PG *pg)
   osd->op_shardedwq.queue(
     make_pair(
       pg->pg_id,
-      PGQueueable(
+      OpQueueItem(
        PGSnapTrim(pg->get_osdmap()->get_epoch()),
        cct->_conf->osd_snap_trim_cost,
        cct->_conf->osd_snap_trim_priority,
@@ -8909,7 +8909,7 @@ void OSD::enqueue_op(spg_t pg, OpRequestRef& op, epoch_t epoch)
   op->osd_trace.keyval("cost", op->get_req()->get_cost());
   op->mark_queued_for_pg();
   logger->tinc(l_osd_op_before_queue_op_lat, latency);
-  op_shardedwq.queue(make_pair(pg, PGQueueable(op, epoch)));
+  op_shardedwq.queue(make_pair(pg, OpQueueItem(op, epoch)));
 }
 
 
@@ -9569,7 +9569,7 @@ void OSD::ShardedOpWQ::_process(uint32_t thread_index, heartbeat_handle_d *hb)
       return;
     }
   }
-  pair<spg_t, PGQueueable> item = sdata->pqueue->dequeue();
+  pair<spg_t, OpQueueItem> item = sdata->pqueue->dequeue();
   if (osd->is_stopping()) {
     sdata->sdata_op_ordering_lock.Unlock();
     return;    // OSD shutdown, discard.
@@ -9609,7 +9609,7 @@ void OSD::ShardedOpWQ::_process(uint32_t thread_index, heartbeat_handle_d *hb)
 
   osd->service.maybe_inject_dispatch_delay();
 
-  boost::optional<PGQueueable> qi;
+  boost::optional<OpQueueItem> qi;
 
   // we don't use a Mutex::Locker here because of the
   // osd->service.release_reserved_pushes() call below
@@ -9743,7 +9743,7 @@ void OSD::ShardedOpWQ::_process(uint32_t thread_index, heartbeat_handle_d *hb)
   pg->unlock();
 }
 
-void OSD::ShardedOpWQ::_enqueue(pair<spg_t, PGQueueable> item) {
+void OSD::ShardedOpWQ::_enqueue(pair<spg_t, OpQueueItem> item) {
   uint32_t shard_index =
     item.first.hash_to_shard(shard_list.size());
 
@@ -9769,7 +9769,7 @@ void OSD::ShardedOpWQ::_enqueue(pair<spg_t, PGQueueable> item) {
 
 }
 
-void OSD::ShardedOpWQ::_enqueue_front(pair<spg_t, PGQueueable> item)
+void OSD::ShardedOpWQ::_enqueue_front(pair<spg_t, OpQueueItem> item)
 {
   uint32_t shard_index = item.first.hash_to_shard(shard_list.size());
   ShardData* sdata = shard_list[shard_index];
index 03e414e1425cbed299ba4ea6262c947c3d93e325..f76091ff2cc87dd0541f18bdbaf24ab38f83e383 100644 (file)
@@ -19,6 +19,7 @@
 
 #include "msg/Dispatcher.h"
 
+#include "common/backport14.h"
 #include "common/Mutex.h"
 #include "common/RWLock.h"
 #include "common/Timer.h"
@@ -40,7 +41,7 @@
 #include "OpRequest.h"
 #include "Session.h"
 
-#include "osd/PGQueueable.h"
+#include "osd/OpQueueItem.h"
 
 #include <atomic>
 #include <map>
@@ -366,8 +367,8 @@ public:
   GenContextWQ recovery_gen_wq;
   ClassHandler  *&class_handler;
 
-  void enqueue_back(spg_t pgid, PGQueueable qi);
-  void enqueue_front(spg_t pgid, PGQueueable qi);
+  void enqueue_back(spg_t pgid, OpQueueItem qi);
+  void enqueue_front(spg_t pgid, OpQueueItem qi);
 
   void maybe_inject_dispatch_delay() {
     if (g_conf->osd_debug_inject_dispatch_delay_probability > 0) {
@@ -866,7 +867,7 @@ public:
     }
     enqueue_back(
       pg->get_pgid(),
-      PGQueueable(
+      OpQueueItem(
        PGScrub(pg->get_osdmap()->get_epoch()),
        cct->_conf->osd_scrub_cost,
        scrub_queue_priority,
@@ -894,7 +895,7 @@ private:
     assert(recovery_lock.is_locked_by_me());
     enqueue_back(
       p.second->get_pgid(),
-      PGQueueable(
+      OpQueueItem(
        PGRecovery(p.first, reserved_pushes),
        cct->_conf->osd_recovery_cost,
        cct->_conf->osd_recovery_priority,
@@ -1617,10 +1618,10 @@ private:
    * wake_pg_waiters, and (2) when wake_pg_waiters just ran, waiting_for_pg
    * and already requeued the items.
    */
-  friend class PGQueueable;
+  friend class OpQueueItem;
 
   class ShardedOpWQ
-    : public ShardedThreadPool::ShardedWQ<pair<spg_t,PGQueueable>>
+    : public ShardedThreadPool::ShardedWQ<pair<spg_t,OpQueueItem>>
   {
     struct ShardData {
       Mutex sdata_lock;
@@ -1631,7 +1632,7 @@ private:
       OSDMapRef waiting_for_pg_osdmap;
       struct pg_slot {
        PGRef pg;                     ///< cached pg reference [optional]
-       deque<PGQueueable> to_process; ///< order items for this slot
+       deque<OpQueueItem> to_process; ///< order items for this slot
        int num_running = 0;          ///< _process threads doing pg lookup/lock
 
        /// true if pg does/did not exist. if so all new items go directly to
@@ -1649,9 +1650,9 @@ private:
       unordered_map<spg_t,pg_slot> pg_slots;
 
       /// priority queue
-      std::unique_ptr<OpQueue< pair<spg_t, PGQueueable>, uint64_t>> pqueue;
+      std::unique_ptr<OpQueue< pair<spg_t, OpQueueItem>, uint64_t>> pqueue;
 
-      void _enqueue_front(pair<spg_t, PGQueueable> item, unsigned cutoff) {
+      void _enqueue_front(pair<spg_t, OpQueueItem> item, unsigned cutoff) {
        unsigned priority = item.second.get_priority();
        unsigned cost = item.second.get_cost();
        if (priority >= cutoff)
@@ -1673,20 +1674,18 @@ private:
                                 false, cct) {
        if (opqueue == io_queue::weightedpriority) {
          pqueue = std::unique_ptr
-           <WeightedPriorityQueue<pair<spg_t,PGQueueable>,uint64_t>>(
-             new WeightedPriorityQueue<pair<spg_t,PGQueueable>,uint64_t>(
+           <WeightedPriorityQueue<pair<spg_t,OpQueueItem>,uint64_t>>(
+             new WeightedPriorityQueue<pair<spg_t,OpQueueItem>,uint64_t>(
                max_tok_per_prio, min_cost));
        } else if (opqueue == io_queue::prioritized) {
          pqueue = std::unique_ptr
-           <PrioritizedQueue<pair<spg_t,PGQueueable>,uint64_t>>(
-             new PrioritizedQueue<pair<spg_t,PGQueueable>,uint64_t>(
+           <PrioritizedQueue<pair<spg_t,OpQueueItem>,uint64_t>>(
+             new PrioritizedQueue<pair<spg_t,OpQueueItem>,uint64_t>(
                max_tok_per_prio, min_cost));
        } else if (opqueue == io_queue::mclock_opclass) {
-         pqueue = std::unique_ptr
-           <ceph::mClockOpClassQueue>(new ceph::mClockOpClassQueue(cct));
+         pqueue = ceph::make_unique<ceph::mClockOpClassQueue>(cct);
        } else if (opqueue == io_queue::mclock_client) {
-         pqueue = std::unique_ptr
-           <ceph::mClockClientQueue>(new ceph::mClockClientQueue(cct));
+         pqueue = ceph::make_unique<ceph::mClockClientQueue>(cct);
        }
       }
     }; // struct ShardData
@@ -1701,7 +1700,7 @@ private:
                time_t ti,
                time_t si,
                ShardedThreadPool* tp)
-      : ShardedThreadPool::ShardedWQ<pair<spg_t,PGQueueable>>(ti, si, tp),
+      : ShardedThreadPool::ShardedWQ<pair<spg_t,OpQueueItem>>(ti, si, tp),
         osd(o),
         num_shards(pnum_shards) {
       for (uint32_t i = 0; i < num_shards; i++) {
@@ -1740,10 +1739,10 @@ private:
     void _process(uint32_t thread_index, heartbeat_handle_d *hb) override;
 
     /// enqueue a new item
-    void _enqueue(pair <spg_t, PGQueueable> item) override;
+    void _enqueue(pair <spg_t, OpQueueItem> item) override;
 
     /// requeue an old item (at the front of the line)
-    void _enqueue_front(pair <spg_t, PGQueueable> item) override;
+    void _enqueue_front(pair <spg_t, OpQueueItem> item) override;
       
     void return_waiting_threads() override {
       for(uint32_t i = 0; i < num_shards; i++) {
@@ -1776,7 +1775,7 @@ private:
       uint64_t reserved_pushes_to_free;
       Pred(spg_t pg, list<OpRequestRef> *out_ops = 0)
        : pgid(pg), out_ops(out_ops), reserved_pushes_to_free(0) {}
-      void accumulate(const PGQueueable &op) {
+      void accumulate(const OpQueueItem &op) {
        reserved_pushes_to_free += op.get_reserved_pushes();
        if (out_ops) {
          boost::optional<OpRequestRef> mop = op.maybe_get_op();
@@ -1784,7 +1783,7 @@ private:
            out_ops->push_front(*mop);
        }
       }
-      bool operator()(const pair<spg_t, PGQueueable> &op) {
+      bool operator()(const pair<spg_t, OpQueueItem> &op) {
        if (op.first == pgid) {
          accumulate(op.second);
          return true;
diff --git a/src/osd/OpQueueItem.cc b/src/osd/OpQueueItem.cc
new file mode 100644 (file)
index 0000000..bf94416
--- /dev/null
@@ -0,0 +1,35 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2016 Red Hat Inc.
+ *
+ * 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
+ * Foundation.  See file COPYING.
+ *
+ */
+
+
+#include "PG.h"
+#include "OpQueueItem.h"
+#include "OSD.h"
+
+
+void OpQueueItem::RunVis::operator()(const OpRequestRef &op) {
+  osd->dequeue_op(pg, op, handle);
+}
+
+void OpQueueItem::RunVis::operator()(const PGSnapTrim &op) {
+  pg->snap_trimmer(op.epoch_queued);
+}
+
+void OpQueueItem::RunVis::operator()(const PGScrub &op) {
+  pg->scrub(op.epoch_queued, handle);
+}
+
+void OpQueueItem::RunVis::operator()(const PGRecovery &op) {
+  osd->do_recovery(pg.get(), op.epoch_queued, op.reserved_pushes, handle);
+}
diff --git a/src/osd/OpQueueItem.h b/src/osd/OpQueueItem.h
new file mode 100644 (file)
index 0000000..97c4c5a
--- /dev/null
@@ -0,0 +1,148 @@
+// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
+// vim: ts=8 sw=2 smarttab
+/*
+ * Ceph - scalable distributed file system
+ *
+ * Copyright (C) 2016 Red Hat Inc.
+ *
+ * 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
+ * Foundation.  See file COPYING.
+ *
+ */
+
+
+#pragma once
+
+#include <ostream>
+
+#include "include/types.h"
+#include "include/utime.h"
+#include "osd/OpRequest.h"
+#include "osd/PG.h"
+
+
+class OSD;
+
+
+struct PGScrub {
+  epoch_t epoch_queued;
+  explicit PGScrub(epoch_t e) : epoch_queued(e) {}
+  ostream &operator<<(ostream &rhs) {
+    return rhs << "PGScrub";
+  }
+};
+
+struct PGSnapTrim {
+  epoch_t epoch_queued;
+  explicit PGSnapTrim(epoch_t e) : epoch_queued(e) {}
+  ostream &operator<<(ostream &rhs) {
+    return rhs << "PGSnapTrim";
+  }
+};
+
+struct PGRecovery {
+  epoch_t epoch_queued;
+  uint64_t reserved_pushes;
+  PGRecovery(epoch_t e, uint64_t reserved_pushes)
+    : epoch_queued(e), reserved_pushes(reserved_pushes) {}
+  ostream &operator<<(ostream &rhs) {
+    return rhs << "PGRecovery(epoch=" << epoch_queued
+              << ", reserved_pushes: " << reserved_pushes << ")";
+  }
+};
+
+
+class OpQueueItem {
+  typedef boost::variant<
+    OpRequestRef,
+    PGSnapTrim,
+    PGScrub,
+    PGRecovery
+    > QVariant;
+  QVariant qvariant;
+  int cost;
+  unsigned priority;
+  utime_t start_time;
+  uint64_t owner;  ///< global id (e.g., client.XXX)
+  epoch_t map_epoch;    ///< an epoch we expect the PG to exist in
+
+  struct RunVis : public boost::static_visitor<> {
+    OSD *osd;
+    PGRef &pg;
+    ThreadPool::TPHandle &handle;
+    RunVis(OSD *osd, PGRef &pg, ThreadPool::TPHandle &handle)
+      : osd(osd), pg(pg), handle(handle) {}
+    void operator()(const OpRequestRef &op);
+    void operator()(const PGSnapTrim &op);
+    void operator()(const PGScrub &op);
+    void operator()(const PGRecovery &op);
+  }; // struct RunVis
+
+  struct StringifyVis : public boost::static_visitor<std::string> {
+    std::string operator()(const OpRequestRef &op) {
+      return stringify(op);
+    }
+    std::string operator()(const PGSnapTrim &op) {
+      return "PGSnapTrim";
+    }
+    std::string operator()(const PGScrub &op) {
+      return "PGScrub";
+    }
+    std::string operator()(const PGRecovery &op) {
+      return "PGRecovery";
+    }
+  };
+
+  friend ostream& operator<<(ostream& out, const OpQueueItem& q) {
+    StringifyVis v;
+    return out << "OpQueueItem(" << boost::apply_visitor(v, q.qvariant)
+              << " prio " << q.priority << " cost " << q.cost
+              << " e" << q.map_epoch << ")";
+  }
+
+public:
+
+  OpQueueItem(OpRequestRef op, epoch_t e)
+    : qvariant(op), cost(op->get_req()->get_cost()),
+      priority(op->get_req()->get_priority()),
+      start_time(op->get_req()->get_recv_stamp()),
+      owner(op->get_req()->get_source().num()),
+      map_epoch(e)
+    {}
+  OpQueueItem(
+    const PGSnapTrim &op, int cost, unsigned priority, utime_t start_time,
+    uint64_t owner, epoch_t e)
+    : qvariant(op), cost(cost), priority(priority), start_time(start_time),
+      owner(owner), map_epoch(e) {}
+  OpQueueItem(
+    const PGScrub &op, int cost, unsigned priority, utime_t start_time,
+    uint64_t owner, epoch_t e)
+    : qvariant(op), cost(cost), priority(priority), start_time(start_time),
+      owner(owner), map_epoch(e) {}
+  OpQueueItem(
+    const PGRecovery &op, int cost, unsigned priority, utime_t start_time,
+    uint64_t owner, epoch_t e)
+    : qvariant(op), cost(cost), priority(priority), start_time(start_time),
+      owner(owner), map_epoch(e) {}
+
+  const boost::optional<OpRequestRef> maybe_get_op() const {
+    const OpRequestRef *op = boost::get<OpRequestRef>(&qvariant);
+    return op ? OpRequestRef(*op) : boost::optional<OpRequestRef>();
+  }
+  uint64_t get_reserved_pushes() const {
+    const PGRecovery *op = boost::get<PGRecovery>(&qvariant);
+    return op ? op->reserved_pushes : 0;
+  }
+  void run(OSD *osd, PGRef &pg, ThreadPool::TPHandle &handle) {
+    RunVis v(osd, pg, handle);
+    boost::apply_visitor(v, qvariant);
+  }
+  unsigned get_priority() const { return priority; }
+  int get_cost() const { return cost; }
+  utime_t get_start_time() const { return start_time; }
+  uint64_t get_owner() const { return owner; }
+  epoch_t get_map_epoch() const { return map_epoch; }
+  const QVariant& get_variant() const { return qvariant; }
+}; // struct OpQueueItem
index 04893e987fd98c35de1ffc8945bc7ef5d7586579..509537d99537a16b269f386eff6b1768b11fbc62 100644 (file)
@@ -3457,7 +3457,7 @@ void PG::requeue_op(OpRequestRef op)
     p->second.push_front(op);
   } else {
     dout(20) << __func__ << " " << op << dendl;
-    osd->enqueue_front(info.pgid, PGQueueable(op, get_osdmap()->get_epoch()));
+    osd->enqueue_front(info.pgid, OpQueueItem(op, get_osdmap()->get_epoch()));
   }
 }
 
@@ -3473,7 +3473,7 @@ void PG::requeue_ops(list<OpRequestRef> &ls)
       p->second.push_front(*i);
     } else {
       dout(20) << __func__ << " " << *i << dendl;
-      osd->enqueue_front(info.pgid, PGQueueable(*i, get_osdmap()->get_epoch()));
+      osd->enqueue_front(info.pgid, OpQueueItem(*i, get_osdmap()->get_epoch()));
     }
   }
   ls.clear();
@@ -3492,7 +3492,7 @@ void PG::requeue_map_waiters()
     } else {
       dout(20) << __func__ << " " << p->first << " " << p->second << dendl;
       for (auto q = p->second.rbegin(); q != p->second.rend(); ++q) {
-       osd->enqueue_front(info.pgid, PGQueueable(*q, epoch));
+       osd->enqueue_front(info.pgid, OpQueueItem(*q, epoch));
       }
       p = waiting_for_map.erase(p);
     }
diff --git a/src/osd/PGQueueable.cc b/src/osd/PGQueueable.cc
deleted file mode 100644 (file)
index 844cdfc..0000000
+++ /dev/null
@@ -1,35 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2016 Red Hat Inc.
- *
- * 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
- * Foundation.  See file COPYING.
- *
- */
-
-
-#include "PG.h"
-#include "PGQueueable.h"
-#include "OSD.h"
-
-
-void PGQueueable::RunVis::operator()(const OpRequestRef &op) {
-  osd->dequeue_op(pg, op, handle);
-}
-
-void PGQueueable::RunVis::operator()(const PGSnapTrim &op) {
-  pg->snap_trimmer(op.epoch_queued);
-}
-
-void PGQueueable::RunVis::operator()(const PGScrub &op) {
-  pg->scrub(op.epoch_queued, handle);
-}
-
-void PGQueueable::RunVis::operator()(const PGRecovery &op) {
-  osd->do_recovery(pg.get(), op.epoch_queued, op.reserved_pushes, handle);
-}
diff --git a/src/osd/PGQueueable.h b/src/osd/PGQueueable.h
deleted file mode 100644 (file)
index 9eeadc0..0000000
+++ /dev/null
@@ -1,148 +0,0 @@
-// -*- mode:C++; tab-width:8; c-basic-offset:2; indent-tabs-mode:t -*-
-// vim: ts=8 sw=2 smarttab
-/*
- * Ceph - scalable distributed file system
- *
- * Copyright (C) 2016 Red Hat Inc.
- *
- * 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
- * Foundation.  See file COPYING.
- *
- */
-
-
-#pragma once
-
-#include <ostream>
-
-#include "include/types.h"
-#include "include/utime.h"
-#include "osd/OpRequest.h"
-#include "osd/PG.h"
-
-
-class OSD;
-
-
-struct PGScrub {
-  epoch_t epoch_queued;
-  explicit PGScrub(epoch_t e) : epoch_queued(e) {}
-  ostream &operator<<(ostream &rhs) {
-    return rhs << "PGScrub";
-  }
-};
-
-struct PGSnapTrim {
-  epoch_t epoch_queued;
-  explicit PGSnapTrim(epoch_t e) : epoch_queued(e) {}
-  ostream &operator<<(ostream &rhs) {
-    return rhs << "PGSnapTrim";
-  }
-};
-
-struct PGRecovery {
-  epoch_t epoch_queued;
-  uint64_t reserved_pushes;
-  PGRecovery(epoch_t e, uint64_t reserved_pushes)
-    : epoch_queued(e), reserved_pushes(reserved_pushes) {}
-  ostream &operator<<(ostream &rhs) {
-    return rhs << "PGRecovery(epoch=" << epoch_queued
-              << ", reserved_pushes: " << reserved_pushes << ")";
-  }
-};
-
-
-class PGQueueable {
-  typedef boost::variant<
-    OpRequestRef,
-    PGSnapTrim,
-    PGScrub,
-    PGRecovery
-    > QVariant;
-  QVariant qvariant;
-  int cost;
-  unsigned priority;
-  utime_t start_time;
-  uint64_t owner;  ///< global id (e.g., client.XXX)
-  epoch_t map_epoch;    ///< an epoch we expect the PG to exist in
-
-  struct RunVis : public boost::static_visitor<> {
-    OSD *osd;
-    PGRef &pg;
-    ThreadPool::TPHandle &handle;
-    RunVis(OSD *osd, PGRef &pg, ThreadPool::TPHandle &handle)
-      : osd(osd), pg(pg), handle(handle) {}
-    void operator()(const OpRequestRef &op);
-    void operator()(const PGSnapTrim &op);
-    void operator()(const PGScrub &op);
-    void operator()(const PGRecovery &op);
-  }; // struct RunVis
-
-  struct StringifyVis : public boost::static_visitor<std::string> {
-    std::string operator()(const OpRequestRef &op) {
-      return stringify(op);
-    }
-    std::string operator()(const PGSnapTrim &op) {
-      return "PGSnapTrim";
-    }
-    std::string operator()(const PGScrub &op) {
-      return "PGScrub";
-    }
-    std::string operator()(const PGRecovery &op) {
-      return "PGRecovery";
-    }
-  };
-
-  friend ostream& operator<<(ostream& out, const PGQueueable& q) {
-    StringifyVis v;
-    return out << "PGQueueable(" << boost::apply_visitor(v, q.qvariant)
-              << " prio " << q.priority << " cost " << q.cost
-              << " e" << q.map_epoch << ")";
-  }
-
-public:
-
-  PGQueueable(OpRequestRef op, epoch_t e)
-    : qvariant(op), cost(op->get_req()->get_cost()),
-      priority(op->get_req()->get_priority()),
-      start_time(op->get_req()->get_recv_stamp()),
-      owner(op->get_req()->get_source().num()),
-      map_epoch(e)
-    {}
-  PGQueueable(
-    const PGSnapTrim &op, int cost, unsigned priority, utime_t start_time,
-    uint64_t owner, epoch_t e)
-    : qvariant(op), cost(cost), priority(priority), start_time(start_time),
-      owner(owner), map_epoch(e) {}
-  PGQueueable(
-    const PGScrub &op, int cost, unsigned priority, utime_t start_time,
-    uint64_t owner, epoch_t e)
-    : qvariant(op), cost(cost), priority(priority), start_time(start_time),
-      owner(owner), map_epoch(e) {}
-  PGQueueable(
-    const PGRecovery &op, int cost, unsigned priority, utime_t start_time,
-    uint64_t owner, epoch_t e)
-    : qvariant(op), cost(cost), priority(priority), start_time(start_time),
-      owner(owner), map_epoch(e) {}
-
-  const boost::optional<OpRequestRef> maybe_get_op() const {
-    const OpRequestRef *op = boost::get<OpRequestRef>(&qvariant);
-    return op ? OpRequestRef(*op) : boost::optional<OpRequestRef>();
-  }
-  uint64_t get_reserved_pushes() const {
-    const PGRecovery *op = boost::get<PGRecovery>(&qvariant);
-    return op ? op->reserved_pushes : 0;
-  }
-  void run(OSD *osd, PGRef &pg, ThreadPool::TPHandle &handle) {
-    RunVis v(osd, pg, handle);
-    boost::apply_visitor(v, qvariant);
-  }
-  unsigned get_priority() const { return priority; }
-  int get_cost() const { return cost; }
-  utime_t get_start_time() const { return start_time; }
-  uint64_t get_owner() const { return owner; }
-  epoch_t get_map_epoch() const { return map_epoch; }
-  const QVariant& get_variant() const { return qvariant; }
-}; // struct PGQueueable
index a73b8dd7bb91dc0db2e253a19cce1236c7f9b557..15231f2c101f20a923200880c9f4d7e8fe7158c8 100644 (file)
@@ -9062,7 +9062,7 @@ void PrimaryLogPG::op_applied(const eversion_t &applied_version)
            scrubber.active_rep_scrub->get_req())->scrub_to) {
        osd->enqueue_back(
          info.pgid,
-         PGQueueable(scrubber.active_rep_scrub, get_osdmap()->get_epoch()));
+         OpQueueItem(scrubber.active_rep_scrub, get_osdmap()->get_epoch()));
        scrubber.active_rep_scrub = OpRequestRef();
       }
     }
@@ -10299,7 +10299,7 @@ void PrimaryLogPG::_applied_recovered_object_replica()
        scrubber.active_rep_scrub->get_req())->chunky) {
     osd->enqueue_back(
       info.pgid,
-      PGQueueable(scrubber.active_rep_scrub, get_osdmap()->get_epoch()));
+      OpQueueItem(scrubber.active_rep_scrub, get_osdmap()->get_epoch()));
     scrubber.active_rep_scrub = OpRequestRef();
   }
   unlock();
index 3d8824914eb3a3ceb6fc42713ce47650fc125cfa..f14c7d647d37cefacb04827c1f596e181250dbeb 100644 (file)
 
 #include "common/config.h"
 #include "common/ceph_context.h"
-#include "osd/PGQueueable.h"
+#include "osd/OpQueueItem.h"
 
 #include "common/mClockPriorityQueue.h"
 
 
 namespace ceph {
 
-  using Request = std::pair<spg_t, PGQueueable>;
+  using Request = std::pair<spg_t, OpQueueItem>;
   using Client = uint64_t;
 
-
   // This class exists to bridge the ceph code, which treats the class
   // as the client, and the queue, where the class is
   // osd_op_type_t. So this adapter class will transform calls
index 6b87335b10946f16e861c7f0f6d9fd5b6f60212e..ccf3f578e67dcd2e7acf10a0cbbd836889d82f1b 100644 (file)
 
 #include "common/config.h"
 #include "common/ceph_context.h"
-#include "osd/PGQueueable.h"
+#include "osd/OpQueueItem.h"
 
 #include "common/mClockPriorityQueue.h"
 
 
 namespace ceph {
 
-  using Request = std::pair<spg_t, PGQueueable>;
+  using Request = std::pair<spg_t, OpQueueItem>;
   using Client = uint64_t;
 
-
   // This class exists to bridge the ceph code, which treats the class
   // as the client, and the queue, where the class is
   // osd_op_type_t. So this adapter class will transform calls
index 498acd9ddda927aa88c006c8295daffdb4c36c96..6bb9b9f0ee162369ca4de0e11de22c8ba38b09d3 100644 (file)
@@ -38,27 +38,27 @@ public:
 
 #if 0 // more work needed here
   Request create_client_op(epoch_t e, uint64_t owner) {
-    return Request(spg_t(), PGQueueable(OpRequestRef(), e));
+    return Request(spg_t(), OpQueueItem(OpRequestRef(), e));
   }
 #endif
 
   Request create_snaptrim(epoch_t e, uint64_t owner) {
     return Request(spg_t(),
-                  PGQueueable(PGSnapTrim(e),
+                  OpQueueItem(PGSnapTrim(e),
                               12, 12,
                               utime_t(), owner, e));
   }
 
   Request create_scrub(epoch_t e, uint64_t owner) {
     return Request(spg_t(),
-                  PGQueueable(PGScrub(e),
+                  OpQueueItem(PGScrub(e),
                               12, 12,
                               utime_t(), owner, e));
   }
 
   Request create_recovery(epoch_t e, uint64_t owner) {
     return Request(spg_t(),
-                  PGQueueable(PGRecovery(e, 64),
+                  OpQueueItem(PGRecovery(e, 64),
                               12, 12,
                               utime_t(), owner, e));
   }
index b18587f491a6e8a17323b5e9d898eaf3cef63547..44a0346d5f34cf513043ce66fbcab835fdc707de 100644 (file)
@@ -40,27 +40,27 @@ public:
 
 #if 0 // more work needed here
   Request create_client_op(epoch_t e, uint64_t owner) {
-    return Request(spg_t(), PGQueueable(OpRequestRef(), e));
+    return Request(spg_t(), OpQueueItem(OpRequestRef(), e));
   }
 #endif
 
   Request create_snaptrim(epoch_t e, uint64_t owner) {
     return Request(spg_t(),
-                  PGQueueable(PGSnapTrim(e),
+                  OpQueueItem(PGSnapTrim(e),
                               12, 12,
                               utime_t(), owner, e));
   }
 
   Request create_scrub(epoch_t e, uint64_t owner) {
     return Request(spg_t(),
-                  PGQueueable(PGScrub(e),
+                  OpQueueItem(PGScrub(e),
                               12, 12,
                               utime_t(), owner, e));
   }
 
   Request create_recovery(epoch_t e, uint64_t owner) {
     return Request(spg_t(),
-                  PGQueueable(PGRecovery(e, 64),
+                  OpQueueItem(PGRecovery(e, 64),
                               12, 12,
                               utime_t(), owner, e));
   }