Merge upstream/main into add-ws-transport

Resolved conflicts in:
- .gitignore: Combined JavaScript interop and Sphinx build ignores
- libp2p/__init__.py: Integrated QUIC transport support with WebSocket transport
- libp2p/network/swarm.py: Used upstream's improved listener handling
- pyproject.toml: Kept both WebSocket and QUIC dependencies

This merge brings in:
- QUIC transport implementation
- Enhanced swarm functionality
- Improved peer discovery
- Better error handling
- Updated dependencies and documentation

WebSocket transport implementation remains intact and functional.
This commit is contained in:
acul71
2025-09-07 23:47:41 +02:00
105 changed files with 13904 additions and 730 deletions

70
libp2p/network/config.py Normal file
View File

@ -0,0 +1,70 @@
from dataclasses import dataclass
@dataclass
class RetryConfig:
"""
Configuration for retry logic with exponential backoff.
This configuration controls how connection attempts are retried when they fail.
The retry mechanism uses exponential backoff with jitter to prevent thundering
herd problems in distributed systems.
Attributes:
max_retries: Maximum number of retry attempts before giving up.
Default: 3 attempts
initial_delay: Initial delay in seconds before the first retry.
Default: 0.1 seconds (100ms)
max_delay: Maximum delay cap in seconds to prevent excessive wait times.
Default: 30.0 seconds
backoff_multiplier: Multiplier for exponential backoff (each retry multiplies
the delay by this factor). Default: 2.0 (doubles each time)
jitter_factor: Random jitter factor (0.0-1.0) to add randomness to delays
and prevent synchronized retries. Default: 0.1 (10% jitter)
"""
max_retries: int = 3
initial_delay: float = 0.1
max_delay: float = 30.0
backoff_multiplier: float = 2.0
jitter_factor: float = 0.1
@dataclass
class ConnectionConfig:
"""
Configuration for multi-connection support.
This configuration controls how multiple connections per peer are managed,
including connection limits, timeouts, and load balancing strategies.
Attributes:
max_connections_per_peer: Maximum number of connections allowed to a single
peer. Default: 3 connections
connection_timeout: Timeout in seconds for establishing new connections.
Default: 30.0 seconds
load_balancing_strategy: Strategy for distributing streams across connections.
Options: "round_robin" (default) or "least_loaded"
"""
max_connections_per_peer: int = 3
connection_timeout: float = 30.0
load_balancing_strategy: str = "round_robin" # or "least_loaded"
def __post_init__(self) -> None:
"""Validate configuration after initialization."""
if not (
self.load_balancing_strategy == "round_robin"
or self.load_balancing_strategy == "least_loaded"
):
raise ValueError(
"Load balancing strategy can only be 'round_robin' or 'least_loaded'"
)
if self.max_connections_per_peer < 1:
raise ValueError("Max connection per peer should be atleast 1")
if self.connection_timeout < 0:
raise ValueError("Connection timeout should be positive")

View File

