"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");
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
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
.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(""),
#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
}
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),
}
}
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) {
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;
<< " " << 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;
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
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 ||
// ** 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);
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;
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 }) {
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;
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,
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;
}
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 {
// 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 >
}
}
+#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);
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()) {
if (!deferred_aggressive) {
if (deferred_queue_size >= deferred_batch_ops.load() ||
- bsthrottle.should_submit_deferred()) {
+ throttle.should_submit_deferred()) {
deferred_try_submit();
}
}
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;
{
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);
}
{
auto tstart = mono_clock::now();
- if (!bsthrottle.try_start_transaction(
+ if (!throttle.try_start_transaction(
*db,
*txc,
tstart)) {
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();
}
}
+#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)
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;
}
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;
{
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
#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"
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";
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)
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),
}
};
-
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);
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,
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;
}
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());
/// 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};
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() >
#include "include/stringify.h"
#include "include/random.h"
#include "common/perf_counters.h"
+#include "common/TracepointProvider.h"
#include <fio.h>
#include <optgroup.h>
}
};
+TracepointProvider::Traits bluestore_tracepoint_traits("libbluestore_tp.so",
+ "bluestore_tracing");
+
Engine::Engine(thread_data* td)
: ref_count(0),
unlink(td->o.unlink),
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"),
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)
--- /dev/null
+#define TRACEPOINT_CREATE_PROBES
+/*
+ * The header containing our TRACEPOINT_EVENTs.
+ */
+#include "tracing/bluestore.h"
--- /dev/null
+#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)
+ )
+)
+