diff --git a/fluffy/conf.nim b/fluffy/conf.nim index e11ebd567..6b181d353 100644 --- a/fluffy/conf.nim +++ b/fluffy/conf.nim @@ -372,6 +372,15 @@ type name: "disable-state-root-validation" .}: bool + disableBanNodes* {. + hidden, + desc: + "Disable node banning functionality for both discv5 and portal sub-protocols", + defaultValue: defaultDisableBanNodes, + defaultValueDesc: $defaultDisableBanNodes, + name: "debug-disable-ban-nodes" + .}: bool + case cmd* {.command, defaultValue: noCommand.}: PortalCmd of noCommand: discard diff --git a/fluffy/fluffy.nim b/fluffy/fluffy.nim index 46144e836..f64aae46a 100644 --- a/fluffy/fluffy.nim +++ b/fluffy/fluffy.nim @@ -148,6 +148,7 @@ proc run( enrAutoUpdate = config.enrAutoUpdate, config = discoveryConfig, rng = rng, + banNodes = not config.disableBanNodes, ) d.open() @@ -184,6 +185,7 @@ proc run( config.tableIpLimit, config.bucketIpLimit, config.bitsPerHop, config.alpha, config.radiusConfig, config.disablePoke, config.maxGossipNodes, config.contentCacheSize, config.disableContentCache, config.maxConcurrentOffers, + config.disableBanNodes, ) portalNodeConfig = PortalNodeConfig( diff --git a/fluffy/network/beacon/beacon_network.nim b/fluffy/network/beacon/beacon_network.nim index 4191293f5..97a29d813 100644 --- a/fluffy/network/beacon/beacon_network.nim +++ b/fluffy/network/beacon/beacon_network.nim @@ -364,6 +364,8 @@ proc validateContent( debug "Received offered content validated successfully", srcNodeId, contentKey else: + if srcNodeId.isSome(): + n.portalProtocol.banNode(srcNodeId.get(), NodeBanDurationOfferFailedValidation) debug "Received offered content failed validation", srcNodeId, contentKey, error = validation.error return false diff --git a/fluffy/network/history/history_network.nim b/fluffy/network/history/history_network.nim index b9c232ee6..236de27dd 100644 --- a/fluffy/network/history/history_network.nim +++ b/fluffy/network/history/history_network.nim @@ -147,6 +147,9 @@ proc getVerifiedBlockHeader*( return Opt.none(Header) header = validateCanonicalHeaderBytes(headerContent.content, id, n.accumulator).valueOr: + n.portalProtocol.banNode( + headerContent.receivedFrom.id, NodeBanDurationContentLookupFailedValidation + ) warn "Validation of block header failed", error = error, node = headerContent.receivedFrom.record.toURI() continue @@ -192,6 +195,9 @@ proc getBlockBody*( return Opt.none(BlockBody) body = validateBlockBodyBytes(bodyContent.content, header).valueOr: + n.portalProtocol.banNode( + bodyContent.receivedFrom.id, NodeBanDurationContentLookupFailedValidation + ) warn "Validation of block body failed", error, node = bodyContent.receivedFrom.record.toURI() continue @@ -266,7 +272,11 @@ proc getReceipts*( receiptsContent = (await n.portalProtocol.contentLookup(contentKey, contentId)).valueOr: debug "Failed fetching receipts from the network" return Opt.none(seq[Receipt]) + receipts = validateReceiptsBytes(receiptsContent.content, header.receiptsRoot).valueOr: + n.portalProtocol.banNode( + receiptsContent.receivedFrom.id, NodeBanDurationContentLookupFailedValidation + ) warn "Validation of receipts failed", error, node = receiptsContent.receivedFrom.record.toURI() continue @@ -384,6 +394,9 @@ proc validateContent( debug "Received offered content validated successfully", srcNodeId, contentKey else: + if srcNodeId.isSome(): + n.portalProtocol.banNode(srcNodeId.get(), NodeBanDurationOfferFailedValidation) + debug "Received offered content failed validation", srcNodeId, contentKey, error = res.error return false diff --git a/fluffy/network/state/state_network.nim b/fluffy/network/state/state_network.nim index 6871b7130..4906fcf6c 100644 --- a/fluffy/network/state/state_network.nim +++ b/fluffy/network/state/state_network.nim @@ -112,6 +112,9 @@ proc getContent( continue validateRetrieval(key, contentValue).isOkOr: + n.portalProtocol.banNode( + lookupRes.receivedFrom.id, NodeBanDurationContentLookupFailedValidation + ) error "Validation of retrieved state content failed" continue @@ -246,6 +249,10 @@ proc processContentLoop(n: StateNetwork) {.async: (raises: []).} = debug "Received offered content validated successfully", srcNodeId, contentKeyBytes else: + if srcNodeId.isSome(): + n.portalProtocol.banNode( + srcNodeId.get(), NodeBanDurationOfferFailedValidation + ) state_network_offers_failed.inc(labelValues = [$n.portalProtocol.protocolId]) error "Received offered content failed validation", srcNodeId, contentKeyBytes, error = offerRes.error() diff --git a/fluffy/network/wire/portal_protocol.nim b/fluffy/network/wire/portal_protocol.nim index c48dab4cb..d90e9546e 100644 --- a/fluffy/network/wire/portal_protocol.nim +++ b/fluffy/network/wire/portal_protocol.nim @@ -126,6 +126,11 @@ const ## value in milliseconds initialLookups = 1 ## Amount of lookups done when populating the routing table + ## Ban durations for banned nodes in the routing table + NodeBanDurationMessageResponseError = 15.minutes + NodeBanDurationContentLookupFailedValidation* = 60.minutes + NodeBanDurationOfferFailedValidation* = 60.minutes + type ToContentIdHandler* = proc(contentKey: ContentKeyByteList): results.Opt[ContentId] {.raises: [], gcsafe.} @@ -285,6 +290,13 @@ func getProtocolId*( of PortalSubnetwork.transactionGossip: [portalPrefix, 0x4F] +proc banNode*(p: PortalProtocol, nodeId: NodeId, period: chronos.Duration) = + if not p.config.disableBanNodes: + p.routingTable.banNode(nodeId, period) + +proc isBanned*(p: PortalProtocol, nodeId: NodeId): bool = + p.config.disableBanNodes == false and p.routingTable.isBanned(nodeId) + func `$`(id: PortalProtocolId): string = id.toHex() @@ -300,8 +312,10 @@ func getNode*(p: PortalProtocol, id: NodeId): Opt[Node] = func localNode*(p: PortalProtocol): Node = p.baseProtocol.localNode -func neighbours*(p: PortalProtocol, id: NodeId, seenOnly = false): seq[Node] = - p.routingTable.neighbours(id = id, seenOnly = seenOnly) +template neighbours*( + p: PortalProtocol, id: NodeId, k: int = BUCKET_SIZE, seenOnly = false +): seq[Node] = + p.routingTable.neighbours(id, k, seenOnly) func distance(p: PortalProtocol, a, b: NodeId): UInt256 = p.routingTable.distance(a, b) @@ -480,7 +494,7 @@ proc handleFindContent( # Node does not have the content, or content is not even in radius, # send closest neighbours to the requested content id. let - closestNodes = p.routingTable.neighbours(NodeId(contentId), seenOnly = true) + closestNodes = p.neighbours(NodeId(contentId), seenOnly = true) enrs = truncateEnrs(closestNodes, maxPayloadSize, enrOverhead) portal_content_enrs_packed.observe(enrs.len().int64, labelValues = [$p.protocolId]) @@ -557,6 +571,12 @@ proc messageHandler( let p = PortalProtocol(protocol) + if p.isBanned(srcId): + # The sender of the message is in the temporary node ban list + # so we don't process the message + debug "Ignoring message from banned node", srcId, srcUdpAddress + return @[] + let decoded = decodeMessage(request) if decoded.isOk(): let message = decoded.get() @@ -695,7 +715,11 @@ proc reqResponse[Request: SomeMessage, Response: SomeMessage]( debug "Error receiving message response", error = messageResponse.error, srcId = dst.id, srcAddress = dst.address p.pingTimings.del(dst.id) - p.routingTable.replaceNode(dst) + + if p.config.disableBanNodes: + p.routingTable.replaceNode(dst) + else: + p.routingTable.banNode(dst.id, NodeBanDurationMessageResponseError) return messageResponse @@ -758,6 +782,9 @@ proc ping*( ): Future[PortalResult[(uint64, CapabilitiesPayload)]] {. async: (raises: [CancelledError]) .} = + if p.isBanned(dst.id): + return err("destination node is banned") + let pongResponse = await p.pingImpl(dst) if pongResponse.isOk(): @@ -783,12 +810,16 @@ proc ping*( proc findNodes*( p: PortalProtocol, dst: Node, distances: seq[uint16] ): Future[PortalResult[seq[Node]]] {.async: (raises: [CancelledError]).} = + if p.isBanned(dst.id): + return err("destination node is banned") + let nodesMessage = await p.findNodesImpl(dst, List[uint16, 256](distances)) if nodesMessage.isOk(): let records = recordsFromBytes(nodesMessage.get().enrs) if records.isOk(): # TODO: distance function is wrong here for state, fix + tests - return ok(verifyNodesRecords(records.get(), dst, enrsResultLimit, distances)) + let res = verifyNodesRecords(records.get(), dst, enrsResultLimit, distances) + return ok(res.filterIt(not p.isBanned(it.id))) else: return err(records.error) else: @@ -801,6 +832,9 @@ proc findContent*( node = dst contentKey + if p.isBanned(dst.id): + return err("destination node is banned") + let contentMessageResponse = await p.findContentImpl(dst, contentKey) if contentMessageResponse.isOk(): @@ -868,8 +902,11 @@ proc findContent*( let records = recordsFromBytes(m.enrs) if records.isOk(): let verifiedNodes = verifyNodesRecords(records.get(), dst, enrsResultLimit) - - return ok(FoundContent(src: dst, kind: Nodes, nodes: verifiedNodes)) + return ok( + FoundContent( + src: dst, kind: Nodes, nodes: verifiedNodes.filterIt(not p.isBanned(it.id)) + ) + ) else: return err("Content message returned invalid ENRs") else: @@ -935,6 +972,9 @@ proc offer( contentKeys.len().int64, labelValues = [$p.protocolId] ) + if p.isBanned(o.dst.id): + return err("destination node is banned") + let acceptMessageResponse = await p.offerImpl(o.dst, contentKeys) if acceptMessageResponse.isOk(): @@ -1088,7 +1128,7 @@ proc lookup*( ## target. Maximum value for n is `BUCKET_SIZE`. # `closestNodes` holds the k closest nodes to target found, sorted by distance # Unvalidated nodes are used for requests as a form of validation. - var closestNodes = p.routingTable.neighbours(target, BUCKET_SIZE, seenOnly = false) + var closestNodes = p.neighbours(target, BUCKET_SIZE, seenOnly = false) var asked, seen = HashSet[NodeId]() asked.incl(p.localNode.id) # No need to ask our own node @@ -1190,7 +1230,7 @@ proc contentLookup*( ## target. # `closestNodes` holds the k closest nodes to target found, sorted by distance # Unvalidated nodes are used for requests as a form of validation. - var closestNodes = p.routingTable.neighbours(targetId, BUCKET_SIZE, seenOnly = false) + var closestNodes = p.neighbours(targetId, BUCKET_SIZE, seenOnly = false) # Shuffling the order of the nodes in order to not always hit the same node # first for the same request. @@ -1316,7 +1356,7 @@ proc traceContentLookup*( # Need to use a system clock and not the mono clock for this. let startedAtMs = int64(times.epochTime() * 1000) - var closestNodes = p.routingTable.neighbours(targetId, BUCKET_SIZE, seenOnly = false) + var closestNodes = p.neighbours(targetId, BUCKET_SIZE, seenOnly = false) # Shuffling the order of the nodes in order to not always hit the same node # first for the same request. p.baseProtocol.rng[].shuffle(closestNodes) @@ -1514,7 +1554,7 @@ proc query*( ## This will take k nodes from the routing table closest to target and ## query them for nodes closest to target. If there are less than k nodes in ## the routing table, nodes returned by the first queries will be used. - var queryBuffer = p.routingTable.neighbours(target, k, seenOnly = false) + var queryBuffer = p.neighbours(target, k, seenOnly = false) var asked, seen = HashSet[NodeId]() asked.incl(p.localNode.id) # No need to ask our own node @@ -1605,7 +1645,7 @@ proc neighborhoodGossip*( # It might still cause issues in data getting propagated in a wider id range. var closestLocalNodes = - p.routingTable.neighbours(NodeId(contentId), k = 16, seenOnly = true) + p.routingTable.neighbours(NodeId(contentId), BUCKET_SIZE, seenOnly = true) # Shuffling the order of the nodes in order to not always hit the same node # first for the same request. @@ -1813,6 +1853,9 @@ proc refreshLoop(p: PortalProtocol) {.async: (raises: []).} = trace "Discovered nodes in random target query", nodes = randomQuery.len debug "Total nodes in routing table", total = p.routingTable.len() + # Remove the expired bans from routing table to limit memory usage + p.routingTable.cleanupExpiredBans() + await sleepAsync(refreshInterval) except CancelledError: trace "refreshLoop canceled" @@ -1865,6 +1908,12 @@ proc resolve*( if id == p.localNode.id: return Opt.some(p.localNode) + # No point in trying to resolve a banned node because it won't exist in the + # routing table and it will be filtered out of any respones in the lookup call + if p.isBanned(id): + debug "Not resolving banned node", nodeId = id + return Opt.none(Node) + let node = p.getNode(id) if node.isSome(): let nodesMessage = await p.findNodes(node.get(), @[0'u16]) diff --git a/fluffy/network/wire/portal_protocol_config.nim b/fluffy/network/wire/portal_protocol_config.nim index 23f67dfae..8384a3f33 100644 --- a/fluffy/network/wire/portal_protocol_config.nim +++ b/fluffy/network/wire/portal_protocol_config.nim @@ -1,5 +1,5 @@ # Fluffy -# Copyright (c) 2021-2024 Status Research & Development GmbH +# Copyright (c) 2021-2025 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). @@ -45,6 +45,7 @@ type contentCacheSize*: int disableContentCache*: bool maxConcurrentOffers*: int + disableBanNodes*: bool const defaultRadiusConfig* = RadiusConfig(kind: Dynamic) @@ -56,6 +57,7 @@ const defaultMaxConcurrentOffers* = 50 defaultAlpha* = 3 revalidationTimeout* = chronos.seconds(30) + defaultDisableBanNodes* = false defaultPortalProtocolConfig* = PortalProtocolConfig( tableIpLimits: DefaultTableIpLimits, @@ -67,6 +69,7 @@ const contentCacheSize: defaultContentCacheSize, disableContentCache: defaultDisableContentCache, maxConcurrentOffers: defaultMaxConcurrentOffers, + disableBanNodes: defaultDisableBanNodes, ) proc init*( @@ -81,6 +84,7 @@ proc init*( contentCacheSize: int, disableContentCache: bool, maxConcurrentOffers: int, + disableBanNodes: bool, ): T = PortalProtocolConfig( tableIpLimits: @@ -93,6 +97,7 @@ proc init*( contentCacheSize: contentCacheSize, disableContentCache: disableContentCache, maxConcurrentOffers: maxConcurrentOffers, + disableBanNodes: disableBanNodes, ) func fromLogRadius*(T: type UInt256, logRadius: uint16): T = diff --git a/fluffy/rpc/rpc_portal_state_api.nim b/fluffy/rpc/rpc_portal_state_api.nim index 5851dce06..c440ba096 100644 --- a/fluffy/rpc/rpc_portal_state_api.nim +++ b/fluffy/rpc/rpc_portal_state_api.nim @@ -46,6 +46,7 @@ proc installPortalStateApiHandlers*(rpcServer: RpcServer, p: PortalProtocol) = of Content: let valueBytes = foundContentResult.content validateRetrieval(key, valueBytes).isOkOr: + p.banNode(node.id, NodeBanDurationContentLookupFailedValidation) raise invalidValueErr() let res = ContentInfo( @@ -97,6 +98,10 @@ proc installPortalStateApiHandlers*(rpcServer: RpcServer, p: PortalProtocol) = valueBytes = contentLookupResult.content validateRetrieval(key, valueBytes).isOkOr: + p.banNode( + contentLookupResult.receivedFrom.id, + NodeBanDurationContentLookupFailedValidation, + ) raise invalidValueErr() p.storeContent(keyBytes, contentId, valueBytes, cacheContent = true) @@ -132,6 +137,10 @@ proc installPortalStateApiHandlers*(rpcServer: RpcServer, p: PortalProtocol) = raise contentNotFoundErrWithTrace(data) validateRetrieval(key, valueBytes).isOkOr: + if res.trace.receivedFrom.isSome(): + p.banNode( + res.trace.receivedFrom.get(), NodeBanDurationContentLookupFailedValidation + ) raise invalidValueErr() p.storeContent(keyBytes, contentId, valueBytes, cacheContent = true) diff --git a/vendor/nim-eth b/vendor/nim-eth index e589cc028..9f5c27466 160000 --- a/vendor/nim-eth +++ b/vendor/nim-eth @@ -1 +1 @@ -Subproject commit e589cc0288b139509a19f3a27cd0cf0368ca0834 +Subproject commit 9f5c274667b054d46997cf18dcf8573eff028f90