From cd1c68dbc57b497daa581e0d095ed376bbceddc3 Mon Sep 17 00:00:00 2001 From: Jacek Sieka Date: Tue, 1 Sep 2020 09:33:03 +0200 Subject: [PATCH 01/10] avoid send deadlock by not allowing send to block (#342) * avoid send deadlock by not allowing send to block * handle message issues more consistently --- libp2p/protocols/pubsub/floodsub.nim | 86 ++++++-------- libp2p/protocols/pubsub/gossipsub.nim | 141 ++++++++++------------- libp2p/protocols/pubsub/pubsub.nim | 116 ++++++++----------- libp2p/protocols/pubsub/pubsubpeer.nim | 80 ++++++++----- libp2p/protocols/pubsub/rpc/messages.nim | 5 + libp2p/stream/bufferstream.nim | 2 +- tests/pubsub/testgossipinternal.nim | 41 ++++--- 7 files changed, 225 insertions(+), 246 deletions(-) diff --git a/libp2p/protocols/pubsub/floodsub.nim b/libp2p/protocols/pubsub/floodsub.nim index 382db1fd0..3562c84e8 100644 --- a/libp2p/protocols/pubsub/floodsub.nim +++ b/libp2p/protocols/pubsub/floodsub.nim @@ -63,50 +63,38 @@ method unsubscribePeer*(f: FloodSub, peer: PeerID) = method rpcHandler*(f: FloodSub, peer: PubSubPeer, - rpcMsgs: seq[RPCMsg]) {.async.} = - await procCall PubSub(f).rpcHandler(peer, rpcMsgs) + rpcMsg: RPCMsg) {.async.} = + await procCall PubSub(f).rpcHandler(peer, rpcMsg) - for m in rpcMsgs: # for all RPC messages - if m.messages.len > 0: # if there are any messages - var toSendPeers = initHashSet[PubSubPeer]() - for msg in m.messages: # for every message - let msgId = f.msgIdProvider(msg) - logScope: msgId + if rpcMsg.messages.len > 0: # if there are any messages + var toSendPeers = initHashSet[PubSubPeer]() + for msg in rpcMsg.messages: # for every message + let msgId = f.msgIdProvider(msg) + logScope: msgId - if msgId notin f.seen: - f.seen.put(msgId) # add the message to the seen cache + if msgId notin f.seen: + f.seen.put(msgId) # add the message to the seen cache - if f.verifySignature and not msg.verify(peer.peerId): - trace "dropping message due to failed signature verification" - continue + if f.verifySignature and not msg.verify(peer.peerId): + trace "dropping message due to failed signature verification" + continue - if not (await f.validate(msg)): - trace "dropping message due to failed validation" - continue + if not (await f.validate(msg)): + trace "dropping message due to failed validation" + continue - for t in msg.topicIDs: # for every topic in the message - if t in f.floodsub: - toSendPeers.incl(f.floodsub[t]) # get all the peers interested in this topic - if t in f.topics: # check that we're subscribed to it - for h in f.topics[t].handler: - trace "calling handler for message", topicId = t, - localPeer = f.peerInfo.id, - fromPeer = msg.fromPeer.pretty + for t in msg.topicIDs: # for every topic in the message + if t in f.floodsub: + toSendPeers.incl(f.floodsub[t]) # get all the peers interested in this topic - try: - await h(t, msg.data) # trigger user provided handler - except CancelledError as exc: - raise exc - except CatchableError as exc: - trace "exception in message handler", exc = exc.msg + await handleData(f, t, msg.data) - # forward the message to all peers interested in it - let published = await f.broadcast( - toSeq(toSendPeers), - RPCMsg(messages: m.messages), - DefaultSendTimeout) + # forward the message to all peers interested in it + f.broadcast( + toSeq(toSendPeers), + RPCMsg(messages: rpcMsg.messages)) - trace "forwared message to peers", peers = published + trace "forwared message to peers", peers = toSendPeers.len method init*(f: FloodSub) = proc handler(conn: Connection, proto: string) {.async.} = @@ -122,14 +110,13 @@ method init*(f: FloodSub) = method publish*(f: FloodSub, topic: string, - data: seq[byte], - timeout: Duration = InfiniteDuration): Future[int] {.async.} = + data: seq[byte]): Future[int] {.async.} = # base returns always 0 - discard await procCall PubSub(f).publish(topic, data, timeout) + discard await procCall PubSub(f).publish(topic, data) if data.len <= 0 or topic.len <= 0: trace "topic or data missing, skipping publish" - return + return 0 if topic notin f.floodsub: trace "missing peers for topic, skipping publish" @@ -137,33 +124,34 @@ method publish*(f: FloodSub, trace "publishing on topic", name = topic inc f.msgSeqno - let msg = Message.init(f.peerInfo, data, topic, f.msgSeqno, f.sign) + let + msg = Message.init(f.peerInfo, data, topic, f.msgSeqno, f.sign) + peers = toSeq(f.floodsub.getOrDefault(topic)) # start the future but do not wait yet - let published = await f.broadcast( - toSeq(f.floodsub.getOrDefault(topic)), - RPCMsg(messages: @[msg]), - timeout) + f.broadcast( + peers, + RPCMsg(messages: @[msg])) when defined(libp2p_expensive_metrics): libp2p_pubsub_messages_published.inc(labelValues = [topic]) - trace "published message to peers", peers = published, + trace "published message to peers", peers = peers.len, msg = msg.shortLog() - return published + return peers.len method unsubscribe*(f: FloodSub, topics: seq[TopicPair]) {.async.} = await procCall PubSub(f).unsubscribe(topics) for p in f.peers.values: - discard await f.sendSubs(p, topics.mapIt(it.topic).deduplicate(), false) + f.sendSubs(p, topics.mapIt(it.topic).deduplicate(), false) method unsubscribeAll*(f: FloodSub, topic: string) {.async.} = await procCall PubSub(f).unsubscribeAll(topic) for p in f.peers.values: - discard await f.sendSubs(p, @[topic], false) + f.sendSubs(p, @[topic], false) method initPubSub*(f: FloodSub) = procCall PubSub(f).initPubSub() diff --git a/libp2p/protocols/pubsub/gossipsub.nim b/libp2p/protocols/pubsub/gossipsub.nim index fbda35dcc..3714cfe7d 100644 --- a/libp2p/protocols/pubsub/gossipsub.nim +++ b/libp2p/protocols/pubsub/gossipsub.nim @@ -157,10 +157,10 @@ proc rebalanceMesh(g: GossipSub, topic: string) {.async.} = # Send changes to peers after table updates to avoid stale state if grafts.len > 0: let graft = RPCMsg(control: some(ControlMessage(graft: @[ControlGraft(topicID: topic)]))) - discard await g.broadcast(grafts, graft, DefaultSendTimeout) + g.broadcast(grafts, graft) if prunes.len > 0: let prune = RPCMsg(control: some(ControlMessage(prune: @[ControlPrune(topicID: topic)]))) - discard await g.broadcast(prunes, prune, DefaultSendTimeout) + g.broadcast(prunes, prune) proc dropFanoutPeers(g: GossipSub) = # drop peers that we haven't published to in @@ -229,14 +229,11 @@ proc heartbeat(g: GossipSub) {.async.} = g.replenishFanout(t) let peers = g.getGossipPeers() - var sent: seq[Future[bool]] for peer, control in peers: g.peers.withValue(peer.peerId, pubsubPeer) do: - sent &= g.send( + g.send( pubsubPeer[], - RPCMsg(control: some(control)), - DefaultSendTimeout) - checkFutures(await allFinished(sent)) + RPCMsg(control: some(control))) g.mcache.shift() # shift the cache except CancelledError as exc: @@ -379,89 +376,68 @@ proc handleIWant(g: GossipSub, method rpcHandler*(g: GossipSub, peer: PubSubPeer, - rpcMsgs: seq[RPCMsg]) {.async.} = - await procCall PubSub(g).rpcHandler(peer, rpcMsgs) + rpcMsg: RPCMsg) {.async.} = + await procCall PubSub(g).rpcHandler(peer, rpcMsg) - for m in rpcMsgs: # for all RPC messages - if m.messages.len > 0: # if there are any messages - var toSendPeers: HashSet[PubSubPeer] - for msg in m.messages: # for every message - let msgId = g.msgIdProvider(msg) - logScope: msgId + if rpcMsg.messages.len > 0: # if there are any messages + var toSendPeers: HashSet[PubSubPeer] + for msg in rpcMsg.messages: # for every message + let msgId = g.msgIdProvider(msg) + logScope: msgId - if msgId in g.seen: - trace "message already processed, skipping" - continue + if msgId in g.seen: + trace "message already processed, skipping" + continue - trace "processing message" + trace "processing message" - g.seen.put(msgId) # add the message to the seen cache + g.seen.put(msgId) # add the message to the seen cache - if g.verifySignature and not msg.verify(peer.peerId): - trace "dropping message due to failed signature verification" - continue + if g.verifySignature and not msg.verify(peer.peerId): + trace "dropping message due to failed signature verification" + continue - if not (await g.validate(msg)): - trace "dropping message due to failed validation" - continue + if not (await g.validate(msg)): + trace "dropping message due to failed validation" + continue - # this shouldn't happen - if g.peerInfo.peerId == msg.fromPeer: - trace "skipping messages from self" - continue + # this shouldn't happen + if g.peerInfo.peerId == msg.fromPeer: + trace "skipping messages from self" + continue - for t in msg.topicIDs: # for every topic in the message - if t in g.floodsub: - toSendPeers.incl(g.floodsub[t]) # get all floodsub peers for topic + for t in msg.topicIDs: # for every topic in the message + if t in g.floodsub: + toSendPeers.incl(g.floodsub[t]) # get all floodsub peers for topic - if t in g.mesh: - toSendPeers.incl(g.mesh[t]) # get all mesh peers for topic + if t in g.mesh: + toSendPeers.incl(g.mesh[t]) # get all mesh peers for topic - if t in g.topics: # if we're subscribed to the topic - for h in g.topics[t].handler: - trace "calling handler for message", topicId = t, - localPeer = g.peerInfo.id, - fromPeer = msg.fromPeer.pretty - try: - await h(t, msg.data) # trigger user provided handler - except CancelledError as exc: - raise exc - except CatchableError as exc: - trace "exception in message handler", exc = exc.msg + await handleData(g, t, msg.data) - # forward the message to all peers interested in it - let published = await g.broadcast( - toSeq(toSendPeers), - RPCMsg(messages: m.messages), - DefaultSendTimeout) + # forward the message to all peers interested in it + g.broadcast( + toSeq(toSendPeers), + RPCMsg(messages: rpcMsg.messages)) - trace "forwared message to peers", peers = published + trace "forwared message to peers", peers = toSendPeers.len + + if rpcMsg.control.isSome: + let control = rpcMsg.control.get() + g.handlePrune(peer, control.prune) var respControl: ControlMessage - if m.control.isSome: - let control = m.control.get() - g.handlePrune(peer, control.prune) + respControl.iwant.add(g.handleIHave(peer, control.ihave)) + respControl.prune.add(g.handleGraft(peer, control.graft)) + let messages = g.handleIWant(peer, control.iwant) - respControl.iwant.add(g.handleIHave(peer, control.ihave)) - respControl.prune.add(g.handleGraft(peer, control.graft)) - let messages = g.handleIWant(peer, control.iwant) + if respControl.graft.len > 0 or respControl.prune.len > 0 or + respControl.ihave.len > 0 or messages.len > 0: - if respControl.graft.len > 0 or respControl.prune.len > 0 or - respControl.ihave.len > 0: - try: - info "sending control message", msg = respControl - let sent = await g.send( - peer, - RPCMsg(control: some(respControl), messages: messages), - DefaultSendTimeout) - - if not sent: - g.unsubscribePeer(peer.peerId) - - except CancelledError as exc: - raise exc - except CatchableError as exc: - trace "exception forwarding control messages", exc = exc.msg + debug "sending control message", msg = shortLog(respControl) + g.send( + peer, + RPCMsg(control: some(respControl), messages: messages)) method subscribe*(g: GossipSub, topic: string, @@ -481,7 +457,7 @@ method unsubscribe*(g: GossipSub, g.mesh.del(topic) let prune = RPCMsg(control: some(ControlMessage(prune: @[ControlPrune(topicID: topic)]))) - discard await g.broadcast(toSeq(peers), prune, DefaultSendTimeout) + g.broadcast(toSeq(peers), prune) method unsubscribeAll*(g: GossipSub, topic: string) {.async.} = await procCall PubSub(g).unsubscribeAll(topic) @@ -491,14 +467,13 @@ method unsubscribeAll*(g: GossipSub, topic: string) {.async.} = g.mesh.del(topic) let prune = RPCMsg(control: some(ControlMessage(prune: @[ControlPrune(topicID: topic)]))) - discard await g.broadcast(toSeq(peers), prune, DefaultSendTimeout) + g.broadcast(toSeq(peers), prune) method publish*(g: GossipSub, topic: string, - data: seq[byte], - timeout: Duration = InfiniteDuration): Future[int] {.async.} = + data: seq[byte]): Future[int] {.async.} = # base returns always 0 - discard await procCall PubSub(g).publish(topic, data, timeout) + discard await procCall PubSub(g).publish(topic, data) trace "publishing message on topic", topic, data = data.shortLog if topic.len <= 0: # data could be 0/empty @@ -533,14 +508,14 @@ method publish*(g: GossipSub, g.mcache.put(msgId, msg) if peers.len > 0: - let published = await g.broadcast(toSeq(peers), RPCMsg(messages: @[msg]), timeout) + g.broadcast(toSeq(peers), RPCMsg(messages: @[msg])) when defined(libp2p_expensive_metrics): - if published > 0: + if peers.len > 0: libp2p_pubsub_messages_published.inc(labelValues = [topic]) - trace "published message to peers", peers = published, + trace "published message to peers", peers = peers.len, msg = msg.shortLog() - return published + return peers.len else: debug "No peers for gossip message", topic, msg = msg.shortLog() return 0 diff --git a/libp2p/protocols/pubsub/pubsub.nim b/libp2p/protocols/pubsub/pubsub.nim index 8c4a6dff2..e3c242010 100644 --- a/libp2p/protocols/pubsub/pubsub.nim +++ b/libp2p/protocols/pubsub/pubsub.nim @@ -66,54 +66,38 @@ method unsubscribePeer*(p: PubSub, peerId: PeerID) {.base.} = ## trace "unsubscribing pubsub peer", peer = $peerId - if peerId in p.peers: - p.peers.del(peerId) + p.peers.del(peerId) libp2p_pubsub_peers.set(p.peers.len.int64) proc send*( p: PubSub, peer: PubSubPeer, - msg: RPCMsg, - timeout: Duration): Future[bool] {.async.} = + msg: RPCMsg) = ## send to remote peer ## trace "sending pubsub message to peer", peer = $peer, msg = shortLog(msg) - try: - await peer.send(msg, timeout) - return true - except CancelledError as exc: - raise exc - except CatchableError as exc: - trace "exception sending pubsub message to peer", - peer = $peer, msg = shortLog(msg) - p.unsubscribePeer(peer.peerId) + peer.send(msg) proc broadcast*( p: PubSub, - sendPeers: seq[PubSubPeer], - msg: RPCMsg, - timeout: Duration): Future[int] {.async.} = - ## send messages and cleanup failed peers + sendPeers: openArray[PubSubPeer], + msg: RPCMsg) = # raises: [Defect] + ## send messages - returns number of send attempts made. ## trace "broadcasting messages to peers", peers = sendPeers.len, message = shortLog(msg) - let sent = await allFinished( - sendPeers.mapIt( p.send(it, msg, timeout) )) - return sent.filterIt( it.finished and it.read ).len + for peer in sendPeers: + p.send(peer, msg) proc sendSubs*(p: PubSub, peer: PubSubPeer, topics: seq[string], - subscribe: bool): Future[bool] = + subscribe: bool) = ## send subscriptions to remote peer - p.send( - peer, - RPCMsg( - subscriptions: topics.mapIt(SubOpts(subscribe: subscribe, topic: it))), - DefaultSendTimeout) + p.send(peer, RPCMsg.withSubs(topics, subscribe)) method subscribeTopic*(p: PubSub, topic: string, @@ -124,16 +108,14 @@ method subscribeTopic*(p: PubSub, method rpcHandler*(p: PubSub, peer: PubSubPeer, - rpcMsgs: seq[RPCMsg]) {.async, base.} = + rpcMsg: RPCMsg) {.async, base.} = ## handle rpc messages - trace "processing RPC message", peer = peer.id, msgs = rpcMsgs.len + logScope: peer = peer.id - for m in rpcMsgs: # for all RPC messages - trace "processing messages", msg = m.shortLog - if m.subscriptions.len > 0: # if there are any subscriptions - for s in m.subscriptions: # subscribe/unsubscribe the peer for each topic - trace "about to subscribe to topic", topicId = s.topic - p.subscribeTopic(s.topic, s.subscribe, peer) + trace "processing RPC message", msg = rpcMsg.shortLog + for s in rpcMsg.subscriptions: # subscribe/unsubscribe the peer for each topic + trace "about to subscribe to topic", topicId = s.topic + p.subscribeTopic(s.topic, s.subscribe, peer) proc getOrCreatePeer*( p: PubSub, @@ -142,16 +124,36 @@ proc getOrCreatePeer*( if peer in p.peers: return p.peers[peer] + proc getConn(): Future[(Connection, RPCMsg)] {.async.} = + let conn = await p.switch.dial(peer, proto) + return (conn, RPCMsg.withSubs(toSeq(p.topics.keys), true)) + # create new pubsub peer - let pubSubPeer = newPubSubPeer(peer, p.switch, proto) + let pubSubPeer = newPubSubPeer(peer, getConn, proto) trace "created new pubsub peer", peerId = $peer p.peers[peer] = pubSubPeer pubSubPeer.observers = p.observers libp2p_pubsub_peers.set(p.peers.len.int64) + + pubsubPeer.connect() + return pubSubPeer +proc handleData*(p: PubSub, topic: string, data: seq[byte]): Future[void] {.async.} = + if topic notin p.topics: return # Not subscribed + + for h in p.topics[topic].handler: + trace "triggering handler", topicID = topic + try: + await h(topic, data) + except CancelledError as exc: + raise exc + except CatchableError as exc: + # Handlers should never raise exceptions + warn "Error in topic handler", msg = exc.msg + method handleConn*(p: PubSub, conn: Connection, proto: string) {.base, async.} = @@ -171,15 +173,13 @@ method handleConn*(p: PubSub, await conn.close() return - proc handler(peer: PubSubPeer, msgs: seq[RPCMsg]) {.async.} = + proc handler(peer: PubSubPeer, msg: RPCMsg): Future[void] = # call pubsub rpc handler - await p.rpcHandler(peer, msgs) + p.rpcHandler(peer, msg) + + let peer = p.getOrCreatePeer(conn.peerInfo.peerId, proto) try: - let peer = p.getOrCreatePeer(conn.peerInfo.peerId, proto) - if p.topics.len > 0: - discard await p.sendSubs(peer, toSeq(p.topics.keys), true) - peer.handler = handler await peer.handle(conn) # spawn peer read loop trace "pubsub peer handler ended", peer = peer.id @@ -195,15 +195,7 @@ method subscribePeer*(p: PubSub, peer: PeerID) {.base.} = ## messages ## - let pubsubPeer = p.getOrCreatePeer(peer, p.codec) - if p.topics.len > 0: - # TODO sendSubs may raise, but doing asyncCheck here causes the exception - # to escape to the poll loop. - # With a bit of luck, it may be harmless to ignore exceptions here - - # some cleanup is eventually done in PubSubPeer.send - asyncCheck p.sendSubs(pubsubPeer, toSeq(p.topics.keys), true) - - pubsubPeer.subscribed = true + discard p.getOrCreatePeer(peer, p.codec) method unsubscribe*(p: PubSub, topics: seq[TopicPair]) {.base, async.} = @@ -249,32 +241,18 @@ method subscribe*(p: PubSub, p.topics[topic].handler.add(handler) - var sent: seq[Future[bool]] - for peer in toSeq(p.peers.values): - sent.add(p.sendSubs(peer, @[topic], true)) - - checkFutures(await allFinished(sent)) + for _, peer in p.peers: + p.sendSubs(peer, @[topic], true) # metrics libp2p_pubsub_topics.set(p.topics.len.int64) method publish*(p: PubSub, topic: string, - data: seq[byte], - timeout: Duration = InfiniteDuration): Future[int] {.base, async.} = + data: seq[byte]): Future[int] {.base, async.} = ## publish to a ``topic`` - if p.triggerSelf and topic in p.topics: - for h in p.topics[topic].handler: - trace "triggering handler", topicID = topic - try: - await h(topic, data) - except CancelledError as exc: - raise exc - except CatchableError as exc: - # TODO these exceptions are ignored since it's likely that if writes are - # are failing, the underlying connection is already closed - this needs - # more cleanup though - debug "Could not write to pubsub connection", msg = exc.msg + if p.triggerSelf: + await handleData(p, topic, data) return 0 diff --git a/libp2p/protocols/pubsub/pubsubpeer.nim b/libp2p/protocols/pubsub/pubsubpeer.nim index 15004d7bc..1d8a0ef5c 100644 --- a/libp2p/protocols/pubsub/pubsubpeer.nim +++ b/libp2p/protocols/pubsub/pubsubpeer.nim @@ -28,27 +28,26 @@ when defined(libp2p_expensive_metrics): declareCounter(libp2p_pubsub_skipped_received_messages, "number of received skipped messages", labels = ["id"]) declareCounter(libp2p_pubsub_skipped_sent_messages, "number of sent skipped messages", labels = ["id"]) -const - DefaultSendTimeout* = 10.seconds - type PubSubObserver* = ref object onRecv*: proc(peer: PubSubPeer; msgs: var RPCMsg) {.gcsafe, raises: [Defect].} onSend*: proc(peer: PubSubPeer; msgs: var RPCMsg) {.gcsafe, raises: [Defect].} + GetConn* = proc(): Future[(Connection, RPCMsg)] {.gcsafe.} + PubSubPeer* = ref object of RootObj - switch*: Switch # switch instance to dial peers + getConn*: GetConn # callback to establish a new send connection codec*: string # the protocol that this peer joined from sendConn: Connection # cached send connection + connections*: seq[Connection] # connections to this peer peerId*: PeerID 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 - subscribed*: bool # are we subscribed to this peer dialLock: AsyncLock - RPCHandler* = proc(peer: PubSubPeer, msg: seq[RPCMsg]): Future[void] {.gcsafe.} + RPCHandler* = proc(peer: PubSubPeer, msg: RPCMsg): Future[void] {.gcsafe.} func hash*(p: PubSubPeer): Hash = # int is either 32/64, so intptr basically, pubsubpeer is a ref @@ -78,13 +77,15 @@ proc sendObservers(p: PubSubPeer, msg: var RPCMsg) = proc handle*(p: PubSubPeer, conn: Connection) {.async.} = logScope: + oid = $conn.oid peer = p.id + closed = conn.closed - debug "starting pubsub read loop for peer", closed = conn.closed + debug "starting pubsub read loop" try: try: while not conn.atEof: - trace "waiting for data", closed = conn.closed + trace "waiting for data" let data = await conn.readLp(64 * 1024) let digest = $(sha256.digest(data)) trace "read data from peer", data = data.shortLog @@ -111,10 +112,9 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} = # metrics libp2p_pubsub_received_messages.inc(labelValues = [p.id, t]) - await p.handler(p, @[msg]) + await p.handler(p, msg) p.recvdRpcCache.put(digest) finally: - debug "exiting pubsub peer read loop" await conn.close() if p.sendConn == conn: @@ -124,9 +124,14 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} = raise exc except CatchableError as exc: trace "Exception occurred in PubSubPeer.handle", exc = exc.msg + finally: + debug "exiting pubsub read loop" proc getSendConn(p: PubSubPeer): Future[Connection] {.async.} = - # get a cached send connection or create a new one + ## get a cached send connection or create a new one - will return nil if + ## getting a new connection fails + ## + block: # check if there's an existing connection that can be reused let current = p.sendConn @@ -147,7 +152,8 @@ proc getSendConn(p: PubSubPeer): Future[Connection] {.async.} = # and later close one of them, other implementations such as rust-libp2p # become deaf to our messages (potentially due to the clean-up associated # with closing connections). To prevent this, we use a lock that ensures - # that only a single dial will be performed for each peer. + # that only a single dial will be performed for each peer and send the + # subscription table every time we reconnect. # # Nevertheless, this approach is still quite problematic because the gossip # sends and their respective dials may be started from the mplex read loop. @@ -172,24 +178,35 @@ proc getSendConn(p: PubSubPeer): Future[Connection] {.async.} = return current # Grab a new send connection - let newConn = await p.switch.dial(p.peerId, p.codec) # ...and here + let (newConn, handshake) = await p.getConn() # ...and here if newConn.isNil: return nil + trace "Sending handshake", oid = $newConn.oid, handshake = shortLog(handshake) + await newConn.writeLp(encodeRpcMsg(handshake)) + trace "Caching new send connection", oid = $newConn.oid p.sendConn = newConn asyncCheck p.handle(newConn) # start a read loop on the new connection return newConn - + except CancelledError as exc: + raise exc + except CatchableError as exc: + return nil finally: if p.dialLock.locked: p.dialLock.release() -proc send*( - p: PubSubPeer, - msg: RPCMsg, - timeout: Duration = DefaultSendTimeout) {.async.} = +proc connectImpl*(p: PubSubPeer) {.async.} = + try: + discard await getSendConn(p) + except CatchableError as exc: + debug "Could not connect to pubsub peer", err = exc.msg +proc connect*(p: PubSubPeer) = + asyncCheck(connectImpl(p)) + +proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} = doAssert(not isNil(p), "pubsubpeer nil!") logScope: @@ -217,16 +234,15 @@ proc send*( libp2p_pubsub_skipped_sent_messages.inc(labelValues = [p.id]) return - var conn: Connection + var conn = await p.getSendConn() try: trace "about to send message" - conn = await p.getSendConn() - if conn == nil: debug "Couldn't get send connection, dropping message" return + trace "sending encoded msgs to peer", connId = $conn.oid - await conn.writeLp(encoded).wait(timeout) + await conn.writeLp(encoded) p.sentRpcCache.put(digest) trace "sent pubsub message to remote", connId = $conn.oid @@ -238,22 +254,32 @@ proc send*( libp2p_pubsub_sent_messages.inc(labelValues = [p.id, t]) except CatchableError as exc: + # Because we detach the send call from the currently executing task using + # asyncCheck, no exceptions may leak out of it trace "unable to send to remote", exc = exc.msg # Next time sendConn is used, it will be have its close flag set and thus # will be recycled if not isNil(conn): - await conn.close() + await conn.close() # This will clean up the send connection - raise exc + if exc is CancelledError: # TODO not handled + debug "Send cancelled" + + # We'll ask for a new send connection whenever possible + if p.sendConn == conn: + p.sendConn = nil + +proc send*(p: PubSubPeer, msg: RPCMsg) = + asyncCheck sendImpl(p, msg) proc `$`*(p: PubSubPeer): string = - p.id + $p.peerId proc newPubSubPeer*(peerId: PeerID, - switch: Switch, + getConn: GetConn, codec: string): PubSubPeer = new result - result.switch = switch + result.getConn = getConn result.codec = codec result.peerId = peerId result.sentRpcCache = newTimedCache[string](2.minutes) diff --git a/libp2p/protocols/pubsub/rpc/messages.nim b/libp2p/protocols/pubsub/rpc/messages.nim index 9f62f5bcb..3930de5a0 100644 --- a/libp2p/protocols/pubsub/rpc/messages.nim +++ b/libp2p/protocols/pubsub/rpc/messages.nim @@ -50,6 +50,11 @@ type messages*: seq[Message] control*: Option[ControlMessage] +func withSubs*( + T: type RPCMsg, topics: openArray[string], subscribe: bool): T = + T( + subscriptions: topics.mapIt(SubOpts(subscribe: subscribe, topic: it))) + func shortLog*(s: ControlIHave): auto = ( topicID: s.topicID.shortLog, diff --git a/libp2p/stream/bufferstream.nim b/libp2p/stream/bufferstream.nim index 5cd33c1e6..25840e498 100644 --- a/libp2p/stream/bufferstream.nim +++ b/libp2p/stream/bufferstream.nim @@ -43,7 +43,7 @@ logScope: topics = "bufferstream" const - DefaultBufferSize* = 102400 + DefaultBufferSize* = 128 const BufferStreamTrackerName* = "libp2p.bufferstream" diff --git a/tests/pubsub/testgossipinternal.nim b/tests/pubsub/testgossipinternal.nim index 779e9fca2..5d48b1905 100644 --- a/tests/pubsub/testgossipinternal.nim +++ b/tests/pubsub/testgossipinternal.nim @@ -16,6 +16,13 @@ type proc noop(data: seq[byte]) {.async, gcsafe.} = discard +proc getPubSubPeer(p: TestGossipSub, peerId: PeerID): auto = + proc getConn(): Future[(Connection, RPCMsg)] {.async.} = + let conn = await p.switch.dial(peerId, GossipSubCodec) + return (conn, RPCMsg.withSubs(toSeq(p.topics.keys), true)) + + newPubSubPeer(peerId, getConn, GossipSubCodec) + proc randomPeerInfo(): PeerInfo = PeerInfo.init(PrivateKey.random(ECDSA, rng[]).get()) @@ -39,7 +46,7 @@ suite "GossipSub internal": conns &= conn let peerInfo = randomPeerInfo() conn.peerInfo = peerInfo - let peer = newPubSubPeer(peerInfo.peerId, gossipSub.switch, GossipSubCodec) + let peer = gossipSub.getPubSubPeer(peerInfo.peerId) gossipSub.peers[peerInfo.peerId] = peer gossipSub.mesh[topic].incl(peer) @@ -69,7 +76,7 @@ suite "GossipSub internal": conns &= conn let peerInfo = PeerInfo.init(PrivateKey.random(ECDSA, rng[]).get()) conn.peerInfo = peerInfo - let peer = newPubSubPeer(peerInfo.peerId, gossipsub.switch, GossipSubCodec) + let peer = gossipSub.getPubSubPeer(peerInfo.peerId) gossipSub.peers[peerInfo.peerId] = peer gossipSub.mesh[topic].incl(peer) @@ -89,7 +96,7 @@ suite "GossipSub internal": proc testRun(): Future[bool] {.async.} = let gossipSub = TestGossipSub.init(newStandardSwitch()) - proc handler(peer: PubSubPeer, msg: seq[RPCMsg]) {.async.} = + proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} = discard let topic = "foobar" @@ -101,7 +108,7 @@ suite "GossipSub internal": conns &= conn var peerInfo = randomPeerInfo() conn.peerInfo = peerInfo - let peer = newPubSubPeer(peerInfo.peerId, gossipsub.switch, GossipSubCodec) + let peer = gossipSub.getPubSubPeer(peerInfo.peerId) peer.handler = handler gossipSub.gossipsub[topic].incl(peer) @@ -121,7 +128,7 @@ suite "GossipSub internal": proc testRun(): Future[bool] {.async.} = let gossipSub = TestGossipSub.init(newStandardSwitch()) - proc handler(peer: PubSubPeer, msg: seq[RPCMsg]) {.async.} = + proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} = discard let topic = "foobar" @@ -135,7 +142,7 @@ suite "GossipSub internal": conns &= conn let peerInfo = PeerInfo.init(PrivateKey.random(ECDSA, rng[]).get()) conn.peerInfo = peerInfo - let peer = newPubSubPeer(peerInfo.peerId, gossipsub.switch, GossipSubCodec) + let peer = gossipSub.getPubSubPeer(peerInfo.peerId) peer.handler = handler gossipSub.fanout[topic].incl(peer) @@ -156,7 +163,7 @@ suite "GossipSub internal": proc testRun(): Future[bool] {.async.} = let gossipSub = TestGossipSub.init(newStandardSwitch()) - proc handler(peer: PubSubPeer, msg: seq[RPCMsg]) {.async.} = + proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} = discard let topic1 = "foobar1" @@ -173,7 +180,7 @@ suite "GossipSub internal": conns &= conn let peerInfo = randomPeerInfo() conn.peerInfo = peerInfo - let peer = newPubSubPeer(peerInfo.peerId, gossipsub.switch, GossipSubCodec) + let peer = gossipSub.getPubSubPeer(peerInfo.peerId) peer.handler = handler gossipSub.fanout[topic1].incl(peer) gossipSub.fanout[topic2].incl(peer) @@ -197,7 +204,7 @@ suite "GossipSub internal": proc testRun(): Future[bool] {.async.} = let gossipSub = TestGossipSub.init(newStandardSwitch()) - proc handler(peer: PubSubPeer, msg: seq[RPCMsg]) {.async.} = + proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} = discard let topic = "foobar" @@ -212,7 +219,7 @@ suite "GossipSub internal": conns &= conn let peerInfo = randomPeerInfo() conn.peerInfo = peerInfo - let peer = newPubSubPeer(peerInfo.peerId, gossipsub.switch, GossipSubCodec) + let peer = gossipSub.getPubSubPeer(peerInfo.peerId) peer.handler = handler if i mod 2 == 0: gossipSub.fanout[topic].incl(peer) @@ -225,7 +232,7 @@ suite "GossipSub internal": conns &= conn let peerInfo = randomPeerInfo() conn.peerInfo = peerInfo - let peer = newPubSubPeer(peerInfo.peerId, gossipsub.switch, GossipSubCodec) + let peer = gossipSub.getPubSubPeer(peerInfo.peerId) peer.handler = handler gossipSub.gossipsub[topic].incl(peer) @@ -262,7 +269,7 @@ suite "GossipSub internal": proc testRun(): Future[bool] {.async.} = let gossipSub = TestGossipSub.init(newStandardSwitch()) - proc handler(peer: PubSubPeer, msg: seq[RPCMsg]) {.async.} = + proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} = discard let topic = "foobar" @@ -274,7 +281,7 @@ suite "GossipSub internal": conns &= conn let peerInfo = randomPeerInfo() conn.peerInfo = peerInfo - let peer = newPubSubPeer(peerInfo.peerId, gossipsub.switch, GossipSubCodec) + let peer = gossipSub.getPubSubPeer(peerInfo.peerId) peer.handler = handler if i mod 2 == 0: gossipSub.fanout[topic].incl(peer) @@ -307,7 +314,7 @@ suite "GossipSub internal": proc testRun(): Future[bool] {.async.} = let gossipSub = TestGossipSub.init(newStandardSwitch()) - proc handler(peer: PubSubPeer, msg: seq[RPCMsg]) {.async.} = + proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} = discard let topic = "foobar" @@ -319,7 +326,7 @@ suite "GossipSub internal": conns &= conn let peerInfo = randomPeerInfo() conn.peerInfo = peerInfo - let peer = newPubSubPeer(peerInfo.peerId, gossipSub.switch, GossipSubCodec) + let peer = gossipSub.getPubSubPeer(peerInfo.peerId) peer.handler = handler if i mod 2 == 0: gossipSub.mesh[topic].incl(peer) @@ -352,7 +359,7 @@ suite "GossipSub internal": proc testRun(): Future[bool] {.async.} = let gossipSub = TestGossipSub.init(newStandardSwitch()) - proc handler(peer: PubSubPeer, msg: seq[RPCMsg]) {.async.} = + proc handler(peer: PubSubPeer, msg: RPCMsg) {.async.} = discard let topic = "foobar" @@ -364,7 +371,7 @@ suite "GossipSub internal": conns &= conn let peerInfo = randomPeerInfo() conn.peerInfo = peerInfo - let peer = newPubSubPeer(peerInfo.peerId, gossipSub.switch, GossipSubCodec) + let peer = gossipSub.getPubSubPeer(peerInfo.peerId) peer.handler = handler if i mod 2 == 0: gossipSub.mesh[topic].incl(peer) From c0bc73ddac718216629a3d91883ed6fb07b98202 Mon Sep 17 00:00:00 2001 From: Eugene Kabanov Date: Thu, 3 Sep 2020 20:13:37 +0300 Subject: [PATCH 02/10] Fix Azure CI x86 problems. (#350) --- azure-pipelines.yml | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/azure-pipelines.yml b/azure-pipelines.yml index 3b0372ef1..3c1eedd03 100644 --- a/azure-pipelines.yml +++ b/azure-pipelines.yml @@ -88,6 +88,13 @@ steps: [[ -z "$ncpu" || $ncpu -le 0 ]] && ncpu=2 echo "Found ${ncpu} cores" + if [[ $PLATFORM == "x86" ]]; then + choco --version + choco install --x86 openssl + export PATH="/c/Program Files (x86)/OpenSSL-Win32/bin:${PATH}" + echo "PATH=${PATH}" + fi + # build nim from our own branch - this to avoid the day-to-day churn and # regressions of the fast-paced Nim development while maintaining the # flexibility to apply patches From 5819c6a9a796812ec3fb011fa3a597b015a17beb Mon Sep 17 00:00:00 2001 From: Jacek Sieka Date: Fri, 4 Sep 2020 08:10:32 +0200 Subject: [PATCH 03/10] 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 --- libp2p/protocols/pubsub/floodsub.nim | 111 ++++++++++++---------- libp2p/protocols/pubsub/gossipsub.nim | 124 ++++++++++++------------- libp2p/protocols/pubsub/mcache.nim | 72 ++++++-------- libp2p/protocols/pubsub/pubsub.nim | 19 ++-- libp2p/protocols/pubsub/pubsubpeer.nim | 32 +------ libp2p/protocols/pubsub/timedcache.nim | 106 +++++++++------------ tests/pubsub/testmcache.nim | 8 +- tests/pubsub/testpubsub.nim | 1 + tests/pubsub/testtimedcache.nim | 34 +++++++ 9 files changed, 250 insertions(+), 257 deletions(-) create mode 100644 tests/pubsub/testtimedcache.nim diff --git a/libp2p/protocols/pubsub/floodsub.nim b/libp2p/protocols/pubsub/floodsub.nim index 3562c84e8..b581b6a66 100644 --- a/libp2p/protocols/pubsub/floodsub.nim +++ b/libp2p/protocols/pubsub/floodsub.nim @@ -7,16 +7,17 @@ ## This file may not be copied, modified, or distributed except according to ## those terms. -import sequtils, tables, sets, strutils +import std/[sequtils, sets, tables] import chronos, chronicles, metrics -import pubsub, - pubsubpeer, - timedcache, - peertable, - rpc/[messages, message], +import ./pubsub, + ./pubsubpeer, + ./timedcache, + ./peertable, + ./rpc/[message, messages], ../../stream/connection, ../../peerid, - ../../peerinfo + ../../peerinfo, + ../../utility logScope: topics = "floodsub" @@ -49,15 +50,13 @@ method subscribeTopic*(f: FloodSub, method unsubscribePeer*(f: FloodSub, peer: PeerID) = ## handle peer disconnects ## - trace "unsubscribing floodsub peer", peer = $peer let pubSubPeer = f.peers.getOrDefault(peer) if pubSubPeer.isNil: return - for t in toSeq(f.floodsub.keys): - if t in f.floodsub: - f.floodsub[t].excl(pubSubPeer) + for _, v in f.floodsub.mpairs(): + v.excl(pubSubPeer) procCall PubSub(f).unsubscribePeer(peer) @@ -66,35 +65,34 @@ method rpcHandler*(f: FloodSub, rpcMsg: RPCMsg) {.async.} = await procCall PubSub(f).rpcHandler(peer, rpcMsg) - if rpcMsg.messages.len > 0: # if there are any messages + for msg in rpcMsg.messages: # for every message + let msgId = f.msgIdProvider(msg) + logScope: + msgId + peer = peer.id + + if f.seen.put(msgId): + trace "Dropping already-seen message" + continue + + if f.verifySignature and not msg.verify(peer.peerId): + debug "Dropping message due to failed signature verification" + continue + + if not (await f.validate(msg)): + trace "Dropping message due to failed validation" + continue + var toSendPeers = initHashSet[PubSubPeer]() - for msg in rpcMsg.messages: # for every message - let msgId = f.msgIdProvider(msg) - logScope: msgId + for t in msg.topicIDs: # for every topic in the message + f.floodsub.withValue(t, peers): toSendPeers.incl(peers[]) - if msgId notin f.seen: - f.seen.put(msgId) # add the message to the seen cache + await handleData(f, t, msg.data) - if f.verifySignature and not msg.verify(peer.peerId): - trace "dropping message due to failed signature verification" - continue - - if not (await f.validate(msg)): - trace "dropping message due to failed validation" - continue - - for t in msg.topicIDs: # for every topic in the message - if t in f.floodsub: - toSendPeers.incl(f.floodsub[t]) # get all the peers interested in this topic - - await handleData(f, t, msg.data) - - # forward the message to all peers interested in it - f.broadcast( - toSeq(toSendPeers), - RPCMsg(messages: rpcMsg.messages)) - - trace "forwared message to peers", peers = toSendPeers.len + # In theory, if topics are the same in all messages, we could batch - we'd + # also have to be careful to only include validated messages + f.broadcast(toSeq(toSendPeers), RPCMsg(messages: @[msg])) + trace "Forwared message to peers", peers = toSendPeers.len method init*(f: FloodSub) = proc handler(conn: Connection, proto: string) {.async.} = @@ -114,30 +112,41 @@ method publish*(f: FloodSub, # base returns always 0 discard await procCall PubSub(f).publish(topic, data) - if data.len <= 0 or topic.len <= 0: - trace "topic or data missing, skipping publish" + logScope: topic + trace "Publishing message on topic", data = data.shortLog + + if topic.len <= 0: # data could be 0/empty + debug "Empty topic, skipping publish" return 0 - if topic notin f.floodsub: - trace "missing peers for topic, skipping publish" - return + let peers = toSeq(f.floodsub.getOrDefault(topic)) + + if peers.len == 0: + debug "No peers for topic, skipping publish" + return 0 - trace "publishing on topic", name = topic inc f.msgSeqno let 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 - f.broadcast( - peers, - RPCMsg(messages: @[msg])) + logScope: msgId + + trace "Created new message", msg = shortLog(msg), peers = peers.len + + 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): libp2p_pubsub_messages_published.inc(labelValues = [topic]) - trace "published message to peers", peers = peers.len, - msg = msg.shortLog() + trace "Published message to peers" + return peers.len method unsubscribe*(f: FloodSub, @@ -156,5 +165,5 @@ method unsubscribeAll*(f: FloodSub, topic: string) {.async.} = method initPubSub*(f: FloodSub) = procCall PubSub(f).initPubSub() f.floodsub = initTable[string, HashSet[PubSubPeer]]() - f.seen = newTimedCache[string](2.minutes) + f.seen = TimedCache[string].init(2.minutes) f.init() diff --git a/libp2p/protocols/pubsub/gossipsub.nim b/libp2p/protocols/pubsub/gossipsub.nim index 3714cfe7d..173e91dd5 100644 --- a/libp2p/protocols/pubsub/gossipsub.nim +++ b/libp2p/protocols/pubsub/gossipsub.nim @@ -7,20 +7,19 @@ ## This file may not be copied, modified, or distributed except according to ## those terms. -import std/[tables, sets, options, sequtils, random] +import std/[options, random, sequtils, sets, tables] import chronos, chronicles, metrics -import pubsub, - floodsub, - pubsubpeer, - peertable, - mcache, - timedcache, - rpc/[messages, message], +import ./pubsub, + ./floodsub, + ./pubsubpeer, + ./peertable, + ./mcache, + ./timedcache, + ./rpc/[messages, message], ../protocol, - ../../peerinfo, ../../stream/connection, + ../../peerinfo, ../../peerid, - ../../errors, ../../utility logScope: @@ -379,47 +378,36 @@ method rpcHandler*(g: GossipSub, rpcMsg: RPCMsg) {.async.} = 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 - let msgId = g.msgIdProvider(msg) - logScope: msgId + for msg in rpcMsg.messages: # for every message + let msgId = g.msgIdProvider(msg) + logScope: + msgId + peer = peer.id - if msgId in g.seen: - trace "message already processed, skipping" - continue + if g.seen.put(msgId): + trace "Dropping already-seen message" + 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): + debug "Dropping message due to failed signature verification" + continue - if g.verifySignature and not msg.verify(peer.peerId): - trace "dropping message due to failed signature verification" - continue + if not (await g.validate(msg)): + trace "Dropping message due to failed validation" + continue - if not (await g.validate(msg)): - trace "dropping message due to failed validation" - continue + var toSendPeers = initHashSet[PubSubPeer]() + for t in msg.topicIDs: # for every topic in the message + g.floodsub.withValue(t, peers): toSendPeers.incl(peers[]) + g.mesh.withValue(t, peers): toSendPeers.incl(peers[]) - # this shouldn't happen - if g.peerInfo.peerId == msg.fromPeer: - trace "skipping messages from self" - continue - - for t in msg.topicIDs: # for every topic in the message - if t in g.floodsub: - toSendPeers.incl(g.floodsub[t]) # get all floodsub peers for topic - - if t in g.mesh: - toSendPeers.incl(g.mesh[t]) # get all mesh peers for topic - - await handleData(g, t, msg.data) - - # forward the message to all peers interested in it - g.broadcast( - toSeq(toSendPeers), - RPCMsg(messages: rpcMsg.messages)) + await handleData(g, t, msg.data) + # In theory, if topics are the same in all messages, we could batch - we'd + # also have to be careful to only include validated messages + g.broadcast(toSeq(toSendPeers), RPCMsg(messages: @[msg])) trace "forwared message to peers", peers = toSendPeers.len if rpcMsg.control.isSome: @@ -451,12 +439,13 @@ method unsubscribe*(g: GossipSub, for (topic, handler) in topics: # 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: - let peers = g.mesh.getOrDefault(topic) + let peers = g.mesh[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) method unsubscribeAll*(g: GossipSub, topic: string) {.async.} = @@ -474,9 +463,12 @@ method publish*(g: GossipSub, data: seq[byte]): Future[int] {.async.} = # base returns always 0 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 + debug "Empty topic, skipping publish" return 0 var peers: HashSet[PubSubPeer] @@ -497,29 +489,35 @@ method publish*(g: GossipSub, # time g.lastFanoutPubSub[topic] = Moment.fromNow(GossipSubFanoutTTL) + if peers.len == 0: + debug "No peers for topic, skipping publish" + return 0 + inc g.msgSeqno let msg = Message.init(g.peerInfo, data, topic, g.msgSeqno, g.sign) msgId = g.msgIdProvider(msg) - trace "created new message", msg, topic, peers = peers.len + logScope: msgId - if msgId notin g.mcache: - g.mcache.put(msgId, msg) + trace "Created new message", msg = shortLog(msg), peers = peers.len - if peers.len > 0: - g.broadcast(toSeq(peers), RPCMsg(messages: @[msg])) - when defined(libp2p_expensive_metrics): - if peers.len > 0: - libp2p_pubsub_messages_published.inc(labelValues = [topic]) - - trace "published message to peers", peers = peers.len, - msg = msg.shortLog() - return peers.len - else: - debug "No peers for gossip message", topic, msg = msg.shortLog() + if g.seen.put(msgId): + # custom msgid providers might cause this + trace "Dropping already-seen message" return 0 + g.mcache.put(msgId, msg) + + g.broadcast(toSeq(peers), RPCMsg(messages: @[msg])) + when defined(libp2p_expensive_metrics): + if peers.len > 0: + libp2p_pubsub_messages_published.inc(labelValues = [topic]) + + trace "Published message to peers" + + return peers.len + method start*(g: GossipSub) {.async.} = trace "gossipsub start" @@ -556,7 +554,7 @@ method initPubSub*(g: GossipSub) = procCall FloodSub(g).initPubSub() randomize() - g.mcache = newMCache(GossipSubHistoryGossip, GossipSubHistoryLength) + g.mcache = MCache.init(GossipSubHistoryGossip, GossipSubHistoryLength) g.mesh = initTable[string, HashSet[PubSubPeer]]() # meshes - 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 diff --git a/libp2p/protocols/pubsub/mcache.nim b/libp2p/protocols/pubsub/mcache.nim index 82231f550..172e8bc62 100644 --- a/libp2p/protocols/pubsub/mcache.nim +++ b/libp2p/protocols/pubsub/mcache.nim @@ -7,69 +7,57 @@ ## This file may not be copied, modified, or distributed except according to ## those terms. -import chronos, chronicles -import tables, options, sets, sequtils -import rpc/[messages], timedcache +import std/[sets, tables, options] +import rpc/[messages] + +export sets, tables, messages, options type CacheEntry* = object mid*: string - msg*: Message + topicIDs*: seq[string] - MCache* = ref object of RootObj - msgs*: TimedCache[Message] + MCache* = object of RootObj + msgs*: Table[string, Message] history*: seq[seq[CacheEntry]] historySize*: Natural windowSize*: Natural -proc get*(c: MCache, mid: string): Option[Message] = +func get*(c: MCache, mid: string): Option[Message] = result = none(Message) if mid in c.msgs: result = some(c.msgs[mid]) -proc contains*(c: MCache, mid: string): bool = +func contains*(c: MCache, mid: string): bool = c.get(mid).isSome -proc put*(c: 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) - ) - +func put*(c: var MCache, msgId: string, msg: Message) = if msgId notin c.msgs: - c.msgs.put(msgId, msg, handler = handler) - c.history[0].add(CacheEntry(mid: msgId, msg: msg)) + c.msgs[msgId] = 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]() - let len = - if c.windowSize > c.history.len: - c.history.len - else: - c.windowSize + let + len = min(c.windowSize, c.history.len) - if c.history.len > 0: - for slot in c.history[0.. c.historySize: - for entry in c.history.pop(): - c.msgs.del(entry.mid) +func shift*(c: var MCache) = + for entry in c.history.pop(): + c.msgs.del(entry.mid) c.history.insert(@[]) -proc newMCache*(window: Natural, history: Natural): MCache = - new result - result.historySize = history - result.windowSize = window - result.history = newSeq[seq[CacheEntry]]() - result.history.add(@[]) # initialize with empty slot - result.msgs = newTimedCache[Message](2.minutes) +func init*(T: type MCache, window, history: Natural): T = + T( + history: newSeq[seq[CacheEntry]](history), + historySize: history, + windowSize: window + ) diff --git a/libp2p/protocols/pubsub/pubsub.nim b/libp2p/protocols/pubsub/pubsub.nim index e3c242010..6946ca463 100644 --- a/libp2p/protocols/pubsub/pubsub.nim +++ b/libp2p/protocols/pubsub/pubsub.nim @@ -201,16 +201,17 @@ method unsubscribe*(p: PubSub, topics: seq[TopicPair]) {.base, async.} = ## unsubscribe from a list of ``topic`` strings for t in topics: - for i, h in p.topics[t.topic].handler: - if h == t.handler: - p.topics[t.topic].handler.del(i) + p.topics.withValue(t.topic, subs): + for i, h in subs[].handler: + if h == t.handler: + subs[].handler.del(i) - # make sure we delete the topic if - # no more handlers are left - if p.topics[t.topic].handler.len <= 0: - p.topics.del(t.topic) - # metrics - libp2p_pubsub_topics.set(p.topics.len.int64) + # make sure we delete the topic if + # no more handlers are left + if subs.handler.len <= 0: + p.topics.del(t.topic) # careful, invalidates subs + # metrics + libp2p_pubsub_topics.set(p.topics.len.int64) proc unsubscribe*(p: PubSub, topic: string, diff --git a/libp2p/protocols/pubsub/pubsubpeer.nim b/libp2p/protocols/pubsub/pubsubpeer.nim index 1d8a0ef5c..8cf50aeba 100644 --- a/libp2p/protocols/pubsub/pubsubpeer.nim +++ b/libp2p/protocols/pubsub/pubsubpeer.nim @@ -10,8 +10,6 @@ import std/[hashes, options, strutils, tables] import chronos, chronicles, nimcrypto/sha2, metrics import rpc/[messages, message, protobuf], - timedcache, - ../../switch, ../../peerid, ../../peerinfo, ../../stream/connection, @@ -42,8 +40,6 @@ type connections*: seq[Connection] # connections to this peer peerId*: PeerID 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 dialLock: AsyncLock @@ -87,33 +83,24 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} = while not conn.atEof: trace "waiting for data" let data = await conn.readLp(64 * 1024) - let digest = $(sha256.digest(data)) 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) if rmsg.isErr(): notice "failed to decode msg from peer" break - var msg = rmsg.get() - - trace "decoded msg from peer", msg = msg.shortLog + trace "decoded msg from peer", msg = rmsg.get().shortLog # trigger hooks - p.recvObservers(msg) + p.recvObservers(rmsg.get()) when defined(libp2p_expensive_metrics): - for m in msg.messages: + for m in rmsg.get().messages: for t in m.topicIDs: # metrics libp2p_pubsub_received_messages.inc(labelValues = [p.id, t]) - await p.handler(p, msg) - p.recvdRpcCache.put(digest) + await p.handler(p, rmsg.get()) finally: await conn.close() @@ -227,13 +214,6 @@ proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} = logScope: 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() try: 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 await conn.writeLp(encoded) - - p.sentRpcCache.put(digest) trace "sent pubsub message to remote", connId = $conn.oid when defined(libp2p_expensive_metrics): @@ -282,6 +260,4 @@ proc newPubSubPeer*(peerId: PeerID, result.getConn = getConn result.codec = codec result.peerId = peerId - result.sentRpcCache = newTimedCache[string](2.minutes) - result.recvdRpcCache = newTimedCache[string](2.minutes) result.dialLock = newAsyncLock() diff --git a/libp2p/protocols/pubsub/timedcache.nim b/libp2p/protocols/pubsub/timedcache.nim index e7d08f3b2..35a12aed3 100644 --- a/libp2p/protocols/pubsub/timedcache.nim +++ b/libp2p/protocols/pubsub/timedcache.nim @@ -7,73 +7,59 @@ ## This file may not be copied, modified, or distributed except according to ## those terms. -import tables -import chronos, chronicles +import std/[heapqueue, sets] -logScope: - topics = "timedcache" +import chronos/timer const Timeout* = 10.seconds # default timeout in ms type - ExpireHandler*[V] = proc(key: string, val: V) {.gcsafe.} - TimedEntry*[V] = object of RootObj - val: V - handler: ExpireHandler[V] + TimedEntry*[K] = ref object of RootObj + key: K + expiresAt: Moment - TimedCache*[V] = ref object of RootObj - cache*: Table[string, TimedEntry[V]] - onExpire*: ExpireHandler[V] - timeout*: Duration + TimedCache*[K] = object of RootObj + expiries: HeapQueue[TimedEntry[K]] + entries: HashSet[K] + timeout: Duration -# TODO: This belong in chronos, temporary left here until chronos is updated -proc addTimer*(at: Duration, cb: CallbackFunc, udata: pointer = nil) = - ## 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) +func `<`*(a, b: TimedEntry): bool = + a.expiresAt < b.expiresAt -proc put*[V](t: TimedCache[V], - key: string, - val: V = "", - timeout: Duration, - handler: ExpireHandler[V] = nil) = - trace "adding entry to timed cache", key = key - t.cache[key] = TimedEntry[V](val: val, handler: handler) +func expire*(t: var TimedCache, now: Moment = Moment.now()) = + while t.expiries.len() > 0 and t.expiries[0].expiresAt < now: + t.entries.excl(t.expiries.pop().key) - addTimer( - timeout, - proc (arg: pointer = nil) {.gcsafe.} = - trace "deleting expired entry from timed cache", key = key - if key in t.cache: - let entry = t.cache[key] - t.cache.del(key) - if not isNil(entry.handler): - entry.handler(key, entry.val) +func del*[K](t: var TimedCache[K], key: K): bool = + # Removes existing key from cache, returning false if it was not present + if not t.entries.missingOrExcl(key): + for i in 0.. Date: Fri, 4 Sep 2020 19:30:45 +0300 Subject: [PATCH 04/10] Remove asyncCheck from codebase. (#345) * Remove asyncCheck from codebase. * Replace all `discard` statements with new `asyncSpawn`. * Bump `nim-chronos` requirement. --- libp2p.nimble | 2 +- libp2p/connmanager.nim | 20 +++++++--- libp2p/muxers/mplex/lpchannel.nim | 41 ++++++++++--------- libp2p/muxers/mplex/mplex.nim | 50 ++++++++++++++--------- libp2p/protocols/pubsub/floodsub.nim | 10 ++++- libp2p/protocols/pubsub/gossipsub.nim | 10 ++++- libp2p/protocols/pubsub/pubsubpeer.nim | 10 +++-- libp2p/protocols/secure/secure.nim | 17 ++++++-- libp2p/switch.nim | 55 +++++++++++++++++++++----- libp2p/transports/tcptransport.nim | 9 +++-- 10 files changed, 155 insertions(+), 69 deletions(-) diff --git a/libp2p.nimble b/libp2p.nimble index ef37b2dca..e1d4a08e3 100644 --- a/libp2p.nimble +++ b/libp2p.nimble @@ -11,7 +11,7 @@ requires "nim >= 1.2.0", "nimcrypto >= 0.4.1", "bearssl >= 0.1.4", "chronicles >= 0.7.2", - "chronos >= 2.3.8", + "chronos >= 2.5.2", "metrics", "secp256k1", "stew >= 0.1.0" diff --git a/libp2p/connmanager.nim b/libp2p/connmanager.nim index 8d101146a..4cbafda98 100644 --- a/libp2p/connmanager.nim +++ b/libp2p/connmanager.nim @@ -120,10 +120,17 @@ proc onClose(c: ConnManager, conn: Connection) {.async.} = ## ## triggers the connections resource cleanup ## - - await conn.join() - trace "triggering connection cleanup", peer = $conn.peerInfo - await c.cleanupConn(conn) + try: + await conn.join() + trace "triggering connection cleanup", peer = $conn.peerInfo + await c.cleanupConn(conn) + except CancelledError: + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. + trace "Unexpected cancellation in connection manager's cleanup" + except CatchableError as exc: + trace "Unexpected exception in connection manager's cleanup", + errMsg = exc.msg proc selectConn*(c: ConnManager, peerId: PeerID, @@ -184,8 +191,9 @@ proc storeConn*(c: ConnManager, conn: Connection) = c.conns[peerId].incl(conn) - # launch on close listener - asyncCheck c.onClose(conn) + # Launch on close listener + # All the errors are handled inside `onClose()` procedure. + asyncSpawn c.onClose(conn) libp2p_peers.set(c.conns.len.int64) trace "stored connection", connections = c.conns.len, peer = peerId diff --git a/libp2p/muxers/mplex/lpchannel.nim b/libp2p/muxers/mplex/lpchannel.nim index 48eadb97d..e62dfebf2 100644 --- a/libp2p/muxers/mplex/lpchannel.nim +++ b/libp2p/muxers/mplex/lpchannel.nim @@ -66,18 +66,6 @@ template withWriteLock(lock: AsyncLock, body: untyped): untyped = if not(isNil(lock)) and lock.locked: lock.release() -template withEOFExceptions(body: untyped): untyped = - try: - body - except CancelledError as exc: - raise exc - except LPStreamEOFError as exc: - trace "muxed connection EOF", exc = exc.msg - except LPStreamClosedError as exc: - trace "muxed connection closed", exc = exc.msg - except LPStreamIncompleteError as exc: - trace "incomplete message", exc = exc.msg - proc closeMessage(s: LPChannel) {.async.} = logScope: id = s.id @@ -104,11 +92,22 @@ proc resetMessage(s: LPChannel) {.async.} = # stack = getStackTrace() ## send reset message - this will not raise - withEOFExceptions: + try: withWriteLock(s.writeLock): trace "sending reset message" - await s.conn.writeMsg(s.id, s.resetCode) # write reset + except CancelledError: + # This procedure is called from one place and never awaited, so there no + # need to re-raise CancelledError. + trace "Unexpected cancellation while resetting channel" + except LPStreamEOFError as exc: + trace "muxed connection EOF", exc = exc.msg + except LPStreamClosedError as exc: + trace "muxed connection closed", exc = exc.msg + except LPStreamIncompleteError as exc: + trace "incomplete message", exc = exc.msg + except CatchableError as exc: + trace "Unhandled exception leak", exc = exc.msg proc open*(s: LPChannel) {.async, gcsafe.} = logScope: @@ -170,10 +169,7 @@ method reset*(s: LPChannel) {.base, async, gcsafe.} = trace "resetting channel" - # we asyncCheck here because the other end - # might be dead already - reset is always - # optimistic - asyncCheck s.resetMessage() + asyncSpawn s.resetMessage() try: # drain the buffer before closing @@ -210,9 +206,11 @@ method close*(s: LPChannel) {.async, gcsafe.} = await s.closeMessage().wait(2.minutes) if s.atEof: # already closed by remote close parent buffer immediately await procCall BufferStream(s).close() - except CancelledError as exc: + except CancelledError: + trace "Unexpected cancellation while closing channel" await s.reset() - raise exc + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. except CatchableError as exc: trace "exception closing channel", exc = exc.msg await s.reset() @@ -220,7 +218,8 @@ method close*(s: LPChannel) {.async, gcsafe.} = trace "lpchannel closed local" s.closedLocal = true - asyncCheck closeInternal() + # All the errors are handled inside `closeInternal()` procedure. + asyncSpawn closeInternal() method initStream*(s: LPChannel) = if s.objName.len == 0: diff --git a/libp2p/muxers/mplex/mplex.nim b/libp2p/muxers/mplex/mplex.nim index 5915f0042..d2611bc2e 100644 --- a/libp2p/muxers/mplex/mplex.nim +++ b/libp2p/muxers/mplex/mplex.nim @@ -48,14 +48,21 @@ proc newTooManyChannels(): ref TooManyChannels = proc cleanupChann(m: Mplex, chann: LPChannel) {.async, inline.} = ## remove the local channel from the internal tables ## - await chann.join() - m.channels[chann.initiator].del(chann.id) - trace "cleaned up channel", id = chann.id, oid = $chann.oid + try: + await chann.join() + m.channels[chann.initiator].del(chann.id) + trace "cleaned up channel", id = chann.id, oid = $chann.oid - when defined(libp2p_expensive_metrics): - libp2p_mplex_channels.set( - m.channels[chann.initiator].len.int64, - labelValues = [$chann.initiator, $m.connection.peerInfo]) + when defined(libp2p_expensive_metrics): + libp2p_mplex_channels.set( + m.channels[chann.initiator].len.int64, + labelValues = [$chann.initiator, $m.connection.peerInfo]) + except CancelledError: + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. + trace "Unexpected cancellation in mplex channel cleanup" + except CatchableError as exc: + trace "error cleaning up mplex channel", exc = exc.msg proc newStreamInternal*(m: Mplex, initiator: bool = true, @@ -90,7 +97,8 @@ proc newStreamInternal*(m: Mplex, m.channels[initiator][id] = result - asyncCheck m.cleanupChann(result) + # All the errors are handled inside `cleanupChann()` procedure. + asyncSpawn m.cleanupChann(result) when defined(libp2p_expensive_metrics): libp2p_mplex_channels.set( @@ -104,12 +112,13 @@ proc handleStream(m: Mplex, chann: LPChannel) {.async.} = await m.streamHandler(chann) trace "finished handling stream" doAssert(chann.closed, "connection not closed by handler!") - except CancelledError as exc: - trace "cancelling stream handler", exc = exc.msg + except CancelledError: + trace "Unexpected cancellation in stream handler" await chann.reset() - raise exc + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. except CatchableError as exc: - trace "exception in stream handler", exc = exc.msg + trace "Exception in mplex stream handler", exc = exc.msg await chann.reset() method handle*(m: Mplex) {.async, gcsafe.} = @@ -145,7 +154,8 @@ method handle*(m: Mplex) {.async, gcsafe.} = tmp else: if m.channels[false].len > m.maxChannCount - 1: - warn "too many channels created by remote peer", allowedMax = MaxChannelCount + warn "too many channels created by remote peer", + allowedMax = MaxChannelCount raise newTooManyChannels() let name = string.fromBytes(data) @@ -160,12 +170,14 @@ method handle*(m: Mplex) {.async, gcsafe.} = trace "created channel" if not isNil(m.streamHandler): - # launch handler task - asyncCheck m.handleStream(channel) + # Launch handler task + # All the errors are handled inside `handleStream()` procedure. + asyncSpawn m.handleStream(channel) of MessageType.MsgIn, MessageType.MsgOut: if data.len > MaxMsgSize: - warn "attempting to send a packet larger than allowed", allowed = MaxMsgSize + warn "attempting to send a packet larger than allowed", + allowed = MaxMsgSize raise newLPStreamLimitError() trace "pushing data to channel" @@ -180,8 +192,10 @@ method handle*(m: Mplex) {.async, gcsafe.} = trace "resetting channel" await channel.reset() trace "reset channel" - except CancelledError as exc: - raise exc + except CancelledError: + # This procedure is spawned as task and it is not part of public API, so + # there no way for this procedure to be cancelled implicitely. + trace "Unexpected cancellation in mplex handler" except CatchableError as exc: trace "Exception occurred", exception = exc.msg, oid = $m.oid diff --git a/libp2p/protocols/pubsub/floodsub.nim b/libp2p/protocols/pubsub/floodsub.nim index b581b6a66..578ecc89d 100644 --- a/libp2p/protocols/pubsub/floodsub.nim +++ b/libp2p/protocols/pubsub/floodsub.nim @@ -100,8 +100,14 @@ method init*(f: FloodSub) = ## connection for a protocol string ## e.g. ``/floodsub/1.0.0``, etc... ## - - await f.handleConn(conn, proto) + try: + await f.handleConn(conn, proto) + except CancelledError: + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. + trace "Unexpected cancellation in floodsub handler" + except CatchableError as exc: + trace "FloodSub handler leaks an error", exc = exc.msg f.handler = handler f.codec = FloodSubCodec diff --git a/libp2p/protocols/pubsub/gossipsub.nim b/libp2p/protocols/pubsub/gossipsub.nim index 173e91dd5..8a32d8bf3 100644 --- a/libp2p/protocols/pubsub/gossipsub.nim +++ b/libp2p/protocols/pubsub/gossipsub.nim @@ -75,8 +75,14 @@ method init*(g: GossipSub) = ## connection for a protocol string ## e.g. ``/floodsub/1.0.0``, etc... ## - - await g.handleConn(conn, proto) + try: + await g.handleConn(conn, proto) + except CancelledError: + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. + trace "Unexpected cancellation in gossipsub handler" + except CatchableError as exc: + trace "GossipSub handler leaks an error", exc = exc.msg g.handler = handler g.codec = GossipSubCodec diff --git a/libp2p/protocols/pubsub/pubsubpeer.nim b/libp2p/protocols/pubsub/pubsubpeer.nim index 8cf50aeba..3c7894ac2 100644 --- a/libp2p/protocols/pubsub/pubsubpeer.nim +++ b/libp2p/protocols/pubsub/pubsubpeer.nim @@ -107,8 +107,10 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} = if p.sendConn == conn: p.sendConn = nil - except CancelledError as exc: - raise exc + except CancelledError: + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. + trace "Unexpected cancellation in PubSubPeer.handle" except CatchableError as exc: trace "Exception occurred in PubSubPeer.handle", exc = exc.msg finally: @@ -174,7 +176,9 @@ proc getSendConn(p: PubSubPeer): Future[Connection] {.async.} = trace "Caching new send connection", oid = $newConn.oid p.sendConn = newConn - asyncCheck p.handle(newConn) # start a read loop on the new connection + # Start a read loop on the new connection. + # All the errors are handled inside `handle()` procedure. + asyncSpawn p.handle(newConn) return newConn except CancelledError as exc: raise exc diff --git a/libp2p/protocols/secure/secure.nim b/libp2p/protocols/secure/secure.nim index f73621d4f..3f5338e1b 100644 --- a/libp2p/protocols/secure/secure.nim +++ b/libp2p/protocols/secure/secure.nim @@ -61,10 +61,21 @@ proc handleConn*(s: Secure, conn: Connection, initiator: bool): Future[Connection] {.async, gcsafe.} = var sconn = await s.handshake(conn, initiator) + + proc cleanup() {.async.} = + try: + await conn.join() + await sconn.close() + except CancelledError: + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. + discard + except CatchableError as exc: + trace "error cleaning up secure connection", errMsg = exc.msg + if not isNil(sconn): - conn.join() - .addCallback do(udata: pointer = nil): - asyncCheck sconn.close() + # All the errors are handled inside `cleanup()` procedure. + asyncSpawn cleanup() return sconn diff --git a/libp2p/switch.nim b/libp2p/switch.nim index 23e120476..f7b9aab1b 100644 --- a/libp2p/switch.nim +++ b/libp2p/switch.nim @@ -354,10 +354,21 @@ proc internalConnect(s: Switch, if isNil(conn): # None of the addresses connected raise newException(CatchableError, "Unable to establish outgoing link") - conn.closeEvent.wait() - .addCallback do(udata: pointer): - asyncCheck s.triggerConnEvent( - peerId, ConnEvent(kind: ConnEventKind.Disconnected)) + proc peerCleanup() {.async.} = + try: + await conn.closeEvent.wait() + await s.triggerConnEvent(peerId, + ConnEvent(kind: ConnEventKind.Disconnected)) + except CancelledError: + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. + trace "Unexpected cancellation in switch peer connect cleanup" + except CatchableError as exc: + trace "Unexpected exception in switch peer connect cleanup", + errMsg = exc.msg + + # All the errors are handled inside `cleanup()` procedure. + asyncSpawn peerCleanup() await s.triggerConnEvent( peerId, ConnEvent(kind: ConnEventKind.Connected, incoming: false)) @@ -489,13 +500,37 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = await s.identify(muxer) let peerId = muxer.connection.peerInfo.peerId - muxer.connection.closeEvent.wait() - .addCallback do(udata: pointer): - asyncCheck s.triggerConnEvent( - peerId, ConnEvent(kind: ConnEventKind.Disconnected)) - asyncCheck s.triggerConnEvent( - peerId, ConnEvent(kind: ConnEventKind.Connected, incoming: true)) + proc peerCleanup() {.async.} = + try: + await muxer.connection.closeEvent.wait() + await s.triggerConnEvent(peerId, + ConnEvent(kind: ConnEventKind.Disconnected)) + except CancelledError: + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. + trace "Unexpected cancellation in switch muxer cleanup" + except CatchableError as exc: + trace "Unexpected exception in switch muxer cleanup", + errMsg = exc.msg + + proc peerStartup() {.async.} = + try: + await s.triggerConnEvent(peerId, + ConnEvent(kind: ConnEventKind.Connected, + incoming: true)) + except CancelledError: + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. + trace "Unexpected cancellation in switch muxer startup" + except CatchableError as exc: + trace "Unexpected exception in switch muxer startup", + errMsg = exc.msg + + # All the errors are handled inside `peerCleanup()` procedure. + asyncSpawn peerCleanup() + # All the errors are handled inside `peerStartup()` procedure. + asyncSpawn peerStartup() except CancelledError as exc: await muxer.close() diff --git a/libp2p/transports/tcptransport.nim b/libp2p/transports/tcptransport.nim index 4d0963b82..3e62e5b2a 100644 --- a/libp2p/transports/tcptransport.nim +++ b/libp2p/transports/tcptransport.nim @@ -76,13 +76,16 @@ proc connHandler*(t: TcpTransport, if not(isNil(conn)): await conn.close() t.clients.keepItIf(it != client) - except CancelledError as exc: - raise exc + except CancelledError: + # This is top-level procedure which will work as separate task, so it + # do not need to propogate CancelledError. + trace "Unexpected cancellation in transport's cleanup" except CatchableError as exc: trace "error cleaning up client", exc = exc.msg t.clients.add(client) - asyncCheck cleanup() + # All the errors are handled inside `cleanup()` procedure. + asyncSpawn cleanup() result = conn proc connCb(server: StreamServer, From 6d91d61844dc7af8ec4e05e74248f68442440cb0 Mon Sep 17 00:00:00 2001 From: Jacek Sieka Date: Fri, 4 Sep 2020 18:31:43 +0200 Subject: [PATCH 05/10] small cleanups & docs (#347) * simplify gossipsub heartbeat start / stop * avoid alloc in peerid check * stop iterating over seq after unsubscribing item (could crash) * don't crash on missing private key with enabled sigs (shouldn't happen but...) --- libp2p/protocols/pubsub/gossipsub.nim | 25 +++++++------------ libp2p/protocols/pubsub/peertable.nim | 8 ++++--- libp2p/protocols/pubsub/pubsub.nim | 32 ++++++++++++------------- libp2p/protocols/pubsub/rpc/message.nim | 13 ++++++---- 4 files changed, 36 insertions(+), 42 deletions(-) diff --git a/libp2p/protocols/pubsub/gossipsub.nim b/libp2p/protocols/pubsub/gossipsub.nim index 8a32d8bf3..52ee8b878 100644 --- a/libp2p/protocols/pubsub/gossipsub.nim +++ b/libp2p/protocols/pubsub/gossipsub.nim @@ -54,7 +54,6 @@ type mcache*: MCache # messages cache heartbeatFut: Future[void] # cancellation future for heartbeat interval heartbeatRunning: bool - heartbeatLock: AsyncLock # heartbeat lock to prevent two consecutive concurrent heartbeats when defined(libp2p_expensive_metrics): declareGauge(libp2p_gossipsub_peers_per_topic_mesh, @@ -244,7 +243,7 @@ proc heartbeat(g: GossipSub) {.async.} = except CancelledError as exc: raise exc except CatchableError as exc: - trace "exception ocurred in gossipsub heartbeat", exc = exc.msg + warn "exception ocurred in gossipsub heartbeat", exc = exc.msg await sleepAsync(GossipSubHeartbeatInterval) @@ -527,25 +526,18 @@ method publish*(g: GossipSub, method start*(g: GossipSub) {.async.} = trace "gossipsub start" - ## start pubsub - ## start long running/repeating procedures + if not g.heartbeatFut.isNil: + warn "Starting gossipsub twice" + return - # interlock start to to avoid overlapping to stops - await g.heartbeatLock.acquire() - - # setup the heartbeat interval g.heartbeatRunning = true g.heartbeatFut = g.heartbeat() - g.heartbeatLock.release() - method stop*(g: GossipSub) {.async.} = trace "gossipsub stop" - - ## stop pubsub - ## stop long running tasks - - await g.heartbeatLock.acquire() + if g.heartbeatFut.isNil: + warn "Stopping gossipsub without starting it" + return # stop heartbeat interval g.heartbeatRunning = false @@ -553,8 +545,8 @@ method stop*(g: GossipSub) {.async.} = trace "awaiting last heartbeat" await g.heartbeatFut trace "heartbeat stopped" + g.heartbeatFut = nil - g.heartbeatLock.release() method initPubSub*(g: GossipSub) = procCall FloodSub(g).initPubSub() @@ -567,4 +559,3 @@ method initPubSub*(g: GossipSub) = g.lastFanoutPubSub = initTable[string, Moment]() # last publish time for fanout topics g.gossip = initTable[string, seq[ControlIHave]]() # pending gossip g.control = initTable[string, ControlMessage]() # pending control messages - g.heartbeatLock = newAsyncLock() diff --git a/libp2p/protocols/pubsub/peertable.nim b/libp2p/protocols/pubsub/peertable.nim index d294c0155..15a1bf5d5 100644 --- a/libp2p/protocols/pubsub/peertable.nim +++ b/libp2p/protocols/pubsub/peertable.nim @@ -14,9 +14,11 @@ type PeerTable* = Table[string, HashSet[PubSubPeer]] # topic string to peer map proc hasPeerID*(t: PeerTable, topic: string, peerId: PeerID): bool = - let peers = toSeq(t.getOrDefault(topic)) - peers.any do (peer: PubSubPeer) -> bool: - peer.peerId == peerId + if topic in t: + for peer in t[topic]: + if peer.peerId == peerId: + return true + false func addPeer*(table: var PeerTable, topic: string, peer: PubSubPeer): bool = # returns true if the peer was added, diff --git a/libp2p/protocols/pubsub/pubsub.nim b/libp2p/protocols/pubsub/pubsub.nim index 6946ca463..fed1512e0 100644 --- a/libp2p/protocols/pubsub/pubsub.nim +++ b/libp2p/protocols/pubsub/pubsub.nim @@ -70,11 +70,8 @@ method unsubscribePeer*(p: PubSub, peerId: PeerID) {.base.} = libp2p_pubsub_peers.set(p.peers.len.int64) -proc send*( - p: PubSub, - peer: PubSubPeer, - msg: RPCMsg) = - ## send to remote peer +proc send*(p: PubSub, peer: PubSubPeer, msg: RPCMsg) = + ## Attempt to send `msg` to remote peer ## trace "sending pubsub message to peer", peer = $peer, msg = shortLog(msg) @@ -84,11 +81,10 @@ proc broadcast*( p: PubSub, sendPeers: openArray[PubSubPeer], msg: RPCMsg) = # raises: [Defect] - ## send messages - returns number of send attempts made. - ## + ## Attempt to send `msg` to the given peers trace "broadcasting messages to peers", - peers = sendPeers.len, message = shortLog(msg) + peers = sendPeers.len, msg = shortLog(msg) for peer in sendPeers: p.send(peer, msg) @@ -201,16 +197,14 @@ method unsubscribe*(p: PubSub, topics: seq[TopicPair]) {.base, async.} = ## unsubscribe from a list of ``topic`` strings for t in topics: - p.topics.withValue(t.topic, subs): - for i, h in subs[].handler: - if h == t.handler: - subs[].handler.del(i) + p.topics.withValue(t.topic, topic): + topic[].handler.keepIf(proc (x: auto): bool = x != t.handler) + + if topic[].handler.len == 0: + # make sure we delete the topic if + # no more handlers are left + p.topics.del(t.topic) - # make sure we delete the topic if - # no more handlers are left - if subs.handler.len <= 0: - p.topics.del(t.topic) # careful, invalidates subs - # metrics libp2p_pubsub_topics.set(p.topics.len.int64) proc unsubscribe*(p: PubSub, @@ -252,6 +246,10 @@ method publish*(p: PubSub, topic: string, data: seq[byte]): Future[int] {.base, async.} = ## publish to a ``topic`` + ## The return value is the number of neighbours that we attempted to send the + ## message to, excluding self. Note that this is an optimistic number of + ## attempts - the number of peers that actually receive the message might + ## be lower. if p.triggerSelf: await handleData(p, topic, data) diff --git a/libp2p/protocols/pubsub/rpc/message.nim b/libp2p/protocols/pubsub/rpc/message.nim index e49b54d9e..e5d15d008 100644 --- a/libp2p/protocols/pubsub/rpc/message.nim +++ b/libp2p/protocols/pubsub/rpc/message.nim @@ -30,8 +30,8 @@ declareCounter(libp2p_pubsub_sig_verify_failure, "pubsub failed validated messag func defaultMsgIdProvider*(m: Message): string = byteutils.toHex(m.seqno) & m.fromPeer.pretty -proc sign*(msg: Message, p: PeerInfo): CryptoResult[seq[byte]] = - ok((? p.privateKey.sign(PubSubPrefix & encodeMessage(msg))).getBytes()) +proc sign*(msg: Message, privateKey: PrivateKey): CryptoResult[seq[byte]] = + ok((? privateKey.sign(PubSubPrefix & encodeMessage(msg))).getBytes()) proc verify*(m: Message, p: PeerID): bool = if m.signature.len > 0 and m.key.len > 0: @@ -63,6 +63,9 @@ proc init*( seqno: @(seqno.toBytesBE), # unefficient, fine for now topicIDs: @[topic]) - if sign and peer.publicKey.isSome: - result.signature = sign(result, peer).tryGet() - result.key = peer.publicKey.get().getBytes().tryGet() + if sign: + if peer.keyType != KeyType.HasPrivate: + raise (ref CatchableError)(msg: "Cannot sign message without private key") + + result.signature = sign(result, peer.privateKey).tryGet() + result.key = peer.privateKey.getKey().tryGet().getBytes().tryGet() From 16a008db75df36183d375dac5a9e0f923bca4617 Mon Sep 17 00:00:00 2001 From: Jacek Sieka Date: Fri, 4 Sep 2020 20:30:26 +0200 Subject: [PATCH 06/10] fix connection event order when connection dies early (#351) if the connection is already closed (because the remote closes during identfiy for example), an exception would be raised which would leave the connection in limbo, beacuse it would not go through the rest of internalConnect. Also, if the connection is already closed, the disconnect event would be scheduled before the connect event :/ --- libp2p/protocols/pubsub/rpc/messages.nim | 2 +- libp2p/switch.nim | 74 ++++++++++++++---------- 2 files changed, 44 insertions(+), 32 deletions(-) diff --git a/libp2p/protocols/pubsub/rpc/messages.nim b/libp2p/protocols/pubsub/rpc/messages.nim index 3930de5a0..92ba5e332 100644 --- a/libp2p/protocols/pubsub/rpc/messages.nim +++ b/libp2p/protocols/pubsub/rpc/messages.nim @@ -86,7 +86,7 @@ func shortLog*(c: ControlMessage): auto = func shortLog*(msg: Message): auto = ( - fromPeer: msg.fromPeer, + fromPeer: msg.fromPeer.pretty, data: msg.data.shortLog, seqno: msg.seqno.shortLog, topicIDs: $msg.topicIDs, diff --git a/libp2p/switch.nim b/libp2p/switch.nim index f7b9aab1b..d31067c71 100644 --- a/libp2p/switch.nim +++ b/libp2p/switch.nim @@ -205,10 +205,6 @@ proc disconnect*(s: Switch, peerId: PeerID): Future[void] {.gcsafe.} = s.connManager.dropPeer(peerId) proc upgradeOutgoing(s: Switch, conn: Connection): Future[Connection] {.async, gcsafe.} = - logScope: - conn = $conn - oid = $conn.oid - let sconn = await s.secure(conn) # secure the connection if isNil(sconn): raise newException(CatchableError, @@ -218,21 +214,29 @@ proc upgradeOutgoing(s: Switch, conn: Connection): Future[Connection] {.async, g raise newException(CatchableError, "current version of nim-libp2p requires that secure protocol negotiates peerid") - trace "upgrading connection" + trace "upgrading connection", conn = $sconn, uoid = $conn.oid let muxer = await s.mux(sconn) # mux it if possible if muxer == nil: # TODO this might be relaxed in the future raise newException(CatchableError, "a muxer is required for outgoing connections") - await s.identify(muxer) + try: + await s.identify(muxer) + except CatchableError as exc: + # Identify is non-essential, though if it fails, it might indicate that + # the connection was closed already - this will be picked up by the read + # loop + debug "Could not identify connection", + err = exc.msg, conn = $conn, uoid = $conn.oid if isNil(sconn.peerInfo): await sconn.close() raise newException(CatchableError, "unable to identify connection, stopping upgrade") - trace "successfully upgraded outgoing connection", oid = sconn.oid + trace "successfully upgraded outgoing connection", + conn = $sconn, uoid = $conn.oid, oid = $sconn.oid return sconn @@ -267,7 +271,8 @@ proc upgradeIncoming(s: Switch, conn: Connection) {.async, gcsafe.} = except CancelledError as exc: raise exc except CatchableError as exc: - debug "ending secured handler", err = exc.msg + debug "ending secured handler", + err = exc.msg, conn = $conn, oid = $conn.oid if (await ms.select(conn)): # just handshake # add the secure handlers @@ -281,9 +286,6 @@ proc upgradeIncoming(s: Switch, conn: Connection) {.async, gcsafe.} = proc internalConnect(s: Switch, peerId: PeerID, addrs: seq[MultiAddress]): Future[Connection] {.async.} = - logScope: - peer = peerId - if s.peerInfo.peerId == peerId: raise newException(CatchableError, "can't dial self!") @@ -300,12 +302,12 @@ proc internalConnect(s: Switch, # This connection should already have been removed from the connection # manager - it's essentially a bug that we end up here - we'll fail # for now, hoping that this will clean themselves up later... - warn "dead connection in connection manager" + warn "dead connection in connection manager", peer = $peerId await conn.close() raise newException(CatchableError, "Zombie connection encountered") - trace "Reusing existing connection", oid = $conn.oid, - direction = $conn.dir + trace "Reusing existing connection", + oid = $conn.oid, direction = $conn.dir, peer = $peerId return conn @@ -317,10 +319,10 @@ proc internalConnect(s: Switch, let dialed = try: await t.dial(a) except CancelledError as exc: - trace "dialing canceled", exc = exc.msg + trace "dialing canceled", exc = exc.msg, peer = $peerId raise exc except CatchableError as exc: - trace "dialing failed", exc = exc.msg + trace "dialing failed", exc = exc.msg, peer = $peerId libp2p_failed_dials.inc() continue # Try the next address @@ -344,6 +346,7 @@ proc internalConnect(s: Switch, conn = upgraded trace "dial successful", + peer = $peerId, oid = $upgraded.oid, peerInfo = shortLog(upgraded.peerInfo) break @@ -354,6 +357,14 @@ proc internalConnect(s: Switch, if isNil(conn): # None of the addresses connected raise newException(CatchableError, "Unable to establish outgoing link") + if conn.closed(): + # This can happen if one of the peer event handlers deems the peer + # unworthy and disconnects it + raise newLPStreamClosedError() + + await s.triggerConnEvent( + peerId, ConnEvent(kind: ConnEventKind.Connected, incoming: false)) + proc peerCleanup() {.async.} = try: await conn.closeEvent.wait() @@ -362,21 +373,15 @@ proc internalConnect(s: Switch, except CancelledError: # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. - trace "Unexpected cancellation in switch peer connect cleanup" + trace "Unexpected cancellation in switch peer connect cleanup", + peer = $peerId except CatchableError as exc: trace "Unexpected exception in switch peer connect cleanup", - errMsg = exc.msg + errMsg = exc.msg, peer = $peerId # All the errors are handled inside `cleanup()` procedure. asyncSpawn peerCleanup() - await s.triggerConnEvent( - peerId, ConnEvent(kind: ConnEventKind.Connected, incoming: false)) - - if conn.closed(): - # This can happen if one of the peer event handlers deems the peer - # unworthy and disconnects it - raise newException(CatchableError, "Connection closed during handshake") return conn @@ -498,7 +503,13 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = try: await s.identify(muxer) + except CatchableError as exc: + # Identify is non-essential, though if it fails, it might indicate that + # the connection was closed already - this will be picked up by the read + # loop + debug "Could not identify connection", err = exc.msg + try: let peerId = muxer.connection.peerInfo.peerId proc peerCleanup() {.async.} = @@ -509,9 +520,9 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = except CancelledError: # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. - trace "Unexpected cancellation in switch muxer cleanup" + debug "Unexpected cancellation in switch muxer cleanup" except CatchableError as exc: - trace "Unexpected exception in switch muxer cleanup", + debug "Unexpected exception in switch muxer cleanup", errMsg = exc.msg proc peerStartup() {.async.} = @@ -522,15 +533,16 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = except CancelledError: # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. - trace "Unexpected cancellation in switch muxer startup" + debug "Unexpected cancellation in switch muxer startup" except CatchableError as exc: - trace "Unexpected exception in switch muxer startup", + debug "Unexpected exception in switch muxer startup", errMsg = exc.msg + # All the errors are handled inside `peerStartup()` procedure. + asyncSpawn peerStartup() + # All the errors are handled inside `peerCleanup()` procedure. asyncSpawn peerCleanup() - # All the errors are handled inside `peerStartup()` procedure. - asyncSpawn peerStartup() except CancelledError as exc: await muxer.close() From 9b815efe8fe70dfaea5b4c101d47844cbd811337 Mon Sep 17 00:00:00 2001 From: Jacek Sieka Date: Fri, 4 Sep 2020 22:53:03 +0200 Subject: [PATCH 07/10] gossipsub: don't subscribe to floodsub also (#352) --- libp2p/protocols/pubsub/gossipsub.nim | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/libp2p/protocols/pubsub/gossipsub.nim b/libp2p/protocols/pubsub/gossipsub.nim index 52ee8b878..2bdd9a8eb 100644 --- a/libp2p/protocols/pubsub/gossipsub.nim +++ b/libp2p/protocols/pubsub/gossipsub.nim @@ -283,7 +283,8 @@ method subscribeTopic*(g: GossipSub, topic: string, subscribe: bool, peer: PubSubPeer) {.gcsafe.} = - procCall FloodSub(g).subscribeTopic(topic, subscribe, peer) + # Skip floodsub - we don't want it to add the peer to `g.floodsub` + procCall PubSub(g).subscribeTopic(topic, subscribe, peer) logScope: peer = $peer.id From c1856fda5307be7e855a42894d2753169ef78e1f Mon Sep 17 00:00:00 2001 From: Jacek Sieka Date: Sun, 6 Sep 2020 10:31:47 +0200 Subject: [PATCH 08/10] simplify and unify logging (#353) * use short format for logging peerid * log peerid:oid for connections --- examples/directchat.nim | 2 +- libp2p/connmanager.nim | 10 +- libp2p/daemon/daemonapi.nim | 17 ---- libp2p/muxers/mplex/coder.nim | 6 +- libp2p/muxers/mplex/lpchannel.nim | 116 +++++++---------------- libp2p/muxers/mplex/mplex.nim | 68 ++++++------- libp2p/muxers/muxer.nim | 10 +- libp2p/peerid.nim | 38 ++++---- libp2p/peerinfo.nim | 11 +-- libp2p/protocols/identify.nim | 14 +-- libp2p/protocols/pubsub/floodsub.nim | 35 +++---- libp2p/protocols/pubsub/gossipsub.nim | 33 ++++--- libp2p/protocols/pubsub/pubsub.nim | 10 +- libp2p/protocols/pubsub/pubsubpeer.nim | 72 +++++++------- libp2p/protocols/pubsub/rpc/message.nim | 2 +- libp2p/protocols/pubsub/rpc/messages.nim | 2 +- libp2p/protocols/pubsub/rpc/protobuf.nim | 2 +- libp2p/protocols/secure/noise.nim | 19 ++-- libp2p/protocols/secure/secio.nim | 23 +++-- libp2p/protocols/secure/secure.nim | 30 +++--- libp2p/stream/bufferstream.nim | 20 ++-- libp2p/stream/chronosstream.nim | 12 ++- libp2p/stream/connection.nim | 23 +++-- libp2p/switch.nim | 87 ++++++++--------- tests/pubsub/testgossipsub.nim | 22 ++--- tests/testpeer.nim | 8 +- 26 files changed, 311 insertions(+), 381 deletions(-) diff --git a/examples/directchat.nim b/examples/directchat.nim index fe1c35374..86d3c8aa5 100644 --- a/examples/directchat.nim +++ b/examples/directchat.nim @@ -190,7 +190,7 @@ proc processInput(rfd: AsyncFD, rng: ref BrHmacDrbgContext) {.async.} = let libp2pFuts = await switch.start() chatProto.started = true - let id = peerInfo.peerId.pretty + let id = $peerInfo.peerId echo "PeerID: " & id echo "listening on: " for a in peerInfo.addrs: diff --git a/libp2p/connmanager.nim b/libp2p/connmanager.nim index 4cbafda98..9be88f400 100644 --- a/libp2p/connmanager.nim +++ b/libp2p/connmanager.nim @@ -113,7 +113,7 @@ proc cleanupConn(c: ConnManager, conn: Connection) {.async.} = finally: await conn.close() - trace "connection cleaned up", peer = $conn.peerInfo + trace "connection cleaned up", conn proc onClose(c: ConnManager, conn: Connection) {.async.} = ## connection close even handler @@ -122,7 +122,7 @@ proc onClose(c: ConnManager, conn: Connection) {.async.} = ## try: await conn.join() - trace "triggering connection cleanup", peer = $conn.peerInfo + trace "triggering connection cleanup", conn await c.cleanupConn(conn) except CancelledError: # This is top-level procedure which will work as separate task, so it @@ -167,7 +167,7 @@ proc selectMuxer*(c: ConnManager, conn: Connection): Muxer = if conn in c.muxed: return c.muxed[conn].muxer else: - debug "no muxer for connection", conn = $conn + debug "no muxer for connection", conn proc storeConn*(c: ConnManager, conn: Connection) = ## store a connection @@ -196,7 +196,7 @@ proc storeConn*(c: ConnManager, conn: Connection) = asyncSpawn c.onClose(conn) libp2p_peers.set(c.conns.len.int64) - trace "stored connection", connections = c.conns.len, peer = peerId + trace "stored connection", connections = c.conns.len, conn proc storeOutgoing*(c: ConnManager, conn: Connection) = conn.dir = Direction.Out @@ -222,7 +222,7 @@ proc storeMuxer*(c: ConnManager, muxer: muxer, handle: handle) - trace "stored muxer", connections = c.conns.len + trace "stored muxer", connections = c.conns.len, muxer proc getMuxedStream*(c: ConnManager, peerId: PeerID, diff --git a/libp2p/daemon/daemonapi.nim b/libp2p/daemon/daemonapi.nim index e610fd384..8aee77875 100644 --- a/libp2p/daemon/daemonapi.nim +++ b/libp2p/daemon/daemonapi.nim @@ -1304,20 +1304,3 @@ proc pubsubSubscribe*(api: DaemonAPI, topic: string, except Exception as exc: await api.closeConnection(transp) raise exc - -proc `$`*(pinfo: PeerInfo): string = - ## Get string representation of ``PeerInfo`` object. - result = newStringOfCap(128) - result.add("{PeerID: '") - result.add($pinfo.peer.pretty()) - result.add("' Addresses: [") - let length = len(pinfo.addresses) - for i in 0.. 0: - result = result diff --git a/libp2p/muxers/mplex/coder.nim b/libp2p/muxers/mplex/coder.nim index 3b7ecbe4d..558e266e0 100644 --- a/libp2p/muxers/mplex/coder.nim +++ b/libp2p/muxers/mplex/coder.nim @@ -31,10 +31,10 @@ proc newInvalidMplexMsgType*(): ref InvalidMplexMsgType = proc readMsg*(conn: Connection): Future[Msg] {.async, gcsafe.} = let header = await conn.readVarint() - trace "read header varint", varint = header + trace "read header varint", varint = header, conn let data = await conn.readLp(MaxMsgSize) - trace "read data", dataLen = data.len, data = shortLog(data) + trace "read data", dataLen = data.len, data = shortLog(data), conn let msgType = header and 0x7 if msgType.int > ord(MessageType.ResetOut): @@ -46,7 +46,7 @@ proc writeMsg*(conn: Connection, id: uint64, msgType: MessageType, data: seq[byte] = @[]) {.async, gcsafe.} = - trace "sending data over mplex", oid = $conn.oid, + trace "sending data over mplex", conn, id, msgType, data = data.len diff --git a/libp2p/muxers/mplex/lpchannel.nim b/libp2p/muxers/mplex/lpchannel.nim index e62dfebf2..4e8a2b534 100644 --- a/libp2p/muxers/mplex/lpchannel.nim +++ b/libp2p/muxers/mplex/lpchannel.nim @@ -7,7 +7,7 @@ ## This file may not be copied, modified, or distributed except according to ## those terms. -import oids, deques +import std/[oids, strformat] import chronos, chronicles, metrics import types, coder, @@ -66,86 +66,60 @@ template withWriteLock(lock: AsyncLock, body: untyped): untyped = if not(isNil(lock)) and lock.locked: lock.release() -proc closeMessage(s: LPChannel) {.async.} = - logScope: - id = s.id - initiator = s.initiator - name = s.name - oid = $s.oid - peer = $s.conn.peerInfo - # stack = getStackTrace() +func shortLog*(s: LPChannel): auto = + if s.isNil: "LPChannel(nil)" + elif s.conn.peerInfo.isNil: $s.oid + elif s.name != $s.oid: &"{shortLog(s.conn.peerInfo.peerId)}:{s.oid}:{s.name}" + else: &"{shortLog(s.conn.peerInfo.peerId)}:{s.oid}" +chronicles.formatIt(LPChannel): shortLog(it) +proc closeMessage(s: LPChannel) {.async.} = ## send close message - this will not raise ## on EOF or Closed withWriteLock(s.writeLock): - trace "sending close message" + trace "sending close message", s await s.conn.writeMsg(s.id, s.closeCode) # write close proc resetMessage(s: LPChannel) {.async.} = - logScope: - id = s.id - initiator = s.initiator - name = s.name - oid = $s.oid - peer = $s.conn.peerInfo - # stack = getStackTrace() - ## send reset message - this will not raise try: withWriteLock(s.writeLock): - trace "sending reset message" + trace "sending reset message", s await s.conn.writeMsg(s.id, s.resetCode) # write reset except CancelledError: # This procedure is called from one place and never awaited, so there no # need to re-raise CancelledError. trace "Unexpected cancellation while resetting channel" except LPStreamEOFError as exc: - trace "muxed connection EOF", exc = exc.msg + trace "muxed connection EOF", exc = exc.msg, s except LPStreamClosedError as exc: - trace "muxed connection closed", exc = exc.msg + trace "muxed connection closed", exc = exc.msg, s except LPStreamIncompleteError as exc: - trace "incomplete message", exc = exc.msg + trace "incomplete message", exc = exc.msg, s except CatchableError as exc: - trace "Unhandled exception leak", exc = exc.msg + debug "Unhandled exception leak", exc = exc.msg, s proc open*(s: LPChannel) {.async, gcsafe.} = - logScope: - id = s.id - initiator = s.initiator - name = s.name - oid = $s.oid - peer = $s.conn.peerInfo - # stack = getStackTrace() - - ## NOTE: Don't call withExcAndLock or withWriteLock, - ## because this already gets called from writeHandler - ## which is locked await s.conn.writeMsg(s.id, MessageType.New, s.name) - trace "opened channel" + trace "opened channel", s s.isOpen = true proc closeRemote*(s: LPChannel) {.async.} = - logScope: - id = s.id - initiator = s.initiator - name = s.name - oid = $s.oid - peer = $s.conn.peerInfo - # stack = getStackTrace() - - trace "got EOF, closing channel" + trace "closing remote", s try: await s.drainBuffer() s.isEof = true # set EOF immediately to prevent further reads # close parent bufferstream to prevent further reads await procCall BufferStream(s).close() - trace "channel closed on EOF" + trace "channel closed on EOF", s except CancelledError as exc: raise exc except CatchableError as exc: - trace "exception closing remote channel", exc = exc.msg + trace "exception closing remote channel", exc = exc.msg, s + + trace "closed remote", s method closed*(s: LPChannel): bool = ## this emulates half-closed behavior @@ -155,19 +129,11 @@ method closed*(s: LPChannel): bool = s.closedLocal method reset*(s: LPChannel) {.base, async, gcsafe.} = - logScope: - id = s.id - initiator = s.initiator - name = s.name - oid = $s.oid - peer = $s.conn.peerInfo - # stack = getStackTrace() - if s.closedLocal and s.isEof: - trace "channel already closed or reset" + trace "channel already closed or reset", s return - trace "resetting channel" + trace "resetting channel", s asyncSpawn s.resetMessage() @@ -182,24 +148,16 @@ method reset*(s: LPChannel) {.base, async, gcsafe.} = except CancelledError as exc: raise exc except CatchableError as exc: - trace "exception in reset", exc = exc.msg + trace "exception in reset", exc = exc.msg, s - trace "channel reset" + trace "channel reset", s method close*(s: LPChannel) {.async, gcsafe.} = - logScope: - id = s.id - initiator = s.initiator - name = s.name - oid = $s.oid - peer = $s.conn.peerInfo - # stack = getStackTrace() - if s.closedLocal: - trace "channel already closed" + trace "channel already closed", s return - trace "closing local lpchannel" + trace "closing local lpchannel", s proc closeInternal() {.async.} = try: @@ -207,15 +165,15 @@ method close*(s: LPChannel) {.async, gcsafe.} = if s.atEof: # already closed by remote close parent buffer immediately await procCall BufferStream(s).close() except CancelledError: - trace "Unexpected cancellation while closing channel" + trace "Unexpected cancellation while closing channel", s await s.reset() # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. except CatchableError as exc: - trace "exception closing channel", exc = exc.msg + trace "exception closing channel", exc = exc.msg, s await s.reset() - trace "lpchannel closed local" + trace "lpchannel closed local", s s.closedLocal = true # All the errors are handled inside `closeInternal()` procedure. @@ -226,7 +184,7 @@ method initStream*(s: LPChannel) = s.objName = "LPChannel" s.timeoutHandler = proc() {.async, gcsafe.} = - trace "idle timeout expired, resetting LPChannel" + trace "idle timeout expired, resetting LPChannel", s await s.reset() procCall BufferStream(s).initStream() @@ -253,27 +211,19 @@ proc init*( resetCode: if initiator: MessageType.ResetOut else: MessageType.ResetIn, dir: if initiator: Direction.Out else: Direction.In) - logScope: - id = chann.id - initiator = chann.initiator - name = chann.name - oid = $chann.oid - peer = $chann.conn.peerInfo - # stack = getStackTrace() - proc writeHandler(data: seq[byte]) {.async, gcsafe.} = try: if chann.isLazy and not(chann.isOpen): await chann.open() # writes should happen in sequence - trace "sending data", len = data.len + trace "sending data", len = data.len, chann await conn.writeMsg(chann.id, chann.msgCode, data) except CatchableError as exc: - trace "exception in lpchannel write handler", exc = exc.msg + trace "exception in lpchannel write handler", exc = exc.msg, chann await chann.reset() raise exc @@ -281,6 +231,6 @@ proc init*( when chronicles.enabledLogLevel == LogLevel.TRACE: chann.name = if chann.name.len > 0: chann.name else: $chann.oid - trace "created new lpchannel" + trace "created new lpchannel", chann return chann diff --git a/libp2p/muxers/mplex/mplex.nim b/libp2p/muxers/mplex/mplex.nim index d2611bc2e..37244258b 100644 --- a/libp2p/muxers/mplex/mplex.nim +++ b/libp2p/muxers/mplex/mplex.nim @@ -42,6 +42,9 @@ type oid*: Oid maxChannCount: int +func shortLog*(m: MPlex): auto = shortLog(m.connection) +chronicles.formatIt(Mplex): shortLog(it) + proc newTooManyChannels(): ref TooManyChannels = newException(TooManyChannels, "max allowed channel count exceeded") @@ -51,18 +54,19 @@ proc cleanupChann(m: Mplex, chann: LPChannel) {.async, inline.} = try: await chann.join() m.channels[chann.initiator].del(chann.id) - trace "cleaned up channel", id = chann.id, oid = $chann.oid + trace "cleaned up channel", m, chann when defined(libp2p_expensive_metrics): libp2p_mplex_channels.set( m.channels[chann.initiator].len.int64, - labelValues = [$chann.initiator, $m.connection.peerInfo]) + labelValues = [$chann.initiator, $m.connection.peerInfo.peerId]) except CancelledError: # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. - trace "Unexpected cancellation in mplex channel cleanup" + trace "Unexpected cancellation in mplex channel cleanup", + m, chann except CatchableError as exc: - trace "error cleaning up mplex channel", exc = exc.msg + trace "error cleaning up mplex channel", exc = exc.msg, m, chann proc newStreamInternal*(m: Mplex, initiator: bool = true, @@ -77,10 +81,10 @@ proc newStreamInternal*(m: Mplex, m.currentId.inc(); m.currentId else: chanId - trace "creating new channel", channelId = id, - initiator = initiator, - name = name, - oid = $m.oid + trace "creating new channel", id, + initiator, + name, + m result = LPChannel.init( id, m.connection, @@ -103,35 +107,30 @@ proc newStreamInternal*(m: Mplex, when defined(libp2p_expensive_metrics): libp2p_mplex_channels.set( m.channels[initiator].len.int64, - labelValues = [$initiator, $m.connection.peerInfo]) + labelValues = [$initiator, $m.connection.peerInfo.peerId]) proc handleStream(m: Mplex, chann: LPChannel) {.async.} = ## call the muxer stream handler for this channel ## try: await m.streamHandler(chann) - trace "finished handling stream" + trace "finished handling stream", m, chann doAssert(chann.closed, "connection not closed by handler!") except CancelledError: - trace "Unexpected cancellation in stream handler" + trace "Unexpected cancellation in stream handler", m, chann await chann.reset() # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. except CatchableError as exc: - trace "Exception in mplex stream handler", exc = exc.msg + trace "Exception in mplex stream handler", + exc = exc.msg, m, chann await chann.reset() method handle*(m: Mplex) {.async, gcsafe.} = - logScope: moid = $m.oid - - trace "starting mplex main loop" + trace "starting mplex main loop", m, peer = m.connection.peerInfo.peerId try: - defer: - trace "stopping mplex main loop" - await m.close() - while not m.connection.atEof: - trace "waiting for data" + trace "waiting for data", m let (id, msgType, data) = await m.connection.readMsg() initiator = bool(ord(msgType) and 1) @@ -142,32 +141,28 @@ method handle*(m: Mplex) {.async, gcsafe.} = msgType = msgType size = data.len - trace "read message from connection", data = data.shortLog + trace "read message from connection", m, data = data.shortLog var channel = if MessageType(msgType) != MessageType.New: let tmp = m.channels[initiator].getOrDefault(id, nil) if tmp == nil: - trace "Channel not found, skipping" + trace "Channel not found, skipping", m continue tmp else: if m.channels[false].len > m.maxChannCount - 1: warn "too many channels created by remote peer", - allowedMax = MaxChannelCount + allowedMax = MaxChannelCount, m raise newTooManyChannels() let name = string.fromBytes(data) m.newStreamInternal(false, id, name, timeout = m.outChannTimeout) - logScope: - name = channel.name - oid = $channel.oid - case msgType: of MessageType.New: - trace "created channel" + trace "created channel", m, channel if not isNil(m.streamHandler): # Launch handler task @@ -177,27 +172,26 @@ method handle*(m: Mplex) {.async, gcsafe.} = of MessageType.MsgIn, MessageType.MsgOut: if data.len > MaxMsgSize: warn "attempting to send a packet larger than allowed", - allowed = MaxMsgSize + allowed = MaxMsgSize, channel raise newLPStreamLimitError() - trace "pushing data to channel" + trace "pushing data to channel", m, channel await channel.pushTo(data) - trace "pushed data to channel" + trace "pushed data to channel", m, channel of MessageType.CloseIn, MessageType.CloseOut: - trace "closing channel" await channel.closeRemote() - trace "closed channel" of MessageType.ResetIn, MessageType.ResetOut: - trace "resetting channel" await channel.reset() - trace "reset channel" except CancelledError: # This procedure is spawned as task and it is not part of public API, so # there no way for this procedure to be cancelled implicitely. trace "Unexpected cancellation in mplex handler" except CatchableError as exc: - trace "Exception occurred", exception = exc.msg, oid = $m.oid + trace "Exception occurred", exception = exc.msg, m + finally: + trace "stopping mplex main loop", m + await m.close() proc init*(M: type Mplex, conn: Connection, @@ -224,7 +218,7 @@ method close*(m: Mplex) {.async, gcsafe.} = if m.isClosed: return - trace "closing mplex muxer", moid = $m.oid + trace "closing mplex muxer", m m.isClosed = true diff --git a/libp2p/muxers/muxer.nim b/libp2p/muxers/muxer.nim index baa76a22d..870a80a79 100644 --- a/libp2p/muxers/muxer.nim +++ b/libp2p/muxers/muxer.nim @@ -35,6 +35,9 @@ type streamHandler*: StreamHandler # triggered every time there is a new stream, called for any muxer instance muxerHandler*: MuxerHandler # triggered every time there is a new muxed connection created +func shortLog*(m: Muxer): auto = shortLog(m.connection) +chronicles.formatIt(Muxer): shortLog(it) + # muxer interface method newStream*(m: Muxer, name: string = "", lazy: bool = false): Future[Connection] {.base, async, gcsafe.} = discard @@ -49,7 +52,7 @@ proc newMuxerProvider*(creator: MuxerConstructor, codec: string): MuxerProvider method init(c: MuxerProvider) = proc handler(conn: Connection, proto: string) {.async, gcsafe, closure.} = - trace "starting muxer handler", proto=proto, peer = $conn + trace "starting muxer handler", proto=proto, conn try: let muxer = c.newMuxer(conn) @@ -68,11 +71,8 @@ method init(c: MuxerProvider) = except CancelledError as exc: raise exc except CatchableError as exc: - trace "exception in muxer handler", exc = exc.msg, peer = $conn, proto=proto + trace "exception in muxer handler", exc = exc.msg, conn, proto finally: await conn.close() c.handler = handler - -proc `$`*(m: Muxer): string = - $m.connection diff --git a/libp2p/peerid.nim b/libp2p/peerid.nim index 9eeb46791..cd01ec149 100644 --- a/libp2p/peerid.nim +++ b/libp2p/peerid.nim @@ -11,11 +11,13 @@ {.push raises: [Defect].} -import hashes +import std/hashes +import chronicles import nimcrypto/utils, stew/base58 import crypto/crypto, multicodec, multihash, vbuffer import protobuf/minprotobuf import stew/results + export results const @@ -27,11 +29,24 @@ type PeerID* = object data*: seq[byte] - PeerIDError* = object of CatchableError - -proc pretty*(pid: PeerID): string {.inline.} = +func `$`*(pid: PeerID): string = ## Return base58 encoded ``pid`` representation. - result = Base58.encode(pid.data) + Base58.encode(pid.data) + +func shortLog*(pid: PeerId): string = + ## Returns compact string representation of ``pid``. + var spid = $pid + if len(spid) <= 10: + result = spid + else: + result = newStringOfCap(10) + for i in 0..<2: + result.add(spid[i]) + result.add("*") + for i in (len(spid) - 6)..spid.high: + result.add(spid[i]) + +chronicles.formatIt(PeerID): shortLog(it) proc toBytes*(pid: PeerID, data: var openarray[byte]): int = ## Store PeerID ``pid`` to array of bytes ``data``. @@ -112,19 +127,6 @@ proc extractPublicKey*(pid: PeerID, pubkey: var PublicKey): bool = let length = len(mh.data.buffer) result = pubkey.init(mh.data.buffer.toOpenArray(mh.dpos, length - 1)) -proc `$`*(pid: PeerID): string = - ## Returns compact string representation of ``pid``. - var spid = pid.pretty() - if len(spid) <= 10: - result = spid - else: - result = newStringOfCap(10) - for i in 0..<2: - result.add(spid[i]) - result.add("*") - for i in (len(spid) - 6)..spid.high: - result.add(spid[i]) - proc init*(pid: var PeerID, data: openarray[byte]): bool = ## Initialize peer id from raw binary representation ``data``. ## diff --git a/libp2p/peerinfo.nim b/libp2p/peerinfo.nim index f43905696..3fac63a3b 100644 --- a/libp2p/peerinfo.nim +++ b/libp2p/peerinfo.nim @@ -41,20 +41,15 @@ type of HasPublic: key: Option[PublicKey] -proc id*(p: PeerInfo): string = - if not(isNil(p)): - return p.peerId.pretty() - -proc `$`*(p: PeerInfo): string = p.id - -proc shortLog*(p: PeerInfo): auto = +func shortLog*(p: PeerInfo): auto = ( - id: p.id(), + peerId: $p.peerId, addrs: mapIt(p.addrs, $it), protocols: mapIt(p.protocols, $it), protoVersion: p.protoVersion, agentVersion: p.agentVersion, ) +chronicles.formatIt(PeerInfo): shortLog(it) template postInit(peerinfo: PeerInfo, addrs: openarray[MultiAddress], diff --git a/libp2p/protocols/identify.nim b/libp2p/protocols/identify.nim index 611469a66..3c7ce2381 100644 --- a/libp2p/protocols/identify.nim +++ b/libp2p/protocols/identify.nim @@ -106,16 +106,16 @@ method init*(p: Identify) = proc handle(conn: Connection, proto: string) {.async, gcsafe, closure.} = try: defer: - trace "exiting identify handler", oid = $conn.oid + trace "exiting identify handler", conn await conn.close() - trace "handling identify request", oid = $conn.oid + trace "handling identify request", conn var pb = encodeMsg(p.peerInfo, conn.observedAddr) await conn.writeLp(pb.buffer) except CancelledError as exc: raise exc except CatchableError as exc: - trace "exception in identify handler", exc = exc.msg + trace "exception in identify handler", exc = exc.msg, conn p.handler = handle p.codec = IdentifyCodec @@ -123,10 +123,10 @@ method init*(p: Identify) = proc identify*(p: Identify, conn: Connection, remotePeerInfo: PeerInfo): Future[IdentifyInfo] {.async, gcsafe.} = - trace "initiating identify", peer = $conn + trace "initiating identify", conn var message = await conn.readLp(64*1024) if len(message) == 0: - trace "identify: Empty message received!" + trace "identify: Empty message received!", conn raise newException(IdentityInvalidMsgError, "Empty message received!") let infoOpt = decodeMsg(message) @@ -144,8 +144,8 @@ proc identify*(p: Identify, # have in most cases if peer.get() != remotePeerInfo.peerId: trace "Peer ids don't match", - remote = peer.get().pretty(), - local = remotePeerInfo.id + remote = peer, + local = remotePeerInfo.peerId raise newException(IdentityNoMatchError, "Peer ids don't match") diff --git a/libp2p/protocols/pubsub/floodsub.nim b/libp2p/protocols/pubsub/floodsub.nim index 578ecc89d..11c480251 100644 --- a/libp2p/protocols/pubsub/floodsub.nim +++ b/libp2p/protocols/pubsub/floodsub.nim @@ -39,18 +39,18 @@ method subscribeTopic*(f: FloodSub, f.floodsub[topic] = initHashSet[PubSubPeer]() if subscribe: - trace "adding subscription for topic", peer = peer.id, name = topic + trace "adding subscription for topic", peer, topic # subscribe the peer to the topic f.floodsub[topic].incl(peer) else: - trace "removing subscription for topic", peer = peer.id, name = topic + trace "removing subscription for topic", peer, topic # unsubscribe the peer from the topic f.floodsub[topic].excl(peer) method unsubscribePeer*(f: FloodSub, peer: PeerID) = ## handle peer disconnects ## - trace "unsubscribing floodsub peer", peer = $peer + trace "unsubscribing floodsub peer", peer let pubSubPeer = f.peers.getOrDefault(peer) if pubSubPeer.isNil: return @@ -67,20 +67,17 @@ method rpcHandler*(f: FloodSub, for msg in rpcMsg.messages: # for every message let msgId = f.msgIdProvider(msg) - logScope: - msgId - peer = peer.id if f.seen.put(msgId): - trace "Dropping already-seen message" + trace "Dropping already-seen message", msgId, peer continue if f.verifySignature and not msg.verify(peer.peerId): - debug "Dropping message due to failed signature verification" + debug "Dropping message due to failed signature verification", msgId, peer continue if not (await f.validate(msg)): - trace "Dropping message due to failed validation" + trace "Dropping message due to failed validation", msgId, peer continue var toSendPeers = initHashSet[PubSubPeer]() @@ -105,9 +102,9 @@ method init*(f: FloodSub) = except CancelledError: # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. - trace "Unexpected cancellation in floodsub handler" + trace "Unexpected cancellation in floodsub handler", conn except CatchableError as exc: - trace "FloodSub handler leaks an error", exc = exc.msg + trace "FloodSub handler leaks an error", exc = exc.msg, conn f.handler = handler f.codec = FloodSubCodec @@ -118,17 +115,16 @@ method publish*(f: FloodSub, # base returns always 0 discard await procCall PubSub(f).publish(topic, data) - logScope: topic - trace "Publishing message on topic", data = data.shortLog + trace "Publishing message on topic", data = data.shortLog, topic if topic.len <= 0: # data could be 0/empty - debug "Empty topic, skipping publish" + debug "Empty topic, skipping publish", topic return 0 let peers = toSeq(f.floodsub.getOrDefault(topic)) if peers.len == 0: - debug "No peers for topic, skipping publish" + debug "No peers for topic, skipping publish", topic return 0 inc f.msgSeqno @@ -136,13 +132,12 @@ method publish*(f: FloodSub, msg = Message.init(f.peerInfo, data, topic, f.msgSeqno, f.sign) msgId = f.msgIdProvider(msg) - logScope: msgId - - trace "Created new message", msg = shortLog(msg), peers = peers.len + trace "Created new message", + msg = shortLog(msg), peers = peers.len, topic, msgId if f.seen.put(msgId): # custom msgid providers might cause this - trace "Dropping already-seen message" + trace "Dropping already-seen message", msgId, topic return 0 # Try to send to all peers that are known to be interested @@ -151,7 +146,7 @@ method publish*(f: FloodSub, when defined(libp2p_expensive_metrics): libp2p_pubsub_messages_published.inc(labelValues = [topic]) - trace "Published message to peers" + trace "Published message to peers", msgId, topic return peers.len diff --git a/libp2p/protocols/pubsub/gossipsub.nim b/libp2p/protocols/pubsub/gossipsub.nim index 2bdd9a8eb..af6fe77d7 100644 --- a/libp2p/protocols/pubsub/gossipsub.nim +++ b/libp2p/protocols/pubsub/gossipsub.nim @@ -79,15 +79,16 @@ method init*(g: GossipSub) = except CancelledError: # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. - trace "Unexpected cancellation in gossipsub handler" + trace "Unexpected cancellation in gossipsub handler", conn except CatchableError as exc: - trace "GossipSub handler leaks an error", exc = exc.msg + trace "GossipSub handler leaks an error", exc = exc.msg, conn g.handler = handler g.codec = GossipSubCodec proc replenishFanout(g: GossipSub, topic: string) = ## get fanout peers for a topic + logScope: topic trace "about to replenish fanout" if g.fanout.peers(topic) < GossipSubDLo: @@ -201,7 +202,7 @@ proc getGossipPeers(g: GossipSub): Table[PubSubPeer, ControlMessage] {.gcsafe.} continue if topic notin g.gossipsub: - trace "topic not in gossip array, skipping", topicID = topic + trace "topic not in gossip array, skipping", topic continue let ihave = ControlIHave(topicID: topic, messageIDs: toSeq(mids)) @@ -251,9 +252,10 @@ method unsubscribePeer*(g: GossipSub, peer: PeerID) = ## handle peer disconnects ## - trace "unsubscribing gossipsub peer", peer = $peer + trace "unsubscribing gossipsub peer", peer let pubSubPeer = g.peers.getOrDefault(peer) if pubSubPeer.isNil: + trace "no peer to unsubscribe", peer return for t in toSeq(g.gossipsub.keys): @@ -287,7 +289,7 @@ method subscribeTopic*(g: GossipSub, procCall PubSub(g).subscribeTopic(topic, subscribe, peer) logScope: - peer = $peer.id + peer topic if subscribe: @@ -319,7 +321,7 @@ proc handleGraft(g: GossipSub, for graft in grafts: let topic = graft.topicID logScope: - peer = peer.id + peer topic trace "peer grafted topic" @@ -350,7 +352,7 @@ proc handleGraft(g: GossipSub, proc handlePrune(g: GossipSub, peer: PubSubPeer, prunes: seq[ControlPrune]) = for prune in prunes: - trace "peer pruned topic", peer = peer.id, topic = prune.topicID + trace "peer pruned topic", peer, topic = prune.topicID g.mesh.removePeer(prune.topicID, peer) when defined(libp2p_expensive_metrics): @@ -362,7 +364,7 @@ proc handleIHave(g: GossipSub, ihaves: seq[ControlIHave]): ControlIWant = for ihave in ihaves: trace "peer sent ihave", - peer = peer.id, topic = ihave.topicID, msgs = ihave.messageIDs + peer, topic = ihave.topicID, msgs = ihave.messageIDs if ihave.topicID in g.mesh: for m in ihave.messageIDs: @@ -374,7 +376,7 @@ proc handleIWant(g: GossipSub, iwants: seq[ControlIWant]): seq[Message] = for iwant in iwants: for mid in iwant.messageIDs: - trace "peer sent iwant", peer = peer.id, messageID = mid + trace "peer sent iwant", peer, messageID = mid let msg = g.mcache.get(mid) if msg.isSome: result.add(msg.get()) @@ -386,22 +388,19 @@ method rpcHandler*(g: GossipSub, for msg in rpcMsg.messages: # for every message let msgId = g.msgIdProvider(msg) - logScope: - msgId - peer = peer.id if g.seen.put(msgId): - trace "Dropping already-seen message" + trace "Dropping already-seen message", msgId, peer continue g.mcache.put(msgId, msg) if g.verifySignature and not msg.verify(peer.peerId): - debug "Dropping message due to failed signature verification" + debug "Dropping message due to failed signature verification", msgId, peer continue if not (await g.validate(msg)): - trace "Dropping message due to failed validation" + trace "Dropping message due to failed validation", msgId, peer continue var toSendPeers = initHashSet[PubSubPeer]() @@ -414,7 +413,7 @@ method rpcHandler*(g: GossipSub, # In theory, if topics are the same in all messages, we could batch - we'd # also have to be careful to only include validated messages g.broadcast(toSeq(toSendPeers), RPCMsg(messages: @[msg])) - trace "forwared message to peers", peers = toSendPeers.len + trace "forwared message to peers", peers = toSendPeers.len, msgId, peer if rpcMsg.control.isSome: let control = rpcMsg.control.get() @@ -428,7 +427,7 @@ method rpcHandler*(g: GossipSub, if respControl.graft.len > 0 or respControl.prune.len > 0 or respControl.ihave.len > 0 or messages.len > 0: - debug "sending control message", msg = shortLog(respControl) + debug "sending control message", msg = shortLog(respControl), peer g.send( peer, RPCMsg(control: some(respControl), messages: messages)) diff --git a/libp2p/protocols/pubsub/pubsub.nim b/libp2p/protocols/pubsub/pubsub.nim index fed1512e0..f89598ef1 100644 --- a/libp2p/protocols/pubsub/pubsub.nim +++ b/libp2p/protocols/pubsub/pubsub.nim @@ -106,11 +106,9 @@ method rpcHandler*(p: PubSub, peer: PubSubPeer, rpcMsg: RPCMsg) {.async, base.} = ## handle rpc messages - logScope: peer = peer.id - - trace "processing RPC message", msg = rpcMsg.shortLog + trace "processing RPC message", msg = rpcMsg.shortLog, peer for s in rpcMsg.subscriptions: # subscribe/unsubscribe the peer for each topic - trace "about to subscribe to topic", topicId = s.topic + trace "about to subscribe to topic", topicId = s.topic, peer p.subscribeTopic(s.topic, s.subscribe, peer) proc getOrCreatePeer*( @@ -178,11 +176,11 @@ method handleConn*(p: PubSub, try: peer.handler = handler await peer.handle(conn) # spawn peer read loop - trace "pubsub peer handler ended", peer = peer.id + trace "pubsub peer handler ended", conn except CancelledError as exc: raise exc except CatchableError as exc: - trace "exception ocurred in pubsub handle", exc = exc.msg + trace "exception ocurred in pubsub handle", exc = exc.msg, conn finally: await conn.close() diff --git a/libp2p/protocols/pubsub/pubsubpeer.nim b/libp2p/protocols/pubsub/pubsubpeer.nim index 3c7894ac2..ac8ca3303 100644 --- a/libp2p/protocols/pubsub/pubsubpeer.nim +++ b/libp2p/protocols/pubsub/pubsubpeer.nim @@ -49,9 +49,10 @@ func hash*(p: PubSubPeer): Hash = # int is either 32/64, so intptr basically, pubsubpeer is a ref cast[pointer](p).hash -proc id*(p: PubSubPeer): string = - doAssert(not p.isNil, "nil pubsubpeer") - p.peerId.pretty +func shortLog*(p: PubSubPeer): string = + if p.isNil: "PubSubPeer(nil)" + else: shortLog(p.peerId) +chronicles.formatIt(PubSubPeer): shortLog(it) proc connected*(p: PubSubPeer): bool = not p.sendConn.isNil and not @@ -72,25 +73,29 @@ proc sendObservers(p: PubSubPeer, msg: var RPCMsg) = obs.onSend(p, msg) proc handle*(p: PubSubPeer, conn: Connection) {.async.} = - logScope: - oid = $conn.oid - peer = p.id - closed = conn.closed - debug "starting pubsub read loop" + debug "starting pubsub read loop", + conn, peer = p, closed = conn.closed try: try: while not conn.atEof: - trace "waiting for data" + trace "waiting for data", conn, peer = p, closed = conn.closed + let data = await conn.readLp(64 * 1024) - trace "read data from peer", data = data.shortLog + trace "read data from peer", + conn, peer = p, closed = conn.closed, + data = data.shortLog var rmsg = decodeRpcMsg(data) if rmsg.isErr(): - notice "failed to decode msg from peer" + notice "failed to decode msg from peer", + conn, peer = p, closed = conn.closed, + err = rmsg.error() break - trace "decoded msg from peer", msg = rmsg.get().shortLog + trace "decoded msg from peer", + conn, peer = p, closed = conn.closed, + msg = rmsg.get().shortLog # trigger hooks p.recvObservers(rmsg.get()) @@ -98,7 +103,7 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} = for m in rmsg.get().messages: for t in m.topicIDs: # metrics - libp2p_pubsub_received_messages.inc(labelValues = [p.id, t]) + libp2p_pubsub_received_messages.inc(labelValues = [$p.peerId, t]) await p.handler(p, rmsg.get()) finally: @@ -112,9 +117,11 @@ proc handle*(p: PubSubPeer, conn: Connection) {.async.} = # do not need to propogate CancelledError. trace "Unexpected cancellation in PubSubPeer.handle" except CatchableError as exc: - trace "Exception occurred in PubSubPeer.handle", exc = exc.msg + trace "Exception occurred in PubSubPeer.handle", + conn, peer = p, closed = conn.closed, exc = exc.msg finally: - debug "exiting pubsub read loop" + debug "exiting pubsub read loop", + conn, peer = p, closed = conn.closed proc getSendConn(p: PubSubPeer): Future[Connection] {.async.} = ## get a cached send connection or create a new one - will return nil if @@ -127,7 +134,7 @@ proc getSendConn(p: PubSubPeer): Future[Connection] {.async.} = if not current.isNil: if not (current.closed() or current.atEof): # The existing send connection looks like it might work - reuse it - trace "Reusing existing connection", oid = $current.oid + trace "Reusing existing connection", current return current # Send connection is set but broken - get rid of it @@ -171,19 +178,15 @@ proc getSendConn(p: PubSubPeer): Future[Connection] {.async.} = if newConn.isNil: return nil - trace "Sending handshake", oid = $newConn.oid, handshake = shortLog(handshake) + trace "Sending handshake", newConn, handshake = shortLog(handshake) await newConn.writeLp(encodeRpcMsg(handshake)) - trace "Caching new send connection", oid = $newConn.oid + trace "Caching new send connection", newConn p.sendConn = newConn # Start a read loop on the new connection. # All the errors are handled inside `handle()` procedure. asyncSpawn p.handle(newConn) return newConn - except CancelledError as exc: - raise exc - except CatchableError as exc: - return nil finally: if p.dialLock.locked: p.dialLock.release() @@ -200,11 +203,7 @@ proc connect*(p: PubSubPeer) = proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} = doAssert(not isNil(p), "pubsubpeer nil!") - logScope: - peer = p.id - rpcMsg = shortLog(msg) - - trace "sending msg to peer" + trace "sending msg to peer", peer = p, rpcMsg = shortLog(msg) # trigger send hooks var mm = msg # hooks can modify the message @@ -215,37 +214,34 @@ proc sendImpl(p: PubSubPeer, msg: RPCMsg) {.async.} = info "empty message, skipping" return - logScope: - encoded = shortLog(encoded) - - var conn = await p.getSendConn() + var conn: Connection try: - trace "about to send message" + conn = await p.getSendConn() if conn == nil: - debug "Couldn't get send connection, dropping message" + debug "Couldn't get send connection, dropping message", peer = p return - trace "sending encoded msgs to peer", connId = $conn.oid + trace "sending encoded msgs to peer", conn, encoded = shortLog(encoded) await conn.writeLp(encoded) - trace "sent pubsub message to remote", connId = $conn.oid + trace "sent pubsub message to remote", conn when defined(libp2p_expensive_metrics): for x in mm.messages: for t in x.topicIDs: # metrics - libp2p_pubsub_sent_messages.inc(labelValues = [p.id, t]) + libp2p_pubsub_sent_messages.inc(labelValues = [$p.peerId, t]) except CatchableError as exc: # Because we detach the send call from the currently executing task using # asyncCheck, no exceptions may leak out of it - trace "unable to send to remote", exc = exc.msg + debug "unable to send to remote", exc = exc.msg, peer = p # Next time sendConn is used, it will be have its close flag set and thus # will be recycled if not isNil(conn): await conn.close() # This will clean up the send connection if exc is CancelledError: # TODO not handled - debug "Send cancelled" + debug "Send cancelled", peer = p # We'll ask for a new send connection whenever possible if p.sendConn == conn: diff --git a/libp2p/protocols/pubsub/rpc/message.nim b/libp2p/protocols/pubsub/rpc/message.nim index e5d15d008..090bd374d 100644 --- a/libp2p/protocols/pubsub/rpc/message.nim +++ b/libp2p/protocols/pubsub/rpc/message.nim @@ -28,7 +28,7 @@ declareCounter(libp2p_pubsub_sig_verify_success, "pubsub successfully validated declareCounter(libp2p_pubsub_sig_verify_failure, "pubsub failed validated messages") func defaultMsgIdProvider*(m: Message): string = - byteutils.toHex(m.seqno) & m.fromPeer.pretty + byteutils.toHex(m.seqno) & $m.fromPeer proc sign*(msg: Message, privateKey: PrivateKey): CryptoResult[seq[byte]] = ok((? privateKey.sign(PubSubPrefix & encodeMessage(msg))).getBytes()) diff --git a/libp2p/protocols/pubsub/rpc/messages.nim b/libp2p/protocols/pubsub/rpc/messages.nim index 92ba5e332..014f065af 100644 --- a/libp2p/protocols/pubsub/rpc/messages.nim +++ b/libp2p/protocols/pubsub/rpc/messages.nim @@ -86,7 +86,7 @@ func shortLog*(c: ControlMessage): auto = func shortLog*(msg: Message): auto = ( - fromPeer: msg.fromPeer.pretty, + fromPeer: msg.fromPeer.shortLog, data: msg.data.shortLog, seqno: msg.seqno.shortLog, topicIDs: $msg.topicIDs, diff --git a/libp2p/protocols/pubsub/rpc/protobuf.nim b/libp2p/protocols/pubsub/rpc/protobuf.nim index 327b81559..ab6fcc711 100644 --- a/libp2p/protocols/pubsub/rpc/protobuf.nim +++ b/libp2p/protocols/pubsub/rpc/protobuf.nim @@ -187,7 +187,7 @@ proc decodeMessage*(pb: ProtoBuffer): ProtoResult[Message] {.inline.} = trace "decodeMessage: decoding message" var msg: Message if ? pb.getField(1, msg.fromPeer): - trace "decodeMessage: read fromPeer", fromPeer = msg.fromPeer.pretty() + trace "decodeMessage: read fromPeer", fromPeer = msg.fromPeer else: trace "decodeMessage: fromPeer is missing" if ? pb.getField(2, msg.data): diff --git a/libp2p/protocols/secure/noise.nim b/libp2p/protocols/secure/noise.nim index 726199259..c48ef5456 100644 --- a/libp2p/protocols/secure/noise.nim +++ b/libp2p/protocols/secure/noise.nim @@ -7,6 +7,7 @@ ## This file may not be copied, modified, or distributed except according to ## those terms. +import std/[oids, strformat] import chronos import chronicles import bearssl @@ -88,6 +89,12 @@ type # Utility +func shortLog*(conn: NoiseConnection): auto = + if conn.isNil: "NoiseConnection(nil)" + elif conn.peerInfo.isNil: $conn.oid + else: &"{shortLog(conn.peerInfo.peerId)}:{conn.oid}" +chronicles.formatIt(NoiseConnection): shortLog(it) + proc genKeyPair(rng: var BrHmacDrbgContext): KeyPair = result.privateKey = Curve25519Key.random(rng) result.publicKey = result.privateKey.public() @@ -392,13 +399,13 @@ method readMessage*(sconn: NoiseConnection): Future[seq[byte]] {.async.} = var besize: array[2, byte] await sconn.stream.readExactly(addr besize[0], besize.len) let size = uint16.fromBytesBE(besize).int # Cannot overflow - trace "receiveEncryptedMessage", size, peer = $sconn + trace "receiveEncryptedMessage", size, sconn if size > 0: var buffer = newSeq[byte](size) await sconn.stream.readExactly(addr buffer[0], buffer.len) return sconn.readCs.decryptWithAd([], buffer) else: - trace "Received 0-length message", conn = $sconn + trace "Received 0-length message", sconn method write*(sconn: NoiseConnection, message: seq[byte]): Future[void] {.async.} = if message.len == 0: @@ -418,14 +425,14 @@ method write*(sconn: NoiseConnection, message: seq[byte]): Future[void] {.async. lesize = cipher.len.uint16 besize = lesize.toBytesBE outbuf = newSeqOfCap[byte](cipher.len + 2) - trace "sendEncryptedMessage", size = lesize, peer = $sconn, left, offset + trace "sendEncryptedMessage", sconn, size = lesize, left, offset outbuf &= besize outbuf &= cipher await sconn.stream.write(outbuf) sconn.activity = true method handshake*(p: Noise, conn: Connection, initiator: bool): Future[SecureConn] {.async.} = - trace "Starting Noise handshake", initiator, peer = $conn + trace "Starting Noise handshake", conn, initiator # https://github.com/libp2p/specs/tree/master/noise#libp2p-data-in-handshake-messages let @@ -469,7 +476,7 @@ method handshake*(p: Noise, conn: Connection, initiator: bool): Future[SecureCon if not remoteSig.verify(verifyPayload, remotePubKey): raise newException(NoiseHandshakeError, "Noise handshake signature verify failed.") else: - trace "Remote signature verified", peer = $conn + trace "Remote signature verified", conn if initiator and not isNil(conn.peerInfo): let pid = PeerID.init(remotePubKey) @@ -480,7 +487,7 @@ method handshake*(p: Noise, conn: Connection, initiator: bool): Future[SecureCon failedKey: PublicKey discard extractPublicKey(conn.peerInfo.peerId, failedKey) debug "Noise handshake, peer infos don't match!", - initiator, dealt_peer = $conn.peerInfo.id, + initiator, dealt_peer = conn, dealt_key = $failedKey, received_peer = $pid, received_key = $remotePubKey raise newException(NoiseHandshakeError, "Noise handshake, peer infos don't match! " & $pid & " != " & $conn.peerInfo.peerId) diff --git a/libp2p/protocols/secure/secio.nim b/libp2p/protocols/secure/secio.nim index e40ab2d00..71bcb4dc2 100644 --- a/libp2p/protocols/secure/secio.nim +++ b/libp2p/protocols/secure/secio.nim @@ -6,7 +6,8 @@ ## at your option. ## This file may not be copied, modified, or distributed except according to ## those terms. -import chronos, chronicles, oids, stew/endians2, bearssl +import std/[oids, strformat] +import chronos, chronicles, stew/endians2, bearssl import nimcrypto/[hmac, sha2, sha, hash, rijndael, twofish, bcmode] import secure, ../../stream/connection, @@ -69,6 +70,12 @@ type SecioError* = object of CatchableError +func shortLog*(conn: SecioConn): auto = + if conn.isNil: "SecioConn(nil)" + elif conn.peerInfo.isNil: $conn.oid + else: &"{shortLog(conn.peerInfo.peerId)}:{conn.oid}" +chronicles.formatIt(SecioConn): shortLog(it) + proc init(mac: var SecureMac, hash: string, key: openarray[byte]) = if hash == "SHA256": mac = SecureMac(kind: SecureMacType.Sha256) @@ -184,17 +191,17 @@ proc readRawMessage(conn: Connection): Future[seq[byte]] {.async.} = trace "Recieved message header", header = lengthBuf.shortLog, length = length if length > SecioMaxMessageSize: # Verify length before casting! - trace "Received size of message exceed limits", conn = $conn, length = length + trace "Received size of message exceed limits", conn, length = length raise (ref SecioError)(msg: "Message exceeds maximum length") if length > 0: var buf = newSeq[byte](int(length)) await conn.readExactly(addr buf[0], buf.len) trace "Received message body", - conn = $conn, length = buf.len, buff = buf.shortLog + conn, length = buf.len, buff = buf.shortLog return buf - trace "Discarding 0-length payload", conn = $conn + trace "Discarding 0-length payload", conn method readMessage*(sconn: SecioConn): Future[seq[byte]] {.async.} = ## Read message from channel secure connection ``sconn``. @@ -312,12 +319,12 @@ method handshake*(s: Secio, conn: Connection, initiator: bool = false): Future[S var answer = await transactMessage(conn, request) if len(answer) == 0: - trace "Proposal exchange failed", conn = $conn + trace "Proposal exchange failed", conn raise (ref SecioError)(msg: "Proposal exchange failed") if not decodeProposal(answer, remoteNonce, remoteBytesPubkey, remoteExchanges, remoteCiphers, remoteHashes): - trace "Remote proposal decoding failed", conn = $conn + trace "Remote proposal decoding failed", conn raise (ref SecioError)(msg: "Remote proposal decoding failed") if not remotePubkey.init(remoteBytesPubkey): @@ -354,11 +361,11 @@ method handshake*(s: Secio, conn: Connection, initiator: bool = false): Future[S var localExchange = createExchange(epubkey, signature.getBytes()) var remoteExchange = await transactMessage(conn, localExchange) if len(remoteExchange) == 0: - trace "Corpus exchange failed", conn = $conn + trace "Corpus exchange failed", conn raise (ref SecioError)(msg: "Corpus exchange failed") if not decodeExchange(remoteExchange, remoteEBytesPubkey, remoteEBytesSig): - trace "Remote exchange decoding failed", conn = $conn + trace "Remote exchange decoding failed", conn raise (ref SecioError)(msg: "Remote exchange decoding failed") if not remoteESignature.init(remoteEBytesSig): diff --git a/libp2p/protocols/secure/secure.nim b/libp2p/protocols/secure/secure.nim index 3f5338e1b..7caf6686c 100644 --- a/libp2p/protocols/secure/secure.nim +++ b/libp2p/protocols/secure/secure.nim @@ -7,7 +7,7 @@ ## This file may not be copied, modified, or distributed except according to ## those terms. -import options +import std/[options, strformat] import chronos, chronicles, bearssl import ../protocol, ../../stream/streamseq, @@ -25,12 +25,18 @@ type stream*: Connection buf: StreamSeq -proc init*[T: SecureConn](C: type T, - conn: Connection, - peerInfo: PeerInfo, - observedAddr: Multiaddress, - timeout: Duration = DefaultConnectionTimeout): T = - result = C(stream: conn, +func shortLog*(conn: SecureConn): auto = + if conn.isNil: "SecureConn(nil)" + elif conn.peerInfo.isNil: $conn.oid + else: &"{shortLog(conn.peerInfo.peerId)}:{conn.oid}" +chronicles.formatIt(SecureConn): shortLog(it) + +proc init*(T: type SecureConn, + conn: Connection, + peerInfo: PeerInfo, + observedAddr: Multiaddress, + timeout: Duration = DefaultConnectionTimeout): T = + result = T(stream: conn, peerInfo: peerInfo, observedAddr: observedAddr, closeEvent: conn.closeEvent, @@ -71,7 +77,7 @@ proc handleConn*(s: Secure, # do not need to propogate CancelledError. discard except CatchableError as exc: - trace "error cleaning up secure connection", errMsg = exc.msg + trace "error cleaning up secure connection", err = exc.msg, sconn if not isNil(sconn): # All the errors are handled inside `cleanup()` procedure. @@ -83,18 +89,18 @@ method init*(s: Secure) {.gcsafe.} = procCall LPProtocol(s).init() proc handle(conn: Connection, proto: string) {.async, gcsafe.} = - trace "handling connection upgrade", proto + trace "handling connection upgrade", proto, conn try: # We don't need the result but we # definitely need to await the handshake discard await s.handleConn(conn, false) - trace "connection secured" + trace "connection secured", conn except CancelledError as exc: - warn "securing connection canceled" + warn "securing connection canceled", conn await conn.close() raise exc except CatchableError as exc: - warn "securing connection failed", msg = exc.msg + warn "securing connection failed", err = exc.msg, conn await conn.close() s.handler = handle diff --git a/libp2p/stream/bufferstream.nim b/libp2p/stream/bufferstream.nim index 25840e498..9ca59d744 100644 --- a/libp2p/stream/bufferstream.nim +++ b/libp2p/stream/bufferstream.nim @@ -30,7 +30,7 @@ ## will suspend until either the amount of elements in the ## buffer goes below ``maxSize`` or more data becomes available. -import deques, math +import std/[deques, math, strformat] import chronos, chronicles, metrics import ../stream/connection @@ -100,6 +100,12 @@ proc newAlreadyPipedError*(): ref CatchableError {.inline.} = proc newNotWritableError*(): ref CatchableError {.inline.} = result = newException(NotWritableError, "stream is not writable") +func shortLog*(s: BufferStream): auto = + if s.isNil: "BufferStream(nil)" + elif s.peerInfo.isNil: $s.oid + else: &"{shortLog(s.peerInfo.peerId)}:{s.oid}" +chronicles.formatIt(BufferStream): shortLog(it) + proc requestReadBytes(s: BufferStream): Future[void] = ## create a future that will complete when more ## data becomes available in the read buffer @@ -142,7 +148,7 @@ proc initBufferStream*(s: BufferStream, await s.writeLock.acquire() await handler(data) - trace "created bufferstream", oid = $s.oid + trace "created bufferstream", s proc newBufferStream*(handler: WriteHandler = nil, size: int = DefaultBufferSize, @@ -206,7 +212,7 @@ proc drainBuffer*(s: BufferStream) {.async.} = ## wait for all data in the buffer to be consumed ## - trace "draining buffer", len = s.len, oid = $s.oid + trace "draining buffer", len = s.len, s while s.len > 0: await s.dataReadEvent.wait() s.dataReadEvent.clear() @@ -296,7 +302,7 @@ method close*(s: BufferStream) {.async, gcsafe.} = try: ## close the stream and clear the buffer if not s.isClosed: - trace "closing bufferstream", oid = $s.oid + trace "closing bufferstream", s s.isEof = true for r in s.readReqs: if not(isNil(r)) and not(r.finished()): @@ -306,11 +312,11 @@ method close*(s: BufferStream) {.async, gcsafe.} = await procCall Connection(s).close() inc getBufferStreamTracker().closed - trace "bufferstream closed", oid = $s.oid + trace "bufferstream closed", s else: trace "attempt to close an already closed bufferstream", - trace = getStackTrace(), oid = $s.oid + trace = getStackTrace(), s except CancelledError as exc: raise exc except CatchableError as exc: - trace "error closing buffer stream", exc = exc.msg + trace "error closing buffer stream", exc = exc.msg, s diff --git a/libp2p/stream/chronosstream.nim b/libp2p/stream/chronosstream.nim index e6d28cf26..b6bb99d73 100644 --- a/libp2p/stream/chronosstream.nim +++ b/libp2p/stream/chronosstream.nim @@ -7,7 +7,7 @@ ## This file may not be copied, modified, or distributed except according to ## those terms. -import oids +import std/[oids, strformat] import chronos, chronicles import connection @@ -21,6 +21,12 @@ type ChronosStream* = ref object of Connection client: StreamTransport +func shortLog*(conn: ChronosStream): string = + if conn.isNil: "ChronosStream(nil)" + elif conn.peerInfo.isNil: $conn.oid + else: &"{shortLog(conn.peerInfo.peerId)}:{conn.oid}" +chronicles.formatIt(ChronosStream): shortLog(it) + method initStream*(s: ChronosStream) = if s.objName.len == 0: s.objName = "ChronosStream" @@ -88,7 +94,7 @@ method close*(s: ChronosStream) {.async.} = try: if not s.isClosed: trace "shutting down chronos stream", address = $s.client.remoteAddress(), - oid = $s.oid + s if not s.client.closed(): await s.client.closeWait() @@ -96,4 +102,4 @@ method close*(s: ChronosStream) {.async.} = except CancelledError as exc: raise exc except CatchableError as exc: - trace "error closing chronosstream", exc = exc.msg + trace "error closing chronosstream", exc = exc.msg, s diff --git a/libp2p/stream/connection.nim b/libp2p/stream/connection.nim index 3359e3344..20343ea81 100644 --- a/libp2p/stream/connection.nim +++ b/libp2p/stream/connection.nim @@ -7,13 +7,13 @@ ## This file may not be copied, modified, or distributed except according to ## those terms. -import hashes, oids +import std/[hashes, oids, strformat] import chronicles, chronos, metrics import lpstream, ../multiaddress, ../peerinfo -export lpstream +export lpstream, peerinfo logScope: topics = "connection" @@ -66,6 +66,12 @@ proc setupConnectionTracker(): ConnectionTracker = result.isLeaked = leakTransport addTracker(ConnectionTrackerName, result) +func shortLog*(conn: Connection): string = + if conn.isNil: "Connection(nil)" + elif conn.peerInfo.isNil: $conn.oid + else: &"{shortLog(conn.peerInfo.peerId)}:{conn.oid}" +chronicles.formatIt(Connection): shortLog(it) + method initStream*(s: Connection) = if s.objName.len == 0: s.objName = "Connection" @@ -77,7 +83,7 @@ method initStream*(s: Connection) = s.timeoutHandler = proc() {.async.} = await s.close() - trace "timeout set at", timeout = s.timeout.millis + trace "timeout set at", timeout = s.timeout.millis, s doAssert(isNil(s.timerTaskFut)) # doAssert(s.timeout > 0.millis) if s.timeout > 0.millis: @@ -94,10 +100,6 @@ method close*(s: Connection) {.async.} = await procCall LPStream(s).close() inc getConnectionTracker().closed -proc `$`*(conn: Connection): string = - if not isNil(conn.peerInfo): - result = conn.peerInfo.id - func hash*(p: Connection): Hash = cast[pointer](p).hash @@ -110,9 +112,6 @@ proc timeoutMonitor(s: Connection) {.async, gcsafe.} = ## be reset ## - logScope: - oid = $s.oid - try: while true: await sleepAsync(s.timeout) @@ -127,14 +126,14 @@ proc timeoutMonitor(s: Connection) {.async, gcsafe.} = break # reset channel on innactivity timeout - trace "Connection timed out" + trace "Connection timed out", s if not(isNil(s.timeoutHandler)): await s.timeoutHandler() except CancelledError as exc: raise exc except CatchableError as exc: - trace "exception in timeout", exc = exc.msg + trace "exception in timeout", exc = exc.msg, s proc init*(C: type Connection, peerInfo: PeerInfo, diff --git a/libp2p/switch.nim b/libp2p/switch.nim index d31067c71..b12805289 100644 --- a/libp2p/switch.nim +++ b/libp2p/switch.nim @@ -30,9 +30,6 @@ import stream/connection, peerid, errors -chronicles.formatIt(PeerInfo): $it -chronicles.formatIt(PeerID): $it - logScope: topics = "switch" @@ -101,7 +98,7 @@ proc triggerConnEvent(s: Switch, peerId: PeerID, event: ConnEvent) {.async, gcsa except CancelledError as exc: raise exc except CatchableError as exc: # handlers should not raise! - warn "exception in trigger ConnEvents", exc = exc.msg + warn "exception in trigger ConnEvents", exc = exc.msg, peerId proc disconnect*(s: Switch, peerId: PeerID) {.async, gcsafe.} @@ -120,7 +117,7 @@ proc secure(s: Switch, conn: Connection): Future[Connection] {.async, gcsafe.} = if manager.len == 0: raise newException(CatchableError, "Unable to negotiate a secure channel!") - trace "securing connection", codec = manager + trace "securing connection", codec = manager, conn let secureProtocol = s.secureManagers.filterIt(it.codec == manager) # ms.select should deal with the correctness of this @@ -154,7 +151,7 @@ proc identify(s: Switch, conn: Connection) {.async, gcsafe.} = if info.protos.len > 0: conn.peerInfo.protocols = info.protos - trace "identify: identified remote peer", peer = $conn.peerInfo + trace "identify: identified remote peer", conn proc identify(s: Switch, muxer: Muxer) {.async, gcsafe.} = # new stream for identify @@ -171,14 +168,14 @@ proc identify(s: Switch, muxer: Muxer) {.async, gcsafe.} = proc mux(s: Switch, conn: Connection): Future[Muxer] {.async, gcsafe.} = ## mux incoming connection - trace "muxing connection", peer = $conn + trace "muxing connection", conn if s.muxers.len == 0: - warn "no muxers registered, skipping upgrade flow" + warn "no muxers registered, skipping upgrade flow", conn return let muxerName = await s.ms.select(conn, toSeq(s.muxers.keys())) if muxerName.len == 0 or muxerName == "na": - debug "no muxer available, early exit", peer = $conn + debug "no muxer available, early exit", conn return # create new muxer for connection @@ -190,13 +187,13 @@ proc mux(s: Switch, conn: Connection): Future[Muxer] {.async, gcsafe.} = s.connManager.storeOutgoing(muxer.connection) s.connManager.storeMuxer(muxer) - trace "found a muxer", name = muxerName, peer = $conn + trace "found a muxer", name = muxerName, conn # start muxer read loop - the future will complete when loop ends let handlerFut = muxer.handle() # store it in muxed connections if we have a peer for it - trace "adding muxer for peer", peer = conn.peerInfo.id + trace "adding muxer for peer", conn s.connManager.storeMuxer(muxer, handlerFut) # update muxer with handler return muxer @@ -214,7 +211,7 @@ proc upgradeOutgoing(s: Switch, conn: Connection): Future[Connection] {.async, g raise newException(CatchableError, "current version of nim-libp2p requires that secure protocol negotiates peerid") - trace "upgrading connection", conn = $sconn, uoid = $conn.oid + trace "upgrading connection", conn let muxer = await s.mux(sconn) # mux it if possible if muxer == nil: # TODO this might be relaxed in the future @@ -227,21 +224,19 @@ proc upgradeOutgoing(s: Switch, conn: Connection): Future[Connection] {.async, g # Identify is non-essential, though if it fails, it might indicate that # the connection was closed already - this will be picked up by the read # loop - debug "Could not identify connection", - err = exc.msg, conn = $conn, uoid = $conn.oid + debug "Could not identify connection", err = exc.msg, conn if isNil(sconn.peerInfo): await sconn.close() raise newException(CatchableError, "unable to identify connection, stopping upgrade") - trace "successfully upgraded outgoing connection", - conn = $sconn, uoid = $conn.oid, oid = $sconn.oid + trace "successfully upgraded outgoing connection", conn, sconn return sconn proc upgradeIncoming(s: Switch, conn: Connection) {.async, gcsafe.} = - trace "upgrading incoming connection", conn = $conn, oid = $conn.oid + trace "upgrading incoming connection", conn let ms = newMultistream() # secure incoming connections @@ -250,7 +245,7 @@ proc upgradeIncoming(s: Switch, conn: Connection) {.async, gcsafe.} = {.async, gcsafe, closure.} = var sconn: Connection - trace "Securing connection", oid = $conn.oid + trace "Securing connection", conn let secure = s.secureManagers.filterIt(it.codec == proto)[0] try: @@ -271,8 +266,7 @@ proc upgradeIncoming(s: Switch, conn: Connection) {.async, gcsafe.} = except CancelledError as exc: raise exc except CatchableError as exc: - debug "ending secured handler", - err = exc.msg, conn = $conn, oid = $conn.oid + debug "ending secured handler", err = exc.msg, conn if (await ms.select(conn)): # just handshake # add the secure handlers @@ -302,27 +296,25 @@ proc internalConnect(s: Switch, # This connection should already have been removed from the connection # manager - it's essentially a bug that we end up here - we'll fail # for now, hoping that this will clean themselves up later... - warn "dead connection in connection manager", peer = $peerId + warn "dead connection in connection manager", peerId await conn.close() raise newException(CatchableError, "Zombie connection encountered") - trace "Reusing existing connection", - oid = $conn.oid, direction = $conn.dir, peer = $peerId - + trace "Reusing existing connection", conn return conn - trace "Dialing peer" + trace "Dialing peer", peerId for t in s.transports: # for each transport for a in addrs: # for each address if t.handles(a): # check if it can dial it - trace "Dialing address", address = $a + trace "Dialing address", address = $a, peerId let dialed = try: await t.dial(a) except CancelledError as exc: - trace "dialing canceled", exc = exc.msg, peer = $peerId + trace "dialing canceled", exc = exc.msg, peerId raise exc except CatchableError as exc: - trace "dialing failed", exc = exc.msg, peer = $peerId + trace "dialing failed", exc = exc.msg, peerId libp2p_failed_dials.inc() continue # Try the next address @@ -337,7 +329,7 @@ proc internalConnect(s: Switch, # If we failed to establish the connection through one transport, # we won't succeeed through another - no use in trying again await dialed.close() - debug "upgrade failed", exc = exc.msg + debug "Upgrade failed", exc = exc.msg, peerId if exc isnot CancelledError: libp2p_failed_upgrade.inc() raise exc @@ -345,10 +337,7 @@ proc internalConnect(s: Switch, doAssert not isNil(upgraded), "connection died after upgradeOutgoing" conn = upgraded - trace "dial successful", - peer = $peerId, - oid = $upgraded.oid, - peerInfo = shortLog(upgraded.peerInfo) + trace "dial successful", conn, peerInfo = upgraded.peerInfo break finally: if lock.locked(): @@ -374,10 +363,10 @@ proc internalConnect(s: Switch, # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. trace "Unexpected cancellation in switch peer connect cleanup", - peer = $peerId + conn except CatchableError as exc: trace "Unexpected exception in switch peer connect cleanup", - errMsg = exc.msg, peer = $peerId + errMsg = exc.msg, conn # All the errors are handled inside `cleanup()` procedure. asyncSpawn peerCleanup() @@ -389,9 +378,7 @@ proc connect*(s: Switch, peerId: PeerID, addrs: seq[MultiAddress]) {.async.} = discard await s.internalConnect(peerId, addrs) proc negotiateStream(s: Switch, stream: Connection, proto: string): Future[Connection] {.async.} = - trace "Attempting to select remote", proto = proto, - streamOid = $stream.oid, - oid = $stream.oid + trace "Attempting to select remote", proto = proto, stream if not await s.ms.select(stream, proto): await stream.close() @@ -430,11 +417,11 @@ proc dial*(s: Switch, return await s.negotiateStream(stream, proto) except CancelledError as exc: - trace "dial canceled" + trace "dial canceled", conn await cleanup() raise exc except CatchableError as exc: - trace "error dialing", exc = exc.msg + trace "error dialing", exc = exc.msg, conn await cleanup() raise exc @@ -450,7 +437,7 @@ proc mount*[T: LPProtocol](s: Switch, proto: T) {.gcsafe.} = s.ms.addHandler(proto.codec, proto) proc start*(s: Switch): Future[seq[Future[void]]] {.async, gcsafe.} = - trace "starting switch for peer", peerInfo = shortLog(s.peerInfo) + trace "starting switch for peer", peerInfo = s.peerInfo proc handle(conn: Connection): Future[void] {.async, closure, gcsafe.} = try: @@ -458,7 +445,7 @@ proc start*(s: Switch): Future[seq[Future[void]]] {.async, gcsafe.} = except CancelledError as exc: raise exc except CatchableError as exc: - trace "Exception occurred in Switch.start", exc = exc.msg + trace "Error in connection handler", exc = exc.msg, conn finally: await conn.close() @@ -470,7 +457,7 @@ proc start*(s: Switch): Future[seq[Future[void]]] {.async, gcsafe.} = s.peerInfo.addrs[i] = t.ma # update peer's address startFuts.add(server) - debug "started libp2p node", peer = $s.peerInfo, addrs = s.peerInfo.addrs + debug "started libp2p node", peerInfo = s.peerInfo result = startFuts # listen for incoming connections proc stop*(s: Switch) {.async.} = @@ -507,7 +494,7 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = # Identify is non-essential, though if it fails, it might indicate that # the connection was closed already - this will be picked up by the read # loop - debug "Could not identify connection", err = exc.msg + debug "Could not identify connection", err = exc.msg, muxer try: let peerId = muxer.connection.peerInfo.peerId @@ -523,7 +510,7 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = debug "Unexpected cancellation in switch muxer cleanup" except CatchableError as exc: debug "Unexpected exception in switch muxer cleanup", - errMsg = exc.msg + errMsg = exc.msg, muxer proc peerStartup() {.async.} = try: @@ -533,10 +520,10 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = except CancelledError: # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. - debug "Unexpected cancellation in switch muxer startup" + debug "Unexpected cancellation in switch muxer startup", muxer except CatchableError as exc: debug "Unexpected exception in switch muxer startup", - errMsg = exc.msg + errMsg = exc.msg, muxer # All the errors are handled inside `peerStartup()` procedure. asyncSpawn peerStartup() @@ -550,7 +537,7 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = except CatchableError as exc: await muxer.close() libp2p_failed_upgrade.inc() - trace "exception in muxer handler", exc = exc.msg + trace "exception in muxer handler", exc = exc.msg, muxer proc newSwitch*(peerInfo: PeerInfo, transports: seq[Transport], @@ -573,7 +560,7 @@ proc newSwitch*(peerInfo: PeerInfo, let s = result # can't capture result result.streamHandler = proc(stream: Connection) {.async, gcsafe.} = try: - trace "handling connection for", peerInfo = $stream + trace "handling connection for", stream defer: if not(isNil(stream)): await stream.close() @@ -581,7 +568,7 @@ proc newSwitch*(peerInfo: PeerInfo, except CancelledError as exc: raise exc except CatchableError as exc: - trace "exception in stream handler", exc = exc.msg + trace "exception in stream handler", exc = exc.msg, stream result.mount(identity) for key, val in muxers: diff --git a/tests/pubsub/testgossipsub.nim b/tests/pubsub/testgossipsub.nim index 84870caeb..924836e4a 100644 --- a/tests/pubsub/testgossipsub.nim +++ b/tests/pubsub/testgossipsub.nim @@ -517,9 +517,9 @@ suite "GossipSub": closureScope: var dialerNode = dialer handler = proc(topic: string, data: seq[byte]) {.async, gcsafe, closure.} = - if dialerNode.peerInfo.id notin seen: - seen[dialerNode.peerInfo.id] = 0 - seen[dialerNode.peerInfo.id].inc + if $dialerNode.peerInfo.peerId notin seen: + seen[$dialerNode.peerInfo.peerId] = 0 + seen[$dialerNode.peerInfo.peerId].inc check topic == "foobar" if not seenFut.finished() and seen.len >= runs: seenFut.complete() @@ -529,8 +529,8 @@ suite "GossipSub": await allFuturesThrowing(subs) tryPublish await wait(nodes[0].publish("foobar", - cast[seq[byte]]("from node " & - nodes[1].peerInfo.id)), + toBytes("from node " & + $nodes[1].peerInfo.peerId)), 1.minutes), runs, 5.seconds await wait(seenFut, 2.minutes) @@ -567,7 +567,7 @@ suite "GossipSub": await allFuturesThrowing(nodes.mapIt(it.start())) await subscribeNodes(nodes) - var seen: Table[string, int] + var seen: Table[PeerID, int] var subs: seq[Future[void]] var seenFut = newFuture[void]() for dialer in nodes: @@ -576,9 +576,9 @@ suite "GossipSub": var dialerNode = dialer handler = proc(topic: string, data: seq[byte]) {.async, gcsafe, closure.} = - if dialerNode.peerInfo.id notin seen: - seen[dialerNode.peerInfo.id] = 0 - seen[dialerNode.peerInfo.id].inc + if dialerNode.peerInfo.peerId notin seen: + seen[dialerNode.peerInfo.peerId] = 0 + seen[dialerNode.peerInfo.peerId].inc check topic == "foobar" if not seenFut.finished() and seen.len >= runs: seenFut.complete() @@ -588,8 +588,8 @@ suite "GossipSub": await allFuturesThrowing(subs) tryPublish await wait(nodes[0].publish("foobar", - cast[seq[byte]]("from node " & - nodes[1].peerInfo.id)), + toBytes("from node " & + $nodes[1].peerInfo.peerId)), 1.minutes), 2, 5.seconds await wait(seenFut, 5.minutes) diff --git a/tests/testpeer.nim b/tests/testpeer.nim index c9425f0cf..3418e2091 100644 --- a/tests/testpeer.nim +++ b/tests/testpeer.nim @@ -200,10 +200,10 @@ suite "Peer testing suite": p1 == p2 p1 == p4 p2 == p4 - p1.pretty() == PeerIDs[i] - p2.pretty() == PeerIDs[i] - p3.pretty() == PeerIDs[i] - p4.pretty() == PeerIDs[i] + $p1 == PeerIDs[i] + $p2 == PeerIDs[i] + $p3 == PeerIDs[i] + $p4 == PeerIDs[i] p1.match(seckey) == true p1.match(pubkey) == true p1.getBytes() == p2.getBytes() From 2b72d485a39715c33dd887b8c8639cc5303e70f4 Mon Sep 17 00:00:00 2001 From: Jacek Sieka Date: Mon, 7 Sep 2020 14:15:11 +0200 Subject: [PATCH 09/10] a few more log fixes (#355) --- libp2p/switch.nim | 119 ++++++++++++++++++++++++---------------------- 1 file changed, 63 insertions(+), 56 deletions(-) diff --git a/libp2p/switch.nim b/libp2p/switch.nim index b12805289..9269898f4 100644 --- a/libp2p/switch.nim +++ b/libp2p/switch.nim @@ -44,7 +44,8 @@ declareCounter(libp2p_failed_dials, "failed dials") declareCounter(libp2p_failed_upgrade, "peers failed upgrade") type - NoPubSubException* = object of CatchableError + UpgradeFailedError* = object of CatchableError + DialFailedError* = object of CatchableError ConnEventKind* {.pure.} = enum Connected, # A connection was made and securely upgraded - there may be @@ -111,13 +112,13 @@ proc isConnected*(s: Switch, peerId: PeerID): bool = proc secure(s: Switch, conn: Connection): Future[Connection] {.async, gcsafe.} = if s.secureManagers.len <= 0: - raise newException(CatchableError, "No secure managers registered!") + raise newException(UpgradeFailedError, "No secure managers registered!") let manager = await s.ms.select(conn, s.secureManagers.mapIt(it.codec)) if manager.len == 0: - raise newException(CatchableError, "Unable to negotiate a secure channel!") + raise newException(UpgradeFailedError, "Unable to negotiate a secure channel!") - trace "securing connection", codec = manager, conn + trace "Securing connection", codec = manager, conn let secureProtocol = s.secureManagers.filterIt(it.codec == manager) # ms.select should deal with the correctness of this @@ -133,7 +134,7 @@ proc identify(s: Switch, conn: Connection) {.async, gcsafe.} = let info = await s.identity.identify(conn, conn.peerInfo) if info.pubKey.isNone and isNil(conn): - raise newException(CatchableError, + raise newException(UpgradeFailedError, "no public key provided and no existing peer identity found") if isNil(conn.peerInfo): @@ -151,7 +152,7 @@ proc identify(s: Switch, conn: Connection) {.async, gcsafe.} = if info.protos.len > 0: conn.peerInfo.protocols = info.protos - trace "identify: identified remote peer", conn + trace "identified remote peer", conn, peerInfo = shortLog(conn.peerInfo) proc identify(s: Switch, muxer: Muxer) {.async, gcsafe.} = # new stream for identify @@ -168,7 +169,7 @@ proc identify(s: Switch, muxer: Muxer) {.async, gcsafe.} = proc mux(s: Switch, conn: Connection): Future[Muxer] {.async, gcsafe.} = ## mux incoming connection - trace "muxing connection", conn + trace "Muxing connection", conn if s.muxers.len == 0: warn "no muxers registered, skipping upgrade flow", conn return @@ -184,7 +185,8 @@ proc mux(s: Switch, conn: Connection): Future[Muxer] {.async, gcsafe.} = # install stream handler muxer.streamHandler = s.streamHandler - s.connManager.storeOutgoing(muxer.connection) + s.connManager.storeOutgoing(conn) + trace "Storing muxer", conn s.connManager.storeMuxer(muxer) trace "found a muxer", name = muxerName, conn @@ -193,7 +195,7 @@ proc mux(s: Switch, conn: Connection): Future[Muxer] {.async, gcsafe.} = let handlerFut = muxer.handle() # store it in muxed connections if we have a peer for it - trace "adding muxer for peer", conn + trace "Storing muxer with handler", conn s.connManager.storeMuxer(muxer, handlerFut) # update muxer with handler return muxer @@ -202,20 +204,21 @@ proc disconnect*(s: Switch, peerId: PeerID): Future[void] {.gcsafe.} = s.connManager.dropPeer(peerId) proc upgradeOutgoing(s: Switch, conn: Connection): Future[Connection] {.async, gcsafe.} = + trace "Upgrading outgoing connection", conn + let sconn = await s.secure(conn) # secure the connection if isNil(sconn): - raise newException(CatchableError, + raise newException(UpgradeFailedError, "unable to secure connection, stopping upgrade") if sconn.peerInfo.isNil: - raise newException(CatchableError, + raise newException(UpgradeFailedError, "current version of nim-libp2p requires that secure protocol negotiates peerid") - trace "upgrading connection", conn let muxer = await s.mux(sconn) # mux it if possible if muxer == nil: # TODO this might be relaxed in the future - raise newException(CatchableError, + raise newException(UpgradeFailedError, "a muxer is required for outgoing connections") try: @@ -228,28 +231,26 @@ proc upgradeOutgoing(s: Switch, conn: Connection): Future[Connection] {.async, g if isNil(sconn.peerInfo): await sconn.close() - raise newException(CatchableError, - "unable to identify connection, stopping upgrade") + raise newException(UpgradeFailedError, + "No peerInfo for connection, stopping upgrade") - trace "successfully upgraded outgoing connection", conn, sconn + trace "Upgraded outgoing connection", conn, sconn return sconn proc upgradeIncoming(s: Switch, conn: Connection) {.async, gcsafe.} = - trace "upgrading incoming connection", conn + trace "Upgrading incoming connection", conn let ms = newMultistream() # secure incoming connections proc securedHandler (conn: Connection, proto: string) {.async, gcsafe, closure.} = - - var sconn: Connection trace "Securing connection", conn let secure = s.secureManagers.filterIt(it.codec == proto)[0] try: - sconn = await secure.secure(conn, false) + var sconn = await secure.secure(conn, false) if isNil(sconn): return @@ -266,7 +267,9 @@ proc upgradeIncoming(s: Switch, conn: Connection) {.async, gcsafe.} = except CancelledError as exc: raise exc except CatchableError as exc: - debug "ending secured handler", err = exc.msg, conn + debug "Exception in secure handler", err = exc.msg, conn + + trace "Ending secured handler", conn if (await ms.select(conn)): # just handshake # add the secure handlers @@ -296,11 +299,12 @@ proc internalConnect(s: Switch, # This connection should already have been removed from the connection # manager - it's essentially a bug that we end up here - we'll fail # for now, hoping that this will clean themselves up later... - warn "dead connection in connection manager", peerId + warn "dead connection in connection manager", conn await conn.close() - raise newException(CatchableError, "Zombie connection encountered") + raise newException(DialFailedError, "Zombie connection encountered") + + trace "Reusing existing connection", conn, direction = $conn.dir - trace "Reusing existing connection", conn return conn trace "Dialing peer", peerId @@ -311,10 +315,10 @@ proc internalConnect(s: Switch, let dialed = try: await t.dial(a) except CancelledError as exc: - trace "dialing canceled", exc = exc.msg, peerId + trace "Dialing canceled", exc = exc.msg, peerId raise exc except CatchableError as exc: - trace "dialing failed", exc = exc.msg, peerId + trace "Dialing failed", exc = exc.msg, peerId libp2p_failed_dials.inc() continue # Try the next address @@ -337,7 +341,7 @@ proc internalConnect(s: Switch, doAssert not isNil(upgraded), "connection died after upgradeOutgoing" conn = upgraded - trace "dial successful", conn, peerInfo = upgraded.peerInfo + trace "Dial successful", conn, peerInfo = conn.peerInfo break finally: if lock.locked(): @@ -371,27 +375,25 @@ proc internalConnect(s: Switch, # All the errors are handled inside `cleanup()` procedure. asyncSpawn peerCleanup() - return conn proc connect*(s: Switch, peerId: PeerID, addrs: seq[MultiAddress]) {.async.} = discard await s.internalConnect(peerId, addrs) -proc negotiateStream(s: Switch, stream: Connection, proto: string): Future[Connection] {.async.} = - trace "Attempting to select remote", proto = proto, stream +proc negotiateStream(s: Switch, conn: Connection, proto: string): Future[Connection] {.async.} = + trace "Negotiating stream", proto = proto, conn + if not await s.ms.select(conn, proto): + await conn.close() + raise newException(DialFailedError, "Unable to select sub-protocol " & proto) - if not await s.ms.select(stream, proto): - await stream.close() - raise newException(CatchableError, "Unable to select sub-protocol" & proto) - - return stream + return conn proc dial*(s: Switch, peerId: PeerID, proto: string): Future[Connection] {.async.} = let stream = await s.connmanager.getMuxedStream(peerId) if stream.isNil: - raise newException(CatchableError, "Couldn't get muxed stream") + raise newException(DialFailedError, "Couldn't get muxed stream") return await s.negotiateStream(stream, proto) @@ -413,7 +415,7 @@ proc dial*(s: Switch, try: if isNil(stream): await conn.close() - raise newException(CatchableError, "Couldn't get muxed stream") + raise newException(DialFailedError, "Couldn't get muxed stream") return await s.negotiateStream(stream, proto) except CancelledError as exc: @@ -421,7 +423,7 @@ proc dial*(s: Switch, await cleanup() raise exc except CatchableError as exc: - trace "error dialing", exc = exc.msg, conn + trace "Error dialing", exc = exc.msg, conn await cleanup() raise exc @@ -440,14 +442,16 @@ proc start*(s: Switch): Future[seq[Future[void]]] {.async, gcsafe.} = trace "starting switch for peer", peerInfo = s.peerInfo proc handle(conn: Connection): Future[void] {.async, closure, gcsafe.} = + trace "Incoming connection", conn try: await s.upgradeIncoming(conn) # perform upgrade on incoming connection except CancelledError as exc: raise exc except CatchableError as exc: - trace "Error in connection handler", exc = exc.msg, conn + trace "Exception occurred in incoming handler", exc = exc.msg, conn finally: await conn.close() + trace "Connection handler done", conn var startFuts: seq[Future[void]] for t in s.transports: # for each transport @@ -457,7 +461,7 @@ proc start*(s: Switch): Future[seq[Future[void]]] {.async, gcsafe.} = s.peerInfo.addrs[i] = t.ma # update peer's address startFuts.add(server) - debug "started libp2p node", peerInfo = s.peerInfo + debug "Started libp2p node", peer = s.peerInfo result = startFuts # listen for incoming connections proc stop*(s: Switch) {.async.} = @@ -477,13 +481,16 @@ proc stop*(s: Switch) {.async.} = trace "switch stopped" proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = - if muxer.connection.peerInfo.isNil: + let + conn = muxer.connection + + if conn.peerInfo.isNil: warn "This version of nim-libp2p requires secure protocol to negotiate peerid" await muxer.close() return # store incoming connection - s.connManager.storeIncoming(muxer.connection) + s.connManager.storeIncoming(conn) # store muxer and muxed connection s.connManager.storeMuxer(muxer) @@ -494,10 +501,10 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = # Identify is non-essential, though if it fails, it might indicate that # the connection was closed already - this will be picked up by the read # loop - debug "Could not identify connection", err = exc.msg, muxer + debug "Could not identify connection", err = exc.msg, conn try: - let peerId = muxer.connection.peerInfo.peerId + let peerId = conn.peerInfo.peerId proc peerCleanup() {.async.} = try: @@ -507,10 +514,10 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = except CancelledError: # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. - debug "Unexpected cancellation in switch muxer cleanup" + debug "Unexpected cancellation in switch muxer cleanup", conn except CatchableError as exc: debug "Unexpected exception in switch muxer cleanup", - errMsg = exc.msg, muxer + err = exc.msg, conn proc peerStartup() {.async.} = try: @@ -520,10 +527,10 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = except CancelledError: # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. - debug "Unexpected cancellation in switch muxer startup", muxer + debug "Unexpected cancellation in switch muxer startup", conn except CatchableError as exc: debug "Unexpected exception in switch muxer startup", - errMsg = exc.msg, muxer + err = exc.msg, conn # All the errors are handled inside `peerStartup()` procedure. asyncSpawn peerStartup() @@ -537,7 +544,7 @@ proc muxerHandler(s: Switch, muxer: Muxer) {.async, gcsafe.} = except CatchableError as exc: await muxer.close() libp2p_failed_upgrade.inc() - trace "exception in muxer handler", exc = exc.msg, muxer + trace "Exception in muxer handler", exc = exc.msg, conn proc newSwitch*(peerInfo: PeerInfo, transports: seq[Transport], @@ -558,17 +565,17 @@ proc newSwitch*(peerInfo: PeerInfo, ) let s = result # can't capture result - result.streamHandler = proc(stream: Connection) {.async, gcsafe.} = + result.streamHandler = proc(conn: Connection) {.async, gcsafe.} = # noraises + trace "Incoming muxed connection", conn try: - trace "handling connection for", stream - defer: - if not(isNil(stream)): - await stream.close() - await s.ms.handle(stream) # handle incoming connection + await s.ms.handle(conn) # handle incoming connection except CancelledError as exc: raise exc except CatchableError as exc: - trace "exception in stream handler", exc = exc.msg, stream + trace "exception in stream handler", exc = exc.msg, conn + finally: + await conn.close() + trace "Muxed connection done", conn result.mount(identity) for key, val in muxers: From 82c179db9ea919304dfa0d22063e11d7aecba8a3 Mon Sep 17 00:00:00 2001 From: Jacek Sieka Date: Tue, 8 Sep 2020 08:24:28 +0200 Subject: [PATCH 10/10] mplex fixes (#356) * close the right connection when channel send fails * don't crash on channel id that is not unique --- libp2p/connmanager.nim | 25 ++++++++++++++++--------- libp2p/muxers/mplex/lpchannel.nim | 6 +++--- libp2p/muxers/mplex/mplex.nim | 24 +++++++++++++----------- 3 files changed, 32 insertions(+), 23 deletions(-) diff --git a/libp2p/connmanager.nim b/libp2p/connmanager.nim index 9be88f400..b17bbf46a 100644 --- a/libp2p/connmanager.nim +++ b/libp2p/connmanager.nim @@ -76,11 +76,12 @@ proc contains*(c: ConnManager, muxer: Muxer): bool = return muxer == c.muxed[conn].muxer proc closeMuxerHolder(muxerHolder: MuxerHolder) {.async.} = - trace "cleaning up muxer for peer" + trace "Cleaning up muxer", m = muxerHolder.muxer await muxerHolder.muxer.close() if not(isNil(muxerHolder.handle)): await muxerHolder.handle # TODO noraises? + trace "Cleaned up muxer", m = muxerHolder.muxer proc delConn(c: ConnManager, conn: Connection) = let peerId = conn.peerInfo.peerId @@ -91,6 +92,8 @@ proc delConn(c: ConnManager, conn: Connection) = c.conns.del(peerId) libp2p_peers.set(c.conns.len.int64) + trace "Removed connection", conn + proc cleanupConn(c: ConnManager, conn: Connection) {.async.} = ## clean connection's resources such as muxers and streams @@ -113,7 +116,7 @@ proc cleanupConn(c: ConnManager, conn: Connection) {.async.} = finally: await conn.close() - trace "connection cleaned up", conn + trace "Connection cleaned up", conn proc onClose(c: ConnManager, conn: Connection) {.async.} = ## connection close even handler @@ -122,15 +125,15 @@ proc onClose(c: ConnManager, conn: Connection) {.async.} = ## try: await conn.join() - trace "triggering connection cleanup", conn + trace "Connection closed, cleaning up", conn await c.cleanupConn(conn) except CancelledError: # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. - trace "Unexpected cancellation in connection manager's cleanup" + debug "Unexpected cancellation in connection manager's cleanup", conn except CatchableError as exc: - trace "Unexpected exception in connection manager's cleanup", - errMsg = exc.msg + debug "Unexpected exception in connection manager's cleanup", + errMsg = exc.msg, conn proc selectConn*(c: ConnManager, peerId: PeerID, @@ -181,7 +184,7 @@ proc storeConn*(c: ConnManager, conn: Connection) = let peerId = conn.peerInfo.peerId if c.conns.getOrDefault(peerId).len > c.maxConns: - trace "too many connections", peer = $peerId, + debug "too many connections", peer = conn, conns = c.conns.getOrDefault(peerId).len raise newTooManyConnections() @@ -196,7 +199,8 @@ proc storeConn*(c: ConnManager, conn: Connection) = asyncSpawn c.onClose(conn) libp2p_peers.set(c.conns.len.int64) - trace "stored connection", connections = c.conns.len, conn + trace "Stored connection", + connections = c.conns.len, conn, direction = $conn.dir proc storeOutgoing*(c: ConnManager, conn: Connection) = conn.dir = Direction.Out @@ -222,7 +226,7 @@ proc storeMuxer*(c: ConnManager, muxer: muxer, handle: handle) - trace "stored muxer", connections = c.conns.len, muxer + trace "Stored muxer", connections = c.conns.len, muxer proc getMuxedStream*(c: ConnManager, peerId: PeerID, @@ -256,8 +260,10 @@ proc getMuxedStream*(c: ConnManager, proc dropPeer*(c: ConnManager, peerId: PeerID) {.async.} = ## drop connections and cleanup resources for peer ## + trace "Dropping peer", peerId let conns = c.conns.getOrDefault(peerId) for conn in conns: + trace "Removing connection", conn delConn(c, conn) var muxers: seq[MuxerHolder] @@ -271,6 +277,7 @@ proc dropPeer*(c: ConnManager, peerId: PeerID) {.async.} = for conn in conns: await conn.close() + trace "Dropped peer", peerId proc close*(c: ConnManager) {.async.} = ## cleanup resources for the connection diff --git a/libp2p/muxers/mplex/lpchannel.nim b/libp2p/muxers/mplex/lpchannel.nim index 4e8a2b534..54c2a8fe1 100644 --- a/libp2p/muxers/mplex/lpchannel.nim +++ b/libp2p/muxers/mplex/lpchannel.nim @@ -90,7 +90,7 @@ proc resetMessage(s: LPChannel) {.async.} = except CancelledError: # This procedure is called from one place and never awaited, so there no # need to re-raise CancelledError. - trace "Unexpected cancellation while resetting channel" + debug "Unexpected cancellation while resetting channel", s except LPStreamEOFError as exc: trace "muxed connection EOF", exc = exc.msg, s except LPStreamClosedError as exc: @@ -217,14 +217,14 @@ proc init*( await chann.open() # writes should happen in sequence - trace "sending data", len = data.len, chann + trace "sending data", len = data.len, conn, chann await conn.writeMsg(chann.id, chann.msgCode, data) except CatchableError as exc: trace "exception in lpchannel write handler", exc = exc.msg, chann - await chann.reset() + asyncSpawn conn.close() raise exc chann.initBufferStream(writeHandler, size) diff --git a/libp2p/muxers/mplex/mplex.nim b/libp2p/muxers/mplex/mplex.nim index 37244258b..71d24c0be 100644 --- a/libp2p/muxers/mplex/mplex.nim +++ b/libp2p/muxers/mplex/mplex.nim @@ -32,6 +32,7 @@ when defined(libp2p_expensive_metrics): type TooManyChannels* = object of CatchableError + InvalidChannelIdError* = object of CatchableError Mplex* = ref object of Muxer channels: array[bool, Table[uint64, LPChannel]] @@ -48,13 +49,16 @@ chronicles.formatIt(Mplex): shortLog(it) proc newTooManyChannels(): ref TooManyChannels = newException(TooManyChannels, "max allowed channel count exceeded") +proc newInvalidChannelIdError(): ref InvalidChannelIdError = + newException(InvalidChannelIdError, "max allowed channel count exceeded") + proc cleanupChann(m: Mplex, chann: LPChannel) {.async, inline.} = ## remove the local channel from the internal tables ## try: await chann.join() m.channels[chann.initiator].del(chann.id) - trace "cleaned up channel", m, chann + debug "cleaned up channel", m, chann when defined(libp2p_expensive_metrics): libp2p_mplex_channels.set( @@ -63,10 +67,10 @@ proc cleanupChann(m: Mplex, chann: LPChannel) {.async, inline.} = except CancelledError: # This is top-level procedure which will work as separate task, so it # do not need to propogate CancelledError. - trace "Unexpected cancellation in mplex channel cleanup", + debug "Unexpected cancellation in mplex channel cleanup", m, chann except CatchableError as exc: - trace "error cleaning up mplex channel", exc = exc.msg, m, chann + debug "error cleaning up mplex channel", exc = exc.msg, m, chann proc newStreamInternal*(m: Mplex, initiator: bool = true, @@ -81,10 +85,9 @@ proc newStreamInternal*(m: Mplex, m.currentId.inc(); m.currentId else: chanId - trace "creating new channel", id, - initiator, - name, - m + if id in m.channels[initiator]: + raise newInvalidChannelIdError() + result = LPChannel.init( id, m.connection, @@ -96,8 +99,7 @@ proc newStreamInternal*(m: Mplex, result.peerInfo = m.connection.peerInfo result.observedAddr = m.connection.observedAddr - doAssert(id notin m.channels[initiator], - "channel slot already taken!") + trace "Creating new channel", id, initiator, name, m, channel = result m.channels[initiator][id] = result @@ -127,7 +129,7 @@ proc handleStream(m: Mplex, chann: LPChannel) {.async.} = await chann.reset() method handle*(m: Mplex) {.async, gcsafe.} = - trace "starting mplex main loop", m, peer = m.connection.peerInfo.peerId + trace "Starting mplex main loop", m try: while not m.connection.atEof: trace "waiting for data", m @@ -186,7 +188,7 @@ method handle*(m: Mplex) {.async, gcsafe.} = except CancelledError: # This procedure is spawned as task and it is not part of public API, so # there no way for this procedure to be cancelled implicitely. - trace "Unexpected cancellation in mplex handler" + debug "Unexpected cancellation in mplex handler", m except CatchableError as exc: trace "Exception occurred", exception = exc.msg, m finally: