diff --git a/p2p/transport/webrtc/listener.go b/p2p/transport/webrtc/listener.go index 0ec05ec0e9..9a97e05de3 100644 --- a/p2p/transport/webrtc/listener.go +++ b/p2p/transport/webrtc/listener.go @@ -1,3 +1,6 @@ +//go:build !js +// +build !js + package libp2pwebrtc import ( @@ -8,8 +11,6 @@ import ( "fmt" "net" "strings" - "sync" - "time" "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" @@ -23,47 +24,6 @@ import ( "github.com/pion/webrtc/v4" ) -type connMultiaddrs struct { - local, remote ma.Multiaddr -} - -var _ network.ConnMultiaddrs = &connMultiaddrs{} - -func (c *connMultiaddrs) LocalMultiaddr() ma.Multiaddr { return c.local } -func (c *connMultiaddrs) RemoteMultiaddr() ma.Multiaddr { return c.remote } - -const ( - candidateSetupTimeout = 10 * time.Second - // This is higher than other transports(64) as there's no way to detect a peer that has gone away after - // sending the initial connection request message(STUN Binding request). Such peers take up a goroutine - // till connection timeout. As the number of handshakes in parallel is still guarded by the resource - // manager, this higher number is okay. - DefaultMaxInFlightConnections = 128 -) - -type listener struct { - transport *WebRTCTransport - - mux *udpmux.UDPMux - - config webrtc.Configuration - localFingerprint webrtc.DTLSFingerprint - localFingerprintMultibase string - - localAddr net.Addr - localMultiaddr ma.Multiaddr - - // buffered incoming connections - acceptQueue chan tpt.CapableConn - - // used to control the lifecycle of the listener - ctx context.Context - cancel context.CancelFunc - wg sync.WaitGroup -} - -var _ tpt.Listener = &listener{} - func newListener(transport *WebRTCTransport, laddr ma.Multiaddr, socket net.PacketConn, config webrtc.Configuration) (*listener, error) { localFingerprints, err := config.Certificates[0].GetFingerprints() if err != nil { @@ -288,65 +248,3 @@ func (l *listener) setupConnection( return conn, err } - -func (l *listener) Accept() (tpt.CapableConn, error) { - select { - case <-l.ctx.Done(): - return nil, tpt.ErrListenerClosed - case conn := <-l.acceptQueue: - return conn, nil - } -} - -func (l *listener) Close() error { - select { - case <-l.ctx.Done(): - default: - } - l.cancel() - l.mux.Close() - l.wg.Wait() -loop: - for { - select { - case conn := <-l.acceptQueue: - conn.Close() - default: - break loop - } - } - return nil -} - -func (l *listener) Addr() net.Addr { - return l.localAddr -} - -func (l *listener) Multiaddr() ma.Multiaddr { - return l.localMultiaddr -} - -// addOnConnectionStateChangeCallback adds the OnConnectionStateChange to the PeerConnection. -// The channel returned here: -// * is closed when the state changes to Connection -// * receives an error when the state changes to Failed or Closed or Disconnected -func addOnConnectionStateChangeCallback(pc *webrtc.PeerConnection) <-chan error { - errC := make(chan error, 1) - var once sync.Once - pc.OnConnectionStateChange(func(state webrtc.PeerConnectionState) { - switch pc.ConnectionState() { - case webrtc.PeerConnectionStateConnected: - once.Do(func() { close(errC) }) - // PeerConnectionStateFailed happens when we fail to negotiate the connection. - // PeerConnectionStateDisconnected happens when we disconnect immediately after connecting. - // PeerConnectionStateClosed happens when we close the peer connection locally, not when remote closes. We don't need - // to error in this case, but it's a no-op, so it doesn't hurt. - case webrtc.PeerConnectionStateFailed, webrtc.PeerConnectionStateClosed, webrtc.PeerConnectionStateDisconnected: - once.Do(func() { - errC <- errors.New("peerconnection failed") - close(errC) - }) - } - }) - return errC -} diff --git a/p2p/transport/webrtc/listener_shared.go b/p2p/transport/webrtc/listener_shared.go new file mode 100644 index 0000000000..472e68a7eb --- /dev/null +++ b/p2p/transport/webrtc/listener_shared.go @@ -0,0 +1,118 @@ +package libp2pwebrtc + +import ( + "context" + "errors" + "net" + "sync" + "time" + + "github.com/libp2p/go-libp2p/core/network" + tpt "github.com/libp2p/go-libp2p/core/transport" + "github.com/libp2p/go-libp2p/p2p/transport/webrtc/udpmux" + ma "github.com/multiformats/go-multiaddr" + "github.com/pion/webrtc/v4" +) + +const ( + candidateSetupTimeout = 10 * time.Second + // This is higher than other transports(64) as there's no way to detect a peer that has gone away after + // sending the initial connection request message(STUN Binding request). Such peers take up a goroutine + // till connection timeout. As the number of handshakes in parallel is still guarded by the resource + // manager, this higher number is okay. + DefaultMaxInFlightConnections = 128 +) + +type connMultiaddrs struct { + local, remote ma.Multiaddr +} + +var _ network.ConnMultiaddrs = &connMultiaddrs{} + +func (c *connMultiaddrs) LocalMultiaddr() ma.Multiaddr { return c.local } +func (c *connMultiaddrs) RemoteMultiaddr() ma.Multiaddr { return c.remote } + +type listener struct { + transport *WebRTCTransport + + mux *udpmux.UDPMux + + config webrtc.Configuration + localFingerprint webrtc.DTLSFingerprint + localFingerprintMultibase string + + localAddr net.Addr + localMultiaddr ma.Multiaddr + + // buffered incoming connections + acceptQueue chan tpt.CapableConn + + // used to control the lifecycle of the listener + ctx context.Context + cancel context.CancelFunc + wg sync.WaitGroup +} + +var _ tpt.Listener = &listener{} + +func (l *listener) Accept() (tpt.CapableConn, error) { + select { + case <-l.ctx.Done(): + return nil, tpt.ErrListenerClosed + case conn := <-l.acceptQueue: + return conn, nil + } +} + +func (l *listener) Addr() net.Addr { + return l.localAddr +} + +func (l *listener) Multiaddr() ma.Multiaddr { + return l.localMultiaddr +} + +func (l *listener) Close() error { + select { + case <-l.ctx.Done(): + default: + } + l.cancel() + l.mux.Close() + l.wg.Wait() +loop: + for { + select { + case conn := <-l.acceptQueue: + conn.Close() + default: + break loop + } + } + return nil +} + +// addOnConnectionStateChangeCallback adds the OnConnectionStateChange to the PeerConnection. +// The channel returned here: +// * is closed when the state changes to Connection +// * receives an error when the state changes to Failed or Closed or Disconnected +func addOnConnectionStateChangeCallback(pc *webrtc.PeerConnection) <-chan error { + errC := make(chan error, 1) + var once sync.Once + pc.OnConnectionStateChange(func(state webrtc.PeerConnectionState) { + switch pc.ConnectionState() { + case webrtc.PeerConnectionStateConnected: + once.Do(func() { close(errC) }) + // PeerConnectionStateFailed happens when we fail to negotiate the connection. + // PeerConnectionStateDisconnected happens when we disconnect immediately after connecting. + // PeerConnectionStateClosed happens when we close the peer connection locally, not when remote closes. We don't need + // to error in this case, but it's a no-op, so it doesn't hurt. + case webrtc.PeerConnectionStateFailed, webrtc.PeerConnectionStateClosed, webrtc.PeerConnectionStateDisconnected: + once.Do(func() { + errC <- errors.New("peerconnection failed") + close(errC) + }) + } + }) + return errC +} diff --git a/p2p/transport/webrtc/stream_test.go b/p2p/transport/webrtc/stream_test.go index 461ed27ff8..44c48e40e3 100644 --- a/p2p/transport/webrtc/stream_test.go +++ b/p2p/transport/webrtc/stream_test.go @@ -1,3 +1,6 @@ +//go:build !js +// +build !js + package libp2pwebrtc import ( diff --git a/p2p/transport/webrtc/transport.go b/p2p/transport/webrtc/transport.go index f3322c8117..559f43796b 100644 --- a/p2p/transport/webrtc/transport.go +++ b/p2p/transport/webrtc/transport.go @@ -1,191 +1,20 @@ +//go:build !js +// +build !js + // Package libp2pwebrtc implements the WebRTC transport for go-libp2p, // as described in https://github.com/libp2p/specs/tree/master/webrtc. + package libp2pwebrtc import ( - "context" - "crypto" - "crypto/ecdsa" - "crypto/elliptic" - "crypto/rand" - "crypto/x509" - "errors" "fmt" "net" - "time" - - mrand "math/rand/v2" - - "google.golang.org/protobuf/proto" - "github.com/libp2p/go-libp2p/core/connmgr" - ic "github.com/libp2p/go-libp2p/core/crypto" - "github.com/libp2p/go-libp2p/core/network" - "github.com/libp2p/go-libp2p/core/peer" - "github.com/libp2p/go-libp2p/core/pnet" - "github.com/libp2p/go-libp2p/core/sec" tpt "github.com/libp2p/go-libp2p/core/transport" - "github.com/libp2p/go-libp2p/p2p/security/noise" - libp2pquic "github.com/libp2p/go-libp2p/p2p/transport/quic" - "github.com/libp2p/go-libp2p/p2p/transport/webrtc/pb" - "github.com/libp2p/go-msgio" - ma "github.com/multiformats/go-multiaddr" manet "github.com/multiformats/go-multiaddr/net" - "github.com/multiformats/go-multihash" - - "github.com/pion/datachannel" - "github.com/pion/webrtc/v4" -) - -var webrtcComponent *ma.Component - -func init() { - var err error - webrtcComponent, err = ma.NewComponent(ma.ProtocolWithCode(ma.P_WEBRTC_DIRECT).Name, "") - if err != nil { - log.Fatal(err) - } -} - -const ( - // handshakeChannelNegotiated is used to specify that the - // handshake data channel does not need negotiation via DCEP. - // A constant is used since the `DataChannelInit` struct takes - // references instead of values. - handshakeChannelNegotiated = true - // handshakeChannelID is the agreed ID for the handshake data - // channel. A constant is used since the `DataChannelInit` struct takes - // references instead of values. We specify the type here as this - // value is only ever copied and passed by reference - handshakeChannelID = uint16(0) ) -// timeout values for the peerconnection -// https://github.com/pion/webrtc/blob/v3.1.50/settingengine.go#L102-L109 -const ( - DefaultDisconnectedTimeout = 20 * time.Second - DefaultFailedTimeout = 30 * time.Second - DefaultKeepaliveTimeout = 15 * time.Second - - // sctpReceiveBufferSize is the size of the buffer for incoming messages. - // - // This is enough space for enqueuing 10 full sized messages. - // Besides throughput, this only matters if an application is using multiple dependent - // streams, say streams 1 & 2. It reads from stream 1 only after receiving message from - // stream 2. A buffer of 10 messages should serve all such situations. - sctpReceiveBufferSize = 10 * maxReceiveMessageSize -) - -type WebRTCTransport struct { - webrtcConfig webrtc.Configuration - rcmgr network.ResourceManager - gater connmgr.ConnectionGater - privKey ic.PrivKey - noiseTpt *noise.Transport - localPeerId peer.ID - - listenUDP func(network string, laddr *net.UDPAddr) (net.PacketConn, error) - - // timeouts - peerConnectionTimeouts iceTimeouts - - // in-flight connections - maxInFlightConnections uint32 -} - -var _ tpt.Transport = &WebRTCTransport{} - -type Option func(*WebRTCTransport) error - -type iceTimeouts struct { - Disconnect time.Duration - Failed time.Duration - Keepalive time.Duration -} - -type ListenUDPFn func(network string, laddr *net.UDPAddr) (net.PacketConn, error) - -func New(privKey ic.PrivKey, psk pnet.PSK, gater connmgr.ConnectionGater, rcmgr network.ResourceManager, listenUDP ListenUDPFn, opts ...Option) (*WebRTCTransport, error) { - if psk != nil { - log.Error("WebRTC doesn't support private networks yet.") - return nil, fmt.Errorf("WebRTC doesn't support private networks yet") - } - if rcmgr == nil { - rcmgr = &network.NullResourceManager{} - } - localPeerID, err := peer.IDFromPrivateKey(privKey) - if err != nil { - return nil, fmt.Errorf("get local peer ID: %w", err) - } - // We use elliptic P-256 since it is widely supported by browsers. - // - // Implementation note: Testing with the browser, - // it seems like Chromium only supports ECDSA P-256 or RSA key signatures in the webrtc TLS certificate. - // We tried using P-228 and P-384 which caused the DTLS handshake to fail with Illegal Parameter - // - // Please refer to this is a list of suggested algorithms for the WebCrypto API. - // The algorithm for generating a certificate for an RTCPeerConnection - // must adhere to the WebCrpyto API. From my observation, - // RSA and ECDSA P-256 is supported on almost all browsers. - // Ed25519 is not present on the list. - pk, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader) - if err != nil { - return nil, fmt.Errorf("generate key for cert: %w", err) - } - cert, err := webrtc.GenerateCertificate(pk) - if err != nil { - return nil, fmt.Errorf("generate certificate: %w", err) - } - config := webrtc.Configuration{ - Certificates: []webrtc.Certificate{*cert}, - } - noiseTpt, err := noise.New(noise.ID, privKey, nil) - if err != nil { - return nil, fmt.Errorf("unable to create noise transport: %w", err) - } - transport := &WebRTCTransport{ - rcmgr: rcmgr, - gater: gater, - webrtcConfig: config, - privKey: privKey, - noiseTpt: noiseTpt, - localPeerId: localPeerID, - - listenUDP: listenUDP, - peerConnectionTimeouts: iceTimeouts{ - Disconnect: DefaultDisconnectedTimeout, - Failed: DefaultFailedTimeout, - Keepalive: DefaultKeepaliveTimeout, - }, - - maxInFlightConnections: DefaultMaxInFlightConnections, - } - for _, opt := range opts { - if err := opt(transport); err != nil { - return nil, err - } - } - return transport, nil -} - -func (t *WebRTCTransport) ListenOrder() int { - return libp2pquic.ListenOrder + 1 // We want to listen after QUIC listens so we can possibly reuse the same port. -} - -func (t *WebRTCTransport) Protocols() []int { - return []int{ma.P_WEBRTC_DIRECT} -} - -func (t *WebRTCTransport) Proxy() bool { - return false -} - -func (t *WebRTCTransport) CanDial(addr ma.Multiaddr) bool { - isValid, n := IsWebRTCDirectMultiaddr(addr) - return isValid && n > 0 -} - // Listen returns a listener for addr. // // The IP, Port combination for addr must be exclusive to this listener as a WebRTC listener cannot @@ -248,425 +77,3 @@ func (t *WebRTCTransport) listenSocket(socket net.PacketConn) (tpt.Listener, err t.webrtcConfig, ) } - -func (t *WebRTCTransport) Dial(ctx context.Context, remoteMultiaddr ma.Multiaddr, p peer.ID) (tpt.CapableConn, error) { - scope, err := t.rcmgr.OpenConnection(network.DirOutbound, false, remoteMultiaddr) - if err != nil { - return nil, err - } - if err := scope.SetPeer(p); err != nil { - scope.Done() - return nil, err - } - conn, err := t.dial(ctx, scope, remoteMultiaddr, p) - if err != nil { - scope.Done() - return nil, err - } - return conn, nil -} - -func (t *WebRTCTransport) dial(ctx context.Context, scope network.ConnManagementScope, remoteMultiaddr ma.Multiaddr, p peer.ID) (tConn tpt.CapableConn, err error) { - var w webRTCConnection - defer func() { - if err != nil { - if w.PeerConnection != nil { - _ = w.PeerConnection.Close() - } - if tConn != nil { - _ = tConn.Close() - tConn = nil - } - } - }() - - remoteMultihash, err := decodeRemoteFingerprint(remoteMultiaddr) - if err != nil { - return nil, fmt.Errorf("decode fingerprint: %w", err) - } - remoteHashFunction, ok := getSupportedSDPHash(remoteMultihash.Code) - if !ok { - return nil, fmt.Errorf("unsupported hash function: %w", nil) - } - - rnw, rhost, err := manet.DialArgs(remoteMultiaddr) - if err != nil { - return nil, fmt.Errorf("generate dial args: %w", err) - } - - raddr, err := net.ResolveUDPAddr(rnw, rhost) - if err != nil { - return nil, fmt.Errorf("resolve udp address: %w", err) - } - - // Instead of encoding the local fingerprint we - // generate a random UUID as the connection ufrag. - // The only requirement here is that the ufrag and password - // must be equal, which will allow the server to determine - // the password using the STUN message. - ufrag := genUfrag() - - settingEngine := webrtc.SettingEngine{ - LoggerFactory: pionLoggerFactory, - } - settingEngine.SetICECredentials(ufrag, ufrag) - settingEngine.DetachDataChannels() - // use the first best address candidate - settingEngine.SetPrflxAcceptanceMinWait(0) - settingEngine.SetICETimeouts( - t.peerConnectionTimeouts.Disconnect, - t.peerConnectionTimeouts.Failed, - t.peerConnectionTimeouts.Keepalive, - ) - // By default, webrtc will not collect candidates on the loopback address. - // This is disallowed in the ICE specification. However, implementations - // do not strictly follow this, for eg. Chrome gathers TCP loopback candidates. - // If you run pion on a system with only the loopback interface UP, - // it will not connect to anything. - settingEngine.SetIncludeLoopbackCandidate(true) - settingEngine.SetSCTPMaxReceiveBufferSize(sctpReceiveBufferSize) - if err := scope.ReserveMemory(sctpReceiveBufferSize, network.ReservationPriorityMedium); err != nil { - return nil, err - } - - w, err = newWebRTCConnection(settingEngine, t.webrtcConfig) - if err != nil { - return nil, fmt.Errorf("instantiating peer connection failed: %w", err) - } - - errC := addOnConnectionStateChangeCallback(w.PeerConnection) - - // do offer-answer exchange - offer, err := w.PeerConnection.CreateOffer(nil) - if err != nil { - return nil, fmt.Errorf("create offer: %w", err) - } - - err = w.PeerConnection.SetLocalDescription(offer) - if err != nil { - return nil, fmt.Errorf("set local description: %w", err) - } - - answerSDPString, err := createServerSDP(raddr, ufrag, *remoteMultihash) - if err != nil { - return nil, fmt.Errorf("render server SDP: %w", err) - } - - answer := webrtc.SessionDescription{SDP: answerSDPString, Type: webrtc.SDPTypeAnswer} - err = w.PeerConnection.SetRemoteDescription(answer) - if err != nil { - return nil, fmt.Errorf("set remote description: %w", err) - } - - // await peerconnection opening - select { - case err := <-errC: - if err != nil { - return nil, err - } - case <-ctx.Done(): - return nil, errors.New("peerconnection opening timed out") - } - - // We are connected, run the noise handshake - detached, err := detachHandshakeDataChannel(ctx, w.HandshakeDataChannel) - if err != nil { - return nil, err - } - channel := newStream(w.HandshakeDataChannel, detached, maxSendMessageSize, nil) - - remotePubKey, err := t.noiseHandshake(ctx, w.PeerConnection, channel, p, remoteHashFunction, false) - if err != nil { - return nil, err - } - - // Setup local and remote address for the connection - cp, err := w.HandshakeDataChannel.Transport().Transport().ICETransport().GetSelectedCandidatePair() - if cp == nil { - return nil, errors.New("ice connection did not have selected candidate pair: nil result") - } - if err != nil { - return nil, fmt.Errorf("ice connection did not have selected candidate pair: error: %w", err) - } - // the local address of the selected candidate pair should be the local address for the connection - localAddr, err := manet.FromNetAddr(&net.UDPAddr{IP: net.ParseIP(cp.Local.Address), Port: int(cp.Local.Port)}) - if err != nil { - return nil, err - } - remoteMultiaddrWithoutCerthash, _ := ma.SplitFunc(remoteMultiaddr, func(c ma.Component) bool { return c.Protocol().Code == ma.P_CERTHASH }) - - conn, err := newConnection( - network.DirOutbound, - w.PeerConnection, - t, - scope, - t.localPeerId, - localAddr, - p, - remotePubKey, - remoteMultiaddrWithoutCerthash, - w.IncomingDataChannels, - w.PeerConnectionClosedCh, - ) - if err != nil { - return nil, err - } - - if t.gater != nil && !t.gater.InterceptSecured(network.DirOutbound, p, conn) { - return nil, fmt.Errorf("secured connection gated") - } - return conn, nil -} - -func genUfrag() string { - const ( - uFragAlphabet = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ1234567890" - uFragPrefix = "libp2p+webrtc+v1/" - uFragIdLength = 32 - uFragLength = len(uFragPrefix) + uFragIdLength - ) - - seed := [32]byte{} - rand.Read(seed[:]) - r := mrand.New(mrand.New(mrand.NewChaCha8(seed))) - b := make([]byte, uFragLength) - for i := 0; i < len(uFragPrefix); i++ { - b[i] = uFragPrefix[i] - } - for i := len(uFragPrefix); i < uFragLength; i++ { - b[i] = uFragAlphabet[r.IntN(len(uFragAlphabet))] - } - return string(b) -} - -func (t *WebRTCTransport) getCertificateFingerprint() (webrtc.DTLSFingerprint, error) { - fps, err := t.webrtcConfig.Certificates[0].GetFingerprints() - if err != nil { - return webrtc.DTLSFingerprint{}, err - } - return fps[0], nil -} - -func (t *WebRTCTransport) generateNoisePrologue(pc *webrtc.PeerConnection, hash crypto.Hash, inbound bool) ([]byte, error) { - raw := pc.SCTP().Transport().GetRemoteCertificate() - cert, err := x509.ParseCertificate(raw) - if err != nil { - return nil, err - } - - // NOTE: should we want we can fork the cert code as well to avoid - // all the extra allocations due to unneeded string interspersing (hex) - localFp, err := t.getCertificateFingerprint() - if err != nil { - return nil, err - } - - remoteFpBytes, err := parseFingerprint(cert, hash) - if err != nil { - return nil, err - } - - localFpBytes, err := decodeInterspersedHexFromASCIIString(localFp.Value) - if err != nil { - return nil, err - } - - localEncoded, err := multihash.Encode(localFpBytes, multihash.SHA2_256) - if err != nil { - log.Debugf("could not encode multihash for local fingerprint") - return nil, err - } - remoteEncoded, err := multihash.Encode(remoteFpBytes, multihash.SHA2_256) - if err != nil { - log.Debugf("could not encode multihash for remote fingerprint") - return nil, err - } - - result := []byte("libp2p-webrtc-noise:") - if inbound { - result = append(result, remoteEncoded...) - result = append(result, localEncoded...) - } else { - result = append(result, localEncoded...) - result = append(result, remoteEncoded...) - } - return result, nil -} - -func (t *WebRTCTransport) noiseHandshake(ctx context.Context, pc *webrtc.PeerConnection, s *stream, peer peer.ID, hash crypto.Hash, inbound bool) (ic.PubKey, error) { - prologue, err := t.generateNoisePrologue(pc, hash, inbound) - if err != nil { - return nil, fmt.Errorf("generate prologue: %w", err) - } - opts := make([]noise.SessionOption, 0, 2) - opts = append(opts, noise.Prologue(prologue)) - if peer == "" { - opts = append(opts, noise.DisablePeerIDCheck()) - } - sessionTransport, err := t.noiseTpt.WithSessionOptions(opts...) - if err != nil { - return nil, fmt.Errorf("failed to instantiate Noise transport: %w", err) - } - var secureConn sec.SecureConn - if inbound { - secureConn, err = sessionTransport.SecureOutbound(ctx, netConnWrapper{s}, peer) - if err != nil { - return nil, fmt.Errorf("failed to secure inbound connection: %w", err) - } - } else { - secureConn, err = sessionTransport.SecureInbound(ctx, netConnWrapper{s}, peer) - if err != nil { - return nil, fmt.Errorf("failed to secure outbound connection: %w", err) - } - } - return secureConn.RemotePublicKey(), nil -} - -func (t *WebRTCTransport) AddCertHashes(addr ma.Multiaddr) (ma.Multiaddr, bool) { - listenerFingerprint, err := t.getCertificateFingerprint() - if err != nil { - return nil, false - } - - encodedLocalFingerprint, err := encodeDTLSFingerprint(listenerFingerprint) - if err != nil { - return nil, false - } - - certComp, err := ma.NewComponent(ma.ProtocolWithCode(ma.P_CERTHASH).Name, encodedLocalFingerprint) - if err != nil { - return nil, false - } - return addr.AppendComponent(certComp), true -} - -type netConnWrapper struct { - *stream -} - -func (netConnWrapper) LocalAddr() net.Addr { return nil } -func (netConnWrapper) RemoteAddr() net.Addr { return nil } -func (w netConnWrapper) Close() error { - // Close called while running the security handshake is an error and we should Reset the - // stream in that case rather than gracefully closing - w.stream.Reset() - return nil -} - -// detachHandshakeDataChannel detaches the handshake data channel -func detachHandshakeDataChannel(ctx context.Context, dc *webrtc.DataChannel) (datachannel.ReadWriteCloser, error) { - done := make(chan struct{}) - var rwc datachannel.ReadWriteCloser - var err error - dc.OnOpen(func() { - defer close(done) - rwc, err = dc.Detach() - }) - // this is safe since for detached datachannels, the peerconnection runs the onOpen - // callback immediately if the SCTP transport is also connected. - select { - case <-done: - return rwc, err - case <-ctx.Done(): - return nil, ctx.Err() - } -} - -// webRTCConnection holds the webrtc.PeerConnection with the handshake channel and the queue for -// incoming data channels created by the peer. -// -// When creating a webrtc.PeerConnection, It is important to set the OnDataChannel handler upfront -// before connecting with the peer. If the handler's set up after connecting with the peer, there's -// a small window of time where datachannels created by the peer may not surface to us and cause a -// memory leak. -type webRTCConnection struct { - PeerConnection *webrtc.PeerConnection - HandshakeDataChannel *webrtc.DataChannel - IncomingDataChannels chan dataChannel - PeerConnectionClosedCh chan struct{} -} - -func newWebRTCConnection(settings webrtc.SettingEngine, config webrtc.Configuration) (webRTCConnection, error) { - api := webrtc.NewAPI(webrtc.WithSettingEngine(settings)) - pc, err := api.NewPeerConnection(config) - if err != nil { - return webRTCConnection{}, fmt.Errorf("failed to create peer connection: %w", err) - } - - negotiated, id := handshakeChannelNegotiated, handshakeChannelID - handshakeDataChannel, err := pc.CreateDataChannel("", &webrtc.DataChannelInit{ - Negotiated: &negotiated, - ID: &id, - }) - if err != nil { - pc.Close() - return webRTCConnection{}, fmt.Errorf("failed to create handshake channel: %w", err) - } - - incomingDataChannels := make(chan dataChannel, maxAcceptQueueLen) - pc.OnDataChannel(func(dc *webrtc.DataChannel) { - dc.OnOpen(func() { - rwc, err := dc.Detach() - if err != nil { - log.Warnf("could not detach datachannel: id: %d", *dc.ID()) - return - } - select { - case incomingDataChannels <- dataChannel{rwc, dc}: - default: - log.Warnf("connection busy, rejecting stream") - b, _ := proto.Marshal(&pb.Message{Flag: pb.Message_RESET.Enum()}) - w := msgio.NewWriter(rwc) - w.WriteMsg(b) - rwc.Close() - } - }) - }) - - connectionClosedCh := make(chan struct{}, 1) - pc.SCTP().OnClose(func(err error) { - // We only need one message. Closing a connection is a problem as pion might invoke the callback more than once. - select { - case connectionClosedCh <- struct{}{}: - default: - } - }) - return webRTCConnection{ - PeerConnection: pc, - HandshakeDataChannel: handshakeDataChannel, - IncomingDataChannels: incomingDataChannels, - PeerConnectionClosedCh: connectionClosedCh, - }, nil -} - -// IsWebRTCDirectMultiaddr returns whether addr is a /webrtc-direct multiaddr with the count of certhashes -// in addr -func IsWebRTCDirectMultiaddr(addr ma.Multiaddr) (bool, int) { - var foundUDP, foundWebRTC bool - certHashCount := 0 - ma.ForEach(addr, func(c ma.Component) bool { - if !foundUDP { - if c.Protocol().Code == ma.P_UDP { - foundUDP = true - } - return true - } - if !foundWebRTC && foundUDP { - // protocol after udp must be webrtc-direct - if c.Protocol().Code != ma.P_WEBRTC_DIRECT { - return false - } - foundWebRTC = true - return true - } - if foundWebRTC { - if c.Protocol().Code == ma.P_CERTHASH { - certHashCount++ - } else { - return false - } - } - return true - }) - return foundUDP && foundWebRTC, certHashCount -} diff --git a/p2p/transport/webrtc/transport_shared.go b/p2p/transport/webrtc/transport_shared.go new file mode 100644 index 0000000000..c0e7c8b01d --- /dev/null +++ b/p2p/transport/webrtc/transport_shared.go @@ -0,0 +1,609 @@ +// Package libp2pwebrtc implements the WebRTC transport for go-libp2p, +// as described in https://github.com/libp2p/specs/tree/master/webrtc. +package libp2pwebrtc + +import ( + "context" + "crypto" + "crypto/ecdsa" + "crypto/elliptic" + "crypto/rand" + "crypto/x509" + "errors" + "fmt" + "net" + "time" + + mrand "math/rand/v2" + + "google.golang.org/protobuf/proto" + + "github.com/libp2p/go-libp2p/core/connmgr" + ic "github.com/libp2p/go-libp2p/core/crypto" + "github.com/libp2p/go-libp2p/core/network" + "github.com/libp2p/go-libp2p/core/peer" + "github.com/libp2p/go-libp2p/core/pnet" + "github.com/libp2p/go-libp2p/core/sec" + tpt "github.com/libp2p/go-libp2p/core/transport" + "github.com/libp2p/go-libp2p/p2p/security/noise" + libp2pquic "github.com/libp2p/go-libp2p/p2p/transport/quic" + "github.com/libp2p/go-libp2p/p2p/transport/webrtc/pb" + "github.com/libp2p/go-msgio" + + ma "github.com/multiformats/go-multiaddr" + manet "github.com/multiformats/go-multiaddr/net" + "github.com/multiformats/go-multihash" + + "github.com/pion/datachannel" + "github.com/pion/webrtc/v4" +) + +var webrtcComponent *ma.Component + +func init() { + var err error + webrtcComponent, err = ma.NewComponent(ma.ProtocolWithCode(ma.P_WEBRTC_DIRECT).Name, "") + if err != nil { + log.Fatal(err) + } +} + +const ( + // handshakeChannelNegotiated is used to specify that the + // handshake data channel does not need negotiation via DCEP. + // A constant is used since the `DataChannelInit` struct takes + // references instead of values. + handshakeChannelNegotiated = true + // handshakeChannelID is the agreed ID for the handshake data + // channel. A constant is used since the `DataChannelInit` struct takes + // references instead of values. We specify the type here as this + // value is only ever copied and passed by reference + handshakeChannelID = uint16(0) +) + +// timeout values for the peerconnection +// https://github.com/pion/webrtc/blob/v3.1.50/settingengine.go#L102-L109 +const ( + DefaultDisconnectedTimeout = 20 * time.Second + DefaultFailedTimeout = 30 * time.Second + DefaultKeepaliveTimeout = 15 * time.Second + + // sctpReceiveBufferSize is the size of the buffer for incoming messages. + // + // This is enough space for enqueuing 10 full sized messages. + // Besides throughput, this only matters if an application is using multiple dependent + // streams, say streams 1 & 2. It reads from stream 1 only after receiving message from + // stream 2. A buffer of 10 messages should serve all such situations. + sctpReceiveBufferSize = 10 * maxReceiveMessageSize +) + +type WebRTCTransport struct { + webrtcConfig webrtc.Configuration + rcmgr network.ResourceManager + gater connmgr.ConnectionGater + privKey ic.PrivKey + noiseTpt *noise.Transport + localPeerId peer.ID + + listenUDP func(network string, laddr *net.UDPAddr) (net.PacketConn, error) + + // timeouts + peerConnectionTimeouts iceTimeouts + + // in-flight connections + maxInFlightConnections uint32 +} + +var _ tpt.Transport = &WebRTCTransport{} + +type Option func(*WebRTCTransport) error + +type iceTimeouts struct { + Disconnect time.Duration + Failed time.Duration + Keepalive time.Duration +} + +type ListenUDPFn func(network string, laddr *net.UDPAddr) (net.PacketConn, error) + +func New(privKey ic.PrivKey, psk pnet.PSK, gater connmgr.ConnectionGater, rcmgr network.ResourceManager, listenUDP ListenUDPFn, opts ...Option) (*WebRTCTransport, error) { + if psk != nil { + log.Error("WebRTC doesn't support private networks yet.") + return nil, fmt.Errorf("WebRTC doesn't support private networks yet") + } + if rcmgr == nil { + rcmgr = &network.NullResourceManager{} + } + localPeerID, err := peer.IDFromPrivateKey(privKey) + if err != nil { + return nil, fmt.Errorf("get local peer ID: %w", err) + } + // We use elliptic P-256 since it is widely supported by browsers. + // + // Implementation note: Testing with the browser, + // it seems like Chromium only supports ECDSA P-256 or RSA key signatures in the webrtc TLS certificate. + // We tried using P-228 and P-384 which caused the DTLS handshake to fail with Illegal Parameter + // + // Please refer to this is a list of suggested algorithms for the WebCrypto API. + // The algorithm for generating a certificate for an RTCPeerConnection + // must adhere to the WebCrpyto API. From my observation, + // RSA and ECDSA P-256 is supported on almost all browsers. + // Ed25519 is not present on the list. + pk, err := ecdsa.GenerateKey(elliptic.P256(), rand.Reader) + if err != nil { + return nil, fmt.Errorf("generate key for cert: %w", err) + } + cert, err := webrtc.GenerateCertificate(pk) + if err != nil { + return nil, fmt.Errorf("generate certificate: %w", err) + } + config := webrtc.Configuration{ + Certificates: []webrtc.Certificate{*cert}, + } + noiseTpt, err := noise.New(noise.ID, privKey, nil) + if err != nil { + return nil, fmt.Errorf("unable to create noise transport: %w", err) + } + transport := &WebRTCTransport{ + rcmgr: rcmgr, + gater: gater, + webrtcConfig: config, + privKey: privKey, + noiseTpt: noiseTpt, + localPeerId: localPeerID, + + listenUDP: listenUDP, + peerConnectionTimeouts: iceTimeouts{ + Disconnect: DefaultDisconnectedTimeout, + Failed: DefaultFailedTimeout, + Keepalive: DefaultKeepaliveTimeout, + }, + + maxInFlightConnections: DefaultMaxInFlightConnections, + } + for _, opt := range opts { + if err := opt(transport); err != nil { + return nil, err + } + } + return transport, nil +} + +func (t *WebRTCTransport) ListenOrder() int { + return libp2pquic.ListenOrder + 1 // We want to listen after QUIC listens so we can possibly reuse the same port. +} + +func (t *WebRTCTransport) Protocols() []int { + return []int{ma.P_WEBRTC_DIRECT} +} + +func (t *WebRTCTransport) Proxy() bool { + return false +} + +func (t *WebRTCTransport) CanDial(addr ma.Multiaddr) bool { + isValid, n := IsWebRTCDirectMultiaddr(addr) + return isValid && n > 0 +} + +func (t *WebRTCTransport) Dial(ctx context.Context, remoteMultiaddr ma.Multiaddr, p peer.ID) (tpt.CapableConn, error) { + scope, err := t.rcmgr.OpenConnection(network.DirOutbound, false, remoteMultiaddr) + if err != nil { + return nil, err + } + if err := scope.SetPeer(p); err != nil { + scope.Done() + return nil, err + } + conn, err := t.dial(ctx, scope, remoteMultiaddr, p) + if err != nil { + scope.Done() + return nil, err + } + return conn, nil +} + +func (t *WebRTCTransport) dial(ctx context.Context, scope network.ConnManagementScope, remoteMultiaddr ma.Multiaddr, p peer.ID) (tConn tpt.CapableConn, err error) { + var w webRTCConnection + defer func() { + if err != nil { + if w.PeerConnection != nil { + _ = w.PeerConnection.Close() + } + if tConn != nil { + _ = tConn.Close() + tConn = nil + } + } + }() + + remoteMultihash, err := decodeRemoteFingerprint(remoteMultiaddr) + if err != nil { + return nil, fmt.Errorf("decode fingerprint: %w", err) + } + remoteHashFunction, ok := getSupportedSDPHash(remoteMultihash.Code) + if !ok { + return nil, fmt.Errorf("unsupported hash function: %w", nil) + } + + rnw, rhost, err := manet.DialArgs(remoteMultiaddr) + if err != nil { + return nil, fmt.Errorf("generate dial args: %w", err) + } + + raddr, err := net.ResolveUDPAddr(rnw, rhost) + if err != nil { + return nil, fmt.Errorf("resolve udp address: %w", err) + } + + // Instead of encoding the local fingerprint we + // generate a random UUID as the connection ufrag. + // The only requirement here is that the ufrag and password + // must be equal, which will allow the server to determine + // the password using the STUN message. + ufrag := genUfrag() + + settingEngine := webrtc.SettingEngine{ + LoggerFactory: pionLoggerFactory, + } + settingEngine.SetICECredentials(ufrag, ufrag) + settingEngine.DetachDataChannels() + // use the first best address candidate + settingEngine.SetPrflxAcceptanceMinWait(0) + settingEngine.SetICETimeouts( + t.peerConnectionTimeouts.Disconnect, + t.peerConnectionTimeouts.Failed, + t.peerConnectionTimeouts.Keepalive, + ) + // By default, webrtc will not collect candidates on the loopback address. + // This is disallowed in the ICE specification. However, implementations + // do not strictly follow this, for eg. Chrome gathers TCP loopback candidates. + // If you run pion on a system with only the loopback interface UP, + // it will not connect to anything. + settingEngine.SetIncludeLoopbackCandidate(true) + settingEngine.SetSCTPMaxReceiveBufferSize(sctpReceiveBufferSize) + if err := scope.ReserveMemory(sctpReceiveBufferSize, network.ReservationPriorityMedium); err != nil { + return nil, err + } + + w, err = newWebRTCConnection(settingEngine, t.webrtcConfig) + if err != nil { + return nil, fmt.Errorf("instantiating peer connection failed: %w", err) + } + + errC := addOnConnectionStateChangeCallback(w.PeerConnection) + + // do offer-answer exchange + offer, err := w.PeerConnection.CreateOffer(nil) + if err != nil { + return nil, fmt.Errorf("create offer: %w", err) + } + + err = w.PeerConnection.SetLocalDescription(offer) + if err != nil { + return nil, fmt.Errorf("set local description: %w", err) + } + + answerSDPString, err := createServerSDP(raddr, ufrag, *remoteMultihash) + if err != nil { + return nil, fmt.Errorf("render server SDP: %w", err) + } + + answer := webrtc.SessionDescription{SDP: answerSDPString, Type: webrtc.SDPTypeAnswer} + err = w.PeerConnection.SetRemoteDescription(answer) + if err != nil { + return nil, fmt.Errorf("set remote description: %w", err) + } + + // await peerconnection opening + select { + case err := <-errC: + if err != nil { + return nil, err + } + case <-ctx.Done(): + return nil, errors.New("peerconnection opening timed out") + } + + // We are connected, run the noise handshake + detached, err := detachHandshakeDataChannel(ctx, w.HandshakeDataChannel) + if err != nil { + return nil, err + } + channel := newStream(w.HandshakeDataChannel, detached, maxSendMessageSize, nil) + + remotePubKey, err := t.noiseHandshake(ctx, w.PeerConnection, channel, p, remoteHashFunction, false) + if err != nil { + return nil, err + } + + // Setup local and remote address for the connection + cp, err := w.HandshakeDataChannel.Transport().Transport().ICETransport().GetSelectedCandidatePair() + if cp == nil { + return nil, errors.New("ice connection did not have selected candidate pair: nil result") + } + if err != nil { + return nil, fmt.Errorf("ice connection did not have selected candidate pair: error: %w", err) + } + // the local address of the selected candidate pair should be the local address for the connection + localAddr, err := manet.FromNetAddr(&net.UDPAddr{IP: net.ParseIP(cp.Local.Address), Port: int(cp.Local.Port)}) + if err != nil { + return nil, err + } + remoteMultiaddrWithoutCerthash, _ := ma.SplitFunc(remoteMultiaddr, func(c ma.Component) bool { return c.Protocol().Code == ma.P_CERTHASH }) + + conn, err := newConnection( + network.DirOutbound, + w.PeerConnection, + t, + scope, + t.localPeerId, + localAddr, + p, + remotePubKey, + remoteMultiaddrWithoutCerthash, + w.IncomingDataChannels, + w.PeerConnectionClosedCh, + ) + if err != nil { + return nil, err + } + + if t.gater != nil && !t.gater.InterceptSecured(network.DirOutbound, p, conn) { + return nil, fmt.Errorf("secured connection gated") + } + return conn, nil +} + +func genUfrag() string { + const ( + uFragAlphabet = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ1234567890" + uFragPrefix = "libp2p+webrtc+v1/" + uFragIdLength = 32 + uFragLength = len(uFragPrefix) + uFragIdLength + ) + + seed := [32]byte{} + rand.Read(seed[:]) + r := mrand.New(mrand.New(mrand.NewChaCha8(seed))) + b := make([]byte, uFragLength) + for i := 0; i < len(uFragPrefix); i++ { + b[i] = uFragPrefix[i] + } + for i := len(uFragPrefix); i < uFragLength; i++ { + b[i] = uFragAlphabet[r.IntN(len(uFragAlphabet))] + } + return string(b) +} + +func (t *WebRTCTransport) getCertificateFingerprint() (webrtc.DTLSFingerprint, error) { + fps, err := t.webrtcConfig.Certificates[0].GetFingerprints() + if err != nil { + return webrtc.DTLSFingerprint{}, err + } + return fps[0], nil +} + +func (t *WebRTCTransport) generateNoisePrologue(pc *webrtc.PeerConnection, hash crypto.Hash, inbound bool) ([]byte, error) { + raw := pc.SCTP().Transport().GetRemoteCertificate() + cert, err := x509.ParseCertificate(raw) + if err != nil { + return nil, err + } + + // NOTE: should we want we can fork the cert code as well to avoid + // all the extra allocations due to unneeded string interspersing (hex) + localFp, err := t.getCertificateFingerprint() + if err != nil { + return nil, err + } + + remoteFpBytes, err := parseFingerprint(cert, hash) + if err != nil { + return nil, err + } + + localFpBytes, err := decodeInterspersedHexFromASCIIString(localFp.Value) + if err != nil { + return nil, err + } + + localEncoded, err := multihash.Encode(localFpBytes, multihash.SHA2_256) + if err != nil { + log.Debugf("could not encode multihash for local fingerprint") + return nil, err + } + remoteEncoded, err := multihash.Encode(remoteFpBytes, multihash.SHA2_256) + if err != nil { + log.Debugf("could not encode multihash for remote fingerprint") + return nil, err + } + + result := []byte("libp2p-webrtc-noise:") + if inbound { + result = append(result, remoteEncoded...) + result = append(result, localEncoded...) + } else { + result = append(result, localEncoded...) + result = append(result, remoteEncoded...) + } + return result, nil +} + +func (t *WebRTCTransport) noiseHandshake(ctx context.Context, pc *webrtc.PeerConnection, s *stream, peer peer.ID, hash crypto.Hash, inbound bool) (ic.PubKey, error) { + prologue, err := t.generateNoisePrologue(pc, hash, inbound) + if err != nil { + return nil, fmt.Errorf("generate prologue: %w", err) + } + opts := make([]noise.SessionOption, 0, 2) + opts = append(opts, noise.Prologue(prologue)) + if peer == "" { + opts = append(opts, noise.DisablePeerIDCheck()) + } + sessionTransport, err := t.noiseTpt.WithSessionOptions(opts...) + if err != nil { + return nil, fmt.Errorf("failed to instantiate Noise transport: %w", err) + } + var secureConn sec.SecureConn + if inbound { + secureConn, err = sessionTransport.SecureOutbound(ctx, netConnWrapper{s}, peer) + if err != nil { + return nil, fmt.Errorf("failed to secure inbound connection: %w", err) + } + } else { + secureConn, err = sessionTransport.SecureInbound(ctx, netConnWrapper{s}, peer) + if err != nil { + return nil, fmt.Errorf("failed to secure outbound connection: %w", err) + } + } + return secureConn.RemotePublicKey(), nil +} + +func (t *WebRTCTransport) AddCertHashes(addr ma.Multiaddr) (ma.Multiaddr, bool) { + listenerFingerprint, err := t.getCertificateFingerprint() + if err != nil { + return nil, false + } + + encodedLocalFingerprint, err := encodeDTLSFingerprint(listenerFingerprint) + if err != nil { + return nil, false + } + + certComp, err := ma.NewComponent(ma.ProtocolWithCode(ma.P_CERTHASH).Name, encodedLocalFingerprint) + if err != nil { + return nil, false + } + return addr.AppendComponent(certComp), true +} + +type netConnWrapper struct { + *stream +} + +func (netConnWrapper) LocalAddr() net.Addr { return nil } +func (netConnWrapper) RemoteAddr() net.Addr { return nil } +func (w netConnWrapper) Close() error { + // Close called while running the security handshake is an error and we should Reset the + // stream in that case rather than gracefully closing + w.stream.Reset() + return nil +} + +// detachHandshakeDataChannel detaches the handshake data channel +func detachHandshakeDataChannel(ctx context.Context, dc *webrtc.DataChannel) (datachannel.ReadWriteCloser, error) { + done := make(chan struct{}) + var rwc datachannel.ReadWriteCloser + var err error + dc.OnOpen(func() { + defer close(done) + rwc, err = dc.Detach() + }) + // this is safe since for detached datachannels, the peerconnection runs the onOpen + // callback immediately if the SCTP transport is also connected. + select { + case <-done: + return rwc, err + case <-ctx.Done(): + return nil, ctx.Err() + } +} + +// webRTCConnection holds the webrtc.PeerConnection with the handshake channel and the queue for +// incoming data channels created by the peer. +// +// When creating a webrtc.PeerConnection, It is important to set the OnDataChannel handler upfront +// before connecting with the peer. If the handler's set up after connecting with the peer, there's +// a small window of time where datachannels created by the peer may not surface to us and cause a +// memory leak. +type webRTCConnection struct { + PeerConnection *webrtc.PeerConnection + HandshakeDataChannel *webrtc.DataChannel + IncomingDataChannels chan dataChannel + PeerConnectionClosedCh chan struct{} +} + +func newWebRTCConnection(settings webrtc.SettingEngine, config webrtc.Configuration) (webRTCConnection, error) { + api := webrtc.NewAPI(webrtc.WithSettingEngine(settings)) + pc, err := api.NewPeerConnection(config) + if err != nil { + return webRTCConnection{}, fmt.Errorf("failed to create peer connection: %w", err) + } + + negotiated, id := handshakeChannelNegotiated, handshakeChannelID + handshakeDataChannel, err := pc.CreateDataChannel("", &webrtc.DataChannelInit{ + Negotiated: &negotiated, + ID: &id, + }) + if err != nil { + pc.Close() + return webRTCConnection{}, fmt.Errorf("failed to create handshake channel: %w", err) + } + + incomingDataChannels := make(chan dataChannel, maxAcceptQueueLen) + pc.OnDataChannel(func(dc *webrtc.DataChannel) { + dc.OnOpen(func() { + rwc, err := dc.Detach() + if err != nil { + log.Warnf("could not detach datachannel: id: %d", *dc.ID()) + return + } + select { + case incomingDataChannels <- dataChannel{rwc, dc}: + default: + log.Warnf("connection busy, rejecting stream") + b, _ := proto.Marshal(&pb.Message{Flag: pb.Message_RESET.Enum()}) + w := msgio.NewWriter(rwc) + w.WriteMsg(b) + rwc.Close() + } + }) + }) + + connectionClosedCh := make(chan struct{}, 1) + pc.SCTP().OnClose(func(err error) { + // We only need one message. Closing a connection is a problem as pion might invoke the callback more than once. + select { + case connectionClosedCh <- struct{}{}: + default: + } + }) + return webRTCConnection{ + PeerConnection: pc, + HandshakeDataChannel: handshakeDataChannel, + IncomingDataChannels: incomingDataChannels, + PeerConnectionClosedCh: connectionClosedCh, + }, nil +} + +// IsWebRTCDirectMultiaddr returns whether addr is a /webrtc-direct multiaddr with the count of certhashes +// in addr +func IsWebRTCDirectMultiaddr(addr ma.Multiaddr) (bool, int) { + var foundUDP, foundWebRTC bool + certHashCount := 0 + ma.ForEach(addr, func(c ma.Component) bool { + if !foundUDP { + if c.Protocol().Code == ma.P_UDP { + foundUDP = true + } + return true + } + if !foundWebRTC && foundUDP { + // protocol after udp must be webrtc-direct + if c.Protocol().Code != ma.P_WEBRTC_DIRECT { + return false + } + foundWebRTC = true + return true + } + if foundWebRTC { + if c.Protocol().Code == ma.P_CERTHASH { + certHashCount++ + } else { + return false + } + } + return true + }) + return foundUDP && foundWebRTC, certHashCount +} diff --git a/p2p/transport/webrtc/transport_test.go b/p2p/transport/webrtc/transport_test.go index 83f65c8c3f..94006b62e4 100644 --- a/p2p/transport/webrtc/transport_test.go +++ b/p2p/transport/webrtc/transport_test.go @@ -1,3 +1,6 @@ +//go:build !js +// +build !js + package libp2pwebrtc import ( diff --git a/p2p/transport/webrtc/transport_wasm.go b/p2p/transport/webrtc/transport_wasm.go new file mode 100644 index 0000000000..5e63c1f2ff --- /dev/null +++ b/p2p/transport/webrtc/transport_wasm.go @@ -0,0 +1,15 @@ +//go:build js +// +build js + +package libp2pwebrtc + +import ( + "fmt" + + tpt "github.com/libp2p/go-libp2p/core/transport" + ma "github.com/multiformats/go-multiaddr" +) + +func (t *WebRTCTransport) Listen(addr ma.Multiaddr) (tpt.Listener, error) { + return nil, fmt.Errorf("listening is not supported in the browser") +}