@ -17,6 +17,7 @@ from libp2p.stream_muxer.exceptions import (
MuxedStreamError,
MuxedStreamReset,
)
from libp2p.transport.quic.exceptions import QUICStreamClosedError, QUICStreamResetError
from .exceptions import (
StreamClosed,
@ -170,7 +171,7 @@ class NetStream(INetStream):
elif self.__stream_state == StreamState.OPEN:
self.__stream_state = StreamState.CLOSE_READ
raise StreamEOF() from error
except MuxedStreamReset as error:
except (MuxedStreamReset, QUICStreamClosedError, QUICStreamResetError) as error:
async with self._state_lock:
if self.__stream_state in [
StreamState.OPEN,
@ -199,7 +200,12 @@ class NetStream(INetStream):
try:
await self.muxed_stream.write(data)
except (MuxedStreamClosed, MuxedStreamError) as error:
except (
MuxedStreamClosed,
MuxedStreamError,
QUICStreamClosedError,
QUICStreamResetError,
) as error:
async with self._state_lock:
if self.__stream_state == StreamState.OPEN:
self.__stream_state = StreamState.CLOSE_WRITE

View File

@ -3,6 +3,8 @@ from collections.abc import (
Callable,
)
import logging
import random
from typing import cast
from multiaddr import (
Multiaddr,
@ -25,6 +27,7 @@ from libp2p.custom_types import (
from libp2p.io.abc import (
ReadWriteCloser,
)
from libp2p.network.config import ConnectionConfig, RetryConfig
from libp2p.peer.id import (
ID,
)
@ -39,6 +42,9 @@ from libp2p.transport.exceptions import (
OpenConnectionError,
SecurityUpgradeFailure,
)
from libp2p.transport.quic.config import QUICTransportConfig
from libp2p.transport.quic.connection import QUICConnection
from libp2p.transport.quic.transport import QUICTransport
from libp2p.transport.upgrader import (
TransportUpgrader,
)
@ -71,9 +77,7 @@ class Swarm(Service, INetworkService):
peerstore: IPeerStore
upgrader: TransportUpgrader
transport: ITransport
# TODO: Connection and `peer_id` are 1-1 mapping in our implementation,
# whereas in Go one `peer_id` may point to multiple connections.
connections: dict[ID, INetConn]
connections: dict[ID, list[INetConn]]
listeners: dict[str, IListener]
common_stream_handler: StreamHandlerFn
listener_nursery: trio.Nursery | None
@ -81,18 +85,31 @@ class Swarm(Service, INetworkService):
notifees: list[INotifee]
# Enhanced: New configuration
retry_config: RetryConfig
connection_config: ConnectionConfig | QUICTransportConfig
_round_robin_index: dict[ID, int]
def __init__(
self,
peer_id: ID,
peerstore: IPeerStore,
upgrader: TransportUpgrader,
transport: ITransport,
retry_config: RetryConfig | None = None,
connection_config: ConnectionConfig | QUICTransportConfig | None = None,
):
self.self_id = peer_id
self.peerstore = peerstore
self.upgrader = upgrader
self.transport = transport
self.connections = dict()
# Enhanced: Initialize retry and connection configuration
self.retry_config = retry_config or RetryConfig()
self.connection_config = connection_config or ConnectionConfig()
# Enhanced: Initialize connections as 1:many mapping
self.connections = {}
self.listeners = dict()
# Create Notifee array
@ -103,11 +120,19 @@ class Swarm(Service, INetworkService):
self.listener_nursery = None
self.event_listener_nursery_created = trio.Event()
# Load balancing state
self._round_robin_index = {}
async def run(self) -> None:
async with trio.open_nursery() as nursery:
# Create a nursery for listener tasks.
self.listener_nursery = nursery
self.event_listener_nursery_created.set()
if isinstance(self.transport, QUICTransport):
self.transport.set_background_nursery(nursery)
self.transport.set_swarm(self)
try:
await self.manager.wait_finished()
finally:
@ -122,18 +147,74 @@ class Swarm(Service, INetworkService):
def set_stream_handler(self, stream_handler: StreamHandlerFn) -> None:
self.common_stream_handler = stream_handler
async def dial_peer(self, peer_id: ID) -> INetConn:
def get_connections(self, peer_id: ID | None = None) -> list[INetConn]:
"""
Try to create a connection to peer_id.
Get connections for peer (like JS getConnections, Go ConnsToPeer).
Parameters
----------
peer_id : ID | None
The peer ID to get connections for. If None, returns all connections.
Returns
-------
list[INetConn]
List of connections to the specified peer, or all connections
if peer_id is None.
"""
if peer_id is not None:
return self.connections.get(peer_id, [])
# Return all connections from all peers
all_conns = []
for conns in self.connections.values():
all_conns.extend(conns)
return all_conns
def get_connections_map(self) -> dict[ID, list[INetConn]]:
"""
Get all connections map (like JS getConnectionsMap).
Returns
-------
dict[ID, list[INetConn]]
The complete mapping of peer IDs to their connection lists.
"""
return self.connections.copy()
def get_connection(self, peer_id: ID) -> INetConn | None:
"""
Get single connection for backward compatibility.
Parameters
----------
peer_id : ID
The peer ID to get a connection for.
Returns
-------
INetConn | None
The first available connection, or None if no connections exist.
"""
conns = self.get_connections(peer_id)
return conns[0] if conns else None
async def dial_peer(self, peer_id: ID) -> list[INetConn]:
"""
Try to create connections to peer_id with enhanced retry logic.
:param peer_id: peer if we want to dial
:raises SwarmException: raised when an error occurs
:return: muxed connection
:return: list of muxed connections
"""
if peer_id in self.connections:
# If muxed connection already exists for peer_id,
# set muxed connection equal to existing muxed connection
return self.connections[peer_id]
# Check if we already have connections
existing_connections = self.get_connections(peer_id)
if existing_connections:
logger.debug(f"Reusing existing connections to peer {peer_id}")
return existing_connections
logger.debug("attempting to dial peer %s", peer_id)
@ -146,12 +227,19 @@ class Swarm(Service, INetworkService):
if not addrs:
raise SwarmException(f"No known addresses to peer {peer_id}")
connections = []
exceptions: list[SwarmException] = []
# Try all known addresses
# Enhanced: Try all known addresses with retry logic
for multiaddr in addrs:
try:
return await self.dial_addr(multiaddr, peer_id)
connection = await self._dial_with_retry(multiaddr, peer_id)
connections.append(connection)
# Limit number of connections per peer
if len(connections) >= self.connection_config.max_connections_per_peer:
break
except SwarmException as e:
exceptions.append(e)
logger.debug(
@ -161,15 +249,73 @@ class Swarm(Service, INetworkService):
exc_info=e,
)
# Tried all addresses, raising exception.
raise SwarmException(
f"unable to connect to {peer_id}, no addresses established a successful "
"connection (with exceptions)"
) from MultiError(exceptions)
if not connections:
# Tried all addresses, raising exception.
raise SwarmException(
f"unable to connect to {peer_id}, no addresses established a "
"successful connection (with exceptions)"
) from MultiError(exceptions)
async def dial_addr(self, addr: Multiaddr, peer_id: ID) -> INetConn:
return connections
async def _dial_with_retry(self, addr: Multiaddr, peer_id: ID) -> INetConn:
"""
Try to create a connection to peer_id with addr.
Enhanced: Dial with retry logic and exponential backoff.
:param addr: the address to dial
:param peer_id: the peer we want to connect to
:raises SwarmException: raised when all retry attempts fail
:return: network connection
"""
last_exception = None
for attempt in range(self.retry_config.max_retries + 1):
try:
return await self._dial_addr_single_attempt(addr, peer_id)
except Exception as e:
last_exception = e
if attempt < self.retry_config.max_retries:
delay = self._calculate_backoff_delay(attempt)
logger.debug(
f"Connection attempt {attempt + 1} failed, "
f"retrying in {delay:.2f}s: {e}"
)
await trio.sleep(delay)
else:
logger.debug(f"All {self.retry_config.max_retries} attempts failed")
# Convert the last exception to SwarmException for consistency
if last_exception is not None:
if isinstance(last_exception, SwarmException):
raise last_exception
else:
raise SwarmException(
f"Failed to connect after {self.retry_config.max_retries} attempts"
) from last_exception
# This should never be reached, but mypy requires it
raise SwarmException("Unexpected error in retry logic")
def _calculate_backoff_delay(self, attempt: int) -> float:
"""
Enhanced: Calculate backoff delay with jitter to prevent thundering herd.
:param attempt: the current attempt number (0-based)
:return: delay in seconds
"""
delay = min(
self.retry_config.initial_delay
* (self.retry_config.backoff_multiplier**attempt),
self.retry_config.max_delay,
)
# Add jitter to prevent synchronized retries
jitter = delay * self.retry_config.jitter_factor
return delay + random.uniform(-jitter, jitter)
async def _dial_addr_single_attempt(self, addr: Multiaddr, peer_id: ID) -> INetConn:
"""
Enhanced: Single attempt to dial an address (extracted from original dial_addr).
:param addr: the address we want to connect with
:param peer_id: the peer we want to connect to
@ -179,6 +325,7 @@ class Swarm(Service, INetworkService):
# Dial peer (connection to peer does not yet exist)
# Transport dials peer (gets back a raw conn)
try:
addr = Multiaddr(f"{addr}/p2p/{peer_id}")
raw_conn = await self.transport.dial(addr)
except OpenConnectionError as error:
logger.debug("fail to dial peer %s over base transport", peer_id)
@ -186,6 +333,15 @@ class Swarm(Service, INetworkService):
f"fail to open connection to peer {peer_id}"
) from error
if isinstance(self.transport, QUICTransport) and isinstance(
raw_conn, IMuxedConn
):
logger.info(
"Skipping upgrade for QUIC, QUIC connections are already multiplexed"
)
swarm_conn = await self.add_conn(raw_conn)
return swarm_conn
logger.debug("dialed peer %s over base transport", peer_id)
# Per, https://discuss.libp2p.io/t/multistream-security/130, we first secure
@ -211,24 +367,103 @@ class Swarm(Service, INetworkService):
logger.debug("upgraded mux for peer %s", peer_id)
swarm_conn = await self.add_conn(muxed_conn)
logger.debug("successfully dialed peer %s", peer_id)
return swarm_conn
async def dial_addr(self, addr: Multiaddr, peer_id: ID) -> INetConn:
"""
Enhanced: Try to create a connection to peer_id with addr using retry logic.
:param addr: the address we want to connect with
:param peer_id: the peer we want to connect to
:raises SwarmException: raised when an error occurs
:return: network connection
"""
return await self._dial_with_retry(addr, peer_id)
async def new_stream(self, peer_id: ID) -> INetStream:
"""
Enhanced: Create a new stream with load balancing across multiple connections.
:param peer_id: peer_id of destination
:raises SwarmException: raised when an error occurs
:return: net stream instance
"""
logger.debug("attempting to open a stream to peer %s", peer_id)
# Get existing connections or dial new ones
connections = self.get_connections(peer_id)
if not connections:
connections = await self.dial_peer(peer_id)
swarm_conn = await self.dial_peer(peer_id)
# Load balancing strategy at interface level
connection = self._select_connection(connections, peer_id)
net_stream = await swarm_conn.new_stream()
logger.debug("successfully opened a stream to peer %s", peer_id)
return net_stream
if isinstance(self.transport, QUICTransport) and connection is not None:
conn = cast(SwarmConn, connection)
return await conn.new_stream()
try:
net_stream = await connection.new_stream()
logger.debug("successfully opened a stream to peer %s", peer_id)
return net_stream
except Exception as e:
logger.debug(f"Failed to create stream on connection: {e}")
# Try other connections if available
for other_conn in connections:
if other_conn != connection:
try:
net_stream = await other_conn.new_stream()
logger.debug(
f"Successfully opened a stream to peer {peer_id} "
"using alternative connection"
)
return net_stream
except Exception:
continue
# All connections failed, raise exception
raise SwarmException(f"Failed to create stream to peer {peer_id}") from e
def _select_connection(self, connections: list[INetConn], peer_id: ID) -> INetConn:
"""
Select connection based on load balancing strategy.
Parameters
----------
connections : list[INetConn]
List of available connections.
peer_id : ID
The peer ID for round-robin tracking.
strategy : str
Load balancing strategy ("round_robin", "least_loaded", etc.).
Returns
-------
INetConn
Selected connection.
"""
if not connections:
raise ValueError("No connections available")
strategy = self.connection_config.load_balancing_strategy
if strategy == "round_robin":
# Simple round-robin selection
if peer_id not in self._round_robin_index:
self._round_robin_index[peer_id] = 0
index = self._round_robin_index[peer_id] % len(connections)
self._round_robin_index[peer_id] += 1
return connections[index]
elif strategy == "least_loaded":
# Find connection with least streams
return min(connections, key=lambda c: len(c.get_streams()))
else:
# Default to first connection
return connections[0]
async def listen(self, *multiaddrs: Multiaddr) -> bool:
"""
@ -248,17 +483,35 @@ class Swarm(Service, INetworkService):
"""
logger.debug(f"Swarm.listen called with multiaddrs: {multiaddrs}")
# We need to wait until `self.listener_nursery` is created.
logger.debug("Starting to listen")
await self.event_listener_nursery_created.wait()
success_count = 0
for maddr in multiaddrs:
logger.debug(f"Swarm.listen processing multiaddr: {maddr}")
if str(maddr) in self.listeners:
logger.debug(f"Swarm.listen: listener already exists for {maddr}")
return True
success_count += 1
continue
async def conn_handler(
read_write_closer: ReadWriteCloser, maddr: Multiaddr = maddr
) -> None:
# No need to upgrade QUIC Connection
if isinstance(self.transport, QUICTransport):
try:
quic_conn = cast(QUICConnection, read_write_closer)
await self.add_conn(quic_conn)
peer_id = quic_conn.peer_id
logger.debug(
f"successfully opened quic connection to peer {peer_id}"
)
# NOTE: This is a intentional barrier to prevent from the
# handler exiting and closing the connection.
await self.manager.wait_finished()
except Exception:
await read_write_closer.close()
return
raw_conn = RawConnection(read_write_closer, False)
# Per, https://discuss.libp2p.io/t/multistream-security/130, we first
@ -309,13 +562,14 @@ class Swarm(Service, INetworkService):
# Call notifiers since event occurred
await self.notify_listen(maddr)
return True
success_count += 1
logger.debug("successfully started listening on: %s", maddr)
except OSError:
# Failed. Continue looping.
logger.debug("fail to listen on: %s", maddr)
# No maddr succeeded
return False
# Return true if at least one address succeeded
return success_count > 0
async def close(self) -> None:
"""
@ -328,9 +582,9 @@ class Swarm(Service, INetworkService):
# Perform alternative cleanup if the manager isn't initialized
# Close all connections manually
if hasattr(self, "connections"):
for conn_id in list(self.connections.keys()):
conn = self.connections[conn_id]
await conn.close()
for peer_id, conns in list(self.connections.items()):
for conn in conns:
await conn.close()
# Clear connection tracking dictionary
self.connections.clear()
@ -360,12 +614,28 @@ class Swarm(Service, INetworkService):
logger.debug("swarm successfully closed")
async def close_peer(self, peer_id: ID) -> None:
if peer_id not in self.connections:
"""
Close all connections to the specified peer.
Parameters
----------
peer_id : ID
The peer ID to close connections for.
"""
connections = self.get_connections(peer_id)
if not connections:
return
connection = self.connections[peer_id]
# NOTE: `connection.close` will delete `peer_id` from `self.connections`
# and `notify_disconnected` for us.
await connection.close()
# Close all connections
for connection in connections:
try:
await connection.close()
except Exception as e:
logger.warning(f"Error closing connection to {peer_id}: {e}")
# Remove from connections dict
self.connections.pop(peer_id, None)
logger.debug("successfully close the connection to peer %s", peer_id)
@ -379,26 +649,77 @@ class Swarm(Service, INetworkService):
muxed_conn,
self,
)
logger.debug("Swarm::add_conn | starting muxed connection")
self.manager.run_task(muxed_conn.start)
await muxed_conn.event_started.wait()
logger.debug("Swarm::add_conn | starting swarm connection")
self.manager.run_task(swarm_conn.start)
await swarm_conn.event_started.wait()
# Store muxed_conn with peer id
self.connections[muxed_conn.peer_id] = swarm_conn
# Add to connections dict with deduplication
peer_id = muxed_conn.peer_id
if peer_id not in self.connections:
self.connections[peer_id] = []
# Check for duplicate connections by comparing the underlying muxed connection
for existing_conn in self.connections[peer_id]:
if existing_conn.muxed_conn == muxed_conn:
logger.debug(f"Connection already exists for peer {peer_id}")
# existing_conn is a SwarmConn since it's stored in the connections list
return existing_conn # type: ignore[return-value]
self.connections[peer_id].append(swarm_conn)
# Trim if we exceed max connections
max_conns = self.connection_config.max_connections_per_peer
if len(self.connections[peer_id]) > max_conns:
self._trim_connections(peer_id)
# Call notifiers since event occurred
await self.notify_connected(swarm_conn)
return swarm_conn
def _trim_connections(self, peer_id: ID) -> None:
"""
Remove oldest connections when limit is exceeded.
"""
connections = self.connections[peer_id]
if len(connections) <= self.connection_config.max_connections_per_peer:
return
# Sort by creation time and remove oldest
# For now, just keep the most recent connections
max_conns = self.connection_config.max_connections_per_peer
connections_to_remove = connections[:-max_conns]
for conn in connections_to_remove:
logger.debug(f"Trimming old connection for peer {peer_id}")
trio.lowlevel.spawn_system_task(self._close_connection_async, conn)
# Keep only the most recent connections
max_conns = self.connection_config.max_connections_per_peer
self.connections[peer_id] = connections[-max_conns:]
async def _close_connection_async(self, connection: INetConn) -> None:
"""Close a connection asynchronously."""
try:
await connection.close()
except Exception as e:
logger.warning(f"Error closing connection: {e}")
def remove_conn(self, swarm_conn: SwarmConn) -> None:
"""
Simply remove the connection from Swarm's records, without closing
the connection.
"""
peer_id = swarm_conn.muxed_conn.peer_id
if peer_id not in self.connections:
return
del self.connections[peer_id]
if peer_id in self.connections:
self.connections[peer_id] = [
conn for conn in self.connections[peer_id] if conn != swarm_conn
]
if not self.connections[peer_id]:
del self.connections[peer_id]
# Notifee
@ -444,3 +765,21 @@ class Swarm(Service, INetworkService):
async with trio.open_nursery() as nursery:
for notifee in self.notifees:
nursery.start_soon(notifier, notifee)
# Backward compatibility properties
@property
def connections_legacy(self) -> dict[ID, INetConn]:
"""
Legacy 1:1 mapping for backward compatibility.
Returns
-------
dict[ID, INetConn]
Legacy mapping with only the first connection per peer.
"""
legacy_conns = {}
for peer_id, conns in self.connections.items():
if conns:
legacy_conns[peer_id] = conns[0]
return legacy_conns