make disk_io_thread responsible for the storage object. torrents refer to it by index

This commit is contained in:
arvidn 2016-12-31 12:35:10 -05:00 committed by Arvid Norberg
parent 6cfcf45214
commit 4ebc344e48
18 changed files with 299 additions and 203 deletions

View File

@ -548,6 +548,8 @@ namespace libtorrent
session_status status() const;
#endif
void get_cache_info(torrent_handle h, cache_status* ret, int flags) const;
void set_peer_id(peer_id const& id);
void set_key(int key);
std::uint16_t listen_port() const override;

View File

@ -42,6 +42,7 @@ POSSIBILITY OF SUCH DAMAGE.
#include "libtorrent/disk_buffer_holder.hpp"
#include "libtorrent/aux_/vector.hpp"
#include "libtorrent/export.hpp"
#include "libtorrent/storage_defs.hpp"
namespace libtorrent
{
@ -66,6 +67,8 @@ namespace libtorrent
file_exist
};
struct storage_holder;
struct TORRENT_EXTRA_EXPORT disk_interface
{
enum flags_t
@ -80,43 +83,47 @@ namespace libtorrent
volatile_read = 0x10,
};
virtual void async_read(storage_interface* storage, peer_request const& r
virtual storage_holder new_torrent(std::unique_ptr<storage_interface> storage) = 0;
virtual void remove_torrent(storage_index_t) = 0;
virtual storage_interface* get_torrent(storage_index_t) = 0;
virtual void async_read(storage_index_t storage, peer_request const& r
, std::function<void(disk_buffer_holder block, int flags, storage_error const& se)> handler
, void* requester, std::uint8_t flags = 0) = 0;
virtual bool async_write(storage_interface* storage, peer_request const& r
virtual bool async_write(storage_index_t storage, peer_request const& r
, char const* buf, std::shared_ptr<disk_observer> o
, std::function<void(storage_error const&)> handler
, std::uint8_t flags = 0) = 0;
virtual void async_hash(storage_interface* storage, piece_index_t piece, std::uint8_t flags
virtual void async_hash(storage_index_t storage, piece_index_t piece, std::uint8_t flags
, std::function<void(piece_index_t, sha1_hash const&, storage_error const&)> handler, void* requester) = 0;
virtual void async_move_storage(storage_interface* storage, std::string const& p, std::uint8_t flags
virtual void async_move_storage(storage_index_t storage, std::string const& p, std::uint8_t flags
, std::function<void(status_t, std::string const&, storage_error const&)> handler) = 0;
virtual void async_release_files(storage_interface* storage
virtual void async_release_files(storage_index_t storage
, std::function<void()> handler = std::function<void()>()) = 0;
virtual void async_check_files(storage_interface* storage
virtual void async_check_files(storage_index_t storage
, add_torrent_params const* resume_data
, aux::vector<std::string, file_index_t>& links
, std::function<void(status_t, storage_error const&)> handler) = 0;
virtual void async_flush_piece(storage_interface* storage, piece_index_t piece
virtual void async_flush_piece(storage_index_t storage, piece_index_t piece
, std::function<void()> handler = std::function<void()>()) = 0;
virtual void async_stop_torrent(storage_interface* storage
virtual void async_stop_torrent(storage_index_t storage
, std::function<void()> handler = std::function<void()>()) = 0;
virtual void async_rename_file(storage_interface* storage
virtual void async_rename_file(storage_index_t storage
, file_index_t index, std::string const& name
, std::function<void(std::string const&, file_index_t, storage_error const&)> handler) = 0;
virtual void async_delete_files(storage_interface* storage, int options
virtual void async_delete_files(storage_index_t storage, int options
, std::function<void(storage_error const&)> handler) = 0;
virtual void async_set_file_priority(storage_interface* storage
virtual void async_set_file_priority(storage_index_t storage
, aux::vector<std::uint8_t, file_index_t> const& prio
, std::function<void(storage_error const&)> handler) = 0;
virtual void async_clear_piece(storage_interface* storage, piece_index_t index
virtual void async_clear_piece(storage_index_t storage, piece_index_t index
, std::function<void(piece_index_t)> handler) = 0;
virtual void clear_piece(storage_interface* storage, piece_index_t index) = 0;
virtual void clear_piece(storage_index_t storage, piece_index_t index) = 0;
virtual void update_stats_counters(counters& c) const = 0;
virtual void get_cache_info(cache_status* ret, bool no_pieces = true
, storage_interface const* storage = 0) const = 0;
virtual void get_cache_info(cache_status* ret, storage_index_t storage
, bool no_pieces = true, bool session = true) const = 0;
virtual file_pool& files() = 0;
@ -126,6 +133,56 @@ namespace libtorrent
protected:
~disk_interface() {}
};
struct storage_holder
{
storage_holder() = default;
storage_holder(storage_index_t idx, disk_interface& disk_io)
: m_disk_io(&disk_io)
, m_idx(idx)
{}
~storage_holder()
{
if (m_disk_io) m_disk_io->remove_torrent(m_idx);
}
explicit operator bool() const { return m_disk_io != nullptr; }
operator storage_index_t() const
{
TORRENT_ASSERT(m_disk_io);
return m_idx;
}
void reset()
{
if (m_disk_io) m_disk_io->remove_torrent(m_idx);
m_disk_io = nullptr;
}
storage_holder(storage_holder const&) = delete;
storage_holder& operator=(storage_holder const&) = delete;
storage_holder(storage_holder&& rhs)
: m_disk_io(rhs.m_disk_io)
, m_idx(rhs.m_idx)
{
rhs.m_disk_io = nullptr;
}
storage_holder& operator=(storage_holder&& rhs)
{
if (m_disk_io) m_disk_io->remove_torrent(m_idx);
m_disk_io = rhs.m_disk_io;
m_idx = rhs.m_idx;
rhs.m_disk_io = nullptr;
return *this;
}
private:
disk_interface* m_disk_io = nullptr;
storage_index_t m_idx{0};
};
}
#endif

View File

@ -291,51 +291,55 @@ namespace libtorrent
void abort(bool wait);
void async_read(storage_interface* storage, peer_request const& r
storage_holder new_torrent(std::unique_ptr<storage_interface> storage) override;
void remove_torrent(storage_index_t) override;
void async_read(storage_index_t storage, peer_request const& r
, std::function<void(disk_buffer_holder block
, int flags, storage_error const& se)> handler, void* requester, std::uint8_t flags = 0) override;
bool async_write(storage_interface* storage, peer_request const& r
bool async_write(storage_index_t storage, peer_request const& r
, char const* buf, std::shared_ptr<disk_observer> o
, std::function<void(storage_error const&)> handler
, std::uint8_t flags = 0) override;
void async_hash(storage_interface* storage, piece_index_t piece, std::uint8_t flags
void async_hash(storage_index_t storage, piece_index_t piece, std::uint8_t flags
, std::function<void(piece_index_t, sha1_hash const&, storage_error const&)> handler, void* requester) override;
void async_move_storage(storage_interface* storage, std::string const& p, std::uint8_t flags
void async_move_storage(storage_index_t storage, std::string const& p, std::uint8_t flags
, std::function<void(status_t, std::string const&, storage_error const&)> handler) override;
void async_release_files(storage_interface* storage
void async_release_files(storage_index_t storage
, std::function<void()> handler = std::function<void()>()) override;
void async_delete_files(storage_interface* storage, int options
void async_delete_files(storage_index_t storage, int options
, std::function<void(storage_error const&)> handler) override;
void async_check_files(storage_interface* storage
void async_check_files(storage_index_t storage
, add_torrent_params const* resume_data
, aux::vector<std::string, file_index_t>& links
, std::function<void(status_t, storage_error const&)> handler) override;
void async_rename_file(storage_interface* storage, file_index_t index, std::string const& name
void async_rename_file(storage_index_t storage, file_index_t index, std::string const& name
, std::function<void(std::string const&, file_index_t, storage_error const&)> handler) override;
void async_stop_torrent(storage_interface* storage
void async_stop_torrent(storage_index_t storage
, std::function<void()> handler) override;
void async_flush_piece(storage_interface* storage, piece_index_t piece
void async_flush_piece(storage_index_t storage, piece_index_t piece
, std::function<void()> handler = std::function<void()>()) override;
void async_set_file_priority(storage_interface* storage
void async_set_file_priority(storage_index_t storage
, aux::vector<std::uint8_t, file_index_t> const& prio
, std::function<void(storage_error const&)> handler) override;
void async_clear_piece(storage_interface* storage, piece_index_t index
void async_clear_piece(storage_index_t storage, piece_index_t index
, std::function<void(piece_index_t)> handler) override;
// this is not asynchronous and requires that the piece does not
// have any pending buffers. It's meant to be used for pieces that
// were just read and hashed and failed the hash check.
// there should be no read-operations left, and all buffers should
// be discardable
void clear_piece(storage_interface* storage, piece_index_t index) override;
void clear_piece(storage_index_t storage, piece_index_t index) override;
// implements buffer_allocator_interface
void reclaim_blocks(span<aux::block_cache_reference> ref) override;
void free_disk_buffer(char* buf) override { m_disk_cache.free_buffer(buf); }
void trigger_cache_trim();
void update_stats_counters(counters& c) const override;
void get_cache_info(cache_status* ret, bool no_pieces = true
, storage_interface const* storage = 0) const override;
void get_cache_info(cache_status* ret, storage_index_t storage
, bool no_pieces, bool session) const override;
storage_interface* get_torrent(storage_index_t) override;
// this submits all queued up jobs to the thread
void submit_jobs();
@ -433,8 +437,7 @@ namespace libtorrent
// this queues up another job to be submitted
void add_job(disk_io_job* j, bool user_add = true);
void add_fence_job(storage_interface* storage, disk_io_job* j
, bool user_add = true);
void add_fence_job(disk_io_job* j, bool user_add = true);
// assumes l is locked (cache std::mutex).
// writes out the blocks [start, end) (releases the lock
@ -579,6 +582,8 @@ namespace libtorrent
// completion callbacks in m_completed jobs
bool m_job_completions_in_flight = false;
aux::vector<std::shared_ptr<storage_interface>, storage_index_t> m_torrents;
#if TORRENT_USE_ASSERTS
int m_magic = 0x1337;
std::atomic<bool> m_jobs_aborted{false};

View File

@ -40,6 +40,7 @@ POSSIBILITY OF SUCH DAMAGE.
#include "libtorrent/file.hpp"
#include "libtorrent/aux_/time.hpp"
#include "libtorrent/units.hpp"
#include "libtorrent/storage_defs.hpp"
namespace libtorrent
{
@ -93,13 +94,14 @@ namespace libtorrent
// return an open file handle to file at ``file_index`` in the
// file_storage ``fs`` opened at save path ``p``. ``m`` is the
// file open mode (see file::open_mode_t).
file_handle open_file(void* st, std::string const& p
file_handle open_file(storage_index_t st, std::string const& p
, file_index_t file_index, file_storage const& fs, int m, error_code& ec);
// release all files belonging to the specified storage_interface (``st``)
// the overload that takes ``file_index`` releases only the file with
// that index in storage ``st``.
void release(void* st = nullptr);
void release(void* st, file_index_t file_index);
void release();
void release(storage_index_t st);
void release(storage_index_t st, file_index_t file_index);
// update the allowed number of open file handles to ``size``.
void resize(int size);
@ -110,10 +112,10 @@ namespace libtorrent
// internal
void set_low_prio_io(bool b) { m_low_prio_io = b; }
std::vector<pool_file_status> get_status(void* st) const;
std::vector<pool_file_status> get_status(storage_index_t st) const;
#if TORRENT_USE_ASSERTS
bool assert_idle_files(void* st) const;
bool assert_idle_files(storage_index_t st) const;
// remember that this storage has had
// its files deleted. We may not open any
@ -137,7 +139,7 @@ namespace libtorrent
// maps storage pointer, file index pairs to the
// lru entry for the file
std::map<std::pair<void*, file_index_t>, lru_file_entry> m_files;
std::map<std::pair<storage_index_t, file_index_t>, lru_file_entry> m_files;
#if TORRENT_USE_ASSERTS
std::vector<std::pair<std::string, void const*>> m_deleted_storages;
#endif

View File

@ -446,6 +446,9 @@ namespace libtorrent
// initialized in disk_io_thread::perform_async_job
aux::session_settings* m_settings = nullptr;
storage_index_t storage_index() const { return m_storage_index; }
void set_storage_index(storage_index_t st) { m_storage_index = st; }
private:
bool m_need_tick = false;
@ -458,6 +461,8 @@ namespace libtorrent
// the storage_interface destructs. This is because
// the torrent_info object is owned by the torrent.
std::shared_ptr<void> m_torrent;
storage_index_t m_storage_index;
};
// The default implementation of storage_interface. Behaves as a normal

View File

@ -47,6 +47,9 @@ namespace libtorrent
struct file_pool;
class torrent_info;
struct storage_index_tag_t {};
using storage_index_t = aux::strong_typedef<std::uint32_t, storage_index_tag_t>;
// types of storage allocation used for add_torrent_params::storage_mode.
enum storage_mode_t
{

View File

@ -396,7 +396,6 @@ namespace libtorrent
, peer_request r, std::shared_ptr<read_piece_struct> rp);
storage_mode_t storage_mode() const;
storage_interface* get_storage();
// this will flag the torrent as aborted. The main
// loop in session_impl will check for this state
@ -925,8 +924,9 @@ namespace libtorrent
int num_known_peers() const { return m_peer_list ? m_peer_list->num_peers() : 0; }
int num_connect_candidates() const { return m_peer_list ? m_peer_list->num_connect_candidates() : 0; }
storage_interface& storage();
bool has_storage() const { return m_storage.get() != nullptr; }
bool has_storage() const { return bool(m_storage); }
storage_index_t storage() const { return m_storage; }
storage_interface* get_storage_impl() const;
torrent_info const& torrent_file() const
{ return *m_torrent_file; }
@ -997,8 +997,6 @@ namespace libtorrent
{ return m_torrent_file->is_valid(); }
bool are_files_checked() const
{ return m_files_checked; }
bool valid_storage() const
{ return m_storage.get() != nullptr; }
// parses the info section from the given
// bencoded tree and moves the torrent
@ -1146,24 +1144,11 @@ namespace libtorrent
std::int64_t m_total_uploaded = 0;
std::int64_t m_total_downloaded = 0;
// if this pointer is 0, the torrent is in
// a state where the metadata hasn't been
// received yet, or during shutdown.
// the storage_interface keeps the torrent object
// alive by holding a shared_ptr to it and
// the torrent keeps the piece manager alive
// with this shared_ptr. This cycle is
// broken when torrent::abort() is called
// Then the torrent releases the storage_interface
// and when the storage_interface is complete with all
// outstanding disk io jobs (that keeps
// the storage_interface alive) it will destruct
// and release the torrent file. The reason for
// this is that the torrent_info is used by
// the storage_interface, and stored in the
// torrent, so the torrent cannot destruct
// before the storage_interface.
std::shared_ptr<storage_interface> m_storage;
// this is a handle that keeps the storage object in the disk io subsystem
// alive, as well as the index referencing the storage/torrent in the disk
// I/O. When this destructs, the torrent will be removed from the disk
// subsystem.
storage_holder m_storage;
#ifdef TORRENT_USE_OPENSSL
std::shared_ptr<boost::asio::ssl::context> m_ssl_ctx;

View File

@ -161,7 +161,7 @@ namespace libtorrent
struct hash_state
{
create_torrent& ct;
std::shared_ptr<storage_interface> storage;
storage_holder storage;
disk_io_thread& iothread;
piece_index_t piece_counter;
piece_index_t completed_piece;
@ -184,7 +184,7 @@ namespace libtorrent
++st->completed_piece;
if (st->piece_counter < st->ct.files().end_piece())
{
st->iothread.async_hash(st->storage.get(), st->piece_counter
st->iothread.async_hash(st->storage, st->piece_counter
, disk_interface::sequential_access
, std::bind(&on_hash, _1, _2, _3, st), nullptr);
++st->piece_counter;
@ -279,9 +279,9 @@ namespace libtorrent
params.pool = &disk_thread.files();
params.mode = storage_mode_sparse;
std::shared_ptr<storage_interface> storage(default_storage_constructor(params));
storage->set_files(&t.files());
std::unique_ptr<storage_interface> stor(default_storage_constructor(params));
stor->set_files(&t.files());
storage_holder storage = disk_thread.new_torrent(std::move(stor));
settings_pack sett;
sett.set_int(settings_pack::cache_size, 0);
@ -291,10 +291,10 @@ namespace libtorrent
int const piece_read_ahead = std::max(1, 15 * 1024 * 1024 / t.piece_length());
hash_state st = { t, storage, disk_thread, piece_index_t(0), piece_index_t(0), f, ec };
hash_state st = { t, std::move(storage), disk_thread, piece_index_t(0), piece_index_t(0), f, ec };
for (piece_index_t i(0); i < piece_index_t(piece_read_ahead); ++i)
{
disk_thread.async_hash(storage.get(), i, disk_interface::sequential_access
disk_thread.async_hash(st.storage, i, disk_interface::sequential_access
, std::bind(&on_hash, _1, _2, _3, &st), nullptr);
++st.piece_counter;
if (st.piece_counter >= t.files().end_piece()) break;

View File

@ -211,6 +211,30 @@ namespace libtorrent
m_file_pool.resize(max_files);
}
storage_interface* disk_io_thread::get_torrent(storage_index_t const storage)
{
return m_torrents[storage].get();
}
storage_holder disk_io_thread::new_torrent(std::unique_ptr<storage_interface> storage)
{
TORRENT_ASSERT(storage);
auto it = std::find(m_torrents.begin(), m_torrents.end()
, std::shared_ptr<storage_interface>());
storage_index_t const idx((it == m_torrents.end())
? m_torrents.end_index()
: storage_index_t(std::uint32_t(it - m_torrents.begin())));
storage->set_storage_index(idx);
if (it == m_torrents.end()) m_torrents.emplace_back(std::move(storage));
else m_torrents[idx] = std::move(storage);
return storage_holder(idx, *this);
}
void disk_io_thread::remove_torrent(storage_index_t const idx)
{
m_torrents[idx].reset();
}
disk_io_thread::~disk_io_thread()
{
DLOG("destructing disk_io_thread\n");
@ -1513,7 +1537,7 @@ namespace libtorrent
return do_uncached_write(j);
}
void disk_io_thread::async_read(storage_interface* storage, peer_request const& r
void disk_io_thread::async_read(storage_index_t storage, peer_request const& r
, std::function<void(disk_buffer_holder block, int flags, storage_error const& se)> handler
, void* requester, std::uint8_t const flags)
{
@ -1524,7 +1548,7 @@ namespace libtorrent
, r.start / m_disk_cache.block_size());
disk_io_job* j = allocate_job(disk_io_job::read);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->piece = r.piece;
j->d.io.offset = r.start;
j->d.io.buffer_size = std::uint16_t(r.length);
@ -1625,7 +1649,7 @@ namespace libtorrent
return 1;
}
bool disk_io_thread::async_write(storage_interface* storage, peer_request const& r
bool disk_io_thread::async_write(storage_index_t const storage, peer_request const& r
, char const* buf, std::shared_ptr<disk_observer> o
, std::function<void(storage_error const&)> handler
, std::uint8_t const flags)
@ -1639,7 +1663,7 @@ namespace libtorrent
std::memcpy(buffer.get(), buf, r.length);
disk_io_job* j = allocate_job(disk_io_job::write);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->piece = r.piece;
j->d.io.offset = r.start;
j->d.io.buffer_size = std::uint16_t(r.length);
@ -1685,7 +1709,7 @@ namespace libtorrent
TORRENT_ASSERT((r.start % m_disk_cache.block_size()) == 0);
if (storage->is_blocked(j))
if (j->storage->is_blocked(j))
{
// this means the job was queued up inside storage
m_stats_counters.inc_stats_counter(counters::blocked_disk_jobs);
@ -1717,7 +1741,7 @@ namespace libtorrent
// the block and write job were successfully inserted
// into the cache. Now, see if we should trigger a flush
j = allocate_job(disk_io_job::flush_hashed);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->piece = r.piece;
j->flags = flags;
add_job(j);
@ -1734,18 +1758,18 @@ namespace libtorrent
return exceeded;
}
void disk_io_thread::async_hash(storage_interface* storage
void disk_io_thread::async_hash(storage_index_t const storage
, piece_index_t piece, std::uint8_t flags
, std::function<void(piece_index_t, sha1_hash const&, storage_error const&)> handler, void* requester)
{
disk_io_job* j = allocate_job(disk_io_job::hash);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->piece = piece;
j->callback = std::move(handler);
j->flags = flags;
j->requester = requester;
int piece_size = storage->files()->piece_size(piece);
int piece_size = j->storage->files()->piece_size(piece);
// first check to see if the hashing is already done
std::unique_lock<std::mutex> l(m_cache_mutex);
@ -1773,30 +1797,30 @@ namespace libtorrent
add_job(j);
}
void disk_io_thread::async_move_storage(storage_interface* storage
void disk_io_thread::async_move_storage(storage_index_t const storage
, std::string const& p, std::uint8_t const flags
, std::function<void(status_t, std::string const&, storage_error const&)> handler)
{
disk_io_job* j = allocate_job(disk_io_job::move_storage);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->buffer.string = allocate_string_copy(p.c_str());
j->callback = std::move(handler);
j->flags = flags;
add_fence_job(storage, j);
add_fence_job(j);
}
void disk_io_thread::async_release_files(storage_interface* storage
void disk_io_thread::async_release_files(storage_index_t const storage
, std::function<void()> handler)
{
disk_io_job* j = allocate_job(disk_io_job::release_files);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->callback = std::move(handler);
add_fence_job(storage, j);
add_fence_job(j);
}
void disk_io_thread::async_delete_files(storage_interface* storage
void disk_io_thread::async_delete_files(storage_index_t const storage
, int const options
, std::function<void(storage_error const&)> handler)
{
@ -1811,13 +1835,14 @@ namespace libtorrent
disk_io_job* qj = m_generic_io_jobs.m_queued_jobs.get_all();
jobqueue_t to_abort;
storage_interface* to_delete = m_torrents[storage].get();
while (qj)
{
disk_io_job* next = qj->next;
#if TORRENT_USE_ASSERTS
qj->next = nullptr;
#endif
if (qj->storage.get() == storage)
if (qj->storage.get() == to_delete)
to_abort.push_back(qj);
else
m_generic_io_jobs.m_queued_jobs.push_back(qj);
@ -1826,14 +1851,14 @@ namespace libtorrent
l2.unlock();
std::unique_lock<std::mutex> l(m_cache_mutex);
flush_cache(storage, flush_delete_cache, completed_jobs, l);
flush_cache(to_delete, flush_delete_cache, completed_jobs, l);
l.unlock();
disk_io_job* j = allocate_job(disk_io_job::delete_files);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->callback = std::move(handler);
j->buffer.delete_options = options;
add_fence_job(storage, j);
add_fence_job(j);
fail_jobs_impl(storage_error(boost::asio::error::operation_aborted)
, to_abort, completed_jobs);
@ -1842,7 +1867,7 @@ namespace libtorrent
add_completed_jobs(completed_jobs);
}
void disk_io_thread::async_check_files(storage_interface* storage
void disk_io_thread::async_check_files(storage_index_t const storage
, add_torrent_params const* resume_data
, aux::vector<std::string, file_index_t>& links
, std::function<void(status_t, storage_error const&)> handler)
@ -1852,32 +1877,34 @@ namespace libtorrent
links_vector->swap(links);
disk_io_job* j = allocate_job(disk_io_job::check_fastresume);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->buffer.check_resume_data = resume_data;
j->d.links = links_vector;
j->callback = std::move(handler);
add_fence_job(storage, j);
add_fence_job(j);
}
void disk_io_thread::async_rename_file(storage_interface* storage
void disk_io_thread::async_rename_file(storage_index_t const storage
, file_index_t index, std::string const& name
, std::function<void(std::string const&, file_index_t, storage_error const&)> handler)
{
disk_io_job* j = allocate_job(disk_io_job::rename_file);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->file_index = index;
j->buffer.string = allocate_string_copy(name.c_str());
j->callback = std::move(handler);
add_fence_job(storage, j);
add_fence_job(j);
}
void disk_io_thread::async_stop_torrent(storage_interface* storage
void disk_io_thread::async_stop_torrent(storage_index_t const storage
, std::function<void()> handler)
{
// remove outstanding hash jobs belonging to this torrent
std::unique_lock<std::mutex> l2(m_job_mutex);
std::shared_ptr<storage_interface> st
= m_torrents[storage]->shared_from_this();
disk_io_job* qj = m_hash_io_jobs.m_queued_jobs.get_all();
jobqueue_t to_abort;
@ -1887,7 +1914,7 @@ namespace libtorrent
#if TORRENT_USE_ASSERTS
qj->next = nullptr;
#endif
if (qj->storage.get() == storage)
if (qj->storage.get() == st.get())
to_abort.push_back(qj);
else
m_hash_io_jobs.m_queued_jobs.push_back(qj);
@ -1896,9 +1923,9 @@ namespace libtorrent
l2.unlock();
disk_io_job* j = allocate_job(disk_io_job::stop_torrent);
j->storage = storage->shared_from_this();
j->storage = st;
j->callback = std::move(handler);
add_fence_job(storage, j);
add_fence_job(j);
jobqueue_t completed_jobs;
fail_jobs_impl(storage_error(boost::asio::error::operation_aborted)
@ -1907,12 +1934,12 @@ namespace libtorrent
add_completed_jobs(completed_jobs);
}
void disk_io_thread::async_flush_piece(storage_interface* storage
void disk_io_thread::async_flush_piece(storage_index_t const storage
, piece_index_t const piece
, std::function<void()> handler)
{
disk_io_job* j = allocate_job(disk_io_job::flush_piece);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->piece = piece;
j->callback = std::move(handler);
@ -1927,25 +1954,25 @@ namespace libtorrent
add_job(j);
}
void disk_io_thread::async_set_file_priority(storage_interface* storage
void disk_io_thread::async_set_file_priority(storage_index_t const storage
, aux::vector<std::uint8_t, file_index_t> const& prios
, std::function<void(storage_error const&)> handler)
{
aux::vector<std::uint8_t, file_index_t>* p = new aux::vector<std::uint8_t, file_index_t>(prios);
disk_io_job* j = allocate_job(disk_io_job::file_priority);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->buffer.priorities = p;
j->callback = std::move(handler);
add_fence_job(storage, j);
add_fence_job(j);
}
void disk_io_thread::async_clear_piece(storage_interface* storage
void disk_io_thread::async_clear_piece(storage_index_t const storage
, piece_index_t const index, std::function<void(piece_index_t)> handler)
{
disk_io_job* j = allocate_job(disk_io_job::clear_piece);
j->storage = storage->shared_from_this();
j->storage = m_torrents[storage]->shared_from_this();
j->piece = index;
j->callback = std::move(handler);
@ -1955,15 +1982,17 @@ namespace libtorrent
// TODO: this is potentially very expensive. One way to solve
// it would be to have a fence for just this one piece.
add_fence_job(storage, j);
add_fence_job(j);
}
void disk_io_thread::clear_piece(storage_interface* storage
void disk_io_thread::clear_piece(storage_index_t const storage
, piece_index_t const index)
{
storage_interface* st = m_torrents[storage].get();
std::unique_lock<std::mutex> l(m_cache_mutex);
cached_piece_entry* pe = m_disk_cache.find_piece(storage, index);
cached_piece_entry* pe = m_disk_cache.find_piece(st, index);
if (pe == nullptr) return;
TORRENT_PIECE_ASSERT(pe->hashing == false, pe);
pe->hashing_done = 0;
@ -2571,9 +2600,11 @@ namespace libtorrent
m_disk_cache.update_stats_counters(c);
}
void disk_io_thread::get_cache_info(cache_status* ret, bool const no_pieces
, storage_interface const* storage) const
void disk_io_thread::get_cache_info(cache_status* ret, storage_index_t st
, bool const no_pieces, bool const session) const
{
std::shared_ptr<storage_interface> storage = m_torrents[st];
std::unique_lock<std::mutex> l(m_cache_mutex);
#ifndef TORRENT_NO_DEPRECATE
@ -2625,13 +2656,14 @@ namespace libtorrent
{
int const block_size = m_disk_cache.block_size();
if (storage)
if (!session)
{
TORRENT_ASSERT(storage);
ret->pieces.reserve(storage->num_pieces());
for (auto pe : storage->cached_pieces())
{
TORRENT_ASSERT(pe->storage.get() == storage);
TORRENT_ASSERT(pe->storage.get() == storage.get());
if (pe->cache_state == cached_piece_entry::read_lru2_ghost
|| pe->cache_state == cached_piece_entry::read_lru1_ghost)
@ -2798,8 +2830,7 @@ namespace libtorrent
return retry_job;
}
void disk_io_thread::add_fence_job(storage_interface* storage, disk_io_job* j
, bool user_add)
void disk_io_thread::add_fence_job(disk_io_job* j, bool const user_add)
{
// if this happens, it means we started to shut down
// the disk threads too early. We have to post all jobs
@ -2815,7 +2846,7 @@ namespace libtorrent
disk_io_job* fj = allocate_job(disk_io_job::flush_storage);
fj->storage = j->storage;
int ret = storage->raise_fence(j, fj, m_stats_counters);
int ret = j->storage->raise_fence(j, fj, m_stats_counters);
if (ret == disk_job_fence::fence_post_fence)
{
std::unique_lock<std::mutex> l(m_job_mutex);
@ -2859,7 +2890,7 @@ namespace libtorrent
immediate_execute();
}
void disk_io_thread::add_job(disk_io_job* j, bool user_add)
void disk_io_thread::add_job(disk_io_job* j, bool const user_add)
{
TORRENT_ASSERT(m_magic == 0x1337);

View File

@ -95,7 +95,7 @@ namespace libtorrent
}
#endif // TORRENT_WINDOWS
file_handle file_pool::open_file(void* st, std::string const& p
file_handle file_pool::open_file(storage_index_t st, std::string const& p
, file_index_t const file_index, file_storage const& fs, int m, error_code& ec)
{
// potentially used to hold a reference to a file object that's
@ -115,7 +115,6 @@ namespace libtorrent
== m_deleted_storages.end());
#endif
TORRENT_ASSERT(st != nullptr);
TORRENT_ASSERT(is_complete(p));
TORRENT_ASSERT((m & file::rw_mask) == file::read_only
|| (m & file::rw_mask) == file::read_write);
@ -178,7 +177,7 @@ namespace libtorrent
return file_ptr;
}
std::vector<pool_file_status> file_pool::get_status(void* st) const
std::vector<pool_file_status> file_pool::get_status(storage_index_t const st) const
{
std::vector<pool_file_status> ret;
{
@ -211,7 +210,7 @@ namespace libtorrent
l.lock();
}
void file_pool::release(void* st, file_index_t file_index)
void file_pool::release(storage_index_t const st, file_index_t file_index)
{
std::unique_lock<std::mutex> l(m_mutex);
@ -228,17 +227,17 @@ namespace libtorrent
}
// closes files belonging to the specified
// storage. If 0 is passed, all files are closed
void file_pool::release(void* st)
// storage, or all if none is specified.
void file_pool::release()
{
std::unique_lock<std::mutex> l(m_mutex);
m_files.clear();
l.unlock();
}
if (st == nullptr)
{
m_files.clear();
l.unlock();
return;
}
void file_pool::release(storage_index_t const st)
{
std::unique_lock<std::mutex> l(m_mutex);
auto begin = m_files.lower_bound(std::make_pair(st, file_index_t(0)));
auto const end = m_files.upper_bound(std::make_pair(st
@ -264,7 +263,7 @@ namespace libtorrent
m_deleted_storages.erase(m_deleted_storages.begin());
}
bool file_pool::assert_idle_files(void* st) const
bool file_pool::assert_idle_files(storage_index_t const st) const
{
std::unique_lock<std::mutex> l(m_mutex);

View File

@ -2795,7 +2795,7 @@ namespace libtorrent
if (t->is_deleted()) return;
bool const exceeded = m_disk_thread.async_write(&t->storage(), p, data, self()
bool const exceeded = m_disk_thread.async_write(t->storage(), p, data, self()
, std::bind(&peer_connection::on_disk_write_complete
, self(), _1, p, t));
@ -3008,7 +3008,7 @@ namespace libtorrent
// when this returns, all outstanding jobs to the
// piece are done, and we can restore it, allowing
// new requests to it
m_disk_thread.async_clear_piece(&t->storage(), p.piece
m_disk_thread.async_clear_piece(t->storage(), p.piece
, std::bind(&torrent::on_piece_fail_sync, t, _1, block_finished));
}
else
@ -5089,7 +5089,7 @@ namespace libtorrent
#endif
// this means we're in seed mode and we haven't yet
// verified this piece (r.piece)
m_disk_thread.async_hash(&t->storage(), r.piece, 0
m_disk_thread.async_hash(t->storage(), r.piece, 0
, std::bind(&peer_connection::on_seed_mode_hashed, self()
, _1, _2, _3), this);
t->verifying(r.piece);
@ -5125,7 +5125,7 @@ namespace libtorrent
TORRENT_ASSERT(r.piece >= piece_index_t(0));
TORRENT_ASSERT(r.piece < t->torrent_file().end_piece());
m_disk_thread.async_read(&t->storage(), r
m_disk_thread.async_read(t->storage(), r
, std::bind(&peer_connection::on_disk_read_complete
, self(), _1, _2, _3, r, clock_type::now()), this);
}

View File

@ -451,16 +451,7 @@ namespace libtorrent
void session_handle::get_cache_info(cache_status* ret
, torrent_handle h, int flags) const
{
storage_interface* st = nullptr;
std::shared_ptr<torrent> t = h.m_torrent.lock();
if (t)
{
if (t->has_storage())
st = &t->storage();
else
flags = session::disk_cache_no_pieces;
}
m_impl->disk_thread().get_cache_info(ret, flags & session::disk_cache_no_pieces, st);
sync_call(&session_impl::get_cache_info, h, ret, flags);
}
#ifndef TORRENT_NO_DEPRECATE

View File

@ -5500,6 +5500,25 @@ namespace aux {
}
#endif // TORRENT_NO_DEPRECATE
void session_impl::get_cache_info(torrent_handle h, cache_status* ret, int flags) const
{
storage_index_t st{0};
bool whole_session = true;
std::shared_ptr<torrent> t = h.m_torrent.lock();
if (t)
{
if (t->has_storage())
{
st = t->storage();
whole_session = false;
}
else
flags = session::disk_cache_no_pieces;
}
m_disk_thread.get_cache_info(ret, st
, flags & session::disk_cache_no_pieces, whole_session);
}
#ifndef TORRENT_DISABLE_DHT
void session_impl::start_dht()

View File

@ -93,7 +93,7 @@ namespace
{
if (i->first.block_index == pb.block_index)
{
m_torrent.session().disk_thread().async_read(&m_torrent.storage()
m_torrent.session().disk_thread().async_read(m_torrent.storage()
, r, std::bind(&smart_ban_plugin::on_read_ok_block
, shared_from_this(), *i, i->second.peer->address(), _1, r.length, _2, _3)
, reinterpret_cast<void*>(1));
@ -150,7 +150,7 @@ namespace
// since the piece has failed, this block is very likely to be replaced with a newly
// downloaded one very soon, and to get a block by reference would fail, since the
// block read will have been deleted by the time it gets back to the network thread
m_torrent.session().disk_thread().async_read(&m_torrent.storage(), r
m_torrent.session().disk_thread().async_read(m_torrent.storage(), r
, std::bind(&smart_ban_plugin::on_read_failed_block
, shared_from_this(), pb, (*i)->address(), _1, r.length, _2, _3)
, reinterpret_cast<torrent_peer*>(1)

View File

@ -408,7 +408,7 @@ namespace libtorrent
// this may be called from a different
// thread than the disk thread
m_pool.release(this);
m_pool.release(storage_index());
}
void default_storage::need_partfile()
@ -586,7 +586,7 @@ namespace libtorrent
}
// close files that were opened in write mode
m_pool.release(this);
m_pool.release(storage_index());
#if defined TORRENT_DEBUG_FILE_LEAKS
print_open_files("release files", m_files.name().c_str());
@ -640,7 +640,7 @@ namespace libtorrent
{
if (index < file_index_t(0) || index >= files().end_file()) return;
std::string old_name = files().file_path(index, m_save_path);
m_pool.release(this, index);
m_pool.release(storage_index(), index);
// if the old file doesn't exist, just succeed and change the filename
// that will be created. This shortcut is important because the
@ -711,7 +711,7 @@ namespace libtorrent
}
// make sure we don't have the files open
m_pool.release(this);
m_pool.release(storage_index());
#if defined TORRENT_DEBUG_FILE_LEAKS
print_open_files("release files", m_files.name().c_str());
@ -739,7 +739,7 @@ namespace libtorrent
// threads to hold any references to any files
// in this file storage. Assert that that's the
// case
if (!m_pool.assert_idle_files(this))
if (!m_pool.assert_idle_files(storage_index()))
{
#if defined TORRENT_DEBUG_FILE_LEAKS
print_open_files("delete-files idle assert failed", m_files.name().c_str());
@ -749,7 +749,7 @@ namespace libtorrent
#endif
// make sure we don't have the files open
m_pool.release(this);
m_pool.release(storage_index());
// if there's a part file open, make sure to destruct it to have it
// release the underlying part file. Otherwise we may not be able to
@ -992,7 +992,7 @@ namespace libtorrent
}
}
m_pool.release(this);
m_pool.release(storage_index());
#if defined TORRENT_DEBUG_FILE_LEAKS
print_open_files("release files", m_files.name().c_str());
@ -1298,8 +1298,8 @@ namespace libtorrent
mode |= file::no_cache;
}
file_handle ret = m_pool.open_file(const_cast<default_storage*>(this)
, m_save_path, file, files(), mode, ec);
file_handle ret = m_pool.open_file(storage_index(), m_save_path, file
, files(), mode, ec);
if (ec && (mode & file::lock_file))
{
// we failed to open the file and we're trying to lock it. It's
@ -1307,8 +1307,8 @@ namespace libtorrent
// file in use (but waiting to be closed). Just retry to open it
// without locking.
mode &= ~file::lock_file;
ret = m_pool.open_file(const_cast<default_storage*>(this)
, m_save_path, file, files(), mode, ec);
ret = m_pool.open_file(storage_index(), m_save_path, file, files()
, mode, ec);
}
return ret;
}

View File

@ -857,7 +857,7 @@ namespace libtorrent
for (int i = 0; i < blocks_in_piece; ++i, r.start += block_size())
{
r.length = (std::min)(piece_size - r.start, block_size());
m_ses.disk_thread().async_read(&storage(), r
m_ses.disk_thread().async_read(m_storage, r
, std::bind(&torrent::on_disk_read_complete
, shared_from_this(), _1, _2, _3, r, rp), reinterpret_cast<void*>(1));
}
@ -1163,10 +1163,9 @@ namespace libtorrent
storage_mode_t torrent::storage_mode() const
{ return storage_mode_t(m_storage_mode); }
storage_interface* torrent::get_storage()
storage_interface* torrent::get_storage_impl() const
{
if (!m_storage) return nullptr;
return m_storage.get();
return m_ses.disk_thread().get_torrent(m_storage);
}
void torrent::need_picker()
@ -1265,7 +1264,7 @@ namespace libtorrent
p.length = std::min(piece_size - p.start, int(block_size()));
m_stats_counters.inc_stats_counter(counters::queued_write_bytes, p.length);
m_ses.disk_thread().async_write(&storage(), p, data + p.start, nullptr
m_ses.disk_thread().async_write(m_storage, p, data + p.start, nullptr
, std::bind(&torrent::on_disk_write_complete
, shared_from_this(), _1, p));
@ -1623,11 +1622,13 @@ namespace libtorrent
params.info = m_torrent_file.get();
TORRENT_ASSERT(m_storage_constructor);
m_storage.reset(m_storage_constructor(params));
m_storage->set_files(&m_torrent_file->files());
std::unique_ptr<storage_interface> storage(m_storage_constructor(params));
storage->set_files(&m_torrent_file->files());
// the shared_from_this() will create an intentional
// cycle of ownership, se the hpp file for description.
m_storage->set_owner(shared_from_this());
storage->set_owner(shared_from_this());
m_storage = m_ses.disk_thread().new_torrent(std::move(storage));
}
peer_connection* torrent::find_lowest_ranking_peer() const
@ -1891,7 +1892,7 @@ namespace libtorrent
m_outstanding_check_files = true;
#endif
m_ses.disk_thread().async_check_files(
m_storage.get(), m_add_torrent_params ? m_add_torrent_params.get() : nullptr
m_storage, m_add_torrent_params ? m_add_torrent_params.get() : nullptr
, links, std::bind(&torrent::on_resume_data_checked
, shared_from_this(), _1, _2));
// async_check_files will gut links
@ -2169,7 +2170,7 @@ namespace libtorrent
// now.
leave_seed_mode(true);
m_ses.disk_thread().async_release_files(m_storage.get());
m_ses.disk_thread().async_release_files(m_storage);
// forget that we have any pieces
m_have_all = false;
@ -2202,7 +2203,7 @@ namespace libtorrent
m_add_torrent_params.reset();
aux::vector<std::string, file_index_t> links;
m_ses.disk_thread().async_check_files(m_storage.get(), nullptr
m_ses.disk_thread().async_check_files(m_storage, nullptr
, links, std::bind(&torrent::on_force_recheck
, shared_from_this(), _1, _2));
}
@ -2269,7 +2270,7 @@ namespace libtorrent
for (int i = 0; i < num_outstanding; ++i)
{
m_ses.disk_thread().async_hash(m_storage.get(), m_checking_piece
m_ses.disk_thread().async_hash(m_storage, m_checking_piece
, disk_interface::sequential_access | disk_interface::volatile_read
, std::bind(&torrent::on_piece_hashed
, shared_from_this(), _1, _2, _3), reinterpret_cast<void*>(1));
@ -2362,7 +2363,7 @@ namespace libtorrent
{
// if the hash failed, remove it from the cache
if (m_storage)
m_ses.disk_thread().clear_piece(m_storage.get(), piece);
m_ses.disk_thread().clear_piece(m_storage, piece);
}
if (m_num_checked_pieces < m_torrent_file->end_piece())
@ -2392,7 +2393,7 @@ namespace libtorrent
return;
}
m_ses.disk_thread().async_hash(m_storage.get(), m_checking_piece
m_ses.disk_thread().async_hash(m_storage, m_checking_piece
, disk_interface::sequential_access | disk_interface::volatile_read
, std::bind(&torrent::on_piece_hashed
, shared_from_this(), _1, _2, _3), reinterpret_cast<void*>(1));
@ -3930,7 +3931,7 @@ namespace libtorrent
// make the disk cache flush the piece to disk
if (m_storage)
m_ses.disk_thread().async_flush_piece(m_storage.get(), index);
m_ses.disk_thread().async_flush_piece(m_storage, index);
m_picker->piece_passed(index);
update_gauge();
we_have(index);
@ -4121,7 +4122,7 @@ namespace libtorrent
// don't do this until after the plugins have had a chance
// to read back the blocks that failed, for blame purposes
// this way they have a chance to hit the cache
m_ses.disk_thread().async_clear_piece(m_storage.get(), index
m_ses.disk_thread().async_clear_piece(m_storage, index
, std::bind(&torrent::on_piece_sync, shared_from_this(), _1));
}
else
@ -4309,9 +4310,9 @@ namespace libtorrent
// post a message to the main thread to destruct
// the torrent object from there
if (m_storage.get())
if (m_storage)
{
m_ses.disk_thread().async_stop_torrent(m_storage.get()
m_ses.disk_thread().async_stop_torrent(m_storage
, std::bind(&torrent::on_cache_flushed, shared_from_this()));
}
else
@ -4465,7 +4466,9 @@ namespace libtorrent
{
uintptr_t const self = reinterpret_cast<uintptr_t>(this);
uintptr_t const ses = reinterpret_cast<uintptr_t>(&m_ses);
uintptr_t const storage = reinterpret_cast<uintptr_t>(m_storage.get());
std::uint32_t const storage = m_storage
? static_cast<std::uint32_t>(static_cast<storage_index_t>(m_storage))
: 0;
sha1_hash const h = hasher(reinterpret_cast<char const*>(&self), sizeof(self))
.update(reinterpret_cast<char const*>(&storage), sizeof(storage))
.update(reinterpret_cast<char const*>(&ses), sizeof(ses))
@ -4931,7 +4934,7 @@ namespace libtorrent
// storage may be nullptr during construction and shutdown
if (m_torrent_file->num_pieces() > 0 && m_storage)
{
m_ses.disk_thread().async_set_file_priority(m_storage.get()
m_ses.disk_thread().async_set_file_priority(m_storage
, m_file_priority, std::bind(&torrent::on_file_priority, this, _1));
}
@ -4970,7 +4973,7 @@ namespace libtorrent
// storage may be nullptr during shutdown
if (m_storage)
{
m_ses.disk_thread().async_set_file_priority(m_storage.get()
m_ses.disk_thread().async_set_file_priority(m_storage
, m_file_priority, std::bind(&torrent::on_file_priority, this, _1));
}
update_piece_priorities();
@ -7276,7 +7279,7 @@ namespace libtorrent
if (m_storage)
{
// we need to keep the object alive during this operation
m_ses.disk_thread().async_release_files(m_storage.get()
m_ses.disk_thread().async_release_files(m_storage
, std::bind(&torrent::on_cache_flushed, shared_from_this()));
}
@ -7569,7 +7572,7 @@ namespace libtorrent
TORRENT_UNUSED(fs);
// storage may be nullptr during shutdown
if (!m_storage.get())
if (!m_storage)
{
if (alerts().should_post<file_rename_failed_alert>())
alerts().emplace_alert<file_rename_failed_alert>(get_handle()
@ -7577,7 +7580,7 @@ namespace libtorrent
return;
}
m_ses.disk_thread().async_rename_file(m_storage.get(), index, name
m_ses.disk_thread().async_rename_file(m_storage, index, name
, std::bind(&torrent::on_file_renamed, shared_from_this(), _1, _2, _3));
return;
}
@ -7606,14 +7609,14 @@ namespace libtorrent
}
// storage may be nullptr during shutdown
if (m_storage.get())
if (m_storage)
{
#if TORRENT_USE_UNC_PATHS
std::string path = canonicalize_path(save_path);
#else
std::string const& path = save_path;
#endif
m_ses.disk_thread().async_move_storage(m_storage.get(), path, std::uint8_t(flags)
m_ses.disk_thread().async_move_storage(m_storage, path, std::uint8_t(flags)
, std::bind(&torrent::on_storage_moved, shared_from_this(), _1, _2, _3));
m_moving_storage = true;
}
@ -7659,13 +7662,6 @@ namespace libtorrent
}
catch (...) { handle_exception(); }
storage_interface& torrent::storage()
{
TORRENT_ASSERT(m_storage.get());
return *m_storage;
}
torrent_handle torrent::get_handle()
{
TORRENT_ASSERT(is_single_thread());
@ -8077,10 +8073,10 @@ namespace libtorrent
stop_announcing();
// storage may be nullptr during shutdown
if (m_storage.get())
if (m_storage)
{
TORRENT_ASSERT(m_storage);
m_ses.disk_thread().async_delete_files(m_storage.get(), options
m_ses.disk_thread().async_delete_files(m_storage, options
, std::bind(&torrent::on_files_deleted, shared_from_this(), _1));
m_deleted = true;
return true;
@ -8356,8 +8352,8 @@ namespace libtorrent
m_save_resume_flags = std::uint8_t(flags);
state_updated();
if ((flags & torrent_handle::flush_disk_cache) && m_storage.get())
m_ses.disk_thread().async_release_files(m_storage.get());
if ((flags & torrent_handle::flush_disk_cache) && m_storage)
m_ses.disk_thread().async_release_files(m_storage);
state_updated();
@ -8386,7 +8382,7 @@ namespace libtorrent
TORRENT_ASSERT(m_abort);
return;
}
m_ses.disk_thread().async_release_files(m_storage.get()
m_ses.disk_thread().async_release_files(m_storage
, std::bind(&torrent::on_cache_flushed, shared_from_this()));
}
@ -8488,10 +8484,10 @@ namespace libtorrent
{
// this will make the storage close all
// files and flush all cached data
if (m_storage.get() && clear_disk_cache)
if (m_storage && clear_disk_cache)
{
// the torrent_paused alert will be posted from on_torrent_paused
m_ses.disk_thread().async_stop_torrent(m_storage.get()
m_ses.disk_thread().async_stop_torrent(m_storage
, std::bind(&torrent::on_torrent_paused, shared_from_this()));
}
else
@ -10099,9 +10095,9 @@ namespace libtorrent
{
// picker().mark_as_checking(piece);
TORRENT_ASSERT(m_storage.get());
TORRENT_ASSERT(m_storage);
m_ses.disk_thread().async_hash(m_storage.get(), piece, 0
m_ses.disk_thread().async_hash(m_storage, piece, 0
, std::bind(&torrent::on_piece_verified, shared_from_this(), _1, _2, _3)
, reinterpret_cast<void*>(1));
}

View File

@ -597,7 +597,7 @@ namespace libtorrent
storage_interface* torrent_handle::get_storage_impl() const
{
return sync_call_ret<storage_interface*>(nullptr, &torrent::get_storage);
return sync_call_ret<storage_interface*>(nullptr, &torrent::get_storage_impl);
}
bool torrent_handle::is_valid() const
@ -669,7 +669,7 @@ namespace libtorrent
std::shared_ptr<torrent> t = m_torrent.lock();
if (!t || !t->has_storage()) return;
session_impl& ses = static_cast<session_impl&>(t->session());
status = ses.disk_thread().files().get_status(&t->storage());
status = ses.disk_thread().files().get_status(t->storage());
}
#endif
@ -690,7 +690,7 @@ namespace libtorrent
std::shared_ptr<torrent> t = m_torrent.lock();
if (!t || !t->has_storage()) return {};
session_impl& ses = static_cast<session_impl&>(t->session());
return ses.disk_thread().files().get_status(&t->storage());
return ses.disk_thread().files().get_status(t->storage());
}
void torrent_handle::scrape_tracker(int idx) const

View File

@ -474,14 +474,15 @@ void test_check_files(std::string const& test_path
p.pool = &fp;
p.mode = storage_mode;
std::shared_ptr<storage_interface> pm = std::make_shared<default_storage>(p);
std::unique_ptr<storage_interface> pm(new default_storage(p));
pm->set_files(&fs);
auto st = io.new_torrent(std::move(pm));
std::mutex lock;
bool done = false;
add_torrent_params frd;
aux::vector<std::string, file_index_t> links;
io.async_check_files(pm.get(), &frd, links
io.async_check_files(st, &frd, links
, std::bind(&on_check_resume_data, _1, _2, &done));
io.submit_jobs();
ios.reset();