You cannot select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
lokinet/llarp/router/router.cpp

1377 lines
36 KiB
C++

#include <memory>
#include "router.hpp"
#include <llarp/config/config.hpp>
#include <llarp/constants/proto.hpp>
#include <llarp/constants/files.hpp>
#include <llarp/crypto/crypto_libsodium.hpp>
#include <llarp/crypto/crypto.hpp>
#include <llarp/dht/context.hpp>
#include <llarp/dht/node.hpp>
#include <llarp/iwp/iwp.hpp>
#include <llarp/link/server.hpp>
#include <llarp/messages/link_message.hpp>
#include <llarp/net/net.hpp>
#include <llarp/net/route.hpp>
#include <stdexcept>
#include <llarp/util/buffer.hpp>
#include <llarp/util/logging/file_logger.hpp>
#include <llarp/util/logging/json_logger.hpp>
#include <llarp/util/logging/logger_syslog.hpp>
#include <llarp/util/logging/logger.hpp>
#include <llarp/util/meta/memfn.hpp>
#include <llarp/util/str.hpp>
#include <llarp/ev/ev.hpp>
#include <llarp/tooling/peer_stats_event.hpp>
7 years ago
#include <llarp/tooling/router_event.hpp>
#include <llarp/util/status.hpp>
#include <fstream>
#include <cstdlib>
#include <iterator>
#include <unordered_map>
#include <utility>
#if defined(ANDROID) || defined(IOS)
#include <unistd.h>
#endif
4 years ago
#if defined(WITH_SYSTEMD)
#include <systemd/sd-daemon.h>
#endif
#include <oxenmq/oxenmq.h>
static constexpr std::chrono::milliseconds ROUTER_TICK_INTERVAL = 100ms;
namespace llarp
{
3 years ago
Router::Router(EventLoop_ptr loop, std::shared_ptr<vpn::Platform> vpnPlatform)
: ready(false)
, m_lmq(std::make_shared<oxenmq::OxenMQ>())
, _loop(std::move(loop))
, _vpnPlatform(std::move(vpnPlatform))
, paths(this)
, _exitContext(this)
, _dht(llarp_dht_context_new(this))
, m_DiskThread(m_lmq->add_tagged_thread("disk"))
, inbound_link_msg_parser(this)
, _hiddenServiceContext(this)
, m_RPCServer(new rpc::RpcServer(m_lmq, this))
#ifdef LOKINET_HIVE
, _randomStartDelay(std::chrono::milliseconds((llarp::randint() % 1250) + 2000))
#else
4 years ago
, _randomStartDelay(std::chrono::seconds((llarp::randint() % 30) + 10))
#endif
, m_lokidRpcClient(std::make_shared<rpc::LokidRpcClient>(m_lmq, this))
{
m_keyManager = std::make_shared<KeyManager>();
4 years ago
// for lokid, so we don't close the connection when syncing the whitelist
m_lmq->MAX_MSG_SIZE = -1;
_stopping.store(false);
_running.store(false);
_lastTick = llarp::time_now_ms();
4 years ago
m_NextExploreAt = Clock_t::now();
}
Router::~Router()
{
llarp_dht_context_free(_dht);
}
5 years ago
util::StatusObject
Router::ExtractStatus() const
{
if (_running)
{
util::StatusObject peerStatsObj = nullptr;
if (m_peerDb)
peerStatsObj = m_peerDb->ExtractStatus();
return util::StatusObject{
{"running", true},
{"numNodesKnown", _nodedb->NumLoaded()},
{"dht", _dht->impl->ExtractStatus()},
{"services", _hiddenServiceContext.ExtractStatus()},
{"exit", _exitContext.ExtractStatus()},
{"links", _linkManager.ExtractStatus()},
{"outboundMessages", _outboundMessageHandler.ExtractStatus()},
{"peerStats", peerStatsObj}};
}
else
{
return util::StatusObject{{"running", false}};
}
}
bool
Router::HandleRecvLinkMessageBuffer(ILinkSession* session, const llarp_buffer_t& buf)
{
if (_stopping)
return true;
if (!session)
{
LogWarn("no link session");
return false;
}
return inbound_link_msg_parser.ProcessFrom(session, buf);
6 years ago
}
void
Router::Thaw()
{
// get pubkeys we are connected to
std::unordered_set<RouterID> peerPubkeys;
linkManager().ForEachPeer([&peerPubkeys](auto peer) {
if (not peer)
return;
peerPubkeys.emplace(peer->GetPubKey());
});
// close our sessions to them on link layer
linkManager().ForEachOutboundLink([peerPubkeys](const auto& link) {
for (const auto& remote : peerPubkeys)
link->CloseSessionTo(remote);
});
// thaw endpoints
hiddenServiceContext().ForEachService([](const auto& name, const auto& ep) -> bool {
LogInfo(name, " thawing...");
ep->Thaw();
return true;
});
LogInfo("We are ready to go bruh... probably");
}
void
Router::PersistSessionUntil(const RouterID& remote, llarp_time_t until)
6 years ago
{
_linkManager.PersistSessionUntil(remote, until);
}
4 years ago
void
Router::GossipRCIfNeeded(const RouterContact rc)
{
if (disableGossipingRC_TestingOnly())
return;
4 years ago
/// if we are not a service node forget about gossip
if (not IsServiceNode())
4 years ago
return;
/// wait for random uptime
if (std::chrono::milliseconds{Uptime()} < _randomStartDelay)
4 years ago
return;
_rcGossiper.GossipRC(rc);
}
6 years ago
bool
Router::GetRandomGoodRouter(RouterID& router)
6 years ago
{
if (whitelistRouters)
{
return _rcLookupHandler.GetRandomWhitelistRouter(router);
}
if (const auto maybe = nodedb()->GetRandom([](const auto&) -> bool { return true; }))
{
router = maybe->pubkey;
return true;
}
return false;
6 years ago
}
5 years ago
void
Router::PumpLL()
{
llarp::LogTrace("Router::PumpLL() start");
if (_stopping.load())
return;
paths.PumpDownstream();
paths.PumpUpstream();
_outboundMessageHandler.Tick();
_linkManager.PumpLinks();
llarp::LogTrace("Router::PumpLL() end");
5 years ago
}
bool
3 years ago
Router::SendToOrQueue(const RouterID& remote, const ILinkMessage& msg, SendStatusHandler handler)
{
return _outboundMessageHandler.QueueMessage(remote, msg, handler);
}
void
Router::ForEachPeer(std::function<void(const ILinkSession*, bool)> visit, bool randomize) const
{
_linkManager.ForEachPeer(visit, randomize);
}
void
Router::ForEachPeer(std::function<void(ILinkSession*)> visit)
{
_linkManager.ForEachPeer(visit);
}
void
Router::try_connect(fs::path rcfile)
6 years ago
{
RouterContact remote;
if (!remote.Read(rcfile.string().c_str()))
{
LogError("failure to decode or verify of remote RC");
return;
}
if (remote.Verify(Now()))
{
LogDebug("verified signature");
_outboundSessionMaker.CreateSessionTo(remote, nullptr);
}
else
LogError(rcfile, " contains invalid RC");
6 years ago
}
bool
Router::EnsureIdentity()
{
_encryption = m_keyManager->encryptionKey;
if (whitelistRouters)
{
#if defined(ANDROID) || defined(IOS)
LogError("running a service node on mobile device is not possible.");
return false;
#else
#if defined(_WIN32)
LogError("running a service node on windows is not possible.");
return false;
#endif
#endif
constexpr int maxTries = 5;
int numTries = 0;
while (numTries < maxTries)
{
numTries++;
try
{
_identity = RpcClient()->ObtainIdentityKey();
LogWarn("Obtained lokid identity keys");
break;
}
catch (const std::exception& e)
{
LogWarn(
"Failed attempt ",
numTries,
" of ",
maxTries,
" to get lokid identity keys because: ",
e.what());
if (numTries == maxTries)
throw;
}
}
}
else
{
_identity = m_keyManager->identityKey;
}
if (_identity.IsZero())
return false;
if (_encryption.IsZero())
return false;
return true;
}
6 years ago
bool
Router::Configure(std::shared_ptr<Config> c, bool isSNode, std::shared_ptr<NodeDB> nodedb)
{
Replace libuv with uvw & related refactoring - removes all the llarp_ev_* functions, replacing with methods/classes/functions in the llarp namespace. - banish ev/ev.h to the void - Passes various things by const lvalue ref, especially shared_ptr's that don't need to be copied (to avoid an atomic refcount increment/decrement). - Add a llarp::UDPHandle abstract class for UDP handling - Removes the UDP tick handler; code that needs tick can just do a separate handler on the event loop outside the UDP socket. - Adds an "OwnedBuffer" which owns its own memory but is implicitly convertible to a llarp_buffer_t. This is mostly needed to take over ownership of buffers from uvw without copying them as, currently, uvw does its own allocation (pending some open upstream issues/PRs). - Logic: - add `make_caller`/`call_forever`/`call_every` utility functions to abstract Call wrapping and dependent timed tasks. - Add inLogicThread() so that code can tell its inside the logic thread (typically for debugging assertions). - get rid of janky integer returns and dealing with cancellations on call_later: the other methods added here and the event loop code remove the need for them. - Event loop: - redo everything with uvw instead of libuv - rename EventLoopWakeup::Wakeup to EventLoopWakeup::Trigger to better reflect what it does. - add EventLoopRepeater for repeated events, and replace the code that reschedules itself every time it is called with a repeater. - Split up `EventLoop::run()` into a non-virtual base method and abstract `run_loop()` methods; the base method does a couple extra setup/teardown things that don't need to be in the derived class. - udp_listen is replaced with ev->udp(...) which returns a new UDPHandle object rather that needing gross C-style-but-not-actually-C-compatible structs. - Remove unused register_poll_fd_(un)readable - Use shared_ptr for EventLoopWakeup rather than returning a raw pointer; uvw lets us not have to worry about having the event loop class maintain ownership of it. - Add factory EventLoop::create() function to create a default (uvw-based) event loop (previously this was one of the llarp_ev_blahblah unnamespaced functions). - ev_libuv: this is mostly rewritten; all of the glue code/structs, in particular, are gone as they are no longer needed with uvw. - DNS: - Rename DnsHandler to DnsInterceptor to better describe what it does (this is the code that intercepts all DNS to the tun IP range for Android). - endpoint: - remove unused "isolated network" code - remove distinct (but actually always the same) variables for router/endpoint logic objects - llarp_buffer_t - make constructors type-safe against being called with points to non-size-1 values - tun packet reading: - read all available packets off the device/file descriptor; previously we were reading one packet at a time then returning to the event loop to poll again. - ReadNextPacket() now returns a 0-size packet if the read would block (so that we can implement the previous point). - ReadNextPacket() now throws on I/O error - Miscellaneous code cleanups/simplifications
3 years ago
m_Config = std::move(c);
auto& conf = *m_Config;
whitelistRouters = conf.lokid.whitelistRouters;
if (whitelistRouters)
lokidRPCAddr = oxenmq::address(conf.lokid.lokidRPCAddr);
enableRPCServer = conf.api.m_enableRPCServer;
if (enableRPCServer)
rpcBindAddr = oxenmq::address(conf.api.m_rpcBindAddr);
if (not StartRpcServer())
throw std::runtime_error("Failed to start rpc server");
Config file improvements (#1397) * Config file API/comment improvements API improvements: ================= Make the config API use position-independent tag parameters (Required, Default{123}, MultiValue) rather than a sequence of bools with overloads. For example, instead of: conf.defineOption<int>("a", "b", false, true, 123, [] { ... }); you now write: conf.defineOption<int>("a", "b", MultiValue, Default{123}, [] { ... }); The tags are: - Required - MultiValue - Default{value} plus new abilities (see below): - Hidden - RelayOnly - ClientOnly - Comment{"line1", "line2", "line3"} Made option definition more powerful: ===================================== - `Hidden` allows you to define an option that won't show up in the generated config file if it isn't set. - `RelayOnly`/`ClientOnly` sets up an option that is only accepted and only shows up for relay or client configs. (If neither is specified the option shows up in both modes). - `Comment{...}` lets the option comments be specified as part of the defineOption. Comment improvements ==================== - Rewrote comments for various options to expand on details. - Inlined all the comments with the option definitions. - Several options that were missing comments got comments added. - Made various options for deprecated and or internal options hidden by default so that they don't show up in a default config file. - show the section comment (but not option comments) *after* the [section] tag instead of before it as it makes more sense that way (particularly for the [bind] section which has a new long comment to describe how it works). Disable profiling by default ============================ We had this weird state where we use and store profiling by default but never *load* it when starting up. This commit makes us just not use profiling at all unless explicitly enabled. Other misc changes: =================== - change default worker threads to 0 (= num cpus) instead of 1, and fix it to allow 0. - Actually apply worker-threads option - fixed default data-dir value erroneously having quotes around it - reordered ifname/ifaddr/mapaddr (was previously mapaddr/ifaddr/ifname) as mapaddr is a sort of specialization of ifaddr and so makes more sense to come after it (particularly because it now references ifaddr in its help message). - removed peer-stats option (since we always require it for relays and never use it for clients) - removed router profiles filename option (this doesn't need to be configurable) - removed defunct `service-node-seed` option - Change default logging output file to "" (which means stdout), and also made "-" work for stdout. * Router hive compilation fixes * Comments for SNApp SRV settings in ini file * Add extra blank line after section comments * Better deprecated option handling Allow {client,relay}-only options in {relay,client} configs to be specified as implicitly deprecated options: they warn, and don't set anything. Add an explicit `Deprecated` tag and move deprecated option handling into definition.cpp. * Move backwards compat options into section definitions Keep the "addBackwardsCompatibleConfigOptions" only for options in sections that no longer exist. * Fix INI parsing issues & C++17-ify - don't allow inline comments because it seems they aren't allowed in ini formats in general, and is going to cause problems if there is a comment character in a value (e.g. an exit auth string). Additionally it was breaking on a line such as: # some comment; see? because it was treating only `; see?` as the comment and then producing an error message about the rest of the line being invalid. - make section parsing stricter: the `[` and `]` have to be at the beginning at end of the line now (after stripping whitespace). - Move whitespace stripping to the top since everything in here does it. - chop off string_view suffix/prefix rather than maintaining position values - fix potential infinite loop/segfault when given a line such as `]foo[` * Make config parsing failure fatal Load() LogError's and returns false on failure, so we weren't aborting on config file errors. * Formatting: allow `{}` for empty functions/structs Instead of using two lines when empty: { } * Make default dns bind 127.0.0.1 on non-Linux * Don't show empty section; fix tests We can conceivably have sections that only make sense for clients or relays, and so want to completely omit that section if we have no options for the type of config being generated. Also fixes missing empty lines between tests. Co-authored-by: Thomas Winget <tewinget@gmail.com>
4 years ago
if (conf.router.m_workerThreads > 0)
m_lmq->set_general_threads(conf.router.m_workerThreads);
m_lmq->start();
Replace libuv with uvw & related refactoring - removes all the llarp_ev_* functions, replacing with methods/classes/functions in the llarp namespace. - banish ev/ev.h to the void - Passes various things by const lvalue ref, especially shared_ptr's that don't need to be copied (to avoid an atomic refcount increment/decrement). - Add a llarp::UDPHandle abstract class for UDP handling - Removes the UDP tick handler; code that needs tick can just do a separate handler on the event loop outside the UDP socket. - Adds an "OwnedBuffer" which owns its own memory but is implicitly convertible to a llarp_buffer_t. This is mostly needed to take over ownership of buffers from uvw without copying them as, currently, uvw does its own allocation (pending some open upstream issues/PRs). - Logic: - add `make_caller`/`call_forever`/`call_every` utility functions to abstract Call wrapping and dependent timed tasks. - Add inLogicThread() so that code can tell its inside the logic thread (typically for debugging assertions). - get rid of janky integer returns and dealing with cancellations on call_later: the other methods added here and the event loop code remove the need for them. - Event loop: - redo everything with uvw instead of libuv - rename EventLoopWakeup::Wakeup to EventLoopWakeup::Trigger to better reflect what it does. - add EventLoopRepeater for repeated events, and replace the code that reschedules itself every time it is called with a repeater. - Split up `EventLoop::run()` into a non-virtual base method and abstract `run_loop()` methods; the base method does a couple extra setup/teardown things that don't need to be in the derived class. - udp_listen is replaced with ev->udp(...) which returns a new UDPHandle object rather that needing gross C-style-but-not-actually-C-compatible structs. - Remove unused register_poll_fd_(un)readable - Use shared_ptr for EventLoopWakeup rather than returning a raw pointer; uvw lets us not have to worry about having the event loop class maintain ownership of it. - Add factory EventLoop::create() function to create a default (uvw-based) event loop (previously this was one of the llarp_ev_blahblah unnamespaced functions). - ev_libuv: this is mostly rewritten; all of the glue code/structs, in particular, are gone as they are no longer needed with uvw. - DNS: - Rename DnsHandler to DnsInterceptor to better describe what it does (this is the code that intercepts all DNS to the tun IP range for Android). - endpoint: - remove unused "isolated network" code - remove distinct (but actually always the same) variables for router/endpoint logic objects - llarp_buffer_t - make constructors type-safe against being called with points to non-size-1 values - tun packet reading: - read all available packets off the device/file descriptor; previously we were reading one packet at a time then returning to the event loop to poll again. - ReadNextPacket() now returns a 0-size packet if the read would block (so that we can implement the previous point). - ReadNextPacket() now throws on I/O error - Miscellaneous code cleanups/simplifications
3 years ago
_nodedb = std::move(nodedb);
m_isServiceNode = conf.router.m_isRelay;
if (whitelistRouters)
{
m_lokidRpcClient->ConnectAsync(lokidRPCAddr);
}
// fetch keys
if (not m_keyManager->initialize(conf, true, isSNode))
throw std::runtime_error("KeyManager failed to initialize");
if (!FromConfig(conf))
throw std::runtime_error("FromConfig() failed");
if (!InitOutboundLinks())
throw std::runtime_error("InitOutboundLinks() failed");
if (not EnsureIdentity())
throw std::runtime_error("EnsureIdentity() failed");
m_RoutePoker.Init(this);
return true;
}
5 years ago
/// called in disk worker thread
void
Router::HandleSaveRC() const
5 years ago
{
std::string fname = our_rc_file.string();
_rc.Write(fname.c_str());
5 years ago
}
bool
Router::SaveRC()
6 years ago
{
LogDebug("verify RC signature");
if (!_rc.Verify(Now()))
{
Dump<MAX_RC_SIZE>(rc());
LogError("RC is invalid, not saving");
return false;
}
QueueDiskIO([&]() { HandleSaveRC(); });
5 years ago
return true;
6 years ago
}
6 years ago
bool
Router::IsServiceNode() const
{
return m_isServiceNode;
}
6 years ago
void
Router::Close()
{
if (_onDown)
_onDown();
LogInfo("closing router");
_loop->stop();
_running.store(false);
}
bool
Router::ParseRoutingMessageBuffer(
const llarp_buffer_t& buf, routing::IMessageHandler* h, const PathID_t& rxid)
{
return inbound_routing_msg_parser.ParseMessageBuffer(buf, h, rxid, this);
}
bool
Router::ConnectionToRouterAllowed(const RouterID& router) const
{
return _rcLookupHandler.RemoteIsAllowed(router);
}
size_t
Router::NumberOfConnectedRouters() const
{
return _linkManager.NumberOfConnectedRouters();
}
size_t
Router::NumberOfConnectedClients() const
{
return _linkManager.NumberOfConnectedClients();
}
bool
Router::UpdateOurRC(bool rotateKeys)
{
SecretKey nextOnionKey;
RouterContact nextRC = _rc;
if (rotateKeys)
{
CryptoManager::instance()->encryption_keygen(nextOnionKey);
std::string f = encryption_keyfile.string();
5 years ago
// TODO: use disk worker
if (nextOnionKey.SaveToFile(f.c_str()))
{
nextRC.enckey = seckey_topublic(nextOnionKey);
_encryption = nextOnionKey;
}
}
if (!nextRC.Sign(identity()))
return false;
if (!nextRC.Verify(time_now_ms(), false))
return false;
_rc = std::move(nextRC);
if (rotateKeys)
{
// propagate RC by renegotiating sessions
ForEachPeer([](ILinkSession* s) {
if (s->RenegotiateSession())
LogInfo("renegotiated session");
else
LogWarn("failed to renegotiate session");
});
}
if (IsServiceNode())
return SaveRC();
else
return true;
5 years ago
}
bool
Router::FromConfig(const Config& conf)
{
// Set netid before anything else
if (!conf.router.m_netId.empty() && strcmp(conf.router.m_netId.c_str(), llarp::DEFAULT_NETID))
{
const auto& netid = conf.router.m_netId;
llarp::LogWarn(
"!!!! you have manually set netid to be '",
netid,
"' which does not equal '",
llarp::DEFAULT_NETID,
"' you will run as a different network, good luck "
"and don't forget: something something MUH traffic "
"shape correlation !!!!");
NetID::DefaultValue() = NetID(reinterpret_cast<const byte_t*>(netid.c_str()));
// reset netid in our rc
_rc.netID = llarp::NetID();
}
// IWP config
m_OutboundPort = conf.links.m_OutboundLink.port;
// Router config
_rc.SetNick(conf.router.m_nickname);
_outboundSessionMaker.maxConnectedRouters = conf.router.m_maxConnectedRouters;
_outboundSessionMaker.minConnectedRouters = conf.router.m_minConnectedRouters;
encryption_keyfile = m_keyManager->m_encKeyPath;
our_rc_file = m_keyManager->m_rcPath;
transport_keyfile = m_keyManager->m_transportKeyPath;
ident_keyfile = m_keyManager->m_idKeyPath;
if (not conf.router.m_publicAddress.isEmpty())
_ourAddress = conf.router.m_publicAddress.createSockAddr();
RouterContact::BlockBogons = conf.router.m_blockBogons;
// Lokid Config
whitelistRouters = conf.lokid.whitelistRouters;
lokidRPCAddr = oxenmq::address(conf.lokid.lokidRPCAddr);
Config file improvements (#1397) * Config file API/comment improvements API improvements: ================= Make the config API use position-independent tag parameters (Required, Default{123}, MultiValue) rather than a sequence of bools with overloads. For example, instead of: conf.defineOption<int>("a", "b", false, true, 123, [] { ... }); you now write: conf.defineOption<int>("a", "b", MultiValue, Default{123}, [] { ... }); The tags are: - Required - MultiValue - Default{value} plus new abilities (see below): - Hidden - RelayOnly - ClientOnly - Comment{"line1", "line2", "line3"} Made option definition more powerful: ===================================== - `Hidden` allows you to define an option that won't show up in the generated config file if it isn't set. - `RelayOnly`/`ClientOnly` sets up an option that is only accepted and only shows up for relay or client configs. (If neither is specified the option shows up in both modes). - `Comment{...}` lets the option comments be specified as part of the defineOption. Comment improvements ==================== - Rewrote comments for various options to expand on details. - Inlined all the comments with the option definitions. - Several options that were missing comments got comments added. - Made various options for deprecated and or internal options hidden by default so that they don't show up in a default config file. - show the section comment (but not option comments) *after* the [section] tag instead of before it as it makes more sense that way (particularly for the [bind] section which has a new long comment to describe how it works). Disable profiling by default ============================ We had this weird state where we use and store profiling by default but never *load* it when starting up. This commit makes us just not use profiling at all unless explicitly enabled. Other misc changes: =================== - change default worker threads to 0 (= num cpus) instead of 1, and fix it to allow 0. - Actually apply worker-threads option - fixed default data-dir value erroneously having quotes around it - reordered ifname/ifaddr/mapaddr (was previously mapaddr/ifaddr/ifname) as mapaddr is a sort of specialization of ifaddr and so makes more sense to come after it (particularly because it now references ifaddr in its help message). - removed peer-stats option (since we always require it for relays and never use it for clients) - removed router profiles filename option (this doesn't need to be configurable) - removed defunct `service-node-seed` option - Change default logging output file to "" (which means stdout), and also made "-" work for stdout. * Router hive compilation fixes * Comments for SNApp SRV settings in ini file * Add extra blank line after section comments * Better deprecated option handling Allow {client,relay}-only options in {relay,client} configs to be specified as implicitly deprecated options: they warn, and don't set anything. Add an explicit `Deprecated` tag and move deprecated option handling into definition.cpp. * Move backwards compat options into section definitions Keep the "addBackwardsCompatibleConfigOptions" only for options in sections that no longer exist. * Fix INI parsing issues & C++17-ify - don't allow inline comments because it seems they aren't allowed in ini formats in general, and is going to cause problems if there is a comment character in a value (e.g. an exit auth string). Additionally it was breaking on a line such as: # some comment; see? because it was treating only `; see?` as the comment and then producing an error message about the rest of the line being invalid. - make section parsing stricter: the `[` and `]` have to be at the beginning at end of the line now (after stripping whitespace). - Move whitespace stripping to the top since everything in here does it. - chop off string_view suffix/prefix rather than maintaining position values - fix potential infinite loop/segfault when given a line such as `]foo[` * Make config parsing failure fatal Load() LogError's and returns false on failure, so we weren't aborting on config file errors. * Formatting: allow `{}` for empty functions/structs Instead of using two lines when empty: { } * Make default dns bind 127.0.0.1 on non-Linux * Don't show empty section; fix tests We can conceivably have sections that only make sense for clients or relays, and so want to completely omit that section if we have no options for the type of config being generated. Also fixes missing empty lines between tests. Co-authored-by: Thomas Winget <tewinget@gmail.com>
4 years ago
m_isServiceNode = conf.router.m_isRelay;
auto& networkConfig = conf.network;
/// build a set of strictConnectPubkeys (
/// TODO: make this consistent with config -- do we support multiple strict connections
// or not?
std::unordered_set<RouterID> strictConnectPubkeys;
if (not networkConfig.m_strictConnect.empty())
{
const auto& val = networkConfig.m_strictConnect;
if (IsServiceNode())
throw std::runtime_error("cannot use strict-connect option as service node");
3 years ago
strictConnectPubkeys.insert(val.begin(), val.end());
}
std::vector<fs::path> configRouters = conf.connect.routers;
configRouters.insert(
configRouters.end(), conf.bootstrap.files.begin(), conf.bootstrap.files.end());
// if our conf had no bootstrap files specified, try the default location of
// <DATA_DIR>/bootstrap.signed. If this isn't present, leave a useful error message
if (configRouters.empty() and conf.bootstrap.routers.empty())
{
// TODO: use constant
fs::path defaultBootstrapFile = conf.router.m_dataDir / "bootstrap.signed";
if (fs::exists(defaultBootstrapFile))
{
configRouters.push_back(defaultBootstrapFile);
}
else if (not conf.bootstrap.seednode)
{
LogError("No bootstrap files specified in config file, and the default");
LogError("bootstrap file ", defaultBootstrapFile, " does not exist.");
LogError("Please provide a bootstrap file (e.g. run 'lokinet-bootstrap)'");
throw std::runtime_error("No bootstrap files available.");
}
}
BootstrapList b_list;
for (const auto& router : configRouters)
{
bool isListFile = false;
{
std::ifstream inf(router.c_str(), std::ios::binary);
if (inf.is_open())
{
const char ch = inf.get();
isListFile = ch == 'l';
}
}
if (isListFile)
{
if (not BDecodeReadFile(router, b_list))
{
throw std::runtime_error(stringify("failed to read bootstrap list file '", router, "'"));
}
}
else
{
RouterContact rc;
if (not rc.Read(router))
{
throw std::runtime_error(
stringify("failed to decode bootstrap RC, file='", router, "' rc=", rc));
}
b_list.insert(rc);
}
}
for (const auto& rc : conf.bootstrap.routers)
{
b_list.emplace(rc);
}
for (auto& rc : b_list)
{
if (not rc.Verify(Now()))
{
LogWarn("ignoring invalid RC: ", RouterID(rc.pubkey));
continue;
}
bootstrapRCList.emplace(std::move(rc));
}
if (bootstrapRCList.empty() and not conf.bootstrap.seednode)
{
throw std::runtime_error{"we have no bootstrap nodes"};
}
if (conf.bootstrap.seednode)
{
LogInfo("we are a seed node");
}
else
LogInfo("Loaded ", bootstrapRCList.size(), " bootstrap routers");
// Init components after relevant config settings loaded
_outboundMessageHandler.Init(&_linkManager, &_rcLookupHandler, _loop);
_outboundSessionMaker.Init(
this,
&_linkManager,
&_rcLookupHandler,
&_routerProfiling,
_loop,
util::memFn(&AbstractRouter::QueueWork, this));
_linkManager.Init(&_outboundSessionMaker);
_rcLookupHandler.Init(
_dht,
_nodedb,
_loop,
util::memFn(&AbstractRouter::QueueWork, this),
&_linkManager,
&_hiddenServiceContext,
strictConnectPubkeys,
bootstrapRCList,
whitelistRouters,
m_isServiceNode);
std::vector<LinksConfig::LinkInfo> inboundLinks = conf.links.m_InboundLinks;
if (inboundLinks.empty() and m_isServiceNode)
{
const auto& publicAddr = conf.router.m_publicAddress;
if (publicAddr.isEmpty() or not publicAddr.hasPort())
{
throw std::runtime_error(
"service node enabled but could not find a public IP to bind to; you need to set the "
"public-ip= and public-port= options");
}
inboundLinks.push_back(LinksConfig::LinkInfo{"0.0.0.0", AF_INET, *publicAddr.getPort()});
}
// create inbound links, if we are a service node
for (const LinksConfig::LinkInfo& serverConfig : inboundLinks)
{
auto server = iwp::NewInboundLink(
m_keyManager,
loop(),
util::memFn(&AbstractRouter::rc, this),
util::memFn(&AbstractRouter::HandleRecvLinkMessageBuffer, this),
util::memFn(&AbstractRouter::Sign, this),
nullptr,
util::memFn(&Router::ConnectionEstablished, this),
util::memFn(&AbstractRouter::CheckRenegotiateValid, this),
util::memFn(&Router::ConnectionTimedOut, this),
util::memFn(&AbstractRouter::SessionClosed, this),
util::memFn(&AbstractRouter::PumpLL, this),
util::memFn(&AbstractRouter::QueueWork, this));
const std::string& key = serverConfig.interface;
int af = serverConfig.addressFamily;
uint16_t port = serverConfig.port;
if (!server->Configure(loop(), key, af, port))
{
throw std::runtime_error(stringify("failed to bind inbound link on ", key, " port ", port));
}
_linkManager.AddLink(std::move(server), true);
}
// profiling
_profilesFile = conf.router.m_dataDir / "profiles.dat";
// Network config
Config file improvements (#1397) * Config file API/comment improvements API improvements: ================= Make the config API use position-independent tag parameters (Required, Default{123}, MultiValue) rather than a sequence of bools with overloads. For example, instead of: conf.defineOption<int>("a", "b", false, true, 123, [] { ... }); you now write: conf.defineOption<int>("a", "b", MultiValue, Default{123}, [] { ... }); The tags are: - Required - MultiValue - Default{value} plus new abilities (see below): - Hidden - RelayOnly - ClientOnly - Comment{"line1", "line2", "line3"} Made option definition more powerful: ===================================== - `Hidden` allows you to define an option that won't show up in the generated config file if it isn't set. - `RelayOnly`/`ClientOnly` sets up an option that is only accepted and only shows up for relay or client configs. (If neither is specified the option shows up in both modes). - `Comment{...}` lets the option comments be specified as part of the defineOption. Comment improvements ==================== - Rewrote comments for various options to expand on details. - Inlined all the comments with the option definitions. - Several options that were missing comments got comments added. - Made various options for deprecated and or internal options hidden by default so that they don't show up in a default config file. - show the section comment (but not option comments) *after* the [section] tag instead of before it as it makes more sense that way (particularly for the [bind] section which has a new long comment to describe how it works). Disable profiling by default ============================ We had this weird state where we use and store profiling by default but never *load* it when starting up. This commit makes us just not use profiling at all unless explicitly enabled. Other misc changes: =================== - change default worker threads to 0 (= num cpus) instead of 1, and fix it to allow 0. - Actually apply worker-threads option - fixed default data-dir value erroneously having quotes around it - reordered ifname/ifaddr/mapaddr (was previously mapaddr/ifaddr/ifname) as mapaddr is a sort of specialization of ifaddr and so makes more sense to come after it (particularly because it now references ifaddr in its help message). - removed peer-stats option (since we always require it for relays and never use it for clients) - removed router profiles filename option (this doesn't need to be configurable) - removed defunct `service-node-seed` option - Change default logging output file to "" (which means stdout), and also made "-" work for stdout. * Router hive compilation fixes * Comments for SNApp SRV settings in ini file * Add extra blank line after section comments * Better deprecated option handling Allow {client,relay}-only options in {relay,client} configs to be specified as implicitly deprecated options: they warn, and don't set anything. Add an explicit `Deprecated` tag and move deprecated option handling into definition.cpp. * Move backwards compat options into section definitions Keep the "addBackwardsCompatibleConfigOptions" only for options in sections that no longer exist. * Fix INI parsing issues & C++17-ify - don't allow inline comments because it seems they aren't allowed in ini formats in general, and is going to cause problems if there is a comment character in a value (e.g. an exit auth string). Additionally it was breaking on a line such as: # some comment; see? because it was treating only `; see?` as the comment and then producing an error message about the rest of the line being invalid. - make section parsing stricter: the `[` and `]` have to be at the beginning at end of the line now (after stripping whitespace). - Move whitespace stripping to the top since everything in here does it. - chop off string_view suffix/prefix rather than maintaining position values - fix potential infinite loop/segfault when given a line such as `]foo[` * Make config parsing failure fatal Load() LogError's and returns false on failure, so we weren't aborting on config file errors. * Formatting: allow `{}` for empty functions/structs Instead of using two lines when empty: { } * Make default dns bind 127.0.0.1 on non-Linux * Don't show empty section; fix tests We can conceivably have sections that only make sense for clients or relays, and so want to completely omit that section if we have no options for the type of config being generated. Also fixes missing empty lines between tests. Co-authored-by: Thomas Winget <tewinget@gmail.com>
4 years ago
if (conf.network.m_enableProfiling.value_or(false))
6 years ago
{
LogInfo("router profiling enabled");
if (not fs::exists(_profilesFile))
{
LogInfo("no profiles file at ", _profilesFile, " skipping");
}
else
{
LogInfo("loading router profiles from ", _profilesFile);
routerProfiling().Load(_profilesFile);
}
6 years ago
}
Config file improvements (#1397) * Config file API/comment improvements API improvements: ================= Make the config API use position-independent tag parameters (Required, Default{123}, MultiValue) rather than a sequence of bools with overloads. For example, instead of: conf.defineOption<int>("a", "b", false, true, 123, [] { ... }); you now write: conf.defineOption<int>("a", "b", MultiValue, Default{123}, [] { ... }); The tags are: - Required - MultiValue - Default{value} plus new abilities (see below): - Hidden - RelayOnly - ClientOnly - Comment{"line1", "line2", "line3"} Made option definition more powerful: ===================================== - `Hidden` allows you to define an option that won't show up in the generated config file if it isn't set. - `RelayOnly`/`ClientOnly` sets up an option that is only accepted and only shows up for relay or client configs. (If neither is specified the option shows up in both modes). - `Comment{...}` lets the option comments be specified as part of the defineOption. Comment improvements ==================== - Rewrote comments for various options to expand on details. - Inlined all the comments with the option definitions. - Several options that were missing comments got comments added. - Made various options for deprecated and or internal options hidden by default so that they don't show up in a default config file. - show the section comment (but not option comments) *after* the [section] tag instead of before it as it makes more sense that way (particularly for the [bind] section which has a new long comment to describe how it works). Disable profiling by default ============================ We had this weird state where we use and store profiling by default but never *load* it when starting up. This commit makes us just not use profiling at all unless explicitly enabled. Other misc changes: =================== - change default worker threads to 0 (= num cpus) instead of 1, and fix it to allow 0. - Actually apply worker-threads option - fixed default data-dir value erroneously having quotes around it - reordered ifname/ifaddr/mapaddr (was previously mapaddr/ifaddr/ifname) as mapaddr is a sort of specialization of ifaddr and so makes more sense to come after it (particularly because it now references ifaddr in its help message). - removed peer-stats option (since we always require it for relays and never use it for clients) - removed router profiles filename option (this doesn't need to be configurable) - removed defunct `service-node-seed` option - Change default logging output file to "" (which means stdout), and also made "-" work for stdout. * Router hive compilation fixes * Comments for SNApp SRV settings in ini file * Add extra blank line after section comments * Better deprecated option handling Allow {client,relay}-only options in {relay,client} configs to be specified as implicitly deprecated options: they warn, and don't set anything. Add an explicit `Deprecated` tag and move deprecated option handling into definition.cpp. * Move backwards compat options into section definitions Keep the "addBackwardsCompatibleConfigOptions" only for options in sections that no longer exist. * Fix INI parsing issues & C++17-ify - don't allow inline comments because it seems they aren't allowed in ini formats in general, and is going to cause problems if there is a comment character in a value (e.g. an exit auth string). Additionally it was breaking on a line such as: # some comment; see? because it was treating only `; see?` as the comment and then producing an error message about the rest of the line being invalid. - make section parsing stricter: the `[` and `]` have to be at the beginning at end of the line now (after stripping whitespace). - Move whitespace stripping to the top since everything in here does it. - chop off string_view suffix/prefix rather than maintaining position values - fix potential infinite loop/segfault when given a line such as `]foo[` * Make config parsing failure fatal Load() LogError's and returns false on failure, so we weren't aborting on config file errors. * Formatting: allow `{}` for empty functions/structs Instead of using two lines when empty: { } * Make default dns bind 127.0.0.1 on non-Linux * Don't show empty section; fix tests We can conceivably have sections that only make sense for clients or relays, and so want to completely omit that section if we have no options for the type of config being generated. Also fixes missing empty lines between tests. Co-authored-by: Thomas Winget <tewinget@gmail.com>
4 years ago
else
{
Config file improvements (#1397) * Config file API/comment improvements API improvements: ================= Make the config API use position-independent tag parameters (Required, Default{123}, MultiValue) rather than a sequence of bools with overloads. For example, instead of: conf.defineOption<int>("a", "b", false, true, 123, [] { ... }); you now write: conf.defineOption<int>("a", "b", MultiValue, Default{123}, [] { ... }); The tags are: - Required - MultiValue - Default{value} plus new abilities (see below): - Hidden - RelayOnly - ClientOnly - Comment{"line1", "line2", "line3"} Made option definition more powerful: ===================================== - `Hidden` allows you to define an option that won't show up in the generated config file if it isn't set. - `RelayOnly`/`ClientOnly` sets up an option that is only accepted and only shows up for relay or client configs. (If neither is specified the option shows up in both modes). - `Comment{...}` lets the option comments be specified as part of the defineOption. Comment improvements ==================== - Rewrote comments for various options to expand on details. - Inlined all the comments with the option definitions. - Several options that were missing comments got comments added. - Made various options for deprecated and or internal options hidden by default so that they don't show up in a default config file. - show the section comment (but not option comments) *after* the [section] tag instead of before it as it makes more sense that way (particularly for the [bind] section which has a new long comment to describe how it works). Disable profiling by default ============================ We had this weird state where we use and store profiling by default but never *load* it when starting up. This commit makes us just not use profiling at all unless explicitly enabled. Other misc changes: =================== - change default worker threads to 0 (= num cpus) instead of 1, and fix it to allow 0. - Actually apply worker-threads option - fixed default data-dir value erroneously having quotes around it - reordered ifname/ifaddr/mapaddr (was previously mapaddr/ifaddr/ifname) as mapaddr is a sort of specialization of ifaddr and so makes more sense to come after it (particularly because it now references ifaddr in its help message). - removed peer-stats option (since we always require it for relays and never use it for clients) - removed router profiles filename option (this doesn't need to be configurable) - removed defunct `service-node-seed` option - Change default logging output file to "" (which means stdout), and also made "-" work for stdout. * Router hive compilation fixes * Comments for SNApp SRV settings in ini file * Add extra blank line after section comments * Better deprecated option handling Allow {client,relay}-only options in {relay,client} configs to be specified as implicitly deprecated options: they warn, and don't set anything. Add an explicit `Deprecated` tag and move deprecated option handling into definition.cpp. * Move backwards compat options into section definitions Keep the "addBackwardsCompatibleConfigOptions" only for options in sections that no longer exist. * Fix INI parsing issues & C++17-ify - don't allow inline comments because it seems they aren't allowed in ini formats in general, and is going to cause problems if there is a comment character in a value (e.g. an exit auth string). Additionally it was breaking on a line such as: # some comment; see? because it was treating only `; see?` as the comment and then producing an error message about the rest of the line being invalid. - make section parsing stricter: the `[` and `]` have to be at the beginning at end of the line now (after stripping whitespace). - Move whitespace stripping to the top since everything in here does it. - chop off string_view suffix/prefix rather than maintaining position values - fix potential infinite loop/segfault when given a line such as `]foo[` * Make config parsing failure fatal Load() LogError's and returns false on failure, so we weren't aborting on config file errors. * Formatting: allow `{}` for empty functions/structs Instead of using two lines when empty: { } * Make default dns bind 127.0.0.1 on non-Linux * Don't show empty section; fix tests We can conceivably have sections that only make sense for clients or relays, and so want to completely omit that section if we have no options for the type of config being generated. Also fixes missing empty lines between tests. Co-authored-by: Thomas Winget <tewinget@gmail.com>
4 years ago
routerProfiling().Disable();
LogInfo("router profiling disabled");
}
// API config
if (not IsServiceNode())
{
hiddenServiceContext().AddEndpoint(conf);
}
// peer stats
Config file improvements (#1397) * Config file API/comment improvements API improvements: ================= Make the config API use position-independent tag parameters (Required, Default{123}, MultiValue) rather than a sequence of bools with overloads. For example, instead of: conf.defineOption<int>("a", "b", false, true, 123, [] { ... }); you now write: conf.defineOption<int>("a", "b", MultiValue, Default{123}, [] { ... }); The tags are: - Required - MultiValue - Default{value} plus new abilities (see below): - Hidden - RelayOnly - ClientOnly - Comment{"line1", "line2", "line3"} Made option definition more powerful: ===================================== - `Hidden` allows you to define an option that won't show up in the generated config file if it isn't set. - `RelayOnly`/`ClientOnly` sets up an option that is only accepted and only shows up for relay or client configs. (If neither is specified the option shows up in both modes). - `Comment{...}` lets the option comments be specified as part of the defineOption. Comment improvements ==================== - Rewrote comments for various options to expand on details. - Inlined all the comments with the option definitions. - Several options that were missing comments got comments added. - Made various options for deprecated and or internal options hidden by default so that they don't show up in a default config file. - show the section comment (but not option comments) *after* the [section] tag instead of before it as it makes more sense that way (particularly for the [bind] section which has a new long comment to describe how it works). Disable profiling by default ============================ We had this weird state where we use and store profiling by default but never *load* it when starting up. This commit makes us just not use profiling at all unless explicitly enabled. Other misc changes: =================== - change default worker threads to 0 (= num cpus) instead of 1, and fix it to allow 0. - Actually apply worker-threads option - fixed default data-dir value erroneously having quotes around it - reordered ifname/ifaddr/mapaddr (was previously mapaddr/ifaddr/ifname) as mapaddr is a sort of specialization of ifaddr and so makes more sense to come after it (particularly because it now references ifaddr in its help message). - removed peer-stats option (since we always require it for relays and never use it for clients) - removed router profiles filename option (this doesn't need to be configurable) - removed defunct `service-node-seed` option - Change default logging output file to "" (which means stdout), and also made "-" work for stdout. * Router hive compilation fixes * Comments for SNApp SRV settings in ini file * Add extra blank line after section comments * Better deprecated option handling Allow {client,relay}-only options in {relay,client} configs to be specified as implicitly deprecated options: they warn, and don't set anything. Add an explicit `Deprecated` tag and move deprecated option handling into definition.cpp. * Move backwards compat options into section definitions Keep the "addBackwardsCompatibleConfigOptions" only for options in sections that no longer exist. * Fix INI parsing issues & C++17-ify - don't allow inline comments because it seems they aren't allowed in ini formats in general, and is going to cause problems if there is a comment character in a value (e.g. an exit auth string). Additionally it was breaking on a line such as: # some comment; see? because it was treating only `; see?` as the comment and then producing an error message about the rest of the line being invalid. - make section parsing stricter: the `[` and `]` have to be at the beginning at end of the line now (after stripping whitespace). - Move whitespace stripping to the top since everything in here does it. - chop off string_view suffix/prefix rather than maintaining position values - fix potential infinite loop/segfault when given a line such as `]foo[` * Make config parsing failure fatal Load() LogError's and returns false on failure, so we weren't aborting on config file errors. * Formatting: allow `{}` for empty functions/structs Instead of using two lines when empty: { } * Make default dns bind 127.0.0.1 on non-Linux * Don't show empty section; fix tests We can conceivably have sections that only make sense for clients or relays, and so want to completely omit that section if we have no options for the type of config being generated. Also fixes missing empty lines between tests. Co-authored-by: Thomas Winget <tewinget@gmail.com>
4 years ago
if (IsServiceNode())
{
LogInfo("Initializing peerdb...");
m_peerDb = std::make_shared<PeerDb>();
m_peerDb->configure(conf.router);
}
// Logging config
LogContext::Instance().Initialize(
conf.logging.m_logLevel,
conf.logging.m_logType,
conf.logging.m_logFile,
conf.router.m_nickname,
util::memFn(&AbstractRouter::QueueDiskIO, this));
6 years ago
return true;
}
bool
Router::CheckRenegotiateValid(RouterContact newrc, RouterContact oldrc)
5 years ago
{
return _rcLookupHandler.CheckRenegotiateValid(newrc, oldrc);
5 years ago
}
bool
Router::IsBootstrapNode(const RouterID r) const
{
return std::count_if(
bootstrapRCList.begin(),
bootstrapRCList.end(),
[r](const RouterContact& rc) -> bool { return rc.pubkey == r; })
> 0;
}
5 years ago
bool
Router::ShouldReportStats(llarp_time_t now) const
{
static constexpr auto ReportStatsInterval = 1h;
return now - m_LastStatsReport > ReportStatsInterval;
}
void
Router::ReportStats()
{
const auto now = Now();
LogInfo(nodedb()->NumLoaded(), " RCs loaded");
LogInfo(bootstrapRCList.size(), " bootstrap peers");
LogInfo(NumberOfConnectedRouters(), " router connections");
if (IsServiceNode())
{
LogInfo(NumberOfConnectedClients(), " client connections");
LogInfo(_rc.Age(now), " since we last updated our RC");
LogInfo(_rc.TimeUntilExpires(now), " until our RC expires");
}
if (m_LastStatsReport > 0s)
LogInfo(now - m_LastStatsReport, " last reported stats");
m_LastStatsReport = now;
}
void
Router::Tick()
{
if (_stopping)
5 years ago
return;
// LogDebug("tick router");
const auto now = Now();
6 years ago
#if defined(WITH_SYSTEMD)
4 years ago
{
std::stringstream ss;
ss << "WATCHDOG=1\nSTATUS=v" << llarp::VERSION_STR;
if (IsServiceNode())
4 years ago
{
ss << " snode | known/svc/clients: " << nodedb()->NumLoaded() << "/"
<< NumberOfConnectedRouters() << "/" << NumberOfConnectedClients() << " | "
<< pathContext().CurrentTransitPaths() << " active paths";
4 years ago
}
else
{
ss << " client | known/connected: " << nodedb()->NumLoaded() << "/"
4 years ago
<< NumberOfConnectedRouters() << " | path success: ";
hiddenServiceContext().ForEachService([&ss](const auto& name, const auto& ep) {
ss << " [" << name << " " << std::setprecision(4)
<< (100.0 * ep->CurrentBuildStats().SuccessRatio()) << "%]";
return true;
});
4 years ago
}
const auto status = ss.str();
::sd_notify(0, status.c_str());
}
#endif
routerProfiling().Tick();
if (ShouldReportStats(now))
{
ReportStats();
}
_rcGossiper.Decay(now);
4 years ago
_rcLookupHandler.PeriodicUpdate(now);
const bool isSvcNode = IsServiceNode();
if (_rc.ExpiresSoon(now, std::chrono::milliseconds(randint() % 10000))
|| (now - _rc.last_updated) > rcRegenInterval)
6 years ago
{
LogInfo("regenerating RC");
if (!UpdateOurRC(false))
LogError("Failed to update our RC");
}
else
{
GossipRCIfNeeded(_rc);
4 years ago
}
const bool gotWhitelist = _rcLookupHandler.HaveReceivedWhitelist();
4 years ago
// remove RCs for nodes that are no longer allowed by network policy
nodedb()->RemoveIf([&](const RouterContact& rc) -> bool {
4 years ago
// don't purge bootstrap nodes from nodedb
if (IsBootstrapNode(rc.pubkey))
4 years ago
return false;
// if for some reason we stored an RC that isn't a valid router
// purge this entry
if (not rc.IsPublicRouter())
4 years ago
return true;
4 years ago
// clients have a notion of a whilelist
// we short circuit logic here so we dont remove
// routers that are not whitelisted for first hops
if (not isSvcNode)
4 years ago
return false;
// if we have a whitelist enabled and we don't
// have the whitelist yet don't remove the entry
if (whitelistRouters and not gotWhitelist)
return false;
// if we have no whitelist enabled or we have
// the whitelist enabled and we got the whitelist
// check against the whitelist and remove if it's not
4 years ago
// in the whitelist OR if there is no whitelist don't remove
4 years ago
return not _rcLookupHandler.RemoteIsAllowed(rc.pubkey);
});
5 years ago
// find all deregistered relays
std::unordered_set<PubKey> closePeers;
_linkManager.ForEachPeer([&](auto session) {
if (whitelistRouters and not gotWhitelist)
return;
if (not session)
return;
const auto pk = session->GetPubKey();
if (session->IsRelay() and not _rcLookupHandler.RemoteIsAllowed(pk))
{
closePeers.emplace(pk);
}
});
// mark peers as de-registered
for (auto& peer : closePeers)
_linkManager.DeregisterPeer(std::move(peer));
_linkManager.CheckPersistingSessions(now);
if (not isSvcNode)
{
if (HasClientExit())
{
m_RoutePoker.Enable();
}
else
{
m_RoutePoker.Disable();
}
m_RoutePoker.Update();
}
size_t connected = NumberOfConnectedRouters();
if (not isSvcNode)
{
connected += _linkManager.NumberOfPendingConnections();
}
const int interval = isSvcNode ? 5 : 2;
4 years ago
const auto timepoint_now = Clock_t::now();
if (timepoint_now >= m_NextExploreAt)
4 years ago
{
_rcLookupHandler.ExploreNetwork();
4 years ago
m_NextExploreAt = timepoint_now + std::chrono::seconds(interval);
}
size_t connectToNum = _outboundSessionMaker.minConnectedRouters;
const auto strictConnect = _rcLookupHandler.NumberOfStrictConnectRouters();
if (strictConnect > 0 && connectToNum > strictConnect)
{
connectToNum = strictConnect;
}
if (connected < connectToNum)
{
size_t dlt = connectToNum - connected;
3 years ago
LogDebug("connecting to ", dlt, " random routers to keep alive");
_outboundSessionMaker.ConnectToRandomRouters(dlt);
}
_hiddenServiceContext.Tick(now);
_exitContext.Tick(now);
// save profiles
if (routerProfiling().ShouldSave(now) and m_Config->network.m_saveProfiles)
5 years ago
{
QueueDiskIO([&]() { routerProfiling().Save(_profilesFile); });
5 years ago
}
_nodedb->Tick(now);
if (m_peerDb)
{
// TODO: throttle this?
// TODO: need to capture session stats when session terminates / is removed from link manager
_linkManager.updatePeerDb(m_peerDb);
if (m_peerDb->shouldFlush(now))
{
LogDebug("Queing database flush...");
QueueDiskIO([this]() { m_peerDb->flushDatabase(); });
}
}
// get connected peers
std::set<dht::Key_t> peersWeHave;
_linkManager.ForEachPeer([&peersWeHave](ILinkSession* s) {
if (!s->IsEstablished())
return;
peersWeHave.emplace(s->GetPubKey());
});
// remove any nodes we don't have connections to
_dht->impl->Nodes()->RemoveIf(
[&peersWeHave](const dht::Key_t& k) -> bool { return peersWeHave.count(k) == 0; });
// expire paths
paths.ExpirePaths(now);
// update tick timestamp
_lastTick = llarp::time_now_ms();
}
bool
Router::Sign(Signature& sig, const llarp_buffer_t& buf) const
{
return CryptoManager::instance()->sign(sig, identity(), buf);
}
void
Router::SessionClosed(RouterID remote)
6 years ago
{
5 years ago
dht::Key_t k(remote);
dht()->impl->Nodes()->DelNode(k);
LogInfo("Session to ", remote, " fully closed");
if (IsServiceNode())
return;
if (const auto maybe = nodedb()->Get(remote); maybe.has_value())
{
for (const auto& addr : maybe->addrs)
m_RoutePoker.DelRoute(addr.toIpAddress().toIP());
}
6 years ago
}
void
Router::ConnectionTimedOut(ILinkSession* session)
{
if (m_peerDb)
{
RouterID id{session->GetPubKey()};
// TODO: make sure this is a public router (on whitelist)?
m_peerDb->modifyPeerStats(id, [&](PeerStats& stats) { stats.numConnectionTimeouts++; });
}
_outboundSessionMaker.OnConnectTimeout(session);
}
void
Router::ModifyOurRC(std::function<std::optional<RouterContact>(RouterContact)> modify)
{
if (auto maybe = modify(rc()))
{
_rc = *maybe;
UpdateOurRC();
_rcGossiper.GossipRC(rc());
}
}
bool
Router::ConnectionEstablished(ILinkSession* session, bool inbound)
{
RouterID id{session->GetPubKey()};
if (m_peerDb)
{
// TODO: make sure this is a public router (on whitelist)?
m_peerDb->modifyPeerStats(id, [&](PeerStats& stats) { stats.numConnectionSuccesses++; });
}
NotifyRouterEvent<tooling::LinkSessionEstablishedEvent>(pubkey(), id, inbound);
return _outboundSessionMaker.OnSessionEstablished(session);
}
bool
Router::GetRandomConnectedRouter(RouterContact& result) const
{
return _linkManager.GetRandomConnectedRouter(result);
}
6 years ago
void
Router::HandleDHTLookupForExplore(RouterID /*remote*/, const std::vector<RouterContact>& results)
6 years ago
{
for (const auto& rc : results)
{
_rcLookupHandler.CheckRC(rc);
}
}
6 years ago
// TODO: refactor callers and remove this function
void
Router::LookupRouter(RouterID remote, RouterLookupHandler resultHandler)
{
_rcLookupHandler.GetRC(
remote,
[=](const RouterID& id, const RouterContact* const rc, const RCRequestResult result) {
(void)id;
if (resultHandler)
{
std::vector<RouterContact> routers;
if (result == RCRequestResult::Success && rc != nullptr)
{
routers.push_back(*rc);
}
resultHandler(routers);
}
});
}
void
Router::SetRouterWhitelist(const std::vector<RouterID> routers)
{
_rcLookupHandler.SetRouterWhitelist(routers);
}
bool
Router::StartRpcServer()
{
if (enableRPCServer)
{
m_RPCServer->AsyncServeRPC(rpcBindAddr);
LogInfo("Bound RPC server to ", rpcBindAddr);
}
return true;
}
bool
Router::Run()
{
if (_running || _stopping)
return false;
// set public signing key
_rc.pubkey = seckey_topublic(identity());
// set router version if service node
if (IsServiceNode())
4 years ago
{
_rc.routerVersion = RouterVersion(llarp::VERSION, LLARP_PROTO_VERSION);
}
5 years ago
_linkManager.ForEachInboundLink([&](LinkLayer_ptr link) {
AddressInfo ai;
if (link->GetOurAddressInfo(ai))
{
// override ip and port
if (_ourAddress)
5 years ago
{
ai.fromSockAddr(*_ourAddress);
5 years ago
}
if (RouterContact::BlockBogons && IsBogon(ai.ip))
return;
LogInfo("adding address: ", ai);
_rc.addrs.push_back(ai);
}
});
if (ExitEnabled() and IsServiceNode())
{
LogError("exit mode not supported while service node");
return false;
}
if (IsServiceNode() and not _rc.IsPublicRouter())
{
LogError("we are configured as relay but have no reachable addresses");
return false;
}
// set public encryption key
_rc.enckey = seckey_topublic(encryption());
5 years ago
LogInfo("Signing rc...");
if (!_rc.Sign(identity()))
{
LogError("failed to sign rc");
return false;
}
if (IsServiceNode())
{
if (!SaveRC())
{
LogError("failed to save RC");
return false;
}
}
_outboundSessionMaker.SetOurRouter(pubkey());
if (!_linkManager.StartLinks())
{
LogWarn("One or more links failed to start.");
return false;
}
if (IsServiceNode())
{
// initialize as service node
if (!InitServiceNode())
{
LogError("Failed to initialize service node");
return false;
}
const RouterID us = pubkey();
LogInfo("initalized service node: ", us);
// init gossiper here
_rcGossiper.Init(&_linkManager, us, this);
// relays do not use profiling
routerProfiling().Disable();
}
else
{
// we are a client
// regenerate keys and resign rc before everything else
CryptoManager::instance()->identity_keygen(_identity);
CryptoManager::instance()->encryption_keygen(_encryption);
_rc.pubkey = seckey_topublic(identity());
_rc.enckey = seckey_topublic(encryption());
if (!_rc.Sign(identity()))
{
LogError("failed to regenerate keys and sign RC");
return false;
}
}
LogInfo("starting hidden service context...");
if (!hiddenServiceContext().StartAll())
{
LogError("Failed to start hidden service context");
return false;
}
{
LogInfo("Loading nodedb from disk...");
_nodedb->LoadFromDisk();
}
llarp_dht_context_start(dht(), pubkey());
for (const auto& rc : bootstrapRCList)
{
nodedb()->Put(rc);
_dht->impl->Nodes()->PutNode(rc);
LogInfo("added bootstrap node ", RouterID{rc.pubkey});
}
LogInfo("have ", _nodedb->NumLoaded(), " routers");
#ifdef _WIN32
// windows uses proactor event loop so we need to constantly pump
_loop->add_ticker([this] { PumpLL(); });
#else
_loop->set_pump_function([this] { PumpLL(); });
#endif
_loop->call_every(ROUTER_TICK_INTERVAL, weak_from_this(), [this] { Tick(); });
_running.store(true);
_startedAt = Now();
#if defined(WITH_SYSTEMD)
::sd_notify(0, "READY=1");
#endif
LogContext::Instance().DropToRuntimeLevel();
return _running;
}
bool
Router::IsRunning() const
{
return _running;
}
llarp_time_t
Router::Uptime() const
{
const llarp_time_t _now = Now();
if (_startedAt > 0s && _now > _startedAt)
return _now - _startedAt;
return 0s;
}
void
Router::AfterStopLinks()
{
Close();
m_lmq.reset();
}
void
Router::AfterStopIssued()
{
StopLinks();
nodedb()->SaveToDisk();
_loop->call_later(200ms, [this] { AfterStopLinks(); });
}
void
Router::StopLinks()
{
_linkManager.Stop();
}
void
Router::Die()
{
if (!_running)
return;
if (_stopping)
return;
_stopping.store(true);
LogContext::Instance().RevertRuntimeLevel();
LogWarn("stopping router hard");
#if defined(WITH_SYSTEMD)
sd_notify(0, "STOPPING=1\nSTATUS=Shutting down HARD");
#endif
hiddenServiceContext().StopAll();
_exitContext.Stop();
StopLinks();
Close();
}
void
Router::Stop()
{
if (!_running)
return;
if (_stopping)
return;
_stopping.store(true);
LogContext::Instance().RevertRuntimeLevel();
LogInfo("stopping router");
4 years ago
#if defined(WITH_SYSTEMD)
sd_notify(0, "STOPPING=1\nSTATUS=Shutting down");
#endif
hiddenServiceContext().StopAll();
_exitContext.Stop();
paths.PumpUpstream();
_linkManager.PumpLinks();
_loop->call_later(200ms, [this] { AfterStopIssued(); });
}
bool
Router::HasSessionTo(const RouterID& remote) const
{
return _linkManager.HasSessionTo(remote);
}
std::string
Router::ShortName() const
{
return RouterID(pubkey()).ToString().substr(0, 8);
}
uint32_t
Router::NextPathBuildNumber()
{
return path_build_count++;
}
void
Router::ConnectToRandomRouters(int _want)
{
const size_t want = _want;
auto connected = NumberOfConnectedRouters();
if (not IsServiceNode())
{
connected += _linkManager.NumberOfPendingConnections();
}
if (connected >= want)
return;
_outboundSessionMaker.ConnectToRandomRouters(want);
}
bool
Router::InitServiceNode()
{
LogInfo("accepting transit traffic");
paths.AllowTransit();
llarp_dht_allow_transit(dht());
_exitContext.AddExitEndpoint("default", m_Config->network, m_Config->dns);
return true;
}
bool
Router::TryConnectAsync(RouterContact rc, uint16_t tries)
{
(void)tries;
if (rc.pubkey == pubkey())
{
return false;
}
if (!_rcLookupHandler.RemoteIsAllowed(rc.pubkey))
{
return false;
}
_outboundSessionMaker.CreateSessionTo(rc, nullptr);
return true;
}
void
Router::QueueWork(std::function<void(void)> func)
{
_loop->call_soon(func);
// m_lmq->job(std::move(func));
}
void
Router::QueueDiskIO(std::function<void(void)> func)
{
m_lmq->job(std::move(func), m_DiskThread);
}
bool
Router::HasClientExit() const
{
if (IsServiceNode())
return false;
const auto ep = hiddenServiceContext().GetDefault();
return ep and ep->HasExit();
}
bool
5 years ago
Router::InitOutboundLinks()
{
auto link = iwp::NewOutboundLink(
m_keyManager,
loop(),
util::memFn(&AbstractRouter::rc, this),
util::memFn(&AbstractRouter::HandleRecvLinkMessageBuffer, this),
util::memFn(&AbstractRouter::Sign, this),
[&](llarp::RouterContact rc) {
if (IsServiceNode())
return;
llarp::LogTrace(
"Before connect, outbound link adding route to (",
rc.addrs[0].toIpAddress().toIP(),
") via gateway.");
m_RoutePoker.AddRoute(rc.addrs[0].toIpAddress().toIP());
},
util::memFn(&Router::ConnectionEstablished, this),
util::memFn(&AbstractRouter::CheckRenegotiateValid, this),
util::memFn(&Router::ConnectionTimedOut, this),
util::memFn(&AbstractRouter::SessionClosed, this),
util::memFn(&AbstractRouter::PumpLL, this),
util::memFn(&AbstractRouter::QueueWork, this));
if (!link)
throw std::runtime_error("NewOutboundLink() failed to provide a link");
Replace libuv with uvw & related refactoring - removes all the llarp_ev_* functions, replacing with methods/classes/functions in the llarp namespace. - banish ev/ev.h to the void - Passes various things by const lvalue ref, especially shared_ptr's that don't need to be copied (to avoid an atomic refcount increment/decrement). - Add a llarp::UDPHandle abstract class for UDP handling - Removes the UDP tick handler; code that needs tick can just do a separate handler on the event loop outside the UDP socket. - Adds an "OwnedBuffer" which owns its own memory but is implicitly convertible to a llarp_buffer_t. This is mostly needed to take over ownership of buffers from uvw without copying them as, currently, uvw does its own allocation (pending some open upstream issues/PRs). - Logic: - add `make_caller`/`call_forever`/`call_every` utility functions to abstract Call wrapping and dependent timed tasks. - Add inLogicThread() so that code can tell its inside the logic thread (typically for debugging assertions). - get rid of janky integer returns and dealing with cancellations on call_later: the other methods added here and the event loop code remove the need for them. - Event loop: - redo everything with uvw instead of libuv - rename EventLoopWakeup::Wakeup to EventLoopWakeup::Trigger to better reflect what it does. - add EventLoopRepeater for repeated events, and replace the code that reschedules itself every time it is called with a repeater. - Split up `EventLoop::run()` into a non-virtual base method and abstract `run_loop()` methods; the base method does a couple extra setup/teardown things that don't need to be in the derived class. - udp_listen is replaced with ev->udp(...) which returns a new UDPHandle object rather that needing gross C-style-but-not-actually-C-compatible structs. - Remove unused register_poll_fd_(un)readable - Use shared_ptr for EventLoopWakeup rather than returning a raw pointer; uvw lets us not have to worry about having the event loop class maintain ownership of it. - Add factory EventLoop::create() function to create a default (uvw-based) event loop (previously this was one of the llarp_ev_blahblah unnamespaced functions). - ev_libuv: this is mostly rewritten; all of the glue code/structs, in particular, are gone as they are no longer needed with uvw. - DNS: - Rename DnsHandler to DnsInterceptor to better describe what it does (this is the code that intercepts all DNS to the tun IP range for Android). - endpoint: - remove unused "isolated network" code - remove distinct (but actually always the same) variables for router/endpoint logic objects - llarp_buffer_t - make constructors type-safe against being called with points to non-size-1 values - tun packet reading: - read all available packets off the device/file descriptor; previously we were reading one packet at a time then returning to the event loop to poll again. - ReadNextPacket() now returns a 0-size packet if the read would block (so that we can implement the previous point). - ReadNextPacket() now throws on I/O error - Miscellaneous code cleanups/simplifications
3 years ago
for (const auto af : {AF_INET, AF_INET6})
{
if (not link->Configure(loop(), "*", af, m_OutboundPort))
5 years ago
continue;
#if defined(ANDROID)
m_OutboundUDPSocket = link->GetUDPFD().value_or(-1);
#endif
_linkManager.AddLink(std::move(link), false);
return true;
}
throw std::runtime_error(
stringify("Failed to init AF_INET and AF_INET6 on port ", m_OutboundPort));
}
void
Router::MessageSent(const RouterID& remote, SendStatus status)
{
if (status == SendStatus::Success)
{
LogDebug("Message successfully sent to ", remote);
}
else
{
LogDebug("Message failed sending to ", remote);
}
}
void
Router::HandleRouterEvent(tooling::RouterEventPtr event) const
{
LogDebug(event->ToString());
}
6 years ago
} // namespace llarp