go-waku/waku/v2/node/wakunode2.go

813 lines
21 KiB
Go
Raw Normal View History

2021-03-11 20:27:12 +00:00
package node
import (
"context"
"errors"
2021-03-11 20:27:12 +00:00
"fmt"
"math/rand"
2021-11-17 16:19:42 +00:00
"net"
"sync"
2021-03-15 23:59:18 +00:00
"time"
2021-03-11 20:27:12 +00:00
2023-02-24 15:58:49 +00:00
golog "github.com/ipfs/go-log/v2"
2022-12-09 18:09:06 +00:00
"github.com/libp2p/go-libp2p"
"go.uber.org/zap"
2022-06-13 18:30:35 +00:00
"github.com/ethereum/go-ethereum/crypto"
"github.com/ethereum/go-ethereum/p2p/enode"
2021-08-31 18:19:49 +00:00
2022-10-19 19:39:32 +00:00
"github.com/libp2p/go-libp2p/core/event"
"github.com/libp2p/go-libp2p/core/host"
"github.com/libp2p/go-libp2p/core/network"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/libp2p/go-libp2p/core/peerstore"
2023-02-16 16:17:52 +00:00
"github.com/libp2p/go-libp2p/core/protocol"
"github.com/libp2p/go-libp2p/p2p/discovery/backoff"
2022-05-27 19:55:35 +00:00
ws "github.com/libp2p/go-libp2p/p2p/transport/websocket"
2021-03-11 20:27:12 +00:00
ma "github.com/multiformats/go-multiaddr"
"go.opencensus.io/stats"
2021-04-22 00:09:37 +00:00
"github.com/waku-org/go-waku/logging"
"github.com/waku-org/go-waku/waku/try"
v2 "github.com/waku-org/go-waku/waku/v2"
"github.com/waku-org/go-waku/waku/v2/discv5"
"github.com/waku-org/go-waku/waku/v2/metrics"
2023-04-20 18:51:13 +00:00
"github.com/waku-org/go-waku/waku/v2/protocol/enr"
"github.com/waku-org/go-waku/waku/v2/protocol/filter"
"github.com/waku-org/go-waku/waku/v2/protocol/legacy_filter"
"github.com/waku-org/go-waku/waku/v2/protocol/lightpush"
"github.com/waku-org/go-waku/waku/v2/protocol/pb"
"github.com/waku-org/go-waku/waku/v2/protocol/peer_exchange"
"github.com/waku-org/go-waku/waku/v2/protocol/relay"
"github.com/waku-org/go-waku/waku/v2/protocol/store"
2023-03-09 15:48:25 +00:00
"github.com/waku-org/go-waku/waku/v2/rendezvous"
2022-12-09 03:08:04 +00:00
"github.com/waku-org/go-waku/waku/v2/timesource"
"github.com/waku-org/go-waku/waku/v2/utils"
2021-03-11 20:27:12 +00:00
)
2021-11-10 13:36:51 +00:00
type Peer struct {
ID peer.ID `json:"peerID"`
2023-02-16 16:17:52 +00:00
Protocols []protocol.ID `json:"protocols"`
Addrs []ma.Multiaddr `json:"addrs"`
Connected bool `json:"connected"`
2021-11-10 13:36:51 +00:00
}
type storeFactory func(w *WakuNode) store.Store
type byte32 = [32]byte
type IdentityCredential = struct {
IDTrapdoor byte32 `json:"idTrapdoor"`
IDNullifier byte32 `json:"idNullifier"`
IDSecretHash byte32 `json:"idSecretHash"`
IDCommitment byte32 `json:"idCommitment"`
2022-08-12 12:44:13 +00:00
}
type RLNRelay interface {
IdentityCredential() (IdentityCredential, error)
MembershipIndex() (uint, error)
2022-08-12 12:44:13 +00:00
AppendRLNProof(msg *pb.WakuMessage, senderEpochTime time.Time) error
2022-08-18 16:27:10 +00:00
Stop()
2022-08-12 12:44:13 +00:00
}
2021-03-11 20:27:12 +00:00
type WakuNode struct {
2022-12-09 03:08:04 +00:00
host host.Host
opts *WakuNodeParameters
log *zap.Logger
timesource timesource.Timesource
relay Service
lightPush Service
peerConnector PeerConnectorService
discoveryV5 Service
peerExchange Service
rendezvous Service
legacyFilter ReceptorService
filterFullnode ReceptorService
filterLightnode Service
store ReceptorService
rlnRelay RLNRelay
2023-01-06 22:37:57 +00:00
2023-04-20 18:51:13 +00:00
wakuFlag enr.WakuEnrBitfield
2022-06-13 18:30:35 +00:00
localNode *enode.LocalNode
2021-11-01 14:42:55 +00:00
bcaster v2.Broadcaster
connectionNotif ConnectionNotifier
2021-06-16 10:14:22 +00:00
protocolEventSub event.Subscription
identificationEventSub event.Subscription
2021-11-17 16:19:42 +00:00
addressChangesSub event.Subscription
2023-02-07 13:45:06 +00:00
enrChangeCh chan struct{}
2021-06-16 10:14:22 +00:00
keepAliveMutex sync.Mutex
keepAliveFails map[peer.ID]int
cancel context.CancelFunc
wg *sync.WaitGroup
2021-06-16 10:14:22 +00:00
// Channel passed to WakuNode constructor
// receiving connection status notifications
connStatusChan chan ConnStatus
storeFactory storeFactory
}
func defaultStoreFactory(w *WakuNode) store.Store {
2023-04-17 00:04:12 +00:00
return store.NewWakuStore(w.opts.messageProvider, w.timesource, w.log)
2021-03-11 20:27:12 +00:00
}
// New is used to instantiate a WakuNode using a set of WakuNodeOptions
2023-01-06 22:37:57 +00:00
func New(opts ...WakuNodeOption) (*WakuNode, error) {
params := new(WakuNodeParameters)
params.libP2POpts = DefaultLibP2POptions
2021-11-17 16:19:42 +00:00
opts = append(DefaultWakuNodeOptions, opts...)
for _, opt := range opts {
err := opt(params)
2021-03-11 20:27:12 +00:00
if err != nil {
return nil, err
}
}
if params.logger == nil {
params.logger = utils.Logger()
2023-02-24 15:58:49 +00:00
golog.SetAllLoggers(params.logLevel)
}
2022-06-13 18:30:35 +00:00
if params.privKey == nil {
prvKey, err := crypto.GenerateKey()
if err != nil {
return nil, err
}
params.privKey = prvKey
}
2022-03-22 13:12:58 +00:00
if params.enableWSS {
params.libP2POpts = append(params.libP2POpts, libp2p.Transport(ws.New, ws.WithTLSConfig(params.tlsConfig)))
} else {
// Enable WS transport by default
2022-03-22 13:12:58 +00:00
params.libP2POpts = append(params.libP2POpts, libp2p.Transport(ws.New))
}
2021-11-17 16:19:42 +00:00
// Setting default host address if none was provided
if params.hostAddr == nil {
err := WithHostAddress(&net.TCPAddr{IP: net.ParseIP("0.0.0.0"), Port: 0})(params)
if err != nil {
return nil, err
}
}
if len(params.multiAddr) > 0 {
params.libP2POpts = append(params.libP2POpts, libp2p.ListenAddrs(params.multiAddr...))
}
2021-03-11 20:27:12 +00:00
2022-06-13 18:30:35 +00:00
params.libP2POpts = append(params.libP2POpts, params.Identity())
2021-03-15 16:07:23 +00:00
2021-10-15 02:15:02 +00:00
if params.addressFactory != nil {
params.libP2POpts = append(params.libP2POpts, libp2p.AddrsFactory(params.addressFactory))
}
2023-04-17 00:04:12 +00:00
var err error
2021-03-11 20:27:12 +00:00
w := new(WakuNode)
2021-11-01 14:42:55 +00:00
w.bcaster = v2.NewBroadcaster(1024)
w.opts = params
w.log = params.logger.Named("node2")
w.wg = &sync.WaitGroup{}
w.keepAliveFails = make(map[peer.ID]int)
2023-04-20 18:51:13 +00:00
w.wakuFlag = enr.NewWakuEnrBitfield(w.opts.enableLightPush, w.opts.enableLegacyFilter, w.opts.enableStore, w.opts.enableRelay)
2021-06-16 10:14:22 +00:00
2022-12-09 03:08:04 +00:00
if params.enableNTP {
w.timesource = timesource.NewNTPTimesource(w.opts.ntpURLs, w.log)
} else {
w.timesource = timesource.NewDefaultClock()
}
2023-01-06 22:37:57 +00:00
w.localNode, err = w.newLocalnode(w.opts.privKey)
if err != nil {
w.log.Error("creating localnode", zap.Error(err))
}
// Setup peer connection strategy
cacheSize := 600
rngSrc := rand.NewSource(rand.Int63())
minBackoff, maxBackoff := time.Second*30, time.Hour
bkf := backoff.NewExponentialBackoff(minBackoff, maxBackoff, backoff.FullJitter, time.Second, 5.0, 0, rand.New(rngSrc))
2023-04-17 00:04:12 +00:00
w.peerConnector, err = v2.NewPeerConnectionStrategy(cacheSize, w.opts.discoveryMinPeers, network.DialPeerTimeout, bkf, w.log)
if err != nil {
w.log.Error("creating peer connection strategy", zap.Error(err))
}
2023-01-06 22:37:57 +00:00
if w.opts.enableDiscV5 {
err := w.mountDiscV5()
if err != nil {
return nil, err
}
}
2023-04-17 00:04:12 +00:00
w.peerExchange, err = peer_exchange.NewWakuPeerExchange(w.DiscV5(), w.peerConnector, w.log)
if err != nil {
return nil, err
}
2023-01-06 22:37:57 +00:00
2023-03-14 00:37:28 +00:00
var rendezvousPoints []peer.ID
for _, p := range w.opts.rendezvousNodes {
peerID, err := utils.GetPeerID(p)
if err != nil {
return nil, err
}
rendezvousPoints = append(rendezvousPoints, peerID)
}
2023-04-17 00:04:12 +00:00
w.rendezvous = rendezvous.NewRendezvous(w.opts.enableRendezvousServer, w.opts.rendezvousDB, w.opts.enableRendezvous, rendezvousPoints, w.peerConnector, w.log)
w.relay = relay.NewWakuRelay(w.bcaster, w.opts.minRelayPeersToPublish, w.timesource, w.log, w.opts.wOpts...)
w.legacyFilter = legacy_filter.NewWakuFilter(w.bcaster, w.opts.isLegacyFilterFullnode, w.timesource, w.log, w.opts.legacyFilterOpts...)
w.filterFullnode = filter.NewWakuFilterFullnode(w.bcaster, w.timesource, w.log, w.opts.filterOpts...)
w.filterLightnode = filter.NewWakuFilterLightnode(w.bcaster, w.timesource, w.log)
w.lightPush = lightpush.NewWakuLightPush(w.Relay(), w.log)
if params.storeFactory != nil {
w.storeFactory = params.storeFactory
} else {
w.storeFactory = defaultStoreFactory
}
if params.connStatusC != nil {
w.connStatusChan = params.connStatusC
2021-06-28 14:14:28 +00:00
}
2021-10-05 02:13:54 +00:00
return w, nil
}
func (w *WakuNode) watchMultiaddressChanges(ctx context.Context) {
defer w.wg.Done()
2021-11-17 16:19:42 +00:00
addrs := w.ListenAddresses()
first := make(chan struct{}, 1)
first <- struct{}{}
for {
select {
2023-01-06 22:37:57 +00:00
case <-ctx.Done():
2021-11-17 16:19:42 +00:00
return
case <-first:
2022-06-13 18:30:35 +00:00
w.log.Info("listening", logging.MultiAddrs("multiaddr", addrs...))
2023-02-07 13:45:06 +00:00
w.enrChangeCh <- struct{}{}
2021-11-17 16:19:42 +00:00
case <-w.addressChangesSub.Out():
newAddrs := w.ListenAddresses()
2022-06-13 18:30:35 +00:00
diff := false
2021-11-17 16:19:42 +00:00
if len(addrs) != len(newAddrs) {
2022-06-13 18:30:35 +00:00
diff = true
2021-11-17 16:19:42 +00:00
} else {
for i := range newAddrs {
if addrs[i].String() != newAddrs[i].String() {
2022-06-13 18:30:35 +00:00
diff = true
2021-11-17 16:19:42 +00:00
break
}
}
}
2022-06-13 18:30:35 +00:00
if diff {
2021-11-17 16:19:42 +00:00
addrs = newAddrs
2022-06-13 18:30:35 +00:00
w.log.Info("listening addresses update received", logging.MultiAddrs("multiaddr", addrs...))
2023-01-06 22:37:57 +00:00
_ = w.setupENR(ctx, addrs)
2023-02-07 13:45:06 +00:00
w.enrChangeCh <- struct{}{}
2021-11-17 16:19:42 +00:00
}
}
}
}
// Start initializes all the protocols that were setup in the WakuNode
2023-01-06 22:37:57 +00:00
func (w *WakuNode) Start(ctx context.Context) error {
ctx, cancel := context.WithCancel(ctx)
w.cancel = cancel
2022-12-09 03:08:04 +00:00
2023-04-17 00:04:12 +00:00
host, err := libp2p.New(w.opts.libP2POpts...)
if err != nil {
return err
}
w.host = host
if w.protocolEventSub, err = host.EventBus().Subscribe(new(event.EvtPeerProtocolsUpdated)); err != nil {
return err
}
if w.identificationEventSub, err = host.EventBus().Subscribe(new(event.EvtPeerIdentificationCompleted)); err != nil {
return err
}
if w.addressChangesSub, err = host.EventBus().Subscribe(new(event.EvtLocalAddressesUpdated)); err != nil {
return err
}
2023-01-06 22:37:57 +00:00
w.connectionNotif = NewConnectionNotifier(ctx, w.host, w.log)
w.host.Network().Notify(w.connectionNotif)
2023-02-15 18:43:51 +00:00
w.enrChangeCh = make(chan struct{}, 10)
w.wg.Add(3)
2023-01-06 22:37:57 +00:00
go w.connectednessListener(ctx)
go w.watchMultiaddressChanges(ctx)
go w.watchENRChanges(ctx)
2023-01-06 22:37:57 +00:00
2023-04-17 00:04:12 +00:00
err = w.bcaster.Start(ctx)
if err != nil {
return err
}
2023-01-06 22:37:57 +00:00
if w.opts.keepAliveInterval > time.Duration(0) {
w.wg.Add(1)
go w.startKeepAlive(ctx, w.opts.keepAliveInterval)
2022-06-19 21:47:39 +00:00
}
2023-04-17 00:04:12 +00:00
w.peerConnector.SetHost(host)
err = w.peerConnector.Start(ctx)
if err != nil {
return err
}
2023-01-06 22:37:57 +00:00
if w.opts.enableNTP {
err := w.timesource.Start(ctx)
2022-12-09 03:08:04 +00:00
if err != nil {
return err
}
}
2023-04-17 00:04:12 +00:00
w.relay.SetHost(host)
2023-01-06 22:37:57 +00:00
if w.opts.enableRelay {
err := w.relay.Start(ctx)
if err != nil {
return err
}
2021-06-10 12:59:51 +00:00
if !w.opts.noDefaultWakuTopic {
sub, err := w.Relay().Subscribe(ctx)
if err != nil {
return err
}
2023-01-06 22:37:57 +00:00
w.Broadcaster().Unregister(&relay.DefaultWakuTopic, sub.C)
}
2021-11-17 16:19:42 +00:00
}
2023-01-06 22:37:57 +00:00
w.store = w.storeFactory(w)
2023-04-17 00:04:12 +00:00
w.store.SetHost(host)
2023-01-06 22:37:57 +00:00
if w.opts.enableStore {
err := w.startStore(ctx)
2022-10-23 13:13:43 +00:00
if err != nil {
return err
}
2023-01-06 22:37:57 +00:00
w.log.Info("Subscribing store to broadcaster")
w.bcaster.Register(nil, w.store.MessageChannel())
2021-11-17 16:19:42 +00:00
}
2023-04-17 00:04:12 +00:00
w.lightPush.SetHost(host)
2023-01-06 22:37:57 +00:00
if w.opts.enableLightPush {
if err := w.lightPush.Start(ctx); err != nil {
2022-12-09 03:08:04 +00:00
return err
}
2021-06-16 10:14:22 +00:00
}
2023-04-17 00:04:12 +00:00
w.legacyFilter.SetHost(host)
if w.opts.enableLegacyFilter {
err := w.legacyFilter.Start(ctx)
2022-07-05 21:28:34 +00:00
if err != nil {
return err
}
2023-01-06 22:37:57 +00:00
w.log.Info("Subscribing filter to broadcaster")
w.bcaster.Register(nil, w.legacyFilter.MessageChannel())
2022-07-05 21:28:34 +00:00
}
2023-04-17 00:04:12 +00:00
w.filterFullnode.SetHost(host)
if w.opts.enableFilterFullNode {
err := w.filterFullnode.Start(ctx)
if err != nil {
return err
}
w.log.Info("Subscribing filterV2 to broadcaster")
w.bcaster.Register(nil, w.filterFullnode.MessageChannel())
}
2023-04-17 00:04:12 +00:00
w.filterLightnode.SetHost(host)
if w.opts.enableFilterLightNode {
err := w.filterLightnode.Start(ctx)
if err != nil {
return err
}
}
err = w.setupENR(ctx, w.ListenAddresses())
2023-01-06 22:37:57 +00:00
if err != nil {
return err
}
2023-04-17 00:04:12 +00:00
w.peerExchange.SetHost(host)
2023-01-06 22:37:57 +00:00
if w.opts.enablePeerExchange {
err := w.peerExchange.Start(ctx)
if err != nil {
2021-11-01 12:38:03 +00:00
return err
}
}
2023-04-17 00:04:12 +00:00
w.rendezvous.SetHost(host)
2023-03-09 22:42:50 +00:00
if w.opts.enableRendezvousServer || w.opts.enableRendezvous {
2023-03-09 15:48:25 +00:00
err := w.rendezvous.Start(ctx)
if err != nil {
return err
}
}
2023-01-06 22:37:57 +00:00
if w.opts.enableRLN {
err = w.mountRlnRelay(ctx)
if err != nil {
return err
}
2021-11-01 14:42:55 +00:00
}
2021-10-05 02:13:54 +00:00
return nil
2021-03-11 20:27:12 +00:00
}
// Stop stops the WakuNode and closess all connections to the host
2021-03-22 16:45:13 +00:00
func (w *WakuNode) Stop() {
2023-01-06 22:37:57 +00:00
if w.cancel == nil {
return
}
w.cancel()
w.bcaster.Stop()
2021-11-01 14:42:55 +00:00
defer w.connectionNotif.Close()
2021-06-16 10:14:22 +00:00
defer w.protocolEventSub.Close()
defer w.identificationEventSub.Close()
2021-11-17 16:19:42 +00:00
defer w.addressChangesSub.Close()
2023-03-09 22:42:50 +00:00
if w.opts.enableRendezvousServer || w.opts.enableRendezvous {
2023-03-09 15:48:25 +00:00
w.rendezvous.Stop()
}
2023-01-06 22:37:57 +00:00
w.relay.Stop()
w.lightPush.Stop()
w.store.Stop()
w.legacyFilter.Stop()
w.filterFullnode.Stop()
2023-01-06 22:37:57 +00:00
w.peerExchange.Stop()
2022-10-23 13:13:43 +00:00
2023-01-06 22:37:57 +00:00
if w.opts.enableDiscV5 {
2022-10-23 13:13:43 +00:00
w.discoveryV5.Stop()
}
w.peerConnector.Stop()
_ = w.stopRlnRelay()
2023-01-06 22:37:57 +00:00
w.timesource.Stop()
2022-12-09 03:08:04 +00:00
w.host.Close()
w.wg.Wait()
close(w.enrChangeCh)
w.cancel = nil
2021-03-15 16:07:23 +00:00
}
// Host returns the libp2p Host used by the WakuNode
2021-03-15 16:07:23 +00:00
func (w *WakuNode) Host() host.Host {
return w.host
}
// ID returns the base58 encoded ID from the host
func (w *WakuNode) ID() string {
return w.host.ID().Pretty()
}
func (w *WakuNode) watchENRChanges(ctx context.Context) {
defer w.wg.Done()
var prevNodeVal string
for {
select {
case <-ctx.Done():
return
2023-02-07 13:45:06 +00:00
case <-w.enrChangeCh:
if w.localNode != nil {
currNodeVal := w.localNode.Node().String()
if prevNodeVal != currNodeVal {
if prevNodeVal == "" {
w.log.Info("enr record", logging.ENode("enr", w.localNode.Node()))
} else {
w.log.Info("new enr record", logging.ENode("enr", w.localNode.Node()))
}
prevNodeVal = currNodeVal
}
}
}
}
}
// ListenAddresses returns all the multiaddresses used by the host
func (w *WakuNode) ListenAddresses() []ma.Multiaddr {
2021-04-04 17:05:33 +00:00
hostInfo, _ := ma.NewMultiaddr(fmt.Sprintf("/p2p/%s", w.host.ID().Pretty()))
var result []ma.Multiaddr
2021-04-04 17:05:33 +00:00
for _, addr := range w.host.Addrs() {
result = append(result, addr.Encapsulate(hostInfo))
2021-04-04 17:05:33 +00:00
}
return result
}
2022-06-13 18:30:35 +00:00
// ENR returns the ENR address of the node
func (w *WakuNode) ENR() *enode.Node {
return w.localNode.Node()
}
2022-12-09 03:08:04 +00:00
// Timesource returns the timesource used by this node to obtain the current wall time
// Depending on the configuration it will be the local time or a ntp syncd time
func (w *WakuNode) Timesource() timesource.Timesource {
return w.timesource
}
// Relay is used to access any operation related to Waku Relay protocol
func (w *WakuNode) Relay() *relay.WakuRelay {
2023-01-06 22:37:57 +00:00
if result, ok := w.relay.(*relay.WakuRelay); ok {
return result
}
return nil
2021-03-15 16:07:23 +00:00
}
// Store is used to access any operation related to Waku Store protocol
func (w *WakuNode) Store() store.Store {
2023-01-06 22:37:57 +00:00
return w.store.(store.Store)
2021-11-01 12:38:03 +00:00
}
// LegacyFilter is used to access any operation related to Waku LegacyFilter protocol
func (w *WakuNode) LegacyFilter() *legacy_filter.WakuFilter {
if result, ok := w.legacyFilter.(*legacy_filter.WakuFilter); ok {
2023-01-06 22:37:57 +00:00
return result
}
return nil
2021-06-10 12:59:51 +00:00
}
// FilterLightnode is used to access any operation related to Waku Filter protocol Full node feature
func (w *WakuNode) FilterFullnode() *filter.WakuFilterFullNode {
if result, ok := w.filterFullnode.(*filter.WakuFilterFullNode); ok {
return result
}
return nil
}
// FilterFullnode is used to access any operation related to Waku Filter protocol Light node feature
func (w *WakuNode) FilterLightnode() *filter.WakuFilterLightnode {
if result, ok := w.filterLightnode.(*filter.WakuFilterLightnode); ok {
return result
}
return nil
}
// Lightpush is used to access any operation related to Waku Lightpush protocol
2021-11-01 12:38:03 +00:00
func (w *WakuNode) Lightpush() *lightpush.WakuLightPush {
2023-01-06 22:37:57 +00:00
if result, ok := w.lightPush.(*lightpush.WakuLightPush); ok {
return result
}
return nil
2021-11-01 12:38:03 +00:00
}
// DiscV5 is used to access any operation related to DiscoveryV5
2021-11-17 16:19:42 +00:00
func (w *WakuNode) DiscV5() *discv5.DiscoveryV5 {
2023-01-06 22:37:57 +00:00
if result, ok := w.discoveryV5.(*discv5.DiscoveryV5); ok {
return result
}
return nil
2021-11-17 16:19:42 +00:00
}
2022-10-23 13:13:43 +00:00
// PeerExchange is used to access any operation related to Peer Exchange
func (w *WakuNode) PeerExchange() *peer_exchange.WakuPeerExchange {
2023-01-06 22:37:57 +00:00
if result, ok := w.peerExchange.(*peer_exchange.WakuPeerExchange); ok {
return result
}
return nil
2022-10-23 13:13:43 +00:00
}
// Broadcaster is used to access the message broadcaster that is used to push
// messages to different protocols
2021-11-18 14:20:58 +00:00
func (w *WakuNode) Broadcaster() v2.Broadcaster {
return w.bcaster
}
// Publish will attempt to publish a message via WakuRelay if there are enough
// peers available, otherwise it will attempt to publish via Lightpush protocol
func (w *WakuNode) Publish(ctx context.Context, msg *pb.WakuMessage) error {
if !w.opts.enableLightPush && !w.opts.enableRelay {
return errors.New("cannot publish message, relay and lightpush are disabled")
}
hash := msg.Hash(relay.DefaultWakuTopic)
err := try.Do(func(attempt int) (bool, error) {
var err error
2023-01-06 22:37:57 +00:00
relay := w.Relay()
lightpush := w.Lightpush()
if relay == nil || !relay.EnoughPeersToPublish() {
w.log.Debug("publishing message via lightpush", logging.HexBytes("hash", hash))
_, err = lightpush.Publish(ctx, msg)
} else {
2022-05-27 13:25:06 +00:00
w.log.Debug("publishing message via relay", logging.HexBytes("hash", hash))
2023-01-06 22:37:57 +00:00
_, err = relay.Publish(ctx, msg)
}
return attempt < maxPublishAttempt, err
})
return err
}
2021-11-17 16:19:42 +00:00
func (w *WakuNode) mountDiscV5() error {
discV5Options := []discv5.DiscoveryV5Option{
discv5.WithBootnodes(w.opts.discV5bootnodes),
discv5.WithUDPPort(w.opts.udpPort),
discv5.WithAutoUpdate(w.opts.discV5autoUpdate),
}
if w.opts.advertiseAddrs != nil {
discV5Options = append(discV5Options, discv5.WithAdvertiseAddr(w.opts.advertiseAddrs))
2022-03-10 22:14:50 +00:00
}
2022-03-30 00:02:33 +00:00
var err error
2023-04-17 00:04:12 +00:00
w.discoveryV5, err = discv5.NewDiscoveryV5(w.opts.privKey, w.localNode, w.peerConnector, w.log, discV5Options...)
2021-11-17 16:19:42 +00:00
2021-12-08 14:21:30 +00:00
return err
2021-11-17 16:19:42 +00:00
}
2023-01-06 22:37:57 +00:00
func (w *WakuNode) startStore(ctx context.Context) error {
err := w.store.Start(ctx)
2022-12-09 03:08:04 +00:00
if err != nil {
w.log.Error("starting store", zap.Error(err))
return err
}
2022-11-25 20:54:11 +00:00
if len(w.opts.resumeNodes) != 0 {
// TODO: extract this to a function and run it when you go offline
// TODO: determine if a store is listening to a topic
2022-11-25 20:54:11 +00:00
var peerIDs []peer.ID
for _, n := range w.opts.resumeNodes {
2023-02-16 16:17:52 +00:00
pID, err := w.AddPeer(n, store.StoreID_v20beta4)
2022-11-25 20:54:11 +00:00
if err != nil {
w.log.Warn("adding peer to peerstore", logging.MultiAddrs("peer", n), zap.Error(err))
}
peerIDs = append(peerIDs, pID)
}
if !w.opts.noDefaultWakuTopic {
w.wg.Add(1)
go func() {
defer w.wg.Done()
ctxWithTimeout, ctxCancel := context.WithTimeout(ctx, 20*time.Second)
defer ctxCancel()
if _, err := w.store.(store.Store).Resume(ctxWithTimeout, string(relay.DefaultWakuTopic), peerIDs); err != nil {
w.log.Error("Could not resume history", zap.Error(err))
time.Sleep(10 * time.Second)
}
}()
}
2021-08-13 11:56:09 +00:00
}
2022-12-09 03:08:04 +00:00
return nil
}
2023-02-16 16:17:52 +00:00
func (w *WakuNode) addPeer(info *peer.AddrInfo, protocols ...protocol.ID) error {
2022-05-27 13:25:06 +00:00
w.log.Info("adding peer to peerstore", logging.HostID("peer", info.ID))
w.host.Peerstore().AddAddrs(info.ID, info.Addrs, peerstore.PermanentAddrTTL)
err := w.host.Peerstore().AddProtocols(info.ID, protocols...)
2021-10-16 21:50:49 +00:00
if err != nil {
return err
}
2021-06-10 12:59:51 +00:00
2021-10-16 21:50:49 +00:00
return nil
2021-08-31 18:19:49 +00:00
}
2021-06-10 12:59:51 +00:00
// AddPeer is used to add a peer and the protocols it support to the node peerstore
2023-02-16 16:17:52 +00:00
func (w *WakuNode) AddPeer(address ma.Multiaddr, protocols ...protocol.ID) (peer.ID, error) {
info, err := peer.AddrInfoFromP2pAddr(address)
if err != nil {
2022-11-25 20:54:11 +00:00
return "", err
}
2021-06-10 12:59:51 +00:00
2022-11-25 20:54:11 +00:00
return info.ID, w.addPeer(info, protocols...)
2021-06-28 14:14:28 +00:00
}
// DialPeerWithMultiAddress is used to connect to a peer using a multiaddress
func (w *WakuNode) DialPeerWithMultiAddress(ctx context.Context, address ma.Multiaddr) error {
info, err := peer.AddrInfoFromP2pAddr(address)
if err != nil {
return err
}
return w.connect(ctx, *info)
}
// DialPeer is used to connect to a peer using a string containing a multiaddress
func (w *WakuNode) DialPeer(ctx context.Context, address string) error {
p, err := ma.NewMultiaddr(address)
if err != nil {
return err
}
info, err := peer.AddrInfoFromP2pAddr(p)
if err != nil {
return err
}
return w.connect(ctx, *info)
}
func (w *WakuNode) connect(ctx context.Context, info peer.AddrInfo) error {
err := w.host.Connect(ctx, info)
if err != nil {
return err
}
2021-10-16 21:50:49 +00:00
stats.Record(ctx, metrics.Dials.M(1))
return nil
}
// DialPeerByID is used to connect to an already known peer
func (w *WakuNode) DialPeerByID(ctx context.Context, peerID peer.ID) error {
2021-08-31 19:17:56 +00:00
info := w.host.Peerstore().PeerInfo(peerID)
return w.connect(ctx, info)
2021-08-31 19:17:56 +00:00
}
// ClosePeerByAddress is used to disconnect from a peer using its multiaddress
func (w *WakuNode) ClosePeerByAddress(address string) error {
p, err := ma.NewMultiaddr(address)
if err != nil {
return err
}
// Extract the peer ID from the multiaddr.
info, err := peer.AddrInfoFromP2pAddr(p)
if err != nil {
return err
}
return w.ClosePeerById(info.ID)
}
// ClosePeerById is used to close a connection to a peer
func (w *WakuNode) ClosePeerById(id peer.ID) error {
err := w.host.Network().ClosePeer(id)
if err != nil {
return err
}
2021-09-06 13:34:58 +00:00
return nil
}
// PeerCount return the number of connected peers
func (w *WakuNode) PeerCount() int {
return len(w.host.Network().Peers())
}
// PeerStats returns a list of peers and the protocols supported by them
2021-11-10 13:36:51 +00:00
func (w *WakuNode) PeerStats() PeerStats {
p := make(PeerStats)
for _, peerID := range w.host.Network().Peers() {
protocols, err := w.host.Peerstore().GetProtocols(peerID)
if err != nil {
continue
}
p[peerID] = protocols
}
return p
}
2022-12-09 18:09:06 +00:00
// Set the bootnodes on discv5
func (w *WakuNode) SetDiscV5Bootnodes(nodes []*enode.Node) error {
w.opts.discV5bootnodes = nodes
2023-01-06 22:37:57 +00:00
return w.DiscV5().SetBootnodes(nodes)
2022-12-09 18:09:06 +00:00
}
// Peers return the list of peers, addresses, protocols supported and connection status
2021-11-10 13:36:51 +00:00
func (w *WakuNode) Peers() ([]*Peer, error) {
var peers []*Peer
for _, peerId := range w.host.Peerstore().Peers() {
connected := w.host.Network().Connectedness(peerId) == network.Connected
protocols, err := w.host.Peerstore().GetProtocols(peerId)
if err != nil {
return nil, err
}
addrs := w.host.Peerstore().Addrs(peerId)
peers = append(peers, &Peer{
ID: peerId,
Protocols: protocols,
Connected: connected,
Addrs: addrs,
})
}
return peers, nil
}