deprecate parts of cache_status and move those counters to session_stats counters. make the performance counters thread safe. fix some issues recently introduced to performance counters and the usage of them in client_test

This commit is contained in:
Arvid Norberg 2014-08-01 06:07:48 +00:00
parent c4b2682599
commit f32f5e96aa
13 changed files with 286 additions and 189 deletions

View File

@ -3,38 +3,40 @@
session_view::session_view()
{
using libtorrent::find_metric_idx;
m_width = 128;
std::vector<lt::stats_metric> metrics = lt::session_stats_metrics();
m_cnt[0].resize(metrics.size(), 0);
m_cnt[1].resize(metrics.size(), 0);
m_queued_bytes_idx = find_metric_idx(metrics, "disk.queued_write_bytes");
m_wasted_bytes_idx = find_metric_idx(metrics, "net.recv_redundant_bytes");
m_failed_bytes_idx = find_metric_idx(metrics, "net.recv_failed_bytes");
m_num_peers_idx = find_metric_idx(metrics, "peer.num_peers_connected");
m_recv_payload_idx = find_metric_idx(metrics, "net.recv_payload_bytes");
m_sent_payload_idx = find_metric_idx(metrics, "net.sent_payload_bytes");
m_unchoked_idx = find_metric_idx(metrics, "peer.num_peers_up_unchoked");
m_unchoke_slots_idx = find_metric_idx(metrics, "ses.num_unchoke_slots");
m_limiter_up_queue_idx = find_metric_idx(metrics, "net.limiter_up_queue");
m_limiter_down_queue_idx = find_metric_idx(metrics, "net.limiter_down_queue");
m_queued_writes_idx = find_metric_idx(metrics, "disk.num_write_jobs");
m_queued_reads_idx = find_metric_idx(metrics, "disk.num_read_jobs");
m_queued_bytes_idx = find_metric_idx("disk.queued_write_bytes");
m_wasted_bytes_idx = find_metric_idx("net.recv_redundant_bytes");
m_failed_bytes_idx = find_metric_idx("net.recv_failed_bytes");
m_num_peers_idx = find_metric_idx("peer.num_peers_connected");
m_recv_payload_idx = find_metric_idx("net.recv_payload_bytes");
m_sent_payload_idx = find_metric_idx("net.sent_payload_bytes");
m_unchoked_idx = find_metric_idx("peer.num_peers_up_unchoked");
m_unchoke_slots_idx = find_metric_idx("ses.num_unchoke_slots");
m_limiter_up_queue_idx = find_metric_idx("net.limiter_up_queue");
m_limiter_down_queue_idx = find_metric_idx("net.limiter_down_queue");
m_queued_writes_idx = find_metric_idx("disk.num_write_jobs");
m_queued_reads_idx = find_metric_idx("disk.num_read_jobs");
m_writes_cache_idx = find_metric_idx(metrics, "disk.write_cache_blocks");
m_reads_cache_idx = find_metric_idx(metrics, "disk.read_cache_blocks");
m_pinned_idx = find_metric_idx(metrics, "disk.pinned_blocks");
m_num_blocks_read_idx = find_metric_idx(metrics, "disk.num_blocks_read");
m_cache_hit_idx = find_metric_idx(metrics, "disk.num_blocks_cache_hits");
m_blocks_in_use_idx = find_metric_idx(metrics, "disk.disk_blocks_in_use");
m_blocks_written_idx = find_metric_idx(metrics, "disk.num_blocks_written");
m_write_ops_idx = find_metric_idx(metrics, "disk.num_write_ops");
m_writes_cache_idx = find_metric_idx("disk.write_cache_blocks");
m_reads_cache_idx = find_metric_idx("disk.read_cache_blocks");
m_pinned_idx = find_metric_idx("disk.pinned_blocks");
m_num_blocks_read_idx = find_metric_idx("disk.num_blocks_read");
m_cache_hit_idx = find_metric_idx("disk.num_blocks_cache_hits");
m_blocks_in_use_idx = find_metric_idx("disk.disk_blocks_in_use");
m_blocks_written_idx = find_metric_idx("disk.num_blocks_written");
m_write_ops_idx = find_metric_idx("disk.num_write_ops");
m_mfu_size_idx = find_metric_idx(metrics, "disk.arc_mfu_size");
m_mfu_ghost_idx = find_metric_idx(metrics, "disk.arc_mfu_ghost_size");
m_mru_size_idx = find_metric_idx(metrics, "disk.arc_mru_size");
m_mru_ghost_idx = find_metric_idx(metrics, "disk.arc_mru_ghost_size");
m_mfu_size_idx = find_metric_idx("disk.arc_mfu_size");
m_mfu_ghost_idx = find_metric_idx("disk.arc_mfu_ghost_size");
m_mru_size_idx = find_metric_idx("disk.arc_mru_size");
m_mru_ghost_idx = find_metric_idx("disk.arc_mru_ghost_size");
}
void session_view::set_pos(int pos)

View File

@ -724,6 +724,8 @@ namespace libtorrent
// the settings for the client
aux::session_settings m_settings;
counters m_stats_counters;
// this is a pool allocator for torrent_peer objects
torrent_peer_allocator m_peer_allocator;
@ -1170,15 +1172,11 @@ namespace libtorrent
// incremented by one
int m_log_seq;
counters m_last_stats_counters;
cache_status m_last_cache_status;
size_type m_last_failed;
size_type m_last_redundant;
size_type m_last_uploaded;
size_type m_last_downloaded;
vm_statistics_data_t m_last_vm_stat;
thread_cpu_usage m_network_thread_cpu_usage;
sliding_average<20> m_read_ops;
sliding_average<20> m_write_ops;
#endif
// each second tick the timer takes a little
@ -1216,8 +1214,6 @@ namespace libtorrent
private:
counters m_stats_counters;
#ifdef TORRENT_UPNP_LOGGING
std::ofstream m_upnp_log;
#endif

View File

