added seed mode support (lazy hash checking)

This commit is contained in:
Arvid Norberg 2009-02-03 07:46:24 +00:00
parent a0333f82db
commit 1ac8f806fa
19 changed files with 481 additions and 230 deletions

View File

@ -1,3 +1,5 @@
* added 'seed mode', which assumes that all files are complete
and checks hashes lazily, as blocks are requested
* added new extension for file attributes (executable and hidden)
* added support for unbuffered I/O for aligned files
* added workaround for sparse file issue on Windows Vista

View File

@ -288,6 +288,7 @@ add_torrent()
bool duplicate_is_error;
storage_constructor_type storage;
void* userdata;
bool seed_mode;
};
torrent_handle add_torrent(add_torrent_params const& params);
@ -361,6 +362,17 @@ that needs to be implemented for a custom storage, see `storage_interface`_.
The ``userdata`` parameter is optional and will be passed on to the extension
constructor functions, if any (see `add_extension()`_).
If ``seed_mode`` is set to true, libtorrent will assume that all files are present
for this torrent and that they all match the hashes in the torrent file. Each time
a peer requests to download a block, the piece is verified against the hash, unless
it has been verified already. If a hash fails, the torrent will automatically leave
the seed mode and recheck all the files. The use case for this mode is if a torrent
is created and seeded, or if the user already know that the files are complete, this
is a way to avoid the initial file checks, and significantly reduce the startup time.
Setting ``seed_mode`` on a torrent without metadata (a .torrent file) is a no-op
and will be ignored.
The torrent_handle_ returned by ``add_torrent()`` can be used to retrieve information
about the torrent's progress, its peers etc. It is also used to abort a torrent.
@ -2620,6 +2632,8 @@ It contains the following fields::
bool has_incoming;
int sparse_regions;
bool seed_mode;
};
``progress`` is a value in the range [0, 1], that represents the progress of the
@ -2814,6 +2828,11 @@ to this torrent.'
torrent. This is an interesting metric on windows vista, since there is
a limit on the number of sparse regions in a single file there.
``seed_mode`` is true if the torrent is in seed_mode. If the torrent was
started in seed mode, it will leave seed mode once all pieces have been
checked or as soon as one piece fails the hash check.
peer_info
=========
@ -4982,7 +5001,7 @@ this::
// non virtual functions
disk_io_thread* io_thread();
disk_buffer_pool* disk_pool();
void set_error(boost::filesystem::path const& file, error_code const& ec) const;
error_code const& error() const;
std::string const& error_file() const;
@ -5189,6 +5208,22 @@ This function should delete all files and directories belonging to this storage.
Returning ``true`` indicates an error occurred.
The ``disk_buffer_pool`` is used to allocate and free disk buffers. It has the
following members::
struct disk_buffer_pool : boost::noncopyable
{
char* allocate_buffer(char const* category);
void free_buffer(char* buf);
char* allocate_buffers(int blocks, char const* category);
void free_buffers(char* buf, int blocks);
int block_size() const { return m_block_size; }
void release_memory();
};
magnet links
============

View File

@ -244,12 +244,14 @@ namespace libtorrent
if (m_size && (bits & 7)) m_bytes[(m_size + 7) / 8 - 1] &= 0xff << (7 - (bits & 7));
}
void free() { dealloc(); m_size = 0; }
private:
void dealloc() { if (m_own) std::free(m_bytes); m_bytes = 0; }
unsigned char* m_bytes;
int m_size; // in bits
bool m_own;
int m_size:31; // in bits
bool m_own:1;
};
}

View File

@ -425,6 +425,7 @@ private:
// the number of bytes in the send buffer
// that have been encrypted (only used for
// encrypted connections)
public:
int m_encrypted_bytes;
#endif

View File

@ -91,6 +91,7 @@ namespace libtorrent
, clear_read_cache
, abort_torrent
, update_settings
, read_and_hash
};
action_t action;
@ -260,6 +261,8 @@ namespace libtorrent
cache_t::iterator find_cached_piece(
cache_t& cache, disk_io_job const& j
, mutex_t::scoped_lock& l);
int copy_from_piece(cache_t::iterator p, bool& hit
, disk_io_job const& j, mutex_t::scoped_lock& l);
// write cache operations
void flush_oldest_piece(mutex_t::scoped_lock& l);
@ -271,13 +274,16 @@ namespace libtorrent
// read cache operations
bool clear_oldest_read_piece(cache_t::iterator ignore
, mutex_t::scoped_lock& l);
int read_into_piece(cached_piece_entry& p, int start_block, mutex_t::scoped_lock& l);
int read_into_piece(cached_piece_entry& p, int start_block
, int options, mutex_t::scoped_lock& l);
int cache_read_block(disk_io_job const& j, mutex_t::scoped_lock& l);
int cache_read_piece(disk_io_job const& j, mutex_t::scoped_lock& l);
void free_piece(cached_piece_entry& p, mutex_t::scoped_lock& l);
bool make_room(int num_blocks
, cache_t::iterator ignore
, mutex_t::scoped_lock& l);
int try_read_from_cache(disk_io_job const& j);
int read_piece_from_cache_and_hash(disk_io_job const& j, sha1_hash& h);
// this mutex only protects m_jobs, m_queue_buffer_size
// and m_abort

