add support for uPnP and PMP nat traversal
This commit is contained in:
parent
4b99b58645
commit
348e61364e
|
@ -55,13 +55,6 @@ when isMainModule:
|
||||||
config.setupLogging()
|
config.setupLogging()
|
||||||
config.setupMetrics()
|
config.setupMetrics()
|
||||||
|
|
||||||
if config.nat == ValidIpAddress.init(IPv4_any()):
|
|
||||||
error "`--nat` cannot be set to the any (`0.0.0.0`) address"
|
|
||||||
quit QuitFailure
|
|
||||||
|
|
||||||
if config.nat == ValidIpAddress.init("127.0.0.1"):
|
|
||||||
warn "`--nat` is set to loopback, your node wont properly announce over the DHT"
|
|
||||||
|
|
||||||
if not(checkAndCreateDataDir((config.dataDir).string)):
|
if not(checkAndCreateDataDir((config.dataDir).string)):
|
||||||
# We are unable to access/create data folder or data folder's
|
# We are unable to access/create data folder or data folder's
|
||||||
# permissions are insecure.
|
# permissions are insecure.
|
||||||
|
|
|
@ -44,6 +44,7 @@ import ./utils/addrutils
|
||||||
import ./namespaces
|
import ./namespaces
|
||||||
import ./codextypes
|
import ./codextypes
|
||||||
import ./logutils
|
import ./logutils
|
||||||
|
import ./nat
|
||||||
|
|
||||||
logScope:
|
logScope:
|
||||||
topics = "codex node"
|
topics = "codex node"
|
||||||
|
@ -158,30 +159,13 @@ proc start*(s: CodexServer) {.async.} =
|
||||||
|
|
||||||
await s.codexNode.switch.start()
|
await s.codexNode.switch.start()
|
||||||
|
|
||||||
let
|
let (announceAddrs,discoveryAddrs)= natedAddress(
|
||||||
# TODO: Can't define these as constants, pity
|
s.config.nat,
|
||||||
natIpPart = MultiAddress.init("/ip4/" & $s.config.nat & "/")
|
s.config.listenAddrs,
|
||||||
.expect("Should create multiaddress")
|
s.config.discoveryPort)
|
||||||
anyAddrIp = MultiAddress.init("/ip4/0.0.0.0/")
|
|
||||||
.expect("Should create multiaddress")
|
|
||||||
loopBackAddrIp = MultiAddress.init("/ip4/127.0.0.1/")
|
|
||||||
.expect("Should create multiaddress")
|
|
||||||
|
|
||||||
# announce addresses should be set to bound addresses,
|
|
||||||
# but the IP should be mapped to the provided nat ip
|
|
||||||
announceAddrs = s.codexNode.switch.peerInfo.addrs.mapIt:
|
|
||||||
block:
|
|
||||||
let
|
|
||||||
listenIPPart = it[multiCodec("ip4")].expect("Should get IP")
|
|
||||||
|
|
||||||
if listenIPPart == anyAddrIp or
|
|
||||||
(listenIPPart == loopBackAddrIp and natIpPart != loopBackAddrIp):
|
|
||||||
it.remapAddr(s.config.nat.some)
|
|
||||||
else:
|
|
||||||
it
|
|
||||||
|
|
||||||
s.codexNode.discovery.updateAnnounceRecord(announceAddrs)
|
s.codexNode.discovery.updateAnnounceRecord(announceAddrs)
|
||||||
s.codexNode.discovery.updateDhtRecord(s.config.nat, s.config.discoveryPort)
|
s.codexNode.discovery.updateDhtRecord(discoveryAddrs)
|
||||||
|
|
||||||
await s.bootstrapInteractions()
|
await s.bootstrapInteractions()
|
||||||
await s.codexNode.start()
|
await s.codexNode.start()
|
||||||
|
@ -243,7 +227,6 @@ proc new*(
|
||||||
discovery = Discovery.new(
|
discovery = Discovery.new(
|
||||||
switch.peerInfo.privateKey,
|
switch.peerInfo.privateKey,
|
||||||
announceAddrs = config.listenAddrs,
|
announceAddrs = config.listenAddrs,
|
||||||
bindIp = config.discoveryIp,
|
|
||||||
bindPort = config.discoveryPort,
|
bindPort = config.discoveryPort,
|
||||||
bootstrapNodes = config.bootstrapNodes,
|
bootstrapNodes = config.bootstrapNodes,
|
||||||
store = discoveryStore)
|
store = discoveryStore)
|
||||||
|
|
|
@ -146,17 +146,14 @@ proc updateAnnounceRecord*(d: Discovery, addrs: openArray[MultiAddress]) =
|
||||||
d.protocol.updateRecord(d.providerRecord)
|
d.protocol.updateRecord(d.providerRecord)
|
||||||
.expect("Should update SPR")
|
.expect("Should update SPR")
|
||||||
|
|
||||||
proc updateDhtRecord*(d: Discovery, ip: ValidIpAddress, port: Port) =
|
proc updateDhtRecord*(d: Discovery, addrs: openArray[MultiAddress]) =
|
||||||
## Update providers record
|
## Update providers record
|
||||||
##
|
##
|
||||||
|
|
||||||
trace "Updating Dht record", ip, port = $port
|
trace "Updating Dht record", addrs = addrs
|
||||||
d.dhtRecord = SignedPeerRecord.init(
|
d.dhtRecord = SignedPeerRecord.init(
|
||||||
d.key, PeerRecord.init(d.peerId, @[
|
d.key, PeerRecord.init(d.peerId, @addrs))
|
||||||
MultiAddress.init(
|
.expect("Should construct signed record").some
|
||||||
ip,
|
|
||||||
IpTransportProtocol.udpProtocol,
|
|
||||||
port)])).expect("Should construct signed record").some
|
|
||||||
|
|
||||||
if not d.protocol.isNil:
|
if not d.protocol.isNil:
|
||||||
d.protocol.updateRecord(d.dhtRecord)
|
d.protocol.updateRecord(d.dhtRecord)
|
||||||
|
|
|
@ -0,0 +1,419 @@
|
||||||
|
# Copyright (c) 2019-2023 Status Research & Development GmbH
|
||||||
|
# Licensed under either of
|
||||||
|
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE))
|
||||||
|
# * MIT license ([LICENSE-MIT](LICENSE-MIT))
|
||||||
|
# at your option.
|
||||||
|
# This file may not be copied, modified, or distributed except according to
|
||||||
|
# those terms.
|
||||||
|
|
||||||
|
{.push raises: [].}
|
||||||
|
|
||||||
|
import
|
||||||
|
std/[options, os, strutils, times, net],stew/shims/net as stewNet,
|
||||||
|
stew/[objects,results], nat_traversal/[miniupnpc, natpmp],
|
||||||
|
json_serialization/std/net
|
||||||
|
|
||||||
|
import pkg/chronos
|
||||||
|
import pkg/chronicles
|
||||||
|
import pkg/libp2p
|
||||||
|
|
||||||
|
import ./utils
|
||||||
|
import ./utils/natutils
|
||||||
|
import ./utils/addrutils
|
||||||
|
|
||||||
|
const
|
||||||
|
UPNP_TIMEOUT = 200 # ms
|
||||||
|
PORT_MAPPING_INTERVAL = 20 * 60 # seconds
|
||||||
|
NATPMP_LIFETIME = 60 * 60 # in seconds, must be longer than PORT_MAPPING_INTERVAL
|
||||||
|
|
||||||
|
var
|
||||||
|
upnp {.threadvar.}: Miniupnp
|
||||||
|
npmp {.threadvar.}: NatPmp
|
||||||
|
strategy = NatStrategy.NatNone
|
||||||
|
internalTcpPort: Port
|
||||||
|
externalTcpPort: Port
|
||||||
|
internalUdpPort: Port
|
||||||
|
externalUdpPort: Port
|
||||||
|
|
||||||
|
logScope:
|
||||||
|
topics = "nat"
|
||||||
|
|
||||||
|
type
|
||||||
|
PrefSrcStatus = enum
|
||||||
|
NoRoutingInfo
|
||||||
|
PrefSrcIsPublic
|
||||||
|
PrefSrcIsPrivate
|
||||||
|
BindAddressIsPublic
|
||||||
|
BindAddressIsPrivate
|
||||||
|
|
||||||
|
## Also does threadvar initialisation.
|
||||||
|
## Must be called before redirectPorts() in each thread.
|
||||||
|
proc getExternalIP*(natStrategy: NatStrategy, quiet = false): Option[IpAddress] =
|
||||||
|
var externalIP: IpAddress
|
||||||
|
|
||||||
|
if natStrategy == NatStrategy.NatAny or natStrategy == NatStrategy.NatUpnp:
|
||||||
|
if upnp == nil:
|
||||||
|
upnp = newMiniupnp()
|
||||||
|
|
||||||
|
upnp.discoverDelay = UPNP_TIMEOUT
|
||||||
|
let dres = upnp.discover()
|
||||||
|
if dres.isErr:
|
||||||
|
debug "UPnP", msg = dres.error
|
||||||
|
else:
|
||||||
|
var
|
||||||
|
msg: cstring
|
||||||
|
canContinue = true
|
||||||
|
case upnp.selectIGD():
|
||||||
|
of IGDNotFound:
|
||||||
|
msg = "Internet Gateway Device not found. Giving up."
|
||||||
|
canContinue = false
|
||||||
|
of IGDFound:
|
||||||
|
msg = "Internet Gateway Device found."
|
||||||
|
of IGDNotConnected:
|
||||||
|
msg = "Internet Gateway Device found but it's not connected. Trying anyway."
|
||||||
|
of NotAnIGD:
|
||||||
|
msg = "Some device found, but it's not recognised as an Internet Gateway Device. Trying anyway."
|
||||||
|
if not quiet:
|
||||||
|
debug "UPnP", msg
|
||||||
|
if canContinue:
|
||||||
|
let ires = upnp.externalIPAddress()
|
||||||
|
if ires.isErr:
|
||||||
|
debug "UPnP", msg = ires.error
|
||||||
|
else:
|
||||||
|
# if we got this far, UPnP is working and we don't need to try NAT-PMP
|
||||||
|
try:
|
||||||
|
externalIP = parseIpAddress(ires.value)
|
||||||
|
strategy = NatStrategy.NatUpnp
|
||||||
|
return some(externalIP)
|
||||||
|
except ValueError as e:
|
||||||
|
error "parseIpAddress() exception", err = e.msg
|
||||||
|
return
|
||||||
|
|
||||||
|
if natStrategy == NatStrategy.NatAny or natStrategy == NatStrategy.NatPmp:
|
||||||
|
if npmp == nil:
|
||||||
|
npmp = newNatPmp()
|
||||||
|
let nres = npmp.init()
|
||||||
|
if nres.isErr:
|
||||||
|
debug "NAT-PMP", msg = nres.error
|
||||||
|
else:
|
||||||
|
let nires = npmp.externalIPAddress()
|
||||||
|
if nires.isErr:
|
||||||
|
debug "NAT-PMP", msg = nires.error
|
||||||
|
else:
|
||||||
|
try:
|
||||||
|
externalIP = parseIpAddress($(nires.value))
|
||||||
|
strategy = NatPmp
|
||||||
|
return some(externalIP)
|
||||||
|
except ValueError as e:
|
||||||
|
error "parseIpAddress() exception", err = e.msg
|
||||||
|
return
|
||||||
|
|
||||||
|
# This queries the routing table to get the "preferred source" attribute and
|
||||||
|
# checks if it's a public IP. If so, then it's our public IP.
|
||||||
|
#
|
||||||
|
# Further more, we check if the bind address (user provided, or a "0.0.0.0"
|
||||||
|
# default) is a public IP. That's a long shot, because code paths involving a
|
||||||
|
# user-provided bind address are not supposed to get here.
|
||||||
|
proc getRoutePrefSrc(
|
||||||
|
bindIp: ValidIpAddress): (Option[ValidIpAddress], PrefSrcStatus) =
|
||||||
|
let bindAddress = initTAddress(bindIp, Port(0))
|
||||||
|
|
||||||
|
if bindAddress.isAnyLocal():
|
||||||
|
let ip = getRouteIpv4()
|
||||||
|
if ip.isErr():
|
||||||
|
# No route was found, log error and continue without IP.
|
||||||
|
error "No routable IP address found, check your network connection",
|
||||||
|
error = ip.error
|
||||||
|
return (none(ValidIpAddress), NoRoutingInfo)
|
||||||
|
elif ip.get().isGlobalUnicast():
|
||||||
|
return (some(ip.get()), PrefSrcIsPublic)
|
||||||
|
else:
|
||||||
|
return (none(ValidIpAddress), PrefSrcIsPrivate)
|
||||||
|
elif bindAddress.isGlobalUnicast():
|
||||||
|
return (some(ValidIpAddress.init(bindIp)), BindAddressIsPublic)
|
||||||
|
else:
|
||||||
|
return (none(ValidIpAddress), BindAddressIsPrivate)
|
||||||
|
|
||||||
|
# Try to detect a public IP assigned to this host, before trying NAT traversal.
|
||||||
|
proc getPublicRoutePrefSrcOrExternalIP*(natStrategy: NatStrategy, bindIp: ValidIpAddress, quiet = true): Option[ValidIpAddress] =
|
||||||
|
let (prefSrcIp, prefSrcStatus) = getRoutePrefSrc(bindIp)
|
||||||
|
|
||||||
|
case prefSrcStatus:
|
||||||
|
of NoRoutingInfo, PrefSrcIsPublic, BindAddressIsPublic:
|
||||||
|
return prefSrcIp
|
||||||
|
of PrefSrcIsPrivate, BindAddressIsPrivate:
|
||||||
|
let extIp = getExternalIP(natStrategy, quiet)
|
||||||
|
if extIp.isSome:
|
||||||
|
return some(ValidIpAddress.init(extIp.get))
|
||||||
|
|
||||||
|
proc doPortMapping(tcpPort, udpPort: Port, description: string): Option[(Port, Port)] {.gcsafe.} =
|
||||||
|
var
|
||||||
|
extTcpPort: Port
|
||||||
|
extUdpPort: Port
|
||||||
|
|
||||||
|
if strategy == NatStrategy.NatUpnp:
|
||||||
|
for t in [(tcpPort, UPNPProtocol.TCP), (udpPort, UPNPProtocol.UDP)]:
|
||||||
|
let
|
||||||
|
(port, protocol) = t
|
||||||
|
pmres = upnp.addPortMapping(externalPort = $port,
|
||||||
|
protocol = protocol,
|
||||||
|
internalHost = upnp.lanAddr,
|
||||||
|
internalPort = $port,
|
||||||
|
desc = description,
|
||||||
|
leaseDuration = 0)
|
||||||
|
if pmres.isErr:
|
||||||
|
error "UPnP port mapping", msg = pmres.error, port
|
||||||
|
return
|
||||||
|
else:
|
||||||
|
# let's check it
|
||||||
|
let cres = upnp.getSpecificPortMapping(externalPort = $port,
|
||||||
|
protocol = protocol)
|
||||||
|
if cres.isErr:
|
||||||
|
warn "UPnP port mapping check failed. Assuming the check itself is broken and the port mapping was done.", msg = cres.error
|
||||||
|
|
||||||
|
info "UPnP: added port mapping", externalPort = port, internalPort = port, protocol = protocol
|
||||||
|
case protocol:
|
||||||
|
of UPNPProtocol.TCP:
|
||||||
|
extTcpPort = port
|
||||||
|
of UPNPProtocol.UDP:
|
||||||
|
extUdpPort = port
|
||||||
|
elif strategy == NatStrategy.NatPmp:
|
||||||
|
for t in [(tcpPort, NatPmpProtocol.TCP), (udpPort, NatPmpProtocol.UDP)]:
|
||||||
|
let
|
||||||
|
(port, protocol) = t
|
||||||
|
pmres = npmp.addPortMapping(eport = port.cushort,
|
||||||
|
iport = port.cushort,
|
||||||
|
protocol = protocol,
|
||||||
|
lifetime = NATPMP_LIFETIME)
|
||||||
|
if pmres.isErr:
|
||||||
|
error "NAT-PMP port mapping", msg = pmres.error, port
|
||||||
|
return
|
||||||
|
else:
|
||||||
|
let extPort = Port(pmres.value)
|
||||||
|
info "NAT-PMP: added port mapping", externalPort = extPort, internalPort = port, protocol = protocol
|
||||||
|
case protocol:
|
||||||
|
of NatPmpProtocol.TCP:
|
||||||
|
extTcpPort = extPort
|
||||||
|
of NatPmpProtocol.UDP:
|
||||||
|
extUdpPort = extPort
|
||||||
|
return some((extTcpPort, extUdpPort))
|
||||||
|
|
||||||
|
type PortMappingArgs = tuple[tcpPort, udpPort: Port, description: string]
|
||||||
|
var
|
||||||
|
natThread: Thread[PortMappingArgs]
|
||||||
|
natCloseChan: Channel[bool]
|
||||||
|
|
||||||
|
proc repeatPortMapping(args: PortMappingArgs) {.thread, raises: [ValueError].} =
|
||||||
|
ignoreSignalsInThread()
|
||||||
|
let
|
||||||
|
(tcpPort, udpPort, description) = args
|
||||||
|
interval = initDuration(seconds = PORT_MAPPING_INTERVAL)
|
||||||
|
sleepDuration = 1_000 # in ms, also the maximum delay after pressing Ctrl-C
|
||||||
|
|
||||||
|
var lastUpdate = now()
|
||||||
|
|
||||||
|
# We can't use copies of Miniupnp and NatPmp objects in this thread, because they share
|
||||||
|
# C pointers with other instances that have already been garbage collected, so
|
||||||
|
# we use threadvars instead and initialise them again with getExternalIP(),
|
||||||
|
# even though we don't need the external IP's value.
|
||||||
|
let ipres = getExternalIP(strategy, quiet = true)
|
||||||
|
if ipres.isSome:
|
||||||
|
while true:
|
||||||
|
# we're being silly here with this channel polling because we can't
|
||||||
|
# select on Nim channels like on Go ones
|
||||||
|
let (dataAvailable, _) = try: natCloseChan.tryRecv()
|
||||||
|
except Exception: (false, false)
|
||||||
|
if dataAvailable:
|
||||||
|
return
|
||||||
|
else:
|
||||||
|
let currTime = now()
|
||||||
|
if currTime >= (lastUpdate + interval):
|
||||||
|
discard doPortMapping(tcpPort, udpPort, description)
|
||||||
|
lastUpdate = currTime
|
||||||
|
sleep(sleepDuration)
|
||||||
|
|
||||||
|
proc stopNatThread() {.noconv.} =
|
||||||
|
# stop the thread
|
||||||
|
|
||||||
|
try:
|
||||||
|
natCloseChan.send(true)
|
||||||
|
natThread.joinThread()
|
||||||
|
natCloseChan.close()
|
||||||
|
except Exception as exc:
|
||||||
|
warn "Failed to stop NAT port mapping renewal thread", exc = exc.msg
|
||||||
|
|
||||||
|
# delete our port mappings
|
||||||
|
|
||||||
|
# FIXME: if the initial port mapping failed because it already existed for the
|
||||||
|
# required external port, we should not delete it. It might have been set up
|
||||||
|
# by another program.
|
||||||
|
|
||||||
|
# In Windows, a new thread is created for the signal handler, so we need to
|
||||||
|
# initialise our threadvars again.
|
||||||
|
let ipres = getExternalIP(strategy, quiet = true)
|
||||||
|
if ipres.isSome:
|
||||||
|
if strategy == NatStrategy.NatUpnp:
|
||||||
|
for t in [(externalTcpPort, internalTcpPort, UPNPProtocol.TCP), (externalUdpPort, internalUdpPort, UPNPProtocol.UDP)]:
|
||||||
|
let
|
||||||
|
(eport, iport, protocol) = t
|
||||||
|
pmres = upnp.deletePortMapping(externalPort = $eport,
|
||||||
|
protocol = protocol)
|
||||||
|
if pmres.isErr:
|
||||||
|
error "UPnP port mapping deletion", msg = pmres.error
|
||||||
|
else:
|
||||||
|
debug "UPnP: deleted port mapping", externalPort = eport, internalPort = iport, protocol = protocol
|
||||||
|
elif strategy == NatStrategy.NatPmp:
|
||||||
|
for t in [(externalTcpPort, internalTcpPort, NatPmpProtocol.TCP), (externalUdpPort, internalUdpPort, NatPmpProtocol.UDP)]:
|
||||||
|
let
|
||||||
|
(eport, iport, protocol) = t
|
||||||
|
pmres = npmp.deletePortMapping(eport = eport.cushort,
|
||||||
|
iport = iport.cushort,
|
||||||
|
protocol = protocol)
|
||||||
|
if pmres.isErr:
|
||||||
|
error "NAT-PMP port mapping deletion", msg = pmres.error
|
||||||
|
else:
|
||||||
|
debug "NAT-PMP: deleted port mapping", externalPort = eport, internalPort = iport, protocol = protocol
|
||||||
|
|
||||||
|
proc redirectPorts*(tcpPort, udpPort: Port, description: string): Option[(Port, Port)] =
|
||||||
|
result = doPortMapping(tcpPort, udpPort, description)
|
||||||
|
if result.isSome:
|
||||||
|
(externalTcpPort, externalUdpPort) = result.get()
|
||||||
|
# needed by NAT-PMP on port mapping deletion
|
||||||
|
internalTcpPort = tcpPort
|
||||||
|
internalUdpPort = udpPort
|
||||||
|
# Port mapping works. Let's launch a thread that repeats it, in case the
|
||||||
|
# NAT-PMP lease expires or the router is rebooted and forgets all about
|
||||||
|
# these mappings.
|
||||||
|
natCloseChan.open()
|
||||||
|
try:
|
||||||
|
natThread.createThread(repeatPortMapping, (externalTcpPort, externalUdpPort, description))
|
||||||
|
# atexit() in disguise
|
||||||
|
addQuitProc(stopNatThread)
|
||||||
|
except Exception as exc:
|
||||||
|
warn "Failed to create NAT port mapping renewal thread", exc = exc.msg
|
||||||
|
|
||||||
|
proc setupNat*(natStrategy: NatStrategy, tcpPort, udpPort: Port,
|
||||||
|
clientId: string):
|
||||||
|
tuple[ip: Option[ValidIpAddress], tcpPort, udpPort: Option[Port]] =
|
||||||
|
## Setup NAT port mapping and get external IP address.
|
||||||
|
## If any of this fails, we don't return any IP address but do return the
|
||||||
|
## original ports as best effort.
|
||||||
|
## TODO: Allow for tcp or udp port mapping to be optional.
|
||||||
|
let extIp = getExternalIP(natStrategy)
|
||||||
|
if extIp.isSome:
|
||||||
|
let ip = ValidIpAddress.init(extIp.get)
|
||||||
|
let extPorts = ({.gcsafe.}:
|
||||||
|
redirectPorts(tcpPort = tcpPort,
|
||||||
|
udpPort = udpPort,
|
||||||
|
description = clientId))
|
||||||
|
if extPorts.isSome:
|
||||||
|
let (extTcpPort, extUdpPort) = extPorts.get()
|
||||||
|
(ip: some(ip), tcpPort: some(extTcpPort), udpPort: some(extUdpPort))
|
||||||
|
else:
|
||||||
|
warn "UPnP/NAT-PMP available but port forwarding failed"
|
||||||
|
(ip: none(ValidIpAddress), tcpPort: some(tcpPort), udpPort: some(udpPort))
|
||||||
|
else:
|
||||||
|
warn "UPnP/NAT-PMP not available"
|
||||||
|
(ip: none(ValidIpAddress), tcpPort: some(tcpPort), udpPort: some(udpPort))
|
||||||
|
|
||||||
|
type
|
||||||
|
NatConfig* = object
|
||||||
|
case hasExtIp*: bool
|
||||||
|
of true: extIp*: ValidIpAddress
|
||||||
|
of false: nat*: NatStrategy
|
||||||
|
|
||||||
|
func parseCmdArg*(T: type NatConfig, p: string): T {.raises: [ValueError].} =
|
||||||
|
case p.toLowerAscii:
|
||||||
|
of "any":
|
||||||
|
NatConfig(hasExtIp: false, nat: NatStrategy.NatAny)
|
||||||
|
of "none":
|
||||||
|
NatConfig(hasExtIp: false, nat: NatStrategy.NatNone)
|
||||||
|
of "upnp":
|
||||||
|
NatConfig(hasExtIp: false, nat: NatStrategy.NatUpnp)
|
||||||
|
of "pmp":
|
||||||
|
NatConfig(hasExtIp: false, nat: NatStrategy.NatPmp)
|
||||||
|
else:
|
||||||
|
if p.startsWith("extip:"):
|
||||||
|
try:
|
||||||
|
let ip = ValidIpAddress.init(p[6..^1])
|
||||||
|
NatConfig(hasExtIp: true, extIp: ip)
|
||||||
|
except ValueError:
|
||||||
|
let error = "Not a valid IP address: " & p[6..^1]
|
||||||
|
raise newException(ValueError, error)
|
||||||
|
else:
|
||||||
|
let error = "Not a valid NAT option: " & p
|
||||||
|
raise newException(ValueError, error)
|
||||||
|
|
||||||
|
proc completeCmdArg*(T: type NatConfig; val: string): seq[string] =
|
||||||
|
return @[]
|
||||||
|
|
||||||
|
|
||||||
|
proc setupAddress*(natConfig: NatConfig, bindIp: ValidIpAddress,
|
||||||
|
tcpPort, udpPort: Port, clientId: string):
|
||||||
|
tuple[ip: Option[ValidIpAddress], tcpPort, udpPort: Option[Port]]
|
||||||
|
{.gcsafe.} =
|
||||||
|
## Set-up of the external address via any of the ways as configured in
|
||||||
|
## `NatConfig`. In case all fails an error is logged and the bind ports are
|
||||||
|
## selected also as external ports, as best effort and in hope that the
|
||||||
|
## external IP can be figured out by other means at a later stage.
|
||||||
|
## TODO: Allow for tcp or udp bind ports to be optional.
|
||||||
|
|
||||||
|
if natConfig.hasExtIp:
|
||||||
|
# any required port redirection must be done by hand
|
||||||
|
return (some(natConfig.extIp), some(tcpPort), some(udpPort))
|
||||||
|
|
||||||
|
case natConfig.nat:
|
||||||
|
of NatStrategy.NatAny:
|
||||||
|
let (prefSrcIp, prefSrcStatus) = getRoutePrefSrc(bindIp)
|
||||||
|
|
||||||
|
case prefSrcStatus:
|
||||||
|
of NoRoutingInfo, PrefSrcIsPublic, BindAddressIsPublic:
|
||||||
|
return (prefSrcIp, some(tcpPort), some(udpPort))
|
||||||
|
of PrefSrcIsPrivate, BindAddressIsPrivate:
|
||||||
|
return setupNat(natConfig.nat, tcpPort, udpPort, clientId)
|
||||||
|
of NatStrategy.NatNone:
|
||||||
|
let (prefSrcIp, prefSrcStatus) = getRoutePrefSrc(bindIp)
|
||||||
|
|
||||||
|
case prefSrcStatus:
|
||||||
|
of NoRoutingInfo, PrefSrcIsPublic, BindAddressIsPublic:
|
||||||
|
return (prefSrcIp, some(tcpPort), some(udpPort))
|
||||||
|
of PrefSrcIsPrivate:
|
||||||
|
error "No public IP address found. Should not use --nat:none option"
|
||||||
|
return (none(ValidIpAddress), some(tcpPort), some(udpPort))
|
||||||
|
of BindAddressIsPrivate:
|
||||||
|
error "Bind IP is not a public IP address. Should not use --nat:none option"
|
||||||
|
return (none(ValidIpAddress), some(tcpPort), some(udpPort))
|
||||||
|
of NatStrategy.NatUpnp, NatStrategy.NatPmp:
|
||||||
|
return setupNat(natConfig.nat, tcpPort, udpPort, clientId)
|
||||||
|
|
||||||
|
proc natedAddress*(natConfig: NatConfig, addrs: seq[MultiAddress], udpPort: Port): tuple[libp2p, discovery: seq[MultiAddress]] =
|
||||||
|
## Takes a NAT configuration, sequence of multiaddresses and UDP port and returns:
|
||||||
|
## - Modified multiaddresses with NAT-mapped addresses for libp2p
|
||||||
|
## - Discovery addresses with NAT-mapped UDP ports
|
||||||
|
|
||||||
|
var discoveryAddrs = newSeq[MultiAddress](0)
|
||||||
|
let
|
||||||
|
newAddrs = addrs.mapIt:
|
||||||
|
block:
|
||||||
|
# Extract IP address and port from the multiaddress
|
||||||
|
let (ipPart, port) = getAddressAndPort(it)
|
||||||
|
if ipPart.isSome and port.isSome:
|
||||||
|
# Try to setup NAT mapping for the address
|
||||||
|
let (newIP, tcp, udp) = setupAddress(natConfig, ipPart.get, port.get, udpPort, "codex")
|
||||||
|
if newIP.isSome:
|
||||||
|
# NAT mapping successful - add discovery address with mapped UDP port
|
||||||
|
discoveryAddrs.add(getMultiAddrWithIPAndUDPPort(newIP.get, udp.get))
|
||||||
|
# Remap original address with NAT IP and TCP port
|
||||||
|
it.remapAddr(ip = newIP, port = tcp)
|
||||||
|
else:
|
||||||
|
# NAT mapping failed - use original address
|
||||||
|
echo "Failed to get external IP, using original address", it
|
||||||
|
discoveryAddrs.add(getMultiAddrWithIPAndUDPPort(ipPart.get, udpPort))
|
||||||
|
it
|
||||||
|
else:
|
||||||
|
# Invalid multiaddress format - return as is
|
||||||
|
it
|
||||||
|
(newAddrs, discoveryAddrs)
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -20,6 +20,8 @@ import ./utils/asynciter
|
||||||
|
|
||||||
export asyncheapqueue, fileutils, asynciter, chronos
|
export asyncheapqueue, fileutils, asynciter, chronos
|
||||||
|
|
||||||
|
when defined(posix):
|
||||||
|
import os, posix
|
||||||
|
|
||||||
func divUp*[T: SomeInteger](a, b : T): T =
|
func divUp*[T: SomeInteger](a, b : T): T =
|
||||||
## Division with result rounded up (rather than truncated as in 'div')
|
## Division with result rounded up (rather than truncated as in 'div')
|
||||||
|
@ -94,3 +96,31 @@ when not declared(parseDuration): # Odd code formatting to minimize diff v. main
|
||||||
result = start #..is no unit to the end of `s`.
|
result = start #..is no unit to the end of `s`.
|
||||||
var sizeF = number * scale + 0.5 # Saturate to int64.high when too big
|
var sizeF = number * scale + 0.5 # Saturate to int64.high when too big
|
||||||
size = seconds(int(sizeF))
|
size = seconds(int(sizeF))
|
||||||
|
|
||||||
|
# Block all/most signals in the current thread, so we don't interfere with regular signal
|
||||||
|
# handling elsewhere.
|
||||||
|
proc ignoreSignalsInThread*() =
|
||||||
|
when defined(posix):
|
||||||
|
var signalMask, oldSignalMask: Sigset
|
||||||
|
|
||||||
|
# sigprocmask() doesn't work on macOS, for multithreaded programs
|
||||||
|
if sigfillset(signalMask) != 0:
|
||||||
|
echo osErrorMsg(osLastError())
|
||||||
|
quit(QuitFailure)
|
||||||
|
when defined(boehmgc):
|
||||||
|
# Turns out Boehm GC needs some signals to deal with threads:
|
||||||
|
# https://www.hboehm.info/gc/debugging.html
|
||||||
|
const
|
||||||
|
SIGPWR = 30
|
||||||
|
SIGXCPU = 24
|
||||||
|
SIGSEGV = 11
|
||||||
|
SIGBUS = 7
|
||||||
|
if sigdelset(signalMask, SIGPWR) != 0 or
|
||||||
|
sigdelset(signalMask, SIGXCPU) != 0 or
|
||||||
|
sigdelset(signalMask, SIGSEGV) != 0 or
|
||||||
|
sigdelset(signalMask, SIGBUS) != 0:
|
||||||
|
echo osErrorMsg(osLastError())
|
||||||
|
quit(QuitFailure)
|
||||||
|
if pthread_sigmask(SIG_BLOCK, signalMask, oldSignalMask) != 0:
|
||||||
|
echo osErrorMsg(osLastError())
|
||||||
|
quit(QuitFailure)
|
|
@ -15,6 +15,7 @@ import std/options
|
||||||
|
|
||||||
import pkg/libp2p
|
import pkg/libp2p
|
||||||
import pkg/stew/shims/net
|
import pkg/stew/shims/net
|
||||||
|
import pkg/stew/endians2
|
||||||
|
|
||||||
func remapAddr*(
|
func remapAddr*(
|
||||||
address: MultiAddress,
|
address: MultiAddress,
|
||||||
|
@ -39,3 +40,59 @@ func remapAddr*(
|
||||||
|
|
||||||
MultiAddress.init(parts.join("/"))
|
MultiAddress.init(parts.join("/"))
|
||||||
.expect("Should construct multiaddress")
|
.expect("Should construct multiaddress")
|
||||||
|
|
||||||
|
proc getMultiAddrWithIPAndUDPPort*(ip: ValidIpAddress, port: Port): MultiAddress =
|
||||||
|
## Creates a MultiAddress with the specified IP address and UDP port
|
||||||
|
##
|
||||||
|
## Parameters:
|
||||||
|
## - ip: A valid IP address (IPv4 or IPv6)
|
||||||
|
## - port: The UDP port number
|
||||||
|
##
|
||||||
|
## Returns:
|
||||||
|
## A MultiAddress in the format "/ip4/<address>/udp/<port>" or "/ip6/<address>/udp/<port>"
|
||||||
|
|
||||||
|
let ipFamily = ip.family
|
||||||
|
if ipFamily == IpAddressFamily.IPv4:
|
||||||
|
# Construct IPv4 multiaddress
|
||||||
|
return MultiAddress.init("/ip4/" & $ip & "/udp/" & $port).expect("valid multiaddr")
|
||||||
|
else:
|
||||||
|
# Construct IPv6 multiaddress
|
||||||
|
return MultiAddress.init("/ip6/" & $ip & "/udp/" & $port).expect("valid multiaddr")
|
||||||
|
|
||||||
|
proc getAddressAndPort*(ma: MultiAddress): tuple[ip: Option[ValidIpAddress], port: Option[Port]] =
|
||||||
|
try:
|
||||||
|
# Try IPv4 first
|
||||||
|
let ipv4Result = ma[multiCodec("ip4")]
|
||||||
|
let ip = if ipv4Result.isOk:
|
||||||
|
let ipBytes = ipv4Result.get()
|
||||||
|
.protoArgument()
|
||||||
|
.expect("Invalid IPv4 format")
|
||||||
|
let ipArray = [ipBytes[0], ipBytes[1], ipBytes[2], ipBytes[3]]
|
||||||
|
some(ipv4(ipArray))
|
||||||
|
else:
|
||||||
|
# Try IPv6 if IPv4 not found
|
||||||
|
let ipv6Result = ma[multiCodec("ip6")]
|
||||||
|
if ipv6Result.isOk:
|
||||||
|
let ipBytes = ipv6Result.get()
|
||||||
|
.protoArgument()
|
||||||
|
.expect("Invalid IPv6 format")
|
||||||
|
var ipArray: array[16, byte]
|
||||||
|
for i in 0..15:
|
||||||
|
ipArray[i] = ipBytes[i]
|
||||||
|
some(ipv6(ipArray))
|
||||||
|
else:
|
||||||
|
none(ValidIpAddress)
|
||||||
|
|
||||||
|
# Get TCP Port
|
||||||
|
let portResult = ma[multiCodec("tcp")]
|
||||||
|
let port = if portResult.isOk:
|
||||||
|
let portBytes = portResult.get()
|
||||||
|
.protoArgument()
|
||||||
|
.expect("Invalid port format")
|
||||||
|
some(Port(fromBytesBE(uint16, portBytes)))
|
||||||
|
else:
|
||||||
|
none(Port)
|
||||||
|
|
||||||
|
result = (ip: ip, port: port)
|
||||||
|
except Exception:
|
||||||
|
result = (ip: none(ValidIpAddress), port: none(Port))
|
|
@ -0,0 +1,68 @@
|
||||||
|
{.push raises: [].}
|
||||||
|
|
||||||
|
import
|
||||||
|
std/[tables, hashes],
|
||||||
|
stew/results, stew/shims/net as stewNet, chronos, chronicles
|
||||||
|
|
||||||
|
import pkg/libp2p
|
||||||
|
|
||||||
|
type
|
||||||
|
NatStrategy* = enum
|
||||||
|
NatAny
|
||||||
|
NatUpnp
|
||||||
|
NatPmp
|
||||||
|
NatNone
|
||||||
|
|
||||||
|
type
|
||||||
|
IpLimits* = object
|
||||||
|
limit*: uint
|
||||||
|
ips: Table[ValidIpAddress, uint]
|
||||||
|
|
||||||
|
func hash*(ip: ValidIpAddress): Hash =
|
||||||
|
case ip.family
|
||||||
|
of IpAddressFamily.IPv6: hash(ip.address_v6)
|
||||||
|
of IpAddressFamily.IPv4: hash(ip.address_v4)
|
||||||
|
|
||||||
|
func inc*(ipLimits: var IpLimits, ip: ValidIpAddress): bool =
|
||||||
|
let val = ipLimits.ips.getOrDefault(ip, 0)
|
||||||
|
if val < ipLimits.limit:
|
||||||
|
ipLimits.ips[ip] = val + 1
|
||||||
|
true
|
||||||
|
else:
|
||||||
|
false
|
||||||
|
|
||||||
|
func dec*(ipLimits: var IpLimits, ip: ValidIpAddress) =
|
||||||
|
let val = ipLimits.ips.getOrDefault(ip, 0)
|
||||||
|
if val == 1:
|
||||||
|
ipLimits.ips.del(ip)
|
||||||
|
elif val > 1:
|
||||||
|
ipLimits.ips[ip] = val - 1
|
||||||
|
|
||||||
|
func isGlobalUnicast*(address: TransportAddress): bool =
|
||||||
|
if address.isGlobal() and address.isUnicast():
|
||||||
|
true
|
||||||
|
else:
|
||||||
|
false
|
||||||
|
|
||||||
|
func isGlobalUnicast*(address: IpAddress): bool =
|
||||||
|
let a = initTAddress(address, Port(0))
|
||||||
|
a.isGlobalUnicast()
|
||||||
|
|
||||||
|
proc getRouteIpv4*(): Result[ValidIpAddress, cstring] =
|
||||||
|
# Avoiding Exception with initTAddress and can't make it work with static.
|
||||||
|
# Note: `publicAddress` is only used an "example" IP to find the best route,
|
||||||
|
# no data is send over the network to this IP!
|
||||||
|
let
|
||||||
|
publicAddress = TransportAddress(family: AddressFamily.IPv4,
|
||||||
|
address_v4: [1'u8, 1, 1, 1], port: Port(0))
|
||||||
|
route = getBestRoute(publicAddress)
|
||||||
|
|
||||||
|
if route.source.isUnspecified():
|
||||||
|
err("No best ipv4 route found")
|
||||||
|
else:
|
||||||
|
let ip = try: route.source.address()
|
||||||
|
except ValueError as e:
|
||||||
|
# This should not occur really.
|
||||||
|
error "Address conversion error", exception = e.name, msg = e.msg
|
||||||
|
return err("Invalid IP address")
|
||||||
|
ok(ValidIpAddress.init(ip))
|
Loading…
Reference in New Issue