mirror of
https://github.com/status-im/go-waku.git
synced 2025-01-12 14:54:19 +00:00
feat: e2e reliable chat example POC (#1153)
This commit is contained in:
parent
bc16c74f2e
commit
8ab0764350
@ -86,7 +86,11 @@ func nonRecoverError(err error) error {
|
||||
func Execute(options NodeOptions) error {
|
||||
// Set encoding for logs (console, json, ...)
|
||||
// Note that libp2p reads the encoding from GOLOG_LOG_FMT env var.
|
||||
utils.InitLogger(options.LogEncoding, options.LogOutput, "gowaku")
|
||||
lvl, err := zapcore.ParseLevel(options.LogLevel)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
utils.InitLogger(options.LogEncoding, options.LogOutput, "gowaku", lvl)
|
||||
|
||||
hostAddr, err := net.ResolveTCPAddr("tcp", fmt.Sprintf("%s:%d", options.Address, options.Port))
|
||||
if err != nil {
|
||||
@ -124,11 +128,6 @@ func Execute(options NodeOptions) error {
|
||||
go metricsServer.Start()
|
||||
}
|
||||
|
||||
lvl, err := zapcore.ParseLevel(options.LogLevel)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
nodeOpts := []node.WakuNodeOption{
|
||||
node.WithLogger(logger),
|
||||
node.WithLogLevel(lvl),
|
||||
|
2
examples/chat2-reliable/.gitignore
vendored
Normal file
2
examples/chat2-reliable/.gitignore
vendored
Normal file
@ -0,0 +1,2 @@
|
||||
chat2
|
||||
chat2-reliable
|
9
examples/chat2-reliable/Makefile
Normal file
9
examples/chat2-reliable/Makefile
Normal file
@ -0,0 +1,9 @@
|
||||
.PHONY: all build run
|
||||
|
||||
all: build
|
||||
|
||||
build:
|
||||
go build -o build/chat2-reliable .
|
||||
|
||||
run:
|
||||
./build/chat2-reliable
|
117
examples/chat2-reliable/README.md
Normal file
117
examples/chat2-reliable/README.md
Normal file
@ -0,0 +1,117 @@
|
||||
# chat2-reliable: A Reliable P2P Chat Application
|
||||
|
||||
## Background
|
||||
|
||||
`chat2-reliable` is an enhanced version of a basic command-line chat application that uses the [Waku v2 suite of protocols](https://specs.vac.dev/specs/waku/v2/waku-v2). This version implements an end-to-end reliability protocol to ensure message delivery and causal ordering in a distributed environment.
|
||||
|
||||
## Features
|
||||
|
||||
- P2P chat capabilities using Waku v2 protocols
|
||||
- Implementation of e2e reliability protocol
|
||||
- Support for group chats and direct communication
|
||||
- Scalable to large groups (up to 10K participants)
|
||||
- Transport-agnostic design
|
||||
|
||||
## E2E Reliability Protocol
|
||||
|
||||
The e2e reliability protocol in `chat2-reliable` is an implementation of the proposal at [Vac Forum](https://forum.vac.dev/t/end-to-end-reliability-for-scalable-distributed-logs/293) and includes the following key features:
|
||||
|
||||
1. **Lamport Clocks**: Each participant maintains a Lamport clock for logical timestamping of messages.
|
||||
|
||||
2. **Causal History**: Messages include a short causal history (preceding message IDs) to establish causal relationships.
|
||||
|
||||
3. **Bloom Filters**: A rolling bloom filter is used to track received message IDs and detect duplicates.
|
||||
|
||||
4. **Lazy Pull Mechanism**: Missing messages are requested from peers when causal dependencies are unmet.
|
||||
|
||||
5. **Eager Push Mechanism**: Unacknowledged messages are resent to ensure delivery.
|
||||
|
||||
## Usage
|
||||
|
||||
### Building the Application
|
||||
|
||||
```
|
||||
make
|
||||
```
|
||||
|
||||
### Starting the Application
|
||||
|
||||
Basic usage:
|
||||
```
|
||||
./build/chat2-reliable
|
||||
```
|
||||
|
||||
With custom DNS server:
|
||||
```
|
||||
./build/chat2-reliable --dns-discovery-name-server 8.8.8.8
|
||||
```
|
||||
|
||||
### In-chat Commands
|
||||
|
||||
- `/help`: Display available commands
|
||||
- `/connect`: Interactively connect to a new peer
|
||||
- `/peers`: Display the list of connected peers
|
||||
|
||||
Example:
|
||||
```
|
||||
/connect /ip4/127.0.0.1/tcp/58426/p2p/16Uiu5rGt2QDLmPKas9zpsBgtr5kRzk473s9wkKSWoYwfcY4Hco33
|
||||
```
|
||||
|
||||
## Message Format
|
||||
|
||||
Messages in `chat2-reliable` use the following protobuf format:
|
||||
|
||||
```protobuf
|
||||
message Message {
|
||||
string sender_id = 1;
|
||||
string message_id = 2;
|
||||
int32 lamport_timestamp = 3;
|
||||
repeated string causal_history = 4;
|
||||
string channel_id = 5;
|
||||
bytes bloom_filter = 6;
|
||||
string content = 7;
|
||||
}
|
||||
```
|
||||
|
||||
## Implementation Details
|
||||
|
||||
1. **Lamport Clocks**: Implemented in the `Chat` struct with methods to increment, update, and retrieve the timestamp.
|
||||
|
||||
2. **Causal History**: Stored in the `CausalHistory` field of each message, containing IDs of recent preceding messages.
|
||||
|
||||
3. **Bloom Filters**: Implemented as a `RollingBloomFilter` to efficiently track received messages and detect duplicates.
|
||||
|
||||
4. **Message Processing**:
|
||||
- Incoming messages are checked against the bloom filter for duplicates.
|
||||
- Causal dependencies are verified before processing.
|
||||
- Messages with unmet dependencies are stored in an incoming buffer.
|
||||
|
||||
5. **Message Recovery**:
|
||||
- Missing messages are requested from peers.
|
||||
- A retry mechanism with exponential backoff is implemented for failed retrievals.
|
||||
|
||||
6. **Conflict Resolution**:
|
||||
- Messages are ordered based on Lamport timestamps and message IDs for consistency.
|
||||
|
||||
7. **Periodic Tasks**:
|
||||
- Buffer sweeps to process buffered messages and resend unacknowledged ones.
|
||||
- Sync messages to maintain consistency across peers.
|
||||
|
||||
## Testing
|
||||
|
||||
The implementation includes various tests to ensure the reliability features work as expected:
|
||||
|
||||
- Lamport timestamp correctness
|
||||
- Causal ordering of messages
|
||||
- Duplicate detection using bloom filters
|
||||
- Message recovery after network partitions
|
||||
- Concurrent message sending
|
||||
- Large group scaling
|
||||
- Eager push mechanism effectiveness
|
||||
- Bloom filter window functionality
|
||||
- Conflict resolution
|
||||
- New node synchronization
|
||||
|
||||
```
|
||||
go test -v
|
||||
```
|
2
examples/chat2-reliable/build/.gitignore
vendored
Normal file
2
examples/chat2-reliable/build/.gitignore
vendored
Normal file
@ -0,0 +1,2 @@
|
||||
*
|
||||
!.gitignore
|
545
examples/chat2-reliable/chat.go
Normal file
545
examples/chat2-reliable/chat.go
Normal file
@ -0,0 +1,545 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"chat2-reliable/pb"
|
||||
"context"
|
||||
"encoding/hex"
|
||||
"errors"
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/multiformats/go-multiaddr"
|
||||
"github.com/waku-org/go-waku/waku/v2/dnsdisc"
|
||||
"github.com/waku-org/go-waku/waku/v2/node"
|
||||
"github.com/waku-org/go-waku/waku/v2/payload"
|
||||
"github.com/waku-org/go-waku/waku/v2/protocol"
|
||||
"github.com/waku-org/go-waku/waku/v2/protocol/filter"
|
||||
"github.com/waku-org/go-waku/waku/v2/protocol/lightpush"
|
||||
wpb "github.com/waku-org/go-waku/waku/v2/protocol/pb"
|
||||
"github.com/waku-org/go-waku/waku/v2/protocol/relay"
|
||||
wrln "github.com/waku-org/go-waku/waku/v2/protocol/rln"
|
||||
"github.com/waku-org/go-waku/waku/v2/protocol/store"
|
||||
"github.com/waku-org/go-waku/waku/v2/utils"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
const (
|
||||
maxMessageHistory = 100
|
||||
)
|
||||
|
||||
type Chat struct {
|
||||
ctx context.Context
|
||||
wg sync.WaitGroup
|
||||
node *node.WakuNode
|
||||
ui UI
|
||||
uiReady chan struct{}
|
||||
inputChan chan string
|
||||
options Options
|
||||
C chan *protocol.Envelope
|
||||
nick string
|
||||
lamportTimestamp int32
|
||||
bloomFilter *RollingBloomFilter
|
||||
outgoingBuffer []UnacknowledgedMessage
|
||||
incomingBuffer []*pb.Message
|
||||
messageHistory []*pb.Message
|
||||
mutex sync.Mutex
|
||||
lamportTSMutex sync.Mutex
|
||||
}
|
||||
|
||||
func NewChat(ctx context.Context, node *node.WakuNode, connNotifier <-chan node.PeerConnection, options Options) *Chat {
|
||||
chat := &Chat{
|
||||
ctx: ctx,
|
||||
node: node,
|
||||
options: options,
|
||||
nick: options.Nickname,
|
||||
uiReady: make(chan struct{}, 1),
|
||||
inputChan: make(chan string, 100),
|
||||
lamportTimestamp: 0,
|
||||
bloomFilter: NewRollingBloomFilter(),
|
||||
outgoingBuffer: make([]UnacknowledgedMessage, 0),
|
||||
incomingBuffer: make([]*pb.Message, 0),
|
||||
messageHistory: make([]*pb.Message, 0),
|
||||
mutex: sync.Mutex{},
|
||||
lamportTSMutex: sync.Mutex{},
|
||||
}
|
||||
|
||||
chat.ui = NewUIModel(chat.uiReady, chat.inputChan)
|
||||
|
||||
topics := options.Relay.Topics.Value()
|
||||
if len(topics) == 0 {
|
||||
topics = append(topics, relay.DefaultWakuTopic)
|
||||
}
|
||||
|
||||
if options.Filter.Enable {
|
||||
cf := protocol.ContentFilter{
|
||||
PubsubTopic: relay.DefaultWakuTopic,
|
||||
ContentTopics: protocol.NewContentTopicSet(options.ContentTopic),
|
||||
}
|
||||
var filterOpt filter.FilterSubscribeOption
|
||||
peerID, err := options.Filter.NodePeerID()
|
||||
if err != nil {
|
||||
filterOpt = filter.WithAutomaticPeerSelection()
|
||||
} else {
|
||||
filterOpt = filter.WithPeer(peerID)
|
||||
chat.ui.InfoMessage(fmt.Sprintf("Subscribing to filter node %s", peerID))
|
||||
}
|
||||
theFilters, err := node.FilterLightnode().Subscribe(ctx, cf, filterOpt)
|
||||
if err != nil {
|
||||
chat.ui.ErrorMessage(err)
|
||||
} else {
|
||||
chat.C = theFilters[0].C // Picking first subscription since there is only 1 contentTopic specified.
|
||||
}
|
||||
} else {
|
||||
for _, topic := range topics {
|
||||
sub, err := node.Relay().Subscribe(ctx, protocol.NewContentFilter(topic))
|
||||
if err != nil {
|
||||
chat.ui.ErrorMessage(err)
|
||||
} else {
|
||||
chat.C = make(chan *protocol.Envelope)
|
||||
go func() {
|
||||
for e := range sub[0].Ch {
|
||||
chat.C <- e
|
||||
}
|
||||
}()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
connWg := sync.WaitGroup{}
|
||||
connWg.Add(2)
|
||||
|
||||
chat.wg.Add(7) // Added 2 more goroutines for periodic tasks
|
||||
go chat.parseInput()
|
||||
go chat.receiveMessages()
|
||||
go chat.welcomeMessage()
|
||||
go chat.connectionWatcher(connNotifier)
|
||||
go chat.staticNodes(&connWg)
|
||||
go chat.discoverNodes(&connWg)
|
||||
go chat.retrieveHistory(&connWg)
|
||||
|
||||
chat.initReliabilityProtocol() // Initialize the reliability protocol
|
||||
|
||||
return chat
|
||||
}
|
||||
|
||||
func (c *Chat) Stop() {
|
||||
c.wg.Wait()
|
||||
close(c.inputChan)
|
||||
}
|
||||
|
||||
func (c *Chat) connectionWatcher(connNotifier <-chan node.PeerConnection) {
|
||||
defer c.wg.Done()
|
||||
for {
|
||||
select {
|
||||
case conn := <-connNotifier:
|
||||
if conn.Connected {
|
||||
c.ui.InfoMessage(fmt.Sprintf("Peer %s connected", conn.PeerID.String()))
|
||||
} else {
|
||||
c.ui.InfoMessage(fmt.Sprintf("Peer %s disconnected", conn.PeerID.String()))
|
||||
}
|
||||
case <-c.ctx.Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) receiveMessages() {
|
||||
defer c.wg.Done()
|
||||
for {
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
return
|
||||
case value := <-c.C:
|
||||
msgContentTopic := value.Message().ContentTopic
|
||||
if msgContentTopic != c.options.ContentTopic {
|
||||
continue // Discard messages from other topics
|
||||
}
|
||||
|
||||
msg, err := decodeMessage(c.options.ContentTopic, value.Message())
|
||||
if err != nil {
|
||||
fmt.Printf("Error decoding message: %v\n", err)
|
||||
continue
|
||||
}
|
||||
|
||||
c.processReceivedMessage(msg)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) parseInput() {
|
||||
defer c.wg.Done()
|
||||
for {
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
return
|
||||
case line := <-c.inputChan:
|
||||
c.ui.SetSending(true)
|
||||
go func() {
|
||||
defer c.ui.SetSending(false)
|
||||
|
||||
// bail if requested
|
||||
if line == "/exit" {
|
||||
c.ui.Quit()
|
||||
fmt.Println("Bye!")
|
||||
return
|
||||
}
|
||||
|
||||
// add peer
|
||||
if strings.HasPrefix(line, "/connect") {
|
||||
peer := strings.TrimPrefix(line, "/connect ")
|
||||
c.wg.Add(1)
|
||||
go func(peer string) {
|
||||
defer c.wg.Done()
|
||||
|
||||
ma, err := multiaddr.NewMultiaddr(peer)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
return
|
||||
}
|
||||
|
||||
peerID, err := ma.ValueForProtocol(multiaddr.P_P2P)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
return
|
||||
}
|
||||
|
||||
c.ui.InfoMessage(fmt.Sprintf("Connecting to peer: %s", peerID))
|
||||
ctx, cancel := context.WithTimeout(c.ctx, time.Duration(10)*time.Second)
|
||||
defer cancel()
|
||||
|
||||
err = c.node.DialPeerWithMultiAddress(ctx, ma)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
}
|
||||
}(peer)
|
||||
return
|
||||
}
|
||||
|
||||
// list peers
|
||||
if line == "/peers" {
|
||||
peers := c.node.Host().Network().Peers()
|
||||
if len(peers) == 0 {
|
||||
c.ui.InfoMessage("No peers available")
|
||||
} else {
|
||||
peerInfoMsg := "Peers: \n"
|
||||
for _, p := range peers {
|
||||
peerInfo := c.node.Host().Peerstore().PeerInfo(p)
|
||||
peerProtocols, err := c.node.Host().Peerstore().GetProtocols(p)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
return
|
||||
}
|
||||
peerInfoMsg += fmt.Sprintf("• %s:\n", p.String())
|
||||
|
||||
var strProtocols []string
|
||||
for _, p := range peerProtocols {
|
||||
strProtocols = append(strProtocols, string(p))
|
||||
}
|
||||
|
||||
peerInfoMsg += fmt.Sprintf(" Protocols: %s\n", strings.Join(strProtocols, ", "))
|
||||
peerInfoMsg += " Addresses:\n"
|
||||
for _, addr := range peerInfo.Addrs {
|
||||
peerInfoMsg += fmt.Sprintf(" - %s/p2p/%s\n", addr.String(), p.String())
|
||||
}
|
||||
}
|
||||
c.ui.InfoMessage(peerInfoMsg)
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// change nick
|
||||
if strings.HasPrefix(line, "/nick") {
|
||||
newNick := strings.TrimSpace(strings.TrimPrefix(line, "/nick "))
|
||||
if newNick != "" {
|
||||
c.nick = newNick
|
||||
} else {
|
||||
c.ui.ErrorMessage(errors.New("invalid nickname"))
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
if line == "/help" {
|
||||
c.ui.InfoMessage(`Available commands:
|
||||
/connect multiaddress - dials a node adding it to the list of connected peers
|
||||
/peers - list of peers connected to this node
|
||||
/nick newNick - change the user's nickname
|
||||
/exit - closes the app`)
|
||||
return
|
||||
}
|
||||
|
||||
c.SendMessage(line)
|
||||
}()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) publish(ctx context.Context, message *pb.Message) error {
|
||||
msgBytes, err := proto.Marshal(message)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
version := uint32(0)
|
||||
timestamp := utils.GetUnixEpochFrom(c.node.Timesource().Now())
|
||||
keyInfo := &payload.KeyInfo{
|
||||
Kind: payload.None,
|
||||
}
|
||||
|
||||
p := new(payload.Payload)
|
||||
p.Data = msgBytes
|
||||
p.Key = keyInfo
|
||||
|
||||
payload, err := p.Encode(version)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
wakuMsg := &wpb.WakuMessage{
|
||||
Payload: payload,
|
||||
Version: proto.Uint32(version),
|
||||
ContentTopic: c.options.ContentTopic,
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
|
||||
if c.options.RLNRelay.Enable {
|
||||
err = c.node.RLNRelay().AppendRLNProof(wakuMsg, c.node.Timesource().Now())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
rateLimitProof, err := wrln.BytesToRateLimitProof(wakuMsg.RateLimitProof)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
c.ui.InfoMessage(fmt.Sprintf("RLN Epoch: %d", rateLimitProof.Epoch.Uint64()))
|
||||
}
|
||||
|
||||
if c.options.LightPush.Enable {
|
||||
lightOpt := []lightpush.RequestOption{lightpush.WithDefaultPubsubTopic()}
|
||||
var peerID peer.ID
|
||||
peerID, err = c.options.LightPush.NodePeerID()
|
||||
if err != nil {
|
||||
lightOpt = append(lightOpt, lightpush.WithAutomaticPeerSelection())
|
||||
} else {
|
||||
lightOpt = append(lightOpt, lightpush.WithPeer(peerID))
|
||||
}
|
||||
|
||||
_, err = c.node.Lightpush().Publish(ctx, wakuMsg, lightOpt...)
|
||||
} else {
|
||||
_, err = c.node.Relay().Publish(ctx, wakuMsg, relay.WithDefaultPubsubTopic())
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
func decodeMessage(contentTopic string, wakumsg *wpb.WakuMessage) (*pb.Message, error) {
|
||||
keyInfo := &payload.KeyInfo{
|
||||
Kind: payload.None,
|
||||
}
|
||||
|
||||
payload, err := payload.DecodePayload(wakumsg, keyInfo)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
msg := &pb.Message{}
|
||||
if err := proto.Unmarshal(payload.Data, msg); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return msg, nil
|
||||
}
|
||||
|
||||
func (c *Chat) retrieveHistory(connectionWg *sync.WaitGroup) {
|
||||
defer c.wg.Done()
|
||||
|
||||
connectionWg.Wait() // Wait until node connection operations are
|
||||
|
||||
if !c.options.Store.Enable {
|
||||
return
|
||||
}
|
||||
|
||||
var storeOpt store.RequestOption
|
||||
if c.options.Store.Node == nil {
|
||||
c.ui.InfoMessage("No store node configured. Choosing one at random...")
|
||||
storeOpt = store.WithAutomaticPeerSelection()
|
||||
} else {
|
||||
pID, err := c.getStoreNodePID()
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
return
|
||||
}
|
||||
storeOpt = store.WithPeer(*pID)
|
||||
c.ui.InfoMessage(fmt.Sprintf("Querying historic messages from %s", pID.String()))
|
||||
}
|
||||
|
||||
tCtx, cancel := context.WithTimeout(c.ctx, 10*time.Second)
|
||||
defer cancel()
|
||||
|
||||
q := store.FilterCriteria{
|
||||
ContentFilter: protocol.NewContentFilter(relay.DefaultWakuTopic, c.options.ContentTopic),
|
||||
}
|
||||
|
||||
response, err := c.node.Store().Request(tCtx, q,
|
||||
store.WithAutomaticRequestID(),
|
||||
storeOpt,
|
||||
store.WithPaging(false, 100))
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(fmt.Errorf("could not query storenode: %w", err))
|
||||
} else {
|
||||
if len(response.Messages()) == 0 {
|
||||
c.ui.InfoMessage("0 historic messages available")
|
||||
} else {
|
||||
for _, msg := range response.Messages() {
|
||||
c.C <- protocol.NewEnvelope(msg.Message, msg.Message.GetTimestamp(), relay.DefaultWakuTopic)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) staticNodes(connectionWg *sync.WaitGroup) {
|
||||
defer c.wg.Done()
|
||||
defer connectionWg.Done()
|
||||
|
||||
<-c.uiReady // wait until UI is ready
|
||||
|
||||
wg := sync.WaitGroup{}
|
||||
|
||||
wg.Add(len(c.options.StaticNodes))
|
||||
for _, n := range c.options.StaticNodes {
|
||||
go func(addr multiaddr.Multiaddr) {
|
||||
defer wg.Done()
|
||||
ctx, cancel := context.WithTimeout(c.ctx, time.Duration(10)*time.Second)
|
||||
defer cancel()
|
||||
|
||||
c.ui.InfoMessage(fmt.Sprintf("Connecting to %s", addr.String()))
|
||||
|
||||
err := c.node.DialPeerWithMultiAddress(ctx, addr)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
}
|
||||
}(n)
|
||||
}
|
||||
|
||||
wg.Wait()
|
||||
}
|
||||
|
||||
func (c *Chat) welcomeMessage() {
|
||||
defer c.wg.Done()
|
||||
|
||||
<-c.uiReady // wait until UI is ready
|
||||
|
||||
c.ui.InfoMessage("Welcome, " + c.nick + "!")
|
||||
c.ui.InfoMessage("type /help to see available commands \n")
|
||||
|
||||
addrMessage := "Listening on:\n"
|
||||
for _, addr := range c.node.ListenAddresses() {
|
||||
addrMessage += " -" + addr.String() + "\n"
|
||||
}
|
||||
c.ui.InfoMessage(addrMessage)
|
||||
|
||||
if !c.options.RLNRelay.Enable {
|
||||
return
|
||||
}
|
||||
|
||||
credential, err := c.node.RLNRelay().IdentityCredential()
|
||||
if err != nil {
|
||||
c.ui.Quit()
|
||||
}
|
||||
|
||||
idx := c.node.RLNRelay().MembershipIndex()
|
||||
|
||||
idTrapdoor := credential.IDTrapdoor
|
||||
idNullifier := credential.IDSecretHash
|
||||
idSecretHash := credential.IDSecretHash
|
||||
idCommitment := credential.IDCommitment
|
||||
|
||||
rlnMessage := "RLN config:\n"
|
||||
rlnMessage += fmt.Sprintf("- Your membership index is: %d\n", idx)
|
||||
rlnMessage += fmt.Sprintf("- Your rln identity trapdoor is: 0x%s\n", hex.EncodeToString(idTrapdoor[:]))
|
||||
rlnMessage += fmt.Sprintf("- Your rln identity nullifier is: 0x%s\n", hex.EncodeToString(idNullifier[:]))
|
||||
rlnMessage += fmt.Sprintf("- Your rln identity secret hash is: 0x%s\n", hex.EncodeToString(idSecretHash[:]))
|
||||
rlnMessage += fmt.Sprintf("- Your rln identity commitment key is: 0x%s\n", hex.EncodeToString(idCommitment[:]))
|
||||
|
||||
c.ui.InfoMessage(rlnMessage)
|
||||
}
|
||||
|
||||
func (c *Chat) discoverNodes(connectionWg *sync.WaitGroup) {
|
||||
defer c.wg.Done()
|
||||
defer connectionWg.Done()
|
||||
|
||||
<-c.uiReady // wait until UI is ready
|
||||
|
||||
var dnsDiscoveryUrl string
|
||||
if c.options.DNSDiscovery.Enable {
|
||||
if c.options.Fleet != fleetNone {
|
||||
if c.options.Fleet == fleetTest {
|
||||
dnsDiscoveryUrl = "enrtree://AOGYWMBYOUIMOENHXCHILPKY3ZRFEULMFI4DOM442QSZ73TT2A7VI@test.waku.nodes.status.im"
|
||||
} else {
|
||||
// Connect to prod by default
|
||||
dnsDiscoveryUrl = "enrtree://AIRVQ5DDA4FFWLRBCHJWUWOO6X6S4ZTZ5B667LQ6AJU6PEYDLRD5O@sandbox.waku.nodes.status.im"
|
||||
}
|
||||
}
|
||||
|
||||
if c.options.DNSDiscovery.URL != "" {
|
||||
dnsDiscoveryUrl = c.options.DNSDiscovery.URL
|
||||
}
|
||||
}
|
||||
|
||||
if dnsDiscoveryUrl != "" {
|
||||
c.ui.InfoMessage(fmt.Sprintf("attempting DNS discovery with %s", dnsDiscoveryUrl))
|
||||
nodes, err := dnsdisc.RetrieveNodes(c.ctx, dnsDiscoveryUrl, dnsdisc.WithNameserver(c.options.DNSDiscovery.Nameserver))
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(errors.New(err.Error()))
|
||||
} else {
|
||||
var nodeList []peer.AddrInfo
|
||||
for _, n := range nodes {
|
||||
nodeList = append(nodeList, n.PeerInfo)
|
||||
}
|
||||
c.ui.InfoMessage(fmt.Sprintf("Discovered and connecting to %v ", nodeList))
|
||||
wg := sync.WaitGroup{}
|
||||
wg.Add(len(nodeList))
|
||||
for _, n := range nodeList {
|
||||
go func(ctx context.Context, info peer.AddrInfo) {
|
||||
defer wg.Done()
|
||||
|
||||
ctx, cancel := context.WithTimeout(ctx, time.Duration(20)*time.Second)
|
||||
defer cancel()
|
||||
err = c.node.DialPeerWithInfo(ctx, info)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(fmt.Errorf("could not connect to %s: %w", info.ID.String(), err))
|
||||
}
|
||||
}(c.ctx, n)
|
||||
}
|
||||
wg.Wait()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func generateUniqueID() string {
|
||||
return uuid.New().String()
|
||||
}
|
||||
|
||||
func (c *Chat) getRecentMessageIDs(n int) []string {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
|
||||
result := make([]string, 0, n)
|
||||
for i := len(c.messageHistory) - 1; i >= 0 && len(result) < n; i-- {
|
||||
result = append(result, c.messageHistory[i].MessageId)
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
func (c *Chat) getStoreNodePID() (*peer.ID, error) {
|
||||
pID, err := utils.GetPeerID(*c.options.Store.Node)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &pID, nil
|
||||
}
|
603
examples/chat2-reliable/chat_reliability_test.go
Normal file
603
examples/chat2-reliable/chat_reliability_test.go
Normal file
@ -0,0 +1,603 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"chat2-reliable/pb"
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
"github.com/urfave/cli/v2"
|
||||
"github.com/waku-org/go-waku/waku/v2/node"
|
||||
"github.com/waku-org/go-waku/waku/v2/peerstore"
|
||||
"github.com/waku-org/go-waku/waku/v2/protocol/relay"
|
||||
)
|
||||
|
||||
type TestEnvironment struct {
|
||||
nodes []*node.WakuNode
|
||||
chats []*Chat
|
||||
}
|
||||
|
||||
func setupTestEnvironment(ctx context.Context, t *testing.T, nodeCount int) (*TestEnvironment, error) {
|
||||
t.Logf("Setting up test environment with %d nodes", nodeCount)
|
||||
env := &TestEnvironment{
|
||||
nodes: make([]*node.WakuNode, nodeCount),
|
||||
chats: make([]*Chat, nodeCount),
|
||||
}
|
||||
|
||||
for i := 0; i < nodeCount; i++ {
|
||||
node, err := setupTestNode(ctx, t)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to set up node %d: %w", i, err)
|
||||
}
|
||||
env.nodes[i] = node
|
||||
|
||||
chat, err := setupTestChat(ctx, node, fmt.Sprintf("Node%d", i))
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to set up chat for node %d: %w", i, err)
|
||||
}
|
||||
env.chats[i] = chat
|
||||
}
|
||||
|
||||
t.Log("Connecting nodes in ring topology")
|
||||
for i := 0; i < nodeCount; i++ {
|
||||
nextIndex := (i + 1) % nodeCount
|
||||
_, err := env.nodes[i].AddPeer(env.nodes[nextIndex].ListenAddresses()[0], peerstore.Static, env.chats[i].options.Relay.Topics.Value())
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to connect node %d to node %d: %w", i, nextIndex, err)
|
||||
}
|
||||
}
|
||||
|
||||
t.Log("Test environment setup complete")
|
||||
return env, nil
|
||||
}
|
||||
|
||||
func setupTestNode(ctx context.Context, t *testing.T) (*node.WakuNode, error) {
|
||||
opts := []node.WakuNodeOption{
|
||||
node.WithWakuRelay(),
|
||||
// node.WithWakuStore(),
|
||||
}
|
||||
node, err := node.New(opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := node.Start(ctx); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// if node.Store() == nil {
|
||||
// t.Logf("Store protocol is not enabled on node %d", index)
|
||||
// }
|
||||
|
||||
return node, nil
|
||||
}
|
||||
|
||||
type PeerConnection = node.PeerConnection
|
||||
|
||||
func setupTestChat(ctx context.Context, node *node.WakuNode, nickname string) (*Chat, error) {
|
||||
topics := cli.StringSlice{}
|
||||
topics.Set(relay.DefaultWakuTopic)
|
||||
|
||||
options := Options{
|
||||
Nickname: nickname,
|
||||
ContentTopic: "/test/1/chat/proto",
|
||||
Relay: RelayOptions{
|
||||
Enable: true,
|
||||
Topics: topics,
|
||||
},
|
||||
}
|
||||
|
||||
// Create a channel of the correct type
|
||||
connNotifier := make(chan PeerConnection)
|
||||
|
||||
chat := NewChat(ctx, node, connNotifier, options)
|
||||
if chat == nil {
|
||||
return nil, fmt.Errorf("failed to create chat instance")
|
||||
}
|
||||
return chat, nil
|
||||
}
|
||||
|
||||
func areNodesConnected(nodes []*node.WakuNode, expectedPeers int) bool {
|
||||
for _, node := range nodes {
|
||||
if len(node.Host().Network().Peers()) != expectedPeers {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}
|
||||
|
||||
// TestLamportTimestamps verifies that Lamport timestamps are correctly updated
|
||||
func TestLamportTimestamps(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
t.Log("Starting TestLamportTimestamps")
|
||||
|
||||
nodeCount := 3
|
||||
env, err := setupTestEnvironment(ctx, t, nodeCount)
|
||||
require.NoError(t, err, "Failed to set up test environment")
|
||||
|
||||
require.Eventually(t, func() bool {
|
||||
return areNodesConnected(env.nodes, 2)
|
||||
}, 30*time.Second, 1*time.Second, "Nodes failed to connect")
|
||||
|
||||
for i, chat := range env.chats {
|
||||
t.Logf("Node %d initial Lamport timestamp: %d", i, chat.getLamportTimestamp())
|
||||
}
|
||||
|
||||
t.Log("Sending message from Node 0")
|
||||
env.chats[0].SendMessage("Message from Node 0")
|
||||
|
||||
t.Log("Waiting for message propagation")
|
||||
require.Eventually(t, func() bool {
|
||||
for _, chat := range env.chats {
|
||||
if chat.getLamportTimestamp() == 0 {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}, 30*time.Second, 1*time.Second, "Message propagation failed")
|
||||
|
||||
assert.Equal(t, int32(1), env.chats[0].getLamportTimestamp(), "Sender's Lamport timestamp should be 1")
|
||||
assert.Greater(t, env.chats[1].getLamportTimestamp(), int32(0), "Node 1's Lamport timestamp should be greater than 0")
|
||||
assert.Greater(t, env.chats[2].getLamportTimestamp(), int32(0), "Node 2's Lamport timestamp should be greater than 0")
|
||||
|
||||
assert.NotEmpty(t, env.chats[1].messageHistory, "Node 1 should have received the message")
|
||||
assert.NotEmpty(t, env.chats[2].messageHistory, "Node 2 should have received the message")
|
||||
|
||||
if len(env.chats[1].messageHistory) > 0 {
|
||||
assert.Equal(t, "Message from Node 0", env.chats[1].messageHistory[0].Content, "Node 1 should have received the correct message")
|
||||
}
|
||||
if len(env.chats[2].messageHistory) > 0 {
|
||||
assert.Equal(t, "Message from Node 0", env.chats[2].messageHistory[0].Content, "Node 2 should have received the correct message")
|
||||
}
|
||||
|
||||
t.Log("TestLamportTimestamps completed successfully")
|
||||
}
|
||||
|
||||
// TestCausalOrdering ensures messages are processed in the correct causal order
|
||||
func TestCausalOrdering(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
t.Log("Starting TestCausalOrdering")
|
||||
|
||||
nodeCount := 3
|
||||
env, err := setupTestEnvironment(ctx, t, nodeCount)
|
||||
require.NoError(t, err, "Failed to set up test environment")
|
||||
|
||||
require.Eventually(t, func() bool {
|
||||
return areNodesConnected(env.nodes, 2)
|
||||
}, 30*time.Second, 1*time.Second, "Nodes failed to connect")
|
||||
|
||||
t.Log("Sending messages from different nodes")
|
||||
env.chats[0].SendMessage("Message 1 from Node 0")
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
env.chats[1].SendMessage("Message 2 from Node 1")
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
env.chats[2].SendMessage("Message 3 from Node 2")
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
t.Log("Waiting for message propagation")
|
||||
require.Eventually(t, func() bool {
|
||||
for i, chat := range env.chats {
|
||||
t.Logf("Node %d message history length: %d", i, len(chat.messageHistory))
|
||||
if len(chat.messageHistory) != 3 {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}, 30*time.Second, 1*time.Second, "Messages did not propagate to all nodes")
|
||||
|
||||
for i, chat := range env.chats {
|
||||
assert.Len(t, chat.messageHistory, 3, "Node %d should have 3 messages", i)
|
||||
assert.Equal(t, "Message 1 from Node 0", chat.messageHistory[0].Content, "Node %d: First message incorrect", i)
|
||||
assert.Equal(t, "Message 2 from Node 1", chat.messageHistory[1].Content, "Node %d: Second message incorrect", i)
|
||||
assert.Equal(t, "Message 3 from Node 2", chat.messageHistory[2].Content, "Node %d: Third message incorrect", i)
|
||||
}
|
||||
|
||||
t.Log("TestCausalOrdering completed successfully")
|
||||
}
|
||||
|
||||
func TestBloomFilterDuplicateDetection(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
t.Log("Starting TestBloomFilterDuplicateDetection")
|
||||
|
||||
nodeCount := 2
|
||||
env, err := setupTestEnvironment(ctx, t, nodeCount)
|
||||
require.NoError(t, err, "Failed to set up test environment")
|
||||
|
||||
require.Eventually(t, func() bool {
|
||||
return areNodesConnected(env.nodes, 1)
|
||||
}, 30*time.Second, 1*time.Second, "Nodes failed to connect")
|
||||
|
||||
t.Log("Sending a message")
|
||||
testMessage := "Test message"
|
||||
env.chats[0].SendMessage(testMessage)
|
||||
|
||||
t.Log("Waiting for message propagation")
|
||||
var receivedMsg *pb.Message
|
||||
require.Eventually(t, func() bool {
|
||||
if len(env.chats[1].messageHistory) == 1 {
|
||||
receivedMsg = env.chats[1].messageHistory[0]
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}, 30*time.Second, 1*time.Second, "Message did not propagate to second node")
|
||||
|
||||
require.NotNil(t, receivedMsg, "Received message should not be nil")
|
||||
|
||||
t.Log("Simulating receiving the same message again")
|
||||
|
||||
// Create a duplicate message
|
||||
duplicateMsg := &pb.Message{
|
||||
SenderId: receivedMsg.SenderId,
|
||||
MessageId: receivedMsg.MessageId, // Use the same MessageId to simulate a true duplicate
|
||||
LamportTimestamp: receivedMsg.LamportTimestamp,
|
||||
CausalHistory: receivedMsg.CausalHistory,
|
||||
ChannelId: receivedMsg.ChannelId,
|
||||
BloomFilter: receivedMsg.BloomFilter,
|
||||
Content: receivedMsg.Content,
|
||||
}
|
||||
|
||||
env.chats[1].processReceivedMessage(duplicateMsg)
|
||||
|
||||
assert.Len(t, env.chats[1].messageHistory, 1, "Node 1 should still have only one message (no duplicates)")
|
||||
|
||||
t.Log("TestBloomFilterDuplicateDetection completed successfully")
|
||||
}
|
||||
|
||||
// TestNetworkPartition ensures that missing messages can be recovered
|
||||
func TestNetworkPartition(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 3*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
t.Log("Starting TestMessageRecovery")
|
||||
|
||||
nodeCount := 3
|
||||
env, err := setupTestEnvironment(ctx, t, nodeCount)
|
||||
require.NoError(t, err, "Failed to set up test environment")
|
||||
|
||||
nc := NewNetworkController(ctx, env.nodes, env.chats)
|
||||
|
||||
require.Eventually(t, func() bool {
|
||||
return areNodesConnected(env.nodes, 2)
|
||||
}, 60*time.Second, 1*time.Second, "Nodes failed to connect")
|
||||
|
||||
t.Log("Stage 1: Sending initial messages")
|
||||
env.chats[0].SendMessage("Message 1")
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
env.chats[1].SendMessage("Message 2")
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
t.Log("Waiting for message propagation")
|
||||
require.Eventually(t, func() bool {
|
||||
for _, chat := range env.chats {
|
||||
if len(chat.messageHistory) != 2 {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}, 30*time.Second, 1*time.Second, "Messages did not propagate to all nodes")
|
||||
|
||||
// Verify that Node 2 has messages before disconnection
|
||||
require.Equal(t, 2, len(env.chats[2].messageHistory), "Node 2 does not have all messages")
|
||||
|
||||
t.Log("Stage 2: Simulating network partition for Node 2")
|
||||
nc.DisconnectNode(env.nodes[2])
|
||||
time.Sleep(1 * time.Second) // Allow time for disconnection to take effect
|
||||
|
||||
t.Log("Stage 3: Sending message that Node 2 will miss")
|
||||
env.chats[0].SendMessage("Missed Message")
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
|
||||
t.Log("Stage 4: Reconnecting Node 2")
|
||||
nc.ReconnectNode(env.nodes[2])
|
||||
time.Sleep(5 * time.Second) // Allow time for reconnection to take effect
|
||||
|
||||
// Verify that Node 2 didn't receive the message
|
||||
require.Equal(t, 2, len(env.chats[2].messageHistory), "Node 2 should not have received the missed message")
|
||||
|
||||
t.Log("Stage 5: Sending a new message that depends on the missed message")
|
||||
env.chats[1].SendMessage("New Message")
|
||||
|
||||
// Verify that Node 2 received the new message
|
||||
require.Eventually(t, func() bool {
|
||||
msgCount := len(env.chats[2].messageHistory)
|
||||
return msgCount >= 3
|
||||
}, 30*time.Second, 5*time.Second, "Node 2 should have received the new message")
|
||||
|
||||
// Stage 6: Wait for message recovery
|
||||
t.Log("Stage 6: Waiting for message recovery")
|
||||
require.Eventually(t, func() bool {
|
||||
msgCount := len(env.chats[2].messageHistory)
|
||||
return msgCount == 4
|
||||
}, 30*time.Second, 5*time.Second, "Message recovery failed")
|
||||
|
||||
// Print final message history for all nodes
|
||||
for i, chat := range env.chats {
|
||||
t.Logf("Node %d final message history:", i)
|
||||
for j, msg := range chat.messageHistory {
|
||||
t.Logf(" Message %d: %s", j+1, msg.Content)
|
||||
}
|
||||
}
|
||||
|
||||
// Verify the results
|
||||
for i, msg := range env.chats[2].messageHistory {
|
||||
t.Logf("Message %d: %s", i+1, msg.Content)
|
||||
}
|
||||
|
||||
assert.Equal(t, "Message 1", env.chats[2].messageHistory[0].Content, "First message incorrect")
|
||||
assert.Equal(t, "Message 2", env.chats[2].messageHistory[1].Content, "Second message incorrect")
|
||||
assert.Equal(t, "Missed Message", env.chats[2].messageHistory[2].Content, "Missed message not recovered")
|
||||
assert.Equal(t, "New Message", env.chats[2].messageHistory[3].Content, "New message incorrect")
|
||||
}
|
||||
|
||||
func TestConcurrentMessageSending(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 3*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
t.Log("Starting TestConcurrentMessageSending")
|
||||
|
||||
nodeCount := 5
|
||||
env, err := setupTestEnvironment(ctx, t, nodeCount)
|
||||
require.NoError(t, err, "Failed to set up test environment")
|
||||
|
||||
require.Eventually(t, func() bool {
|
||||
return areNodesConnected(env.nodes, 2)
|
||||
}, 60*time.Second, 3*time.Second, "Nodes failed to connect")
|
||||
|
||||
messageCount := 10
|
||||
var wg sync.WaitGroup
|
||||
|
||||
t.Log("Sending messages concurrently")
|
||||
for i := 0; i < len(env.chats); i++ {
|
||||
wg.Add(1)
|
||||
go func(index int) {
|
||||
defer wg.Done()
|
||||
for j := 0; j < messageCount; j++ {
|
||||
env.chats[index].SendMessage(fmt.Sprintf("Message %d from Node %d", j, index))
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
}
|
||||
}(i)
|
||||
}
|
||||
|
||||
wg.Wait()
|
||||
|
||||
t.Log("Waiting for message propagation")
|
||||
totalExpectedMessages := len(env.chats) * messageCount
|
||||
require.Eventually(t, func() bool {
|
||||
for _, chat := range env.chats {
|
||||
if len(chat.messageHistory) != totalExpectedMessages {
|
||||
return false
|
||||
}
|
||||
}
|
||||
return true
|
||||
}, 2*time.Minute, 1*time.Second, "Messages did not propagate to all nodes")
|
||||
|
||||
for i, chat := range env.chats {
|
||||
assert.Len(t, chat.messageHistory, totalExpectedMessages, "Node %d should have received all messages", i)
|
||||
}
|
||||
|
||||
t.Log("TestConcurrentMessageSending completed successfully")
|
||||
}
|
||||
|
||||
func TestLargeGroupScaling(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
t.Log("Starting TestLargeGroupScaling")
|
||||
|
||||
nodeCount := 20
|
||||
env, err := setupTestEnvironment(ctx, t, nodeCount)
|
||||
require.NoError(t, err, "Failed to set up test environment")
|
||||
|
||||
require.Eventually(t, func() bool {
|
||||
return areNodesConnected(env.nodes, 2)
|
||||
}, 2*time.Minute, 3*time.Second, "Nodes failed to connect")
|
||||
|
||||
// Send a message from the first node
|
||||
env.chats[0].SendMessage("Broadcast message to large group")
|
||||
|
||||
// Allow time for propagation
|
||||
time.Sleep(time.Duration(nodeCount*100) * time.Millisecond)
|
||||
|
||||
// Verify all nodes received the message
|
||||
for i, chat := range env.chats {
|
||||
assert.Len(t, chat.messageHistory, 1, "Node %d should have received the broadcast message", i)
|
||||
assert.Equal(t, "Broadcast message to large group", chat.messageHistory[0].Content)
|
||||
}
|
||||
|
||||
t.Log("TestLargeGroupScaling completed successfully")
|
||||
}
|
||||
|
||||
func TestEagerPushMechanism(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
nodeCount := 2
|
||||
env, err := setupTestEnvironment(ctx, t, nodeCount)
|
||||
require.NoError(t, err, "Failed to set up test environment")
|
||||
|
||||
nc := NewNetworkController(ctx, env.nodes, env.chats)
|
||||
|
||||
// Disconnect node 1
|
||||
nc.DisconnectNode(env.nodes[1])
|
||||
|
||||
// Send a message from node 0
|
||||
env.chats[0].SendMessage("Test eager push")
|
||||
|
||||
// Wait for the message to be added to the outgoing buffer
|
||||
time.Sleep(1 * time.Second)
|
||||
|
||||
// Reconnect node 1
|
||||
nc.ReconnectNode(env.nodes[1])
|
||||
|
||||
// Wait for eager push to resend the message
|
||||
time.Sleep(5 * time.Second)
|
||||
|
||||
// Check if node 1 received the message
|
||||
assert.Eventually(t, func() bool {
|
||||
return len(env.chats[1].messageHistory) == 1
|
||||
}, 10*time.Second, 1*time.Second, "Node 1 should have received the message via eager push")
|
||||
}
|
||||
|
||||
func TestBloomFilterWindow(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
nodeCount := 2
|
||||
env, err := setupTestEnvironment(ctx, t, nodeCount)
|
||||
require.NoError(t, err, "Failed to set up test environment")
|
||||
|
||||
// Reduce bloom filter window for testing
|
||||
for _, chat := range env.chats {
|
||||
chat.bloomFilter.window = 2 * time.Second
|
||||
}
|
||||
|
||||
// Send a message
|
||||
env.chats[0].SendMessage("Test bloom filter window")
|
||||
messageID := env.chats[0].messageHistory[0].MessageId
|
||||
|
||||
// Check if the message is in the bloom filter
|
||||
assert.Eventually(t, func() bool {
|
||||
return env.chats[1].bloomFilter.Test(messageID)
|
||||
}, 30*time.Second, 1*time.Second, "Message should be in the bloom filter")
|
||||
|
||||
// Wait for the bloom filter window to pass
|
||||
time.Sleep(3 * time.Second)
|
||||
|
||||
// Clean the bloom filter
|
||||
env.chats[1].bloomFilter.Clean()
|
||||
|
||||
time.Sleep(3 * time.Second)
|
||||
|
||||
// Check if the message is no longer in the bloom filter
|
||||
assert.False(t, env.chats[1].bloomFilter.Test(messageID), "Message should no longer be in the bloom filter")
|
||||
|
||||
// Send another message to ensure the filter still works for new messages
|
||||
env.chats[0].SendMessage("New test message")
|
||||
time.Sleep(1 * time.Second)
|
||||
|
||||
newMessageID := env.chats[0].messageHistory[1].MessageId
|
||||
// Check if the new message is in the bloom filter
|
||||
assert.Eventually(t, func() bool {
|
||||
return env.chats[1].bloomFilter.Test(newMessageID)
|
||||
}, 30*time.Second, 1*time.Second, "New message should be in the bloom filter")
|
||||
}
|
||||
|
||||
func TestConflictResolution(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
nodeCount := 3
|
||||
env, err := setupTestEnvironment(ctx, t, nodeCount)
|
||||
require.NoError(t, err, "Failed to set up test environment")
|
||||
|
||||
// Create conflicting messages with the same Lamport timestamp
|
||||
conflictingMsg1 := &pb.Message{
|
||||
SenderId: "Node0",
|
||||
MessageId: "msg1",
|
||||
LamportTimestamp: 1,
|
||||
Content: "Conflict 1",
|
||||
}
|
||||
conflictingMsg2 := &pb.Message{
|
||||
SenderId: "Node1",
|
||||
MessageId: "msg2",
|
||||
LamportTimestamp: 1,
|
||||
Content: "Conflict 2",
|
||||
}
|
||||
|
||||
// Process the conflicting messages in different orders on different nodes
|
||||
env.chats[0].processReceivedMessage(conflictingMsg1)
|
||||
env.chats[0].processReceivedMessage(conflictingMsg2)
|
||||
|
||||
env.chats[1].processReceivedMessage(conflictingMsg2)
|
||||
env.chats[1].processReceivedMessage(conflictingMsg1)
|
||||
|
||||
// Check if the messages are ordered consistently across nodes
|
||||
assert.Equal(t, env.chats[0].messageHistory[0].MessageId, env.chats[1].messageHistory[0].MessageId, "Conflicting messages should be ordered consistently")
|
||||
assert.Equal(t, env.chats[0].messageHistory[1].MessageId, env.chats[1].messageHistory[1].MessageId, "Conflicting messages should be ordered consistently")
|
||||
}
|
||||
|
||||
func TestNewNodeSyncAndMessagePropagation(t *testing.T) {
|
||||
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute)
|
||||
defer cancel()
|
||||
|
||||
t.Log("Starting TestNewNodeSyncAndMessagePropagation")
|
||||
|
||||
// Set up initial network with 2 nodes
|
||||
initialNodeCount := 2
|
||||
env, err := setupTestEnvironment(ctx, t, initialNodeCount)
|
||||
require.NoError(t, err, "Failed to set up initial test environment")
|
||||
|
||||
// Ensure initial nodes are connected
|
||||
require.Eventually(t, func() bool {
|
||||
return areNodesConnected(env.nodes, 1)
|
||||
}, 60*time.Second, 1*time.Second, "Initial nodes failed to connect")
|
||||
|
||||
t.Log("Sending initial messages")
|
||||
env.chats[0].SendMessage("Initial message 1")
|
||||
env.chats[1].SendMessage("Initial message 2")
|
||||
|
||||
// Wait for message propagation
|
||||
time.Sleep(5 * time.Second)
|
||||
|
||||
// Verify initial messages are received by both nodes
|
||||
for i, chat := range env.chats {
|
||||
assert.Len(t, chat.messageHistory, 2, "Node %d should have 2 initial messages", i)
|
||||
}
|
||||
|
||||
t.Log("Adding new node to the network")
|
||||
newNode, err := setupTestNode(ctx, t)
|
||||
require.NoError(t, err, "Failed to set up new node")
|
||||
newChat, err := setupTestChat(ctx, newNode, "NewNode")
|
||||
require.NoError(t, err, "Failed to set up new chat")
|
||||
|
||||
env.nodes = append(env.nodes, newNode)
|
||||
env.chats = append(env.chats, newChat)
|
||||
|
||||
// Connect new node to the network
|
||||
_, err = env.nodes[2].AddPeer(env.nodes[0].ListenAddresses()[0], peerstore.Static, env.chats[2].options.Relay.Topics.Value())
|
||||
require.NoError(t, err, "Failed to connect new node to the network")
|
||||
|
||||
t.Log("Waiting for new node to sync")
|
||||
require.Eventually(t, func() bool {
|
||||
msgCount := len(env.chats[2].messageHistory)
|
||||
return msgCount == 2
|
||||
}, 1*time.Minute, 5*time.Second, "New node failed to sync message history")
|
||||
|
||||
t.Log("Sending message from old node")
|
||||
env.chats[0].SendMessage("Message from old node")
|
||||
|
||||
// Wait for message propagation
|
||||
time.Sleep(10 * time.Second)
|
||||
|
||||
// Verify the message is received by all nodes
|
||||
for i, chat := range env.chats {
|
||||
assert.Len(t, chat.messageHistory, 3, "Node %d should have 3 messages", i)
|
||||
}
|
||||
|
||||
t.Log("Sending message from new node")
|
||||
env.chats[2].SendMessage("Message from new node")
|
||||
|
||||
// Wait for message propagation
|
||||
time.Sleep(10 * time.Second)
|
||||
|
||||
// Verify the message from new node is received by all nodes
|
||||
for i, chat := range env.chats {
|
||||
assert.Len(t, chat.messageHistory, 4, "Node %d should have 4 messages", i)
|
||||
}
|
||||
|
||||
for i := 0; i < 3; i++ {
|
||||
lastMsg := env.chats[i].messageHistory[len(env.chats[i].messageHistory)-1]
|
||||
assert.Equal(t, "Message from new node", lastMsg.Content, "The last message is incorrect for node %d", i)
|
||||
}
|
||||
|
||||
t.Log("TestNewNodeSyncAndMessagePropagation completed")
|
||||
}
|
136
examples/chat2-reliable/exec.go
Normal file
136
examples/chat2-reliable/exec.go
Normal file
@ -0,0 +1,136 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
|
||||
tea "github.com/charmbracelet/bubbletea"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
"github.com/ethereum/go-ethereum/p2p/enode"
|
||||
"github.com/libp2p/go-libp2p/core/protocol"
|
||||
"github.com/multiformats/go-multiaddr"
|
||||
"github.com/waku-org/go-waku/waku/v2/node"
|
||||
"github.com/waku-org/go-waku/waku/v2/peerstore"
|
||||
"github.com/waku-org/go-waku/waku/v2/protocol/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/store"
|
||||
)
|
||||
|
||||
func execute(options Options) {
|
||||
var err error
|
||||
hostAddr, _ := net.ResolveTCPAddr("tcp", fmt.Sprintf("0.0.0.0:%d", options.Port))
|
||||
|
||||
if options.NodeKey == nil {
|
||||
options.NodeKey, err = crypto.GenerateKey()
|
||||
if err != nil {
|
||||
fmt.Println("Could not generate random key")
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
connNotifier := make(chan node.PeerConnection)
|
||||
|
||||
opts := []node.WakuNodeOption{
|
||||
node.WithPrivateKey(options.NodeKey),
|
||||
node.WithNTP(),
|
||||
node.WithHostAddress(hostAddr),
|
||||
node.WithConnectionNotification(connNotifier),
|
||||
}
|
||||
|
||||
if options.Relay.Enable {
|
||||
opts = append(opts, node.WithWakuRelay())
|
||||
}
|
||||
|
||||
if options.RLNRelay.Enable {
|
||||
spamHandler := func(message *pb.WakuMessage, topic string) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
if options.RLNRelay.Dynamic {
|
||||
fmt.Println("Setting up dynamic rln...")
|
||||
opts = append(opts, node.WithDynamicRLNRelay(
|
||||
options.RLNRelay.CredentialsPath,
|
||||
options.RLNRelay.CredentialsPassword,
|
||||
"", // Will use default tree path
|
||||
options.RLNRelay.MembershipContractAddress,
|
||||
options.RLNRelay.MembershipIndex,
|
||||
spamHandler,
|
||||
options.RLNRelay.ETHClientAddress,
|
||||
))
|
||||
} else {
|
||||
opts = append(opts, node.WithStaticRLNRelay(
|
||||
options.RLNRelay.MembershipIndex,
|
||||
spamHandler))
|
||||
}
|
||||
}
|
||||
|
||||
if options.DiscV5.Enable {
|
||||
nodes := []*enode.Node{}
|
||||
for _, n := range options.DiscV5.Nodes.Value() {
|
||||
parsedNode, err := enode.Parse(enode.ValidSchemes, n)
|
||||
if err != nil {
|
||||
fmt.Println("Failed to parse DiscV5 node ", err)
|
||||
return
|
||||
}
|
||||
nodes = append(nodes, parsedNode)
|
||||
}
|
||||
opts = append(opts, node.WithDiscoveryV5(uint(options.DiscV5.Port), nodes, options.DiscV5.AutoUpdate))
|
||||
}
|
||||
|
||||
if options.Filter.Enable {
|
||||
opts = append(opts, node.WithWakuFilterLightNode())
|
||||
}
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
wakuNode, err := node.New(opts...)
|
||||
if err != nil {
|
||||
fmt.Println(err.Error())
|
||||
return
|
||||
}
|
||||
|
||||
if err := wakuNode.Start(ctx); err != nil {
|
||||
fmt.Println(err.Error())
|
||||
return
|
||||
}
|
||||
|
||||
err = addPeer(wakuNode, options.Store.Node, options.Relay.Topics.Value(), store.StoreQueryID_v300)
|
||||
if err != nil {
|
||||
fmt.Println(err.Error())
|
||||
return
|
||||
}
|
||||
|
||||
err = addPeer(wakuNode, options.LightPush.Node, options.Relay.Topics.Value(), lightpush.LightPushID_v20beta1)
|
||||
if err != nil {
|
||||
fmt.Println(err.Error())
|
||||
return
|
||||
}
|
||||
|
||||
err = addPeer(wakuNode, options.Filter.Node, options.Relay.Topics.Value(), filter.FilterSubscribeID_v20beta1)
|
||||
|
||||
if err != nil {
|
||||
fmt.Println(err.Error())
|
||||
return
|
||||
}
|
||||
|
||||
chat := NewChat(ctx, wakuNode, connNotifier, options)
|
||||
p := tea.NewProgram(chat.ui)
|
||||
if err := p.Start(); err != nil {
|
||||
fmt.Println(err.Error())
|
||||
}
|
||||
|
||||
cancel()
|
||||
|
||||
wakuNode.Stop()
|
||||
chat.Stop()
|
||||
}
|
||||
|
||||
func addPeer(wakuNode *node.WakuNode, addr *multiaddr.Multiaddr, topics []string, protocols ...protocol.ID) error {
|
||||
if addr == nil {
|
||||
return nil
|
||||
}
|
||||
_, err := wakuNode.AddPeer(*addr, peerstore.Static, topics, protocols...)
|
||||
return err
|
||||
}
|
230
examples/chat2-reliable/flags.go
Normal file
230
examples/chat2-reliable/flags.go
Normal file
@ -0,0 +1,230 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/waku-org/go-waku/waku/cliutils"
|
||||
wcli "github.com/waku-org/go-waku/waku/cliutils"
|
||||
"github.com/waku-org/go-waku/waku/v2/protocol"
|
||||
|
||||
"github.com/urfave/cli/v2"
|
||||
)
|
||||
|
||||
type FleetValue struct {
|
||||
Value *Fleet
|
||||
Default Fleet
|
||||
}
|
||||
|
||||
func (v *FleetValue) Set(value string) error {
|
||||
if value == string(fleetProd) || value == string(fleetTest) || value == string(fleetNone) {
|
||||
*v.Value = Fleet(value)
|
||||
return nil
|
||||
}
|
||||
return fmt.Errorf("%s is not a valid option. need %+v", value, []Fleet{fleetProd, fleetTest, fleetNone})
|
||||
}
|
||||
|
||||
func (v *FleetValue) String() string {
|
||||
if v.Value == nil {
|
||||
return string(v.Default)
|
||||
}
|
||||
return string(*v.Value)
|
||||
}
|
||||
|
||||
func getFlags() []cli.Flag {
|
||||
// Defaults
|
||||
options.Fleet = fleetProd
|
||||
|
||||
testCT, err := protocol.NewContentTopic("toy-chat", "3", "mingde", "proto")
|
||||
if err != nil {
|
||||
panic("invalid contentTopic")
|
||||
}
|
||||
testnetContentTopic := testCT.String()
|
||||
|
||||
return []cli.Flag{
|
||||
&cli.GenericFlag{
|
||||
Name: "nodekey",
|
||||
Usage: "P2P node private key as hex. (default random)",
|
||||
Value: &wcli.PrivateKeyValue{
|
||||
Value: &options.NodeKey,
|
||||
},
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "listen-address",
|
||||
Aliases: []string{"host", "address"},
|
||||
Value: "0.0.0.0",
|
||||
Usage: "Listening address",
|
||||
Destination: &options.Address,
|
||||
},
|
||||
&cli.IntFlag{
|
||||
Name: "tcp-port",
|
||||
Aliases: []string{"port", "p"},
|
||||
Value: 0,
|
||||
Usage: "Libp2p TCP listening port (0 for random)",
|
||||
Destination: &options.Port,
|
||||
},
|
||||
&cli.IntFlag{
|
||||
Name: "udp-port",
|
||||
Value: 60000,
|
||||
Usage: "Listening UDP port for Node Discovery v5.",
|
||||
Destination: &options.DiscV5.Port,
|
||||
},
|
||||
&cli.GenericFlag{
|
||||
Name: "log-level",
|
||||
Aliases: []string{"l"},
|
||||
Value: &cliutils.ChoiceValue{
|
||||
Choices: []string{"DEBUG", "INFO", "WARN", "ERROR", "DPANIC", "PANIC", "FATAL"},
|
||||
Value: &options.LogLevel,
|
||||
},
|
||||
Usage: "Define the logging level,",
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "content-topic",
|
||||
Usage: "content topic to use for the chat",
|
||||
Value: testnetContentTopic,
|
||||
Destination: &options.ContentTopic,
|
||||
},
|
||||
&cli.GenericFlag{
|
||||
Name: "fleet",
|
||||
Usage: "Select the fleet to connect to",
|
||||
Value: &FleetValue{
|
||||
Default: fleetProd,
|
||||
Value: &options.Fleet,
|
||||
},
|
||||
},
|
||||
&cli.GenericFlag{
|
||||
Name: "staticnode",
|
||||
Usage: "Multiaddr of peer to directly connect with. Option may be repeated",
|
||||
Value: &wcli.MultiaddrSlice{
|
||||
Values: &options.StaticNodes,
|
||||
},
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "nickname",
|
||||
Usage: "nickname to use in chat.",
|
||||
Destination: &options.Nickname,
|
||||
Value: "Anonymous",
|
||||
},
|
||||
&cli.BoolFlag{
|
||||
Name: "relay",
|
||||
Value: true,
|
||||
Usage: "Enable relay protocol",
|
||||
Destination: &options.Relay.Enable,
|
||||
},
|
||||
&cli.StringSliceFlag{
|
||||
Name: "topic",
|
||||
Usage: "Pubsub topics to subscribe to. Option can be repeated",
|
||||
Destination: &options.Relay.Topics,
|
||||
},
|
||||
&cli.BoolFlag{
|
||||
Name: "store",
|
||||
Usage: "Enable store protocol",
|
||||
Value: true,
|
||||
Destination: &options.Store.Enable,
|
||||
},
|
||||
&cli.GenericFlag{
|
||||
Name: "storenode",
|
||||
Usage: "Multiaddr of a peer that supports store protocol.",
|
||||
Value: &wcli.MultiaddrValue{
|
||||
Value: &options.Store.Node,
|
||||
},
|
||||
},
|
||||
&cli.BoolFlag{
|
||||
Name: "filter",
|
||||
Usage: "Enable filter protocol",
|
||||
Destination: &options.Filter.Enable,
|
||||
},
|
||||
&cli.GenericFlag{
|
||||
Name: "filternode",
|
||||
Usage: "Multiaddr of a peer that supports filter protocol.",
|
||||
Value: &wcli.MultiaddrValue{
|
||||
Value: &options.Filter.Node,
|
||||
},
|
||||
},
|
||||
&cli.BoolFlag{
|
||||
Name: "lightpush",
|
||||
Usage: "Enable lightpush protocol",
|
||||
Destination: &options.LightPush.Enable,
|
||||
},
|
||||
&cli.GenericFlag{
|
||||
Name: "lightpushnode",
|
||||
Usage: "Multiaddr of a peer that supports lightpush protocol.",
|
||||
Value: &wcli.MultiaddrValue{
|
||||
Value: &options.LightPush.Node,
|
||||
},
|
||||
},
|
||||
&cli.BoolFlag{
|
||||
Name: "discv5-discovery",
|
||||
Usage: "Enable discovering nodes via Node Discovery v5",
|
||||
Destination: &options.DiscV5.Enable,
|
||||
},
|
||||
&cli.StringSliceFlag{
|
||||
Name: "discv5-bootstrap-node",
|
||||
Usage: "Text-encoded ENR for bootstrap node. Used when connecting to the network. Option may be repeated",
|
||||
Destination: &options.DiscV5.Nodes,
|
||||
},
|
||||
&cli.BoolFlag{
|
||||
Name: "discv5-enr-auto-update",
|
||||
Usage: "Discovery can automatically update its ENR with the IP address as seen by other nodes it communicates with.",
|
||||
Destination: &options.DiscV5.AutoUpdate,
|
||||
},
|
||||
&cli.BoolFlag{
|
||||
Name: "dns-discovery",
|
||||
Usage: "Enable DNS discovery",
|
||||
Destination: &options.DNSDiscovery.Enable,
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "dns-discovery-url",
|
||||
Usage: "URL for DNS node list in format 'enrtree://<key>@<fqdn>'",
|
||||
Destination: &options.DNSDiscovery.URL,
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "dns-discovery-name-server",
|
||||
Aliases: []string{"dns-discovery-nameserver"},
|
||||
Usage: "DNS nameserver IP to query (empty to use system's default)",
|
||||
Destination: &options.DNSDiscovery.Nameserver,
|
||||
},
|
||||
&cli.BoolFlag{
|
||||
Name: "rln-relay",
|
||||
Value: false,
|
||||
Usage: "Enable spam protection through rln-relay",
|
||||
Destination: &options.RLNRelay.Enable,
|
||||
},
|
||||
&cli.GenericFlag{
|
||||
Name: "rln-relay-cred-index",
|
||||
Usage: "the index of the onchain commitment to use",
|
||||
Value: &wcli.OptionalUint{
|
||||
Value: &options.RLNRelay.MembershipIndex,
|
||||
},
|
||||
},
|
||||
&cli.BoolFlag{
|
||||
Name: "rln-relay-dynamic",
|
||||
Usage: "Enable waku-rln-relay with on-chain dynamic group management",
|
||||
Destination: &options.RLNRelay.Dynamic,
|
||||
},
|
||||
&cli.PathFlag{
|
||||
Name: "rln-relay-cred-path",
|
||||
Usage: "The path for persisting rln-relay credential",
|
||||
Value: "",
|
||||
Destination: &options.RLNRelay.CredentialsPath,
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "rln-relay-cred-password",
|
||||
Value: "",
|
||||
Usage: "Password for encrypting RLN credentials",
|
||||
Destination: &options.RLNRelay.CredentialsPassword,
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "rln-relay-eth-client-address",
|
||||
Usage: "Ethereum testnet client address",
|
||||
Value: "ws://localhost:8545",
|
||||
Destination: &options.RLNRelay.ETHClientAddress,
|
||||
},
|
||||
&cli.GenericFlag{
|
||||
Name: "rln-relay-eth-contract-address",
|
||||
Usage: "Address of membership contract on an Ethereum testnet",
|
||||
Value: &wcli.AddressValue{
|
||||
Value: &options.RLNRelay.MembershipContractAddress,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
171
examples/chat2-reliable/go.mod
Normal file
171
examples/chat2-reliable/go.mod
Normal file
@ -0,0 +1,171 @@
|
||||
module chat2-reliable
|
||||
|
||||
go 1.21
|
||||
|
||||
toolchain go1.21.10
|
||||
|
||||
replace github.com/waku-org/go-waku => ../..
|
||||
|
||||
replace github.com/ethereum/go-ethereum v1.10.26 => github.com/status-im/go-ethereum v1.10.25-status.15
|
||||
|
||||
replace github.com/libp2p/go-libp2p-pubsub v0.11.0 => github.com/waku-org/go-libp2p-pubsub v0.0.0-20240703191659-2cbb09eac9b5
|
||||
|
||||
require (
|
||||
github.com/bits-and-blooms/bloom/v3 v3.7.0
|
||||
github.com/charmbracelet/bubbles v0.13.0
|
||||
github.com/charmbracelet/bubbletea v0.22.0
|
||||
github.com/charmbracelet/lipgloss v0.5.0
|
||||
github.com/ethereum/go-ethereum v1.10.26
|
||||
github.com/google/uuid v1.4.0
|
||||
github.com/ipfs/go-log/v2 v2.5.1
|
||||
github.com/libp2p/go-libp2p v0.35.2
|
||||
github.com/libp2p/go-msgio v0.3.0
|
||||
github.com/muesli/reflow v0.3.0
|
||||
github.com/multiformats/go-multiaddr v0.12.4
|
||||
github.com/stretchr/testify v1.9.0
|
||||
github.com/urfave/cli/v2 v2.27.2
|
||||
github.com/waku-org/go-waku v0.2.3-0.20221109195301-b2a5a68d28ba
|
||||
go.uber.org/zap v1.27.0
|
||||
golang.org/x/term v0.20.0
|
||||
google.golang.org/protobuf v1.34.1
|
||||
)
|
||||
|
||||
require (
|
||||
github.com/BurntSushi/toml v1.3.2 // indirect
|
||||
github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6 // indirect
|
||||
github.com/atotto/clipboard v0.1.4 // indirect
|
||||
github.com/avast/retry-go/v4 v4.5.1 // indirect
|
||||
github.com/beevik/ntp v0.3.0 // indirect
|
||||
github.com/benbjohnson/clock v1.3.5 // indirect
|
||||
github.com/beorn7/perks v1.0.1 // indirect
|
||||
github.com/bits-and-blooms/bitset v1.10.0 // indirect
|
||||
github.com/btcsuite/btcd v0.20.1-beta // indirect
|
||||
github.com/btcsuite/btcd/btcec/v2 v2.2.1 // indirect
|
||||
github.com/btcsuite/btcutil v0.0.0-20190425235716-9e5f4b9a998d // indirect
|
||||
github.com/cenkalti/backoff/v3 v3.2.2 // indirect
|
||||
github.com/cenkalti/backoff/v4 v4.1.2 // indirect
|
||||
github.com/cespare/xxhash/v2 v2.2.0 // indirect
|
||||
github.com/containerd/cgroups v1.1.0 // indirect
|
||||
github.com/containerd/console v1.0.3 // indirect
|
||||
github.com/coreos/go-systemd/v22 v22.5.0 // indirect
|
||||
github.com/cpuguy83/go-md2man/v2 v2.0.4 // indirect
|
||||
github.com/cruxic/go-hmac-drbg v0.0.0-20170206035330-84c46983886d // indirect
|
||||
github.com/davecgh/go-spew v1.1.1 // indirect
|
||||
github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c // indirect
|
||||
github.com/deckarep/golang-set v1.8.0 // indirect
|
||||
github.com/decred/dcrd/dcrec/secp256k1/v4 v4.3.0 // indirect
|
||||
github.com/docker/go-units v0.5.0 // indirect
|
||||
github.com/elastic/gosigar v0.14.2 // indirect
|
||||
github.com/flynn/noise v1.1.0 // indirect
|
||||
github.com/francoispqt/gojay v1.2.13 // indirect
|
||||
github.com/go-ole/go-ole v1.2.1 // indirect
|
||||
github.com/go-stack/stack v1.8.1 // indirect
|
||||
github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 // indirect
|
||||
github.com/godbus/dbus/v5 v5.1.0 // indirect
|
||||
github.com/gogo/protobuf v1.3.2 // indirect
|
||||
github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb // indirect
|
||||
github.com/google/gopacket v1.1.19 // indirect
|
||||
github.com/google/pprof v0.0.0-20240207164012-fb44976bdcd5 // indirect
|
||||
github.com/gorilla/websocket v1.5.3 // indirect
|
||||
github.com/hashicorp/golang-lru v0.5.5-0.20210104140557-80c98217689d // indirect
|
||||
github.com/hashicorp/golang-lru/v2 v2.0.7 // indirect
|
||||
github.com/holiman/uint256 v1.2.2-0.20230321075855-87b91420868c // indirect
|
||||
github.com/huin/goupnp v1.3.0 // indirect
|
||||
github.com/ipfs/go-cid v0.4.1 // indirect
|
||||
github.com/jackpal/go-nat-pmp v1.0.2 // indirect
|
||||
github.com/jbenet/go-temp-err-catcher v0.1.0 // indirect
|
||||
github.com/klauspost/compress v1.17.8 // indirect
|
||||
github.com/klauspost/cpuid/v2 v2.2.7 // indirect
|
||||
github.com/koron/go-ssdp v0.0.4 // 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-libp2p-asn-util v0.4.1 // indirect
|
||||
github.com/libp2p/go-libp2p-pubsub v0.11.0 // indirect
|
||||
github.com/libp2p/go-nat v0.2.0 // indirect
|
||||
github.com/libp2p/go-netroute v0.2.1 // indirect
|
||||
github.com/libp2p/go-reuseport v0.4.0 // indirect
|
||||
github.com/libp2p/go-yamux/v4 v4.0.1 // indirect
|
||||
github.com/lucasb-eyer/go-colorful v1.2.0 // indirect
|
||||
github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd // indirect
|
||||
github.com/mattn/go-isatty v0.0.20 // indirect
|
||||
github.com/mattn/go-runewidth v0.0.13 // indirect
|
||||
github.com/miekg/dns v1.1.58 // 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.1 // indirect
|
||||
github.com/mr-tron/base58 v1.2.0 // indirect
|
||||
github.com/muesli/ansi v0.0.0-20211018074035-2e021307bc4b // indirect
|
||||
github.com/muesli/cancelreader v0.2.1 // indirect
|
||||
github.com/muesli/termenv v0.11.1-0.20220212125758-44cd13922739 // indirect
|
||||
github.com/multiformats/go-base32 v0.1.0 // indirect
|
||||
github.com/multiformats/go-base36 v0.2.0 // indirect
|
||||
github.com/multiformats/go-multiaddr-dns v0.3.1 // indirect
|
||||
github.com/multiformats/go-multiaddr-fmt v0.1.0 // indirect
|
||||
github.com/multiformats/go-multibase v0.2.0 // indirect
|
||||
github.com/multiformats/go-multicodec v0.9.0 // indirect
|
||||
github.com/multiformats/go-multihash v0.2.3 // indirect
|
||||
github.com/multiformats/go-multistream v0.5.0 // indirect
|
||||
github.com/multiformats/go-varint v0.0.7 // indirect
|
||||
github.com/onsi/ginkgo/v2 v2.15.0 // indirect
|
||||
github.com/opencontainers/runtime-spec v1.2.0 // indirect
|
||||
github.com/pbnjay/memory v0.0.0-20210728143218-7b4eea64cf58 // indirect
|
||||
github.com/pion/datachannel v1.5.6 // indirect
|
||||
github.com/pion/dtls/v2 v2.2.11 // indirect
|
||||
github.com/pion/ice/v2 v2.3.25 // indirect
|
||||
github.com/pion/interceptor v0.1.29 // indirect
|
||||
github.com/pion/logging v0.2.2 // indirect
|
||||
github.com/pion/mdns v0.0.12 // indirect
|
||||
github.com/pion/randutil v0.1.0 // indirect
|
||||
github.com/pion/rtcp v1.2.14 // indirect
|
||||
github.com/pion/rtp v1.8.6 // indirect
|
||||
github.com/pion/sctp v1.8.16 // indirect
|
||||
github.com/pion/sdp/v3 v3.0.9 // indirect
|
||||
github.com/pion/srtp/v2 v2.0.18 // indirect
|
||||
github.com/pion/stun v0.6.1 // indirect
|
||||
github.com/pion/transport/v2 v2.2.5 // indirect
|
||||
github.com/pion/turn/v2 v2.1.6 // indirect
|
||||
github.com/pion/webrtc/v3 v3.2.40 // indirect
|
||||
github.com/pkg/errors v0.9.1 // indirect
|
||||
github.com/pmezard/go-difflib v1.0.0 // indirect
|
||||
github.com/prometheus/client_golang v1.19.1 // indirect
|
||||
github.com/prometheus/client_model v0.6.1 // indirect
|
||||
github.com/prometheus/common v0.48.0 // indirect
|
||||
github.com/prometheus/procfs v0.12.0 // indirect
|
||||
github.com/quic-go/qpack v0.4.0 // indirect
|
||||
github.com/quic-go/quic-go v0.44.0 // indirect
|
||||
github.com/quic-go/webtransport-go v0.8.0 // indirect
|
||||
github.com/raulk/go-watchdog v1.3.0 // indirect
|
||||
github.com/rivo/uniseg v0.2.0 // indirect
|
||||
github.com/rjeczalik/notify v0.9.3 // indirect
|
||||
github.com/russross/blackfriday/v2 v2.1.0 // indirect
|
||||
github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible // indirect
|
||||
github.com/spaolacci/murmur3 v1.1.0 // indirect
|
||||
github.com/status-im/status-go/extkeys v1.1.2 // indirect
|
||||
github.com/syndtr/goleveldb v1.0.1-0.20220614013038-64ee5596c38a // indirect
|
||||
github.com/tklauser/go-sysconf v0.3.5 // indirect
|
||||
github.com/tklauser/numcpus v0.2.2 // indirect
|
||||
github.com/waku-org/go-discover v0.0.0-20240506173252-4912704efdc5 // indirect
|
||||
github.com/waku-org/go-libp2p-rendezvous v0.0.0-20240110193335-a67d1cc760a0 // indirect
|
||||
github.com/waku-org/go-zerokit-rln v0.1.14-0.20240102145250-fa738c0bdf59 // indirect
|
||||
github.com/waku-org/go-zerokit-rln-apple v0.0.0-20230916172309-ee0ee61dde2b // indirect
|
||||
github.com/waku-org/go-zerokit-rln-arm v0.0.0-20230916171929-1dd9494ff065 // indirect
|
||||
github.com/waku-org/go-zerokit-rln-x86_64 v0.0.0-20230916171518-2a77c3734dd1 // indirect
|
||||
github.com/wk8/go-ordered-map v1.0.0 // indirect
|
||||
github.com/xrash/smetrics v0.0.0-20240312152122-5f08fbb34913 // indirect
|
||||
go.uber.org/dig v1.17.1 // indirect
|
||||
go.uber.org/fx v1.22.1 // indirect
|
||||
go.uber.org/mock v0.4.0 // indirect
|
||||
go.uber.org/multierr v1.11.0 // indirect
|
||||
golang.org/x/crypto v0.23.0 // indirect
|
||||
golang.org/x/exp v0.0.0-20240506185415-9bf2ced13842 // indirect
|
||||
golang.org/x/mod v0.17.0 // indirect
|
||||
golang.org/x/net v0.25.0 // indirect
|
||||
golang.org/x/sync v0.7.0 // indirect
|
||||
golang.org/x/sys v0.20.0 // indirect
|
||||
golang.org/x/text v0.15.0 // indirect
|
||||
golang.org/x/time v0.5.0 // indirect
|
||||
golang.org/x/tools v0.21.0 // indirect
|
||||
gopkg.in/natefinch/npipe.v2 v2.0.0-20160621034901-c1b8fa8bdcce // indirect
|
||||
gopkg.in/yaml.v3 v3.0.1 // indirect
|
||||
lukechampine.com/blake3 v1.2.1 // indirect
|
||||
)
|
1165
examples/chat2-reliable/go.sum
Normal file
1165
examples/chat2-reliable/go.sum
Normal file
File diff suppressed because it is too large
Load Diff
36
examples/chat2-reliable/main.go
Normal file
36
examples/chat2-reliable/main.go
Normal file
@ -0,0 +1,36 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"os"
|
||||
|
||||
logging "github.com/ipfs/go-log/v2"
|
||||
"github.com/urfave/cli/v2"
|
||||
"github.com/waku-org/go-waku/waku/v2/utils"
|
||||
"go.uber.org/zap/zapcore"
|
||||
)
|
||||
|
||||
var options Options
|
||||
|
||||
func main() {
|
||||
app := &cli.App{
|
||||
Flags: getFlags(),
|
||||
Action: func(c *cli.Context) error {
|
||||
|
||||
lvl, err := zapcore.ParseLevel(options.LogLevel)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
logging.SetAllLoggers(logging.LogLevel(lvl))
|
||||
utils.InitLogger("console", "file:chat2.log", "chat2", lvl)
|
||||
|
||||
execute(options)
|
||||
return nil
|
||||
},
|
||||
}
|
||||
|
||||
err := app.Run(os.Args)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
126
examples/chat2-reliable/options.go
Normal file
126
examples/chat2-reliable/options.go
Normal file
@ -0,0 +1,126 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"crypto/ecdsa"
|
||||
"errors"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/multiformats/go-multiaddr"
|
||||
"github.com/urfave/cli/v2"
|
||||
)
|
||||
|
||||
// DiscV5Options are settings to enable a modified version of Ethereum’s Node
|
||||
// Discovery Protocol v5 as a means for ambient node discovery.
|
||||
type DiscV5Options struct {
|
||||
Enable bool
|
||||
Nodes cli.StringSlice
|
||||
Port int
|
||||
AutoUpdate bool
|
||||
}
|
||||
|
||||
// RelayOptions are settings to enable the relay protocol which is a pubsub
|
||||
// approach to peer-to-peer messaging with a strong focus on privacy,
|
||||
// censorship-resistance, security and scalability.
|
||||
type RelayOptions struct {
|
||||
Enable bool
|
||||
Topics cli.StringSlice
|
||||
}
|
||||
|
||||
type RLNRelayOptions struct {
|
||||
Enable bool
|
||||
CredentialsPath string
|
||||
CredentialsPassword string
|
||||
MembershipIndex *uint
|
||||
Dynamic bool
|
||||
ETHClientAddress string
|
||||
MembershipContractAddress common.Address
|
||||
}
|
||||
|
||||
func nodePeerID(node *multiaddr.Multiaddr) (peer.ID, error) {
|
||||
if node == nil {
|
||||
return peer.ID(""), errors.New("node is nil")
|
||||
}
|
||||
|
||||
peerID, err := (*node).ValueForProtocol(multiaddr.P_P2P)
|
||||
if err != nil {
|
||||
return peer.ID(""), err
|
||||
}
|
||||
|
||||
return peer.Decode(peerID)
|
||||
}
|
||||
|
||||
// FilterOptions are settings used to enable filter protocol. This is a protocol
|
||||
// that enables subscribing to messages that a peer receives. This is a more
|
||||
// lightweight version of WakuRelay specifically designed for bandwidth
|
||||
// restricted devices.
|
||||
type FilterOptions struct {
|
||||
Enable bool
|
||||
Node *multiaddr.Multiaddr
|
||||
}
|
||||
|
||||
func (f FilterOptions) NodePeerID() (peer.ID, error) {
|
||||
return nodePeerID(f.Node)
|
||||
}
|
||||
|
||||
// LightpushOptions are settings used to enable the lightpush protocol. This is
|
||||
// a lightweight protocol used to avoid having to run the relay protocol which
|
||||
// is more resource intensive. With this protocol a message is pushed to a peer
|
||||
// that supports both the lightpush protocol and relay protocol. That peer will
|
||||
// broadcast the message and return a confirmation that the message was
|
||||
// broadcasted
|
||||
type LightpushOptions struct {
|
||||
Enable bool
|
||||
Node *multiaddr.Multiaddr
|
||||
}
|
||||
|
||||
func (f LightpushOptions) NodePeerID() (peer.ID, error) {
|
||||
return nodePeerID(f.Node)
|
||||
}
|
||||
|
||||
// StoreOptions are settings used for enabling the store protocol, used to
|
||||
// retrieve message history from other nodes
|
||||
type StoreOptions struct {
|
||||
Enable bool
|
||||
Node *multiaddr.Multiaddr
|
||||
}
|
||||
|
||||
func (f StoreOptions) NodePeerID() (peer.ID, error) {
|
||||
return nodePeerID(f.Node)
|
||||
}
|
||||
|
||||
// DNSDiscoveryOptions are settings used for enabling DNS-based discovery
|
||||
// protocol that stores merkle trees in DNS records which contain connection
|
||||
// information for nodes. It's very useful for bootstrapping a p2p network.
|
||||
type DNSDiscoveryOptions struct {
|
||||
Enable bool
|
||||
URL string
|
||||
Nameserver string
|
||||
}
|
||||
|
||||
type Fleet string
|
||||
|
||||
const fleetNone Fleet = "none"
|
||||
const fleetProd Fleet = "prod"
|
||||
const fleetTest Fleet = "test"
|
||||
|
||||
// Options contains all the available features and settings that can be
|
||||
// configured via flags when executing chat2
|
||||
type Options struct {
|
||||
Port int
|
||||
Fleet Fleet
|
||||
Address string
|
||||
NodeKey *ecdsa.PrivateKey
|
||||
ContentTopic string
|
||||
Nickname string
|
||||
LogLevel string
|
||||
StaticNodes []multiaddr.Multiaddr
|
||||
|
||||
Relay RelayOptions
|
||||
Store StoreOptions
|
||||
Filter FilterOptions
|
||||
LightPush LightpushOptions
|
||||
RLNRelay RLNRelayOptions
|
||||
DiscV5 DiscV5Options
|
||||
DNSDiscovery DNSDiscoveryOptions
|
||||
}
|
330
examples/chat2-reliable/pb/chat2.pb.go
Normal file
330
examples/chat2-reliable/pb/chat2.pb.go
Normal file
@ -0,0 +1,330 @@
|
||||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// versions:
|
||||
// protoc-gen-go v1.34.2
|
||||
// protoc v3.19.4
|
||||
// source: chat2.proto
|
||||
|
||||
package pb
|
||||
|
||||
import (
|
||||
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
|
||||
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
|
||||
reflect "reflect"
|
||||
sync "sync"
|
||||
)
|
||||
|
||||
const (
|
||||
// Verify that this generated code is sufficiently up-to-date.
|
||||
_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
|
||||
// Verify that runtime/protoimpl is sufficiently up-to-date.
|
||||
_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
|
||||
)
|
||||
|
||||
type Message struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
SenderId string `protobuf:"bytes,1,opt,name=sender_id,json=senderId,proto3" json:"sender_id,omitempty"`
|
||||
MessageId string `protobuf:"bytes,2,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"`
|
||||
LamportTimestamp int32 `protobuf:"varint,3,opt,name=lamport_timestamp,json=lamportTimestamp,proto3" json:"lamport_timestamp,omitempty"`
|
||||
CausalHistory []string `protobuf:"bytes,4,rep,name=causal_history,json=causalHistory,proto3" json:"causal_history,omitempty"`
|
||||
ChannelId string `protobuf:"bytes,5,opt,name=channel_id,json=channelId,proto3" json:"channel_id,omitempty"`
|
||||
BloomFilter []byte `protobuf:"bytes,6,opt,name=bloom_filter,json=bloomFilter,proto3" json:"bloom_filter,omitempty"`
|
||||
Content string `protobuf:"bytes,7,opt,name=content,proto3" json:"content,omitempty"`
|
||||
}
|
||||
|
||||
func (x *Message) Reset() {
|
||||
*x = Message{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_chat2_proto_msgTypes[0]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *Message) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*Message) ProtoMessage() {}
|
||||
|
||||
func (x *Message) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_chat2_proto_msgTypes[0]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use Message.ProtoReflect.Descriptor instead.
|
||||
func (*Message) Descriptor() ([]byte, []int) {
|
||||
return file_chat2_proto_rawDescGZIP(), []int{0}
|
||||
}
|
||||
|
||||
func (x *Message) GetSenderId() string {
|
||||
if x != nil {
|
||||
return x.SenderId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *Message) GetMessageId() string {
|
||||
if x != nil {
|
||||
return x.MessageId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *Message) GetLamportTimestamp() int32 {
|
||||
if x != nil {
|
||||
return x.LamportTimestamp
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (x *Message) GetCausalHistory() []string {
|
||||
if x != nil {
|
||||
return x.CausalHistory
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *Message) GetChannelId() string {
|
||||
if x != nil {
|
||||
return x.ChannelId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (x *Message) GetBloomFilter() []byte {
|
||||
if x != nil {
|
||||
return x.BloomFilter
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (x *Message) GetContent() string {
|
||||
if x != nil {
|
||||
return x.Content
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// only for peer retrieval instead of store
|
||||
type MessageRequest struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
MessageId string `protobuf:"bytes,1,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"`
|
||||
}
|
||||
|
||||
func (x *MessageRequest) Reset() {
|
||||
*x = MessageRequest{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_chat2_proto_msgTypes[1]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *MessageRequest) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*MessageRequest) ProtoMessage() {}
|
||||
|
||||
func (x *MessageRequest) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_chat2_proto_msgTypes[1]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use MessageRequest.ProtoReflect.Descriptor instead.
|
||||
func (*MessageRequest) Descriptor() ([]byte, []int) {
|
||||
return file_chat2_proto_rawDescGZIP(), []int{1}
|
||||
}
|
||||
|
||||
func (x *MessageRequest) GetMessageId() string {
|
||||
if x != nil {
|
||||
return x.MessageId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type MessageResponse struct {
|
||||
state protoimpl.MessageState
|
||||
sizeCache protoimpl.SizeCache
|
||||
unknownFields protoimpl.UnknownFields
|
||||
|
||||
Message *Message `protobuf:"bytes,1,opt,name=message,proto3" json:"message,omitempty"`
|
||||
}
|
||||
|
||||
func (x *MessageResponse) Reset() {
|
||||
*x = MessageResponse{}
|
||||
if protoimpl.UnsafeEnabled {
|
||||
mi := &file_chat2_proto_msgTypes[2]
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
}
|
||||
|
||||
func (x *MessageResponse) String() string {
|
||||
return protoimpl.X.MessageStringOf(x)
|
||||
}
|
||||
|
||||
func (*MessageResponse) ProtoMessage() {}
|
||||
|
||||
func (x *MessageResponse) ProtoReflect() protoreflect.Message {
|
||||
mi := &file_chat2_proto_msgTypes[2]
|
||||
if protoimpl.UnsafeEnabled && x != nil {
|
||||
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
|
||||
if ms.LoadMessageInfo() == nil {
|
||||
ms.StoreMessageInfo(mi)
|
||||
}
|
||||
return ms
|
||||
}
|
||||
return mi.MessageOf(x)
|
||||
}
|
||||
|
||||
// Deprecated: Use MessageResponse.ProtoReflect.Descriptor instead.
|
||||
func (*MessageResponse) Descriptor() ([]byte, []int) {
|
||||
return file_chat2_proto_rawDescGZIP(), []int{2}
|
||||
}
|
||||
|
||||
func (x *MessageResponse) GetMessage() *Message {
|
||||
if x != nil {
|
||||
return x.Message
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
var File_chat2_proto protoreflect.FileDescriptor
|
||||
|
||||
var file_chat2_proto_rawDesc = []byte{
|
||||
0x0a, 0x0b, 0x63, 0x68, 0x61, 0x74, 0x32, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x02, 0x70,
|
||||
0x62, 0x22, 0xf5, 0x01, 0x0a, 0x07, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1b, 0x0a,
|
||||
0x09, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
|
||||
0x52, 0x08, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6d, 0x65,
|
||||
0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09,
|
||||
0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x2b, 0x0a, 0x11, 0x6c, 0x61, 0x6d,
|
||||
0x70, 0x6f, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03,
|
||||
0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x6c, 0x61, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x69, 0x6d,
|
||||
0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x61, 0x75, 0x73, 0x61, 0x6c,
|
||||
0x5f, 0x68, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d,
|
||||
0x63, 0x61, 0x75, 0x73, 0x61, 0x6c, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x1d, 0x0a,
|
||||
0x0a, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28,
|
||||
0x09, 0x52, 0x09, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c,
|
||||
0x62, 0x6c, 0x6f, 0x6f, 0x6d, 0x5f, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01,
|
||||
0x28, 0x0c, 0x52, 0x0b, 0x62, 0x6c, 0x6f, 0x6f, 0x6d, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12,
|
||||
0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09,
|
||||
0x52, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x22, 0x2f, 0x0a, 0x0e, 0x4d, 0x65, 0x73,
|
||||
0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x6d,
|
||||
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
|
||||
0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x22, 0x38, 0x0a, 0x0f, 0x4d, 0x65,
|
||||
0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a,
|
||||
0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0b,
|
||||
0x2e, 0x70, 0x62, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x07, 0x6d, 0x65, 0x73,
|
||||
0x73, 0x61, 0x67, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
|
||||
}
|
||||
|
||||
var (
|
||||
file_chat2_proto_rawDescOnce sync.Once
|
||||
file_chat2_proto_rawDescData = file_chat2_proto_rawDesc
|
||||
)
|
||||
|
||||
func file_chat2_proto_rawDescGZIP() []byte {
|
||||
file_chat2_proto_rawDescOnce.Do(func() {
|
||||
file_chat2_proto_rawDescData = protoimpl.X.CompressGZIP(file_chat2_proto_rawDescData)
|
||||
})
|
||||
return file_chat2_proto_rawDescData
|
||||
}
|
||||
|
||||
var file_chat2_proto_msgTypes = make([]protoimpl.MessageInfo, 3)
|
||||
var file_chat2_proto_goTypes = []any{
|
||||
(*Message)(nil), // 0: pb.Message
|
||||
(*MessageRequest)(nil), // 1: pb.MessageRequest
|
||||
(*MessageResponse)(nil), // 2: pb.MessageResponse
|
||||
}
|
||||
var file_chat2_proto_depIdxs = []int32{
|
||||
0, // 0: pb.MessageResponse.message:type_name -> pb.Message
|
||||
1, // [1:1] is the sub-list for method output_type
|
||||
1, // [1:1] is the sub-list for method input_type
|
||||
1, // [1:1] is the sub-list for extension type_name
|
||||
1, // [1:1] is the sub-list for extension extendee
|
||||
0, // [0:1] is the sub-list for field type_name
|
||||
}
|
||||
|
||||
func init() { file_chat2_proto_init() }
|
||||
func file_chat2_proto_init() {
|
||||
if File_chat2_proto != nil {
|
||||
return
|
||||
}
|
||||
if !protoimpl.UnsafeEnabled {
|
||||
file_chat2_proto_msgTypes[0].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*Message); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_chat2_proto_msgTypes[1].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*MessageRequest); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
file_chat2_proto_msgTypes[2].Exporter = func(v any, i int) any {
|
||||
switch v := v.(*MessageResponse); i {
|
||||
case 0:
|
||||
return &v.state
|
||||
case 1:
|
||||
return &v.sizeCache
|
||||
case 2:
|
||||
return &v.unknownFields
|
||||
default:
|
||||
return nil
|
||||
}
|
||||
}
|
||||
}
|
||||
type x struct{}
|
||||
out := protoimpl.TypeBuilder{
|
||||
File: protoimpl.DescBuilder{
|
||||
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
|
||||
RawDescriptor: file_chat2_proto_rawDesc,
|
||||
NumEnums: 0,
|
||||
NumMessages: 3,
|
||||
NumExtensions: 0,
|
||||
NumServices: 0,
|
||||
},
|
||||
GoTypes: file_chat2_proto_goTypes,
|
||||
DependencyIndexes: file_chat2_proto_depIdxs,
|
||||
MessageInfos: file_chat2_proto_msgTypes,
|
||||
}.Build()
|
||||
File_chat2_proto = out.File
|
||||
file_chat2_proto_rawDesc = nil
|
||||
file_chat2_proto_goTypes = nil
|
||||
file_chat2_proto_depIdxs = nil
|
||||
}
|
22
examples/chat2-reliable/pb/chat2.proto
Normal file
22
examples/chat2-reliable/pb/chat2.proto
Normal file
@ -0,0 +1,22 @@
|
||||
syntax = "proto3";
|
||||
|
||||
package pb;
|
||||
|
||||
message Message {
|
||||
string sender_id = 1;
|
||||
string message_id = 2;
|
||||
int32 lamport_timestamp = 3;
|
||||
repeated string causal_history = 4;
|
||||
string channel_id = 5;
|
||||
bytes bloom_filter = 6;
|
||||
string content = 7;
|
||||
}
|
||||
|
||||
// only for peer retrieval instead of store
|
||||
message MessageRequest {
|
||||
string message_id = 1;
|
||||
}
|
||||
|
||||
message MessageResponse {
|
||||
Message message = 1;
|
||||
}
|
3
examples/chat2-reliable/pb/generate.go
Normal file
3
examples/chat2-reliable/pb/generate.go
Normal file
@ -0,0 +1,3 @@
|
||||
package pb
|
||||
|
||||
//go:generate protoc -I. --go_opt=paths=source_relative --go_opt=Mchat2.proto=./pb --go_out=. ./chat2.proto
|
171
examples/chat2-reliable/peer_retrieval.go
Normal file
171
examples/chat2-reliable/peer_retrieval.go
Normal file
@ -0,0 +1,171 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"chat2-reliable/pb"
|
||||
"context"
|
||||
"encoding/base64"
|
||||
"errors"
|
||||
"fmt"
|
||||
"math"
|
||||
"time"
|
||||
|
||||
"github.com/libp2p/go-libp2p/core/network"
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/libp2p/go-libp2p/core/protocol"
|
||||
"github.com/libp2p/go-msgio/pbio"
|
||||
"github.com/waku-org/go-waku/waku/v2/peermanager"
|
||||
wpb "github.com/waku-org/go-waku/waku/v2/protocol/pb"
|
||||
"github.com/waku-org/go-waku/waku/v2/protocol/relay"
|
||||
"github.com/waku-org/go-waku/waku/v2/protocol/store"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
const messageRequestProtocolID = protocol.ID("/chat2-reliable/message-request/1.0.0")
|
||||
|
||||
// below functions are specifically for peer retrieval of missing msgs instead of store
|
||||
func (c *Chat) doRequestMissingMessageFromPeers(messageID string) (*pb.Message, error) {
|
||||
peers := c.node.Host().Network().Peers()
|
||||
for _, peerID := range peers {
|
||||
msg, err := c.requestMessageFromPeer(peerID, messageID)
|
||||
if err == nil && msg != nil {
|
||||
return msg, nil
|
||||
}
|
||||
}
|
||||
return nil, errors.New("no peers could provide the missing message")
|
||||
}
|
||||
|
||||
func (c *Chat) requestMessageFromPeer(peerID peer.ID, messageID string) (*pb.Message, error) {
|
||||
ctx, cancel := context.WithTimeout(c.ctx, 30*time.Second)
|
||||
defer cancel()
|
||||
|
||||
stream, err := c.node.Host().NewStream(ctx, peerID, messageRequestProtocolID)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to open stream to peer: %w", err)
|
||||
}
|
||||
|
||||
writer := pbio.NewDelimitedWriter(stream)
|
||||
reader := pbio.NewDelimitedReader(stream, math.MaxInt32)
|
||||
|
||||
// Send message request
|
||||
request := &pb.MessageRequest{MessageId: messageID}
|
||||
err = writeProtobufMessage(writer, request)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to send message request: %w", err)
|
||||
}
|
||||
|
||||
// Read response
|
||||
response := &pb.MessageResponse{}
|
||||
err = readProtobufMessage(reader, response)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("failed to read message response: %w", err)
|
||||
}
|
||||
|
||||
if response.Message == nil {
|
||||
return nil, fmt.Errorf("peer did not have the requested message")
|
||||
}
|
||||
|
||||
return response.Message, nil
|
||||
}
|
||||
|
||||
// Helper functions for protobuf message reading/writing
|
||||
func writeProtobufMessage(stream pbio.WriteCloser, msg proto.Message) error {
|
||||
err := stream.WriteMsg(msg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func readProtobufMessage(stream pbio.ReadCloser, msg proto.Message) error {
|
||||
err := stream.ReadMsg(msg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *Chat) handleMessageRequest(stream network.Stream) {
|
||||
writer := pbio.NewDelimitedWriter(stream)
|
||||
reader := pbio.NewDelimitedReader(stream, math.MaxInt32)
|
||||
|
||||
request := &pb.MessageRequest{}
|
||||
err := readProtobufMessage(reader, request)
|
||||
if err != nil {
|
||||
stream.Reset()
|
||||
c.ui.ErrorMessage(fmt.Errorf("failed to read message request: %w", err))
|
||||
return
|
||||
}
|
||||
|
||||
c.mutex.Lock()
|
||||
var foundMessage *pb.Message
|
||||
for _, msg := range c.messageHistory {
|
||||
if msg.MessageId == request.MessageId {
|
||||
foundMessage = msg
|
||||
break
|
||||
}
|
||||
}
|
||||
c.mutex.Unlock()
|
||||
|
||||
response := &pb.MessageResponse{Message: foundMessage}
|
||||
err = writeProtobufMessage(writer, response)
|
||||
if err != nil {
|
||||
stream.Reset()
|
||||
c.ui.ErrorMessage(fmt.Errorf("failed to send message response: %w", err))
|
||||
return
|
||||
}
|
||||
|
||||
stream.Close()
|
||||
}
|
||||
|
||||
func (c *Chat) setupMessageRequestHandler() {
|
||||
c.node.Host().SetStreamHandler(messageRequestProtocolID, c.handleMessageRequest)
|
||||
}
|
||||
|
||||
func (c *Chat) _doRequestMissingMessageFromStore(messageID string) error {
|
||||
ctx, cancel := context.WithTimeout(c.ctx, 10*time.Second)
|
||||
defer cancel()
|
||||
|
||||
hash, err := base64.URLEncoding.DecodeString(messageID)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to parse message hash: %w", err)
|
||||
}
|
||||
|
||||
x := store.MessageHashCriteria{
|
||||
MessageHashes: []wpb.MessageHash{wpb.ToMessageHash(hash)},
|
||||
}
|
||||
|
||||
peers, err := c.node.PeerManager().SelectPeers(peermanager.PeerSelectionCriteria{
|
||||
SelectionType: peermanager.Automatic,
|
||||
Proto: store.StoreQueryID_v300,
|
||||
PubsubTopics: []string{relay.DefaultWakuTopic},
|
||||
Ctx: ctx,
|
||||
})
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to find a store node: %w", err)
|
||||
}
|
||||
response, err := c.node.Store().Request(ctx, x,
|
||||
store.WithAutomaticRequestID(),
|
||||
store.WithPeer(peers[0]),
|
||||
//store.WithAutomaticPeerSelection(),
|
||||
store.WithPaging(true, 100), // Use paging to handle potentially large result sets
|
||||
)
|
||||
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to retrieve missing message: %w", err)
|
||||
}
|
||||
|
||||
for _, msg := range response.Messages() {
|
||||
decodedMsg, err := decodeMessage(c.options.ContentTopic, msg.Message)
|
||||
if err != nil {
|
||||
continue
|
||||
}
|
||||
if decodedMsg.MessageId == messageID {
|
||||
c.processReceivedMessage(decodedMsg)
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
return fmt.Errorf("missing message not found: %s", messageID)
|
||||
}
|
416
examples/chat2-reliable/reliability.go
Normal file
416
examples/chat2-reliable/reliability.go
Normal file
@ -0,0 +1,416 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"chat2-reliable/pb"
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/bits-and-blooms/bloom/v3"
|
||||
)
|
||||
|
||||
const (
|
||||
bloomFilterSize = 10000
|
||||
bloomFilterFPRate = 0.01
|
||||
bloomFilterWindow = 1 * time.Hour
|
||||
bloomFilterCleanInterval = 30 * time.Minute
|
||||
bufferSweepInterval = 5 * time.Second
|
||||
syncMessageInterval = 30 * time.Second
|
||||
messageAckTimeout = 60 * time.Second
|
||||
maxRetries = 3
|
||||
retryBaseDelay = 1 * time.Second
|
||||
maxRetryDelay = 10 * time.Second
|
||||
ackTimeout = 5 * time.Second
|
||||
maxResendAttempts = 5
|
||||
resendBaseDelay = 1 * time.Second
|
||||
maxResendDelay = 30 * time.Second
|
||||
)
|
||||
|
||||
func (c *Chat) initReliabilityProtocol() {
|
||||
c.wg.Add(4)
|
||||
c.setupMessageRequestHandler()
|
||||
|
||||
go c.periodicBufferSweep()
|
||||
go c.periodicSyncMessage()
|
||||
go c.startBloomFilterCleaner()
|
||||
go c.startEagerPushMechanism()
|
||||
}
|
||||
|
||||
func (c *Chat) startEagerPushMechanism() {
|
||||
defer c.wg.Done()
|
||||
|
||||
ticker := time.NewTicker(1 * time.Second)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
c.checkUnacknowledgedMessages()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
type UnacknowledgedMessage struct {
|
||||
Message *pb.Message
|
||||
SendTime time.Time
|
||||
ResendAttempts int
|
||||
}
|
||||
|
||||
func (c *Chat) startBloomFilterCleaner() {
|
||||
defer c.wg.Done()
|
||||
|
||||
ticker := time.NewTicker(bloomFilterCleanInterval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
c.bloomFilter.Clean()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) SendMessage(line string) {
|
||||
c.incLamportTimestamp()
|
||||
|
||||
bloomBytes, err := c.bloomFilter.MarshalBinary()
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(fmt.Errorf("failed to marshal bloom filter: %w", err))
|
||||
return
|
||||
}
|
||||
|
||||
msg := &pb.Message{
|
||||
SenderId: c.node.Host().ID().String(),
|
||||
MessageId: generateUniqueID(),
|
||||
LamportTimestamp: c.getLamportTimestamp(),
|
||||
CausalHistory: c.getRecentMessageIDs(10),
|
||||
ChannelId: c.options.ContentTopic,
|
||||
BloomFilter: bloomBytes,
|
||||
Content: line,
|
||||
}
|
||||
|
||||
unackMsg := UnacknowledgedMessage{
|
||||
Message: msg,
|
||||
SendTime: time.Now(),
|
||||
ResendAttempts: 0,
|
||||
}
|
||||
c.outgoingBuffer = append(c.outgoingBuffer, unackMsg)
|
||||
|
||||
ctx, cancel := context.WithTimeout(c.ctx, messageAckTimeout)
|
||||
defer cancel()
|
||||
|
||||
err = c.publish(ctx, msg)
|
||||
if err != nil {
|
||||
if err.Error() == "validation failed" {
|
||||
err = errors.New("message rate violation")
|
||||
}
|
||||
c.ui.ErrorMessage(err)
|
||||
} else {
|
||||
c.bloomFilter.Add(msg.MessageId)
|
||||
c.addToMessageHistory(msg)
|
||||
c.ui.ChatMessage(int64(c.getLamportTimestamp()), msg.SenderId, msg.Content)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) processReceivedMessage(msg *pb.Message) {
|
||||
// Check if the message is already in the bloom filter
|
||||
if c.bloomFilter.Test(msg.MessageId) {
|
||||
return
|
||||
}
|
||||
|
||||
// Update bloom filter
|
||||
c.bloomFilter.Add(msg.MessageId)
|
||||
|
||||
// Update Lamport timestamp
|
||||
c.updateLamportTimestamp(msg.LamportTimestamp)
|
||||
|
||||
// Review ACK status of messages in the unacknowledged outgoing buffer
|
||||
c.reviewAckStatus(msg)
|
||||
|
||||
// Check causal dependencies
|
||||
missingDeps := c.checkCausalDependencies(msg)
|
||||
if len(missingDeps) == 0 {
|
||||
if msg.Content != "" {
|
||||
// Process the message
|
||||
c.ui.ChatMessage(int64(c.getLamportTimestamp()), msg.SenderId, msg.Content)
|
||||
// Add to message history
|
||||
c.addToMessageHistory(msg)
|
||||
}
|
||||
|
||||
// Process any messages in the buffer that now have their dependencies met
|
||||
c.processBufferedMessages()
|
||||
} else {
|
||||
// Request missing dependencies
|
||||
for _, depID := range missingDeps {
|
||||
c.requestMissingMessage(depID)
|
||||
}
|
||||
// Add to incoming buffer
|
||||
c.addToIncomingBuffer(msg)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) processBufferedMessages() {
|
||||
c.mutex.Lock()
|
||||
|
||||
remainingBuffer := make([]*pb.Message, 0, len(c.incomingBuffer))
|
||||
processedBuffer := make([]*pb.Message, 0)
|
||||
|
||||
for _, msg := range c.incomingBuffer {
|
||||
missingDeps := c.checkCausalDependencies(msg)
|
||||
if len(missingDeps) == 0 {
|
||||
if msg.Content != "" {
|
||||
c.ui.ChatMessage(int64(c.getLamportTimestamp()), msg.SenderId, msg.Content)
|
||||
processedBuffer = append(processedBuffer, msg)
|
||||
}
|
||||
} else {
|
||||
remainingBuffer = append(remainingBuffer, msg)
|
||||
}
|
||||
}
|
||||
|
||||
c.incomingBuffer = remainingBuffer
|
||||
c.mutex.Unlock()
|
||||
|
||||
for _, msg := range processedBuffer {
|
||||
c.addToMessageHistory(msg)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) reviewAckStatus(msg *pb.Message) {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
|
||||
// Review causal history
|
||||
for _, msgID := range msg.CausalHistory {
|
||||
for i, outMsg := range c.outgoingBuffer {
|
||||
if outMsg.Message.MessageId == msgID {
|
||||
// acknowledged and remove from outgoing buffer
|
||||
c.outgoingBuffer = append(c.outgoingBuffer[:i], c.outgoingBuffer[i+1:]...)
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Review bloom filter
|
||||
if msg.BloomFilter != nil {
|
||||
receivedFilter := bloom.NewWithEstimates(bloomFilterSize, bloomFilterFPRate)
|
||||
err := receivedFilter.UnmarshalBinary(msg.BloomFilter)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(fmt.Errorf("failed to unmarshal bloom filter: %w", err))
|
||||
} else {
|
||||
for i := 0; i < len(c.outgoingBuffer); i++ {
|
||||
if receivedFilter.Test([]byte(c.outgoingBuffer[i].Message.MessageId)) {
|
||||
// possibly acknowledged and remove it from the outgoing buffer
|
||||
c.outgoingBuffer = append(c.outgoingBuffer[:i], c.outgoingBuffer[i+1:]...)
|
||||
i--
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) requestMissingMessage(messageID string) {
|
||||
for retry := 0; retry < maxRetries; retry++ {
|
||||
missedMsg, err := c.doRequestMissingMessageFromPeers(messageID)
|
||||
if err == nil {
|
||||
c.processReceivedMessage(missedMsg)
|
||||
return
|
||||
}
|
||||
|
||||
// Exponential backoff
|
||||
delay := retryBaseDelay * time.Duration(1<<uint(retry))
|
||||
if delay > maxRetryDelay {
|
||||
delay = maxRetryDelay
|
||||
}
|
||||
time.Sleep(delay)
|
||||
}
|
||||
|
||||
c.ui.ErrorMessage(fmt.Errorf("failed to retrieve missing message %s after %d attempts", messageID, maxRetries))
|
||||
}
|
||||
|
||||
func (c *Chat) checkCausalDependencies(msg *pb.Message) []string {
|
||||
var missingDeps []string
|
||||
seenMessages := make(map[string]bool)
|
||||
|
||||
for _, historicalMsg := range c.messageHistory {
|
||||
seenMessages[historicalMsg.MessageId] = true
|
||||
}
|
||||
|
||||
for _, depID := range msg.CausalHistory {
|
||||
if !seenMessages[depID] {
|
||||
missingDeps = append(missingDeps, depID)
|
||||
}
|
||||
}
|
||||
return missingDeps
|
||||
}
|
||||
|
||||
func (c *Chat) addToMessageHistory(msg *pb.Message) {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
|
||||
// Find the correct position to insert the new message
|
||||
insertIndex := len(c.messageHistory)
|
||||
for i, existingMsg := range c.messageHistory {
|
||||
if existingMsg.LamportTimestamp > msg.LamportTimestamp {
|
||||
insertIndex = i
|
||||
break
|
||||
} else if existingMsg.LamportTimestamp == msg.LamportTimestamp {
|
||||
// If timestamps are equal, use MessageId for deterministic ordering
|
||||
if existingMsg.MessageId > msg.MessageId {
|
||||
insertIndex = i
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// Insert the new message at the correct position
|
||||
if insertIndex == len(c.messageHistory) {
|
||||
c.messageHistory = append(c.messageHistory, msg)
|
||||
} else {
|
||||
c.messageHistory = append(c.messageHistory[:insertIndex+1], c.messageHistory[insertIndex:]...)
|
||||
c.messageHistory[insertIndex] = msg
|
||||
}
|
||||
|
||||
// Trim the history if it exceeds the maximum size
|
||||
if len(c.messageHistory) > maxMessageHistory {
|
||||
c.messageHistory = c.messageHistory[len(c.messageHistory)-maxMessageHistory:]
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) periodicBufferSweep() {
|
||||
defer c.wg.Done()
|
||||
|
||||
ticker := time.NewTicker(bufferSweepInterval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
// Process incoming buffer
|
||||
c.processBufferedMessages()
|
||||
|
||||
// Resend unacknowledged messages from outgoing buffer
|
||||
c.checkUnacknowledgedMessages()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) checkUnacknowledgedMessages() {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
|
||||
now := time.Now()
|
||||
for i := 0; i < len(c.outgoingBuffer); i++ {
|
||||
unackMsg := c.outgoingBuffer[i]
|
||||
if now.Sub(unackMsg.SendTime) > ackTimeout {
|
||||
if unackMsg.ResendAttempts < maxResendAttempts {
|
||||
c.resendMessage(unackMsg.Message, unackMsg.ResendAttempts)
|
||||
c.outgoingBuffer[i].ResendAttempts++
|
||||
c.outgoingBuffer[i].SendTime = now
|
||||
} else {
|
||||
// Remove the message from the buffer after max attempts
|
||||
c.outgoingBuffer = append(c.outgoingBuffer[:i], c.outgoingBuffer[i+1:]...)
|
||||
i-- // Adjust index after removal
|
||||
c.ui.ErrorMessage(fmt.Errorf("message %s dropped: failed to be acknowledged after %d attempts", unackMsg.Message.Content, maxResendAttempts))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) resendMessage(msg *pb.Message, resendAttempts int) {
|
||||
go func() {
|
||||
delay := resendBaseDelay * time.Duration(1<<uint(resendAttempts))
|
||||
if delay > maxResendDelay {
|
||||
delay = maxResendDelay
|
||||
}
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
return
|
||||
case <-time.After(delay):
|
||||
// do nothing
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(c.ctx, ackTimeout)
|
||||
defer cancel()
|
||||
|
||||
err := c.publish(ctx, msg)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(fmt.Errorf("failed to resend message: %w", err))
|
||||
}
|
||||
}()
|
||||
}
|
||||
|
||||
func (c *Chat) periodicSyncMessage() {
|
||||
defer c.wg.Done()
|
||||
|
||||
ticker := time.NewTicker(syncMessageInterval)
|
||||
defer ticker.Stop()
|
||||
|
||||
for {
|
||||
select {
|
||||
case <-c.ctx.Done():
|
||||
return
|
||||
case <-ticker.C:
|
||||
c.sendSyncMessage()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) sendSyncMessage() {
|
||||
bloomBytes, err := c.bloomFilter.MarshalBinary()
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(fmt.Errorf("failed to marshal bloom filter: %w", err))
|
||||
return
|
||||
}
|
||||
|
||||
syncMsg := &pb.Message{
|
||||
SenderId: c.node.Host().ID().String(),
|
||||
MessageId: generateUniqueID(),
|
||||
LamportTimestamp: c.getLamportTimestamp(),
|
||||
CausalHistory: c.getRecentMessageIDs(10),
|
||||
ChannelId: c.options.ContentTopic,
|
||||
BloomFilter: bloomBytes,
|
||||
Content: "", // Empty content for sync messages
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(c.ctx, messageAckTimeout)
|
||||
defer cancel()
|
||||
|
||||
err = c.publish(ctx, syncMsg)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(fmt.Errorf("failed to send sync message: %w", err))
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) addToIncomingBuffer(msg *pb.Message) {
|
||||
c.mutex.Lock()
|
||||
defer c.mutex.Unlock()
|
||||
|
||||
c.incomingBuffer = append(c.incomingBuffer, msg)
|
||||
}
|
||||
|
||||
func (c *Chat) incLamportTimestamp() {
|
||||
c.lamportTSMutex.Lock()
|
||||
defer c.lamportTSMutex.Unlock()
|
||||
c.lamportTimestamp++
|
||||
}
|
||||
|
||||
func (c *Chat) updateLamportTimestamp(msgTs int32) {
|
||||
c.lamportTSMutex.Lock()
|
||||
defer c.lamportTSMutex.Unlock()
|
||||
if msgTs > c.lamportTimestamp {
|
||||
c.lamportTimestamp = msgTs
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) getLamportTimestamp() int32 {
|
||||
c.lamportTSMutex.Lock()
|
||||
defer c.lamportTSMutex.Unlock()
|
||||
return c.lamportTimestamp
|
||||
}
|
79
examples/chat2-reliable/rolling_bloom_filter.go
Normal file
79
examples/chat2-reliable/rolling_bloom_filter.go
Normal file
@ -0,0 +1,79 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/bits-and-blooms/bloom/v3"
|
||||
)
|
||||
|
||||
type TimestampedMessageID struct {
|
||||
ID string
|
||||
Timestamp time.Time
|
||||
}
|
||||
|
||||
type RollingBloomFilter struct {
|
||||
filter *bloom.BloomFilter
|
||||
window time.Duration
|
||||
messages []TimestampedMessageID
|
||||
mutex sync.Mutex
|
||||
}
|
||||
|
||||
func NewRollingBloomFilter() *RollingBloomFilter {
|
||||
return &RollingBloomFilter{
|
||||
filter: bloom.NewWithEstimates(bloomFilterSize, bloomFilterFPRate),
|
||||
window: bloomFilterWindow,
|
||||
messages: make([]TimestampedMessageID, 0),
|
||||
}
|
||||
}
|
||||
|
||||
func (rbf *RollingBloomFilter) Add(messageID string) {
|
||||
rbf.mutex.Lock()
|
||||
defer rbf.mutex.Unlock()
|
||||
|
||||
rbf.filter.Add([]byte(messageID))
|
||||
rbf.messages = append(rbf.messages, TimestampedMessageID{
|
||||
ID: messageID,
|
||||
Timestamp: time.Now(),
|
||||
})
|
||||
}
|
||||
|
||||
func (rbf *RollingBloomFilter) Test(messageID string) bool {
|
||||
rbf.mutex.Lock()
|
||||
defer rbf.mutex.Unlock()
|
||||
|
||||
return rbf.filter.Test([]byte(messageID))
|
||||
}
|
||||
|
||||
func (rbf *RollingBloomFilter) Clean() {
|
||||
rbf.mutex.Lock()
|
||||
defer rbf.mutex.Unlock()
|
||||
|
||||
cutoff := time.Now().Add(-rbf.window)
|
||||
newMessages := make([]TimestampedMessageID, 0)
|
||||
newFilter := bloom.NewWithEstimates(bloomFilterSize, bloomFilterFPRate)
|
||||
|
||||
for _, msg := range rbf.messages {
|
||||
if msg.Timestamp.After(cutoff) {
|
||||
newMessages = append(newMessages, msg)
|
||||
newFilter.Add([]byte(msg.ID))
|
||||
}
|
||||
}
|
||||
|
||||
rbf.messages = newMessages
|
||||
rbf.filter = newFilter
|
||||
}
|
||||
|
||||
// MarshalBinary implements the encoding.BinaryMarshaler interface for RollingBloomFilter
|
||||
func (rbf *RollingBloomFilter) MarshalBinary() ([]byte, error) {
|
||||
rbf.mutex.Lock()
|
||||
defer rbf.mutex.Unlock()
|
||||
return rbf.filter.MarshalBinary()
|
||||
}
|
||||
|
||||
// UnmarshalBinary implements the encoding.BinaryUnmarshaler interface for RollingBloomFilter
|
||||
func (rbf *RollingBloomFilter) UnmarshalBinary(data []byte) error {
|
||||
rbf.mutex.Lock()
|
||||
defer rbf.mutex.Unlock()
|
||||
return rbf.filter.UnmarshalBinary(data)
|
||||
}
|
15
examples/chat2-reliable/screen.go
Normal file
15
examples/chat2-reliable/screen.go
Normal file
@ -0,0 +1,15 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"os"
|
||||
|
||||
"golang.org/x/term"
|
||||
)
|
||||
|
||||
func GetTerminalDimensions() (int, int) {
|
||||
physicalWidth, physicalHeight, err := term.GetSize(int(os.Stdout.Fd()))
|
||||
if err != nil {
|
||||
panic("Could not determine terminal size")
|
||||
}
|
||||
return physicalWidth, physicalHeight
|
||||
}
|
71
examples/chat2-reliable/test_utils.go
Normal file
71
examples/chat2-reliable/test_utils.go
Normal file
@ -0,0 +1,71 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"github.com/libp2p/go-libp2p/core/host"
|
||||
"github.com/libp2p/go-libp2p/core/network"
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/waku-org/go-waku/waku/v2/node"
|
||||
)
|
||||
|
||||
type TestNetworkController struct {
|
||||
nodes []*node.WakuNode
|
||||
chats []*Chat
|
||||
mu sync.Mutex
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
func NewNetworkController(ctx context.Context, nodes []*node.WakuNode, chats []*Chat) *TestNetworkController {
|
||||
return &TestNetworkController{
|
||||
nodes: nodes,
|
||||
chats: chats,
|
||||
ctx: ctx,
|
||||
}
|
||||
}
|
||||
|
||||
func (nc *TestNetworkController) DisconnectNode(node *node.WakuNode) {
|
||||
nc.mu.Lock()
|
||||
defer nc.mu.Unlock()
|
||||
|
||||
for _, other := range nc.nodes {
|
||||
if node != other {
|
||||
nc.disconnectPeers(node.Host(), other.Host())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (nc *TestNetworkController) ReconnectNode(node *node.WakuNode) {
|
||||
nc.mu.Lock()
|
||||
defer nc.mu.Unlock()
|
||||
|
||||
for _, other := range nc.nodes {
|
||||
if node != other && !nc.IsConnected(node, other) {
|
||||
nc.connectPeers(node.Host(), other.Host())
|
||||
fmt.Printf("Reconnected node %s to node %s\n", node.Host().ID().String(), other.Host().ID().String())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (nc *TestNetworkController) disconnectPeers(h1, h2 host.Host) {
|
||||
h1.Network().ClosePeer(h2.ID())
|
||||
h2.Network().ClosePeer(h1.ID())
|
||||
}
|
||||
|
||||
func (nc *TestNetworkController) connectPeers(h1, h2 host.Host) {
|
||||
_, err := h1.Network().DialPeer(nc.ctx, h2.ID())
|
||||
if err != nil {
|
||||
fmt.Printf("Error connecting peers: %v\n", err)
|
||||
}
|
||||
}
|
||||
|
||||
func (nc *TestNetworkController) IsConnected(n1, n2 *node.WakuNode) bool {
|
||||
peerID, err := peer.Decode(n2.ID())
|
||||
if err != nil {
|
||||
fmt.Printf("Error decoding peer ID: %v\n", err)
|
||||
return false
|
||||
}
|
||||
return n1.Host().Network().Connectedness(peerID) == network.Connected
|
||||
}
|
344
examples/chat2-reliable/ui.go
Normal file
344
examples/chat2-reliable/ui.go
Normal file
@ -0,0 +1,344 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/charmbracelet/bubbles/key"
|
||||
"github.com/charmbracelet/bubbles/spinner"
|
||||
"github.com/charmbracelet/bubbles/textarea"
|
||||
"github.com/charmbracelet/bubbles/viewport"
|
||||
tea "github.com/charmbracelet/bubbletea"
|
||||
"github.com/charmbracelet/lipgloss"
|
||||
"github.com/muesli/reflow/wordwrap"
|
||||
"github.com/waku-org/go-waku/waku/v2/utils"
|
||||
)
|
||||
|
||||
const viewportMargin = 6
|
||||
|
||||
var (
|
||||
appStyle = lipgloss.NewStyle().Padding(1, 2)
|
||||
|
||||
titleStyle = func() lipgloss.Style {
|
||||
b := lipgloss.RoundedBorder()
|
||||
b.Right = "├"
|
||||
return lipgloss.NewStyle().BorderStyle(b).Padding(0, 1)
|
||||
}().Render
|
||||
|
||||
errorStyle = lipgloss.NewStyle().Foreground(lipgloss.Color("9")).Render
|
||||
infoStyle = lipgloss.NewStyle().Foreground(lipgloss.Color("4")).Render
|
||||
senderStyle = lipgloss.NewStyle().Foreground(lipgloss.Color("5")).Render
|
||||
)
|
||||
|
||||
type errMsg error
|
||||
|
||||
type sending bool
|
||||
|
||||
type quit bool
|
||||
|
||||
type MessageType int
|
||||
|
||||
const (
|
||||
ChatMessageType MessageType = iota
|
||||
InfoMessageType
|
||||
ErrorMessageType
|
||||
)
|
||||
|
||||
type message struct {
|
||||
mType MessageType
|
||||
err error
|
||||
author string
|
||||
clock time.Time
|
||||
content string
|
||||
}
|
||||
|
||||
type UI struct {
|
||||
ready bool
|
||||
err error
|
||||
|
||||
quitChan chan struct{}
|
||||
readyChan chan<- struct{}
|
||||
inputChan chan<- string
|
||||
|
||||
messageChan chan message
|
||||
messages []message
|
||||
|
||||
isSendingChan chan sending
|
||||
isSending bool
|
||||
|
||||
width int
|
||||
height int
|
||||
|
||||
viewport viewport.Model
|
||||
textarea textarea.Model
|
||||
|
||||
spinner spinner.Model
|
||||
}
|
||||
|
||||
func NewUIModel(readyChan chan<- struct{}, inputChan chan<- string) UI {
|
||||
width, height := GetTerminalDimensions()
|
||||
|
||||
ta := textarea.New()
|
||||
ta.Placeholder = "Send a message..."
|
||||
ta.Focus()
|
||||
|
||||
ta.Prompt = "┃ "
|
||||
ta.CharLimit = 2000
|
||||
|
||||
// Remove cursor line styling
|
||||
ta.FocusedStyle.CursorLine = lipgloss.NewStyle()
|
||||
ta.SetHeight(3)
|
||||
ta.SetWidth(width)
|
||||
ta.ShowLineNumbers = false
|
||||
|
||||
ta.KeyMap.InsertNewline.SetEnabled(false)
|
||||
|
||||
s := spinner.New()
|
||||
s.Spinner = spinner.Jump
|
||||
s.Style = lipgloss.NewStyle().Foreground(lipgloss.Color("205"))
|
||||
|
||||
m := UI{
|
||||
messageChan: make(chan message, 100),
|
||||
isSendingChan: make(chan sending, 100),
|
||||
quitChan: make(chan struct{}),
|
||||
readyChan: readyChan,
|
||||
inputChan: inputChan,
|
||||
width: width,
|
||||
height: height,
|
||||
textarea: ta,
|
||||
spinner: s,
|
||||
err: nil,
|
||||
}
|
||||
|
||||
return m
|
||||
}
|
||||
|
||||
func (m UI) Init() tea.Cmd {
|
||||
return tea.Batch(
|
||||
recvQuitSignal(m.quitChan),
|
||||
recvMessages(m.messageChan),
|
||||
recvSendingState(m.isSendingChan),
|
||||
textarea.Blink,
|
||||
spinner.Tick,
|
||||
)
|
||||
}
|
||||
|
||||
func (m UI) Update(msg tea.Msg) (tea.Model, tea.Cmd) {
|
||||
var (
|
||||
tiCmd tea.Cmd
|
||||
vpCmd tea.Cmd
|
||||
)
|
||||
|
||||
m.textarea, tiCmd = m.textarea.Update(msg)
|
||||
m.viewport, vpCmd = m.viewport.Update(msg)
|
||||
|
||||
var cmdToReturn []tea.Cmd = []tea.Cmd{tiCmd, vpCmd}
|
||||
|
||||
headerHeight := lipgloss.Height(m.headerView())
|
||||
|
||||
printMessages := false
|
||||
|
||||
switch msg := msg.(type) {
|
||||
|
||||
case tea.WindowSizeMsg:
|
||||
m.width, m.height = msg.Width, msg.Height
|
||||
|
||||
if !m.ready {
|
||||
// Since this program is using the full size of the viewport we
|
||||
// need to wait until we've received the window dimensions before
|
||||
// we can initialize the viewport. The initial dimensions come in
|
||||
// quickly, though asynchronously, which is why we wait for them
|
||||
// here.
|
||||
m.viewport = viewport.New(msg.Width, msg.Height-headerHeight-viewportMargin)
|
||||
m.viewport.SetContent("")
|
||||
m.viewport.YPosition = headerHeight + 1
|
||||
m.viewport.KeyMap = DefaultKeyMap()
|
||||
m.ready = true
|
||||
|
||||
close(m.readyChan)
|
||||
} else {
|
||||
m.viewport.Width = msg.Width
|
||||
m.viewport.Height = msg.Height - headerHeight - viewportMargin
|
||||
}
|
||||
|
||||
printMessages = true
|
||||
|
||||
case tea.KeyMsg:
|
||||
switch msg.Type {
|
||||
case tea.KeyCtrlC, tea.KeyEsc:
|
||||
return m, tea.Quit
|
||||
case tea.KeyEnter:
|
||||
line := m.textarea.Value()
|
||||
if len(line) != 0 {
|
||||
m.inputChan <- line
|
||||
m.textarea.Reset()
|
||||
}
|
||||
}
|
||||
|
||||
// We handle errors just like any other message
|
||||
case errMsg:
|
||||
m.err = msg
|
||||
return m, nil
|
||||
|
||||
case message:
|
||||
m.messages = append(m.messages, msg)
|
||||
printMessages = true
|
||||
cmdToReturn = append(cmdToReturn, recvMessages(m.messageChan))
|
||||
|
||||
case quit:
|
||||
fmt.Println("Bye!")
|
||||
return m, tea.Quit
|
||||
|
||||
case sending:
|
||||
m.isSending = bool(msg)
|
||||
cmdToReturn = append(cmdToReturn, recvSendingState(m.isSendingChan))
|
||||
|
||||
case spinner.TickMsg:
|
||||
var cmd tea.Cmd
|
||||
m.spinner, cmd = m.spinner.Update(msg)
|
||||
return m, cmd
|
||||
}
|
||||
|
||||
if printMessages {
|
||||
var sb strings.Builder
|
||||
for i, msg := range m.messages {
|
||||
line := ""
|
||||
|
||||
switch msg.mType {
|
||||
case ChatMessageType:
|
||||
line += m.breaklineIfNeeded(i, ChatMessageType)
|
||||
msgLine := "[" + msg.clock.Format("Jan 02 15:04") + " " + senderStyle(msg.author) + "] "
|
||||
msgLine += msg.content
|
||||
line += wordwrap.String(line+msgLine, m.width-10)
|
||||
case ErrorMessageType:
|
||||
line += m.breaklineIfNeeded(i, ErrorMessageType)
|
||||
line += wordwrap.String(errorStyle("ERROR:")+" "+msg.err.Error(), m.width-10)
|
||||
utils.Logger().Error(msg.err.Error())
|
||||
case InfoMessageType:
|
||||
line += m.breaklineIfNeeded(i, InfoMessageType)
|
||||
line += wordwrap.String(infoStyle("INFO:")+" "+msg.content, m.width-10)
|
||||
utils.Logger().Info(msg.content)
|
||||
}
|
||||
|
||||
sb.WriteString(line + "\n")
|
||||
|
||||
}
|
||||
|
||||
m.viewport.SetContent(sb.String())
|
||||
m.viewport.GotoBottom()
|
||||
}
|
||||
|
||||
return m, tea.Batch(cmdToReturn...)
|
||||
}
|
||||
|
||||
func (m UI) breaklineIfNeeded(i int, mt MessageType) string {
|
||||
result := ""
|
||||
if i > 0 {
|
||||
if (mt == ChatMessageType && m.messages[i-1].mType != ChatMessageType) || (mt != ChatMessageType && m.messages[i-1].mType == ChatMessageType) {
|
||||
result += "\n"
|
||||
}
|
||||
}
|
||||
return result
|
||||
}
|
||||
|
||||
func (m UI) headerView() string {
|
||||
title := titleStyle("Chat2 •")
|
||||
line := strings.Repeat("─", max(0, m.viewport.Width-lipgloss.Width(title)-4))
|
||||
return lipgloss.JoinHorizontal(lipgloss.Center, title, line)
|
||||
}
|
||||
|
||||
func max(a, b int) int {
|
||||
if a > b {
|
||||
return a
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
||||
func (m UI) View() string {
|
||||
spinnerStr := ""
|
||||
inputStr := ""
|
||||
if m.isSending {
|
||||
spinnerStr = m.spinner.View() + " Sending message..."
|
||||
} else {
|
||||
inputStr = m.textarea.View()
|
||||
}
|
||||
|
||||
return appStyle.Render(fmt.Sprintf(
|
||||
"%s\n%s\n%s%s\n",
|
||||
m.headerView(),
|
||||
m.viewport.View(),
|
||||
inputStr,
|
||||
spinnerStr,
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
func recvMessages(sub chan message) tea.Cmd {
|
||||
return func() tea.Msg {
|
||||
return <-sub
|
||||
}
|
||||
}
|
||||
|
||||
func recvSendingState(sub chan sending) tea.Cmd {
|
||||
return func() tea.Msg {
|
||||
return <-sub
|
||||
}
|
||||
}
|
||||
|
||||
func recvQuitSignal(q chan struct{}) tea.Cmd {
|
||||
return func() tea.Msg {
|
||||
<-q
|
||||
return quit(true)
|
||||
}
|
||||
}
|
||||
|
||||
func (m UI) Quit() {
|
||||
m.quitChan <- struct{}{}
|
||||
}
|
||||
|
||||
func (m UI) SetSending(isSending bool) {
|
||||
m.isSendingChan <- sending(isSending)
|
||||
}
|
||||
|
||||
func (m UI) ErrorMessage(err error) {
|
||||
m.messageChan <- message{mType: ErrorMessageType, err: err}
|
||||
}
|
||||
|
||||
func (m UI) InfoMessage(text string) {
|
||||
m.messageChan <- message{mType: InfoMessageType, content: text}
|
||||
}
|
||||
|
||||
func (m UI) ChatMessage(clock int64, author string, text string) {
|
||||
m.messageChan <- message{mType: ChatMessageType, author: author, content: text, clock: time.Unix(clock, 0)}
|
||||
}
|
||||
|
||||
// DefaultKeyMap returns a set of pager-like default keybindings.
|
||||
func DefaultKeyMap() viewport.KeyMap {
|
||||
return viewport.KeyMap{
|
||||
PageDown: key.NewBinding(
|
||||
key.WithKeys("pgdown"),
|
||||
key.WithHelp("pgdn", "page down"),
|
||||
),
|
||||
PageUp: key.NewBinding(
|
||||
key.WithKeys("pgup"),
|
||||
key.WithHelp("pgup", "page up"),
|
||||
),
|
||||
HalfPageUp: key.NewBinding(
|
||||
key.WithKeys("ctrl+u"),
|
||||
key.WithHelp("ctrl+u", "½ page up"),
|
||||
),
|
||||
HalfPageDown: key.NewBinding(
|
||||
key.WithKeys("ctrl+d"),
|
||||
key.WithHelp("ctrl+d", "½ page down"),
|
||||
),
|
||||
Up: key.NewBinding(
|
||||
key.WithKeys("up"),
|
||||
key.WithHelp("↑", "up"),
|
||||
),
|
||||
Down: key.NewBinding(
|
||||
key.WithKeys("down"),
|
||||
key.WithHelp("↓", "down"),
|
||||
),
|
||||
}
|
||||
}
|
@ -6,6 +6,7 @@ import (
|
||||
logging "github.com/ipfs/go-log/v2"
|
||||
"github.com/urfave/cli/v2"
|
||||
"github.com/waku-org/go-waku/waku/v2/utils"
|
||||
"go.uber.org/zap/zapcore"
|
||||
)
|
||||
|
||||
var options Options
|
||||
@ -14,13 +15,13 @@ func main() {
|
||||
app := &cli.App{
|
||||
Flags: getFlags(),
|
||||
Action: func(c *cli.Context) error {
|
||||
utils.InitLogger("console", "file:chat2.log", "chat2")
|
||||
|
||||
lvl, err := logging.LevelFromString(options.LogLevel)
|
||||
lvl, err := zapcore.ParseLevel(options.LogLevel)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
logging.SetAllLoggers(lvl)
|
||||
|
||||
logging.SetAllLoggers(logging.LogLevel(lvl))
|
||||
utils.InitLogger("console", "file:chat2.log", "chat2", lvl)
|
||||
|
||||
execute(options)
|
||||
return nil
|
||||
|
@ -6,6 +6,7 @@ import (
|
||||
logging "github.com/ipfs/go-log/v2"
|
||||
|
||||
"go.uber.org/zap"
|
||||
"go.uber.org/zap/zapcore"
|
||||
)
|
||||
|
||||
var log *zap.Logger
|
||||
@ -19,7 +20,7 @@ func Logger(name ...string) *zap.Logger {
|
||||
}
|
||||
|
||||
if log == nil {
|
||||
InitLogger("console", "stdout", loggerName)
|
||||
InitLogger("console", "stdout", loggerName, zapcore.InfoLevel)
|
||||
}
|
||||
return log
|
||||
}
|
||||
@ -39,8 +40,9 @@ func MessagesLogger(prefix string) *zap.Logger {
|
||||
}
|
||||
|
||||
// InitLogger initializes a global logger using an specific encoding
|
||||
func InitLogger(encoding string, output string, name string) {
|
||||
func InitLogger(encoding string, output string, name string, level zapcore.Level) {
|
||||
cfg := logging.GetConfig()
|
||||
cfg.Level = logging.LogLevel(level)
|
||||
|
||||
if encoding == "json" {
|
||||
cfg.Format = logging.JSONOutput
|
||||
|
Loading…
x
Reference in New Issue
Block a user