View File

@ -49,7 +49,8 @@ namespace libtorrent
enum error_code_enum
{
no_error = 0,
file_collision
file_collision,
failed_hash_check
};
}

View File

@ -173,6 +173,7 @@ namespace libtorrent
, duplicate_is_error(false)
, storage(sc)
, userdata(0)
, seed_mode(false)
{}
boost::intrusive_ptr<torrent_info> ti;
@ -187,6 +188,7 @@ namespace libtorrent
bool duplicate_is_error;
storage_constructor_type storage;
void* userdata;
bool seed_mode;
};
class TORRENT_EXPORT session: public boost::noncopyable, aux::eh_initializer

View File

@ -191,7 +191,7 @@ namespace libtorrent
session_settings* m_settings;
};
typedef storage_interface* (&storage_constructor_type)(
typedef storage_interface* (*storage_constructor_type)(
file_storage const&, fs::path const&, file_pool&);
TORRENT_EXPORT storage_interface* default_storage_constructor(
@ -234,6 +234,11 @@ namespace libtorrent
, boost::function<void(int, disk_io_job const&)> const& handler
, int priority = 0);
void async_read_and_hash(
peer_request const& r
, boost::function<void(int, disk_io_job const&)> const& handler
, int priority = 0);
void async_write(
peer_request const& r
, disk_buffer_holder& buffer

View File

@ -83,6 +83,7 @@ namespace libtorrent
struct bitfield;
struct announce_entry;
struct tracker_request;
struct add_torrent_params;
namespace aux
{
@ -118,36 +119,8 @@ namespace libtorrent
{
public:
torrent(
aux::session_impl& ses
, boost::intrusive_ptr<torrent_info> tf
, fs::path const& save_path
, tcp::endpoint const& net_interface
, storage_mode_t m_storage_mode
, int block_size
, storage_constructor_type sc
, bool paused
, std::vector<char>* resume_data
, int seq
, bool auto_managed);
// used with metadata-less torrents
// (the metadata is downloaded from the peers)
torrent(
aux::session_impl& ses
, char const* tracker_url
, sha1_hash const& info_hash
, char const* name
, fs::path const& save_path
, tcp::endpoint const& net_interface
, storage_mode_t m_storage_mode
, int block_size
, storage_constructor_type sc
, bool paused
, std::vector<char>* resume_data
, int seq
, bool auto_managed);
torrent(aux::session_impl& ses, tcp::endpoint const& net_interface
, int block_size, int seq, add_torrent_params const& p);
~torrent();
#ifndef TORRENT_DISABLE_ENCRYPTION
@ -691,6 +664,34 @@ namespace libtorrent
int sequence_number() const { return m_sequence_number; }
bool seed_mode() const { return m_seed_mode; }
void leave_seed_mode(bool seed)
{
if (!m_seed_mode) return;
m_seed_mode = false;
// seed is false if we turned out not
// to be a seed after all
if (!seed) force_recheck();
m_num_verified = 0;
m_verified.free();
}
bool all_verified() const
{ return m_num_verified == m_torrent_file->num_pieces(); }
bool verified_piece(int piece) const
{
TORRENT_ASSERT(piece < int(m_verified.size()));
TORRENT_ASSERT(piece >= 0);
return m_verified.get_bit(piece);
}
void verified(int piece)
{
TORRENT_ASSERT(piece < int(m_verified.size()));
TORRENT_ASSERT(piece >= 0);
TORRENT_ASSERT(m_verified.get_bit(piece) == false);
++m_num_verified;
m_verified.set_bit(piece);
}
private:
void on_files_deleted(int ret, disk_io_job const& j);
@ -866,6 +867,14 @@ namespace libtorrent
fs::path m_save_path;
// each bit represents a piece. a set bit means
// the piece has had its hash verified. This
// is only used in seed mode (when m_seed_mode
// is true)
bitfield m_verified;
// m_num_verified = m_verified.count()
int m_num_verified;
// determines the storage state for this torrent.
storage_mode_t m_storage_mode;
@ -1029,6 +1038,12 @@ namespace libtorrent
// is in use. i.e. one or more trackers are waiting
// for a reannounce
bool m_waiting_tracker:1;
// this means we haven't verified the file content
// of the files we're seeding. the m_verified bitfield
// indicates which pieces have been verified and which
// haven't
bool m_seed_mode:1;
};
inline ptime torrent::next_announce() const

View File

@ -121,6 +121,7 @@ namespace libtorrent
, last_scrape(0)
, has_incoming(false)
, sparse_regions(0)
, seed_mode(false)
{}
enum state_t
@ -274,6 +275,9 @@ namespace libtorrent
// the number of "holes" in the torrent
int sparse_regions;
// is true if this torrent is (still) in seed_mode
bool seed_mode;
};
struct TORRENT_EXPORT block_info

View File

@ -36,6 +36,7 @@ POSSIBILITY OF SUCH DAMAGE.
#include "libtorrent/disk_buffer_holder.hpp"
#include "libtorrent/alloca.hpp"
#include "libtorrent/invariant_check.hpp"
#include "libtorrent/error_code.hpp"
#include <boost/scoped_array.hpp>
#ifdef TORRENT_DISK_STATS
@ -507,16 +508,23 @@ namespace libtorrent
m_pieces.push_back(p);
}
enum read_options_t
{
ignore_cache_size = 1
};
// fills a piece with data from disk, returns the total number of bytes
// read or -1 if there was an error
int disk_io_thread::read_into_piece(cached_piece_entry& p, int start_block, mutex_t::scoped_lock& l)
int disk_io_thread::read_into_piece(cached_piece_entry& p, int start_block
, int options, mutex_t::scoped_lock& l)
{
int piece_size = p.storage->info()->piece_size(p.piece);
int blocks_in_piece = (piece_size + m_block_size - 1) / m_block_size;
int end_block = start_block;
for (int i = start_block; i < blocks_in_piece
&& m_cache_stats.cache_size < m_settings.cache_size; ++i)
&& (m_cache_stats.cache_size < m_settings.cache_size
|| (options && ignore_cache_size)); ++i)
{
// this is a block that is already allocated
// stop allocating and don't read more than
@ -610,6 +618,35 @@ namespace libtorrent
return m_settings.cache_size - m_cache_stats.cache_size >= num_blocks;
}
// returns -1 on read error, -2 on out of memory error or the number of bytes read
// this function ignores the cache size limit, it will read the entire
// piece regardless of the offset in j
int disk_io_thread::cache_read_piece(disk_io_job const& j, mutex_t::scoped_lock& l)
{
INVARIANT_CHECK;
int piece_size = j.storage->info()->piece_size(j.piece);
int blocks_in_piece = (piece_size + m_block_size - 1) / m_block_size;
make_room(blocks_in_piece, m_read_pieces.end(), l);
cached_piece_entry p;
p.piece = j.piece;
p.storage = j.storage;
p.last_use = time_now();
p.num_blocks = 0;
p.blocks.reset(new char*[blocks_in_piece]);
std::memset(&p.blocks[0], 0, blocks_in_piece * sizeof(char*));
int ret = read_into_piece(p, 0, ignore_cache_size, l);
if (ret == -1)
free_piece(p, l);
else
m_read_pieces.push_back(p);
return ret;
}
// returns -1 on read error, -2 if there isn't any space in the cache
// or the number of bytes read
int disk_io_thread::cache_read_block(disk_io_job const& j, mutex_t::scoped_lock& l)
@ -631,7 +668,7 @@ namespace libtorrent
p.num_blocks = 0;
p.blocks.reset(new char*[blocks_in_piece]);
std::memset(&p.blocks[0], 0, blocks_in_piece * sizeof(char*));
int ret = read_into_piece(p, start_block, l);
int ret = read_into_piece(p, start_block, 0, l);
if (ret == -1)
free_piece(p, l);
@ -702,6 +739,111 @@ namespace libtorrent
}
#endif
int disk_io_thread::read_piece_from_cache_and_hash(disk_io_job const& j, sha1_hash& h)
{
TORRENT_ASSERT(j.buffer);
mutex_t::scoped_lock l(m_piece_mutex);
cache_t::iterator p
= find_cached_piece(m_read_pieces, j, l);
bool hit = true;
int ret = 0;
// if the piece cannot be found in the cache,
// read the whole piece starting at the block
// we got a request for.
if (p == m_read_pieces.end())
{
ret = cache_read_piece(j, l);
hit = false;
if (ret < 0) return ret;
p = m_read_pieces.end();
--p;
TORRENT_ASSERT(!m_read_pieces.empty());
TORRENT_ASSERT(p->piece == j.piece);
TORRENT_ASSERT(p->storage == j.storage);
}
hasher ctx;
int piece_size = j.storage->info()->piece_size(j.piece);
int blocks_in_piece = (piece_size + m_block_size - 1) / m_block_size;
for (int i = 0; i < blocks_in_piece; ++i)
{
TORRENT_ASSERT(p->blocks[i]);
ctx.update((char const*)p->blocks[i], std::min(piece_size, m_block_size));
piece_size -= m_block_size;
}
h = ctx.final();
ret = copy_from_piece(p, hit, j, l);
TORRENT_ASSERT(ret > 0);
if (ret < 0) return ret;
// if read cache is disabled or we exceeded the
// limit, remove this piece from the cache
if (m_cache_stats.cache_size >= m_settings.cache_size
|| !m_settings.use_read_cache)
{
TORRENT_ASSERT(!m_read_pieces.empty());
TORRENT_ASSERT(p->piece == j.piece);
TORRENT_ASSERT(p->storage == j.storage);
if (p != m_read_pieces.end())
{
free_piece(*p, l);
m_read_pieces.erase(p);
}
}
ret = j.buffer_size;
++m_cache_stats.blocks_read;
if (hit) ++m_cache_stats.blocks_read_hit;
return ret;
}
int disk_io_thread::copy_from_piece(cache_t::iterator p, bool& hit
, disk_io_job const& j, mutex_t::scoped_lock& l)
{
TORRENT_ASSERT(j.buffer);
// copy from the cache and update the last use timestamp
int block = j.offset / m_block_size;
int block_offset = j.offset & (m_block_size-1);
int buffer_offset = 0;
int size = j.buffer_size;
if (p->blocks[block] == 0)
{
int piece_size = j.storage->info()->piece_size(j.piece);
int blocks_in_piece = (piece_size + m_block_size - 1) / m_block_size;
int end_block = block;
while (end_block < blocks_in_piece && p->blocks[end_block] == 0) ++end_block;
if (!make_room(end_block - block, p, l)) return -2;
int ret = read_into_piece(*p, block, 0, l);
hit = false;
if (ret < 0) return ret;
TORRENT_ASSERT(p->blocks[block]);
}
p->last_use = time_now();
while (size > 0)
{
TORRENT_ASSERT(p->blocks[block]);
int to_copy = (std::min)(m_block_size
- block_offset, size);
std::memcpy(j.buffer + buffer_offset
, p->blocks[block] + block_offset
, to_copy);
size -= to_copy;
block_offset = 0;
buffer_offset += to_copy;
++block;
}
return j.buffer_size;
}
int disk_io_thread::try_read_from_cache(disk_io_job const& j)
{
TORRENT_ASSERT(j.buffer);
@ -730,44 +872,14 @@ namespace libtorrent
TORRENT_ASSERT(p->storage == j.storage);
}
if (p != m_read_pieces.end())
{
// copy from the cache and update the last use timestamp
int block = j.offset / m_block_size;
int block_offset = j.offset & (m_block_size-1);
int buffer_offset = 0;
int size = j.buffer_size;
if (p->blocks[block] == 0)
{
int piece_size = j.storage->info()->piece_size(j.piece);
int blocks_in_piece = (piece_size + m_block_size - 1) / m_block_size;
int end_block = block;
while (end_block < blocks_in_piece && p->blocks[end_block] == 0) ++end_block;
if (!make_room(end_block - block, p, l)) return -2;
ret = read_into_piece(*p, block, l);
hit = false;
if (ret < 0) return ret;
TORRENT_ASSERT(p->blocks[block]);
}
p->last_use = time_now();
while (size > 0)
{
TORRENT_ASSERT(p->blocks[block]);
int to_copy = (std::min)(m_block_size
- block_offset, size);
std::memcpy(j.buffer + buffer_offset
, p->blocks[block] + block_offset
, to_copy);
size -= to_copy;
block_offset = 0;
buffer_offset += to_copy;
++block;
}
ret = j.buffer_size;
++m_cache_stats.blocks_read;
if (hit) ++m_cache_stats.blocks_read_hit;
}
if (p == m_read_pieces.end()) return ret;
ret = copy_from_piece(p, hit, j, l);
if (ret < 0) return ret;
ret = j.buffer_size;
++m_cache_stats.blocks_read;
if (hit) ++m_cache_stats.blocks_read_hit;
return ret;
}
@ -977,6 +1089,53 @@ namespace libtorrent
}
break;
}
case disk_io_job::read_and_hash:
{
if (test_error(j))
{
ret = -1;
break;
}
#ifdef TORRENT_DISK_STATS
m_log << log_time() << " read_and_hash " << j.buffer_size << std::endl;
#endif
INVARIANT_CHECK;
TORRENT_ASSERT(j.buffer == 0);
j.buffer = allocate_buffer("send buffer");
TORRENT_ASSERT(j.buffer_size <= m_block_size);
if (j.buffer == 0)
{
ret = -1;
j.error = error_code(ENOMEM, get_posix_category());
j.str = j.error.message();
break;
}
disk_buffer_holder read_holder(*this, j.buffer);
// read the entire piece and verify the piece hash
// since we need to check the hash, this function
// will ignore the cache size limit (at least for
// reading and hashing, not for keeping it around)
sha1_hash h;
ret = read_piece_from_cache_and_hash(j, h);
if (ret == -1)
{
j.buffer = 0;
test_error(j);
break;
}
ret = (j.storage->info()->hash_for_piece(j.piece) == h)?ret:-3;
if (ret == -3)
{
j.storage->mark_failed(j.piece);
j.error = error_code(errors::failed_hash_check, libtorrent_category);
j.str = j.error.message();
}
read_holder.release();
break;
}
case disk_io_job::read:
{
if (test_error(j))

View File

@ -46,7 +46,11 @@ namespace libtorrent
std::string libtorrent_error_category::message(int ev) const
{
static char const* msgs[] =
{ "no error", "torrent file collides with file from another torrent" };
{
"no error",
"torrent file collides with file from another torrent",
"hash check failed"
};
if (ev < 0 || ev >= sizeof(msgs)/sizeof(msgs[0]))
return "Unknown error";
return msgs[ev];

View File

@ -3233,8 +3233,20 @@ namespace libtorrent
TORRENT_ASSERT(r.start + r.length <= t->torrent_file().piece_size(r.piece));
TORRENT_ASSERT(r.length > 0 && r.start >= 0);
t->filesystem().async_read(r, bind(&peer_connection::on_disk_read_complete
, self(), _1, _2, r));
if (!t->seed_mode() || t->verified_piece(r.piece))
{
t->filesystem().async_read(r, bind(&peer_connection::on_disk_read_complete
, self(), _1, _2, r));
}
else
{
// this means we're in seed mode and we haven't yet
// verified this piece (r.piece)
t->filesystem().async_read_and_hash(r, bind(&peer_connection::on_disk_read_complete
, self(), _1, _2, r));
t->verified(r.piece);
}
m_reading_bytes += r.length;
m_requests.erase(m_requests.begin());
@ -3249,22 +3261,36 @@ namespace libtorrent
disk_buffer_holder buffer(m_ses, j.buffer);
boost::shared_ptr<torrent> t = m_torrent.lock();
if (ret != r.length || m_torrent.expired())
{
boost::shared_ptr<torrent> t = m_torrent.lock();
if (!t)
{
disconnect(j.str.c_str());
return;
}
if (t->alerts().should_post<file_error_alert>())
t->alerts().post_alert(file_error_alert(j.error_file, t->get_handle(), j.str));
t->set_error(j.str);
t->pause();
if (ret == -3)
{
if (t->seed_mode()) t->leave_seed_mode(false);
write_reject_request(r);
}
else
{
if (t->alerts().should_post<file_error_alert>())
t->alerts().post_alert(file_error_alert(j.error_file, t->get_handle(), j.str));
t->set_error(j.str);
t->pause();
}
return;
}
if (t)
{
if (t->seed_mode() && t->all_verified())
t->leave_seed_mode(true);
}
#ifdef TORRENT_VERBOSE_LOGGING
(*m_logger) << time_now_string()
<< " ==> PIECE [ piece: " << r.piece << " | s: " << r.start
@ -3340,7 +3366,6 @@ namespace libtorrent
(*m_logger) << time_now_string() << " *** REQUEST_BANDWIDTH [ upload prio: "
<< priority << "]\n";
#endif
}
return;
}