@ -383,7 +383,7 @@ namespace libtorrent
// returns the number of bytes read on success (cache hit)
// -1 on cache miss
int try_read(disk_io_job* j, bool count_stats = true, bool expect_no_fail = false);
int try_read(disk_io_job* j, bool expect_no_fail = false);
// called when we're reading and we found the piece we're
// reading from in the hash table (not necessarily that we
@ -518,8 +518,6 @@ namespace libtorrent
// no buffer duplication
boost::uint32_t m_send_buffer_blocks;
boost::uint32_t m_blocks_read_hit;
// the number of blocks with a refcount > 0, i.e.
// they may not be evicted
int m_pinned_blocks;

View File

@ -42,6 +42,7 @@ POSSIBILITY OF SUCH DAMAGE.
#include "libtorrent/block_cache.hpp"
#include "libtorrent/file_pool.hpp"
#include "libtorrent/disk_interface.hpp"
#include "libtorrent/performance_counters.hpp"
#include <boost/function/function0.hpp>
#include <boost/noncopyable.hpp>
@ -99,12 +100,13 @@ namespace libtorrent
{
// initializes all counters to 0
cache_status()
: blocks_written(0)
: pieces()
#ifndef TORRENT_NO_DEPRECATE
, blocks_written(0)
, writes(0)
, blocks_read(0)
, blocks_read_hit(0)
, reads(0)
#ifndef TORRENT_NO_DEPRECATE
, queued_bytes(0)
, cache_size(0)
#endif
@ -112,6 +114,7 @@ namespace libtorrent
, read_cache_size(0)
, pinned_blocks(0)
, total_used_buffers(0)
#ifndef TORRENT_NO_DEPRECATE
, average_read_time(0)
, average_write_time(0)
, average_hash_time(0)
@ -121,6 +124,7 @@ namespace libtorrent
, cumulative_write_time(0)
, cumulative_hash_time(0)
, total_read_back(0)
#endif
, read_queue_size(0)
, blocked_jobs(0)
, queued_jobs(0)
@ -275,6 +279,7 @@ namespace libtorrent
{
disk_io_thread(io_service& ios
, alert_dispatcher* alert_disp
, counters& cnt
, void* userdata
, int block_size = 16 * 1024);
~disk_io_thread();
@ -502,10 +507,6 @@ namespace libtorrent
// shutting down. This last thread is responsible for cleanup
atomic_count m_num_running_threads;
// this is the number of threads currently writing bytes
// to disk
atomic_count m_num_writing_threads;
// the actual threads running disk jobs
std::vector<boost::shared_ptr<thread> > m_threads;
@ -527,11 +528,14 @@ namespace libtorrent
mutable mutex m_cache_mutex;
block_cache m_disk_cache;
void flip_stats();
// total number of blocks in use by both the read
// and the write cache. This is not supposed to
// exceed m_cache_size
counters& m_stats_counters;
// TODO: 3 turn these counters and gauges into session_stats
// counters (which also would need to be thread safe)
cache_status m_cache_stats;
// average read time for cache misses (in microseconds)
@ -546,10 +550,6 @@ namespace libtorrent
// average time to serve a job (any job) in microseconds
average_accumulator m_job_time;
// the last time we reset the average time and store the
// latest value in m_cache_stats
ptime m_last_stats_flip;
// the total number of outstanding jobs. This is used to
// limit the number of jobs issued in parallel. It also creates
// an opportunity to sort the jobs by physical offset before

View File

@ -34,6 +34,7 @@ POSSIBILITY OF SUCH DAMAGE.
#define TORRENT_PERFORMANCE_COUNTERS_HPP_INCLUDED
#include <boost/cstdint.hpp>
#include <boost/atomic.hpp>
#include "libtorrent/config.hpp"
namespace libtorrent
@ -178,9 +179,11 @@ namespace libtorrent
num_blocks_written,
num_blocks_read,
num_blocks_hashed,
num_blocks_cache_hits,
num_write_ops,
num_read_ops,
num_read_back,
disk_read_time,
disk_write_time,
@ -386,8 +389,11 @@ namespace libtorrent
counters();
counters(counters const&);
counters& operator=(counters const&);
// returns the new value
boost::uint64_t inc_stats_counter(int c, boost::int64_t value = 1);
boost::int64_t inc_stats_counter(int c, boost::int64_t value = 1);
boost::int64_t operator[](int i) const;
void set_value(int c, boost::int64_t value);
@ -396,8 +402,14 @@ namespace libtorrent
private:
// TODO: some space could be saved here by making gauges 32 bits
#if BOOST_ATOMIC_LLONG_LOCK_FREE == 2
boost::atomic<boost::int64_t> m_stats_counter[num_counters];
#else
// if the atomic type is't lock-free, use a single lock instead, for
// the whole array
mutex m_mutex;
boost::int64_t m_stats_counter[num_counters];
#endif
};
}

View File

