lokinet/llarp/handlers/tun.cpp

1183 lines
40 KiB
C++
Raw Normal View History

#include "tun.hpp"
2018-10-10 12:06:28 +00:00
#include <algorithm>
#include <iterator>
#include <variant>
2018-09-25 08:31:29 +00:00
#ifndef _WIN32
#include <sys/socket.h>
2018-09-25 08:31:29 +00:00
#endif
2018-08-15 15:36:34 +00:00
#include <llarp/auth/auth.hpp>
#include <llarp/constants/platform.hpp>
#include <llarp/dns/dns.hpp>
QUIC lokinet integration refactor Refactors how quic packets get handled: the actual tunnels now live in tunnel.hpp's TunnelManager which holds and manages all the quic<->tcp tunnelling. service::Endpoint now holds a TunnelManager rather than a quic::Server. We only need one quic server, but we need a separate quic client instance per outgoing quic tunnel, and TunnelManager handles all that glue now. Adds QUIC packet handling to get to the right tunnel code. This required multiplexing incoming quic packets, as follows: Adds a very small quic tunnel packet header of 4 bytes: [1, SPORT, ECN] for client->server packets, where SPORT is our source "port" (really: just a uint16_t unique quic instance identifier) or [2, DPORT, ECN] for server->client packets where the DPORT is the SPORT from above. (This also reworks ECN bits to get properly carried over lokinet.) We don't need a destination/source port for the server-side because there is only ever one quic server (and we know we're going to it when the first byte of the header is 1). Removes the config option for quic exposing ports; a full lokinet will simply accept anything incoming on quic and tunnel it to the requested port on the the local endpoint IP (this handler will come in a following commit). Replace ConvoTags with full addresses: we need to carry the port, as well, which the ConvoTag can't give us, so change those to more general SockAddrs from which we can extract both the ConvoTag *and* the port. Add a pending connection queue along with new quic-side handlers to call when a stream becomes available (TunnelManager uses this to wire up pending incoming conns with quic streams as streams open up). Completely get rid of tunnel_server/tunnel_client.cpp code; it is now moved to tunnel.hpp. Add listen()/forget() methods in TunnelManager for setting up quic listening sockets (for liblokinet usage). Add open()/close() methods in TunnelManager for spinning up new quic clients for outgoing quic connections.
2021-03-23 19:26:32 +00:00
#include <llarp/net/net.hpp>
#include <llarp/nodedb.hpp>
#include <llarp/router/route_poker.hpp>
#include <llarp/router/router.hpp>
#include <llarp/service/name.hpp>
#include <llarp/service/types.hpp>
#include <llarp/util/str.hpp>
2023-10-03 20:00:23 +00:00
namespace llarp::handlers
2018-08-15 15:36:34 +00:00
{
2024-02-01 12:43:43 +00:00
static auto logcat = log::Cat("tun");
bool TunEndpoint::maybe_hook_dns(
2024-02-01 12:43:43 +00:00
std::shared_ptr<dns::PacketSource_Base> source,
const dns::Message& query,
const oxen::quic::Address& to,
const oxen::quic::Address& from)
2023-10-03 20:00:23 +00:00
{
if (not should_hook_dns_message(query))
2024-02-01 12:43:43 +00:00
return false;
2024-02-01 12:43:43 +00:00
auto job = std::make_shared<dns::QueryJob>(source, query, to, from);
// if (HandleHookedDNSMessage(query, [job](auto msg) { job->SendReply(msg.ToBuffer()); }))
// router().TriggerPump();
// else
// job->Cancel();
2024-02-01 12:43:43 +00:00
return true;
2023-10-03 20:00:23 +00:00
}
2024-02-01 12:43:43 +00:00
/// Intercepts DNS IP packets on platforms where binding to a low port isn't viable.
/// (windows/macos/ios/android ... aka everything that is not linux... funny that)
class DnsInterceptor : public dns::PacketSource_Base
{
ip_pkt_hook _hook;
oxen::quic::Address _our_ip; // maybe should be an IP type...?
llarp::DnsConfig _config;
2024-03-17 14:02:46 +00:00
public:
explicit DnsInterceptor(ip_pkt_hook reply, oxen::quic::Address our_ip, llarp::DnsConfig conf)
: _hook{std::move(reply)}, _our_ip{std::move(our_ip)}, _config{std::move(conf)}
2024-02-01 12:43:43 +00:00
{}
2024-02-01 12:43:43 +00:00
~DnsInterceptor() override = default;
void send_to(const oxen::quic::Address& to, const oxen::quic::Address& from, IPPacket data) const override
2024-02-01 12:43:43 +00:00
{
if (data.empty())
2024-02-01 12:43:43 +00:00
return;
2024-03-17 14:02:46 +00:00
// TOFIX: this
(void)to;
(void)from;
(void)data;
// _hook(data.make_udp(to, from));
2024-02-01 12:43:43 +00:00
}
2023-10-03 20:00:23 +00:00
void stop() override{};
std::optional<oxen::quic::Address> bound_on() const override { return std::nullopt; }
2024-02-01 12:43:43 +00:00
bool would_loop(const oxen::quic::Address& to, const oxen::quic::Address& from) const override
2023-10-03 20:00:23 +00:00
{
2024-02-01 12:43:43 +00:00
if constexpr (platform::is_apple)
{
// DNS on Apple is a bit weird because in order for the NetworkExtension itself to
// send data through the tunnel we have to proxy DNS requests through Apple APIs
// (and so our actual upstream DNS won't be set in our resolvers, which is why the
// vanilla WouldLoop won't work for us). However when active the mac also only
// queries the main tunnel IP for DNS, so we consider anything else to be
// upstream-bound DNS to let it through the tunnel.
return to != _our_ip;
2024-02-01 12:43:43 +00:00
}
else if (auto maybe_addr = _config._query_bind)
{
2024-02-01 12:43:43 +00:00
const auto& addr = *maybe_addr;
// omit traffic to and from our dns socket
return addr == to or addr == from;
}
2024-02-01 12:43:43 +00:00
return false;
}
2024-02-01 12:43:43 +00:00
};
2023-10-03 20:00:23 +00:00
2024-02-01 12:43:43 +00:00
class TunDNS : public dns::Server
2023-10-03 20:00:23 +00:00
{
2024-03-17 14:02:46 +00:00
const TunEndpoint* _tun;
std::optional<oxen::quic::Address> _query_bind;
oxen::quic::Address _our_ip;
2019-02-08 19:43:25 +00:00
2024-03-17 14:02:46 +00:00
public:
std::shared_ptr<dns::PacketSource_Base> pkt_source;
2024-02-01 12:43:43 +00:00
~TunDNS() override = default;
2024-02-01 12:43:43 +00:00
explicit TunDNS(TunEndpoint* ep, const llarp::DnsConfig& conf)
: dns::Server{ep->router().loop(), conf, 0},
_tun{ep},
_query_bind{conf._query_bind},
_our_ip{ep->get_if_addr()}
{
if (_query_bind)
_our_ip.set_port(_query_bind->port());
}
std::shared_ptr<dns::PacketSource_Base> make_packet_source_on(
const oxen::quic::Address&, const llarp::DnsConfig& conf) override
2024-02-01 12:43:43 +00:00
{
2024-02-13 16:18:35 +00:00
(void)_tun;
2024-02-01 12:43:43 +00:00
auto ptr = std::make_shared<DnsInterceptor>(
[](IPPacket pkt) {
2024-02-13 16:18:35 +00:00
(void)pkt;
// ep->handle_write_ip_packet(pkt.ConstBuffer(), pkt.srcv6(), pkt.dstv6(), 0);
},
_our_ip,
2024-02-01 12:43:43 +00:00
conf);
pkt_source = ptr;
2024-02-01 12:43:43 +00:00
return ptr;
}
};
TunEndpoint::TunEndpoint(Router& r) : _router{r}
2018-08-16 14:34:15 +00:00
{
auto& keyfile = _router.config()->network.keyfile;
try
{
if (keyfile.has_value())
_identity.ensure_keys(*keyfile, _router.key_manager()->needs_backup());
else
_identity.regenerate_keys();
}
catch (const std::exception& e)
{
auto err = "API endpoint keyfile failed to load: {}"_format(e.what());
log::error(logcat, "{}", err);
throw std::runtime_error{err};
}
_packet_router =
std::make_shared<vpn::PacketRouter>([this](IPPacket pkt) { handle_outbound_packet(std::move(pkt)); });
2023-10-03 20:00:23 +00:00
}
2024-02-01 12:43:43 +00:00
void TunEndpoint::setup_dns()
2023-10-03 20:00:23 +00:00
{
log::debug(logcat, "{} setting up DNS...", name());
auto& dns_config = _router.config()->dns;
const auto& info = get_vpn_interface()->interface_info();
if (dns_config.l3_intercept)
2024-02-01 12:43:43 +00:00
{
auto dns = std::make_shared<TunDNS>(this, dns_config);
_dns = dns;
2024-02-01 12:43:43 +00:00
2024-08-30 21:26:53 +00:00
uint16_t p = 53;
2024-08-30 21:26:53 +00:00
while (p < 100)
{
try
{
_packet_router->add_udp_handler(p, [this, dns](NetworkPacket pkt) {
auto dns_pkt_src = dns->pkt_source;
auto& pkt_path = pkt.path;
if (dns->maybe_handle_packet(
std::move(dns_pkt_src), pkt_path.remote, pkt_path.local, IPPacket::from_netpkt(pkt)))
return;
2024-08-30 21:26:53 +00:00
handle_outbound_packet(IPPacket::from_netpkt(pkt));
});
2024-02-01 12:43:43 +00:00
2024-08-30 21:26:53 +00:00
} catch (const std::exception& e)
{
if (p += 1; p >= 100)
throw std::runtime_error{"Failed to port map udp handler: {}"_format(e.what())};
}
}
2024-02-01 12:43:43 +00:00
}
else
_dns = std::make_shared<dns::Server>(_router.loop(), dns_config, info.index);
_dns->add_resolver(weak_from_this());
_dns->start();
2020-06-02 21:10:42 +00:00
if (dns_config.l3_intercept)
2024-02-01 12:43:43 +00:00
{
if (auto vpn = _router.vpn_platform())
2024-02-01 12:43:43 +00:00
{
// get the first local address we know of
std::optional<oxen::quic::Address> localaddr;
for (auto res : _dns->get_all_resolvers())
2024-02-01 12:43:43 +00:00
{
if (auto ptr = res.lock())
{
localaddr = ptr->get_local_addr();
2024-02-01 12:43:43 +00:00
if (localaddr)
break;
}
}
if (platform::is_windows)
{
// auto dns_io = vpn->create_packet_io(0, localaddr);
// router().loop()->add_ticker([dns_io, handler = m_PacketRouter]() {
// net::IPPacket pkt = dns_io->ReadNextPacket();
// while (not pkt.empty())
// {
// handler->HandleIPPacket(std::move(pkt));
// pkt = dns_io->ReadNextPacket();
// }
// });
// m_RawDNS = dns_io;
2024-02-01 12:43:43 +00:00
}
(void)vpn;
2024-02-01 12:43:43 +00:00
}
2021-04-14 19:40:57 +00:00
if (_raw_DNS)
_raw_DNS->Start();
2024-02-01 12:43:43 +00:00
}
2023-10-03 20:00:23 +00:00
}
nlohmann::json TunEndpoint::ExtractStatus() const
2023-10-03 20:00:23 +00:00
{
// auto obj = service::Endpoint::ExtractStatus();
// obj["ifaddr"] = m_OurRange.to_string();
// obj["ifname"] = m_IfName;
// std::vector<std::string> upstreamRes;
// for (const auto& ent : m_DnsConfig.upstream_dns)
// upstreamRes.emplace_back(ent.to_string());
// obj["ustreamResolvers"] = upstreamRes;
// std::vector<std::string> localRes;
// for (const auto& ent : m_DnsConfig.bind_addr)
// localRes.emplace_back(ent.to_string());
// obj["localResolvers"] = localRes;
// // for backwards compat
// if (not m_DnsConfig.bind_addr.empty())
// obj["localResolver"] = localRes[0];
// nlohmann::json ips{};
// for (const auto& item : m_IPActivity)
// {
// nlohmann::json ipObj{{"lastActive", to_json(item.second)}};
// std::string remoteStr;
// AlignedBuffer<32> addr = m_IPToAddr.at(item.first);
// if (m_SNodes.at(addr))
// remoteStr = RouterID(addr.as_array()).to_string();
// else
// remoteStr = service::Address(addr.as_array()).to_string();
// ipObj["remote"] = remoteStr;
// std::string ipaddr = item.first.to_string();
// ips[ipaddr] = ipObj;
// }
// obj["addrs"] = ips;
// obj["ourIP"] = m_OurIP.to_string();
// obj["nextIP"] = m_NextIP.to_string();
// obj["maxIP"] = m_MaxIP.to_string();
// return obj;
return {};
2023-10-03 20:00:23 +00:00
}
void TunEndpoint::reconfigure_dns(std::vector<oxen::quic::Address> servers)
2023-10-03 20:00:23 +00:00
{
if (_dns)
2024-02-01 12:43:43 +00:00
{
for (auto weak : _dns->get_all_resolvers())
2024-02-01 12:43:43 +00:00
{
if (auto ptr = weak.lock())
ptr->reset_resolver(servers);
2024-02-01 12:43:43 +00:00
}
}
2023-10-03 20:00:23 +00:00
}
2021-02-16 15:59:18 +00:00
void TunEndpoint::configure()
2023-10-03 20:00:23 +00:00
{
log::debug(logcat, "{} called", __PRETTY_FUNCTION__);
auto& net_conf = _router.config()->network;
2024-02-01 12:43:43 +00:00
/** DISCUSS: Can the auth objects be further simplified?
- In the original implementation, the AuthPolicy async logic was for the instance receiving the connection
request to execute its aynchronous logic and queue the authentication job
Static Token Auth:
- In the re-designed auth paradigm, static tokens are either independantly coordinated with the exit/service
operator
- The session initiator will automatically include any static tokens that are either (A) loaded into the
config mapping or (B) passed to the lokinet-vpn cli utility
- As a result, the session initiator doesn't necessarily need an AuthPolicy object
RPC Auth:
- Why can't the functionality of this be entirely subsumed by the RPCClient?
- If the config specifies the auth_type as RPC plus
*/
// switch (net_conf.auth_type)
// {
// case auth::AuthType::WHITELIST:
// case auth::AuthType::OMQ:
// // The RPCAuthPolicy constructor will throw if auth_{endpoint,method} are empty
// _auth_policy = auth::make_auth_policy<auth::RPCAuthPolicy>(
// router(), *net_conf.auth_endpoint, *net_conf.auth_method, router().lmq(), shared_from_this());
// std::static_pointer_cast<auth::RPCAuthPolicy>(_auth_policy)->start();
// break;
// case auth::AuthType::FILE:
// _auth_policy = auth::make_auth_policy<auth::FileAuthPolicy>(
// router(), net_conf.auth_files, net_conf.auth_file_type);
// break;
// case auth::AuthType::NONE:
// default:
// break;
// }
_traffic_policy = net_conf.traffic_policy;
_base_ipv6_range = net_conf._base_ipv6_range;
if (net_conf.path_alignment_timeout)
2024-02-01 12:43:43 +00:00
{
if (is_service_node())
throw std::runtime_error{"Service nodes cannot specify path alignment timeout!"};
2024-02-01 12:43:43 +00:00
_path_alignment_timeout = *net_conf.path_alignment_timeout;
2024-02-01 12:43:43 +00:00
}
_if_name = *net_conf._if_name;
_local_range = *net_conf._local_ip_range;
_local_addr = *net_conf._local_addr;
_local_base_ip = *net_conf._local_base_ip;
2024-02-01 12:43:43 +00:00
ipv6_enabled = not _local_range.is_ipv4();
if (ipv6_enabled and not net_conf.enable_ipv6)
throw std::runtime_error{"Config must explicitly enable IPv6 to use local range: {}"_format(_local_range)};
_persisting_addr_file = net_conf.addr_map_persist_file;
if (not net_conf._reserved_local_ips.empty())
2024-02-01 12:43:43 +00:00
{
for (auto& [remote, local] : net_conf._reserved_local_ips)
2023-10-03 20:00:23 +00:00
{
_local_ip_mapping.insert_or_assign(local, remote);
2023-10-03 20:00:23 +00:00
}
}
2024-08-28 21:31:39 +00:00
_local_netaddr = NetworkAddress::from_pubkey(_router.local_rid(), not _router.is_service_node());
_local_ip_mapping.insert_or_assign(_local_base_ip, std::move(_local_netaddr));
vpn::InterfaceInfo info;
info.ifname = _if_name;
info.if_info = net_conf._if_info;
info.addrs.emplace_back(_local_range);
if (net_conf.enable_ipv6 and _base_ipv6_range)
{
log::info(logcat, "{} using ipv6 range:{}", name(), *_base_ipv6_range);
info.addrs.emplace_back(*_base_ipv6_range);
}
log::debug(logcat, "{} setting up network...", name());
_net_if = router().vpn_platform()->CreateInterface(std::move(info), &_router);
_if_name = _net_if->interface_info().ifname;
log::info(logcat, "{} got network interface:{}", name(), _if_name);
auto handle_packet = [netif = _net_if, pktrouter = _packet_router](IPPacket pkt) {
// TODO: packets used to have reply hooks
// pkt.reply = [netif](auto pkt) { netif->WritePacket(std::move(pkt)); };
pktrouter->handle_ip_packet(std::move(pkt));
};
if (not router().loop()->add_network_interface(_net_if, std::move(handle_packet)))
{
auto err = "{} failed to add network interface!"_format(name());
log::error(logcat, "{}", err);
throw std::runtime_error{std::move(err)};
}
_local_ipv6 = ipv6_enabled ? _local_addr : _local_addr.mapped_ipv4_as_ipv6();
if (ipv6_enabled)
{
if constexpr (not llarp::platform::is_apple)
{
if (auto maybe = router().net().get_interface_ipv6_addr(_if_name))
{
_local_ipv6 = *maybe;
}
}
}
2024-08-28 21:31:39 +00:00
log::info(
logcat, "{} has interface ipv4 address ({}) with ipv6 address ({})", name(), _local_addr, _local_ipv6);
2024-02-01 12:43:43 +00:00
// if (auto* quic = GetQUICTunnel())
// {
// TODO:
// quic->listen([this](std::string_view, uint16_t port) {
// return llarp::SockAddr{net::TruncateV6(GetIfAddr()), huint16_t{port}};
// });
// }
setup_dns();
2024-02-01 12:43:43 +00:00
}
2018-12-15 16:56:35 +00:00
2024-02-01 12:43:43 +00:00
static bool is_random_snode(const dns::Message& msg)
{
2024-02-01 12:43:43 +00:00
return msg.questions[0].IsName("random.snode");
2023-10-03 20:00:23 +00:00
}
2024-02-01 12:43:43 +00:00
static bool is_localhost_loki(const dns::Message& msg)
{
2024-02-01 12:43:43 +00:00
return msg.questions[0].IsLocalhost();
}
2024-02-01 12:43:43 +00:00
static dns::Message& clear_dns_message(dns::Message& msg)
{
2024-02-01 12:43:43 +00:00
msg.authorities.resize(0);
msg.additional.resize(0);
msg.answers.resize(0);
msg.hdr_fields &= ~dns::flags_RCODENameError;
return msg;
}
2024-02-01 12:43:43 +00:00
bool TunEndpoint::handle_hooked_dns_message(dns::Message msg, std::function<void(dns::Message)> reply)
2024-02-01 12:43:43 +00:00
{
(void)msg;
(void)reply;
// auto ReplyToSNodeDNSWhenReady = [this, reply](RouterID snode, auto msg, bool isV6) ->
// bool {
// return EnsurePathToSNode(
// snode,
// [this, snode, msg, reply, isV6](
// const RouterID&,
// std::shared_ptr<session::BaseSession> s,
// [[maybe_unused]] service::SessionTag tag) {
// SendDNSReply(snode, s, msg, reply, isV6);
// });
// };
// auto ReplyToLokiDNSWhenReady = [this, reply, timeout = PathAlignmentTimeout()](
// service::Address addr, auto msg, bool isV6) -> bool {
// using service::Address;
// using service::OutboundContext;
// if (HasInboundConvo(addr))
// {
// // if we have an inbound convo to this address don't mark as outbound so we don't
// have a
// // state race this codepath is hit when an application verifies that reverse and
// forward
// // dns records match for an inbound session
// SendDNSReply(addr, this, msg, reply, isV6);
// return true;
// }
// MarkAddressOutbound(addr);
// return EnsurePathToService(
// addr,
// [this, addr, msg, reply, isV6](const Address&, OutboundContext* ctx) {
// SendDNSReply(addr, ctx, msg, reply, isV6);
// },
// timeout);
// };
// auto ReplyToDNSWhenReady = [ReplyToLokiDNSWhenReady, ReplyToSNodeDNSWhenReady](
// std::string name, auto msg, bool isV6) {
// if (auto saddr = service::Address(); saddr.FromString(name))
// ReplyToLokiDNSWhenReady(saddr, msg, isV6);
// if (auto rid = RouterID(); rid.from_snode_address(name))
// ReplyToSNodeDNSWhenReady(rid, msg, isV6);
// };
// auto ReplyToLokiSRVWhenReady = [this, reply, timeout = PathAlignmentTimeout()](
// service::Address addr, auto msg) -> bool {
// using service::Address;
// using service::OutboundContext;
// // TODO: how do we handle SRV record lookups for inbound sessions?
// MarkAddressOutbound(addr);
// return EnsurePathToService(
// addr,
// [msg, addr, reply](const Address&, OutboundContext* ctx) {
// if (ctx == nullptr)
// return;
// const auto& introset = ctx->GetCurrentIntroSet();
// msg->AddSRVReply(introset.GetMatchingSRVRecords(addr.subdomain));
// reply(*msg);
// },
// timeout);
// };
// if (msg.answers.size() > 0)
// {
// const auto& answer = msg.answers[0];
// if (answer.HasCNameForTLD(".snode"))
// {
// llarp_buffer_t buf(answer.rData);
// auto qname = dns::DecodeName(&buf, true);
// if (not qname)
// return false;
// RouterID addr;
// if (not addr.from_snode_address(*qname))
// return false;
// auto replyMsg = std::make_shared<dns::Message>(clear_dns_message(msg));
// return ReplyToSNodeDNSWhenReady(addr, std::move(replyMsg), false);
// }
// else if (answer.HasCNameForTLD(".loki"))
// {
// llarp_buffer_t buf(answer.rData);
// auto qname = dns::DecodeName(&buf, true);
// if (not qname)
// return false;
// service::Address addr;
// if (not addr.FromString(*qname))
// return false;
// auto replyMsg = std::make_shared<dns::Message>(clear_dns_message(msg));
// return ReplyToLokiDNSWhenReady(addr, replyMsg, false);
// }
// }
// if (msg.questions.size() != 1)
// {
2024-03-17 16:11:46 +00:00
// log::warning(logcat, "bad number of dns questions: {}", msg.questions.size());
// return false;
// }
// std::string qname = msg.questions[0].Name();
// const auto nameparts = split(qname, ".");
// std::string ons_name;
// if (nameparts.size() >= 2 and ends_with(qname, ".loki"))
// {
// ons_name = nameparts[nameparts.size() - 2];
// ons_name += ".loki"sv;
// }
// if (msg.questions[0].qtype == dns::qTypeTXT)
// {
// RouterID snode;
// if (snode.from_snode_address(qname))
// {
// if (auto rc = router().node_db()->get_rc(snode))
// msg.AddTXTReply(std::string{rc->view()});
// else
// msg.AddNXReply();
// reply(msg);
// return true;
// }
// if (msg.questions[0].IsLocalhost() and msg.questions[0].HasSubdomains())
// {
// const auto subdomain = msg.questions[0].Subdomains();
// if (subdomain == "exit")
// {
// if (HasExit())
// {
// std::string s;
// _exit_map.ForEachEntry([&s](const auto& range, const auto& exit) {
// fmt::format_to(std::back_inserter(s), "{}={}; ", range, exit);
// });
// msg.AddTXTReply(std::move(s));
// }
// else
// {
// msg.AddNXReply();
// }
// }
// else if (subdomain == "netid")
// {
// msg.AddTXTReply(fmt::format("netid={};", RouterContact::ACTIVE_NETID));
// }
// else
// {
// msg.AddNXReply();
// }
// }
// else
// {
// msg.AddNXReply();
// }
// reply(msg);
// }
// else if (msg.questions[0].qtype == dns::qTypeMX)
// {
// // mx record
// service::Address addr;
// if (addr.FromString(qname, ".loki") || addr.FromString(qname, ".snode")
// || is_random_snode(msg) || is_localhost_loki(msg))
// {
// msg.AddMXReply(qname, 1);
// }
// else if (service::is_valid_name(ons_name))
// {
// lookup_name(
// ons_name, [msg, ons_name, reply](std::string name_result, bool success) mutable {
// if (success)
// {
// msg.AddMXReply(name_result, 1);
// }
// else
// msg.AddNXReply();
// reply(msg);
// });
// return true;
// }
// else
// msg.AddNXReply();
// reply(msg);
// }
// else if (msg.questions[0].qtype == dns::qTypeCNAME)
// {
// if (is_random_snode(msg))
// {
// if (auto random = router().GetRandomGoodRouter())
// {
// msg.AddCNAMEReply(random->to_string(), 1);
// }
// else
// msg.AddNXReply();
// }
// else if (msg.questions[0].IsLocalhost() and msg.questions[0].HasSubdomains())
// {
// const auto subdomain = msg.questions[0].Subdomains();
// if (subdomain == "exit" and HasExit())
// {
// _exit_map.ForEachEntry(
// [&msg](const auto&, const auto& exit) { msg.AddCNAMEReply(exit.to_string(), 1);
// });
// }
// else
// {
// msg.AddNXReply();
// }
// }
// else if (is_localhost_loki(msg))
// {
// size_t counter = 0;
// context->ForEachService(
// [&](const std::string&, const std::shared_ptr<service::Endpoint>& service) ->
// bool {
// const service::Address addr = service->GetIdentity().pub.Addr();
// msg.AddCNAMEReply(addr.to_string(), 1);
// ++counter;
// return true;
// });
// if (counter == 0)
// msg.AddNXReply();
// }
// else
// msg.AddNXReply();
// reply(msg);
// }
// else if (msg.questions[0].qtype == dns::qTypeA || msg.questions[0].qtype ==
// dns::qTypeAAAA)
// {
// const bool isV6 = msg.questions[0].qtype == dns::qTypeAAAA;
// const bool isV4 = msg.questions[0].qtype == dns::qTypeA;
// llarp::service::Address addr;
// if (isV6 && !SupportsV6())
// { // empty reply but not a NXDOMAIN so that client can retry IPv4
// msg.AddNSReply("localhost.loki.");
// }
// // on MacOS this is a typeA query
// else if (is_random_snode(msg))
// {
// if (auto random = router().GetRandomGoodRouter())
// {
// msg.AddCNAMEReply(random->to_string(), 1);
// return ReplyToSNodeDNSWhenReady(*random, std::make_shared<dns::Message>(msg),
// isV6);
// }
// msg.AddNXReply();
// }
// else if (is_localhost_loki(msg))
// {
// const bool lookingForExit = msg.questions[0].Subdomains() == "exit";
// huint128_t ip = GetIfAddr();
// if (ip.h)
// {
// if (lookingForExit)
// {
// if (HasExit())
// {
// _exit_map.ForEachEntry(
// [&msg](const auto&, const auto& exit) { msg.AddCNAMEReply(exit.to_string());
// });
// msg.AddINReply(ip, isV6);
// }
// else
// {
// msg.AddNXReply();
// }
// }
// else
// {
// msg.AddCNAMEReply(_identity.pub.Name(), 1);
// msg.AddINReply(ip, isV6);
// }
// }
// else
// {
// msg.AddNXReply();
// }
// }
// else if (addr.FromString(qname, ".loki"))
// {
// if (isV4 && SupportsV6())
// {
// msg.hdr_fields |= dns::flags_QR | dns::flags_AA | dns::flags_RA;
// }
// else
// {
// return ReplyToLokiDNSWhenReady(addr, std::make_shared<dns::Message>(msg), isV6);
// }
// }
// else if (addr.FromString(qname, ".snode"))
// {
// if (isV4 && SupportsV6())
// {
// msg.hdr_fields |= dns::flags_QR | dns::flags_AA | dns::flags_RA;
// }
// else
// {
// return ReplyToSNodeDNSWhenReady(
// addr.as_array(), std::make_shared<dns::Message>(msg), isV6);
// }
// }
// else if (service::is_valid_name(ons_name))
// {
// lookup_name(
// ons_name,
// [msg = std::make_shared<dns::Message>(msg),
// name = Name(),
// ons_name,
// isV6,
// reply,
// ReplyToDNSWhenReady](std::string name_result, bool success) mutable {
// if (not success)
// {
// log::warning(logcat, "{} (ONS name: {}) not resolved", name, ons_name);
// msg->AddNXReply();
// reply(*msg);
// }
// ReplyToDNSWhenReady(name_result, msg, isV6);
// });
// return true;
// }
// else
// msg.AddNXReply();
// reply(msg);
// }
// else if (msg.questions[0].qtype == dns::qTypePTR)
// {
// // reverse dns
// if (auto ip = dns::DecodePTR(msg.questions[0].qname))
// {
// if (auto maybe = ObtainAddrForIP(*ip))
// {
// var::visit([&msg](auto&& result) { msg.AddAReply(result.to_string()); }, *maybe);
// reply(msg);
// return true;
// }
// }
// msg.AddNXReply();
// reply(msg);
// return true;
// }
// else if (msg.questions[0].qtype == dns::qTypeSRV)
// {
// auto srv_for = msg.questions[0].Subdomains();
// auto name = msg.questions[0].qname;
// if (is_localhost_loki(msg))
// {
// msg.AddSRVReply(intro_set().GetMatchingSRVRecords(srv_for));
// reply(msg);
// return true;
// }
// LookupServiceAsync(
// name,
// srv_for,
// [reply, msg = std::make_shared<dns::Message>(std::move(msg))](auto records) {
// if (records.empty())
// {
// msg->AddNXReply();
// }
// else
// {
// msg->AddSRVReply(records);
// }
// reply(*msg);
// });
// return true;
// }
// else
// {
// msg.AddNXReply();
// reply(msg);
// }
2022-04-16 16:41:34 +00:00
return true;
2019-05-07 17:46:38 +00:00
}
2023-10-03 20:00:23 +00:00
bool TunEndpoint::supports_ipv6() const
2023-10-03 20:00:23 +00:00
{
return ipv6_enabled;
2018-12-03 22:22:59 +00:00
}
2024-02-01 12:43:43 +00:00
// FIXME: pass in which question it should be addressing
bool TunEndpoint::should_hook_dns_message(const dns::Message& msg) const
2018-08-22 15:52:10 +00:00
{
// llarp::service::Address addr;
2024-02-01 12:43:43 +00:00
if (msg.questions.size() == 1)
{
/// hook every .loki
if (msg.questions[0].HasTLD(".loki"))
return true;
/// hook every .snode
if (msg.questions[0].HasTLD(".snode"))
return true;
// hook any ranges we own
if (msg.questions[0].qtype == llarp::dns::qTypePTR)
{
if (auto ip = dns::DecodePTR(msg.questions[0].qname))
2024-02-13 16:18:35 +00:00
return _local_range.contains(*ip);
2024-02-01 12:43:43 +00:00
return false;
}
}
for (const auto& answer : msg.answers)
{
if (answer.HasCNameForTLD(".loki"))
return true;
if (answer.HasCNameForTLD(".snode"))
return true;
}
2018-08-22 15:52:10 +00:00
return false;
2023-10-03 20:00:23 +00:00
}
2024-02-01 12:43:43 +00:00
std::string TunEndpoint::get_if_name() const
2023-10-03 20:00:23 +00:00
{
return _if_name;
2020-08-21 15:07:37 +00:00
}
bool TunEndpoint::is_service_node() const
2023-10-03 20:00:23 +00:00
{
return _router.is_service_node();
}
bool TunEndpoint::is_exit_node() const
{
return _router.is_exit_node();
2023-10-03 20:00:23 +00:00
}
2024-02-01 12:43:43 +00:00
bool TunEndpoint::stop()
{
2024-02-01 12:43:43 +00:00
// stop vpn tunnel
if (_net_if)
_net_if->Stop();
if (_raw_DNS)
_raw_DNS->Stop();
2024-02-13 16:18:35 +00:00
2024-02-01 12:43:43 +00:00
// save address map if applicable
if (_persisting_addr_file and not platform::is_android)
2024-02-01 12:43:43 +00:00
{
const auto& file = *_persisting_addr_file;
log::debug(logcat, "{} saving address map to {}", name(), file);
// if (auto maybe = util::OpenFileStream<fs::ofstream>(file, std::ios_base::binary))
// {
// std::map<std::string, std::string> addrmap;
// for (const auto& [ip, addr] : m_IPToAddr)
// {
// if (not m_SNodes.at(addr))
// {
// const service::Address a{addr.as_array()};
// if (HasInboundConvo(a))
// addrmap[ip.to_string()] = a.to_string();
// }
// }
// const auto data = oxenc::bt_serialize(addrmap);
// maybe->write(data.data(), data.size());
// }
2024-02-01 12:43:43 +00:00
}
2024-02-13 16:18:35 +00:00
if (_dns)
_dns->stop();
2024-02-13 16:18:35 +00:00
return true;
}
std::optional<ip_v> TunEndpoint::get_next_local_ip()
{
// if our IP range is exhausted, we loop back around to see if any have been unmapped from terminated sessions;
// we only want to reset the iterator and loop back through once though
bool has_reset = false;
do
{
// this will be std::nullopt if IP range is exhausted OR the IP incrementing overflowed (basically equal)
if (auto maybe_next_ip = _local_range_iterator.next_ip(); maybe_next_ip)
{
if (not _local_ip_mapping.has_local(*maybe_next_ip))
return maybe_next_ip;
// local IP is already assigned; try again
continue;
}
if (not has_reset)
{
log::debug(logcat, "Resetting IP range iterator for range: {}...", _local_range);
_local_range_iterator.reset();
has_reset = true;
}
else
break;
} while (true);
return std::nullopt;
}
std::optional<ip_v> TunEndpoint::map_session_to_local_ip(const NetworkAddress& remote)
{
std::optional<ip_v> ret = std::nullopt;
// first: check if we have a config value for this remote
if (auto maybe_ip = _local_ip_mapping.get_local_from_remote(remote); maybe_ip)
{
ret = maybe_ip;
log::debug(
logcat,
"Local IP for session to remote ({}) pre-loaded from config: {}",
remote,
std::holds_alternative<ipv4>(*maybe_ip) ? std::get<ipv4>(*maybe_ip).to_string()
: std::get<ipv6>(*maybe_ip).to_string());
}
else
{
// We need to check that we both have a valid IP in our local range and that it is not already pre-assigned
// to a remote from the config
if (auto maybe_next_ip = get_next_local_ip(); maybe_next_ip)
{
ret = maybe_next_ip;
_local_ip_mapping.insert_or_assign(*maybe_next_ip, remote);
log::debug(
logcat,
"Local IP for session to remote ({}) assigned: {}",
remote,
std::holds_alternative<ipv4>(*maybe_next_ip) ? std::get<ipv4>(*maybe_next_ip).to_string()
: std::get<ipv6>(*maybe_next_ip).to_string());
}
else
log::critical(logcat, "TUN device failed to assign local private IP for session to remote: {}", remote);
}
return ret;
}
void TunEndpoint::unmap_session_to_local_ip(const NetworkAddress& remote)
{
if (_local_ip_mapping.has_remote(remote))
{
_local_ip_mapping.unmap(remote);
log::debug(logcat, "TUN device unmapped session to remote: {}", remote);
}
else
{
log::debug(logcat, "TUN device could not unmap session (remote: {})", remote);
}
}
void TunEndpoint::handle_outbound_packet(IPPacket pkt)
2024-02-13 16:18:35 +00:00
{
ip_v src, dest;
auto pkt_is_ipv4 = pkt.is_ipv4();
if (pkt_is_ipv4)
{
src = pkt.source_ipv4();
dest = pkt.dest_ipv4();
}
else
{
src = pkt.source_ipv6();
dest = pkt.dest_ipv6();
}
if constexpr (llarp::platform::is_apple)
{
if (ip_equals_address(dest, _local_addr, pkt_is_ipv4))
{
rewrite_and_send_packet(std::move(pkt), src, dest);
return;
}
}
2023-10-12 20:37:45 +00:00
// we pass `dest` because that is our local private IP on the outgoing IPPacket
if (auto maybe_remote = _local_ip_mapping.get_remote_from_local(dest))
{
auto& remote = *maybe_remote;
pkt.clear_addresses();
2023-10-12 20:37:45 +00:00
if (auto session = _router.session_endpoint()->get_session(remote))
{
log::debug(logcat, "Dispatching outbound packet for session (remote: {})", remote);
session->send_path_data_message(std::move(pkt).steal_payload());
return;
}
throw std::runtime_error{"Could not find session (remote: {}) for outbound packet!"_format(remote)};
}
throw std::runtime_error{"Could not find remote mapped to local ip: {}"_format(dest)};
}
2023-10-12 20:37:45 +00:00
bool TunEndpoint::obtain_src_for_remote(const NetworkAddress& remote, ip_v& src, bool use_ipv4)
{
// we are receiving traffic from a session to a local exit node
if (auto maybe_src = _local_ip_mapping.get_local_from_remote(remote))
{
if (std::holds_alternative<ipv4>(*maybe_src))
{
if (use_ipv4)
src = *maybe_src;
else
{
auto quicaddr = oxen::quic::Address{std::get<ipv4>(*maybe_src)};
src = quicaddr.to_ipv6();
}
}
else
{
if (use_ipv4)
{
auto quicaddr = oxen::quic::Address{std::get<ipv6>(*maybe_src)};
src = quicaddr.to_ipv4();
}
else
src = *maybe_src;
}
}
else
{
log::critical(logcat, "Unable to find local IP for inbound packet from remote: {}", remote);
return false;
}
return true;
2021-04-14 15:07:06 +00:00
}
void TunEndpoint::send_packet_to_net_if(IPPacket&& pkt)
2020-05-21 14:18:23 +00:00
{
_router.loop()->call([this, pkt = std::move(pkt)]() { _net_if->WritePacket(std::move(pkt)); });
}
void TunEndpoint::rewrite_and_send_packet(IPPacket&& pkt, ip_v src, ip_v dest)
{
if (pkt.is_ipv4())
pkt.update_ipv4_address(std::get<ipv4>(src), std::get<ipv4>(dest));
else
pkt.update_ipv6_address(std::get<ipv6>(src), std::get<ipv6>(dest));
send_packet_to_net_if(std::move(pkt));
2023-10-03 20:00:23 +00:00
}
2020-05-21 14:18:23 +00:00
bool TunEndpoint::handle_inbound_packet(
IPPacket pkt, NetworkAddress remote, bool is_exit_session, bool is_outbound_session)
2023-10-03 20:00:23 +00:00
{
ip_v src, dest;
auto pkt_is_ipv4 = pkt.is_ipv4();
if (is_exit_session and is_outbound_session)
2024-02-01 12:43:43 +00:00
{
// we are receiving traffic from a session to a remote exit node
if (pkt_is_ipv4)
{
src = pkt.source_ipv4();
dest = _local_addr.to_ipv4();
}
else
{
src = pkt.source_ipv6();
dest = _local_ipv6.to_ipv6();
}
assert(remote.is_client());
auto maybe_remote = _local_ip_mapping.get_remote_from_local(src);
if (not maybe_remote)
{
log::critical(
logcat, "Could not find mapping of local IP (ip:{}) for session to remote: {}", src, remote);
return false;
}
if (*maybe_remote != remote)
{
log::critical(
logcat,
"Internal mapping of local IP (ip:{}, remote:{}) did not match inbound packet from remote: {}",
src,
*maybe_remote,
remote);
return false;
}
}
else
{
if (is_exit_session and not is_outbound_session)
{
// we are receiving traffic from a session to a local exit node
if (not is_allowing_traffic(pkt))
return false;
if (pkt_is_ipv4)
dest = pkt.dest_ipv4();
else
dest = pkt.dest_ipv6();
}
else
{
// we are receiving hidden service traffic
if (pkt_is_ipv4)
dest = _local_addr.to_ipv4();
else
dest = _local_ipv6.to_ipv6();
}
if (not obtain_src_for_remote(remote, src, pkt_is_ipv4))
return false;
2024-02-01 12:43:43 +00:00
}
rewrite_and_send_packet(std::move(pkt), src, dest);
return true;
}
bool TunEndpoint::is_allowing_traffic(const IPPacket& pkt) const
{
if (auto exitPolicy = get_traffic_policy())
return exitPolicy->allow_ip_traffic(pkt);
2024-02-01 12:43:43 +00:00
return true;
2023-10-03 20:00:23 +00:00
}
2024-02-01 12:43:43 +00:00
bool TunEndpoint::has_mapping_to_remote(const NetworkAddress& addr) const
{
return _local_ip_mapping.has_remote(addr);
}
std::optional<ip_v> TunEndpoint::get_mapped_ip(const NetworkAddress& addr)
2023-10-03 20:00:23 +00:00
{
return _local_ip_mapping.get_local_from_remote(addr);
2023-10-03 20:00:23 +00:00
}
oxen::quic::Address TunEndpoint::get_if_addr() const
2023-10-03 20:00:23 +00:00
{
return _local_addr;
2023-10-03 20:00:23 +00:00
}
2024-02-01 12:43:43 +00:00
TunEndpoint::~TunEndpoint() = default;
2018-08-15 15:36:34 +00:00
2023-10-03 20:00:23 +00:00
} // namespace llarp::handlers