Move Portal wire protocol from nim-eth to fluffy (#749)

* Move Portal wire protocol from nim-eth to fluffy

* Rename fluffy make targets

* Move all_fluffy_tests a folder down
This commit is contained in:
Kim De Mey 2021-07-09 13:34:16 +02:00 committed by GitHub
parent 1c5c2cba8b
commit 424fd405e9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 914 additions and 21 deletions

View File

@ -169,8 +169,9 @@ jobs:
gcc --version
DEFAULT_MAKE_FLAGS="-j${ncpu}"
mingw32-make ${DEFAULT_MAKE_FLAGS} fluffy
mingw32-make ${DEFAULT_MAKE_FLAGS} fluffy-tools
build/fluffy.exe --help
mingw32-make ${DEFAULT_MAKE_FLAGS} test-fluffy
mingw32-make ${DEFAULT_MAKE_FLAGS} fluffy-test
- name: Run fluffy tests (Linux)
if: runner.os == 'Linux'
@ -178,15 +179,17 @@ jobs:
export LD_LIBRARY_PATH="${LD_LIBRARY_PATH}:/usr/local/lib"
DEFAULT_MAKE_FLAGS="-j${ncpu}"
env CC=gcc make ${DEFAULT_MAKE_FLAGS} fluffy
env CC=gcc make ${DEFAULT_MAKE_FLAGS} fluffy-tools
build/fluffy --help
# CC is needed to select correct compiler 32/64 bit
env CC=gcc CXX=g++ make ${DEFAULT_MAKE_FLAGS} test-fluffy test-fluffy-reproducibility
env CC=gcc CXX=g++ make ${DEFAULT_MAKE_FLAGS} fluffy-test fluffy-test-reproducibility
- name: Run fluffy tests (Macos)
if: runner.os == 'Macos'
run: |
DEFAULT_MAKE_FLAGS="-j${ncpu}"
make ${DEFAULT_MAKE_FLAGS} fluffy
make ${DEFAULT_MAKE_FLAGS} fluffy-tools
build/fluffy --help
# "-static" option will not work for osx unless static system libraries are provided
make ${DEFAULT_MAKE_FLAGS} test-fluffy test-fluffy-reproducibility
make ${DEFAULT_MAKE_FLAGS} fluffy-test fluffy-test-reproducibility

View File

@ -110,10 +110,6 @@ nimbus: | build deps
echo -e $(BUILD_MSG) "build/$@" && \
$(ENV_SCRIPT) nim nimbus $(NIM_PARAMS) nimbus.nims
fluffy: | build deps
echo -e $(BUILD_MSG) "build/$@" && \
$(ENV_SCRIPT) nim fluffy $(NIM_PARAMS) nimbus.nims
# symlink
nimbus.nims:
ln -s nimbus.nimble $@
@ -122,14 +118,10 @@ nimbus.nims:
libbacktrace:
+ $(MAKE) -C vendor/nim-libbacktrace --no-print-directory BUILD_CXX_LIB=0
# builds and runs the test suite
# builds and runs the nimbus test suite
test: | build deps
$(ENV_SCRIPT) nim test $(NIM_PARAMS) nimbus.nims
# builds and runs the test suite
test-fluffy: | build deps
$(ENV_SCRIPT) nim testfluffy $(NIM_PARAMS) nimbus.nims
# primitive reproducibility test
test-reproducibility:
+ [ -e build/nimbus ] || $(MAKE) V=0 nimbus; \
@ -140,8 +132,14 @@ test-reproducibility:
[ "$$MD5SUM1" = "$$MD5SUM2" ] && echo -e "\e[92mSuccess: identical binaries.\e[39m" || \
{ echo -e "\e[91mFailure: the binary changed between builds.\e[39m"; exit 1; }
# Fluffy related targets
# builds the fluffy client
fluffy: | build deps
echo -e $(BUILD_MSG) "build/$@" && \
$(ENV_SCRIPT) nim fluffy $(NIM_PARAMS) nimbus.nims
# primitive reproducibility test
test-fluffy-reproducibility:
fluffy-test-reproducibility:
+ [ -e build/fluffy ] || $(MAKE) V=0 fluffy; \
MD5SUM1=$$($(MD5SUM) build/fluffy | cut -d ' ' -f 1) && \
rm -rf nimcache/*/fluffy && \
@ -150,9 +148,17 @@ test-fluffy-reproducibility:
[ "$$MD5SUM1" = "$$MD5SUM2" ] && echo -e "\e[92mSuccess: identical binaries.\e[39m" || \
{ echo -e "\e[91mFailure: the binary changed between builds.\e[39m"; exit 1; }
# builds and runs the fluffy test suite
fluffy-test: | build deps
$(ENV_SCRIPT) nim testfluffy $(NIM_PARAMS) nimbus.nims
# builds the fluffy tools
fluffy-tools: | build deps
$(ENV_SCRIPT) nim portalcli $(NIM_PARAMS) nimbus.nims
# usual cleaning
clean: | clean-common
rm -rf build/{nimbus,fluffy,$(TOOLS_CSV),all_tests,test_rpc, all_fluffy_tests}
rm -rf build/{nimbus,fluffy,$(TOOLS_CSV),all_tests,test_rpc,all_fluffy_tests,portalcli}
ifneq ($(USE_LIBBACKTRACE), 0)
+ $(MAKE) -C vendor/nim-libbacktrace clean $(HANDLE_OUTPUT)
endif

View File

@ -9,12 +9,11 @@
import
confutils, confutils/std/net, chronicles, chronicles/topics_registry,
chronos, metrics, metrics/chronos_httpserver, json_rpc/clients/httpclient, json_rpc/rpcproxy,
chronos, metrics, metrics/chronos_httpserver, json_rpc/clients/httpclient,
json_rpc/rpcproxy,
eth/keys, eth/net/nat,
eth/p2p/discoveryv5/protocol as discv5_protocol,
eth/p2p/portal/protocol as portal_protocol,
./conf, ./rpc/eth_api, ./rpc/bridge_client
./conf, ./network/portal_protocol, ./rpc/eth_api, ./rpc/bridge_client
proc initializeBridgeClient(maybeUri: Option[string]): Option[BridgeClient] =
try:

56
fluffy/network/README.md Normal file
View File

@ -0,0 +1,56 @@
# Portal Network Wire Protocol
## Introduction
The `fluffy/network` directory holds a Nim implementation of the
[Portal Network Wire Protocol](https://github.com/ethereum/stateless-ethereum-specs/blob/master/state-network.md#wire-protocol).
Both specification, at above link, and implementations are still WIP.
The protocol builds on top of the Node Discovery v5.1 protocol its `talkreq` and
`talkresp` messages.
For further information on the Nim implementation of the Node Discovery v5.1
protocol check out the
[discv5](https://github.com/status-im/nim-eth/blob/master/doc/discv5.md) page.
## Test suite
To run the test suite specifically for the Portal wire protocol, run following
commands:
```sh
git clone git@github.com:status-im/nimbus-eth1.git
cd nimbus-eth1
# To bring the git submodules up to date
make update
# Build & run Portal wire protocol encoding test
./env.sh nim c -r ./fluffy/tests/test_portal_encoding
# Build & run Portal wire protocol network test
./env.sh nim c -r ./fluffy/tests/test_portal
```
## portalcli
This is a small command line application that allows you to run a node running
Discovery v5.1 + Portal wire protocol.
*Note:* Its objective is only to test the protocol wire component, not to actually
serve content. This means it will always return empty lists on content requests
currently. Perhaps in the future some hardcoded data could added and/or maybe
some test vectors can be created in such form.
The `portalcli` application allows you to either run a node, or to specifically
send one of the Portal message types, wait for the response, and then shut down.
### Example usage
```sh
git clone git@github.com:status-im/nimbus-eth1.git
cd nimbus-eth1
# Build the fluffy tools
make tools-fluffy
# See all options
./build/portalcli --help
# Example command: Ping another node
./build/portalcli ping enr:<base64 encoding of ENR>
# Example command: Run discovery + portal node
./build/portalcli --log-level:debug --bootnode:enr:<base64 encoding of ENR>

153
fluffy/network/messages.nim Normal file
View File

@ -0,0 +1,153 @@
# Nimbus - Portal Network- Message types
# Copyright (c) 2021 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
# As per spec:
# https://github.com/ethereum/stateless-ethereum-specs/blob/master/state-network.md#wire-protocol
{.push raises: [Defect].}
import
stint, stew/[results, objects],
eth/ssz/ssz_serialization
export ssz_serialization, stint
type
ByteList* = List[byte, 2048]
MessageKind* = enum
unused = 0x00
ping = 0x01
pong = 0x02
findnode = 0x03
nodes = 0x04
findcontent = 0x05
foundcontent = 0x06
advertise = 0x07
requestproofs = 0x08
PingMessage* = object
enrSeq*: uint64
dataRadius*: UInt256
PongMessage* = object
enrSeq*: uint64
dataRadius*: UInt256
FindNodeMessage* = object
distances*: List[uint16, 256]
NodesMessage* = object
total*: uint8
enrs*: List[ByteList, 32] # ByteList here is the rlp encoded ENR. This could
# also be limited to 300 bytes instead of 2048
FindContentMessage* = object
contentKey*: ByteList
FoundContentMessage* = object
enrs*: List[ByteList, 32]
payload*: ByteList
AdvertiseMessage* = List[ByteList, 32] # No container, heh...
# This would be more consistent with the other messages
# AdvertiseMessage* = object
# contentKeys*: List[ByteList, 32]
RequestProofsMessage* = object
connectionId*: List[byte, 4]
contentKeys*: List[ByteList, 32]
Message* = object
case kind*: MessageKind
of ping:
ping*: PingMessage
of pong:
pong*: PongMessage
of findnode:
findNode*: FindNodeMessage
of nodes:
nodes*: NodesMessage
of findcontent:
findcontent*: FindContentMessage
of foundcontent:
foundcontent*: FoundContentMessage
of advertise:
advertise*: AdvertiseMessage
of requestproofs:
requestproofs*: RequestProofsMessage
else:
discard
SomeMessage* =
PingMessage or PongMessage or
FindNodeMessage or NodesMessage or
FindContentMessage or FoundContentMessage or
AdvertiseMessage or RequestProofsMessage
template messageKind*(T: typedesc[SomeMessage]): MessageKind =
when T is PingMessage: ping
elif T is PongMessage: pong
elif T is FindNodeMessage: findNode
elif T is NodesMessage: nodes
elif T is FindContentMessage: findcontent
elif T is FoundContentMessage: foundcontent
elif T is AdvertiseMessage: advertise
elif T is RequestProofsMessage: requestproofs
template toSszType*(x: auto): auto =
mixin toSszType
when x is UInt256: toBytesLE(x)
else: x
func fromSszBytes*(T: type UInt256, data: openArray[byte]):
T {.raises: [MalformedSszError, Defect].} =
if data.len != sizeof(result):
raiseIncorrectSize T
T.fromBytesLE(data)
proc encodeMessage*[T: SomeMessage](m: T): seq[byte] =
ord(messageKind(T)).byte & SSZ.encode(m)
proc decodeMessage*(body: openarray[byte]): Result[Message, cstring] =
# Decodes to the specific `Message` type.
if body.len < 1:
return err("No message data")
var kind: MessageKind
if not checkedEnumAssign(kind, body[0]):
return err("Invalid message type")
var message = Message(kind: kind)
try:
case kind
of unused: return err("Invalid message type")
of ping:
message.ping = SSZ.decode(body.toOpenArray(1, body.high), PingMessage)
of pong:
message.pong = SSZ.decode(body.toOpenArray(1, body.high), PongMessage)
of findNode:
message.findNode = SSZ.decode(body.toOpenArray(1, body.high), FindNodeMessage)
of nodes:
message.nodes = SSZ.decode(body.toOpenArray(1, body.high), NodesMessage)
of findcontent:
message.findcontent = SSZ.decode(body.toOpenArray(1, body.high), FindContentMessage)
of foundcontent:
message.foundcontent = SSZ.decode(body.toOpenArray(1, body.high), FoundContentMessage)
of advertise:
message.advertise = SSZ.decode(body.toOpenArray(1, body.high), AdvertiseMessage)
of requestproofs:
message.requestproofs = SSZ.decode(body.toOpenArray(1, body.high), RequestProofsMessage)
except SszError:
return err("Invalid message encoding")
ok(message)

View File

@ -0,0 +1,163 @@
# Nimbus - Portal Network
# Copyright (c) 2021 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.push raises: [Defect].}
import
stew/[results, byteutils], chronicles,
eth/rlp, eth/p2p/discoveryv5/[protocol, node],
./messages
export messages
logScope:
topics = "portal"
const
PortalProtocolId* = "portal".toBytes()
type
PortalProtocol* = ref object of TalkProtocol
baseProtocol*: protocol.Protocol
dataRadius*: UInt256
proc handlePing(p: PortalProtocol, ping: PingMessage):
seq[byte] =
let p = PongMessage(enrSeq: p.baseProtocol.localNode.record.seqNum,
dataRadius: p.dataRadius)
encodeMessage(p)
proc handleFindNode(p: PortalProtocol, fn: FindNodeMessage): seq[byte] =
if fn.distances.len == 0:
let enrs = List[ByteList, 32](@[])
encodeMessage(NodesMessage(total: 1, enrs: enrs))
elif fn.distances.contains(0):
# A request for our own record.
let enr = ByteList(rlp.encode(p.baseProtocol.localNode.record))
encodeMessage(NodesMessage(total: 1, enrs: List[ByteList, 32](@[enr])))
else:
# TODO: Not implemented for now, sending empty back.
let enrs = List[ByteList, 32](@[])
encodeMessage(NodesMessage(total: 1, enrs: enrs))
proc handleFindContent(p: PortalProtocol, ping: FindContentMessage): seq[byte] =
# TODO: Neither payload nor enrs implemented, sending empty back.
let
enrs = List[ByteList, 32](@[])
payload = ByteList(@[])
encodeMessage(FoundContentMessage(enrs: enrs, payload: payload))
proc handleAdvertise(p: PortalProtocol, ping: AdvertiseMessage): seq[byte] =
# TODO: Not implemented
let
connectionId = List[byte, 4](@[])
contentKeys = List[ByteList, 32](@[])
encodeMessage(RequestProofsMessage(connectionId: connectionId,
contentKeys: contentKeys))
proc messageHandler*(protocol: TalkProtocol, request: seq[byte]): seq[byte] =
doAssert(protocol of PortalProtocol)
let p = PortalProtocol(protocol)
let decoded = decodeMessage(request)
if decoded.isOk():
let message = decoded.get()
trace "Received message response", kind = message.kind
case message.kind
of MessageKind.ping:
p.handlePing(message.ping)
of MessageKind.findnode:
p.handleFindNode(message.findNode)
of MessageKind.findcontent:
p.handleFindContent(message.findcontent)
of MessageKind.advertise:
p.handleAdvertise(message.advertise)
else:
@[]
else:
@[]
proc new*(T: type PortalProtocol, baseProtocol: protocol.Protocol,
dataRadius = UInt256.high()): T =
let proto = PortalProtocol(
protocolHandler: messageHandler,
baseProtocol: baseProtocol,
dataRadius: dataRadius)
proto.baseProtocol.registerTalkProtocol(PortalProtocolId, proto).expect(
"Only one protocol should have this id")
return proto
proc ping*(p: PortalProtocol, dst: Node):
Future[DiscResult[PongMessage]] {.async.} =
let ping = PingMessage(enrSeq: p.baseProtocol.localNode.record.seqNum,
dataRadius: p.dataRadius)
# TODO: This send and response handling code could be more generalized for the
# different message types.
trace "Send message request", dstId = dst.id, kind = MessageKind.ping
let talkresp = await talkreq(p.baseProtocol, dst, PortalProtocolId,
encodeMessage(ping))
if talkresp.isOk():
let decoded = decodeMessage(talkresp.get().response)
if decoded.isOk():
let message = decoded.get()
if message.kind == pong:
return ok(message.pong)
else:
return err("Invalid message response received")
else:
return err(decoded.error)
else:
return err(talkresp.error)
proc findNode*(p: PortalProtocol, dst: Node, distances: List[uint16, 256]):
Future[DiscResult[NodesMessage]] {.async.} =
let fn = FindNodeMessage(distances: distances)
trace "Send message request", dstId = dst.id, kind = MessageKind.findnode
let talkresp = await talkreq(p.baseProtocol, dst, PortalProtocolId,
encodeMessage(fn))
if talkresp.isOk():
let decoded = decodeMessage(talkresp.get().response)
if decoded.isOk():
let message = decoded.get()
if message.kind == nodes:
# TODO: Verify nodes here
return ok(message.nodes)
else:
return err("Invalid message response received")
else:
return err(decoded.error)
else:
return err(talkresp.error)
proc findContent*(p: PortalProtocol, dst: Node, contentKey: ByteList):
Future[DiscResult[FoundContentMessage]] {.async.} =
let fc = FindContentMessage(contentKey: contentKey)
trace "Send message request", dstId = dst.id, kind = MessageKind.findcontent
let talkresp = await talkreq(p.baseProtocol, dst, PortalProtocolId,
encodeMessage(fc))
if talkresp.isOk():
let decoded = decodeMessage(talkresp.get().response)
if decoded.isOk():
let message = decoded.get()
if message.kind == foundcontent:
return ok(message.foundcontent)
else:
return err("Invalid message response received")
else:
return err(decoded.error)
else:
return err(talkresp.error)

View File

@ -0,0 +1,222 @@
# Nimbus - Portal Network
# Copyright (c) 2021 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
import
std/[options, strutils, tables],
confutils, confutils/std/net, chronicles, chronicles/topics_registry,
chronos, metrics, metrics/chronos_httpserver, stew/byteutils,
eth/[keys, net/nat],
eth/p2p/discoveryv5/[enr, node],
eth/p2p/discoveryv5/protocol as discv5_protocol,
./messages, ./portal_protocol
type
PortalCmd* = enum
noCommand
ping
findnode
findcontent
DiscoveryConf* = object
logLevel* {.
defaultValue: LogLevel.DEBUG
desc: "Sets the log level"
name: "log-level" .}: LogLevel
udpPort* {.
defaultValue: 9009
desc: "UDP listening port"
name: "udp-port" .}: uint16
listenAddress* {.
defaultValue: defaultListenAddress(config)
desc: "Listening address for the Discovery v5 traffic"
name: "listen-address" }: ValidIpAddress
bootnodes* {.
desc: "ENR URI of node to bootstrap discovery with. Argument may be repeated"
name: "bootnode" .}: seq[enr.Record]
nat* {.
desc: "Specify method to use for determining public address. " &
"Must be one of: any, none, upnp, pmp, extip:<IP>"
defaultValue: NatConfig(hasExtIp: false, nat: NatAny)
name: "nat" .}: NatConfig
enrAutoUpdate* {.
defaultValue: false
desc: "Discovery can automatically update its ENR with the IP address " &
"and UDP port as seen by other nodes it communicates with. " &
"This option allows to enable/disable this functionality"
name: "enr-auto-update" .}: bool
nodeKey* {.
desc: "P2P node private key as hex",
defaultValue: PrivateKey.random(keys.newRng()[])
name: "nodekey" .}: PrivateKey
metricsEnabled* {.
defaultValue: false
desc: "Enable the metrics server"
name: "metrics" .}: bool
metricsAddress* {.
defaultValue: defaultAdminListenAddress(config)
desc: "Listening address of the metrics server"
name: "metrics-address" .}: ValidIpAddress
metricsPort* {.
defaultValue: 8008
desc: "Listening HTTP port of the metrics server"
name: "metrics-port" .}: Port
case cmd* {.
command
defaultValue: noCommand }: PortalCmd
of noCommand:
discard
of ping:
pingTarget* {.
argument
desc: "ENR URI of the node to a send ping message"
name: "node" .}: Node
of findnode:
distance* {.
defaultValue: 255
desc: "Distance parameter for the findNode message"
name: "distance" .}: uint16
# TODO: Order here matters as else the help message does not show all the
# information, see: https://github.com/status-im/nim-confutils/issues/15
findNodeTarget* {.
argument
desc: "ENR URI of the node to send a findNode message"
name: "node" .}: Node
of findcontent:
findContentTarget* {.
argument
desc: "ENR URI of the node to send a findContent message"
name: "node" .}: Node
func defaultListenAddress*(conf: DiscoveryConf): ValidIpAddress =
(static ValidIpAddress.init("0.0.0.0"))
func defaultAdminListenAddress*(conf: DiscoveryConf): ValidIpAddress =
(static ValidIpAddress.init("127.0.0.1"))
proc parseCmdArg*(T: type enr.Record, p: TaintedString): T =
if not fromURI(result, p):
raise newException(ConfigurationError, "Invalid ENR")
proc completeCmdArg*(T: type enr.Record, val: TaintedString): seq[string] =
return @[]
proc parseCmdArg*(T: type Node, p: TaintedString): T =
var record: enr.Record
if not fromURI(record, p):
raise newException(ConfigurationError, "Invalid ENR")
let n = newNode(record)
if n.isErr:
raise newException(ConfigurationError, $n.error)
if n[].address.isNone():
raise newException(ConfigurationError, "ENR without address")
n[]
proc completeCmdArg*(T: type Node, val: TaintedString): seq[string] =
return @[]
proc parseCmdArg*(T: type PrivateKey, p: TaintedString): T =
try:
result = PrivateKey.fromHex(string(p)).tryGet()
except CatchableError:
raise newException(ConfigurationError, "Invalid private key")
proc completeCmdArg*(T: type PrivateKey, val: TaintedString): seq[string] =
return @[]
proc discover(d: discv5_protocol.Protocol) {.async.} =
while true:
let discovered = await d.queryRandom()
info "Lookup finished", nodes = discovered.len
await sleepAsync(30.seconds)
proc run(config: DiscoveryConf) =
let
rng = newRng()
bindIp = config.listenAddress
udpPort = Port(config.udpPort)
# TODO: allow for no TCP port mapping!
(extIp, _, extUdpPort) = setupAddress(config.nat,
config.listenAddress, udpPort, udpPort, "dcli")
let d = newProtocol(config.nodeKey,
extIp, none(Port), extUdpPort,
bootstrapRecords = config.bootnodes,
bindIp = bindIp, bindPort = udpPort,
enrAutoUpdate = config.enrAutoUpdate,
rng = rng)
d.open()
let portal = PortalProtocol.new(d)
if config.metricsEnabled:
let
address = config.metricsAddress
port = config.metricsPort
notice "Starting metrics HTTP server",
url = "http://" & $address & ":" & $port & "/metrics"
try:
chronos_httpserver.startMetricsHttpServer($address, port)
except CatchableError as exc: raise exc
except Exception as exc: raiseAssert exc.msg # TODO fix metrics
case config.cmd
of ping:
let pong = waitFor portal.ping(config.pingTarget)
if pong.isOk():
echo pong.get()
else:
echo pong.error
of findnode:
let distances = List[uint16, 256](@[config.distance])
let nodes = waitFor portal.findNode(config.findNodeTarget, distances)
if nodes.isOk():
echo nodes.get()
else:
echo nodes.error
of findcontent:
proc random(T: type UInt256, rng: var BrHmacDrbgContext): T =
var key: UInt256
brHmacDrbgGenerate(addr rng, addr key, csize_t(sizeof(key)))
key
# For now just random content keys
let contentKey = ByteList(@(UInt256.random(rng[]).toBytes()))
let foundContent = waitFor portal.findContent(config.findContentTarget,
contentKey)
if foundContent.isOk():
echo foundContent.get()
else:
echo foundContent.error
of noCommand:
d.start()
waitfor(discover(d))
when isMainModule:
let config = DiscoveryConf.load()
setLogLevel(config.logLevel)
run(config)

View File

@ -1,5 +1,5 @@
# Nimbus
# Copyright (c) 2018-2019 Status Research & Development GmbH
# Copyright (c) 2021 Status Research & Development GmbH
# Licensed under either of
# * Apache License, version 2.0, ([LICENSE-APACHE](LICENSE-APACHE) or http://www.apache.org/licenses/LICENSE-2.0)
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or http://opensource.org/licenses/MIT)
@ -9,5 +9,10 @@ import ../../test_macro
{. warning[UnusedImport]:off .}
import
./test_portal_encoding,
./test_portal
cliBuilder:
import ./tests/test_bridge_parser
import
./test_bridge_parser

View File

@ -0,0 +1,127 @@
# Nimbus - Portal Network
# Copyright (c) 2021 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.used.}
import
chronos, testutils/unittests, stew/shims/net,
eth/keys, # for rng
eth/p2p/discoveryv5/[enr, node, routing_table],
eth/p2p/discoveryv5/protocol as discv5_protocol,
../network/portal_protocol
proc localAddress(port: int): Address =
Address(ip: ValidIpAddress.init("127.0.0.1"), port: Port(port))
proc initDiscoveryNode(rng: ref BrHmacDrbgContext, privKey: PrivateKey,
address: Address,
bootstrapRecords: openarray[Record] = [],
localEnrFields: openarray[(string, seq[byte])] = [],
previousRecord = none[enr.Record]()):
discv5_protocol.Protocol =
# set bucketIpLimit to allow bucket split
let tableIpLimits = TableIpLimits(tableIpLimit: 1000, bucketIpLimit: 24)
result = newProtocol(privKey,
some(address.ip),
some(address.port), some(address.port),
bindPort = address.port,
bootstrapRecords = bootstrapRecords,
localEnrFields = localEnrFields,
previousRecord = previousRecord,
tableIpLimits = tableIpLimits,
rng = rng)
result.open()
proc random(T: type UInt256, rng: var BrHmacDrbgContext): T =
var key: UInt256
brHmacDrbgGenerate(addr rng, addr key, csize_t(sizeof(key)))
key
procSuite "Portal Tests":
let rng = newRng()
asyncTest "Portal Ping/Pong":
let
node1 = initDiscoveryNode(
rng, PrivateKey.random(rng[]), localAddress(20302))
node2 = initDiscoveryNode(
rng, PrivateKey.random(rng[]), localAddress(20303))
proto1 = PortalProtocol.new(node1)
proto2 = PortalProtocol.new(node2)
let pong = await proto1.ping(proto2.baseProtocol.localNode)
check:
pong.isOk()
pong.get().enrSeq == 1'u64
pong.get().dataRadius == UInt256.high()
await node1.closeWait()
await node2.closeWait()
asyncTest "Portal FindNode/Nodes":
let
node1 = initDiscoveryNode(
rng, PrivateKey.random(rng[]), localAddress(20302))
node2 = initDiscoveryNode(
rng, PrivateKey.random(rng[]), localAddress(20303))
proto1 = PortalProtocol.new(node1)
proto2 = PortalProtocol.new(node2)
block: # Find itself
let nodes = await proto1.findNode(proto2.baseProtocol.localNode,
List[uint16, 256](@[0'u16]))
check:
nodes.isOk()
nodes.get().total == 1'u8
nodes.get().enrs.len() == 1
block: # Find nothing
let nodes = await proto1.findNode(proto2.baseProtocol.localNode,
List[uint16, 256](@[]))
check:
nodes.isOk()
nodes.get().total == 1'u8
nodes.get().enrs.len() == 0
block: # Find for distance
# TODO: Add test when implemented
discard
await node1.closeWait()
await node2.closeWait()
asyncTest "Portal FindContent/FoundContent":
let
node1 = initDiscoveryNode(
rng, PrivateKey.random(rng[]), localAddress(20302))
node2 = initDiscoveryNode(
rng, PrivateKey.random(rng[]), localAddress(20303))
proto1 = PortalProtocol.new(node1)
proto2 = PortalProtocol.new(node2)
let contentKey = ByteList(@(UInt256.random(rng[]).toBytes()))
let foundContent = await proto1.findContent(proto2.baseProtocol.localNode,
contentKey)
check:
foundContent.isOk()
# TODO: adjust when implemented
foundContent.get().enrs.len() == 0
foundContent.get().payload.len() == 0
await node1.closeWait()
await node2.closeWait()

View File

@ -0,0 +1,156 @@
# Nimbus - Portal Network
# Copyright (c) 2021 Status Research & Development GmbH
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.used.}
import
std/unittest,
stint, stew/[byteutils, results],
../network/messages
suite "Portal Protocol Message Encodings":
test "Ping Request":
var dataRadius: UInt256
let
enrSeq = 1'u64
p = PingMessage(enrSeq: enrSeq, dataRadius: dataRadius)
let encoded = encodeMessage(p)
check encoded.toHex ==
"0101000000000000000000000000000000000000000000000000000000000000000000000000000000"
let decoded = decodeMessage(encoded)
check decoded.isOk()
let message = decoded.get()
check:
message.kind == ping
message.ping.enrSeq == enrSeq
message.ping.dataRadius == dataRadius
test "Pong Response":
var dataRadius: UInt256
let
enrSeq = 1'u64
p = PongMessage(enrSeq: enrSeq, dataRadius: dataRadius)
let encoded = encodeMessage(p)
check encoded.toHex ==
"0201000000000000000000000000000000000000000000000000000000000000000000000000000000"
let decoded = decodeMessage(encoded)
check decoded.isOk()
let message = decoded.get()
check:
message.kind == pong
message.pong.enrSeq == enrSeq
message.pong.dataRadius == dataRadius
test "FindNode Request":
let
distances = List[uint16, 256](@[0x0100'u16])
fn = FindNodeMessage(distances: distances)
let encoded = encodeMessage(fn)
check encoded.toHex == "03040000000001"
let decoded = decodeMessage(encoded)
check decoded.isOk()
let message = decoded.get()
check:
message.kind == findnode
message.findnode.distances == distances
test "Nodes Response (empty)":
let
total = 0x1'u8
n = NodesMessage(total: total)
let encoded = encodeMessage(n)
check encoded.toHex == "040105000000"
let decoded = decodeMessage(encoded)
check decoded.isOk()
let message = decoded.get()
check:
message.kind == nodes
message.nodes.total == total
message.nodes.enrs.len() == 0
test "FindContent Request":
let
contentKey = ByteList(@[byte 0x01, 0x02, 0x03])
fn = FindContentMessage(contentKey: contentKey)
let encoded = encodeMessage(fn)
check encoded.toHex == "0504000000010203"
let decoded = decodeMessage(encoded)
check decoded.isOk()
let message = decoded.get()
check:
message.kind == findcontent
message.findcontent.contentKey == contentKey
test "FoundContent Response (empty enrs)":
let
enrs = List[ByteList, 32](@[])
payload = ByteList(@[byte 0x01, 0x02, 0x03])
n = FoundContentMessage(enrs: enrs, payload: payload)
let encoded = encodeMessage(n)
check encoded.toHex == "060800000008000000010203"
let decoded = decodeMessage(encoded)
check decoded.isOk()
let message = decoded.get()
check:
message.kind == foundcontent
message.foundcontent.enrs.len() == 0
message.foundcontent.payload == payload
test "Advertise Request":
let
contentKeys = List[ByteList, 32](List(@[ByteList(@[byte 0x01, 0x02, 0x03])]))
am = AdvertiseMessage(contentKeys)
# am = AdvertiseMessage(contentKeys: contentKeys)
let encoded = encodeMessage(am)
check encoded.toHex == "0704000000010203"
# "070400000004000000010203"
let decoded = decodeMessage(encoded)
check decoded.isOk()
let message = decoded.get()
check:
message.kind == advertise
message.advertise == contentKeys
# message.advertise.contentKeys == contentKeys
test "RequestProofs Response": # That sounds weird
let
connectionId = List[byte, 4](@[byte 0x01, 0x02, 0x03, 0x04])
contentKeys =
List[ByteList, 32](List(@[ByteList(@[byte 0x01, 0x02, 0x03])]))
n = RequestProofsMessage(connectionId: connectionId,
contentKeys: contentKeys)
let encoded = encodeMessage(n)
check encoded.toHex == "08080000000c0000000102030404000000010203"
let decoded = decodeMessage(encoded)
check decoded.isOk()
let message = decoded.get()
check:
message.kind == requestproofs
message.requestproofs.connectionId == connectionId
message.requestproofs.contentKeys == contentKeys

View File

@ -54,5 +54,8 @@ task nimbus, "Build Nimbus":
task fluffy, "Build fluffy":
buildBinary "fluffy", "fluffy/", "-d:chronicles_log_level=TRACE"
task portalcli, "Build portalcli":
buildBinary "portalcli", "fluffy/network/", "-d:chronicles_log_level=TRACE"
task testfluffy, "Run fluffy tests":
test "fluffy", "all_fluffy_tests"
test "fluffy/tests", "all_fluffy_tests"