@ -269,14 +269,13 @@ block_cache::block_cache(int block_size, io_service& ios
, m_read_cache_size(0)
, m_write_cache_size(0)
, m_send_buffer_blocks(0)
, m_blocks_read_hit(0)
, m_pinned_blocks(0)
{}
// returns:
// -1: not in cache
// -2: no memory
int block_cache::try_read(disk_io_job* j, bool count_stats, bool expect_no_fail)
int block_cache::try_read(disk_io_job* j, bool expect_no_fail)
{
INVARIANT_CHECK;
@ -308,8 +307,6 @@ int block_cache::try_read(disk_io_job* j, bool count_stats, bool expect_no_fail)
if (ret < 0) return ret;
ret = j->d.io.buffer_size;
if (count_stats)
++m_blocks_read_hit;
return ret;
}
@ -1450,7 +1447,6 @@ int block_cache::drain_piece_bufs(cached_piece_entry& p, std::vector<char*>& buf
void block_cache::update_stats_counters(counters& c) const
{
c.set_value(counters::num_blocks_cache_hits, m_blocks_read_hit);
c.set_value(counters::write_cache_blocks, m_write_cache_size);
c.set_value(counters::read_cache_blocks, m_read_cache_size);
c.set_value(counters::pinned_blocks, m_pinned_blocks);
@ -1465,7 +1461,6 @@ void block_cache::update_stats_counters(counters& c) const
void block_cache::get_stats(cache_status* ret) const
{
ret->blocks_read_hit = m_blocks_read_hit;
ret->write_cache_size = m_write_cache_size;
ret->read_cache_size = m_read_cache_size;
ret->pinned_blocks = m_pinned_blocks;

View File

@ -36,6 +36,7 @@ POSSIBILITY OF SUCH DAMAGE.
#include "libtorrent/escape_string.hpp"
#include "libtorrent/disk_io_thread.hpp"
#include "libtorrent/torrent_info.hpp" // for merkle_*()
#include "libtorrent/performance_counters.hpp" // for counters
#include <boost/bind.hpp>
#include <boost/next_prior.hpp>
@ -201,7 +202,8 @@ namespace libtorrent
// dummy torrent object pointer
boost::shared_ptr<char> dummy;
disk_io_thread disk_thread(ios, 0, 0);
counters cnt;
disk_io_thread disk_thread(ios, 0, cnt, 0);
storage_params params;
params.files = &t.files();

View File

@ -163,17 +163,17 @@ namespace libtorrent
disk_io_thread::disk_io_thread(io_service& ios
, alert_dispatcher* alert_disp
, counters& cnt
, void* userdata
, int block_size)
: m_num_threads(0)
, m_num_running_threads(0)
, m_num_writing_threads(0)
, m_userdata(userdata)
, m_last_cache_expiry(min_time())
, m_last_file_check(time_now_hires())
, m_file_pool(40)
, m_disk_cache(block_size, ios, boost::bind(&disk_io_thread::trigger_cache_trim, this), alert_disp)
, m_last_stats_flip(time_now())
, m_stats_counters(cnt)
, m_outstanding_jobs(0)
, m_ios(ios)
, m_num_blocked_jobs(0)
@ -635,7 +635,7 @@ namespace libtorrent
{
TORRENT_PIECE_ASSERT(!error, pe);
TORRENT_PIECE_ASSERT(num_blocks > 0, pe);
++m_num_writing_threads;
m_stats_counters.inc_stats_counter(counters::num_writing_threads, 1);
ptime start_time = time_now_hires();
int block_size = m_disk_cache.block_size();
@ -666,18 +666,18 @@ namespace libtorrent
flushing_start = i;
}
--m_num_writing_threads;
m_stats_counters.inc_stats_counter(counters::num_writing_threads, -1);
if (!failed)
{
TORRENT_PIECE_ASSERT(!error, pe);
boost::uint32_t write_time = total_microseconds(time_now_hires() - start_time);
m_write_time.add_sample(write_time / num_blocks);
m_cache_stats.cumulative_write_time += write_time / 1000;
m_cache_stats.cumulative_job_time += write_time / 1000;
m_cache_stats.blocks_written += num_blocks;
++m_cache_stats.writes;
m_stats_counters.inc_stats_counter(counters::num_blocks_written, num_blocks);
m_stats_counters.inc_stats_counter(counters::num_write_ops);
m_stats_counters.inc_stats_counter(counters::disk_write_time, write_time);
m_stats_counters.inc_stats_counter(counters::disk_job_time, write_time);
#if DEBUG_DISK_THREAD
DLOG("flush_iovec: %d\n", num_blocks);
#endif
@ -941,7 +941,7 @@ namespace libtorrent
// not have had its flush_hashed job run on it
// so only do it if no other thread is currently flushing
if (num == 0 || m_num_writing_threads > 0) return;
if (num == 0 || m_stats_counters[counters::num_writing_threads] > 0) return;
// if we still need to flush blocks, start over and flush
// everything in LRU order (degrade to lru cache eviction)
@ -1101,7 +1101,7 @@ namespace libtorrent
// don't evict write jobs if at least one other thread
// is flushing right now. Doing so could result in
// unnecessary flushing of the wrong pieces
if (evict > 0 && m_num_writing_threads == 0)
if (evict > 0 && m_stats_counters[counters::num_writing_threads] == 0)
{
try_flush_write_blocks(evict, completed_jobs, l);
}
@ -1138,9 +1138,6 @@ namespace libtorrent
TORRENT_ASSERT(j->action < sizeof(job_functions)/sizeof(job_functions[0]));
// TODO: hold disk_io_thread mutex here!
if (time_now() > m_last_stats_flip + seconds(1)) flip_stats();
ptime start_time = time_now_hires();
++m_outstanding_jobs;
@ -1216,11 +1213,12 @@ namespace libtorrent
{
boost::uint32_t read_time = total_microseconds(time_now_hires() - start_time);
m_read_time.add_sample(read_time);
m_cache_stats.cumulative_read_time += read_time / 1000;
m_cache_stats.cumulative_job_time += read_time / 1000;
++m_cache_stats.total_read_back;
++m_cache_stats.blocks_read;
++m_cache_stats.reads;
m_stats_counters.inc_stats_counter(counters::num_read_back);
m_stats_counters.inc_stats_counter(counters::num_blocks_read);
m_stats_counters.inc_stats_counter(counters::num_read_ops);
m_stats_counters.inc_stats_counter(counters::disk_read_time, read_time);
m_stats_counters.inc_stats_counter(counters::disk_job_time, read_time);
}
return ret;
}
@ -1318,10 +1316,11 @@ namespace libtorrent
{
boost::uint32_t read_time = total_microseconds(time_now_hires() - start_time);
m_read_time.add_sample(read_time / iov_len);
m_cache_stats.cumulative_read_time += read_time / 1000;
m_cache_stats.cumulative_job_time += read_time / 1000;
m_cache_stats.blocks_read += iov_len;
++m_cache_stats.reads;
m_stats_counters.inc_stats_counter(counters::num_blocks_read, iov_len);
m_stats_counters.inc_stats_counter(counters::num_read_ops);
m_stats_counters.inc_stats_counter(counters::disk_read_time, read_time);
m_stats_counters.inc_stats_counter(counters::disk_job_time, read_time);
}
l.lock();
@ -1363,7 +1362,7 @@ namespace libtorrent
TORRENT_ASSERT(pe->blocks[block].buf);
int tmp = m_disk_cache.try_read(j, false, true);
int tmp = m_disk_cache.try_read(j, true);
TORRENT_ASSERT(tmp >= 0);
maybe_issue_queued_read_jobs(pe, completed_jobs);
@ -1411,6 +1410,7 @@ namespace libtorrent
if (ret >= 0)
{
// cache-hit
m_stats_counters.inc_stats_counter(counters::num_blocks_cache_hits);
DLOG("do_read: cache hit\n");
j->flags |= disk_io_job::cache_hit;
j->ret = ret;
@ -1460,22 +1460,23 @@ namespace libtorrent
file::iovec_t b = { j->buffer, size_t(j->d.io.buffer_size) };
int file_flags = file_flags_for_job(j);
++m_num_writing_threads;
m_stats_counters.inc_stats_counter(counters::num_writing_threads, 1);
// the actual write operation
int ret = j->storage->get_storage_impl()->writev(&b, 1
, j->piece, j->d.io.offset, file_flags, j->error);
--m_num_writing_threads;
m_stats_counters.inc_stats_counter(counters::num_writing_threads, -1);
if (!j->error.ec)
{
boost::uint32_t write_time = total_microseconds(time_now_hires() - start_time);
m_write_time.add_sample(write_time);
m_cache_stats.cumulative_write_time += write_time / 1000;
m_cache_stats.cumulative_job_time += write_time / 1000;
++m_cache_stats.blocks_written;
++m_cache_stats.writes;
m_stats_counters.inc_stats_counter(counters::num_blocks_written);
m_stats_counters.inc_stats_counter(counters::num_write_ops);
m_stats_counters.inc_stats_counter(counters::disk_write_time, write_time);
m_stats_counters.inc_stats_counter(counters::disk_job_time, write_time);
}
m_disk_cache.free_buffer(j->buffer);
@ -1614,6 +1615,7 @@ namespace libtorrent
int ret = m_disk_cache.try_read(j);
if (ret >= 0)
{
m_stats_counters.inc_stats_counter(counters::num_blocks_cache_hits);
DLOG("do_read: cache hit\n");
j->flags |= disk_io_job::cache_hit;
j->ret = ret;
@ -2200,8 +2202,10 @@ namespace libtorrent
TORRENT_PIECE_ASSERT(pe->hash, pe);
m_hash_time.add_sample(hash_time / (end - cursor));
m_cache_stats.cumulative_hash_time += hash_time / 1000;
m_cache_stats.cumulative_job_time += hash_time / 1000;
m_stats_counters.inc_stats_counter(counters::num_blocks_hashed, end - cursor);
m_stats_counters.inc_stats_counter(counters::disk_hash_time, hash_time);
m_stats_counters.inc_stats_counter(counters::disk_job_time, hash_time);
pe->hashing = 0;
@ -2280,10 +2284,11 @@ namespace libtorrent
{
boost::uint32_t read_time = total_microseconds(time_now_hires() - start_time);
m_read_time.add_sample(read_time);
m_cache_stats.cumulative_read_time += read_time / 1000;
m_cache_stats.cumulative_job_time += read_time / 1000;
++m_cache_stats.blocks_read;
++m_cache_stats.reads;
m_stats_counters.inc_stats_counter(counters::num_blocks_read);
m_stats_counters.inc_stats_counter(counters::num_read_ops);
m_stats_counters.inc_stats_counter(counters::disk_read_time, read_time);
m_stats_counters.inc_stats_counter(counters::disk_job_time, read_time);
}
offset += block_size;
@ -2495,11 +2500,12 @@ namespace libtorrent
{
boost::uint32_t read_time = total_microseconds(time_now_hires() - start_time);
m_read_time.add_sample(read_time);
m_cache_stats.cumulative_read_time += read_time / 1000;
m_cache_stats.cumulative_job_time += read_time / 1000;
++m_cache_stats.total_read_back;
++m_cache_stats.blocks_read;
++m_cache_stats.reads;
m_stats_counters.inc_stats_counter(counters::num_read_back);
m_stats_counters.inc_stats_counter(counters::num_blocks_read);
m_stats_counters.inc_stats_counter(counters::num_read_ops);
m_stats_counters.inc_stats_counter(counters::disk_read_time, read_time);
m_stats_counters.inc_stats_counter(counters::disk_job_time, read_time);
}
TORRENT_PIECE_ASSERT(ph->offset == i * block_size, pe);
@ -2723,10 +2729,11 @@ namespace libtorrent
{
boost::uint32_t read_time = total_microseconds(time_now_hires() - start_time);
m_read_time.add_sample(read_time);
m_cache_stats.cumulative_read_time += read_time / 1000;
m_cache_stats.cumulative_job_time += read_time / 1000;
++m_cache_stats.blocks_read;
++m_cache_stats.reads;
m_stats_counters.inc_stats_counter(counters::num_blocks_read);
m_stats_counters.inc_stats_counter(counters::num_read_ops);
m_stats_counters.inc_stats_counter(counters::disk_read_time, read_time);
m_stats_counters.inc_stats_counter(counters::disk_job_time, read_time);
}
offset += block_size;
@ -2748,16 +2755,6 @@ namespace libtorrent
}
#endif
void disk_io_thread::flip_stats()
{
// calling mean() will actually reset the accumulators
m_cache_stats.average_read_time = m_read_time.mean();
m_cache_stats.average_write_time = m_write_time.mean();
m_cache_stats.average_hash_time = m_hash_time.mean();
m_cache_stats.average_job_time = m_job_time.mean();
m_last_stats_flip = time_now();
}
void get_cache_info_impl(cached_piece_info& info, cached_piece_entry const* i, int block_size)
{
info.piece = i->piece;
@ -2781,23 +2778,8 @@ namespace libtorrent
// These are atomic_counts, so it's safe to access them from
// a different thread
// TODO: 3 instead of updating these counters in this function,
// they could be updated every time a job completes
c.set_value(counters::disk_read_time, m_cache_stats.cumulative_read_time);
c.set_value(counters::disk_write_time, m_cache_stats.cumulative_write_time);
c.set_value(counters::disk_hash_time, m_cache_stats.cumulative_hash_time);
c.set_value(counters::disk_job_time, m_cache_stats.cumulative_job_time);
c.set_value(counters::num_writing_threads, m_num_writing_threads);
c.set_value(counters::num_running_threads, m_num_running_threads);
c.set_value(counters::blocked_disk_jobs, m_num_blocked_jobs);
// counters
c.set_value(counters::num_blocks_written, m_cache_stats.blocks_written);
c.set_value(counters::num_blocks_read, m_cache_stats.blocks_read);
c.set_value(counters::num_write_ops, m_cache_stats.writes);
c.set_value(counters::num_read_ops, m_cache_stats.reads);
mutex::scoped_lock jl(m_job_mutex);
c.set_value(counters::queued_disk_jobs, m_num_blocked_jobs
@ -2823,6 +2805,32 @@ namespace libtorrent
ret->queued_jobs = m_queued_jobs.size() + m_queued_hash_jobs.size();
jl.unlock();
#ifndef TORRENT_NO_DEPRECATE
ret->blocks_read_hit = m_stats_counters[counters::num_blocks_cache_hits];
ret->blocks_read = m_stats_counters[counters::num_blocks_read];
ret->blocks_written = m_stats_counters[counters::num_blocks_written];
ret->writes = m_stats_counters[counters::num_write_ops];
ret->reads = m_stats_counters[counters::num_read_ops];
int num_read_jobs = (std::max)(boost::int64_t(1)
, m_stats_counters[counters::num_read_ops]);
int num_write_jobs = (std::max)(boost::int64_t(1)
, m_stats_counters[counters::num_write_ops]);
int num_hash_jobs = (std::max)(boost::int64_t(1)
, m_stats_counters[counters::num_blocks_hashed]);
ret->average_read_time = m_stats_counters[counters::disk_read_time] / num_read_jobs;
ret->average_write_time = m_stats_counters[counters::disk_write_time] / num_write_jobs;
ret->average_hash_time = m_stats_counters[counters::disk_hash_time] / num_hash_jobs;
ret->average_job_time = m_stats_counters[counters::disk_job_time]
/ (num_read_jobs + num_write_jobs + num_hash_jobs);
ret->cumulative_job_time = m_stats_counters[counters::disk_job_time];
ret->cumulative_read_time = m_stats_counters[counters::disk_read_time];
ret->cumulative_write_time = m_stats_counters[counters::disk_write_time];
ret->cumulative_hash_time = m_stats_counters[counters::disk_hash_time];
ret->total_read_back = m_stats_counters[counters::num_read_back];
#endif
mutex::scoped_lock l(m_cache_mutex);
*ret = m_cache_stats;
ret->total_used_buffers = m_disk_cache.in_use();
@ -2832,7 +2840,7 @@ namespace libtorrent
ret->num_jobs = jobs_in_use();
ret->num_read_jobs = read_jobs_in_use();
ret->num_write_jobs = write_jobs_in_use();
ret->num_writing_threads = m_num_writing_threads;
ret->num_writing_threads = m_stats_counters[counters::num_writing_threads];
m_disk_cache.get_stats(ret);
@ -3153,6 +3161,7 @@ namespace libtorrent
DLOG("started disk thread %d\n", int(thread_id));
++m_num_running_threads;
m_stats_counters.inc_stats_counter(counters::num_running_threads, 1);
mutex::scoped_lock l(m_job_mutex);
for (;;)
@ -3220,6 +3229,7 @@ namespace libtorrent
l.unlock();
// do cleanup in the last running thread
m_stats_counters.inc_stats_counter(counters::num_running_threads, -1);
if (--m_num_running_threads > 0)
{
DLOG("exiting disk thread %d. num_threads: %d\n", thread_id, int(m_num_threads));

View File

@ -43,7 +43,43 @@ namespace libtorrent {
counters::counters()
{
#if BOOST_ATOMIC_LLONG_LOCK_FREE == 2
for (int i = 0; i < sizeof(m_stats_counter)
/ sizeof(m_stats_counter[0]); ++i)
m_stats_counter[i].store(0, boost::memory_order_relaxed);
#else
memset(m_stats_counter, 0, sizeof(m_stats_counter));
#endif
}
counters::counters(counters const& c)
{
#if BOOST_ATOMIC_LLONG_LOCK_FREE == 2
for (int i = 0; i < sizeof(m_stats_counter)
/ sizeof(m_stats_counter[0]); ++i)
m_stats_counter[i].store(
c.m_stats_counter[i].load(boost::memory_order_relaxed)
, boost::memory_order_relaxed);
#else
mutex::scoped_lock l(c.m_mutex);
memcpy(m_stats_counter, c.m_stats_counter, sizeof(m_stats_counter));
#endif
}
counters& counters::operator=(counters const& c)
{
#if BOOST_ATOMIC_LLONG_LOCK_FREE == 2
for (int i = 0; i < sizeof(m_stats_counter)
/ sizeof(m_stats_counter[0]); ++i)
m_stats_counter[i].store(
c.m_stats_counter[i].load(boost::memory_order_relaxed)
, boost::memory_order_relaxed);
#else
mutex::scoped_lock l(m_mutex);
mutex::scoped_lock l(c.m_mutex);
memcpy(m_stats_counter, c.m_stats_counter, sizeof(m_stats_counter));
#endif
return *this;
}
boost::int64_t counters::operator[](int i) const
@ -53,12 +89,18 @@ namespace libtorrent {
#ifdef TORRENT_USE_VALGRIND
VALGRIND_CHECK_VALUE_IS_DEFINED(m_stats_counter[i]);
#endif
#if BOOST_ATOMIC_LLONG_LOCK_FREE == 2
return m_stats_counter[i].load(boost::memory_order_relaxed);
#else
mutex::scoped_lock l(m_mutex);
return m_stats_counter[i];
#endif
}
// the argument specifies which counter to
// increment or decrement
boost::uint64_t counters::inc_stats_counter(int c, boost::int64_t value)
boost::int64_t counters::inc_stats_counter(int c, boost::int64_t value)
{
// if c >= num_stats_counters, it means it's not
// a monotonically increasing counter, but a gauge
@ -67,8 +109,15 @@ namespace libtorrent {
TORRENT_ASSERT(c >= 0);
TORRENT_ASSERT(c < num_counters);
#if BOOST_ATOMIC_LLONG_LOCK_FREE == 2
boost::int64_t pv = m_stats_counter[c].fetch_add(value, boost::memory_order_relaxed);
TORRENT_ASSERT(pv + value >= 0);
return pv + value;
#else
mutex::scoped_lock l(m_mutex);
TORRENT_ASSERT(m_stats_counter[c] + value >= 0);
return m_stats_counter[c] += value;
#endif
}
// ratio is a vaue between 0 and 100 representing the percentage the value
@ -82,9 +131,20 @@ namespace libtorrent {
TORRENT_ASSERT(num_stats_counters);
// TODO: 2 to make this thread safe, use compare_exchange_weak
boost::uint64_t current = m_stats_counter[c];
#if BOOST_ATOMIC_LLONG_LOCK_FREE == 2
boost::int64_t current = m_stats_counter[c].load(boost::memory_order_relaxed);
boost::int64_t new_value = (current * (100-ratio) + value * ratio) / 100;
while (!m_stats_counter[c].compare_exchange_weak(current, new_value
, boost::memory_order_relaxed))
{
new_value = (current * (100-ratio) + value * ratio) / 100;
}
#else
mutex::scoped_lock l(m_mutex);
boost::int64_t current = m_stats_counter[c];
m_stats_counter[c] = (current * (100-ratio) + value * ratio) / 100;
#endif
}
void counters::set_value(int c, boost::int64_t value)
@ -92,11 +152,17 @@ namespace libtorrent {
TORRENT_ASSERT(c >= 0);
TORRENT_ASSERT(c < num_counters);
#if BOOST_ATOMIC_LLONG_LOCK_FREE == 2
m_stats_counter[c].store(value);
#else
mutex::scoped_lock l(m_mutex);
// if this assert fires, someone is trying to decrement a counter
// which is not allowed. Counters are monotonically increasing
TORRENT_ASSERT(value >= m_stats_counter[c] || c >= num_stats_counters);
m_stats_counter[c] = value;
#endif
}
}

View File

@ -441,7 +441,8 @@ namespace aux {
, m_ssl_ctx(m_io_service, asio::ssl::context::sslv23)
#endif
, m_alerts(m_settings.get_int(settings_pack::alert_queue_size), alert::all_categories)
, m_disk_thread(m_io_service, this, (uncork_interface*)this)
, m_disk_thread(m_io_service, this, m_stats_counters
, (uncork_interface*)this)
, m_half_open(m_io_service)
, m_download_rate(peer_connection::download_channel)
#ifdef TORRENT_VERBOSE_BANDWIDTH_LIMIT
@ -715,10 +716,6 @@ namespace aux {
get_vm_stats(&vst, ec);
if (!ec) m_last_vm_stat = vst;
m_last_failed = 0;
m_last_redundant = 0;
m_last_uploaded = 0;
m_last_downloaded = 0;
get_thread_cpu_usage(&m_network_thread_cpu_usage);
rotate_stats_log();
@ -4080,26 +4077,25 @@ retry:
if (m_stats_logger)
{
counters cnt = m_stats_counters;
cache_status cs;
m_disk_thread.get_cache_info(&cs);
session_status sst = status();
m_read_ops.add_sample((cs.reads - m_last_cache_status.reads) * 1000000.0 / float(tick_interval_ms));
m_write_ops.add_sample((cs.writes - m_last_cache_status.writes) * 1000000.0 / float(tick_interval_ms));
#ifdef TORRENT_USE_VALGRIND
#define STAT_LOGL(type, val) VALGRIND_CHECK_VALUE_IS_DEFINED(val); fprintf(m_stats_logger, "%" #type "\t", val)
#else
#define STAT_LOGL(type, val) fprintf(m_stats_logger, "%" #type "\t", val)
#endif
#define STAT_COUNTER(cnt) fprintf(m_stats_logger, "%" PRId64 "\t", m_stats_counters[counters:: cnt])
#define STAT_COUNTER(c) fprintf(m_stats_logger, "%" PRId64 "\t", cnt[counters:: c])
#define STAT_COUNTER_DELTA(c) fprintf(m_stats_logger, "%" PRId64 "\t", cnt[counters:: c] \
- m_last_stats_counters[counters:: c])
#define STAT_LOG(type, val) fprintf(m_stats_logger, "%" #type "\t", val)
STAT_LOG(f, total_milliseconds(now - m_last_log_rotation) / 1000.f);
size_type uploaded = m_stat.total_upload() - m_last_uploaded;
STAT_LOG(d, int(uploaded));
size_type downloaded = m_stat.total_download() - m_last_downloaded;
STAT_LOG(d, int(downloaded));
STAT_COUNTER_DELTA(sent_bytes);
STAT_COUNTER_DELTA(recv_bytes);
STAT_COUNTER(num_downloading_torrents);
STAT_COUNTER(num_seeding_torrents);
STAT_COUNTER(num_peers_connected);
@ -4158,29 +4154,44 @@ retry:
STAT_COUNTER(connect_timeouts);
STAT_COUNTER(uninteresting_peers);
STAT_COUNTER(timeout_peers);
STAT_LOG(f, float(m_stats_counters[counters::recv_failed_bytes]) * 100.f
/ (std::max)(m_stats_counters[counters::recv_bytes], boost::int64_t(1)));
STAT_LOG(f, float(m_stats_counters[counters::recv_redundant_bytes]) * 100.f
/ (std::max)(m_stats_counters[counters::recv_bytes], boost::int64_t(1)));
STAT_LOG(f, float(m_stats_counters[counters::recv_bytes]
- m_stats_counters[counters::recv_payload_bytes]) * 100.f
/ (std::max)(m_stats_counters[counters::recv_bytes], boost::int64_t(1)));
STAT_LOG(f, float(cs.average_read_time) / 1000000.f);
STAT_LOG(f, float(cs.average_write_time) / 1000000.f);
STAT_LOG(f, float(cnt[counters::recv_failed_bytes]) * 100.f
/ (std::max)(cnt[counters::recv_bytes], boost::int64_t(1)));
STAT_LOG(f, float(cnt[counters::recv_redundant_bytes]) * 100.f
/ (std::max)(cnt[counters::recv_bytes], boost::int64_t(1)));
STAT_LOG(f, float(cnt[counters::recv_bytes]
- cnt[counters::recv_payload_bytes]) * 100.f
/ (std::max)(cnt[counters::recv_bytes], boost::int64_t(1)));
int delta_read_jobs = cnt[counters::num_read_ops]
- m_last_stats_counters[counters::num_read_ops];
int delta_read_time = cnt[counters::disk_read_time]
- m_last_stats_counters[counters::disk_read_time];
int delta_write_jobs = cnt[counters::num_write_ops]
- m_last_stats_counters[counters::num_write_ops];
int delta_write_time = cnt[counters::disk_write_time]
- m_last_stats_counters[counters::disk_write_time];
int delta_hash_jobs = cnt[counters::num_blocks_hashed]
- m_last_stats_counters[counters::num_blocks_hashed];
int delta_hash_time = cnt[counters::disk_hash_time]
- m_last_stats_counters[counters::disk_hash_time];
STAT_LOG(f, float(delta_read_jobs == 0 ? 0.f
: delta_read_time / delta_read_jobs) / 1000000.f);
STAT_LOG(f, float(delta_write_jobs == 0 ? 0.f
: delta_write_time / delta_write_jobs) / 1000000.f);
STAT_LOG(d, int(cs.pending_jobs + cs.queued_jobs));
STAT_COUNTER(queued_write_bytes);
STAT_LOG(d, int(cs.blocks_read_hit - m_last_cache_status.blocks_read_hit));
STAT_LOG(d, int(cs.blocks_read - m_last_cache_status.blocks_read));
STAT_LOG(d, int(cs.blocks_written - m_last_cache_status.blocks_written));
STAT_LOG(d, int(m_stats_counters[counters::recv_failed_bytes]
- m_last_failed));
STAT_LOG(d, int(m_stats_counters[counters::recv_redundant_bytes]
- m_last_redundant));
STAT_COUNTER_DELTA(num_blocks_cache_hits);
STAT_COUNTER_DELTA(num_blocks_read);
STAT_COUNTER_DELTA(num_blocks_written);
STAT_COUNTER_DELTA(recv_failed_bytes);
STAT_COUNTER_DELTA(recv_redundant_bytes);
STAT_COUNTER(num_error_torrents);
STAT_LOGL(d, cs.read_cache_size);
STAT_LOG(d, cs.write_cache_size + cs.read_cache_size);
STAT_COUNTER(disk_blocks_in_use);
STAT_LOG(f, float(cs.average_hash_time) / 1000000.f);
STAT_LOG(f, float(delta_hash_jobs == 0 ? 0.f
: delta_hash_time / delta_hash_jobs) / 1000000.f);
STAT_COUNTER(connection_attempts);
STAT_COUNTER(num_banned_peers);
STAT_COUNTER(banned_for_hash_failure);
@ -4189,14 +4200,14 @@ retry:
STAT_LOGL(d, connect_candidates);
STAT_LOG(d, int(m_settings.get_int(settings_pack::cache_size)
- m_settings.get_int(settings_pack::max_queued_disk_bytes) / 0x4000));
STAT_LOG(f, float(m_stats_counters[counters::disk_read_time] * 100.f
/ (std::max)(m_stats_counters[counters::disk_job_time], boost::int64_t(1))));
STAT_LOG(f, float(m_stats_counters[counters::disk_write_time] * 100.f
/ (std::max)(m_stats_counters[counters::disk_job_time], boost::int64_t(1))));
STAT_LOG(f, float(m_stats_counters[counters::disk_hash_time] * 100.f
/ (std::max)(m_stats_counters[counters::disk_job_time], boost::int64_t(1))));
STAT_LOG(d, int(cs.total_read_back - m_last_cache_status.total_read_back));
STAT_LOG(f, float(cs.total_read_back * 100.f / (std::max)(1, int(cs.blocks_written))));
STAT_LOG(f, float(cnt[counters::disk_read_time] * 100.f
/ (std::max)(cnt[counters::disk_job_time], boost::int64_t(1))));
STAT_LOG(f, float(cnt[counters::disk_write_time] * 100.f
/ (std::max)(cnt[counters::disk_job_time], boost::int64_t(1))));
STAT_LOG(f, float(cnt[counters::disk_hash_time] * 100.f
/ (std::max)(cnt[counters::disk_job_time], boost::int64_t(1))));
STAT_COUNTER_DELTA(num_read_back);
STAT_LOG(f, float(cnt[counters::num_read_back] * 100.f / (std::max)(1, int(cnt[counters::num_blocks_written]))));
STAT_COUNTER(num_read_jobs);
STAT_LOG(f, float(tick_interval_ms) / 1000.f);
STAT_LOG(f, float(m_tick_residual) / 1000.f);
@ -4214,8 +4225,8 @@ retry:
STAT_LOG(f, float(utp_num_delay_sockets ? float(utp_send_delay_sum) / float(utp_num_delay_sockets) : 0) / 1000000.f);
STAT_LOG(f, float(utp_peak_recv_delay) / 1000000.f);
STAT_LOG(f, float(utp_num_recv_delay_sockets ? float(utp_recv_delay_sum) / float(utp_num_recv_delay_sockets) : 0) / 1000000.f);
STAT_LOG(f, float(cs.reads - m_last_cache_status.reads) * 1000.0 / float(tick_interval_ms));
STAT_LOG(f, float(cs.writes - m_last_cache_status.writes) * 1000.0 / float(tick_interval_ms));
STAT_LOG(f, float(delta_read_jobs) * 1000.0 / float(tick_interval_ms));
STAT_LOG(f, float(delta_write_jobs) * 1000.0 / float(tick_interval_ms));
STAT_LOG(d, int(vm_stat.active_count));
STAT_LOG(d, int(vm_stat.inactive_count));
@ -4225,8 +4236,8 @@ retry:
STAT_LOG(d, int(vm_stat.pageouts - m_last_vm_stat.pageouts));
STAT_LOG(d, int(vm_stat.faults - m_last_vm_stat.faults));
STAT_LOG(f, m_read_ops.mean() / 1000.f);
STAT_LOG(f, m_write_ops.mean() / 1000.f);
STAT_LOG(f, float(delta_read_jobs) * 1000.f / float(tick_interval_ms));
STAT_LOG(f, float(delta_write_jobs) * 1000.f / float(tick_interval_ms));
STAT_COUNTER(pinned_blocks);
STAT_LOGL(d, partial_pieces);
@ -4243,16 +4254,16 @@ retry:
for (int i = counters::on_read_counter; i <= counters::on_disk_counter; ++i)
{
STAT_LOG(d, int(m_stats_counters[i]));
STAT_LOG(d, int(cnt[i]));
}
for (int i = counters::socket_send_size3; i <= counters::socket_send_size20; ++i)
{
STAT_LOG(d, int(m_stats_counters[i]));
STAT_LOG(d, int(cnt[i]));
}
for (int i = counters::socket_recv_size3; i <= counters::socket_recv_size20; ++i)
{
STAT_LOG(d, int(m_stats_counters[i]));
STAT_LOG(d, int(cnt[i]));
}
STAT_LOG(f, total_microseconds(cur_cpu_usage.user_time
@ -4310,7 +4321,7 @@ retry:
STAT_LOG(d, int(m_connections.size()));
STAT_LOGL(d, pending_incoming_reqs);
STAT_LOG(f, m_stats_counters[counters::num_peers_connected] == 0 ? 0.f : (float(pending_incoming_reqs) / m_stats_counters[counters::num_peers_connected]));
STAT_LOG(f, cnt[counters::num_peers_connected] == 0 ? 0.f : (float(pending_incoming_reqs) / cnt[counters::num_peers_connected]));
STAT_LOGL(d, num_want_more_peers);
STAT_LOG(f, total_peers_limit == 0 ? 0 : float(num_limited_peers) / total_peers_limit);
@ -4423,10 +4434,8 @@ retry:
m_last_cache_status = cs;
if (!vm_ec) m_last_vm_stat = vm_stat;
m_network_thread_cpu_usage = cur_cpu_usage;
m_last_failed = m_stats_counters[counters::recv_failed_bytes];
m_last_redundant = m_stats_counters[counters::recv_redundant_bytes];
m_last_uploaded = m_stat.total_upload();
m_last_downloaded = m_stat.total_download();
m_last_stats_counters = cnt;
}
}
#endif // TORRENT_STATS

View File

@ -44,9 +44,6 @@ namespace libtorrent
int value_index;
};
// TODO: 3 the type of counter does not need to be stored in this array.
// when the user asks for the list of counters, that field could be
// generated based on the range of the counter index.
#define METRIC(category, name) { #category "." #name, counters:: name },
const static stats_metric_impl metrics[] =
{
@ -316,12 +313,26 @@ namespace libtorrent
METRIC(disk, arc_write_size)
METRIC(disk, arc_volatile_size)
// the number of blocks written and read from disk in total. A block is
// 16 kiB.
METRIC(disk, num_blocks_written)
METRIC(disk, num_blocks_read)
// the total number of blocks run through SHA-1 hashing
METRIC(disk, num_blocks_hashed)
// the number of blocks read from the disk cache
METRIC(disk, num_blocks_cache_hits)
// the number of disk I/O operation for reads and writes. One disk
// operation may transfer more then one block.
METRIC(disk, num_write_ops)
METRIC(disk, num_read_ops)
// the number of blocks that had to be read back from disk in order to
// hash a piece (when verifying against the piece hash)
METRIC(disk, num_read_back)
// cumulative time spent in various disk jobs, as well
// as total for all disk jobs. Measured in microseconds
METRIC(disk, disk_read_time)
@ -466,7 +477,7 @@ namespace libtorrent
int find_metric_idx(char const* name)
{
stats_metric_impl const* end = metrics + sizeof(metrics)/sizeof(metrics);
stats_metric_impl const* end = metrics + sizeof(metrics)/sizeof(metrics[0]);
stats_metric_impl const* i = std::find_if(metrics, end , boost::bind(&strcmp
, boost::bind(&stats_metric_impl::name, _1), name) == 0);
if (i == end) return -1;

View File

@ -323,7 +323,6 @@ void test_arc_promote()
RETURN_BUFFER;
bc.get_stats(&status);
TEST_EQUAL(status.blocks_read_hit, 1);
TEST_EQUAL(status.write_cache_size, 0);
TEST_EQUAL(status.read_cache_size, 1);
TEST_EQUAL(status.pinned_blocks, 0);
@ -342,7 +341,6 @@ void test_arc_promote()
RETURN_BUFFER;
bc.get_stats(&status);
TEST_EQUAL(status.blocks_read_hit, 2);
TEST_EQUAL(status.write_cache_size, 0);
TEST_EQUAL(status.read_cache_size, 1);
TEST_EQUAL(status.pinned_blocks, 0);
@ -364,7 +362,6 @@ void test_arc_unghost()
INSERT(0, 0);
bc.get_stats(&status);
TEST_EQUAL(status.blocks_read_hit, 0);
TEST_EQUAL(status.write_cache_size, 0);
TEST_EQUAL(status.read_cache_size, 1);
TEST_EQUAL(status.pinned_blocks, 0);
@ -379,7 +376,6 @@ void test_arc_unghost()
bc.evict_piece(pe, jobs);
bc.get_stats(&status);
TEST_EQUAL(status.blocks_read_hit, 0);
TEST_EQUAL(status.write_cache_size, 0);
TEST_EQUAL(status.read_cache_size, 0);
TEST_EQUAL(status.pinned_blocks, 0);
@ -395,7 +391,6 @@ void test_arc_unghost()
bc.cache_hit(pe, (void*)1, false);
bc.get_stats(&status);
TEST_EQUAL(status.blocks_read_hit, 0);
TEST_EQUAL(status.write_cache_size, 0);
// we didn't actually read in any blocks, so the cache size
// is still 0

View File

@ -383,7 +383,8 @@ void test_check_files(std::string const& test_path
aux::session_settings set;
file_pool fp;
libtorrent::asio::io_service ios;
disk_io_thread io(ios, NULL, NULL);
counters cnt;
disk_io_thread io(ios, NULL, cnt, NULL);
disk_buffer_pool dp(16 * 1024, ios, boost::bind(&nop), NULL);
storage_params p;
p.files = &fs;