update go-libp2p to v0.22.0 (#498)

* update go-libp2p to v0.22.0

* skip TestGossipsubConnTagMessageDeliveries
This commit is contained in:
Marten Seemann 2022-08-26 12:45:41 +03:00 committed by GitHub
parent 60cf380032
commit 4f56e8f0a7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
44 changed files with 352 additions and 480 deletions

View File

@ -7,7 +7,7 @@ import (
"sync" "sync"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
const ( const (

View File

@ -7,7 +7,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
func TestBackoff_Update(t *testing.T) { func TestBackoff_Update(t *testing.T) {

View File

@ -4,7 +4,7 @@ import (
"sync" "sync"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/whyrusleeping/timecache" "github.com/whyrusleeping/timecache"
) )

View File

@ -5,7 +5,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
func TestMapBlacklist(t *testing.T) { func TestMapBlacklist(t *testing.T) {

View File

@ -6,8 +6,8 @@ import (
"io" "io"
"time" "time"
"github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"

View File

@ -5,9 +5,9 @@ import (
"math/rand" "math/rand"
"time" "time"
"github.com/libp2p/go-libp2p-core/discovery" "github.com/libp2p/go-libp2p/core/discovery"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
discimpl "github.com/libp2p/go-libp2p/p2p/discovery/backoff" discimpl "github.com/libp2p/go-libp2p/p2p/discovery/backoff"
) )

View File

@ -9,9 +9,9 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/discovery" "github.com/libp2p/go-libp2p/core/discovery"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
type mockDiscoveryServer struct { type mockDiscoveryServer struct {

View File

@ -3,9 +3,9 @@ package pubsub
import ( import (
"context" "context"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
) )
const ( const (

View File

@ -15,10 +15,10 @@ import (
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
bhost "github.com/libp2p/go-libp2p/p2p/host/blank" bhost "github.com/libp2p/go-libp2p/p2p/host/blank"
swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing" swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing"
@ -153,7 +153,7 @@ func TestBasicFloodsub(t *testing.T) {
msgs = append(msgs, subch) msgs = append(msgs, subch)
} }
//connectAll(t, hosts) // connectAll(t, hosts)
sparseConnect(t, hosts) sparseConnect(t, hosts)
time.Sleep(time.Millisecond * 100) time.Sleep(time.Millisecond * 100)

89
go.mod
View File

@ -1,16 +1,93 @@
module github.com/libp2p/go-libp2p-pubsub module github.com/libp2p/go-libp2p-pubsub
go 1.13 go 1.17
require ( require (
github.com/benbjohnson/clock v1.3.0 github.com/benbjohnson/clock v1.3.0
github.com/gogo/protobuf v1.3.2 github.com/gogo/protobuf v1.3.2
github.com/ipfs/go-log v1.0.5 github.com/ipfs/go-log v1.0.5
github.com/libp2p/go-libp2p v0.20.3 github.com/libp2p/go-libp2p v0.22.0
github.com/libp2p/go-libp2p-connmgr v0.2.4 github.com/libp2p/go-libp2p-testing v0.12.0
github.com/libp2p/go-libp2p-core v0.17.0
github.com/libp2p/go-libp2p-testing v0.9.2
github.com/libp2p/go-msgio v0.2.0 github.com/libp2p/go-msgio v0.2.0
github.com/multiformats/go-multiaddr v0.5.0 github.com/multiformats/go-multiaddr v0.6.0
github.com/whyrusleeping/timecache v0.0.0-20160911033111-cfcb2f1abfee github.com/whyrusleeping/timecache v0.0.0-20160911033111-cfcb2f1abfee
) )
require (
github.com/beorn7/perks v1.0.1 // indirect
github.com/cespare/xxhash/v2 v2.1.2 // indirect
github.com/cheekybits/genny v1.0.0 // indirect
github.com/containerd/cgroups v1.0.4 // indirect
github.com/coreos/go-systemd/v22 v22.3.2 // indirect
github.com/davecgh/go-spew v1.1.1 // indirect
github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c // indirect
github.com/decred/dcrd/dcrec/secp256k1/v4 v4.1.0 // indirect
github.com/docker/go-units v0.4.0 // indirect
github.com/elastic/gosigar v0.14.2 // indirect
github.com/francoispqt/gojay v1.2.13 // indirect
github.com/fsnotify/fsnotify v1.5.4 // indirect
github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0 // indirect
github.com/godbus/dbus/v5 v5.1.0 // indirect
github.com/golang/protobuf v1.5.2 // indirect
github.com/google/gopacket v1.1.19 // indirect
github.com/hashicorp/golang-lru v0.5.4 // indirect
github.com/ipfs/go-cid v0.2.0 // indirect
github.com/ipfs/go-log/v2 v2.5.1 // indirect
github.com/jbenet/go-temp-err-catcher v0.1.0 // indirect
github.com/klauspost/compress v1.15.1 // indirect
github.com/klauspost/cpuid/v2 v2.1.0 // indirect
github.com/libp2p/go-buffer-pool v0.1.0 // indirect
github.com/libp2p/go-flow-metrics v0.1.0 // indirect
github.com/libp2p/go-netroute v0.2.0 // indirect
github.com/libp2p/go-openssl v0.1.0 // indirect
github.com/libp2p/go-reuseport v0.2.0 // indirect
github.com/libp2p/go-yamux/v3 v3.1.2 // indirect
github.com/lucas-clemente/quic-go v0.28.1 // indirect
github.com/marten-seemann/qtls-go1-16 v0.1.5 // indirect
github.com/marten-seemann/qtls-go1-17 v0.1.2 // indirect
github.com/marten-seemann/qtls-go1-18 v0.1.2 // indirect
github.com/marten-seemann/qtls-go1-19 v0.1.0 // indirect
github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd // indirect
github.com/mattn/go-isatty v0.0.16 // indirect
github.com/mattn/go-pointer v0.0.1 // indirect
github.com/matttproud/golang_protobuf_extensions v1.0.1 // indirect
github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b // indirect
github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc // indirect
github.com/minio/sha256-simd v1.0.0 // indirect
github.com/mr-tron/base58 v1.2.0 // indirect
github.com/multiformats/go-base32 v0.0.4 // indirect
github.com/multiformats/go-base36 v0.1.0 // indirect
github.com/multiformats/go-multiaddr-fmt v0.1.0 // indirect
github.com/multiformats/go-multibase v0.1.1 // indirect
github.com/multiformats/go-multicodec v0.5.0 // indirect
github.com/multiformats/go-multihash v0.2.1 // indirect
github.com/multiformats/go-multistream v0.3.3 // indirect
github.com/multiformats/go-varint v0.0.6 // indirect
github.com/nxadm/tail v1.4.8 // indirect
github.com/onsi/ginkgo v1.16.5 // indirect
github.com/opencontainers/runtime-spec v1.0.2 // indirect
github.com/opentracing/opentracing-go v1.2.0 // indirect
github.com/pkg/errors v0.9.1 // indirect
github.com/pmezard/go-difflib v1.0.0 // indirect
github.com/prometheus/client_golang v1.12.1 // indirect
github.com/prometheus/client_model v0.2.0 // indirect
github.com/prometheus/common v0.37.0 // indirect
github.com/prometheus/procfs v0.8.0 // indirect
github.com/raulk/go-watchdog v1.3.0 // indirect
github.com/spacemonkeygo/spacelog v0.0.0-20180420211403-2296661a0572 // indirect
github.com/spaolacci/murmur3 v1.1.0 // indirect
github.com/stretchr/testify v1.8.0 // indirect
go.uber.org/atomic v1.10.0 // indirect
go.uber.org/multierr v1.8.0 // indirect
go.uber.org/zap v1.22.0 // indirect
golang.org/x/crypto v0.0.0-20220525230936-793ad666bf5e // indirect
golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4 // indirect
golang.org/x/net v0.0.0-20220812174116-3211cb980234 // indirect
golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab // indirect
golang.org/x/tools v0.1.12 // indirect
google.golang.org/protobuf v1.28.1 // indirect
gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c // indirect
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect
gopkg.in/yaml.v3 v3.0.1 // indirect
lukechampine.com/blake3 v1.1.7 // indirect
)

486
go.sum

File diff suppressed because it is too large Load Diff

View File

@ -5,8 +5,8 @@ import (
"sync" "sync"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
) )
// gossipTracer is an internal tracer that tracks IWANT requests in order to penalize // gossipTracer is an internal tracer that tracks IWANT requests in order to penalize

View File

@ -6,7 +6,7 @@ import (
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
func TestBrokenPromises(t *testing.T) { func TestBrokenPromises(t *testing.T) {

View File

@ -9,12 +9,12 @@ import (
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/peerstore" "github.com/libp2p/go-libp2p/core/peerstore"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
"github.com/libp2p/go-libp2p-core/record" "github.com/libp2p/go-libp2p/core/record"
) )
const ( const (

View File

@ -6,15 +6,16 @@ import (
"time" "time"
"github.com/benbjohnson/clock" "github.com/benbjohnson/clock"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing" swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing"
connmgr "github.com/libp2p/go-libp2p-connmgr" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/peer"
bhost "github.com/libp2p/go-libp2p/p2p/host/blank" bhost "github.com/libp2p/go-libp2p/p2p/host/blank"
"github.com/libp2p/go-libp2p/p2p/net/connmgr"
) )
func TestGossipsubConnTagMessageDeliveries(t *testing.T) { func TestGossipsubConnTagMessageDeliveries(t *testing.T) {
t.Skip("Test disabled with go-libp2p v0.22.0") // TODO: reenable test when updating to v0.23.0
ctx, cancel := context.WithCancel(context.Background()) ctx, cancel := context.WithCancel(context.Background())
defer cancel() defer cancel()
@ -24,8 +25,6 @@ func TestGossipsubConnTagMessageDeliveries(t *testing.T) {
oldGossipSubConnTagDecayInterval := GossipSubConnTagDecayInterval oldGossipSubConnTagDecayInterval := GossipSubConnTagDecayInterval
oldGossipSubConnTagMessageDeliveryCap := GossipSubConnTagMessageDeliveryCap oldGossipSubConnTagMessageDeliveryCap := GossipSubConnTagMessageDeliveryCap
oldSilencePeriod := connmgr.SilencePeriod
// set the gossipsub D parameters low, so that we have some peers outside the mesh // set the gossipsub D parameters low, so that we have some peers outside the mesh
GossipSubDlo = 3 GossipSubDlo = 3
GossipSubD = 3 GossipSubD = 3
@ -37,7 +36,6 @@ func TestGossipsubConnTagMessageDeliveries(t *testing.T) {
// will be forced out even if they end up in someone's mesh // will be forced out even if they end up in someone's mesh
GossipSubConnTagMessageDeliveryCap = 50 GossipSubConnTagMessageDeliveryCap = 50
connmgr.SilencePeriod = time.Millisecond
// reset globals after test // reset globals after test
defer func() { defer func() {
GossipSubD = oldGossipSubD GossipSubD = oldGossipSubD
@ -45,7 +43,6 @@ func TestGossipsubConnTagMessageDeliveries(t *testing.T) {
GossipSubDhi = oldGossipSubDHi GossipSubDhi = oldGossipSubDHi
GossipSubConnTagDecayInterval = oldGossipSubConnTagDecayInterval GossipSubConnTagDecayInterval = oldGossipSubConnTagDecayInterval
GossipSubConnTagMessageDeliveryCap = oldGossipSubConnTagMessageDeliveryCap GossipSubConnTagMessageDeliveryCap = oldGossipSubConnTagMessageDeliveryCap
connmgr.SilencePeriod = oldSilencePeriod
}() }()
decayClock := clock.NewMock() decayClock := clock.NewMock()
@ -63,8 +60,15 @@ func TestGossipsubConnTagMessageDeliveries(t *testing.T) {
honestPeers := make(map[peer.ID]struct{}) honestPeers := make(map[peer.ID]struct{})
for i := 0; i < nHonest; i++ { for i := 0; i < nHonest; i++ {
connmgrs[i] = connmgr.NewConnManager(nHonest, connLimit, 0, var err error
connmgr.DecayerConfig(&decayCfg)) connmgrs[i], err = connmgr.NewConnManager(nHonest, connLimit,
connmgr.WithGracePeriod(0),
connmgr.WithSilencePeriod(time.Millisecond),
connmgr.DecayerConfig(&decayCfg),
)
if err != nil {
t.Fatal(err)
}
netw := swarmt.GenSwarm(t) netw := swarmt.GenSwarm(t)
defer netw.Close() defer netw.Close()

View File

@ -3,7 +3,7 @@ package pubsub
import ( import (
"fmt" "fmt"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
) )
// GossipSubFeatureTest is a feature test function; it takes a feature and a protocol ID and // GossipSubFeatureTest is a feature test function; it takes a feature and a protocol ID and

View File

@ -8,7 +8,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
) )
func TestDefaultGossipSubFeatures(t *testing.T) { func TestDefaultGossipSubFeatures(t *testing.T) {

View File

@ -6,7 +6,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
) )
func TestGossipSubMatchingFn(t *testing.T) { func TestGossipSubMatchingFn(t *testing.T) {

View File

@ -8,10 +8,10 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"

View File

@ -13,11 +13,11 @@ import (
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/peerstore" "github.com/libp2p/go-libp2p/core/peerstore"
"github.com/libp2p/go-libp2p-core/record" "github.com/libp2p/go-libp2p/core/record"
bhost "github.com/libp2p/go-libp2p/p2p/host/blank" bhost "github.com/libp2p/go-libp2p/p2p/host/blank"
swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing" swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing"

View File

@ -3,7 +3,7 @@ package pubsub
import ( import (
"fmt" "fmt"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
// NewMessageCache creates a sliding window cache that remembers messages for as // NewMessageCache creates a sliding window cache that remembers messages for as

View File

@ -1,8 +1,8 @@
package pubsub package pubsub
import ( import (
"github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
ma "github.com/multiformats/go-multiaddr" ma "github.com/multiformats/go-multiaddr"
) )

View File

@ -8,10 +8,10 @@ import (
"sync" "sync"
"time" "time"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
manet "github.com/multiformats/go-multiaddr/net" manet "github.com/multiformats/go-multiaddr/net"
) )

View File

@ -5,7 +5,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
func TestPeerGater(t *testing.T) { func TestPeerGater(t *testing.T) {

View File

@ -12,12 +12,12 @@ import (
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p-core/crypto" "github.com/libp2p/go-libp2p/core/crypto"
"github.com/libp2p/go-libp2p-core/discovery" "github.com/libp2p/go-libp2p/core/discovery"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log"
"github.com/whyrusleeping/timecache" "github.com/whyrusleeping/timecache"

View File

@ -4,9 +4,9 @@ import (
"context" "context"
"math" "math"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
) )
const ( const (

View File

@ -6,7 +6,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
) )
func getRandomsub(ctx context.Context, h host.Host, size int, opts ...Option) *PubSub { func getRandomsub(ctx context.Context, h host.Host, size int, opts ...Option) *PubSub {

View File

@ -7,9 +7,9 @@ import (
"sync" "sync"
"time" "time"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
manet "github.com/multiformats/go-multiaddr/net" manet "github.com/multiformats/go-multiaddr/net"
) )
@ -147,6 +147,7 @@ type TopicScoreSnapshot struct {
// - ExtendedPeerScoreInspectFn, which takes a map of peer IDs to // - ExtendedPeerScoreInspectFn, which takes a map of peer IDs to
// PeerScoreSnapshots and allows inspection of individual score // PeerScoreSnapshots and allows inspection of individual score
// components for debugging peer scoring. // components for debugging peer scoring.
//
// This option must be passed _after_ the WithPeerScore option. // This option must be passed _after_ the WithPeerScore option.
func WithPeerScoreInspect(inspect interface{}, period time.Duration) Option { func WithPeerScoreInspect(inspect interface{}, period time.Duration) Option {
return func(ps *PubSub) error { return func(ps *PubSub) error {

View File

@ -6,7 +6,7 @@ import (
"net" "net"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
type PeerScoreThresholds struct { type PeerScoreThresholds struct {

View File

@ -5,7 +5,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
func TestPeerScoreThresholdsValidation(t *testing.T) { func TestPeerScoreThresholdsValidation(t *testing.T) {

View File

@ -7,7 +7,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
func TestScoreTimeInMesh(t *testing.T) { func TestScoreTimeInMesh(t *testing.T) {

View File

@ -5,8 +5,8 @@ import (
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p-core/crypto" "github.com/libp2p/go-libp2p/core/crypto"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
// MessageSignaturePolicy describes if signatures are produced, expected, and/or verified. // MessageSignaturePolicy describes if signatures are produced, expected, and/or verified.

View File

@ -5,8 +5,8 @@ import (
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p-core/crypto" "github.com/libp2p/go-libp2p/core/crypto"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
func TestSigning(t *testing.T) { func TestSigning(t *testing.T) {

View File

@ -6,7 +6,7 @@ import (
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
// ErrTooManySubscriptions may be returned by a SubscriptionFilter to signal that there are too many // ErrTooManySubscriptions may be returned by a SubscriptionFilter to signal that there are too many

View File

@ -8,7 +8,7 @@ import (
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
func TestBasicSubscriptionFilter(t *testing.T) { func TestBasicSubscriptionFilter(t *testing.T) {

View File

@ -5,9 +5,9 @@ import (
"sync" "sync"
"time" "time"
"github.com/libp2p/go-libp2p-core/connmgr" "github.com/libp2p/go-libp2p/core/connmgr"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
) )
var ( var (

View File

@ -6,9 +6,9 @@ import (
"time" "time"
"github.com/benbjohnson/clock" "github.com/benbjohnson/clock"
connmgr "github.com/libp2p/go-libp2p-connmgr" connmgri "github.com/libp2p/go-libp2p/core/connmgr"
connmgri "github.com/libp2p/go-libp2p-core/connmgr" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/p2p/net/connmgr"
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
) )
@ -16,7 +16,10 @@ import (
func TestTagTracerMeshTags(t *testing.T) { func TestTagTracerMeshTags(t *testing.T) {
// test that tags are applied when the tagTracer sees graft and prune events // test that tags are applied when the tagTracer sees graft and prune events
cmgr := connmgr.NewConnManager(5, 10, time.Minute) cmgr, err := connmgr.NewConnManager(5, 10, connmgr.WithGracePeriod(time.Minute))
if err != nil {
t.Fatal(err)
}
tt := newTagTracer(cmgr) tt := newTagTracer(cmgr)
p := peer.ID("a-peer") p := peer.ID("a-peer")
@ -38,7 +41,10 @@ func TestTagTracerMeshTags(t *testing.T) {
func TestTagTracerDirectPeerTags(t *testing.T) { func TestTagTracerDirectPeerTags(t *testing.T) {
// test that we add a tag to direct peers // test that we add a tag to direct peers
cmgr := connmgr.NewConnManager(5, 10, time.Minute) cmgr, err := connmgr.NewConnManager(5, 10, connmgr.WithGracePeriod(time.Minute))
if err != nil {
t.Fatal(err)
}
tt := newTagTracer(cmgr) tt := newTagTracer(cmgr)
p1 := peer.ID("1") p1 := peer.ID("1")
@ -75,7 +81,10 @@ func TestTagTracerDeliveryTags(t *testing.T) {
Clock: clk, Clock: clk,
Resolution: time.Minute, Resolution: time.Minute,
} }
cmgr := connmgr.NewConnManager(5, 10, time.Minute, connmgr.DecayerConfig(decayCfg)) cmgr, err := connmgr.NewConnManager(5, 10, connmgr.WithGracePeriod(time.Minute), connmgr.DecayerConfig(decayCfg))
if err != nil {
t.Fatal(err)
}
tt := newTagTracer(cmgr) tt := newTagTracer(cmgr)
@ -163,7 +172,10 @@ func TestTagTracerDeliveryTagsNearFirst(t *testing.T) {
Clock: clk, Clock: clk,
Resolution: time.Minute, Resolution: time.Minute,
} }
cmgr := connmgr.NewConnManager(5, 10, time.Minute, connmgr.DecayerConfig(decayCfg)) cmgr, err := connmgr.NewConnManager(5, 10, connmgr.WithGracePeriod(time.Minute), connmgr.DecayerConfig(decayCfg))
if err != nil {
t.Fatal(err)
}
tt := newTagTracer(cmgr) tt := newTagTracer(cmgr)

View File

@ -7,10 +7,10 @@ import (
"sync" "sync"
"time" "time"
"github.com/libp2p/go-libp2p-core/crypto"
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p/core/crypto"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
// ErrTopicClosed is returned if a Topic is utilized after it has been closed // ErrTopicClosed is returned if a Topic is utilized after it has been closed

View File

@ -12,9 +12,9 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer"
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
tnet "github.com/libp2p/go-libp2p-testing/net" tnet "github.com/libp2p/go-libp2p-testing/net"
"github.com/libp2p/go-libp2p/core/peer"
) )
func getTopics(psubs []*PubSub, topicID string, opts ...TopicOpt) []*Topic { func getTopics(psubs []*PubSub, topicID string, opts ...TopicOpt) []*Topic {

View File

@ -3,8 +3,8 @@ package pubsub
import ( import (
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
) )

View File

@ -13,9 +13,9 @@ import (
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/peerstore" "github.com/libp2p/go-libp2p/core/peerstore"
bhost "github.com/libp2p/go-libp2p/p2p/host/blank" bhost "github.com/libp2p/go-libp2p/p2p/host/blank"
swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing" swarmt "github.com/libp2p/go-libp2p/p2p/net/swarm/testing"
@ -125,7 +125,7 @@ type traceStats struct {
} }
func (t *traceStats) process(evt *pb.TraceEvent) { func (t *traceStats) process(evt *pb.TraceEvent) {
//fmt.Printf("process event %s\n", evt.GetType()) // fmt.Printf("process event %s\n", evt.GetType())
switch evt.GetType() { switch evt.GetType() {
case pb.TraceEvent_PUBLISH_MESSAGE: case pb.TraceEvent_PUBLISH_MESSAGE:
t.publish++ t.publish++

View File

@ -11,11 +11,11 @@ import (
pb "github.com/libp2p/go-libp2p-pubsub/pb" pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/libp2p/go-libp2p-core/host" "github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p-core/network" "github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p-core/peerstore" "github.com/libp2p/go-libp2p/core/peerstore"
"github.com/libp2p/go-libp2p-core/protocol" "github.com/libp2p/go-libp2p/core/protocol"
"github.com/libp2p/go-msgio/protoio" "github.com/libp2p/go-msgio/protoio"
) )

View File

@ -7,7 +7,7 @@ import (
"sync" "sync"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
const ( const (
@ -478,7 +478,7 @@ func (val *topicVal) validateMsg(ctx context.Context, src peer.ID, msg *Message)
} }
} }
/// Options // / Options
// WithValidateQueueSize sets the buffer of validate queue. Defaults to 32. // WithValidateQueueSize sets the buffer of validate queue. Defaults to 32.
// When queue is full, validation is throttled and new messages are dropped. // When queue is full, validation is throttled and new messages are dropped.

View File

@ -8,7 +8,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p/core/peer"
) )
func TestRegisterUnregisterValidator(t *testing.T) { func TestRegisterUnregisterValidator(t *testing.T) {