gossipsub / floodsub fixes (#348)

* mcache fixes

* remove timed cache - the window shifting already removes old messages
* ref -> object
* avoid unnecessary allocations with `[]` operator

* simplify init

* fix several gossipsub/floodsub issues

* floodsub, gossipsub: don't rebroadcast messages that fail validation
(!)
* floodsub, gossipsub: don't crash when unsubscribing from unknown
topics (!)
* gossipsub: don't send message to peers that are not interested in the
topic, when messages don't share topic list
* floodsub: don't repeat all messages for each message when
rebroadcasting
* floodsub: allow sending empty data
* floodsub: fix inefficient unsubscribe
* sync floodsub/gossipsub logging
* gossipsub: include incoming messages in mcache (!)
* gossipsub: don't rebroadcast already-seen messages (!)
* pubsubpeer: remove incoming/outgoing seen caches - these are already
handled in gossipsub, floodsub and will cause trouble when peers try to
resubscribe / regraft topics (because control messages will have same
digest)
* timedcache: reimplement without timers (fixes timer leaks and extreme
inefficiency due to per-message closures, futures etc)
* timedcache: ref -> obj
This commit is contained in:
Jacek Sieka 2020-09-04 08:10:32 +02:00 committed by GitHub
parent c0bc73ddac
commit 5819c6a9a7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 250 additions and 257 deletions

View File

@ -7,16 +7,17 @@
## This file may not be copied, modified, or distributed except according to ## This file may not be copied, modified, or distributed except according to
## those terms. ## those terms.
import sequtils, tables, sets, strutils import std/[sequtils, sets, tables]
import chronos, chronicles, metrics import chronos, chronicles, metrics
import pubsub, import ./pubsub,
pubsubpeer, ./pubsubpeer,
timedcache, ./timedcache,
peertable, ./peertable,
rpc/[messages, message], ./rpc/[message, messages],
../../stream/connection, ../../stream/connection,
../../peerid, ../../peerid,
../../peerinfo ../../peerinfo,
../../utility
logScope: logScope:
topics = "floodsub" topics = "floodsub"
@ -49,15 +50,13 @@ method subscribeTopic*(f: FloodSub,
method unsubscribePeer*(f: FloodSub, peer: PeerID) = method unsubscribePeer*(f: FloodSub, peer: PeerID) =
## handle peer disconnects ## handle peer disconnects
## ##
trace "unsubscribing floodsub peer", peer = $peer trace "unsubscribing floodsub peer", peer = $peer
let pubSubPeer = f.peers.getOrDefault(peer) let pubSubPeer = f.peers.getOrDefault(peer)
if pubSubPeer.isNil: if pubSubPeer.isNil:
return return
for t in toSeq(f.floodsub.keys): for _, v in f.floodsub.mpairs():
if t in f.floodsub: v.excl(pubSubPeer)
f.floodsub[t].excl(pubSubPeer)
procCall PubSub(f).unsubscribePeer(peer) procCall PubSub(f).unsubscribePeer(peer)
@ -66,35 +65,34 @@ method rpcHandler*(f: FloodSub,
rpcMsg: RPCMsg) {.async.} = rpcMsg: RPCMsg) {.async.} =
await procCall PubSub(f).rpcHandler(peer, rpcMsg) await procCall PubSub(f).rpcHandler(peer, rpcMsg)
if rpcMsg.messages.len > 0: # if there are any messages
var toSendPeers = initHashSet[PubSubPeer]()
for msg in rpcMsg.messages: # for every message for msg in rpcMsg.messages: # for every message
let msgId = f.msgIdProvider(msg) let msgId = f.msgIdProvider(msg)
logScope: msgId logScope:
msgId
peer = peer.id
if msgId notin f.seen: if f.seen.put(msgId):
f.seen.put(msgId) # add the message to the seen cache trace "Dropping already-seen message"
continue
if f.verifySignature and not msg.verify(peer.peerId): if f.verifySignature and not msg.verify(peer.peerId):
trace "dropping message due to failed signature verification" debug "Dropping message due to failed signature verification"
continue continue
if not (await f.validate(msg)): if not (await f.validate(msg)):
trace "dropping message due to failed validation" trace "Dropping message due to failed validation"
continue continue
var toSendPeers = initHashSet[PubSubPeer]()
for t in msg.topicIDs: # for every topic in the message for t in msg.topicIDs: # for every topic in the message
if t in f.floodsub: f.floodsub.withValue(t, peers): toSendPeers.incl(peers[])
toSendPeers.incl(f.floodsub[t]) # get all the peers interested in this topic
await handleData(f, t, msg.data) await handleData(f, t, msg.data)
# forward the message to all peers interested in it # In theory, if topics are the same in all messages, we could batch - we'd
f.broadcast( # also have to be careful to only include validated messages
toSeq(toSendPeers), f.broadcast(toSeq(toSendPeers), RPCMsg(messages: @[msg]))
RPCMsg(messages: rpcMsg.messages)) trace "Forwared message to peers", peers = toSendPeers.len
trace "forwared message to peers", peers = toSendPeers.len
method init*(f: FloodSub) = method init*(f: FloodSub) =
proc handler(conn: Connection, proto: string) {.async.} = proc handler(conn: Connection, proto: string) {.async.} =
@ -114,30 +112,41 @@ method publish*(f: FloodSub,
# base returns always 0 # base returns always 0
discard await procCall PubSub(f).publish(topic, data) discard await procCall PubSub(f).publish(topic, data)
if data.len <= 0 or topic.len <= 0: logScope: topic
trace "topic or data missing, skipping publish" trace "Publishing message on topic", data = data.shortLog
if topic.len <= 0: # data could be 0/empty
debug "Empty topic, skipping publish"
return 0 return 0
if topic notin f.floodsub: let peers = toSeq(f.floodsub.getOrDefault(topic))
trace "missing peers for topic, skipping publish"
return if peers.len == 0:
debug "No peers for topic, skipping publish"
return 0
trace "publishing on topic", name = topic
inc f.msgSeqno inc f.msgSeqno
let let
msg = Message.init(f.peerInfo, data, topic, f.msgSeqno, f.sign) msg = Message.init(f.peerInfo, data, topic, f.msgSeqno, f.sign)
peers = toSeq(f.floodsub.getOrDefault(topic)) msgId = f.msgIdProvider(msg)
# start the future but do not wait yet logScope: msgId
f.broadcast(
peers, trace "Created new message", msg = shortLog(msg), peers = peers.len
RPCMsg(messages: @[msg]))
if f.seen.put(msgId):
# custom msgid providers might cause this
trace "Dropping already-seen message"
return 0
# Try to send to all peers that are known to be interested
f.broadcast(peers, RPCMsg(messages: @[msg]))
when defined(libp2p_expensive_metrics): when defined(libp2p_expensive_metrics):
libp2p_pubsub_messages_published.inc(labelValues = [topic]) libp2p_pubsub_messages_published.inc(labelValues = [topic])
trace "published message to peers", peers = peers.len, trace "Published message to peers"
msg = msg.shortLog()
return peers.len return peers.len
method unsubscribe*(f: FloodSub, method unsubscribe*(f: FloodSub,
@ -156,5 +165,5 @@ method unsubscribeAll*(f: FloodSub, topic: string) {.async.} =
method initPubSub*(f: FloodSub) = method initPubSub*(f: FloodSub) =
procCall PubSub(f).initPubSub() procCall PubSub(f).initPubSub()
f.floodsub = initTable[string, HashSet[PubSubPeer]]() f.floodsub = initTable[string, HashSet[PubSubPeer]]()
f.seen = newTimedCache[string](2.minutes) f.seen = TimedCache[string].init(2.minutes)
f.init() f.init()

View File

@ -7,20 +7,19 @@
## This file may not be copied, modified, or distributed except according to ## This file may not be copied, modified, or distributed except according to
## those terms. ## those terms.
import std/[tables, sets, options, sequtils, random] import std/[options, random, sequtils, sets, tables]
import chronos, chronicles, metrics import chronos, chronicles, metrics
import pubsub, import ./pubsub,
floodsub, ./floodsub,
pubsubpeer, ./pubsubpeer,
peertable, ./peertable,
mcache, ./mcache,
timedcache, ./timedcache,
rpc/[messages, message], ./rpc/[messages, message],
../protocol, ../protocol,
../../peerinfo,
../../stream/connection, ../../stream/connection,
../../peerinfo,
../../peerid, ../../peerid,
../../errors,
../../utility ../../utility
logScope: logScope:
@ -379,47 +378,36 @@ method rpcHandler*(g: GossipSub,
rpcMsg: RPCMsg) {.async.} = rpcMsg: RPCMsg) {.async.} =
await procCall PubSub(g).rpcHandler(peer, rpcMsg) await procCall PubSub(g).rpcHandler(peer, rpcMsg)
if rpcMsg.messages.len > 0: # if there are any messages
var toSendPeers: HashSet[PubSubPeer]
for msg in rpcMsg.messages: # for every message for msg in rpcMsg.messages: # for every message
let msgId = g.msgIdProvider(msg) let msgId = g.msgIdProvider(msg)
logScope: msgId logScope:
msgId
peer = peer.id
if msgId in g.seen: if g.seen.put(msgId):
trace "message already processed, skipping" trace "Dropping already-seen message"
continue continue
trace "processing message" g.mcache.put(msgId, msg)
g.seen.put(msgId) # add the message to the seen cache
if g.verifySignature and not msg.verify(peer.peerId): if g.verifySignature and not msg.verify(peer.peerId):
trace "dropping message due to failed signature verification" debug "Dropping message due to failed signature verification"
continue continue
if not (await g.validate(msg)): if not (await g.validate(msg)):
trace "dropping message due to failed validation" trace "Dropping message due to failed validation"
continue
# this shouldn't happen
if g.peerInfo.peerId == msg.fromPeer:
trace "skipping messages from self"
continue continue
var toSendPeers = initHashSet[PubSubPeer]()
for t in msg.topicIDs: # for every topic in the message for t in msg.topicIDs: # for every topic in the message
if t in g.floodsub: g.floodsub.withValue(t, peers): toSendPeers.incl(peers[])
toSendPeers.incl(g.floodsub[t]) # get all floodsub peers for topic g.mesh.withValue(t, peers): toSendPeers.incl(peers[])
if t in g.mesh:
toSendPeers.incl(g.mesh[t]) # get all mesh peers for topic
await handleData(g, t, msg.data) await handleData(g, t, msg.data)
# forward the message to all peers interested in it # In theory, if topics are the same in all messages, we could batch - we'd
g.broadcast( # also have to be careful to only include validated messages
toSeq(toSendPeers), g.broadcast(toSeq(toSendPeers), RPCMsg(messages: @[msg]))
RPCMsg(messages: rpcMsg.messages))
trace "forwared message to peers", peers = toSendPeers.len trace "forwared message to peers", peers = toSendPeers.len
if rpcMsg.control.isSome: if rpcMsg.control.isSome:
@ -451,12 +439,13 @@ method unsubscribe*(g: GossipSub,
for (topic, handler) in topics: for (topic, handler) in topics:
# delete from mesh only if no handlers are left # delete from mesh only if no handlers are left
if g.topics[topic].handler.len <= 0: if topic notin g.topics:
if topic in g.mesh: if topic in g.mesh:
let peers = g.mesh.getOrDefault(topic) let peers = g.mesh[topic]
g.mesh.del(topic) g.mesh.del(topic)
let prune = RPCMsg(control: some(ControlMessage(prune: @[ControlPrune(topicID: topic)]))) let prune = RPCMsg(
control: some(ControlMessage(prune: @[ControlPrune(topicID: topic)])))
g.broadcast(toSeq(peers), prune) g.broadcast(toSeq(peers), prune)
method unsubscribeAll*(g: GossipSub, topic: string) {.async.} = method unsubscribeAll*(g: GossipSub, topic: string) {.async.} =
@ -474,9 +463,12 @@ method publish*(g: GossipSub,
data: seq[byte]): Future[int] {.async.} = data: seq[byte]): Future[int] {.async.} =
# base returns always 0 # base returns always 0
discard await procCall PubSub(g).publish(topic, data) discard await procCall PubSub(g).publish(topic, data)
trace "publishing message on topic", topic, data = data.shortLog
logScope: topic
trace "Publishing message on topic", data = data.shortLog
if topic.len <= 0: # data could be 0/empty if topic.len <= 0: # data could be 0/empty
debug "Empty topic, skipping publish"
return 0 return 0
var peers: HashSet[PubSubPeer] var peers: HashSet[PubSubPeer]
@ -497,28 +489,34 @@ method publish*(g: GossipSub,
# time # time
g.lastFanoutPubSub[topic] = Moment.fromNow(GossipSubFanoutTTL) g.lastFanoutPubSub[topic] = Moment.fromNow(GossipSubFanoutTTL)
if peers.len == 0:
debug "No peers for topic, skipping publish"
return 0
inc g.msgSeqno inc g.msgSeqno
let let
msg = Message.init(g.peerInfo, data, topic, g.msgSeqno, g.sign) msg = Message.init(g.peerInfo, data, topic, g.msgSeqno, g.sign)
msgId = g.msgIdProvider(msg) msgId = g.msgIdProvider(msg)
trace "created new message", msg, topic, peers = peers.len logScope: msgId
trace "Created new message", msg = shortLog(msg), peers = peers.len
if g.seen.put(msgId):
# custom msgid providers might cause this
trace "Dropping already-seen message"
return 0
if msgId notin g.mcache:
g.mcache.put(msgId, msg) g.mcache.put(msgId, msg)
if peers.len > 0:
g.broadcast(toSeq(peers), RPCMsg(messages: @[msg])) g.broadcast(toSeq(peers), RPCMsg(messages: @[msg]))
when defined(libp2p_expensive_metrics): when defined(libp2p_expensive_metrics):
if peers.len > 0: if peers.len > 0:
libp2p_pubsub_messages_published.inc(labelValues = [topic]) libp2p_pubsub_messages_published.inc(labelValues = [topic])
trace "published message to peers", peers = peers.len, trace "Published message to peers"
msg = msg.shortLog()
return peers.len return peers.len
else:
debug "No peers for gossip message", topic, msg = msg.shortLog()
return 0
method start*(g: GossipSub) {.async.} = method start*(g: GossipSub) {.async.} =
trace "gossipsub start" trace "gossipsub start"
@ -556,7 +554,7 @@ method initPubSub*(g: GossipSub) =
procCall FloodSub(g).initPubSub() procCall FloodSub(g).initPubSub()
randomize() randomize()
g.mcache = newMCache(GossipSubHistoryGossip, GossipSubHistoryLength) g.mcache = MCache.init(GossipSubHistoryGossip, GossipSubHistoryLength)
g.mesh = initTable[string, HashSet[PubSubPeer]]() # meshes - topic to peer g.mesh = initTable[string, HashSet[PubSubPeer]]() # meshes - topic to peer
g.fanout = initTable[string, HashSet[PubSubPeer]]() # fanout - topic to peer g.fanout = initTable[string, HashSet[PubSubPeer]]() # fanout - topic to peer
g.gossipsub = initTable[string, HashSet[PubSubPeer]]()# topic to peer map of all gossipsub peers g.gossipsub = initTable[string, HashSet[PubSubPeer]]()# topic to peer map of all gossipsub peers

View File

@ -7,69 +7,57 @@
## This file may not be copied, modified, or distributed except according to ## This file may not be copied, modified, or distributed except according to
## those terms. ## those terms.
import chronos, chronicles import std/[sets, tables, options]
import tables, options, sets, sequtils import rpc/[messages]
import rpc/[messages], timedcache
export sets, tables, messages, options
type type
CacheEntry* = object CacheEntry* = object
mid*: string mid*: string
msg*: Message topicIDs*: seq[string]
MCache* = ref object of RootObj MCache* = object of RootObj
msgs*: TimedCache[Message] msgs*: Table[string, Message]
history*: seq[seq[CacheEntry]] history*: seq[seq[CacheEntry]]
historySize*: Natural historySize*: Natural
windowSize*: Natural windowSize*: Natural
proc get*(c: MCache, mid: string): Option[Message] = func get*(c: MCache, mid: string): Option[Message] =
result = none(Message) result = none(Message)
if mid in c.msgs: if mid in c.msgs:
result = some(c.msgs[mid]) result = some(c.msgs[mid])
proc contains*(c: MCache, mid: string): bool = func contains*(c: MCache, mid: string): bool =
c.get(mid).isSome c.get(mid).isSome
proc put*(c: MCache, msgId: string, msg: Message) = func put*(c: var MCache, msgId: string, msg: Message) =
proc handler(key: string, val: Message) {.gcsafe.} =
## make sure we remove the message from history
## to keep things consisten
c.history.applyIt(
it.filterIt(it.mid != msgId)
)
if msgId notin c.msgs: if msgId notin c.msgs:
c.msgs.put(msgId, msg, handler = handler) c.msgs[msgId] = msg
c.history[0].add(CacheEntry(mid: msgId, msg: msg)) c.history[0].add(CacheEntry(mid: msgId, topicIDs: msg.topicIDs))
proc window*(c: MCache, topic: string): HashSet[string] = func window*(c: MCache, topic: string): HashSet[string] =
result = initHashSet[string]() result = initHashSet[string]()
let len = let
if c.windowSize > c.history.len: len = min(c.windowSize, c.history.len)
c.history.len
else:
c.windowSize
if c.history.len > 0: for i in 0..<len:
for slot in c.history[0..<len]: for entry in c.history[i]:
for entry in slot: for t in entry.topicIDs:
for t in entry.msg.topicIDs:
if t == topic: if t == topic:
result.incl(entry.mid) result.incl(entry.mid)
break break
proc shift*(c: MCache) = func shift*(c: var MCache) =
while c.history.len > c.historySize:
for entry in c.history.pop(): for entry in c.history.pop():
c.msgs.del(entry.mid) c.msgs.del(entry.mid)
c.history.insert(@[]) c.history.insert(@[])
proc newMCache*(window: Natural, history: Natural): MCache = func init*(T: type MCache, window, history: Natural): T =
new result T(
result.historySize = history history: newSeq[seq[CacheEntry]](history),
result.windowSize = window historySize: history,
result.history = newSeq[seq[CacheEntry]]() windowSize: window
result.history.add(@[]) # initialize with empty slot )
result.msgs = newTimedCache[Message](2.minutes)

View File

@ -201,14 +201,15 @@ method unsubscribe*(p: PubSub,
topics: seq[TopicPair]) {.base, async.} = topics: seq[TopicPair]) {.base, async.} =
## unsubscribe from a list of ``topic`` strings ## unsubscribe from a list of ``topic`` strings
for t in topics: for t in topics:
for i, h in p.topics[t.topic].handler: p.topics.withValue(t.topic, subs):
for i, h in subs[].handler:
if h == t.handler: if h == t.handler:
p.topics[t.topic].handler.del(i) subs[].handler.del(i)
# make sure we delete the topic if # make sure we delete the topic if
# no more handlers are left # no more handlers are left
if p.topics[t.topic].handler.len <= 0: if subs.handler.len <= 0:
p.topics.del(t.topic) p.topics.del(t.topic) # careful, invalidates subs
# metrics # metrics
libp2p_pubsub_topics.set(p.topics.len.int64) libp2p_pubsub_topics.set(p.topics.len.int64)

View File

@ -10,8 +10,6 @@
import std/[hashes, options, strutils, tables] import std/[hashes, options, strutils, tables]
import chronos, chronicles, nimcrypto/sha2, metrics import chronos, chronicles, nimcrypto/sha2, metrics
import rpc/[messages, message, protobuf], import rpc/[messages, message, protobuf],
timedcache,
../../switch,
../../peerid, ../../peerid,
../../peerinfo, ../../peerinfo,
../../stream/connection, ../../stream/connection,
@ -42,8 +40,6 @@ type
connections*: seq[Connection] # connections to this peer connections*: seq[Connection] # connections to this peer
peerId*: PeerID peerId*: PeerID
handler*: RPCHandler handler*: RPCHandler
sentRpcCache: TimedCache[string] # cache for already sent messages
recvdRpcCache: TimedCache[string] # cache for already received messages
observers*: ref seq[PubSubObserver] # ref as in smart_ptr observers*: ref seq[PubSubObserver] # ref as in smart_ptr
dialLock: AsyncLock dialLock: AsyncLock
@ -87,33 +83,24 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} =
while not conn.atEof: while not conn.atEof:
trace "waiting for data" trace "waiting for data"
let data = await conn.readLp(64 * 1024) let data = await conn.readLp(64 * 1024)
let digest = $(sha256.digest(data))
trace "read data from peer", data = data.shortLog trace "read data from peer", data = data.shortLog
if digest in p.recvdRpcCache:
when defined(libp2p_expensive_metrics):
libp2p_pubsub_skipped_received_messages.inc(labelValues = [p.id])
trace "message already received, skipping"
continue
var rmsg = decodeRpcMsg(data) var rmsg = decodeRpcMsg(data)
if rmsg.isErr(): if rmsg.isErr():
notice "failed to decode msg from peer" notice "failed to decode msg from peer"
break break
var msg = rmsg.get() trace "decoded msg from peer", msg = rmsg.get().shortLog
trace "decoded msg from peer", msg = msg.shortLog
# trigger hooks # trigger hooks
p.recvObservers(msg) p.recvObservers(rmsg.get())
when defined(libp2p_expensive_metrics): when defined(libp2p_expensive_metrics):
for m in msg.messages: for m in rmsg.get().messages:
for t in m.topicIDs: for t in m.topicIDs:
# metrics # metrics
libp2p_pubsub_received_messages.inc(labelValues = [p.id, t]) libp2p_pubsub_received_messages.inc(labelValues = [p.id, t])
await p.handler(p, msg) await p.handler(p, rmsg.get())
p.recvdRpcCache.put(digest)
finally: finally:
await conn.close() await conn.close()
@ -227,13 +214,6 @@ proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} =
logScope: logScope:
encoded = shortLog(encoded) encoded = shortLog(encoded)
let digest = $(sha256.digest(encoded))
if digest in p.sentRpcCache:
trace "message already sent to peer, skipping"
when defined(libp2p_expensive_metrics):
libp2p_pubsub_skipped_sent_messages.inc(labelValues = [p.id])
return
var conn = await p.getSendConn() var conn = await p.getSendConn()
try: try:
trace "about to send message" trace "about to send message"
@ -243,8 +223,6 @@ proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} =
trace "sending encoded msgs to peer", connId = $conn.oid trace "sending encoded msgs to peer", connId = $conn.oid
await conn.writeLp(encoded) await conn.writeLp(encoded)
p.sentRpcCache.put(digest)
trace "sent pubsub message to remote", connId = $conn.oid trace "sent pubsub message to remote", connId = $conn.oid
when defined(libp2p_expensive_metrics): when defined(libp2p_expensive_metrics):
@ -282,6 +260,4 @@ proc newPubSubPeer*(peerId: PeerID,
result.getConn = getConn result.getConn = getConn
result.codec = codec result.codec = codec
result.peerId = peerId result.peerId = peerId
result.sentRpcCache = newTimedCache[string](2.minutes)
result.recvdRpcCache = newTimedCache[string](2.minutes)
result.dialLock = newAsyncLock() result.dialLock = newAsyncLock()

View File

@ -7,73 +7,59 @@
## This file may not be copied, modified, or distributed except according to ## This file may not be copied, modified, or distributed except according to
## those terms. ## those terms.
import tables import std/[heapqueue, sets]
import chronos, chronicles
logScope: import chronos/timer
topics = "timedcache"
const Timeout* = 10.seconds # default timeout in ms const Timeout* = 10.seconds # default timeout in ms
type type
ExpireHandler*[V] = proc(key: string, val: V) {.gcsafe.} TimedEntry*[K] = ref object of RootObj
TimedEntry*[V] = object of RootObj key: K
val: V expiresAt: Moment
handler: ExpireHandler[V]
TimedCache*[V] = ref object of RootObj TimedCache*[K] = object of RootObj
cache*: Table[string, TimedEntry[V]] expiries: HeapQueue[TimedEntry[K]]
onExpire*: ExpireHandler[V] entries: HashSet[K]
timeout*: Duration timeout: Duration
# TODO: This belong in chronos, temporary left here until chronos is updated func `<`*(a, b: TimedEntry): bool =
proc addTimer*(at: Duration, cb: CallbackFunc, udata: pointer = nil) = a.expiresAt < b.expiresAt
## Arrange for the callback ``cb`` to be called at the given absolute
## timestamp ``at``. You can also pass ``udata`` to callback.
addTimer(Moment.fromNow(at), cb, udata)
proc put*[V](t: TimedCache[V], func expire*(t: var TimedCache, now: Moment = Moment.now()) =
key: string, while t.expiries.len() > 0 and t.expiries[0].expiresAt < now:
val: V = "", t.entries.excl(t.expiries.pop().key)
timeout: Duration,
handler: ExpireHandler[V] = nil) =
trace "adding entry to timed cache", key = key
t.cache[key] = TimedEntry[V](val: val, handler: handler)
addTimer( func del*[K](t: var TimedCache[K], key: K): bool =
timeout, # Removes existing key from cache, returning false if it was not present
proc (arg: pointer = nil) {.gcsafe.} = if not t.entries.missingOrExcl(key):
trace "deleting expired entry from timed cache", key = key for i in 0..<t.expiries.len:
if key in t.cache: if t.expiries[i].key == key:
let entry = t.cache[key] t.expiries.del(i)
t.cache.del(key) break
if not isNil(entry.handler): true
entry.handler(key, entry.val) else:
false
func put*[K](t: var TimedCache[K], k: K, now = Moment.now()): bool =
# Puts k in cache, returning true if the item was already present and false
# otherwise. If the item was already present, its expiry timer will be
# refreshed.
t.expire(now)
var res = t.del(k) # Refresh existing item
t.entries.incl(k)
t.expiries.push(TimedEntry[K](key: k, expiresAt: now + t.timeout))
res
func contains*[K](t: TimedCache[K], k: K): bool =
k in t.entries
func init*[K](T: type TimedCache[K], timeout: Duration = Timeout): T =
T(
expiries: initHeapQueue[TimedEntry[K]](),
entries: initHashSet[K](),
timeout: timeout
) )
proc put*[V](t: TimedCache[V],
key: string,
val: V = "",
handler: ExpireHandler[V] = nil) =
t.put(key, val, t.timeout, handler)
proc contains*[V](t: TimedCache[V], key: string): bool =
t.cache.contains(key)
proc del*[V](t: TimedCache[V], key: string) =
trace "deleting entry from timed cache", key = key
t.cache.del(key)
proc get*[V](t: TimedCache[V], key: string): V =
t.cache[key].val
proc `[]`*[V](t: TimedCache[V], key: string): V =
t.get(key)
proc `[]=`*[V](t: TimedCache[V], key: string, val: V): V =
t.put(key, val)
proc newTimedCache*[V](timeout: Duration = Timeout): TimedCache[V] =
new result
result.cache = initTable[string, TimedEntry[V]]()
result.timeout = timeout

View File

@ -15,14 +15,14 @@ proc randomPeerID(): PeerID =
suite "MCache": suite "MCache":
test "put/get": test "put/get":
var mCache = newMCache(3, 5) var mCache = MCache.init(3, 5)
var msg = Message(fromPeer: randomPeerID(), seqno: "12345".toBytes()) var msg = Message(fromPeer: randomPeerID(), seqno: "12345".toBytes())
let msgId = defaultMsgIdProvider(msg) let msgId = defaultMsgIdProvider(msg)
mCache.put(msgId, msg) mCache.put(msgId, msg)
check mCache.get(msgId).isSome and mCache.get(msgId).get() == msg check mCache.get(msgId).isSome and mCache.get(msgId).get() == msg
test "window": test "window":
var mCache = newMCache(3, 5) var mCache = MCache.init(3, 5)
for i in 0..<3: for i in 0..<3:
var msg = Message(fromPeer: randomPeerID(), var msg = Message(fromPeer: randomPeerID(),
@ -43,7 +43,7 @@ suite "MCache":
check mCache.get(id).get().topicIDs[0] == "foo" check mCache.get(id).get().topicIDs[0] == "foo"
test "shift - shift 1 window at a time": test "shift - shift 1 window at a time":
var mCache = newMCache(1, 5) var mCache = MCache.init(1, 5)
for i in 0..<3: for i in 0..<3:
var msg = Message(fromPeer: randomPeerID(), var msg = Message(fromPeer: randomPeerID(),
@ -73,7 +73,7 @@ suite "MCache":
check mCache.window("baz").len == 0 check mCache.window("baz").len == 0
test "shift - 2 windows at a time": test "shift - 2 windows at a time":
var mCache = newMCache(1, 5) var mCache = MCache.init(1, 5)
for i in 0..<3: for i in 0..<3:
var msg = Message(fromPeer: randomPeerID(), var msg = Message(fromPeer: randomPeerID(),

View File

@ -4,4 +4,5 @@ import testgossipinternal,
testfloodsub, testfloodsub,
testgossipsub, testgossipsub,
testmcache, testmcache,
testtimedcache,
testmessage testmessage

View File

@ -0,0 +1,34 @@
{.used.}
import std/unittest
import chronos/timer
import ../../libp2p/protocols/pubsub/timedcache
suite "TimedCache":
test "put/get":
var cache = TimedCache[int].init(5.seconds)
let now = Moment.now()
check:
not cache.put(1, now)
not cache.put(2, now + 3.seconds)
check:
1 in cache
2 in cache
check: not cache.put(3, now + 6.seconds) # expires 1
check:
1 notin cache
2 in cache
3 in cache
check:
cache.put(2, now + 7.seconds) # refreshes 2
not cache.put(4, now + 12.seconds) # expires 3
check:
2 in cache
3 notin cache
4 in cache