make the torrent hold a pointer to the ip_filter instead of relying on a session reference

This commit is contained in:
Arvid Norberg 2015-05-16 06:33:37 +00:00
parent e7f9f35556
commit 2bf4519bee
11 changed files with 153 additions and 121 deletions

View File

@ -361,11 +361,12 @@ namespace libtorrent
void pause();
void resume();
void set_ip_filter(ip_filter const& f);
ip_filter& get_ip_filter();
void set_ip_filter(boost::shared_ptr<ip_filter> const& f);
ip_filter const& get_ip_filter();
void set_port_filter(port_filter const& f);
port_filter const& get_port_filter() const;
port_filter const& get_port_filter() const TORRENT_OVERRIDE;
void ban_ip(address addr) TORRENT_OVERRIDE;
void queue_tracker_request(tracker_request& req
, boost::weak_ptr<request_callback> c);
@ -808,7 +809,7 @@ namespace libtorrent
peer_class_type_filter m_peer_class_type_filter;
// filters incoming connections
ip_filter m_ip_filter;
boost::shared_ptr<ip_filter> m_ip_filter;
// filters outgoing connections
port_filter m_port_filter;

View File

@ -174,9 +174,9 @@ namespace libtorrent { namespace aux
virtual void remove_torrent(torrent_handle const& h, int options = 0) = 0;
virtual void remove_torrent_impl(boost::shared_ptr<torrent> tptr, int options) = 0;
// ip and port filter
virtual ip_filter& get_ip_filter() = 0;
// port filter
virtual port_filter const& get_port_filter() const = 0;
virtual void ban_ip(address addr) = 0;
virtual boost::int64_t session_time() const = 0;

View File

@ -273,8 +273,8 @@ namespace libtorrent
// for a specific download. It updates itself against
// the tracker
class TORRENT_EXTRA_EXPORT torrent
: public single_threaded
, public torrent_hot_members
: private single_threaded
, private torrent_hot_members
, public request_callback
, public peer_class_set
, public boost::enable_shared_from_this<torrent>
@ -450,8 +450,9 @@ namespace libtorrent
void sent_syn(bool ipv6);
void received_synack(bool ipv6);
void ip_filter_updated();
void set_ip_filter(boost::shared_ptr<const ip_filter> ipf);
void port_filter_updated();
ip_filter const* get_ip_filter() { return m_ip_filter.get(); }
std::string resolve_filename(int file) const;
void handle_disk_error(disk_io_job const* j, peer_connection* c = 0);
@ -1148,6 +1149,8 @@ namespace libtorrent
private:
void ip_filter_updated();
void inc_stats_counter(int c, int value = 1);
// initialize the torrent_state structure passed to peer_list
@ -1199,6 +1202,8 @@ namespace libtorrent
void need_policy();
boost::shared_ptr<const ip_filter> m_ip_filter;
// all time totals of uploaded and downloaded payload
// stored in resume data
boost::int64_t m_total_uploaded;

View File

@ -75,6 +75,7 @@ namespace libtorrent
class udp_socket;
struct resolver_interface;
struct counters;
struct ip_filter;
#if TORRENT_USE_I2P
class i2p_connection;
#endif
@ -97,7 +98,6 @@ namespace libtorrent
, key(0)
, num_want(0)
, send_stats(true)
, apply_ip_filter(true)
#ifdef TORRENT_USE_OPENSSL
, ssl_ctx(0)
#endif
@ -125,6 +125,8 @@ namespace libtorrent
std::string trackerid;
std::string auth;
boost::shared_ptr<const ip_filter> filter;
boost::int64_t downloaded;
boost::int64_t uploaded;
boost::int64_t left;
@ -145,7 +147,6 @@ namespace libtorrent
address bind_ip;
bool send_stats;
bool apply_ip_filter;
#ifdef TORRENT_USE_OPENSSL
boost::asio::ssl::context* ssl_ctx;
#endif
@ -332,7 +333,6 @@ namespace libtorrent
tracker_manager(udp_socket& sock
, counters& stats_counters
, resolver_interface& resolver
, struct ip_filter& ipf
, aux::session_settings const& sett
#if !defined TORRENT_DISABLE_LOGGING || TORRENT_USE_ASSERTS
, aux::session_logger& ses
@ -368,7 +368,6 @@ namespace libtorrent
aux::session_settings const& settings() const { return m_settings; }
udp_socket& get_udp_socket() { return m_udp_socket; }
struct ip_filter const& ip_filter() const { return m_ip_filter; }
resolver_interface& host_resolver() { return m_host_resolver; }
private:
@ -385,7 +384,6 @@ namespace libtorrent
typedef std::vector<boost::shared_ptr<http_tracker_connection> > http_conns_t;
http_conns_t m_http_conns;
struct ip_filter const& m_ip_filter;
class udp_socket& m_udp_socket;
resolver_interface& m_host_resolver;
aux::session_settings const& m_settings;

View File

@ -253,13 +253,13 @@ namespace libtorrent
void http_tracker_connection::on_filter(http_connection& c, std::vector<tcp::endpoint>& endpoints)
{
if (tracker_req().apply_ip_filter == false) return;
if (!tracker_req().filter) return;
// remove endpoints that are filtered by the IP filter
for (std::vector<tcp::endpoint>::iterator i = endpoints.begin();
i != endpoints.end();)
{
if (m_man.ip_filter().access(i->address()) == ip_filter::blocked)
if (tracker_req().filter->access(i->address()) == ip_filter::blocked)
i = endpoints.erase(i);
else
++i;

View File

@ -1204,7 +1204,7 @@ namespace libtorrent
// as part of DHT traffic. The fact that we got an incoming
// connection on this info-hash, means the other end, making this
// connection fished it out of the DHT chatter. That's suspicious.
m_ses.get_ip_filter().add_rule(m_remote.address(), m_remote.address(), 0);
m_ses.ban_ip(m_remote.address());
}
#endif
disconnect(errors::invalid_info_hash, op_bittorrent, 1);

View File

@ -534,7 +534,8 @@ namespace libtorrent
void session::set_ip_filter(ip_filter const& f)
{
TORRENT_ASYNC_CALL1(set_ip_filter, f);
boost::shared_ptr<ip_filter> copy = boost::make_shared<ip_filter>(f);
TORRENT_ASYNC_CALL1(set_ip_filter, copy);
}
ip_filter session::get_ip_filter() const

View File

@ -361,7 +361,7 @@ namespace aux {
, m_tcp_peer_class(0)
, m_local_peer_class(0)
, m_tracker_manager(m_udp_socket, m_stats_counters, m_host_resolver
, m_ip_filter, m_settings
, m_settings
#if !defined TORRENT_DISABLE_LOGGING || TORRENT_USE_ASSERTS
, *this
#endif
@ -509,7 +509,7 @@ namespace aux {
session_log("config: %s\n"
"version: %s\n"
"revision: %s\n\n"
, TORRENT_CFG_STRING
, TORRENT_CFG_STRING
, LIBTORRENT_VERSION
, LIBTORRENT_REVISION);
@ -1085,7 +1085,7 @@ namespace aux {
TORRENT_ASSERT(!c->m_in_constructor);
m_connections.insert(c);
}
void session_impl::set_port_filter(port_filter const& f)
{
m_port_filter = f;
@ -1095,10 +1095,10 @@ namespace aux {
// by the new ip-filter
for (torrent_map::iterator i = m_torrents.begin()
, end(m_torrents.end()); i != end; ++i)
i->second->ip_filter_updated();
i->second->port_filter_updated();
}
void session_impl::set_ip_filter(ip_filter const& f)
void session_impl::set_ip_filter(boost::shared_ptr<ip_filter> const& f)
{
INVARIANT_CHECK;
@ -1108,21 +1108,33 @@ namespace aux {
// by the new ip-filter
for (torrent_map::iterator i = m_torrents.begin()
, end(m_torrents.end()); i != end; ++i)
i->second->port_filter_updated();
i->second->set_ip_filter(m_ip_filter);
}
ip_filter& session_impl::get_ip_filter()
void session_impl::ban_ip(address addr)
{
return m_ip_filter;
TORRENT_ASSERT(is_single_thread());
if (!m_ip_filter) m_ip_filter = boost::make_shared<ip_filter>();
m_ip_filter->add_rule(addr, addr, ip_filter::blocked);
}
ip_filter const& session_impl::get_ip_filter()
{
TORRENT_ASSERT(is_single_thread());
return *m_ip_filter;
}
port_filter const& session_impl::get_port_filter() const
{
TORRENT_ASSERT(is_single_thread());
return m_port_filter;
}
namespace
{
template <class Socket>
void static set_socket_buffer_size(Socket& s, session_settings const& sett, error_code& ec)
void set_socket_buffer_size(Socket& s, session_settings const& sett, error_code& ec)
{
int snd_size = sett.get_int(settings_pack::send_socket_buffer_size);
if (snd_size)
@ -1160,13 +1172,17 @@ namespace aux {
}
}
} // anonymous namespace
int session_impl::create_peer_class(char const* name)
{
TORRENT_ASSERT(is_single_thread());
return m_classes.new_peer_class(name);
}
void session_impl::delete_peer_class(int cid)
{
TORRENT_ASSERT(is_single_thread());
// if you hit this assert, you're deleting a non-existent peer class
TORRENT_ASSERT(m_classes.at(cid));
if (m_classes.at(cid) == 0) return;
@ -1314,7 +1330,7 @@ namespace aux {
while (i != NULL && i != t) i = (torrent*)i->next;
TORRENT_ASSERT(i == t);
#endif
// this torrent is in the list already.
// first remove it
m_torrent_lru.erase(t);
@ -1354,7 +1370,7 @@ namespace aux {
while (i != NULL && i != t) i = (torrent*)i->next;
TORRENT_ASSERT(i == t);
#endif
int loaded_limit = m_settings.get_int(settings_pack::active_loaded_limit);
// 0 means unlimited, never evict enything
@ -1369,7 +1385,7 @@ namespace aux {
m_torrent_lru.erase(t);
return;
}
// move this torrent to be the first to be evicted whenever
// another torrent need its slot
bump_torrent(t, false);
@ -1696,7 +1712,7 @@ namespace aux {
#endif
return ret;
}
void session_impl::open_listen_port()
{
#ifndef TORRENT_DISABLE_LOGGING
@ -1706,7 +1722,8 @@ namespace aux {
TORRENT_ASSERT(is_single_thread());
TORRENT_ASSERT(!m_abort);
int flags = m_settings.get_bool(settings_pack::listen_system_port_fallback) ? 0 : listen_no_system_port;
int flags = m_settings.get_bool(settings_pack::listen_system_port_fallback)
? 0 : listen_no_system_port;
error_code ec;
// reset the retry counter
@ -1735,7 +1752,7 @@ retry:
{
// this means we should open two listen sockets
// one for IPv4 and one for IPv6
listen_socket_t s = setup_listener("0.0.0.0", true
, m_listen_interface.port()
, m_listen_port_retries, flags, ec);
@ -2468,7 +2485,8 @@ retry:
// is set to ignore the filter, since this peer might be
// for that torrent
if (m_stats_counters[counters::non_filter_torrents] == 0
&& (m_ip_filter.access(endp.address()) & ip_filter::blocked))
&& m_ip_filter
&& (m_ip_filter->access(endp.address()) & ip_filter::blocked))
{
#ifndef TORRENT_DISABLE_LOGGING
session_log("filtered blocked ip");
@ -3373,7 +3391,7 @@ retry:
boost::shared_ptr<torrent> t;
do
{
t = m_dht_torrents.front().lock();
t = m_dht_torrents.front().lock();
m_dht_torrents.pop_front();
} while (!t && !m_dht_torrents.empty());
@ -3393,7 +3411,7 @@ retry:
++m_next_dht_torrent;
if (m_next_dht_torrent == m_torrents.end())
m_next_dht_torrent = m_torrents.begin();
}
}
#endif
void session_impl::on_lsd_announce(error_code const& e)
@ -4264,8 +4282,8 @@ retry:
} // anonymous namespace
boost::weak_ptr<torrent> session_impl::find_disconnect_candidate_torrent() const
{
boost::weak_ptr<torrent> session_impl::find_disconnect_candidate_torrent() const
{
aux::session_impl::torrent_map::const_iterator i = std::min_element(m_torrents.begin(), m_torrents.end()
, boost::bind(&compare_disconnect_torrent, _1, _2));
@ -4513,7 +4531,7 @@ retry:
return torrent_handle();
}
#ifndef TORRENT_DISABLE_DHT
#ifndef TORRENT_DISABLE_DHT
// add p.dht_nodes to the DHT, if enabled
if (m_dht && !p.dht_nodes.empty())
{
@ -4649,8 +4667,8 @@ retry:
int queue_pos = ++m_max_queue_pos;
torrent_ptr.reset(new torrent(*this
, 16 * 1024, queue_pos, params, *ih));
torrent_ptr = boost::make_shared<torrent>(boost::ref(*this)
, 16 * 1024, queue_pos, boost::cref(params), boost::cref(*ih));
torrent_ptr->start();
#ifndef TORRENT_DISABLE_EXTENSIONS
@ -5016,7 +5034,7 @@ retry:
// by the new ip-filter
for (torrent_map::iterator i = m_torrents.begin()
, end(m_torrents.end()); i != end; ++i)
i->second->ip_filter_updated();
i->second->port_filter_updated();
}
else
{

View File

@ -410,7 +410,7 @@ namespace libtorrent
{ m_ses.stats_counters().inc_stats_counter(c, value); }
#if 0
// NON BOTTLED VERSION. SUPPORTS PROGRESS REPORTING
// since this download is not bottled, this callback will
@ -480,7 +480,7 @@ namespace libtorrent
return;
}
std::vector<char>().swap(m_torrent_file_buf);
// update our torrent_info object and move the
// torrent from the old info-hash to the new one
// as we replace the torrent_info object
@ -630,7 +630,7 @@ namespace libtorrent
pause();
return;
}
// update our torrent_info object and move the
// torrent from the old info-hash to the new one
// as we replace the torrent_info object
@ -724,7 +724,7 @@ namespace libtorrent
// an error.
// TODO: 2 post alert
#ifndef TORRENT_DISABLE_LOGGING
debug_log("*** FAILED SEED MODE, rechecking");
#endif
@ -811,7 +811,7 @@ namespace libtorrent
, _1, _2, _3, _4)
, true // bottled
//bottled buffer size
, m_ses.settings().get_int(settings_pack::max_http_recv_buffer_size)
, settings().get_int(settings_pack::max_http_recv_buffer_size)
, http_connect_handler()
, http_filter_handler()
#ifdef TORRENT_USE_OPENSSL
@ -820,7 +820,7 @@ namespace libtorrent
));
proxy_settings ps = m_ses.proxy();
conn->get(m_url, seconds(30), 0, &ps
, 5, m_ses.settings().get_str(settings_pack::user_agent));
, 5, settings().get_str(settings_pack::user_agent));
set_state(torrent_status::downloading_metadata);
}
@ -840,6 +840,13 @@ namespace libtorrent
state_updated();
}
void torrent::set_ip_filter(boost::shared_ptr<const ip_filter> ipf)
{
m_ip_filter = ipf;
if (!m_apply_ip_filter) return;
ip_filter_updated();
}
#ifndef TORRENT_DISABLE_DHT
bool torrent::should_announce_dht() const
{
@ -865,7 +872,7 @@ namespace libtorrent
for (std::vector<announce_entry>::const_iterator i = m_trackers.begin()
, end(m_trackers.end()); i != end; ++i)
if (i->verified) ++verified_trackers;
return verified_trackers == 0;
}
@ -896,7 +903,7 @@ namespace libtorrent
// is being destructed, all weak references to it have been
// reset, which means that all its peers already have an
// invalidated torrent pointer (so it cannot be verified to be correct)
// i.e. the invariant can only be maintained if all connections have
// been closed by the time the torrent is destructed. And they are
// supposed to be closed. So we can still do the invariant check.
@ -1307,7 +1314,7 @@ namespace libtorrent
}
disk_buffer_holder holder(m_ses, buffer);
std::memcpy(buffer, data + p.start, p.length);
if (!need_loaded())
{
// failed to load .torrent file
@ -1374,7 +1381,7 @@ namespace libtorrent
picker().mark_as_finished(block_finished, 0);
maybe_done_flushing();
}
void torrent::on_disk_cache_complete(disk_io_job const* j)
{
TORRENT_ASSERT(have_piece(j->piece));
@ -1446,7 +1453,7 @@ namespace libtorrent
if (!tp) return;
add_extension(tp);
for (peer_iterator i = m_connections.begin();
i != m_connections.end(); ++i)
{
@ -1767,7 +1774,7 @@ namespace libtorrent
{
ev = errors::invalid_file_tag;
}
std::string info_hash = m_resume_data->node.dict_find_string_value("info-hash");
if (!ev && info_hash.empty())
ev = errors::missing_info_hash;
@ -1794,7 +1801,7 @@ namespace libtorrent
read_resume_data(m_resume_data->node);
}
}
#if TORRENT_USE_ASSERTS
m_resume_data_loaded = true;
#endif
@ -1855,7 +1862,7 @@ namespace libtorrent
m_web_seeds.insert(m_web_seeds.end(), web_seeds.begin(), web_seeds.end());
set_state(torrent_status::checking_resume_data);
#if TORRENT_USE_ASSERTS
m_resume_data_loaded = true;
#endif
@ -1878,7 +1885,7 @@ namespace libtorrent
if (!fs.pad_file_at(i) || fs.file_size(i) == 0) continue;
m_padding += boost::uint32_t(fs.file_size(i));
// TODO: instead of creating the picker up front here,
// maybe this whole section should move to need_picker()
need_picker();
@ -2557,7 +2564,7 @@ namespace libtorrent
{
TORRENT_ASSERT(should_check_files());
int num_outstanding = m_ses.settings().get_int(settings_pack::checking_mem_usage) * block_size()
int num_outstanding = settings().get_int(settings_pack::checking_mem_usage) * block_size()
/ m_torrent_file->piece_length();
if (num_outstanding <= 0) num_outstanding = 1;
@ -2680,7 +2687,7 @@ namespace libtorrent
return;
}
if (m_ses.settings().get_bool(settings_pack::disable_hash_checks)
if (settings().get_bool(settings_pack::disable_hash_checks)
|| sha1_hash(j->d.piece_hash) == m_torrent_file->hash_for_piece(j->piece))
{
if (has_picker() || !m_have_all)
@ -2824,7 +2831,7 @@ namespace libtorrent
// announce with the local discovery service
m_ses.announce_lsd(m_torrent_file->info_hash(), port
, m_ses.settings().get_bool(settings_pack::broadcast_lsd) && m_lsd_seq == 0);
, settings().get_bool(settings_pack::broadcast_lsd) && m_lsd_seq == 0);
++m_lsd_seq;
}
@ -2983,8 +2990,10 @@ namespace libtorrent
e = tracker_request::paused;
tracker_request req;
req.apply_ip_filter = m_apply_ip_filter
&& m_ses.settings().get_bool(settings_pack::apply_ip_filter_to_trackers);
if (settings().get_bool(settings_pack::apply_ip_filter_to_trackers)
&& m_apply_ip_filter)
req.filter = m_ip_filter;
req.info_hash = m_torrent_file->info_hash();
req.pid = m_ses.get_peer_id();
req.downloaded = m_stat.total_payload_download() - m_total_failed_bytes;
@ -3052,7 +3061,7 @@ namespace libtorrent
if (ae.is_working()) sent_announce = true;
continue;
}
req.url = ae.url;
req.event = e;
if (req.event == tracker_request::none)
@ -3069,7 +3078,7 @@ namespace libtorrent
// we only allow trackers if there is a proxy and issue
// a warning if there isn't one
std::string protocol = req.url.substr(0, req.url.find(':'));
int proxy_type = m_ses.settings().get_int(settings_pack::proxy_type);
int proxy_type = settings().get_int(settings_pack::proxy_type);
// http can run over any proxy, so as long as one is used
// it's OK. If no proxy is configured, skip this tracker
@ -3154,8 +3163,10 @@ namespace libtorrent
if (i == -1) i = 0;
tracker_request req;
req.apply_ip_filter = m_apply_ip_filter
&& m_ses.settings().get_bool(settings_pack::apply_ip_filter_to_trackers);
if (settings().get_bool(settings_pack::apply_ip_filter_to_trackers)
&& m_apply_ip_filter)
req.filter = m_ip_filter;
req.info_hash = m_torrent_file->info_hash();
req.kind = tracker_request::scrape_request;
req.url = m_trackers[i].url;
@ -3438,7 +3449,7 @@ namespace libtorrent
void torrent::update_auto_sequential()
{
if (!m_ses.settings().get_bool(settings_pack::auto_sequential))
if (!settings().get_bool(settings_pack::auto_sequential))
{
m_auto_sequential = false;
return;
@ -3469,8 +3480,8 @@ namespace libtorrent
// instead of waiting one second for session_impl::on_tick()
// to be called, connect to a few peers immediately
int conns = (std::min)(
m_ses.settings().get_int(settings_pack::torrent_connect_boost)
, m_ses.settings().get_int(settings_pack::connections_limit) - m_ses.num_connections());
settings().get_int(settings_pack::torrent_connect_boost)
, settings().get_int(settings_pack::connections_limit) - m_ses.num_connections());
if (conns > 0) m_need_connect_boost = false;
@ -3593,8 +3604,7 @@ namespace libtorrent
// TODO: add one peer per IP the hostname resolves to
tcp::endpoint host(host_list.front(), port);
if (m_apply_ip_filter
&& m_ses.get_ip_filter().access(host.address()) & ip_filter::blocked)
if (m_ip_filter && m_ip_filter->access(host.address()) & ip_filter::blocked)
{
#ifndef TORRENT_DISABLE_LOGGING
error_code ec;
@ -3936,7 +3946,7 @@ namespace libtorrent
dec_refcount("verify_piece");
int ret = j->ret;
if (m_ses.settings().get_bool(settings_pack::disable_hash_checks))
if (settings().get_bool(settings_pack::disable_hash_checks))
{
ret = 0;
}
@ -4377,7 +4387,7 @@ namespace libtorrent
allow_disconnect = peer->received_invalid_data(index, single_peer);
}
if (m_ses.settings().get_bool(settings_pack::use_parole_mode))
if (settings().get_bool(settings_pack::use_parole_mode))
p->on_parole = true;
int hashfails = p->hashfails;
@ -5128,7 +5138,7 @@ namespace libtorrent
// update the average download time and average
// download time deviation
int dl_time = total_milliseconds(aux::time_now() - i->first_requested);
if (m_average_piece_time == 0)
{
m_average_piece_time = dl_time;
@ -5138,7 +5148,7 @@ namespace libtorrent
int diff = abs(int(dl_time - m_average_piece_time));
if (m_piece_time_deviation == 0) m_piece_time_deviation = diff;
else m_piece_time_deviation = (m_piece_time_deviation * 9 + diff) / 10;
m_average_piece_time = (m_average_piece_time * 9 + dl_time) / 10;
}
}
@ -5225,7 +5235,7 @@ namespace libtorrent
TORRENT_ASSERT(num_have() >= m_picker->num_have_filtered());
update_gauge();
if (filter_updated)
{
update_peer_interest(was_finished);
@ -5361,7 +5371,7 @@ namespace libtorrent
// the vector need to have exactly one element for every file
// in the torrent
TORRENT_ASSERT(int(files.size()) == m_torrent_file->num_files());
int limit = int(files.size());
if (valid_metadata() && limit > m_torrent_file->num_files())
limit = m_torrent_file->num_files();
@ -5998,7 +6008,7 @@ namespace libtorrent
if (web->resolving) return;
if (int(m_connections.size()) >= m_max_connections
|| m_ses.num_connections() >= m_ses.settings().get_int(settings_pack::connections_limit))
|| m_ses.num_connections() >= settings().get_int(settings_pack::connections_limit))
return;
std::string protocol;
@ -6182,7 +6192,7 @@ namespace libtorrent
if (m_ses.is_aborted()) return;
if (int(m_connections.size()) >= m_max_connections
|| m_ses.num_connections() >= m_ses.settings().get_int(settings_pack::connections_limit))
|| m_ses.num_connections() >= settings().get_int(settings_pack::connections_limit))
return;
tcp::endpoint a(addrs[0], port);
@ -6206,8 +6216,7 @@ namespace libtorrent
return;
}
if (m_apply_ip_filter
&& m_ses.get_ip_filter().access(a.address()) & ip_filter::blocked)
if (m_ip_filter && m_ip_filter->access(a.address()) & ip_filter::blocked)
{
if (m_ses.alerts().should_post<peer_blocked_alert>())
m_ses.alerts().emplace_alert<peer_blocked_alert>(get_handle()
@ -6273,7 +6282,7 @@ namespace libtorrent
}
if (int(m_connections.size()) >= m_max_connections
|| m_ses.num_connections() >= m_ses.settings().get_int(settings_pack::connections_limit))
|| m_ses.num_connections() >= settings().get_int(settings_pack::connections_limit))
return;
connect_web_seed(web, web->endpoints.front());
@ -6286,8 +6295,7 @@ namespace libtorrent
TORRENT_ASSERT(is_single_thread());
if (m_abort) return;
if (m_apply_ip_filter
&& m_ses.get_ip_filter().access(a.address()) & ip_filter::blocked)
if (m_ip_filter && m_ip_filter->access(a.address()) & ip_filter::blocked)
{
if (m_ses.alerts().should_post<peer_blocked_alert>())
m_ses.alerts().emplace_alert<peer_blocked_alert>(get_handle()
@ -6343,8 +6351,8 @@ namespace libtorrent
return;
}
bool proxy_hostnames = m_ses.settings().get_bool(settings_pack::proxy_hostnames);
int proxy_type = m_ses.settings().get_int(settings_pack::proxy_type);
bool proxy_hostnames = settings().get_bool(settings_pack::proxy_hostnames);
int proxy_type = settings().get_int(settings_pack::proxy_type);
if (proxy_hostnames
&& (proxy_type == settings_pack::socks5
@ -6373,7 +6381,7 @@ namespace libtorrent
boost::shared_ptr<peer_connection> c;
peer_connection_args pack;
pack.ses = &m_ses;
pack.sett = &m_ses.settings();
pack.sett = &settings();
pack.stats_counters = &m_ses.stats_counters();
pack.allocator = &m_ses;
pack.disk_thread = &m_ses.disk_thread();
@ -6471,7 +6479,7 @@ namespace libtorrent
bool torrent::resolving_countries() const
{
return m_resolve_countries && !m_ses.settings().get_bool(settings_pack::force_proxy);
return m_resolve_countries && !settings().get_bool(settings_pack::force_proxy);
}
void torrent::resolve_peer_country(boost::shared_ptr<peer_connection> const& p) const
@ -6717,7 +6725,7 @@ namespace libtorrent
m_torrent_file->rename_file(i, new_filename);
}
}
m_added_time = rd.dict_find_int_value("added_time", m_added_time);
m_completed_time = rd.dict_find_int_value("completed_time", m_completed_time);
if (m_completed_time != 0 && m_completed_time < m_added_time)
@ -6738,7 +6746,7 @@ namespace libtorrent
int end_range = num_files - 1;
for (; end_range >= 0; --end_range) if (m_file_priority[end_range] != 1) break;
m_file_priority.resize(end_range + 1);
// initialize pad files to priority 0
file_storage const& fs = m_torrent_file->files();
for (int i = 0; i < (std::min)(fs.num_files(), end_range + 1); ++i)
@ -6842,7 +6850,7 @@ namespace libtorrent
return m_torrent_file;
}
void torrent::write_resume_data(entry& ret) const
{
using namespace libtorrent::detail; // for write_*_endpoint()
@ -6875,7 +6883,7 @@ namespace libtorrent
if (!m_url.empty()) ret["url"] = m_url;
if (!m_uuid.empty()) ret["uuid"] = m_uuid;
if (!m_source_feed_url.empty()) ret["feed"] = m_source_feed_url;
const sha1_hash& info_hash = torrent_file().info_hash();
ret["info-hash"] = std::string((char*)info_hash.begin(), (char*)info_hash.end());
@ -7329,11 +7337,12 @@ namespace libtorrent
TORRENT_ASSERT(want_peers() || ignore_limit);
TORRENT_ASSERT(m_ses.num_connections()
< m_ses.settings().get_int(settings_pack::connections_limit) || ignore_limit);
< settings().get_int(settings_pack::connections_limit) || ignore_limit);
tcp::endpoint a(peerinfo->ip());
TORRENT_ASSERT(!m_apply_ip_filter
|| (m_ses.get_ip_filter().access(peerinfo->address()) & ip_filter::blocked) == 0);
|| !m_ip_filter
|| (m_ip_filter->access(peerinfo->address()) & ip_filter::blocked) == 0);
boost::shared_ptr<socket_type> s(new socket_type(m_ses.get_io_service()));
@ -7366,18 +7375,18 @@ namespace libtorrent
// we'll instantiate a TCP connection
utp_socket_manager* sm = 0;
if (m_ses.settings().get_bool(settings_pack::enable_outgoing_utp)
&& (!m_ses.settings().get_bool(settings_pack::enable_outgoing_tcp)
if (settings().get_bool(settings_pack::enable_outgoing_utp)
&& (!settings().get_bool(settings_pack::enable_outgoing_tcp)
|| peerinfo->supports_utp
|| peerinfo->confirmed_supports_utp))
sm = m_ses.utp_socket_manager();
// don't make a TCP connection if it's disabled
if (sm == 0 && !m_ses.settings().get_bool(settings_pack::enable_outgoing_tcp)) return false;
if (sm == 0 && !settings().get_bool(settings_pack::enable_outgoing_tcp)) return false;
void* userdata = 0;
#ifdef TORRENT_USE_OPENSSL
if (is_ssl_torrent() && m_ses.settings().get_int(settings_pack::ssl_listen) != 0)
if (is_ssl_torrent() && settings().get_int(settings_pack::ssl_listen) != 0)
{
userdata = m_ssl_ctx.get();
}
@ -7414,7 +7423,7 @@ namespace libtorrent
peer_connection_args pack;
pack.ses = &m_ses;
pack.sett = &m_ses.settings();
pack.sett = &settings();
pack.stats_counters = &m_ses.stats_counters();
pack.allocator = &m_ses;
pack.disk_thread = &m_ses.disk_thread();
@ -7433,10 +7442,10 @@ namespace libtorrent
c->m_in_constructor = false;
#endif
c->add_stat(boost::int64_t(peerinfo->prev_amount_download) << 10
c->add_stat(boost::int64_t(peerinfo->prev_amount_download) << 10
, boost::int64_t(peerinfo->prev_amount_upload) << 10);
peerinfo->prev_amount_download = 0;
peerinfo->prev_amount_upload = 0;
peerinfo->prev_amount_download = 0;
peerinfo->prev_amount_upload = 0;
#ifndef TORRENT_DISABLE_EXTENSIONS
for (extension_list_t::iterator i = m_extensions.begin()
@ -7660,7 +7669,8 @@ namespace libtorrent
m_has_incoming = true;
if (m_apply_ip_filter
&& m_ses.get_ip_filter().access(p->remote().address()) & ip_filter::blocked)
&& m_ip_filter
&& m_ip_filter->access(p->remote().address()) & ip_filter::blocked)
{
if (m_ses.alerts().should_post<peer_blocked_alert>())
m_ses.alerts().emplace_alert<peer_blocked_alert>(get_handle()
@ -7878,7 +7888,7 @@ namespace libtorrent
// if the user disabled outgoing connections for seeding torrents,
// don't make any
if (!m_ses.settings().get_bool(settings_pack::seeding_outgoing_connections)
if (!settings().get_bool(settings_pack::seeding_outgoing_connections)
&& (m_state == torrent_status::seeding
|| m_state == torrent_status::finished))
return false;
@ -9790,10 +9800,10 @@ namespace libtorrent
bool is_inactive = is_inactive_internal();
if (is_inactive != m_inactive
&& m_ses.settings().get_bool(settings_pack::dont_count_slow_torrents))
&& settings().get_bool(settings_pack::dont_count_slow_torrents))
{
m_last_active_change = m_ses.session_time();
int delay = m_ses.settings().get_int(settings_pack::auto_manage_startup);
int delay = settings().get_int(settings_pack::auto_manage_startup);
m_inactivity_timer.expires_from_now(seconds(delay));
m_inactivity_timer.async_wait(boost::bind(&torrent::on_inactivity_tick
, shared_from_this(), _1));
@ -9806,10 +9816,10 @@ namespace libtorrent
{
if (is_finished())
return m_stat.upload_payload_rate()
< m_ses.settings().get_int(settings_pack::inactive_up_rate);
< settings().get_int(settings_pack::inactive_up_rate);
else
return m_stat.download_payload_rate()
< m_ses.settings().get_int(settings_pack::inactive_down_rate);
< settings().get_int(settings_pack::inactive_down_rate);
}
void torrent::on_inactivity_tick(error_code const& ec)
@ -9817,7 +9827,7 @@ namespace libtorrent
if (ec) return;
int now = m_ses.session_time();
int delay = m_ses.settings().get_int(settings_pack::auto_manage_startup);
int delay = settings().get_int(settings_pack::auto_manage_startup);
if (now - m_last_active_change < delay) return;
bool is_inactive = is_inactive_internal();
@ -9825,7 +9835,7 @@ namespace libtorrent
m_inactive = is_inactive;
if (m_ses.settings().get_bool(settings_pack::dont_count_slow_torrents))
if (settings().get_bool(settings_pack::dont_count_slow_torrents))
m_ses.trigger_auto_manage();
}
@ -9836,7 +9846,7 @@ namespace libtorrent
// if we have everything we want we don't need to connect to any web-seed
if (!is_finished() && !m_web_seeds.empty() && m_files_checked
&& int(m_connections.size()) < m_max_connections
&& m_ses.num_connections() < m_ses.settings().get_int(settings_pack::connections_limit))
&& m_ses.num_connections() < settings().get_int(settings_pack::connections_limit))
{
// keep trying web-seeds if there are any
// first find out which web seeds we are connected to
@ -10728,7 +10738,7 @@ namespace libtorrent
TORRENT_ASSERT(i != m_web_seeds.end());
if (i == m_web_seeds.end()) return;
if (retry == 0) retry = m_ses.settings().get_int(settings_pack::urlseed_wait_retry);
if (retry == 0) retry = settings().get_int(settings_pack::urlseed_wait_retry);
i->retry = aux::time_now() + seconds(retry);
}
@ -10799,7 +10809,8 @@ namespace libtorrent
#endif
if (m_apply_ip_filter
&& m_ses.get_ip_filter().access(adr.address()) & ip_filter::blocked)
&& m_ip_filter
&& m_ip_filter->access(adr.address()) & ip_filter::blocked)
{
if (alerts().should_post<peer_blocked_alert>())
alerts().emplace_alert<peer_blocked_alert>(get_handle()
@ -10951,8 +10962,8 @@ namespace libtorrent
torrent_state st = get_policy_state();
std::vector<address> banned;
m_peer_list->apply_ip_filter(m_ses.get_ip_filter(), &st, banned);
m_peer_list->apply_ip_filter(*m_ip_filter, &st, banned);
if (alerts().should_post<peer_blocked_alert>())
{
for (std::vector<address>::iterator i = banned.begin()
@ -10960,7 +10971,7 @@ namespace libtorrent
alerts().emplace_alert<peer_blocked_alert>(get_handle(), *i
, peer_blocked_alert::ip_filter);
}
peers_erased(st.erased);
}

View File

@ -202,14 +202,12 @@ namespace libtorrent
tracker_manager::tracker_manager(class udp_socket& sock
, counters& stats_counters
, resolver_interface& resolver
, struct ip_filter& ipf
, aux::session_settings const& sett
#if !defined TORRENT_DISABLE_LOGGING || TORRENT_USE_ASSERTS
, aux::session_logger& ses
#endif
)
: m_ip_filter(ipf)
, m_udp_socket(sock)
: m_udp_socket(sock)
, m_host_resolver(resolver)
, m_settings(sett)
, m_stats_counters(stats_counters)

View File

@ -208,13 +208,13 @@ namespace libtorrent
, end(addresses.end()); i != end; ++i)
m_endpoints.push_back(tcp::endpoint(*i, port));
if (tracker_req().apply_ip_filter)
if (tracker_req().filter)
{
// remove endpoints that are filtered by the IP filter
for (std::vector<tcp::endpoint>::iterator k = m_endpoints.begin();
k != m_endpoints.end();)
{
if (m_man.ip_filter().access(k->address()) == ip_filter::blocked)
if (tracker_req().filter->access(k->address()) == ip_filter::blocked)
{
#ifndef TORRENT_DISABLE_LOGGING
if (cb) cb->debug_log("*** UDP_TRACKER [ IP blocked by filter: %s ]"