]> git.apps.os.sepia.ceph.com Git - ceph.git/commitdiff
bluestore: revamp tracepoints, add sampling
authorSamuel Just <sjust@redhat.com>
Fri, 9 Aug 2019 19:00:26 +0000 (12:00 -0700)
committerSamuel Just <sjust@redhat.com>
Fri, 27 Sep 2019 22:55:18 +0000 (15:55 -0700)
This patch adds per-io bluestore specific tracepoints detailing the
throttle state at queue time as well as state latencies during
execution.  Additionally, bluestore_throttle_trace_rate will limit
the rate at which ios have tracepoints emitted.

Signed-off-by: Samuel Just <sjust@redhat.com>
src/ceph_osd.cc
src/common/ceph_time.h
src/common/options.cc
src/os/bluestore/BlueStore.cc
src/os/bluestore/BlueStore.h
src/test/fio/fio_ceph_objectstore.cc
src/tracing/CMakeLists.txt
src/tracing/bluestore.c [new file with mode: 0644]
src/tracing/bluestore.tp [new file with mode: 0644]

index 145f404f42352240fdbebd3c1917c41c0906f314..b24ca32ab02562e0d7736181b43a02720af1c862 100644 (file)
@@ -59,6 +59,8 @@ TracepointProvider::Traits osd_tracepoint_traits("libosd_tp.so",
                                                  "osd_tracing");
 TracepointProvider::Traits os_tracepoint_traits("libos_tp.so",
                                                 "osd_objectstore_tracing");
+TracepointProvider::Traits bluestore_tracepoint_traits("libbluestore_tp.so",
+                                                      "bluestore_tracing");
 #ifdef WITH_OSD_INSTRUMENT_FUNCTIONS
 TracepointProvider::Traits cyg_profile_traits("libcyg_profile_tp.so",
                                                  "osd_function_tracing");
@@ -655,6 +657,7 @@ flushjournal_out:
 
   TracepointProvider::initialize<osd_tracepoint_traits>(g_ceph_context);
   TracepointProvider::initialize<os_tracepoint_traits>(g_ceph_context);
+  TracepointProvider::initialize<bluestore_tracepoint_traits>(g_ceph_context);
 #ifdef WITH_OSD_INSTRUMENT_FUNCTIONS
   TracepointProvider::initialize<cyg_profile_traits>(g_ceph_context);
 #endif
index 74236bfd4ef288f64b4bf0263f0a27edeed9b862..ae4a17c781296e851fdce9d71423b04cc74e81f9 100644 (file)
@@ -503,6 +503,20 @@ struct converts_to_timespec<Clock, std::void_t<decltype(
 template <typename Clock>
 constexpr bool converts_to_timespec_v = converts_to_timespec<Clock>::value;
 
+template<typename Rep, typename T>
+static Rep to_seconds(T t) {
+  return std::chrono::duration_cast<
+    std::chrono::duration<Rep>>(t).count();
+}
+
+template<typename Rep, typename T>
+static Rep to_microseconds(T t) {
+  return std::chrono::duration_cast<
+    std::chrono::duration<
+      Rep,
+      std::micro>>(t).count();
+}
+
 } // namespace ceph
 
 #endif // COMMON_CEPH_TIME_H
index f60182b5fa539a5402e72b55f9f72219d7534dae..db043b5ac3f56ea92453fdb9806bf3141101f8bb 100644 (file)
@@ -5564,6 +5564,14 @@ std::vector<Option> get_global_options() {
     .set_default(false)
     .set_description(""),
 
+    Option("bluestore_tracing", Option::TYPE_BOOL, Option::LEVEL_ADVANCED)
+    .set_default(false)
+    .set_description("Enable bluestore event tracing."),
+
+    Option("bluestore_throttle_trace_rate", Option::TYPE_FLOAT, Option::LEVEL_ADVANCED)
+    .set_default(0)
+    .set_description("Rate at which to sample bluestore transactions (per second)"),
+
     Option("debug_deliberately_leak_memory", Option::TYPE_BOOL, Option::LEVEL_DEV)
     .set_default(false)
     .set_description(""),
index 4c3110eff44eea38ea4241bc17411c723b378d81..3e462415e67f5ed3dff14efa98a9407b6f18b247 100644 (file)
 #include "common/blkdev.h"
 #include "common/numa.h"
 
+#if defined(WITH_LTTNG)
+#define TRACEPOINT_DEFINE
+#define TRACEPOINT_PROBE_DYNAMIC_LINKAGE
+#include "tracing/bluestore.h"
+#undef TRACEPOINT_PROBE_DYNAMIC_LINKAGE
+#undef TRACEPOINT_DEFINE
+#else
+#define tracepoint(...)
+#endif
+
 #define dout_context cct
 #define dout_subsys ceph_subsys_bluestore
 
@@ -4102,23 +4112,13 @@ void BlueStore::handle_discard(interval_set<uint64_t>& to_release)
 }
 
 BlueStore::BlueStore(CephContext *cct, const string& path)
