rcmgr: make it possible to embed the NullResourceManager

This commit is contained in:
Marten Seemann 2022-10-16 18:00:26 +01:00
parent 012b8ddac4
commit c0a0aa010e
17 changed files with 107 additions and 110 deletions

View File

@ -271,60 +271,57 @@ type ScopeStat struct {
}
// NullResourceManager is a stub for tests and initialization of default values
var NullResourceManager ResourceManager = &nullResourceManager{}
type NullResourceManager struct{}
type nullResourceManager struct{}
type nullScope struct{}
var _ ResourceScope = (*nullScope)(nil)
var _ ResourceScopeSpan = (*nullScope)(nil)
var _ ServiceScope = (*nullScope)(nil)
var _ ProtocolScope = (*nullScope)(nil)
var _ PeerScope = (*nullScope)(nil)
var _ ConnManagementScope = (*nullScope)(nil)
var _ ConnScope = (*nullScope)(nil)
var _ StreamManagementScope = (*nullScope)(nil)
var _ StreamScope = (*nullScope)(nil)
var _ ResourceScope = (*NullScope)(nil)
var _ ResourceScopeSpan = (*NullScope)(nil)
var _ ServiceScope = (*NullScope)(nil)
var _ ProtocolScope = (*NullScope)(nil)
var _ PeerScope = (*NullScope)(nil)
var _ ConnManagementScope = (*NullScope)(nil)
var _ ConnScope = (*NullScope)(nil)
var _ StreamManagementScope = (*NullScope)(nil)
var _ StreamScope = (*NullScope)(nil)
// NullScope is a stub for tests and initialization of default values
var NullScope = &nullScope{}
type NullScope struct{}
func (n *nullResourceManager) ViewSystem(f func(ResourceScope) error) error {
return f(NullScope)
func (n *NullResourceManager) ViewSystem(f func(ResourceScope) error) error {
return f(&NullScope{})
}
func (n *nullResourceManager) ViewTransient(f func(ResourceScope) error) error {
return f(NullScope)
func (n *NullResourceManager) ViewTransient(f func(ResourceScope) error) error {
return f(&NullScope{})
}
func (n *nullResourceManager) ViewService(svc string, f func(ServiceScope) error) error {
return f(NullScope)
func (n *NullResourceManager) ViewService(svc string, f func(ServiceScope) error) error {
return f(&NullScope{})
}
func (n *nullResourceManager) ViewProtocol(p protocol.ID, f func(ProtocolScope) error) error {
return f(NullScope)
func (n *NullResourceManager) ViewProtocol(p protocol.ID, f func(ProtocolScope) error) error {
return f(&NullScope{})
}
func (n *nullResourceManager) ViewPeer(p peer.ID, f func(PeerScope) error) error {
return f(NullScope)
func (n *NullResourceManager) ViewPeer(p peer.ID, f func(PeerScope) error) error {
return f(&NullScope{})
}
func (n *nullResourceManager) OpenConnection(dir Direction, usefd bool, endpoint multiaddr.Multiaddr) (ConnManagementScope, error) {
return NullScope, nil
func (n *NullResourceManager) OpenConnection(dir Direction, usefd bool, endpoint multiaddr.Multiaddr) (ConnManagementScope, error) {
return &NullScope{}, nil
}
func (n *nullResourceManager) OpenStream(p peer.ID, dir Direction) (StreamManagementScope, error) {
return NullScope, nil
func (n *NullResourceManager) OpenStream(p peer.ID, dir Direction) (StreamManagementScope, error) {
return &NullScope{}, nil
}
func (n *nullResourceManager) Close() error {
func (n *NullResourceManager) Close() error {
return nil
}
func (n *nullScope) ReserveMemory(size int, prio uint8) error { return nil }
func (n *nullScope) ReleaseMemory(size int) {}
func (n *nullScope) Stat() ScopeStat { return ScopeStat{} }
func (n *nullScope) BeginSpan() (ResourceScopeSpan, error) { return NullScope, nil }
func (n *nullScope) Done() {}
func (n *nullScope) Name() string { return "" }
func (n *nullScope) Protocol() protocol.ID { return "" }
func (n *nullScope) Peer() peer.ID { return "" }
func (n *nullScope) PeerScope() PeerScope { return NullScope }
func (n *nullScope) SetPeer(peer.ID) error { return nil }
func (n *nullScope) ProtocolScope() ProtocolScope { return NullScope }
func (n *nullScope) SetProtocol(proto protocol.ID) error { return nil }
func (n *nullScope) ServiceScope() ServiceScope { return NullScope }
func (n *nullScope) SetService(srv string) error { return nil }
func (n *NullScope) ReserveMemory(size int, prio uint8) error { return nil }
func (n *NullScope) ReleaseMemory(size int) {}
func (n *NullScope) Stat() ScopeStat { return ScopeStat{} }
func (n *NullScope) BeginSpan() (ResourceScopeSpan, error) { return &NullScope{}, nil }
func (n *NullScope) Done() {}
func (n *NullScope) Name() string { return "" }
func (n *NullScope) Protocol() protocol.ID { return "" }
func (n *NullScope) Peer() peer.ID { return "" }
func (n *NullScope) PeerScope() PeerScope { return &NullScope{} }
func (n *NullScope) SetPeer(peer.ID) error { return nil }
func (n *NullScope) ProtocolScope() ProtocolScope { return &NullScope{} }
func (n *NullScope) SetProtocol(proto protocol.ID) error { return nil }
func (n *NullScope) ServiceScope() ServiceScope { return &NullScope{} }
func (n *NullScope) SetService(srv string) error { return nil }

View File

@ -189,5 +189,5 @@ func (c *conn) Stat() network.ConnStats {
}
func (c *conn) Scope() network.ConnScope {
return network.NullScope
return &network.NullScope{}
}

View File

@ -370,5 +370,5 @@ func (pn *peernet) notifyAll(notification func(f network.Notifiee)) {
}
func (pn *peernet) ResourceManager() network.ResourceManager {
return network.NullResourceManager
return &network.NullResourceManager{}
}

View File

@ -287,7 +287,7 @@ func (s *stream) transport() {
}
func (s *stream) Scope() network.StreamScope {
return network.NullScope
return &network.NullScope{}
}
func (s *stream) cancelWrite(err error) {

View File

@ -178,7 +178,7 @@ func NewSwarm(local peer.ID, peers peerstore.Peerstore, opts ...Option) (*Swarm,
}
}
if s.rcmgr == nil {
s.rcmgr = network.NullResourceManager
s.rcmgr = &network.NullResourceManager{}
}
s.dsync = newDialSync(s.dialWorkerLoop)

View File

@ -45,7 +45,7 @@ func TestAddrsForDial(t *testing.T) {
ps.AddPrivKey(id, priv)
t.Cleanup(func() { ps.Close() })
tpt, err := websocket.New(nil, network.NullResourceManager)
tpt, err := websocket.New(nil, &network.NullResourceManager{})
require.NoError(t, err)
s, err := NewSwarm(id, ps, WithMultiaddrResolver(resolver))
require.NoError(t, err)
@ -81,7 +81,7 @@ func newTestSwarmWithResolver(t *testing.T, resolver *madns.Resolver) *Swarm {
})
// Add a tcp transport so that we know we can dial a tcp multiaddr and we don't filter it out.
tpt, err := tcp.NewTCPTransport(nil, network.NullResourceManager)
tpt, err := tcp.NewTCPTransport(nil, &network.NullResourceManager{})
require.NoError(t, err)
err = s.AddTransport(tpt)
require.NoError(t, err)

View File

@ -56,7 +56,7 @@ func TestAcceptSingleConn(t *testing.T) {
ln := createListener(t, u)
defer ln.Close()
cconn, err := dial(t, u, ln.Multiaddr(), id, network.NullScope)
cconn, err := dial(t, u, ln.Multiaddr(), id, &network.NullScope{})
require.NoError(err)
sconn, err := ln.Accept()
@ -80,7 +80,7 @@ func TestAcceptMultipleConns(t *testing.T) {
}()
for i := 0; i < 10; i++ {
cconn, err := dial(t, u, ln.Multiaddr(), id, network.NullScope)
cconn, err := dial(t, u, ln.Multiaddr(), id, &network.NullScope{})
require.NoError(err)
toClose = append(toClose, cconn)
@ -104,7 +104,7 @@ func TestConnectionsClosedIfNotAccepted(t *testing.T) {
ln := createListener(t, u)
defer ln.Close()
conn, err := dial(t, u, ln.Multiaddr(), id, network.NullScope)
conn, err := dial(t, u, ln.Multiaddr(), id, &network.NullScope{})
require.NoError(err)
errCh := make(chan error)
@ -143,7 +143,7 @@ func TestFailedUpgradeOnListen(t *testing.T) {
errCh <- err
}()
_, err := dial(t, u, ln.Multiaddr(), id, network.NullScope)
_, err := dial(t, u, ln.Multiaddr(), id, &network.NullScope{})
require.Error(err)
// close the listener.
@ -177,7 +177,7 @@ func TestListenerClose(t *testing.T) {
require.Contains(err.Error(), "use of closed network connection")
// doesn't accept new connections when it is closed
_, err = dial(t, u, ln.Multiaddr(), peer.ID("1"), network.NullScope)
_, err = dial(t, u, ln.Multiaddr(), peer.ID("1"), &network.NullScope{})
require.Error(err)
}
@ -189,7 +189,7 @@ func TestListenerCloseClosesQueued(t *testing.T) {
var conns []transport.CapableConn
for i := 0; i < 10; i++ {
conn, err := dial(t, upgrader, ln.Multiaddr(), id, network.NullScope)
conn, err := dial(t, upgrader, ln.Multiaddr(), id, &network.NullScope{})
require.NoError(err)
conns = append(conns, conn)
}
@ -249,7 +249,7 @@ func TestConcurrentAccept(t *testing.T) {
go func() {
defer wg.Done()
conn, err := dial(t, u, ln.Multiaddr(), id, network.NullScope)
conn, err := dial(t, u, ln.Multiaddr(), id, &network.NullScope{})
if err != nil {
errCh <- err
return
@ -279,7 +279,7 @@ func TestAcceptQueueBacklogged(t *testing.T) {
// setup AcceptQueueLength connections, but don't accept any of them
var counter int32 // to be used atomically
doDial := func() {
conn, err := dial(t, u, ln.Multiaddr(), id, network.NullScope)
conn, err := dial(t, u, ln.Multiaddr(), id, &network.NullScope{})
require.NoError(err)
atomic.AddInt32(&counter, 1)
t.Cleanup(func() { conn.Close() })
@ -315,7 +315,7 @@ func TestListenerConnectionGater(t *testing.T) {
defer ln.Close()
// no gating.
conn, err := dial(t, u, ln.Multiaddr(), id, network.NullScope)
conn, err := dial(t, u, ln.Multiaddr(), id, &network.NullScope{})
require.NoError(err)
require.False(conn.IsClosed())
_ = conn.Close()
@ -323,28 +323,28 @@ func TestListenerConnectionGater(t *testing.T) {
// rejecting after handshake.
testGater.BlockSecured(true)
testGater.BlockAccept(false)
conn, err = dial(t, u, ln.Multiaddr(), "invalid", network.NullScope)
conn, err = dial(t, u, ln.Multiaddr(), "invalid", &network.NullScope{})
require.Error(err)
require.Nil(conn)
// rejecting on accept will trigger firupgrader.
testGater.BlockSecured(true)
testGater.BlockAccept(true)
conn, err = dial(t, u, ln.Multiaddr(), "invalid", network.NullScope)
conn, err = dial(t, u, ln.Multiaddr(), "invalid", &network.NullScope{})
require.Error(err)
require.Nil(conn)
// rejecting only on acceptance.
testGater.BlockSecured(false)
testGater.BlockAccept(true)
conn, err = dial(t, u, ln.Multiaddr(), "invalid", network.NullScope)
conn, err = dial(t, u, ln.Multiaddr(), "invalid", &network.NullScope{})
require.Error(err)
require.Nil(conn)
// back to normal
testGater.BlockSecured(false)
testGater.BlockAccept(false)
conn, err = dial(t, u, ln.Multiaddr(), id, network.NullScope)
conn, err = dial(t, u, ln.Multiaddr(), id, &network.NullScope{})
require.NoError(err)
require.False(conn.IsClosed())
_ = conn.Close()
@ -366,7 +366,7 @@ func TestListenerResourceManagement(t *testing.T) {
connScope.EXPECT().PeerScope(),
)
cconn, err := dial(t, upgrader, ln.Multiaddr(), id, network.NullScope)
cconn, err := dial(t, upgrader, ln.Multiaddr(), id, &network.NullScope{})
require.NoError(t, err)
defer cconn.Close()
@ -384,7 +384,7 @@ func TestListenerResourceManagementDenied(t *testing.T) {
ln := createListener(t, upgrader)
rcmgr.EXPECT().OpenConnection(network.DirInbound, true, gomock.Not(ln.Multiaddr())).Return(nil, errors.New("nope"))
_, err := dial(t, upgrader, ln.Multiaddr(), id, network.NullScope)
_, err := dial(t, upgrader, ln.Multiaddr(), id, &network.NullScope{})
require.Error(t, err)
done := make(chan struct{})

View File

@ -90,7 +90,7 @@ func New(secureMuxer sec.SecureMuxer, muxer network.Multiplexer, opts ...Option)
}
}
if u.rcmgr == nil {
u.rcmgr = network.NullResourceManager
u.rcmgr = &network.NullResourceManager{}
}
return u, nil
}

View File

@ -121,21 +121,21 @@ func TestOutboundConnectionGating(t *testing.T) {
testGater := &testGater{}
_, dialUpgrader := createUpgrader(t, upgrader.WithConnectionGater(testGater))
conn, err := dial(t, dialUpgrader, ln.Multiaddr(), id, network.NullScope)
conn, err := dial(t, dialUpgrader, ln.Multiaddr(), id, &network.NullScope{})
require.NoError(err)
require.NotNil(conn)
_ = conn.Close()
// blocking accepts doesn't affect the dialling side, only the listener.
testGater.BlockAccept(true)
conn, err = dial(t, dialUpgrader, ln.Multiaddr(), id, network.NullScope)
conn, err = dial(t, dialUpgrader, ln.Multiaddr(), id, &network.NullScope{})
require.NoError(err)
require.NotNil(conn)
_ = conn.Close()
// now let's block all connections after being secured.
testGater.BlockSecured(true)
conn, err = dial(t, dialUpgrader, ln.Multiaddr(), id, network.NullScope)
conn, err = dial(t, dialUpgrader, ln.Multiaddr(), id, &network.NullScope{})
require.Error(err)
require.Contains(err.Error(), "gater rejected connection")
require.Nil(conn)
@ -153,7 +153,7 @@ func TestOutboundResourceManagement(t *testing.T) {
gomock.InOrder(
connScope.EXPECT().PeerScope(),
connScope.EXPECT().SetPeer(id),
connScope.EXPECT().PeerScope().Return(network.NullScope),
connScope.EXPECT().PeerScope().Return(&network.NullScope{}),
)
_, dialUpgrader := createUpgrader(t)
conn, err := dial(t, dialUpgrader, ln.Multiaddr(), id, connScope)
@ -174,7 +174,7 @@ func TestOutboundResourceManagement(t *testing.T) {
gomock.InOrder(
connScope.EXPECT().PeerScope(),
connScope.EXPECT().SetPeer(id),
connScope.EXPECT().PeerScope().Return(network.NullScope),
connScope.EXPECT().PeerScope().Return(&network.NullScope{}),
connScope.EXPECT().Done(),
)
_, dialUpgrader := createUpgrader(t)

View File

@ -89,7 +89,7 @@ func TestReservationFailures(t *testing.T) {
host.SetStreamHandler(proto.ProtoIDv2Hop, tc.streamHandler)
}
cl, err := libp2p.New(libp2p.ResourceManager(network.NullResourceManager))
cl, err := libp2p.New(libp2p.ResourceManager(&network.NullResourceManager{}))
require.NoError(t, err)
defer cl.Close()
_, err = client.Reserve(context.Background(), cl, peer.AddrInfo{ID: host.ID(), Addrs: host.Addrs()})

View File

@ -17,7 +17,7 @@ func (d *RelayTransport) Dial(ctx context.Context, a ma.Multiaddr, p peer.ID) (t
return nil, err
}
c.tagHop()
scope, _ := network.NullResourceManager.OpenConnection(network.DirOutbound, false, a)
scope, _ := (&network.NullResourceManager{}).OpenConnection(network.DirOutbound, false, a)
return d.upgrader.Upgrade(ctx, d, c, network.DirOutbound, p, scope)
}

View File

@ -189,7 +189,7 @@ func NewTransport(key ic.PrivKey, psk pnet.PSK, gater connmgr.ConnectionGater, r
return nil, err
}
if rcmgr == nil {
rcmgr = network.NullResourceManager
rcmgr = &network.NullResourceManager{}
}
qconfig := quicConfig.Clone()
keyBytes, err := key.Raw()

View File

@ -136,7 +136,7 @@ var _ transport.Transport = &TcpTransport{}
// created. It represents an entire TCP stack (though it might not necessarily be).
func NewTCPTransport(upgrader transport.Upgrader, rcmgr network.ResourceManager, opts ...Option) (*TcpTransport, error) {
if rcmgr == nil {
rcmgr = network.NullResourceManager
rcmgr = &network.NullResourceManager{}
}
tr := &TcpTransport{
upgrader: upgrader,

View File

@ -86,7 +86,7 @@ func TestResourceManager(t *testing.T) {
scope := mocknetwork.NewMockConnManagementScope(ctrl)
rcmgr.EXPECT().OpenConnection(network.DirOutbound, true, ln.Multiaddr()).Return(scope, nil)
scope.EXPECT().SetPeer(peerA)
scope.EXPECT().PeerScope().Return(network.NullScope).AnyTimes() // called by the upgrader
scope.EXPECT().PeerScope().Return(&network.NullScope{}).AnyTimes() // called by the upgrader
conn, err := tb.Dial(context.Background(), ln.Multiaddr(), peerA)
require.NoError(t, err)
scope.EXPECT().Done()

View File

@ -93,7 +93,7 @@ var _ transport.Transport = (*WebsocketTransport)(nil)
func New(u transport.Upgrader, rcmgr network.ResourceManager, opts ...Option) (*WebsocketTransport, error) {
if rcmgr == nil {
rcmgr = network.NullResourceManager
rcmgr = &network.NullResourceManager{}
}
t := &WebsocketTransport{
upgrader: u,

View File

@ -163,7 +163,7 @@ func testWSSServer(t *testing.T, listenAddr ma.Multiaddr) (ma.Multiaddr, peer.ID
}
id, u := newSecureUpgrader(t)
tpt, err := New(u, network.NullResourceManager, WithTLSConfig(tlsConf))
tpt, err := New(u, &network.NullResourceManager{}, WithTLSConfig(tlsConf))
if err != nil {
t.Fatal(err)
}
@ -246,7 +246,7 @@ func TestHostHeaderWss(t *testing.T) {
tlsConfig := &tls.Config{InsecureSkipVerify: true} // Our test server doesn't have a cert signed by a CA
_, u := newSecureUpgrader(t)
tpt, err := New(u, network.NullResourceManager, WithTLSClientConfig(tlsConfig))
tpt, err := New(u, &network.NullResourceManager{}, WithTLSClientConfig(tlsConfig))
require.NoError(t, err)
masToDial, err := tpt.Resolve(context.Background(), serverMA)
@ -265,7 +265,7 @@ func TestDialWss(t *testing.T) {
tlsConfig := &tls.Config{InsecureSkipVerify: true} // Our test server doesn't have a cert signed by a CA
_, u := newSecureUpgrader(t)
tpt, err := New(u, network.NullResourceManager, WithTLSClientConfig(tlsConfig))
tpt, err := New(u, &network.NullResourceManager{}, WithTLSClientConfig(tlsConfig))
require.NoError(t, err)
masToDial, err := tpt.Resolve(context.Background(), serverMA)
@ -288,7 +288,7 @@ func TestDialWssNoClientCert(t *testing.T) {
require.Contains(t, serverMA.String(), "tls")
_, u := newSecureUpgrader(t)
tpt, err := New(u, network.NullResourceManager)
tpt, err := New(u, &network.NullResourceManager{})
require.NoError(t, err)
masToDial, err := tpt.Resolve(context.Background(), serverMA)
@ -325,7 +325,7 @@ func connectAndExchangeData(t *testing.T, laddr ma.Multiaddr, secure bool) {
opts = append(opts, WithTLSConfig(tlsConf))
}
server, u := newUpgrader(t)
tpt, err := New(u, network.NullResourceManager, opts...)
tpt, err := New(u, &network.NullResourceManager{}, opts...)
require.NoError(t, err)
l, err := tpt.Listen(laddr)
require.NoError(t, err)
@ -344,7 +344,7 @@ func connectAndExchangeData(t *testing.T, laddr ma.Multiaddr, secure bool) {
opts = append(opts, WithTLSClientConfig(&tls.Config{InsecureSkipVerify: true}))
}
_, u := newUpgrader(t)
tpt, err := New(u, network.NullResourceManager, opts...)
tpt, err := New(u, &network.NullResourceManager{}, opts...)
require.NoError(t, err)
c, err := tpt.Dial(context.Background(), l.Multiaddr(), server)
require.NoError(t, err)
@ -378,7 +378,7 @@ func TestWebsocketConnection(t *testing.T) {
func TestWebsocketListenSecureFailWithoutTLSConfig(t *testing.T) {
_, u := newUpgrader(t)
tpt, err := New(u, network.NullResourceManager)
tpt, err := New(u, &network.NullResourceManager{})
require.NoError(t, err)
addr := ma.StringCast("/ip4/127.0.0.1/tcp/0/wss")
_, err = tpt.Listen(addr)
@ -387,7 +387,7 @@ func TestWebsocketListenSecureFailWithoutTLSConfig(t *testing.T) {
func TestWebsocketListenSecureAndInsecure(t *testing.T) {
serverID, serverUpgrader := newUpgrader(t)
server, err := New(serverUpgrader, network.NullResourceManager, WithTLSConfig(generateTLSConfig(t)))
server, err := New(serverUpgrader, &network.NullResourceManager{}, WithTLSConfig(generateTLSConfig(t)))
require.NoError(t, err)
lnInsecure, err := server.Listen(ma.StringCast("/ip4/127.0.0.1/tcp/0/ws"))
@ -397,7 +397,7 @@ func TestWebsocketListenSecureAndInsecure(t *testing.T) {
t.Run("insecure", func(t *testing.T) {
_, clientUpgrader := newUpgrader(t)
client, err := New(clientUpgrader, network.NullResourceManager, WithTLSClientConfig(&tls.Config{InsecureSkipVerify: true}))
client, err := New(clientUpgrader, &network.NullResourceManager{}, WithTLSClientConfig(&tls.Config{InsecureSkipVerify: true}))
require.NoError(t, err)
// dialing the insecure address should succeed
@ -414,7 +414,7 @@ func TestWebsocketListenSecureAndInsecure(t *testing.T) {
t.Run("secure", func(t *testing.T) {
_, clientUpgrader := newUpgrader(t)
client, err := New(clientUpgrader, network.NullResourceManager, WithTLSClientConfig(&tls.Config{InsecureSkipVerify: true}))
client, err := New(clientUpgrader, &network.NullResourceManager{}, WithTLSClientConfig(&tls.Config{InsecureSkipVerify: true}))
require.NoError(t, err)
// dialing the insecure address should succeed
@ -432,7 +432,7 @@ func TestWebsocketListenSecureAndInsecure(t *testing.T) {
func TestConcurrentClose(t *testing.T) {
_, u := newUpgrader(t)
tpt, err := New(u, network.NullResourceManager)
tpt, err := New(u, &network.NullResourceManager{})
require.NoError(t, err)
l, err := tpt.maListen(ma.StringCast("/ip4/127.0.0.1/tcp/0/ws"))
if err != nil {
@ -470,7 +470,7 @@ func TestConcurrentClose(t *testing.T) {
func TestWriteZero(t *testing.T) {
_, u := newUpgrader(t)
tpt, err := New(u, network.NullResourceManager)
tpt, err := New(u, &network.NullResourceManager{})
if err != nil {
t.Fatal(err)
}

View File

@ -88,7 +88,7 @@ func getCerthashComponent(t *testing.T, b []byte) ma.Multiaddr {
func TestTransport(t *testing.T) {
serverID, serverKey := newIdentity(t)
tr, err := libp2pwebtransport.New(serverKey, nil, network.NullResourceManager)
tr, err := libp2pwebtransport.New(serverKey, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer tr.(io.Closer).Close()
ln, err := tr.Listen(ma.StringCast("/ip4/127.0.0.1/udp/0/quic/webtransport"))
@ -98,7 +98,7 @@ func TestTransport(t *testing.T) {
addrChan := make(chan ma.Multiaddr)
go func() {
_, clientKey := newIdentity(t)
tr2, err := libp2pwebtransport.New(clientKey, nil, network.NullResourceManager)
tr2, err := libp2pwebtransport.New(clientKey, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer tr2.(io.Closer).Close()
@ -134,7 +134,7 @@ func TestTransport(t *testing.T) {
func TestHashVerification(t *testing.T) {
serverID, serverKey := newIdentity(t)
tr, err := libp2pwebtransport.New(serverKey, nil, network.NullResourceManager)
tr, err := libp2pwebtransport.New(serverKey, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer tr.(io.Closer).Close()
ln, err := tr.Listen(ma.StringCast("/ip4/127.0.0.1/udp/0/quic/webtransport"))
@ -147,7 +147,7 @@ func TestHashVerification(t *testing.T) {
}()
_, clientKey := newIdentity(t)
tr2, err := libp2pwebtransport.New(clientKey, nil, network.NullResourceManager)
tr2, err := libp2pwebtransport.New(clientKey, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer tr2.(io.Closer).Close()
@ -185,7 +185,7 @@ func TestCanDial(t *testing.T) {
}
_, key := newIdentity(t)
tr, err := libp2pwebtransport.New(key, nil, network.NullResourceManager)
tr, err := libp2pwebtransport.New(key, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer tr.(io.Closer).Close()
@ -211,7 +211,7 @@ func TestListenAddrValidity(t *testing.T) {
}
_, key := newIdentity(t)
tr, err := libp2pwebtransport.New(key, nil, network.NullResourceManager)
tr, err := libp2pwebtransport.New(key, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer tr.(io.Closer).Close()
@ -228,7 +228,7 @@ func TestListenAddrValidity(t *testing.T) {
func TestListenerAddrs(t *testing.T) {
_, key := newIdentity(t)
tr, err := libp2pwebtransport.New(key, nil, network.NullResourceManager)
tr, err := libp2pwebtransport.New(key, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer tr.(io.Closer).Close()
@ -266,7 +266,7 @@ func TestResourceManagerDialing(t *testing.T) {
func TestResourceManagerListening(t *testing.T) {
clientID, key := newIdentity(t)
cl, err := libp2pwebtransport.New(key, nil, network.NullResourceManager)
cl, err := libp2pwebtransport.New(key, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer cl.(io.Closer).Close()
@ -345,7 +345,7 @@ func TestConnectionGaterDialing(t *testing.T) {
connGater := NewMockConnectionGater(ctrl)
serverID, serverKey := newIdentity(t)
tr, err := libp2pwebtransport.New(serverKey, nil, network.NullResourceManager)
tr, err := libp2pwebtransport.New(serverKey, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer tr.(io.Closer).Close()
ln, err := tr.Listen(ma.StringCast("/ip4/127.0.0.1/udp/0/quic/webtransport"))
@ -356,7 +356,7 @@ func TestConnectionGaterDialing(t *testing.T) {
require.Equal(t, stripCertHashes(ln.Multiaddr()), addrs.RemoteMultiaddr())
})
_, key := newIdentity(t)
cl, err := libp2pwebtransport.New(key, connGater, network.NullResourceManager)
cl, err := libp2pwebtransport.New(key, connGater, &network.NullResourceManager{})
require.NoError(t, err)
defer cl.(io.Closer).Close()
_, err = cl.Dial(context.Background(), ln.Multiaddr(), serverID)
@ -369,7 +369,7 @@ func TestConnectionGaterInterceptAccept(t *testing.T) {
connGater := NewMockConnectionGater(ctrl)
serverID, serverKey := newIdentity(t)
tr, err := libp2pwebtransport.New(serverKey, connGater, network.NullResourceManager)
tr, err := libp2pwebtransport.New(serverKey, connGater, &network.NullResourceManager{})
require.NoError(t, err)
defer tr.(io.Closer).Close()
ln, err := tr.Listen(ma.StringCast("/ip4/127.0.0.1/udp/0/quic/webtransport"))
@ -382,7 +382,7 @@ func TestConnectionGaterInterceptAccept(t *testing.T) {
})
_, key := newIdentity(t)
cl, err := libp2pwebtransport.New(key, nil, network.NullResourceManager)
cl, err := libp2pwebtransport.New(key, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer cl.(io.Closer).Close()
_, err = cl.Dial(context.Background(), ln.Multiaddr(), serverID)
@ -395,7 +395,7 @@ func TestConnectionGaterInterceptSecured(t *testing.T) {
connGater := NewMockConnectionGater(ctrl)
serverID, serverKey := newIdentity(t)
tr, err := libp2pwebtransport.New(serverKey, connGater, network.NullResourceManager)
tr, err := libp2pwebtransport.New(serverKey, connGater, &network.NullResourceManager{})
require.NoError(t, err)
defer tr.(io.Closer).Close()
ln, err := tr.Listen(ma.StringCast("/ip4/127.0.0.1/udp/0/quic/webtransport"))
@ -403,7 +403,7 @@ func TestConnectionGaterInterceptSecured(t *testing.T) {
defer ln.Close()
clientID, key := newIdentity(t)
cl, err := libp2pwebtransport.New(key, nil, network.NullResourceManager)
cl, err := libp2pwebtransport.New(key, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer cl.(io.Closer).Close()
@ -461,7 +461,7 @@ func TestStaticTLSConf(t *testing.T) {
tlsConf := getTLSConf(t, net.ParseIP("127.0.0.1"), time.Now(), time.Now().Add(365*24*time.Hour))
serverID, serverKey := newIdentity(t)
tr, err := libp2pwebtransport.New(serverKey, nil, network.NullResourceManager, libp2pwebtransport.WithTLSConfig(tlsConf))
tr, err := libp2pwebtransport.New(serverKey, nil, &network.NullResourceManager{}, libp2pwebtransport.WithTLSConfig(tlsConf))
require.NoError(t, err)
defer tr.(io.Closer).Close()
ln, err := tr.Listen(ma.StringCast("/ip4/127.0.0.1/udp/0/quic/webtransport"))
@ -471,7 +471,7 @@ func TestStaticTLSConf(t *testing.T) {
t.Run("fails when the certificate is invalid", func(t *testing.T) {
_, key := newIdentity(t)
cl, err := libp2pwebtransport.New(key, nil, network.NullResourceManager)
cl, err := libp2pwebtransport.New(key, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer cl.(io.Closer).Close()
@ -485,7 +485,7 @@ func TestStaticTLSConf(t *testing.T) {
t.Run("fails when dialing with a wrong certhash", func(t *testing.T) {
_, key := newIdentity(t)
cl, err := libp2pwebtransport.New(key, nil, network.NullResourceManager)
cl, err := libp2pwebtransport.New(key, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer cl.(io.Closer).Close()
@ -500,7 +500,7 @@ func TestStaticTLSConf(t *testing.T) {
store := x509.NewCertPool()
store.AddCert(tlsConf.Certificates[0].Leaf)
tlsConf := &tls.Config{RootCAs: store}
cl, err := libp2pwebtransport.New(key, nil, network.NullResourceManager, libp2pwebtransport.WithTLSClientConfig(tlsConf))
cl, err := libp2pwebtransport.New(key, nil, &network.NullResourceManager{}, libp2pwebtransport.WithTLSClientConfig(tlsConf))
require.NoError(t, err)
defer cl.(io.Closer).Close()
@ -513,7 +513,7 @@ func TestStaticTLSConf(t *testing.T) {
func TestAcceptQueueFilledUp(t *testing.T) {
serverID, serverKey := newIdentity(t)
tr, err := libp2pwebtransport.New(serverKey, nil, network.NullResourceManager)
tr, err := libp2pwebtransport.New(serverKey, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer tr.(io.Closer).Close()
ln, err := tr.Listen(ma.StringCast("/ip4/127.0.0.1/udp/0/quic/webtransport"))
@ -523,7 +523,7 @@ func TestAcceptQueueFilledUp(t *testing.T) {
newConn := func() (tpt.CapableConn, error) {
t.Helper()
_, key := newIdentity(t)
cl, err := libp2pwebtransport.New(key, nil, network.NullResourceManager)
cl, err := libp2pwebtransport.New(key, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer cl.(io.Closer).Close()
return cl.Dial(context.Background(), ln.Multiaddr(), serverID)
@ -553,7 +553,7 @@ func TestSNIIsSent(t *testing.T) {
return tlsConf, nil
},
}
tr, err := libp2pwebtransport.New(key, nil, network.NullResourceManager, libp2pwebtransport.WithTLSConfig(tlsConf))
tr, err := libp2pwebtransport.New(key, nil, &network.NullResourceManager{}, libp2pwebtransport.WithTLSConfig(tlsConf))
require.NoError(t, err)
defer tr.(io.Closer).Close()
@ -561,7 +561,7 @@ func TestSNIIsSent(t *testing.T) {
require.NoError(t, err)
_, key2 := newIdentity(t)
clientTr, err := libp2pwebtransport.New(key2, nil, network.NullResourceManager)
clientTr, err := libp2pwebtransport.New(key2, nil, &network.NullResourceManager{})
require.NoError(t, err)
defer tr.(io.Closer).Close()