Stream rearchitecture (#126)

* Add generic protocol handler

* Add generic protocol handler to stream muxing pipeline

* Modify conn_handler to only deal with connections

* mplex accept stream architecture changes

* Add create generic protocol handler

* Fix minor bugs

* who would win 4 devs or one not

* Debugging

* rearch with handle_incoming infinite loop, seems to work, needs cleanup"

* passing linting, still needs cleanup

* fixing linting again; code still needs cleanup

* fixing tests; code still needs cleanup

* adding test cleanup and task cleanup, removing prints

* linting, and cleanup complete

* storing connections based on peer id

* remove dead code

* remove unnecessary peer_id
This commit is contained in:
Robert Zajac 2019-02-24 20:58:23 -05:00 committed by GitHub
parent 17c778de15
commit 82840b5e6c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 367 additions and 120 deletions

View File

@ -1,7 +1,7 @@
from Crypto.PublicKey import RSA
import asyncio
import multiaddr
from Crypto.PublicKey import RSA
from .peer.peerstore import PeerStore
from .peer.id import id_from_public_key
from .network.swarm import Swarm
@ -10,6 +10,16 @@ from .transport.upgrader import TransportUpgrader
from .transport.tcp.tcp import TCP
async def cleanup_done_tasks():
while True:
for task in asyncio.all_tasks():
if task.done():
await task
# Need not run often
# Some sleep necessary to context switch
await asyncio.sleep(3)
async def new_node(
id_opt=None, transport_opt=None,
muxer_opt=None, sec_opt=None, peerstore=None):
@ -35,4 +45,7 @@ async def new_node(
# TODO routing unimplemented
host = BasicHost(swarm)
# Kick off cleanup job
asyncio.ensure_future(cleanup_done_tasks())
return host

View File

@ -1,12 +1,13 @@
import asyncio
from libp2p.protocol_muxer.multiselect_client import MultiselectClient
from libp2p.protocol_muxer.multiselect import Multiselect
from libp2p.peer.id import id_b58_decode
from .network_interface import INetwork
from .stream.net_stream import NetStream
from .connection.raw_connection import RawConnection
class Swarm(INetwork):
# pylint: disable=too-many-instance-attributes, cell-var-from-loop
@ -23,6 +24,9 @@ class Swarm(INetwork):
self.multiselect = Multiselect()
self.multiselect_client = MultiselectClient()
# Create generic protocol handler
self.generic_protocol_handler = create_generic_protocol_handler(self)
def get_peer_id(self):
return self.self_id
@ -58,10 +62,10 @@ class Swarm(INetwork):
muxed_conn = self.connections[peer_id]
else:
# Transport dials peer (gets back a raw conn)
raw_conn = await self.transport.dial(multiaddr)
raw_conn = await self.transport.dial(multiaddr, self.self_id)
# Use upgrader to upgrade raw conn to muxed conn
muxed_conn = self.upgrader.upgrade_connection(raw_conn)
muxed_conn = self.upgrader.upgrade_connection(raw_conn, self.generic_protocol_handler)
# Store muxed connection in connections
self.connections[peer_id] = muxed_conn
@ -87,7 +91,7 @@ class Swarm(INetwork):
# Use muxed conn to open stream, which returns
# a muxed stream
# TODO: Remove protocol id from being passed into muxed_conn
muxed_stream = await muxed_conn.open_stream(protocol_ids[0], peer_id, multiaddr)
muxed_stream = await muxed_conn.open_stream(protocol_ids[0], multiaddr)
# Perform protocol muxing to determine protocol to use
selected_protocol = await self.multiselect_client.select_one_of(protocol_ids, muxed_stream)
@ -117,26 +121,21 @@ class Swarm(INetwork):
return True
async def conn_handler(reader, writer):
# Read in first message (should be peer_id of initiator) and ack
peer_id = id_b58_decode((await reader.read(1024)).decode())
writer.write("received peer id".encode())
await writer.drain()
# Upgrade reader/write to a net_stream and pass \
# to appropriate stream handler (using multiaddr)
raw_conn = RawConnection(multiaddr.value_for_protocol('ip4'),
multiaddr.value_for_protocol('tcp'), reader, writer, False)
muxed_conn = self.upgrader.upgrade_connection(raw_conn)
muxed_conn = self.upgrader.upgrade_connection(raw_conn, \
self.generic_protocol_handler)
# TODO: Remove protocol id from muxed_conn accept stream or
# move protocol muxing into accept_stream
muxed_stream, _, _ = await muxed_conn.accept_stream()
# Perform protocol muxing to determine protocol to use
selected_protocol, handler = await self.multiselect.negotiate(muxed_stream)
net_stream = NetStream(muxed_stream)
net_stream.set_protocol(selected_protocol)
# Give to stream handler
# TODO: handle case when stream handler is set
# TODO: handle case of multiple protocols over same raw connection
await handler(net_stream)
# Store muxed_conn with peer id
self.connections[peer_id] = muxed_conn
try:
# Success
@ -155,6 +154,23 @@ class Swarm(INetwork):
# TODO: Support more than one transport
self.transport = transport
def create_generic_protocol_handler(swarm):
"""
Create a generic protocol handler from the given swarm. We use swarm
to extract the multiselect module so that generic_protocol_handler
can use multiselect when generic_protocol_handler is called
from a different class
"""
multiselect = swarm.multiselect
async def generic_protocol_handler(muxed_stream):
# Perform protocol muxing to determine protocol to use
_, handler = await multiselect.negotiate(muxed_stream)
# Give to stream handler
asyncio.ensure_future(handler(muxed_stream))
return generic_protocol_handler
class SwarmException(Exception):
pass

View File

@ -31,7 +31,6 @@ class Multiselect(IMultiselectMuxer):
:return: selected protocol name, handler function
:raise Exception: negotiation failed exception
"""
# Create a communicator to handle all communication across the stream
communicator = MultiselectCommunicator(stream)

View File

@ -1,6 +1,6 @@
import asyncio
from .utils import encode_uvarint, decode_uvarint_from_stream
from .utils import encode_uvarint, decode_uvarint_from_stream, get_flag
from .mplex_stream import MplexStream
from ..muxed_connection_interface import IMuxedConn
@ -11,25 +11,28 @@ class Mplex(IMuxedConn):
reference: https://github.com/libp2p/go-mplex/blob/master/multiplex.go
"""
def __init__(self, conn):
def __init__(self, conn, generic_protocol_handler):
"""
create a new muxed connection
:param conn: an instance of raw connection
:param initiator: boolean to prevent multiplex with self
:param generic_protocol_handler: generic protocol handler
for new muxed streams
"""
super(Mplex, self).__init__(conn, generic_protocol_handler)
self.raw_conn = conn
self.initiator = conn.initiator
# Store generic protocol handler
self.generic_protocol_handler = generic_protocol_handler
# Mapping from stream ID -> buffer of messages for that stream
self.buffers = {}
self.stream_queue = asyncio.Queue()
self.data_buffer = bytearray()
# The initiator of the raw connection need not read upon construction time.
# It should read when the user decides that it wants to read from the constructed stream.
if not self.initiator:
asyncio.ensure_future(self.handle_incoming(None))
# Kick off reading
asyncio.ensure_future(self.handle_incoming())
def close(self):
"""
@ -49,39 +52,29 @@ class Mplex(IMuxedConn):
:param stream_id: stream id of stream to read from
:return: message read
"""
# Empty buffer or nonexistent stream
# TODO: propagate up timeout exception and catch
if stream_id not in self.buffers or self.buffers[stream_id].empty():
await self.handle_incoming(stream_id)
# TODO: pass down timeout from user and use that
if stream_id in self.buffers:
return await self._read_buffer_exists(stream_id)
return None
async def _read_buffer_exists(self, stream_id):
"""
Reads from raw connection with the assumption that the message buffer for stream_id exsits
:param stream_id: stream id of stream to read from
:return: message read
"""
try:
data = await asyncio.wait_for(self.buffers[stream_id].get(), timeout=5)
data = await asyncio.wait_for(self.buffers[stream_id].get(), timeout=3)
return data
except asyncio.TimeoutError:
return None
async def open_stream(self, protocol_id, peer_id, multi_addr):
# Stream not created yet
return None
async def open_stream(self, protocol_id, multi_addr):
"""
creates a new muxed_stream
:param protocol_id: protocol_id of stream
:param stream_id: stream_id of stream
:param peer_id: peer_id that stream connects to
:param multi_addr: multi_addr that stream connects to
:return: a new stream
"""
stream_id = self.raw_conn.next_stream_id()
stream = MplexStream(stream_id, multi_addr, self)
self.buffers[stream_id] = asyncio.Queue()
await self.send_message(get_flag(self.initiator, "NEW_STREAM"), None, stream_id)
return stream
async def accept_stream(self):
@ -89,11 +82,9 @@ class Mplex(IMuxedConn):
accepts a muxed stream opened by the other end
:return: the accepted stream
"""
# TODO update to pull out protocol_id from message
protocol_id = "/echo/1.0.0"
stream_id = await self.stream_queue.get()
stream = MplexStream(stream_id, False, self)
return stream, stream_id, protocol_id
asyncio.ensure_future(self.generic_protocol_handler(stream))
async def send_message(self, flag, data, stream_id):
"""
@ -126,38 +117,29 @@ class Mplex(IMuxedConn):
await self.raw_conn.writer.drain()
return len(_bytes)
async def handle_incoming(self, my_stream_id):
async def handle_incoming(self):
"""
Read a message off of the raw connection and add it to the corresponding message buffer
"""
# TODO Deal with other types of messages using flag (currently _)
continue_reading = True
i = 0
while continue_reading:
i += 1
stream_id, _, message = await self.read_message()
continue_reading = (stream_id is not None and
stream_id != my_stream_id and
my_stream_id is not None)
while True:
stream_id, flag, message = await self.read_message()
if stream_id is not None and flag is not None and message is not None:
if stream_id not in self.buffers:
self.buffers[stream_id] = asyncio.Queue()
await self.stream_queue.put(stream_id)
if flag is get_flag(True, "NEW_STREAM"):
# new stream detected on connection
await self.accept_stream()
if message:
await self.buffers[stream_id].put(message)
async def read_chunk(self):
"""
Read a chunk of bytes off of the raw connection into data_buffer
"""
# unused now but possibly useful in the future
try:
chunk = await asyncio.wait_for(self.raw_conn.reader.read(-1), timeout=5)
self.data_buffer += chunk
except asyncio.TimeoutError:
print('timeout!')
return
# Force context switch
await asyncio.sleep(0)
async def read_message(self):
"""
@ -167,7 +149,7 @@ class Mplex(IMuxedConn):
# Timeout is set to a relatively small value to alleviate wait time to exit
# loop in handle_incoming
timeout = .1
timeout = 0.1
try:
header = await decode_uvarint_from_stream(self.raw_conn.reader, timeout)
length = await decode_uvarint_from_stream(self.raw_conn.reader, timeout)

View File

@ -1,6 +1,6 @@
import asyncio
from .constants import HEADER_TAGS
from .utils import get_flag
from ..muxed_stream_interface import IMuxedStream
@ -26,17 +26,6 @@ class MplexStream(IMuxedStream):
self.remote_closed = False
self.stream_lock = asyncio.Lock()
def get_flag(self, action):
"""
get header flag based on action for mplex
:param action: action type in str
:return: int flag
"""
if self.initiator:
return HEADER_TAGS[action]
return HEADER_TAGS[action] - 1
async def read(self):
"""
read messages associated with stream from buffer til end of file
@ -49,7 +38,8 @@ class MplexStream(IMuxedStream):
write to stream
:return: number of bytes written
"""
return await self.mplex_conn.send_message(self.get_flag("MESSAGE"), data, self.stream_id)
return await self.mplex_conn.send_message(
get_flag(self.initiator, "MESSAGE"), data, self.stream_id)
async def close(self):
"""
@ -59,7 +49,7 @@ class MplexStream(IMuxedStream):
"""
# TODO error handling with timeout
# TODO understand better how mutexes are used from go repo
await self.mplex_conn.send_message(self.get_flag("CLOSE"), None, self.stream_id)
await self.mplex_conn.send_message(get_flag(self.initiator, "CLOSE"), None, self.stream_id)
remote_lock = ""
async with self.stream_lock:
@ -87,7 +77,8 @@ class MplexStream(IMuxedStream):
return True
if not self.remote_closed:
await self.mplex_conn.send_message(self.get_flag("RESET"), None, self.stream_id)
await self.mplex_conn.send_message(
get_flag(self.initiator, "RESET"), None, self.stream_id)
self.local_closed = True
self.remote_closed = True

View File

@ -1,5 +1,6 @@
import asyncio
import struct
from .constants import HEADER_TAGS
def encode_uvarint(number):
@ -41,3 +42,14 @@ async def decode_uvarint_from_stream(reader, timeout):
break
return result
def get_flag(initiator, action):
"""
get header flag based on action for mplex
:param action: action type in str
:return: int flag
"""
if initiator or HEADER_TAGS[action] == 0:
return HEADER_TAGS[action]
return HEADER_TAGS[action] - 1

View File

@ -6,6 +6,15 @@ class IMuxedConn(ABC):
reference: https://github.com/libp2p/go-stream-muxer/blob/master/muxer.go
"""
@abstractmethod
def __init__(self, conn, generic_protocol_handler):
"""
create a new muxed connection
:param conn: an instance of raw connection
:param generic_protocol_handler: generic protocol handler
for new muxed streams
"""
@abstractmethod
def close(self):
"""
@ -21,12 +30,10 @@ class IMuxedConn(ABC):
"""
@abstractmethod
def open_stream(self, protocol_id, peer_id, multi_addr):
def open_stream(self, protocol_id, multi_addr):
"""
creates a new muxed_stream
:param protocol_id: protocol_id of stream
:param stream_id: stream_id of stream
:param peer_id: peer_id that stream connects to
:param multi_addr: multi_addr that stream connects to
:return: a new stream
"""

View File

@ -3,6 +3,7 @@ import asyncio
import multiaddr
from libp2p.network.connection.raw_connection import RawConnection
from libp2p.peer.id import id_b58_encode
from ..listener_interface import IListener
from ..transport_interface import ITransport
@ -63,10 +64,11 @@ class TCP(ITransport):
self.server = None
return True
async def dial(self, multiaddr, options=None):
async def dial(self, multiaddr, self_id, options=None):
"""
dial a transport to peer listening on multiaddr
:param multiaddr: multiaddr of peer
:param self_id: peer_id of the dialer (to send to receier)
:param options: optional object
:return: True if successful
"""
@ -75,6 +77,16 @@ class TCP(ITransport):
reader, writer = await asyncio.open_connection(host, port)
# First: send our peer ID so receiver knows it
writer.write(id_b58_encode(self_id).encode())
await writer.drain()
# Await ack for peer id
ack = (await reader.read(1024)).decode()
if ack != "received peer id":
raise Exception("Receiver did not receive peer id")
return RawConnection(host, port, reader, writer, True)
def create_listener(self, handler_function, options=None):

View File

@ -4,10 +4,11 @@ from abc import ABC, abstractmethod
class ITransport(ABC):
@abstractmethod
def dial(self, multiaddr, options=None):
def dial(self, multiaddr, self_id, options=None):
"""
dial a transport to peer listening on multiaddr
:param multiaddr: multiaddr of peer
:param self_id: peer_id of the dialer (to send to receier)
:param options: optional object
:return: list of multiaddrs
"""

View File

@ -17,11 +17,11 @@ class TransportUpgrader:
def upgrade_security(self):
pass
def upgrade_connection(self, conn):
def upgrade_connection(self, conn, generic_protocol_handler):
"""
upgrade raw connection to muxed connection
"""
# For PoC, no security, default to mplex
# TODO do exchange to determine multiplexer
return Mplex(conn)
return Mplex(conn, generic_protocol_handler)

View File

@ -1,5 +1,7 @@
import pytest
import asyncio
from tests.utils import cleanup
from libp2p import new_node
from libp2p.peer.peerinfo import info_from_p2p_addr
from libp2p.protocol_muxer.multiselect_client import MultiselectClientError
@ -27,25 +29,28 @@ async def hello_world(host_a, host_b):
async def connect_write(host_a, host_b):
messages = [b'data %d' % i for i in range(5)]
messages = ['data %d' % i for i in range(5)]
received = []
async def stream_handler(stream):
received = []
while True:
try:
received.append((await stream.read()).decode())
except Exception: # exception is raised when other side close the stream ?
break
await stream.close()
assert received == messages
host_a.set_stream_handler(PROTOCOL_ID, stream_handler)
# Start a stream with the destination.
# Multiaddress of the destination peer is fetched from the peerstore using 'peerId'.
stream = await host_b.new_stream(host_a.get_id(), [PROTOCOL_ID])
for message in messages:
await stream.write(message)
await stream.write(message.encode())
# Reader needs time due to async reads
await asyncio.sleep(2)
await stream.close()
assert received == messages
async def connect_read(host_a, host_b):
@ -103,3 +108,5 @@ async def test_chat(test):
await host_b.connect(info)
await test(host_a, host_b)
await cleanup()

View File

@ -1,9 +1,12 @@
import multiaddr
import pytest
from tests.utils import cleanup
from libp2p import new_node
from libp2p.peer.peerinfo import info_from_p2p_addr
# pylint: disable=too-many-locals
@pytest.mark.asyncio
async def test_simple_messages():
@ -13,10 +16,8 @@ async def test_simple_messages():
async def stream_handler(stream):
while True:
read_string = (await stream.read()).decode()
print("host B received:" + read_string)
response = "ack:" + read_string
print("sending response:" + response)
await stream.write(response.encode())
node_b.set_stream_handler("/echo/1.0.0", stream_handler)
@ -32,11 +33,10 @@ async def test_simple_messages():
response = (await stream.read()).decode()
print("res: " + response)
assert response == ("ack:" + message)
# Success, terminate pending tasks.
return
await cleanup()
@pytest.mark.asyncio
@ -47,38 +47,31 @@ async def test_double_response():
async def stream_handler(stream):
while True:
read_string = (await stream.read()).decode()
print("host B received:" + read_string)
response = "ack1:" + read_string
print("sending response:" + response)
await stream.write(response.encode())
response = "ack2:" + read_string
print("sending response:" + response)
await stream.write(response.encode())
node_b.set_stream_handler("/echo/1.0.0", stream_handler)
# Associate the peer with local ip address (see default parameters of Libp2p())
node_a.get_peerstore().add_addrs(node_b.get_id(), node_b.get_addrs(), 10)
print("node_a about to open stream")
stream = await node_a.new_stream(node_b.get_id(), ["/echo/1.0.0"])
messages = ["hello" + str(x) for x in range(10)]
for message in messages:
await stream.write(message.encode())
response1 = (await stream.read()).decode()
print("res1: " + response1)
assert response1 == ("ack1:" + message)
response2 = (await stream.read()).decode()
print("res2: " + response2)
assert response2 == ("ack2:" + message)
# Success, terminate pending tasks.
return
await cleanup()
@pytest.mark.asyncio
@ -127,7 +120,197 @@ async def test_multiple_streams():
assert response_a == ("ack_b:" + a_message) and response_b == ("ack_a:" + b_message)
# Success, terminate pending tasks.
return
await cleanup()
@pytest.mark.asyncio
async def test_multiple_streams_same_initiator_different_protocols():
node_a = await new_node(transport_opt=["/ip4/127.0.0.1/tcp/0"])
node_b = await new_node(transport_opt=["/ip4/127.0.0.1/tcp/0"])
async def stream_handler_a1(stream):
while True:
read_string = (await stream.read()).decode()
response = "ack_a1:" + read_string
await stream.write(response.encode())
async def stream_handler_a2(stream):
while True:
read_string = (await stream.read()).decode()
response = "ack_a2:" + read_string
await stream.write(response.encode())
async def stream_handler_a3(stream):
while True:
read_string = (await stream.read()).decode()
response = "ack_a3:" + read_string
await stream.write(response.encode())
node_b.set_stream_handler("/echo_a1/1.0.0", stream_handler_a1)
node_b.set_stream_handler("/echo_a2/1.0.0", stream_handler_a2)
node_b.set_stream_handler("/echo_a3/1.0.0", stream_handler_a3)
# Associate the peer with local ip address (see default parameters of Libp2p())
node_a.get_peerstore().add_addrs(node_b.get_id(), node_b.get_addrs(), 10)
node_b.get_peerstore().add_addrs(node_a.get_id(), node_a.get_addrs(), 10)
# Open streams to node_b over echo_a1 echo_a2 echo_a3 protocols
stream_a1 = await node_a.new_stream(node_b.get_id(), ["/echo_a1/1.0.0"])
stream_a2 = await node_a.new_stream(node_b.get_id(), ["/echo_a2/1.0.0"])
stream_a3 = await node_a.new_stream(node_b.get_id(), ["/echo_a3/1.0.0"])
messages = ["hello" + str(x) for x in range(10)]
for message in messages:
a1_message = message + "_a1"
a2_message = message + "_a2"
a3_message = message + "_a3"
await stream_a1.write(a1_message.encode())
await stream_a2.write(a2_message.encode())
await stream_a3.write(a3_message.encode())
response_a1 = (await stream_a1.read()).decode()
response_a2 = (await stream_a2.read()).decode()
response_a3 = (await stream_a3.read()).decode()
assert (response_a1 == ("ack_a1:" + a1_message)
and response_a2 == ("ack_a2:" + a2_message)
and response_a3 == ("ack_a3:" + a3_message))
# Success, terminate pending tasks.
await cleanup()
@pytest.mark.asyncio
async def test_multiple_streams_two_initiators():
node_a = await new_node(transport_opt=["/ip4/127.0.0.1/tcp/0"])
node_b = await new_node(transport_opt=["/ip4/127.0.0.1/tcp/0"])
async def stream_handler_a1(stream):
while True:
read_string = (await stream.read()).decode()
response = "ack_a1:" + read_string
await stream.write(response.encode())
async def stream_handler_a2(stream):
while True:
read_string = (await stream.read()).decode()
response = "ack_a2:" + read_string
await stream.write(response.encode())
async def stream_handler_b1(stream):
while True:
read_string = (await stream.read()).decode()
response = "ack_b1:" + read_string
await stream.write(response.encode())
async def stream_handler_b2(stream):
while True:
read_string = (await stream.read()).decode()
response = "ack_b2:" + read_string
await stream.write(response.encode())
node_a.set_stream_handler("/echo_b1/1.0.0", stream_handler_b1)
node_a.set_stream_handler("/echo_b2/1.0.0", stream_handler_b2)
node_b.set_stream_handler("/echo_a1/1.0.0", stream_handler_a1)
node_b.set_stream_handler("/echo_a2/1.0.0", stream_handler_a2)
# Associate the peer with local ip address (see default parameters of Libp2p())
node_a.get_peerstore().add_addrs(node_b.get_id(), node_b.get_addrs(), 10)
node_b.get_peerstore().add_addrs(node_a.get_id(), node_a.get_addrs(), 10)
stream_a1 = await node_a.new_stream(node_b.get_id(), ["/echo_a1/1.0.0"])
stream_a2 = await node_a.new_stream(node_b.get_id(), ["/echo_a2/1.0.0"])
stream_b1 = await node_b.new_stream(node_a.get_id(), ["/echo_b1/1.0.0"])
stream_b2 = await node_b.new_stream(node_a.get_id(), ["/echo_b2/1.0.0"])
# A writes to /echo_b via stream_a, and B writes to /echo_a via stream_b
messages = ["hello" + str(x) for x in range(10)]
for message in messages:
a1_message = message + "_a1"
a2_message = message + "_a2"
b1_message = message + "_b1"
b2_message = message + "_b2"
await stream_a1.write(a1_message.encode())
await stream_a2.write(a2_message.encode())
await stream_b1.write(b1_message.encode())
await stream_b2.write(b2_message.encode())
response_a1 = (await stream_a1.read()).decode()
response_a2 = (await stream_a2.read()).decode()
response_b1 = (await stream_b1.read()).decode()
response_b2 = (await stream_b2.read()).decode()
assert (response_a1 == ("ack_a1:" + a1_message)
and response_a2 == ("ack_a2:" + a2_message)
and response_b1 == ("ack_b1:" + b1_message)
and response_b2 == ("ack_b2:" + b2_message))
# Success, terminate pending tasks.
await cleanup()
@pytest.mark.asyncio
async def test_triangle_nodes_connection():
node_a = await new_node(transport_opt=["/ip4/127.0.0.1/tcp/0"])
node_b = await new_node(transport_opt=["/ip4/127.0.0.1/tcp/0"])
node_c = await new_node(transport_opt=["/ip4/127.0.0.1/tcp/0"])
async def stream_handler(stream):
while True:
read_string = (await stream.read()).decode()
response = "ack:" + read_string
await stream.write(response.encode())
node_a.set_stream_handler("/echo/1.0.0", stream_handler)
node_b.set_stream_handler("/echo/1.0.0", stream_handler)
node_c.set_stream_handler("/echo/1.0.0", stream_handler)
# Associate the peer with local ip address (see default parameters of Libp2p())
# Associate all permutations
node_a.get_peerstore().add_addrs(node_b.get_id(), node_b.get_addrs(), 10)
node_a.get_peerstore().add_addrs(node_c.get_id(), node_c.get_addrs(), 10)
node_b.get_peerstore().add_addrs(node_a.get_id(), node_a.get_addrs(), 10)
node_b.get_peerstore().add_addrs(node_c.get_id(), node_c.get_addrs(), 10)
node_c.get_peerstore().add_addrs(node_a.get_id(), node_a.get_addrs(), 10)
node_c.get_peerstore().add_addrs(node_b.get_id(), node_b.get_addrs(), 10)
stream_a_to_b = await node_a.new_stream(node_b.get_id(), ["/echo/1.0.0"])
stream_a_to_c = await node_a.new_stream(node_c.get_id(), ["/echo/1.0.0"])
stream_b_to_a = await node_b.new_stream(node_a.get_id(), ["/echo/1.0.0"])
stream_b_to_c = await node_b.new_stream(node_c.get_id(), ["/echo/1.0.0"])
stream_c_to_a = await node_c.new_stream(node_a.get_id(), ["/echo/1.0.0"])
stream_c_to_b = await node_c.new_stream(node_b.get_id(), ["/echo/1.0.0"])
messages = ["hello" + str(x) for x in range(5)]
streams = [stream_a_to_b, stream_a_to_c, stream_b_to_a, stream_b_to_c,
stream_c_to_a, stream_c_to_b]
for message in messages:
for stream in streams:
await stream.write(message.encode())
response = (await stream.read()).decode()
assert response == ("ack:" + message)
# Success, terminate pending tasks.
await cleanup()
@pytest.mark.asyncio
@ -152,3 +335,6 @@ async def test_host_connect():
ma_node_b = multiaddr.Multiaddr('/p2p/%s' % node_b.get_id().pretty())
for addr in node_a.get_peerstore().addrs(node_b.get_id()):
assert addr.encapsulate(ma_node_b) in node_b.get_addrs()
# Success, terminate pending tasks.
await cleanup()

View File

@ -1,5 +1,6 @@
import pytest
from tests.utils import cleanup
from libp2p import new_node
from libp2p.protocol_muxer.multiselect_client import MultiselectClientError
@ -49,7 +50,7 @@ async def perform_simple_test(expected_selected_protocol,
assert expected_selected_protocol == stream.get_protocol()
# Success, terminate pending tasks.
return
await cleanup()
@pytest.mark.asyncio
@ -65,6 +66,9 @@ async def test_single_protocol_fails():
await perform_simple_test("", ["/echo/1.0.0"],
["/potato/1.0.0"])
# Cleanup not reached on error
await cleanup()
@pytest.mark.asyncio
async def test_multiple_protocol_first_is_valid_succeeds():
@ -91,3 +95,6 @@ async def test_multiple_protocol_fails():
with pytest.raises(MultiselectClientError):
await perform_simple_test("", protocols_for_client,
protocols_for_listener)
# Cleanup not reached on error
await cleanup()

14
tests/utils.py Normal file
View File

@ -0,0 +1,14 @@
import asyncio
from contextlib import suppress
async def cleanup():
pending = asyncio.all_tasks()
for task in pending:
task.cancel()
# Now we should await task to execute it's cancellation.
# Cancelled task raises asyncio.CancelledError that we can suppress:
with suppress(asyncio.CancelledError):
await task