View File

@ -1882,23 +1882,8 @@ namespace aux {
if (pos >= queue_pos) queue_pos = pos + 1;
}
// create the torrent and the data associated with
// the checker thread and store it before starting
// the thread
if (params.ti)
{
torrent_ptr.reset(new torrent(*this, params.ti, params.save_path
, m_listen_interface, params.storage_mode, 16 * 1024
, params.storage, params.paused, params.resume_data
, queue_pos, params.auto_managed));
}
else
{
torrent_ptr.reset(new torrent(*this, params.tracker_url, *ih, params.name
, params.save_path, m_listen_interface, params.storage_mode, 16 * 1024
, params.storage, params.paused, params.resume_data
, queue_pos, params.auto_managed));
}
torrent_ptr.reset(new torrent(*this, m_listen_interface
, 16 * 1024, queue_pos, params));
torrent_ptr->start();
#ifndef TORRENT_DISABLE_EXTENSIONS

View File

@ -1405,6 +1405,29 @@ ret:
m_io_thread.add_job(j, handler);
}
void piece_manager::async_read_and_hash(
peer_request const& r
, boost::function<void(int, disk_io_job const&)> const& handler
, int priority)
{
disk_io_job j;
j.storage = this;
j.action = disk_io_job::read_and_hash;
j.piece = r.piece;
j.offset = r.start;
j.buffer_size = r.length;
j.buffer = 0;
j.priority = priority;
TORRENT_ASSERT(r.length <= 16 * 1024);
m_io_thread.add_job(j, handler);
#ifdef TORRENT_DEBUG
boost::recursive_mutex::scoped_lock l(m_mutex);
// if this assert is hit, it suggests
// that check_files was not successful
TORRENT_ASSERT(slot_for(r.piece) >= 0);
#endif
}
void piece_manager::async_read(
peer_request const& r
, boost::function<void(int, disk_io_job const&)> const& handler

View File

@ -130,16 +130,10 @@ namespace libtorrent
torrent::torrent(
session_impl& ses
, boost::intrusive_ptr<torrent_info> tf
, fs::path const& save_path
, tcp::endpoint const& net_interface
, storage_mode_t storage_mode
, int block_size
, storage_constructor_type sc
, bool paused
, std::vector<char>* resume_data
, int seq
, bool auto_managed)
, add_torrent_params const& p)
: m_policy(this)
, m_active_time(seconds(0))
, m_seeding_time(seconds(0))
@ -147,7 +141,7 @@ namespace libtorrent
, m_total_downloaded(0)
, m_started(time_now())
, m_last_scrape(min_time())
, m_torrent_file(tf)
, m_torrent_file(p.ti ? p.ti : new torrent_info(p.info_hash))
, m_storage(0)
, m_host_resolver(ses.m_io_service)
, m_lsd_announce_timer(ses.m_io_service)
@ -161,17 +155,18 @@ namespace libtorrent
, m_total_redundant_bytes(0)
, m_padding(0)
, m_net_interface(net_interface.address(), 0)
, m_save_path(complete(save_path))
, m_storage_mode(storage_mode)
, m_save_path(complete(p.save_path))
, m_num_verified(0)
, m_storage_mode(p.storage_mode)
, m_state(torrent_status::checking_resume_data)
, m_settings(ses.settings())
, m_storage_constructor(sc)
, m_storage_constructor(p.storage)
, m_progress(0.f)
, m_ratio(0.f)
, m_max_uploads((std::numeric_limits<int>::max)())
, m_num_uploads(0)
, m_max_connections((std::numeric_limits<int>::max)())
, m_block_size((std::min)(block_size, tf->piece_length()))
, m_block_size(p.ti ? (std::min)(block_size, m_torrent_file->piece_length()) : block_size)
, m_complete(-1)
, m_incomplete(-1)
, m_deficit_counter(0)
@ -179,22 +174,26 @@ namespace libtorrent
, m_last_working_tracker(-1)
, m_time_scaler(0)
, m_abort(false)
, m_paused(paused)
, m_auto_managed(auto_managed)
, m_paused(p.paused)
, m_auto_managed(p.auto_managed)
#ifndef TORRENT_DISABLE_RESOLVE_COUNTRIES
, m_resolving_country(false)
, m_resolve_countries(false)
#endif
, m_sequential_download(false)
, m_got_tracker_response(false)
, m_connections_initialized(true)
, m_connections_initialized(p.ti)
, m_super_seeding(false)
, m_has_incoming(false)
, m_files_checked(false)
, m_announcing(false)
, m_waiting_tracker(false)
, m_seed_mode(p.seed_mode && m_torrent_file->is_valid())
{
if (resume_data) m_resume_data.swap(*resume_data);
if (m_seed_mode)
m_verified.resize(m_torrent_file->num_pieces(), false);
if (p.resume_data) m_resume_data.swap(*p.resume_data);
if (m_settings.prefer_udp_trackers)
prioritize_udp_trackers();
@ -202,84 +201,7 @@ namespace libtorrent
#ifndef TORRENT_DISABLE_ENCRYPTION
hasher h;
h.update("req2", 4);
h.update((char*)&tf->info_hash()[0], 20);
m_obfuscated_hash = h.final();
#endif
}
torrent::torrent(
session_impl& ses
, char const* tracker_url
, sha1_hash const& info_hash
, char const* name
, fs::path const& save_path
, tcp::endpoint const& net_interface
, storage_mode_t storage_mode
, int block_size
, storage_constructor_type sc
, bool paused
, std::vector<char>* resume_data
, int seq
, bool auto_managed)
: m_policy(this)
, m_active_time(seconds(0))
, m_seeding_time(seconds(0))
, m_total_uploaded(0)
, m_total_downloaded(0)
, m_started(time_now())
, m_last_scrape(min_time())
, m_torrent_file(new torrent_info(info_hash))
, m_storage(0)
, m_host_resolver(ses.m_io_service)
, m_lsd_announce_timer(ses.m_io_service)
, m_tracker_timer(ses.m_io_service)
#ifndef TORRENT_DISABLE_DHT
, m_last_dht_announce(time_now() - minutes(15))
#endif
, m_ses(ses)
, m_total_failed_bytes(0)
, m_total_redundant_bytes(0)
, m_padding(0)
, m_net_interface(net_interface.address(), 0)
, m_save_path(complete(save_path))
, m_storage_mode(storage_mode)
, m_state(torrent_status::checking_resume_data)
, m_settings(ses.settings())
, m_storage_constructor(sc)
, m_progress(0.f)
, m_ratio(0.f)
, m_max_uploads((std::numeric_limits<int>::max)())
, m_num_uploads(0)
, m_max_connections((std::numeric_limits<int>::max)())
, m_block_size(block_size)
, m_complete(-1)
, m_incomplete(-1)
, m_deficit_counter(0)
, m_sequence_number(seq)
, m_last_working_tracker(-1)
, m_time_scaler(0)
, m_abort(false)
, m_paused(paused)
, m_auto_managed(auto_managed)
#ifndef TORRENT_DISABLE_RESOLVE_COUNTRIES
, m_resolving_country(false)
, m_resolve_countries(false)
#endif
, m_sequential_download(false)
, m_got_tracker_response(false)
, m_connections_initialized(false)
, m_super_seeding(false)
, m_has_incoming(false)
, m_files_checked(false)
, m_announcing(false)
, m_waiting_tracker(false)
{
if (resume_data) m_resume_data.swap(*resume_data);
#ifndef TORRENT_DISABLE_ENCRYPTION
hasher h;
h.update("req2", 4);
h.update((char*)&info_hash[0], 20);
h.update((char*)&m_torrent_file->info_hash()[0], 20);
m_obfuscated_hash = h.final();
#endif
@ -288,35 +210,40 @@ namespace libtorrent
#endif
INVARIANT_CHECK;
if (name) m_name.reset(new std::string(name));
if (p.name && !p.ti) m_name.reset(new std::string(p.name));
if (tracker_url)
if (p.tracker_url)
{
m_trackers.push_back(announce_entry(tracker_url));
m_trackers.push_back(announce_entry(p.tracker_url));
m_trackers.back().fail_limit = 0;
m_trackers.back().source = announce_entry::source_magnet_link;
m_torrent_file->add_tracker(tracker_url);
m_torrent_file->add_tracker(p.tracker_url);
}
}
void torrent::start()
{
TORRENT_ASSERT(!m_picker);
m_picker.reset(new piece_picker());
if (!m_resume_data.empty())
if (!m_seed_mode)
{
if (lazy_bdecode(&m_resume_data[0], &m_resume_data[0]
+ m_resume_data.size(), m_resume_entry) != 0)
m_picker.reset(new piece_picker());
if (!m_resume_data.empty())
{
std::vector<char>().swap(m_resume_data);
if (m_ses.m_alerts.should_post<fastresume_rejected_alert>())
if (lazy_bdecode(&m_resume_data[0], &m_resume_data[0]
+ m_resume_data.size(), m_resume_entry) != 0)
{
m_ses.m_alerts.post_alert(fastresume_rejected_alert(get_handle(), "parse failed"));
std::vector<char>().swap(m_resume_data);
if (m_ses.m_alerts.should_post<fastresume_rejected_alert>())
{
m_ses.m_alerts.post_alert(fastresume_rejected_alert(get_handle(), "parse failed"));
#if defined TORRENT_VERBOSE_LOGGING || defined TORRENT_LOGGING || defined TORRENT_ERROR_LOGGING
(*m_ses.m_logger) << "fastresume data for "
<< torrent_file().name() << " rejected: parse failed\n";
(*m_ses.m_logger) << "fastresume data for "
<< torrent_file().name() << " rejected: parse failed\n";
#endif
}
}
}
}
@ -572,7 +499,6 @@ namespace libtorrent
TORRENT_ASSERT(m_torrent_file->num_files() > 0);
TORRENT_ASSERT(m_torrent_file->total_size() >= 0);
m_file_priority.clear();
m_file_priority.resize(m_torrent_file->num_files(), 1);
m_block_size = (std::min)(m_block_size, m_torrent_file->piece_length());
@ -590,8 +516,12 @@ namespace libtorrent
, m_save_path, m_ses.m_files, m_ses.m_disk_thread, m_storage_constructor
, m_storage_mode);
m_storage = m_owning_storage.get();
m_picker->init((std::max)(m_torrent_file->piece_length() / m_block_size, 1)
, int((m_torrent_file->total_size()+m_block_size-1)/m_block_size));
if (has_picker())
{
m_picker->init((std::max)(m_torrent_file->piece_length() / m_block_size, 1)
, int((m_torrent_file->total_size()+m_block_size-1)/m_block_size));
}
std::vector<std::string> const& url_seeds = m_torrent_file->url_seeds();
for (std::vector<std::string>::const_iterator i = url_seeds.begin()
@ -603,6 +533,14 @@ namespace libtorrent
, end(http_seeds.end()); i != end; ++i)
add_web_seed(*i, web_seed_entry::http_seed);
if (m_seed_mode)
{
m_ses.m_io_service.post(boost::bind(&torrent::files_checked, shared_from_this()));
std::vector<char>().swap(m_resume_data);
lazy_entry().swap(m_resume_entry);
return;
}
set_state(torrent_status::checking_resume_data);
if (m_resume_entry.type() == lazy_entry::dict_t)
@ -1602,6 +1540,9 @@ namespace libtorrent
// the following call may cause picker to become invalid
// in case we just became a seed
piece_passed(index);
// if we're in seed mode, we just acquired this piece
// mark it as verified
if (m_seed_mode) verified(index);
}
else if (passed_hash_check == -2)
{
@ -3662,7 +3603,15 @@ namespace libtorrent
// buils a list of all connected peers and sort it by 'disconnectability'.
std::vector<peer_connection*> peers(m_connections.size());
std::copy(m_connections.begin(), m_connections.end(), peers.begin());
std::sort(peers.begin(), peers.end(), boost::bind(&compare_disconnect_peer, _1, _2));
#ifdef TORRENT_DEBUG
for (std::vector<peer_connection*>::iterator i = peers.begin()
, end(peers.end()); i != end; ++i)
{
// make sure this peer is not a dangling pointer
TORRENT_ASSERT(m_ses.has_peer(*i));
}
#endif
std::sort(peers.begin(), peers.end(), compare_disconnect_peer);
// never disconnect peers that connected less than 90 seconds ago
ptime cut_off = time_now() - seconds(90);
@ -3897,7 +3846,8 @@ namespace libtorrent
if (m_abort) return;
// we might be finished already, in which case we should
// not switch to downloading mode.
// not switch to downloading mode. If all files are
// filtered, we're finished when we start.
if (m_state != torrent_status::finished)
set_state(torrent_status::downloading);
@ -3917,14 +3867,17 @@ namespace libtorrent
if (m_ses.m_auto_manage_time_scaler > 1)
m_ses.m_auto_manage_time_scaler = 1;
if (is_finished() && m_state != torrent_status::finished) finished();
if (is_finished() && m_state != torrent_status::finished)
finished();
}
else
{
for (std::vector<announce_entry>::iterator i = m_trackers.begin()
, end(m_trackers.end()); i != end; ++i)
i->complete_sent = true;
if (m_state != torrent_status::finished) finished();
if (m_state != torrent_status::finished)
finished();
}
#ifndef TORRENT_DISABLE_EXTENSIONS
@ -4165,9 +4118,11 @@ namespace libtorrent
}
}
// This check is very expensive.
TORRENT_ASSERT(!valid_metadata() || m_block_size > 0);
TORRENT_ASSERT(!valid_metadata() || (m_torrent_file->piece_length() % m_block_size) == 0);
if (valid_metadata())
{
TORRENT_ASSERT(m_block_size > 0);
TORRENT_ASSERT((m_torrent_file->piece_length() & (m_block_size-1)) == 0);
}
// if (is_seed()) TORRENT_ASSERT(m_picker.get() == 0);
}
#endif
@ -5057,6 +5012,7 @@ namespace libtorrent
st.has_incoming = m_has_incoming;
st.error = m_error;
st.seed_mode = m_seed_mode;
if (m_last_scrape == min_time())
{

View File

@ -245,7 +245,8 @@ boost::tuple<torrent_handle, torrent_handle, torrent_handle>
setup_transfer(session* ses1, session* ses2, session* ses3
, bool clear_files, bool use_metadata_transfer, bool connect_peers
, std::string suffix, int piece_size
, boost::intrusive_ptr<torrent_info>* torrent, bool super_seeding)
, boost::intrusive_ptr<torrent_info>* torrent, bool super_seeding
, add_torrent_params const* p)
{
using namespace boost::filesystem;
@ -289,22 +290,39 @@ setup_transfer(session* ses1, session* ses2, session* ses3
// file pool will complain if two torrents are trying to
// use the same files
sha1_hash info_hash = t->info_hash();
torrent_handle tor1 = ses1->add_torrent(clone_ptr(t), "./tmp1" + suffix);
add_torrent_params param;
if (p) param = *p;
param.ti = clone_ptr(t);
param.save_path = "./tmp1" + suffix;
torrent_handle tor1 = ses1->add_torrent(param);
tor1.super_seeding(super_seeding);
TEST_CHECK(!ses1->get_torrents().empty());
torrent_handle tor2;
torrent_handle tor3;
// the downloader cannot use seed_mode
param.seed_mode = false;
if (ses3)
{
tor3 = ses3->add_torrent(clone_ptr(t), "./tmp3" + suffix);
param.ti = clone_ptr(t);
param.save_path = "./tmp3" + suffix;
tor3 = ses3->add_torrent(param);
TEST_CHECK(!ses3->get_torrents().empty());
}
if (use_metadata_transfer)
tor2 = ses2->add_torrent("http://non-existent-name.com/announce"
, t->info_hash(), 0, "./tmp2" + suffix);
{
param.ti = 0;
param.info_hash = t->info_hash();
}
else
tor2 = ses2->add_torrent(clone_ptr(t), "./tmp2" + suffix);
{
param.ti = clone_ptr(t);
}
param.save_path = "./tmp2" + suffix;
tor2 = ses2->add_torrent(param);
TEST_CHECK(!ses2->get_torrents().empty());
assert(ses1->get_torrents().size() == 1);

View File

@ -39,6 +39,7 @@ POSSIBILITY OF SUCH DAMAGE.
namespace libtorrent
{
class alert;
class add_torrent_params;
}
bool print_alerts(libtorrent::session& ses, char const* name
@ -51,12 +52,14 @@ void test_sleep(int millisec);
boost::intrusive_ptr<libtorrent::torrent_info> create_torrent(std::ostream* file = 0, int piece_size = 16 * 1024, int num_pieces = 1024 / 8);
boost::tuple<libtorrent::torrent_handle, libtorrent::torrent_handle
boost::tuple<libtorrent::torrent_handle
, libtorrent::torrent_handle
, libtorrent::torrent_handle>
setup_transfer(libtorrent::session* ses1, libtorrent::session* ses2
, libtorrent::session* ses3, bool clear_files, bool use_metadata_transfer = true
, bool connect = true, std::string suffix = "", int piece_size = 16 * 1024
, boost::intrusive_ptr<libtorrent::torrent_info>* torrent = 0, bool super_seeding = false);
, boost::intrusive_ptr<libtorrent::torrent_info>* torrent = 0, bool super_seeding = false
, libtorrent::add_torrent_params const* p = 0);
void start_web_server(int port, bool ssl = false);
void stop_web_server(int port);

View File

@ -44,7 +44,7 @@ POSSIBILITY OF SUCH DAMAGE.
using boost::filesystem::remove_all;
using boost::filesystem::exists;
void test_swarm(bool super_seeding = false, bool strict = false)
void test_swarm(bool super_seeding = false, bool strict = false, bool seed_mode = false)
{
using namespace libtorrent;
@ -92,9 +92,11 @@ void test_swarm(bool super_seeding = false, bool strict = false)
torrent_handle tor2;
torrent_handle tor3;
add_torrent_params p;
p.seed_mode = seed_mode;
// test using piece sizes smaller than 16kB
boost::tie(tor1, tor2, tor3) = setup_transfer(&ses1, &ses2, &ses3, true
, false, true, "_swarm", 8 * 1024, 0, super_seeding);
, false, true, "_swarm", 8 * 1024, 0, super_seeding, &p);
float sum_dl_rate2 = 0.f;
float sum_dl_rate3 = 0.f;
@ -201,6 +203,9 @@ int test_main()
using namespace libtorrent;
using namespace boost::filesystem;
// with seed mode
test_swarm(false, false, true);
test_swarm();
// with super seeding