-  : ObjectStore(cct, path),
-    bsthrottle(cct),
-    finisher(cct, "commit_finisher", "cfin"),
-    kv_sync_thread(this),
-    kv_finalize_thread(this),
-    mempool_thread(this)
-{
-  _init_logger();
-  cct->_conf.add_observer(this);
-  set_cache_shards(1);
-}
+  : BlueStore(cct, path, 0) {}
 
 BlueStore::BlueStore(CephContext *cct,
   const string& path,
   uint64_t _min_alloc_size)
   : ObjectStore(cct, path),
-    bsthrottle(cct),
+    throttle(cct),
     finisher(cct, "commit_finisher", "cfin"),
     kv_sync_thread(this),
     kv_finalize_thread(this),
@@ -4242,8 +4242,9 @@ void BlueStore::handle_conf_change(const ConfigProxy& conf,
     }
   }
   if (changed.count("bluestore_throttle_bytes") ||
-      changed.count("bluestore_throttle_deferred_bytes")) {
-    bsthrottle.reset_throttle(conf);
+      changed.count("bluestore_throttle_deferred_bytes") ||
+      changed.count("bluestore_throttle_trace_rate")) {
+    throttle.reset_throttle(conf);
   }
   if (changed.count("bluestore_max_defer_interval")) {
     if (bdev) {
@@ -10879,6 +10880,7 @@ void BlueStore::_txc_calc_cost(TransContext *txc)
   auto ios = 1 + txc->ioc.get_num_ios();
   auto cost = throttle_cost_per_io.load();
   txc->cost = ios * cost + txc->bytes;
+  txc->ios = ios;
   dout(10) << __func__ << " " << txc << " cost " << txc->cost << " ("
           << ios << " ios * " << cost << " + " << txc->bytes
           << " bytes)" << dendl;
@@ -10924,7 +10926,7 @@ void BlueStore::_txc_state_proc(TransContext *txc)
             << " " << txc->get_state_name() << dendl;
     switch (txc->state) {
     case TransContext::STATE_PREPARE:
-      bsthrottle.log_state_latency(*txc, logger, l_bluestore_state_prepare_lat);
+      throttle.log_state_latency(*txc, logger, l_bluestore_state_prepare_lat);
       if (txc->ioc.has_pending_aios()) {
        txc->state = TransContext::STATE_AIO_WAIT;
        txc->had_ios = true;
@@ -10935,7 +10937,7 @@ void BlueStore::_txc_state_proc(TransContext *txc)
 
     case TransContext::STATE_AIO_WAIT:
       {
-       utime_t lat = bsthrottle.log_state_latency(
+       utime_t lat = throttle.log_state_latency(
          *txc, logger, l_bluestore_state_aio_wait_lat);
        if (lat >= cct->_conf->bluestore_log_op_age) {
          dout(0) << __func__ << " slow aio_wait, txc = " << txc
@@ -10952,7 +10954,7 @@ void BlueStore::_txc_state_proc(TransContext *txc)
       if (txc->had_ios) {
        ++txc->osr->txc_with_unstable_io;
       }
-      bsthrottle.log_state_latency(*txc, logger, l_bluestore_state_io_done_lat);
+      throttle.log_state_latency(*txc, logger, l_bluestore_state_io_done_lat);
       txc->state = TransContext::STATE_KV_QUEUED;
       if (cct->_conf->bluestore_sync_submit_transaction) {
        if (txc->last_nid >= nid_max ||
@@ -11000,7 +11002,7 @@ void BlueStore::_txc_state_proc(TransContext *txc)
       // ** fall-thru **
 
     case TransContext::STATE_KV_DONE:
-      bsthrottle.log_state_latency(*txc, logger, l_bluestore_state_kv_done_lat);
+      throttle.log_state_latency(*txc, logger, l_bluestore_state_kv_done_lat);
       if (txc->deferred_txn) {
        txc->state = TransContext::STATE_DEFERRED_QUEUED;
        _deferred_queue(txc);
@@ -11010,12 +11012,12 @@ void BlueStore::_txc_state_proc(TransContext *txc)
       break;
 
     case TransContext::STATE_DEFERRED_CLEANUP:
-      bsthrottle.log_state_latency(*txc, logger, l_bluestore_state_deferred_cleanup_lat);
+      throttle.log_state_latency(*txc, logger, l_bluestore_state_deferred_cleanup_lat);
       txc->state = TransContext::STATE_FINISHING;
       // ** fall-thru **
 
     case TransContext::STATE_FINISHING:
-      bsthrottle.log_state_latency(*txc, logger, l_bluestore_state_finishing_lat);
+      throttle.log_state_latency(*txc, logger, l_bluestore_state_finishing_lat);
       _txc_finish(txc);
       return;
 
@@ -11174,8 +11176,24 @@ void BlueStore::_txc_apply_kv(TransContext *txc, bool sync_submit_transaction)
   ceph_assert(txc->state == TransContext::STATE_KV_QUEUED);
   txc->state = TransContext::STATE_KV_SUBMITTED;
   {
+#if defined(WITH_LTTNG)
+    auto start = mono_clock::now();
+#endif
+
     int r = cct->_conf->bluestore_debug_omit_kv_commit ? 0 : db->submit_transaction(txc->t);
     ceph_assert(r == 0);
+
+#if defined(WITH_LTTNG)
+    if (txc->tracing) {
+      tracepoint(
+       bluestore,
+       transaction_kv_submit_latency,
+       txc->osr->get_sequencer_id(),
+       txc->seq,
+       sync_submit_transaction,
+       ceph::to_seconds<double>(mono_clock::now() - start));
+    }
+#endif
   }
 
   for (auto ls : { &txc->onodes, &txc->modified_objects }) {
@@ -11193,7 +11211,7 @@ void BlueStore::_txc_apply_kv(TransContext *txc, bool sync_submit_transaction)
 void BlueStore::_txc_committed_kv(TransContext *txc)
 {
   dout(20) << __func__ << " txc " << txc << dendl;
-  bsthrottle.complete_kv(*txc);
+  throttle.complete_kv(*txc);
   {
     std::lock_guard l(txc->osr->qlock);
     txc->state = TransContext::STATE_KV_DONE;
@@ -11203,7 +11221,7 @@ void BlueStore::_txc_committed_kv(TransContext *txc)
       finisher.queue(txc->oncommits);
     }
   }
-  bsthrottle.log_state_latency(*txc, logger, l_bluestore_state_kv_committing_lat);
+  throttle.log_state_latency(*txc, logger, l_bluestore_state_kv_committing_lat);
   log_latency_fn(
     __func__,
     l_bluestore_commit_lat,
@@ -11277,8 +11295,8 @@ void BlueStore::_txc_finish(TransContext *txc)
     auto txc = &releasing_txc.front();
     _txc_release_alloc(txc);
     releasing_txc.pop_front();
-    bsthrottle.log_state_latency(*txc, logger, l_bluestore_state_done_lat);
-    bsthrottle.complete(*txc);
+    throttle.log_state_latency(*txc, logger, l_bluestore_state_done_lat);
+    throttle.complete(*txc);
     delete txc;
   }
 
@@ -11339,7 +11357,7 @@ void BlueStore::_osr_attach(Collection *c)
     std::lock_guard l(zombie_osr_lock);
     auto p = zombie_osr_set.find(c->cid);
     if (p == zombie_osr_set.end()) {
-      c->osr = ceph::make_ref<OpSequencer>(this, c->cid);
+      c->osr = ceph::make_ref<OpSequencer>(this, next_sequencer_id++, c->cid);
       ldout(cct, 10) << __func__ << " " << c->cid
                     << " fresh osr " << c->osr << dendl;
     } else {
@@ -11655,7 +11673,7 @@ void BlueStore::_kv_sync_thread()
       // iteration there will already be ops awake.  otherwise, we
       // end up going to sleep, and then wake up when the very first
       // transaction is ready for commit.
-      bsthrottle.release_kv_throttle(costs);
+      throttle.release_kv_throttle(costs);
 
       if (bluefs &&
          after_flush - bluefs_last_balance >
@@ -11676,6 +11694,9 @@ void BlueStore::_kv_sync_thread()
        }
       }
 
+#if defined(WITH_LTTNG)
+      auto sync_start = mono_clock::now();
+#endif
       // submit synct synchronously (block and wait for it to commit)
       int r = cct->_conf->bluestore_debug_omit_kv_commit ? 0 : db->submit_transaction_sync(synct);
       ceph_assert(r == 0);
@@ -11683,6 +11704,23 @@ void BlueStore::_kv_sync_thread()
       int committing_size = kv_committing.size();
       int deferred_size = deferred_stable.size();
 
+#if defined(WITH_LTTNG)
+      double sync_latency = ceph::to_seconds<double>(sync_start - mono_clock::now());
+      for (auto txc: kv_committing) {
+       if (txc->tracing) {
+         tracepoint(
+           bluestore,
+           transaction_kv_sync_latency,
+           txc->osr->get_sequencer_id(),
+           txc->seq,
+           kv_committing.size(),
+           deferred_done.size(),
+           deferred_stable.size(),
+           sync_latency);
+       }
+      }
+#endif
+
       {
        std::unique_lock m{kv_finalize_lock};
        if (kv_committing_to_finalize.empty()) {
@@ -11823,7 +11861,7 @@ void BlueStore::_kv_finalize_thread()
 
       if (!deferred_aggressive) {
        if (deferred_queue_size >= deferred_batch_ops.load() ||
-           bsthrottle.should_submit_deferred()) {
+           throttle.should_submit_deferred()) {
          deferred_try_submit();
        }
       }
@@ -11932,7 +11970,7 @@ void BlueStore::_deferred_submit_unlock(OpSequencer *osr)
   deferred_lock.unlock();
 
   for (auto& txc : b->txcs) {
-    bsthrottle.log_state_latency(txc, logger, l_bluestore_state_deferred_queued_lat);
+    throttle.log_state_latency(txc, logger, l_bluestore_state_deferred_queued_lat);
   }
   uint64_t start = 0, pos = 0;
   bufferlist bl;
@@ -12010,12 +12048,12 @@ void BlueStore::_deferred_aio_finish(OpSequencer *osr)
     {
       for (auto& i : b->txcs) {
        TransContext *txc = &i;
-       bsthrottle.log_state_latency(*txc, logger, l_bluestore_state_deferred_aio_wait_lat);
+       throttle.log_state_latency(*txc, logger, l_bluestore_state_deferred_aio_wait_lat);
        txc->state = TransContext::STATE_DEFERRED_CLEANUP;
        costs += txc->cost;
       }
     }
-    bsthrottle.release_deferred_throttle(costs);
+    throttle.release_deferred_throttle(costs);
   }
 
   {
@@ -12125,7 +12163,7 @@ int BlueStore::queue_transactions(
 
   auto tstart = mono_clock::now();
 
-  if (!bsthrottle.try_start_transaction(
+  if (!throttle.try_start_transaction(
        *db,
        *txc,
        tstart)) {
@@ -12142,7 +12180,7 @@ int BlueStore::queue_transactions(
        kv_cond.notify_one();
       }
     }
-    bsthrottle.finish_start_transaction(*db, *txc, tstart);
+    throttle.finish_start_transaction(*db, *txc, tstart);
     --deferred_aggressive;
   }
   auto tend = mono_clock::now();
@@ -14741,6 +14779,84 @@ void BlueStore::log_latency_fn(
   }
 }
 
+#if defined(WITH_LTTNG)
+void BlueStore::BlueStoreThrottle::emit_initial_tracepoint(
+  KeyValueDB &db,
+  TransContext &txc,
+  mono_clock::time_point start_throttle_acquire)
+{
+  pending_kv_ios += txc.ios;
+  if (txc.deferred_txn) {
+    pending_deferred_ios += txc.ios;
+  }
+
+  uint64_t started = 0;
+  uint64_t completed = 0;
+  if (should_trace(&started, &completed)) {
+    txc.tracing = true;
+    uint64_t rocksdb_base_level,
+      rocksdb_estimate_pending_compaction_bytes,
+      rocksdb_cur_size_all_mem_tables,
+      rocksdb_compaction_pending,
+      rocksdb_mem_table_flush_pending,
+      rocksdb_num_running_compactions,
+      rocksdb_num_running_flushes,
+      rocksdb_actual_delayed_write_rate;
+    db.get_property(
+      "rocksdb.base-level",
+      &rocksdb_base_level);
+    db.get_property(
+      "rocksdb.estimate-pending-compaction-bytes",
+      &rocksdb_estimate_pending_compaction_bytes);
+    db.get_property(
+      "rocksdb.cur-size-all-mem-tables",
+      &rocksdb_cur_size_all_mem_tables);
+    db.get_property(
+      "rocksdb.compaction-pending",
+      &rocksdb_compaction_pending);
+    db.get_property(
+      "rocksdb.mem-table-flush-pending",
+      &rocksdb_mem_table_flush_pending);
+    db.get_property(
+      "rocksdb.num-running-compactions",
+      &rocksdb_num_running_compactions);
+    db.get_property(
+      "rocksdb.num-running-flushes",
+      &rocksdb_num_running_flushes);
+    db.get_property(
+      "rocksdb.actual-delayed-write-rate",
+      &rocksdb_actual_delayed_write_rate);
+
+  
+    tracepoint(
+      bluestore,
+      transaction_initial_state,
+      txc.osr->get_sequencer_id(),
+      txc.seq,
+      throttle_bytes.get_current(),
+      throttle_deferred_bytes.get_current(),
+      pending_kv_ios,
+      pending_deferred_ios,
+      started,
+      completed,
+      ceph::to_seconds<double>(mono_clock::now() - start_throttle_acquire));
+
+    tracepoint(
+      bluestore,
+      transaction_initial_state_rocksdb,
+      txc.osr->get_sequencer_id(),
+      txc.seq,
+      rocksdb_base_level,
+      rocksdb_estimate_pending_compaction_bytes,
+      rocksdb_cur_size_all_mem_tables,
+      rocksdb_compaction_pending,
+      rocksdb_mem_table_flush_pending,
+      rocksdb_num_running_compactions,
+      rocksdb_num_running_flushes,
+      rocksdb_actual_delayed_write_rate);
+  }
+}
+#endif
 
 utime_t BlueStore::BlueStoreThrottle::log_state_latency(
   TransContext &txc, PerfCounters *logger, int state)
@@ -14748,6 +14864,20 @@ utime_t BlueStore::BlueStoreThrottle::log_state_latency(
   utime_t now = ceph_clock_now();
   utime_t lat = now - txc.last_stamp;
   logger->tinc(state, lat);
+#if defined(WITH_LTTNG)
+  if (txc.tracing &&
+      state >= l_bluestore_state_prepare_lat &&
+      state <= l_bluestore_state_done_lat) {
+    OID_ELAPSED("", lat.to_nsec() / 1000.0, txc.get_state_latency_name(state));
+    tracepoint(
+      bluestore,
+      transaction_state_duration,
+      txc.osr->get_sequencer_id(),
+      txc.seq,
+      state,
+      (double)lat);
+  }
+#endif
   txc.last_stamp = now;
   return lat;
 }
@@ -14760,6 +14890,7 @@ bool BlueStore::BlueStoreThrottle::try_start_transaction(
   throttle_bytes.get(txc.cost);
 
   if (!txc.deferred_txn || throttle_deferred_bytes.get_or_fail(txc.cost)) {
+    emit_initial_tracepoint(db, txc, start_throttle_acquire);
     return true;
   } else {
     return false;
@@ -14773,8 +14904,44 @@ void BlueStore::BlueStoreThrottle::finish_start_transaction(
 {
   ceph_assert(txc.deferred_txn);
   throttle_deferred_bytes.get(txc.cost);
+  emit_initial_tracepoint(db, txc, start_throttle_acquire);
 }
 
+#if defined(WITH_LTTNG)
+void BlueStore::BlueStoreThrottle::complete_kv(TransContext &txc)
+{
+  pending_kv_ios -= 1;
+  ios_completed_since_last_traced++;
+  if (txc.tracing) {
+    tracepoint(
+      bluestore,
+      transaction_commit_latency,
+      txc.osr->get_sequencer_id(),
+      txc.seq,
+      ((double)ceph_clock_now()) - ((double)txc.start));
+  }
+}
+#endif
+
+#if defined(WITH_LTTNG)
+void BlueStore::BlueStoreThrottle::complete(TransContext &txc)
+{
+  if (txc.deferred_txn) {
+    pending_deferred_ios -= 1;
+  }
+  if (txc.tracing) {
+    utime_t now = ceph_clock_now();
+    double usecs = ((double)(now.to_nsec()-txc.start.to_nsec()))/1000;
+    tracepoint(
+      bluestore,
+      transaction_total_duration,
+      txc.osr->get_sequencer_id(),
+      txc.seq,
+      usecs);
+  }
+}
+#endif
+
 // DB key value Histogram
 #define KEY_SLAB 32
 #define VALUE_SLAB 64
index a06c7633c2c51b9b8058be1e0b02c54d4ba490cf..9baeca02188298d260f21d8f8a1460a62ddf74a3 100644 (file)
@@ -20,6 +20,8 @@
 #include <unistd.h>
 
 #include <atomic>
+#include <chrono>
+#include <ratio>
 #include <mutex>
 #include <condition_variable>
 
 #include <boost/intrusive/set.hpp>
 #include <boost/functional/hash.hpp>
 #include <boost/dynamic_bitset.hpp>
+#include <boost/circular_buffer.hpp>
 
 #include "include/cpp-btree/btree_set.h"
 
 #include "include/ceph_assert.h"
 #include "include/unordered_map.h"
 #include "include/mempool.h"
+#include "include/hash.h"
 #include "common/bloom_filter.hpp"
 #include "common/Finisher.h"
 #include "common/ceph_mutex.h"
@@ -1494,7 +1498,7 @@ public:
       return "???";
     }
 
-#if defined(WITH_LTTNG) && defined(WITH_EVENTTRACE)
+#if defined(WITH_LTTNG)
     const char *get_state_latency_name(int state) {
       switch (state) {
       case l_bluestore_state_prepare_lat: return "prepare";
@@ -1516,7 +1520,7 @@ public:
     OpSequencerRef osr;  // this should be ch->osr
     boost::intrusive::list_member_hook<> sequencer_item;
 
-    uint64_t bytes = 0, cost = 0;
+    uint64_t bytes = 0, ios = 0, cost = 0;
 
     set<OnodeRef> onodes;     ///< these need to be updated/written
     set<OnodeRef> modified_objects;  ///< objects we modified (and need a ref)
@@ -1544,6 +1548,10 @@ public:
     uint64_t last_nid = 0;     ///< if non-zero, highest new nid we allocated
     uint64_t last_blobid = 0;  ///< if non-zero, highest new blobid we allocated
 
+#if defined(WITH_LTTNG)
+    bool tracing = false;
+#endif
+
     explicit TransContext(CephContext* cct, Collection *c, OpSequencer *o,
                          list<Context*> *on_commits)
       : ch(c),
@@ -1584,21 +1592,82 @@ public:
     }
   };
 
-
   class BlueStoreThrottle {
+#if defined(WITH_LTTNG)
+    const std::chrono::time_point<mono_clock> time_base = mono_clock::now();
+
+    // Time of last chosen io (microseconds)
+    std::atomic<uint64_t> previous_emitted_tp_time_mono_mcs = {0};
+    std::atomic<uint64_t> ios_started_since_last_traced = {0};
+    std::atomic<uint64_t> ios_completed_since_last_traced = {0};
+
+    std::atomic_uint pending_kv_ios = {0};
+    std::atomic_uint pending_deferred_ios = {0};
+
+    // Min period between trace points (microseconds)
+    std::atomic<uint64_t> trace_period_mcs = {0};
+
+    bool should_trace(
+      uint64_t *started,
+      uint64_t *completed) {
+      uint64_t min_period_mcs = trace_period_mcs.load(
+       std::memory_order_relaxed);
+
+      if (min_period_mcs == 0) {
+       *started = 1;
+       *completed = ios_completed_since_last_traced.exchange(0);
+       return true;
+      } else {
+       ios_started_since_last_traced++;
+       auto now_mcs = ceph::to_microseconds<uint64_t>(
+         mono_clock::now() - time_base);
+       uint64_t previous_mcs = previous_emitted_tp_time_mono_mcs;
+       uint64_t period_mcs = now_mcs - previous_mcs;
+       if (period_mcs > min_period_mcs) {
+         if (previous_emitted_tp_time_mono_mcs.compare_exchange_strong(
+               previous_mcs, now_mcs)) {
+           // This would be racy at a sufficiently extreme trace rate, but isn't
+           // worth the overhead of doing it more carefully.
+           *started = ios_started_since_last_traced.exchange(0);
+           *completed = ios_completed_since_last_traced.exchange(0);
+           return true;
+         }
+       }
+       return false;
+      }
+    }
+#endif
+
+#if defined(WITH_LTTNG)
+    void emit_initial_tracepoint(
+      KeyValueDB &db,
+      TransContext &txc,
+      mono_clock::time_point);
+#else
+    void emit_initial_tracepoint(
+      KeyValueDB &db,
+      TransContext &txc,
+      mono_clock::time_point) {}
+#endif
+
     Throttle throttle_bytes;           ///< submit to commit
     Throttle throttle_deferred_bytes;  ///< submit to deferred complete
 
-
   public:
     BlueStoreThrottle(CephContext *cct) :
-      throttle_bytes(cct, "bluestore_throttle_bytes",
-                    cct->_conf->bluestore_throttle_bytes),
-      throttle_deferred_bytes(cct, "bluestore_throttle_deferred_bytes",
-                             cct->_conf->bluestore_throttle_bytes +
-                             cct->_conf->bluestore_throttle_deferred_bytes)
-    {}
+      throttle_bytes(cct, "bluestore_throttle_bytes", 0),
+      throttle_deferred_bytes(cct, "bluestore_throttle_deferred_bytes", 0)
+    {
+      reset_throttle(cct->_conf);
+    }
 
+#if defined(WITH_LTTNG)
+    void complete_kv(TransContext &txc);
+    void complete(TransContext &txc);
+#else
+    void complete_kv(TransContext &txc) {}
+    void complete(TransContext &txc) {}
+#endif
 
     utime_t log_state_latency(
       TransContext &txc, PerfCounters *logger, int state);
@@ -1624,8 +1693,12 @@ public:
       throttle_deferred_bytes.reset_max(
        conf->bluestore_throttle_bytes +
        conf->bluestore_throttle_deferred_bytes);
+#if defined(WITH_LTTNG)
+      double rate = conf.get_val<double>("bluestore_throttle_trace_rate");
+      trace_period_mcs = rate > 0 ? floor((1/rate) * 1000000.0) : 0;
+#endif
     }
-  } bsthrottle;
+  } throttle;
 
   typedef boost::intrusive::list<
     TransContext,
@@ -1694,6 +1767,12 @@ public:
 
     std::atomic_bool zombie = {false};    ///< in zombie_osr set (collection going away)
 
+    const uint32_t sequencer_id;
+
+    uint32_t get_sequencer_id() const {
+      return sequencer_id;
+    }
+
     void queue_new(TransContext *txc) {
       std::lock_guard l(qlock);
       txc->seq = ++last_seq;
@@ -1776,9 +1855,9 @@ public:
     }
   private:
     FRIEND_MAKE_REF(OpSequencer);
-    OpSequencer(BlueStore *store, const coll_t& c)
+    OpSequencer(BlueStore *store, uint32_t sequencer_id, const coll_t& c)
       : RefCountedObject(store->cct),
-       store(store), cid(c) {
+       store(store), cid(c), sequencer_id(sequencer_id) {
     }
     ~OpSequencer() {
       ceph_assert(q.empty());
@@ -1860,6 +1939,7 @@ private:
 
   /// protect zombie_osr_set
   ceph::mutex zombie_osr_lock = ceph::make_mutex("BlueStore::zombie_osr_lock");
+  uint32_t next_sequencer_id = 0;
   std::map<coll_t,OpSequencerRef> zombie_osr_set; ///< set of OpSequencers for deleted collections
 
   std::atomic<uint64_t> nid_last = {0};
@@ -1914,7 +1994,7 @@ private:
   uint64_t block_mask = 0;     ///< mask to get just the block offset
   size_t block_size_order = 0; ///< bits to shift to get block size
 
-  uint64_t min_alloc_size = 0; ///< minimum allocation unit (power of 2)
+  uint64_t min_alloc_size; ///< minimum allocation unit (power of 2)
   ///< bits for min_alloc_size
   uint8_t min_alloc_size_order = 0;
   static_assert(std::numeric_limits<uint8_t>::max() >
index 3a4a89038c3ab1d78a9c1712a9dfc303b1a6b387..878b9f21af1008fdb142ad4533fa1b9783684307 100644 (file)
@@ -20,6 +20,7 @@
 #include "include/stringify.h"
 #include "include/random.h"
 #include "common/perf_counters.h"
+#include "common/TracepointProvider.h"
 
 #include <fio.h>
 #include <optgroup.h>
@@ -335,6 +336,9 @@ struct Engine {
   }
 };
 
+TracepointProvider::Traits bluestore_tracepoint_traits("libbluestore_tp.so",
+                                                      "bluestore_tracing");
+
 Engine::Engine(thread_data* td)
   : ref_count(0),
     unlink(td->o.unlink),
@@ -363,6 +367,8 @@ Engine::Engine(thread_data* td)
                    CINIT_FLAG_NO_DEFAULT_CONFIG_FILE);
   common_init_finish(g_ceph_context);
 
+  TracepointProvider::initialize<bluestore_tracepoint_traits>(g_ceph_context);
+
   // create the ObjectStore
   os.reset(ObjectStore::create(g_ceph_context,
                                g_conf().get_val<std::string>("osd objectstore"),
index c8217c795cffac3457337c8b4ca25b4f345e042f..62d17c17fe80c65ffc141673ab554b664f8d16a0 100644 (file)
@@ -45,6 +45,7 @@ set(osd_traces oprequest.tp osd.tp pg.tp)
 add_tracing_library(osd_tp "${osd_traces}" 1.0.0)
 add_tracing_library(rados_tp librados.tp 2.0.0)
 add_tracing_library(os_tp objectstore.tp 1.0.0)
+add_tracing_library(bluestore_tp bluestore.tp 1.0.0)
 add_tracing_library(rgw_op_tp rgw_op.tp 1.0.0)
 add_tracing_library(rgw_rados_tp rgw_rados.tp 1.0.0)
 
diff --git a/src/tracing/bluestore.c b/src/tracing/bluestore.c
new file mode 100644 (file)
index 0000000..25984b9
--- /dev/null
@@ -0,0 +1,5 @@
+#define TRACEPOINT_CREATE_PROBES
+/*
+ * The header containing our TRACEPOINT_EVENTs.
+ */
+#include "tracing/bluestore.h"
diff --git a/src/tracing/bluestore.tp b/src/tracing/bluestore.tp
new file mode 100644 (file)
index 0000000..41466f1
--- /dev/null
@@ -0,0 +1,148 @@
+#include "include/int_types.h"
+
+TRACEPOINT_EVENT(bluestore, transaction_state_duration,
+    TP_ARGS(
+        uint32_t, sequencer_id,
+        uint64_t, tid,
+        uint8_t,  state,
+        double, elapsed),
+    TP_FIELDS(
+        ctf_integer(uint32_t, sequencer_id, sequencer_id)
+        ctf_integer(uint64_t, tid, tid)
+        ctf_integer(int8_t, state, state)
+        ctf_float(double, elapsed, elapsed)
+    )
+)
+
+TRACEPOINT_EVENT(bluestore, transaction_total_duration,
+    TP_ARGS(
+        uint32_t, sequencer_id,
+        uint64_t, tid,
+        double, elapsed),
+    TP_FIELDS(
+        ctf_integer(uint32_t, sequencer_id, sequencer_id)
+        ctf_integer(uint64_t, tid, tid)
+        ctf_float(double, elapsed, elapsed)
+    )
+)
+
+TRACEPOINT_EVENT(bluestore, transaction_commit_latency,
+    TP_ARGS(
+        uint32_t, sequencer_id,
+        uint64_t, tid,
+        double, elapsed),
+    TP_FIELDS(
+        ctf_integer(uint32_t, sequencer_id, sequencer_id)
+        ctf_integer(uint64_t, tid, tid)
+        ctf_float(double, elapsed, elapsed)
+    )
+)
+
+TRACEPOINT_EVENT(bluestore, transaction_kv_submit_latency,
+    TP_ARGS(
+        uint32_t, sequencer_id,
+        uint64_t, tid,
+       uint8_t, sync,
+        double, elapsed),
+    TP_FIELDS(
+        ctf_integer(uint32_t, sequencer_id, sequencer_id)
+        ctf_integer(uint64_t, tid, tid)
+        ctf_integer(int8_t, sync, sync)
+        ctf_float(double, elapsed, elapsed)
+    )
+)
+
+TRACEPOINT_EVENT(bluestore, transaction_kv_sync_latency,
+    TP_ARGS(
+        uint32_t, sequencer_id,
+        uint64_t, tid,
+       uint32_t, kv_batch_size,
+       uint32_t, deferred_done_batch_size,
+       uint32_t, deferred_stable_batch_size,
+        double, elapsed),
+    TP_FIELDS(
+        ctf_integer(uint32_t, sequencer_id, sequencer_id)
+        ctf_integer(uint64_t, tid, tid)
+       ctf_integer(uint32_t, kv_batch_size, kv_batch_size)
+       ctf_integer(uint32_t, deferred_done_batch_size, deferred_done_batch_size)
+       ctf_integer(uint32_t, deferred_stable_batch_size, deferred_stable_batch_size)
+        ctf_float(double, elapsed, elapsed)
+    )
+)
+
+TRACEPOINT_EVENT(bluestore, transaction_initial_state,
+    TP_ARGS(
+        uint32_t, sequencer_id,
+        uint64_t, tid,
+       int64_t, current_kv_throttle_cost,
+       int64_t, current_deferred_throttle_cost,
+       uint64_t, pending_kv_ios,
+       uint64_t, pending_deferred_ios,
+       uint64_t, ios_started_since_last_traced_io,
+       uint64_t, ios_completed_since_last_traced_io,
+       double, throttle_time),
+    TP_FIELDS(
+        ctf_integer(uint32_t, sequencer_id, sequencer_id)
+        ctf_integer(uint64_t, tid, tid)
+        ctf_integer(int64_t, current_kv_throttle_cost, current_kv_throttle_cost)
+        ctf_integer(int64_t, current_deferred_throttle_cost, current_deferred_throttle_cost)
+        ctf_integer(uint64_t, pending_kv_ios, pending_kv_ios)
+        ctf_integer(uint64_t, pending_deferred_ios, pending_deferred_ios)
+        ctf_integer(uint64_t, ios_started_since_last_traced_io, ios_started_since_last_traced_io)
+        ctf_integer(uint64_t, ios_completed_since_last_traced_io, ios_completed_since_last_traced_io)
+        ctf_float(double, throttle_time, throttle_time)
+    )
+)
+
+TRACEPOINT_EVENT(bluestore, transaction_initial_state_rocksdb,
+    TP_ARGS(
+        uint32_t, sequencer_id,
+        uint64_t, tid,
+       uint64_t, rocksdb_base_level,
+       uint64_t, rocksdb_estimate_pending_compaction_bytes,
+       uint64_t, rocksdb_cur_size_all_mem_tables,
+        uint64_t, rocksdb_compaction_pending,
+        uint64_t, rocksdb_mem_table_flush_pending,
+        uint64_t, rocksdb_num_running_compactions,
+        uint64_t, rocksdb_num_running_flushes,
+        uint64_t, rocksdb_actual_delayed_write_rate),
+    TP_FIELDS(
+        ctf_integer(uint32_t, sequencer_id, sequencer_id)
+        ctf_integer(uint64_t, tid, tid)
+       ctf_integer(uint64_t, rocksdb_base_level, rocksdb_base_level)
+       ctf_integer(uint64_t, rocksdb_estimate_pending_compaction_bytes, rocksdb_estimate_pending_compaction_bytes)
+       ctf_integer(uint64_t, rocksdb_cur_size_all_mem_tables, rocksdb_cur_size_all_mem_tables)
+        ctf_integer(uint64_t, rocksdb_compaction_pending,rocksdb_compaction_pending)
+        ctf_integer(uint64_t, rocksdb_mem_table_flush_pending, rocksdb_mem_table_flush_pending)
+        ctf_integer(uint64_t, rocksdb_num_running_compactions, rocksdb_num_running_compactions)
+        ctf_integer(uint64_t, rocksdb_num_running_flushes, rocksdb_num_running_flushes)
+        ctf_integer(uint64_t, rocksdb_actual_delayed_write_rate, rocksdb_actual_delayed_write_rate)
+    )
+)
+
+TRACEPOINT_EVENT(bluestore, kv_submit,
+    TP_ARGS(
+        uint32_t, sequencer_id,
+        uint64_t, tid,
+       uint64_t, rocksdb_base_level,
+       uint64_t, rocksdb_estimate_pending_compaction_bytes,
+       uint64_t, rocksdb_cur_size_all_mem_tables,
+        uint64_t, rocksdb_compaction_pending,
+        uint64_t, rocksdb_mem_table_flush_pending,
+        uint64_t, rocksdb_num_running_compactions,
+        uint64_t, rocksdb_num_running_flushes,
+        uint64_t, rocksdb_actual_delayed_write_rate),
+    TP_FIELDS(
+        ctf_integer(uint32_t, sequencer_id, sequencer_id)
+        ctf_integer(uint64_t, tid, tid)
+       ctf_integer(uint64_t, rocksdb_base_level, rocksdb_base_level)
+       ctf_integer(uint64_t, rocksdb_estimate_pending_compaction_bytes, rocksdb_estimate_pending_compaction_bytes)
+       ctf_integer(uint64_t, rocksdb_cur_size_all_mem_tables, rocksdb_cur_size_all_mem_tables)
+        ctf_integer(uint64_t, rocksdb_compaction_pending,rocksdb_compaction_pending)
+        ctf_integer(uint64_t, rocksdb_mem_table_flush_pending, rocksdb_mem_table_flush_pending)
+        ctf_integer(uint64_t, rocksdb_num_running_compactions, rocksdb_num_running_compactions)
+        ctf_integer(uint64_t, rocksdb_num_running_flushes, rocksdb_num_running_flushes)
+        ctf_integer(uint64_t, rocksdb_actual_delayed_write_rate, rocksdb_actual_delayed_write_rate)
+    )
+)
+