mirror of https://github.com/status-im/go-waku.git
feat: chat2 refactor / redesign
This commit is contained in:
parent
8538013f29
commit
8cf011f718
|
@ -1,18 +1,9 @@
|
|||
.PHONY: all build fleets-remove fleets-update
|
||||
.PHONY: all build run
|
||||
|
||||
all: build
|
||||
|
||||
FLEETS := fleets.json
|
||||
$(FLEETS):
|
||||
echo -e $(BUILD_MSG) "Getting latest $(FLEETS)"
|
||||
curl -s https://fleets.status.im/ \
|
||||
| jq --indent 4 --sort-keys . \
|
||||
> $(FLEETS)
|
||||
build:
|
||||
go build -tags="gowaku_rln" -o build/chat2 .
|
||||
|
||||
fleets-remove:
|
||||
rm -f $(FLEETS)
|
||||
|
||||
fleets-update: fleets-remove $(FLEETS)
|
||||
|
||||
build: $(FLEETS)
|
||||
go build -o build/chat2 .
|
||||
run:
|
||||
./build/chat2
|
|
@ -17,18 +17,6 @@ To start the `chat2` application in its most basic form, run the following from
|
|||
./build/chat2
|
||||
```
|
||||
|
||||
The app will randomly select and connect to a peer from the test fleet.
|
||||
|
||||
```
|
||||
No static peers configured. Choosing one at random from test fleet...
|
||||
```
|
||||
|
||||
Wait for the chat prompt (`>`) and chat away!
|
||||
|
||||
## Retrieving historical messages
|
||||
|
||||
TODO
|
||||
|
||||
## Specifying a static peer
|
||||
|
||||
In order to connect to a *specific* node as [`relay`](https://specs.vac.dev/specs/waku/v2/waku-relay) peer, define that node's `multiaddr` as a `staticnode` when starting the app:
|
||||
|
|
|
@ -1,95 +1,252 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"chat2/pb"
|
||||
"chat3/pb"
|
||||
"context"
|
||||
"crypto/sha256"
|
||||
"encoding/hex"
|
||||
"errors"
|
||||
"fmt"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/libp2p/go-libp2p-core/peer"
|
||||
"github.com/multiformats/go-multiaddr"
|
||||
"github.com/status-im/go-rln/rln"
|
||||
"github.com/status-im/go-waku/waku/v2/dnsdisc"
|
||||
"github.com/status-im/go-waku/waku/v2/node"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/filter"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/lightpush"
|
||||
wpb "github.com/status-im/go-waku/waku/v2/protocol/pb"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/relay"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/store"
|
||||
"github.com/status-im/go-waku/waku/v2/utils"
|
||||
"golang.org/x/crypto/pbkdf2"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
// Chat represents a subscription to a single PubSub topic. Messages
|
||||
// can be published to the topic with Chat.Publish, and received
|
||||
// messages are pushed to the Messages channel.
|
||||
type Chat struct {
|
||||
// Messages is a channel of messages received from other peers in the chat room
|
||||
Messages chan *pb.Chat2Message
|
||||
ctx context.Context
|
||||
wg sync.WaitGroup
|
||||
node *node.WakuNode
|
||||
ui UI
|
||||
uiReady chan struct{}
|
||||
inputChan chan string
|
||||
options Options
|
||||
|
||||
C chan *protocol.Envelope
|
||||
node *node.WakuNode
|
||||
C chan *protocol.Envelope
|
||||
|
||||
self peer.ID
|
||||
contentTopic string
|
||||
useV1Payload bool
|
||||
useLightPush bool
|
||||
nick string
|
||||
spamChan chan *wpb.WakuMessage
|
||||
nick string
|
||||
}
|
||||
|
||||
// NewChat tries to subscribe to the PubSub topic for the room name, returning
|
||||
// a ChatRoom on success.
|
||||
func NewChat(ctx context.Context, n *node.WakuNode, selfID peer.ID, contentTopic string, useV1Payload bool, useLightPush bool, nickname string, spamChan chan *wpb.WakuMessage) (*Chat, error) {
|
||||
// join the default waku topic and subscribe to it
|
||||
|
||||
func NewChat(ctx context.Context, node *node.WakuNode, options Options) *Chat {
|
||||
chat := &Chat{
|
||||
node: n,
|
||||
self: selfID,
|
||||
contentTopic: contentTopic,
|
||||
nick: nickname,
|
||||
useV1Payload: useV1Payload,
|
||||
useLightPush: useLightPush,
|
||||
Messages: make(chan *pb.Chat2Message, 1024),
|
||||
spamChan: spamChan,
|
||||
ctx: ctx,
|
||||
node: node,
|
||||
options: options,
|
||||
nick: options.Nickname,
|
||||
uiReady: make(chan struct{}, 1),
|
||||
inputChan: make(chan string, 100),
|
||||
}
|
||||
|
||||
if useLightPush {
|
||||
chat.ui = NewUIModel(chat.uiReady, chat.inputChan)
|
||||
|
||||
if options.Filter.Enable {
|
||||
cf := filter.ContentFilter{
|
||||
Topic: relay.DefaultWakuTopic,
|
||||
ContentTopics: []string{contentTopic},
|
||||
ContentTopics: []string{options.ContentTopic},
|
||||
}
|
||||
var err error
|
||||
_, theFilter, err := n.Filter().Subscribe(ctx, cf)
|
||||
chat.C = theFilter.Chan
|
||||
|
||||
var filterOpt filter.FilterSubscribeOption
|
||||
peerID, err := options.Filter.NodePeerID()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
filterOpt = filter.WithAutomaticPeerSelection()
|
||||
} else {
|
||||
filterOpt = filter.WithPeer(peerID)
|
||||
chat.ui.InfoMessage(fmt.Sprintf("Subscribing to filter node %s", peerID))
|
||||
}
|
||||
|
||||
_, theFilter, err := node.Filter().Subscribe(ctx, cf, filterOpt)
|
||||
if err != nil {
|
||||
chat.ui.ErrorMessage(err)
|
||||
} else {
|
||||
chat.C = theFilter.Chan
|
||||
}
|
||||
} else {
|
||||
sub, err := n.Relay().Subscribe(ctx)
|
||||
sub, err := node.Relay().Subscribe(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
chat.ui.ErrorMessage(err)
|
||||
} else {
|
||||
chat.C = sub.C
|
||||
}
|
||||
chat.C = sub.C
|
||||
}
|
||||
|
||||
// start reading messages from the subscription in a loop
|
||||
go chat.readLoop()
|
||||
chat.wg.Add(6)
|
||||
go chat.parseInput()
|
||||
go chat.receiveMessages()
|
||||
|
||||
go chat.readSpamMessages()
|
||||
connectionWg := sync.WaitGroup{}
|
||||
connectionWg.Add(2)
|
||||
|
||||
return chat, nil
|
||||
go chat.welcomeMessage()
|
||||
|
||||
go chat.staticNodes(&connectionWg)
|
||||
go chat.discoverNodes(&connectionWg)
|
||||
go chat.retrieveHistory(&connectionWg)
|
||||
|
||||
return chat
|
||||
}
|
||||
|
||||
func generateSymKey(password string) []byte {
|
||||
// AesKeyLength represents the length (in bytes) of an private key
|
||||
AESKeyLength := 256 / 8
|
||||
return pbkdf2.Key([]byte(password), nil, 65356, AESKeyLength, sha256.New)
|
||||
func (c *Chat) Stop() {
|
||||
c.wg.Wait()
|
||||
close(c.inputChan)
|
||||
}
|
||||
|
||||
// Publish sends a message to the pubsub topic.
|
||||
func (cr *Chat) Publish(ctx context.Context, message string) error {
|
||||
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 || (c.options.RLNRelay.Enable && msgContentTopic != c.options.RLNRelay.ContentTopic) {
|
||||
continue // Discard messages from other topics
|
||||
}
|
||||
|
||||
msg, err := decodeMessage(c.options.UsePayloadV1, c.options.ContentTopic, value.Message())
|
||||
if err == nil {
|
||||
// send valid messages to the UI
|
||||
c.ui.ChatMessage(int64(msg.Timestamp), msg.Nick, string(msg.Payload))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
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 == "/quit" {
|
||||
c.ui.Quit()
|
||||
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(5)*time.Second)
|
||||
defer cancel()
|
||||
|
||||
err = c.node.DialPeerWithMultiAddress(ctx, ma)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
} else {
|
||||
c.ui.InfoMessage(fmt.Sprintf("Connected to %s", peerID))
|
||||
}
|
||||
}(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(" Protocols: %s\n", strings.Join(peerProtocols, ", "))
|
||||
peerInfoMsg += " Addresses:\n"
|
||||
peerInfoMsg += fmt.Sprintf(" - %s:\n", p.Pretty())
|
||||
for _, addr := range peerInfo.Addrs {
|
||||
peerInfoMsg += fmt.Sprintf(" %s/p2p/%s\n", addr.String(), p.Pretty())
|
||||
}
|
||||
}
|
||||
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
|
||||
/quit - closes the app`)
|
||||
return
|
||||
}
|
||||
|
||||
c.SendMessage(line)
|
||||
}()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) SendMessage(line string) {
|
||||
tCtx, cancel := context.WithTimeout(c.ctx, 3*time.Second)
|
||||
defer func() {
|
||||
cancel()
|
||||
}()
|
||||
|
||||
err := c.publish(tCtx, line)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
}
|
||||
}
|
||||
|
||||
func (c *Chat) publish(ctx context.Context, message string) error {
|
||||
msg := &pb.Chat2Message{
|
||||
Timestamp: uint64(time.Now().Unix()),
|
||||
Nick: cr.nick,
|
||||
Nick: c.nick,
|
||||
Payload: []byte(message),
|
||||
}
|
||||
|
||||
|
@ -103,9 +260,9 @@ func (cr *Chat) Publish(ctx context.Context, message string) error {
|
|||
var timestamp int64 = utils.GetUnixEpochFrom(t)
|
||||
var keyInfo *node.KeyInfo = &node.KeyInfo{}
|
||||
|
||||
if cr.useV1Payload { // Use WakuV1 encryption
|
||||
if c.options.UsePayloadV1 { // Use WakuV1 encryption
|
||||
keyInfo.Kind = node.Symmetric
|
||||
keyInfo.SymKey = generateSymKey(cr.contentTopic)
|
||||
keyInfo.SymKey = generateSymKey(c.options.ContentTopic)
|
||||
version = 1
|
||||
} else {
|
||||
keyInfo.Kind = node.None
|
||||
|
@ -124,31 +281,40 @@ func (cr *Chat) Publish(ctx context.Context, message string) error {
|
|||
wakuMsg := &wpb.WakuMessage{
|
||||
Payload: payload,
|
||||
Version: version,
|
||||
ContentTopic: cr.contentTopic,
|
||||
ContentTopic: options.ContentTopic,
|
||||
Timestamp: timestamp,
|
||||
}
|
||||
|
||||
if cr.node.RLNRelay() != nil {
|
||||
if c.options.RLNRelay.Enable {
|
||||
// for future version when we support more than one rln protected content topic,
|
||||
// we should check the message content topic as well
|
||||
err = cr.node.RLNRelay().AppendRLNProof(wakuMsg, t)
|
||||
err = c.node.RLNRelay().AppendRLNProof(wakuMsg, t)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
c.ui.InfoMessage(fmt.Sprintf("RLN Epoch: %d", rln.BytesToEpoch(wakuMsg.RateLimitProof.Epoch).Uint64()))
|
||||
}
|
||||
|
||||
if cr.useLightPush {
|
||||
_, err = cr.node.Lightpush().Publish(ctx, wakuMsg)
|
||||
if c.options.LightPush.Enable {
|
||||
var lightOpt lightpush.LightPushOption
|
||||
var peerID peer.ID
|
||||
peerID, err = options.LightPush.NodePeerID()
|
||||
if err != nil {
|
||||
lightOpt = lightpush.WithAutomaticPeerSelection()
|
||||
} else {
|
||||
lightOpt = lightpush.WithPeer(peerID)
|
||||
}
|
||||
|
||||
_, err = c.node.Lightpush().Publish(c.ctx, wakuMsg, lightOpt)
|
||||
} else {
|
||||
_, err = cr.node.Relay().Publish(ctx, wakuMsg)
|
||||
|
||||
_, err = c.node.Relay().Publish(ctx, wakuMsg)
|
||||
}
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
func DecodeMessage(useV1Payload bool, contentTopic string, wakumsg *wpb.WakuMessage) (*pb.Chat2Message, error) {
|
||||
func decodeMessage(useV1Payload bool, contentTopic string, wakumsg *wpb.WakuMessage) (*pb.Chat2Message, error) {
|
||||
var keyInfo *node.KeyInfo = &node.KeyInfo{}
|
||||
if useV1Payload { // Use WakuV1 encryption
|
||||
keyInfo.Kind = node.Symmetric
|
||||
|
@ -170,34 +336,180 @@ func DecodeMessage(useV1Payload bool, contentTopic string, wakumsg *wpb.WakuMess
|
|||
return msg, nil
|
||||
}
|
||||
|
||||
// readLoop pulls messages from the pubsub topic and pushes them onto the Messages channel.
|
||||
func (cr *Chat) readLoop() {
|
||||
for value := range cr.C {
|
||||
msg, err := DecodeMessage(cr.useV1Payload, cr.contentTopic, value.Message())
|
||||
if err == nil {
|
||||
// send valid messages onto the Messages channel
|
||||
cr.Messages <- msg
|
||||
func generateSymKey(password string) []byte {
|
||||
// AesKeyLength represents the length (in bytes) of an private key
|
||||
AESKeyLength := 256 / 8
|
||||
return pbkdf2.Key([]byte(password), nil, 65356, AESKeyLength, sha256.New)
|
||||
}
|
||||
|
||||
func (c *Chat) retrieveHistory(connectionWg *sync.WaitGroup) {
|
||||
defer c.wg.Done()
|
||||
|
||||
connectionWg.Wait() // Wait until node connection operations are done
|
||||
|
||||
if !c.options.Store.Enable {
|
||||
return
|
||||
}
|
||||
|
||||
var storeOpt store.HistoryRequestOption
|
||||
if c.options.Store.Node == nil {
|
||||
c.ui.InfoMessage("No store node configured. Choosing one at random...")
|
||||
storeOpt = store.WithAutomaticPeerSelection()
|
||||
} else {
|
||||
peerID, err := (*c.options.Store.Node).ValueForProtocol(multiaddr.P_P2P)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
return
|
||||
}
|
||||
pID, err := peer.Decode(peerID)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
return
|
||||
}
|
||||
storeOpt = store.WithPeer(pID)
|
||||
c.ui.InfoMessage(fmt.Sprintf("Querying historic messages from %s", peerID))
|
||||
|
||||
}
|
||||
|
||||
tCtx, cancel := context.WithTimeout(c.ctx, 10*time.Second)
|
||||
defer cancel()
|
||||
|
||||
q := store.Query{
|
||||
ContentTopics: []string{options.ContentTopic},
|
||||
}
|
||||
|
||||
response, err := c.node.Store().Query(tCtx, q,
|
||||
store.WithAutomaticRequestId(),
|
||||
storeOpt,
|
||||
store.WithPaging(true, 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, msg.Timestamp, relay.DefaultWakuTopic)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// readSpam prints messages that are spam (to demonstrate RLN functionality)
|
||||
func (cr *Chat) readSpamMessages() {
|
||||
for value := range cr.C {
|
||||
msg, err := DecodeMessage(cr.useV1Payload, cr.contentTopic, value.Message())
|
||||
if err == nil {
|
||||
msg.Payload = append([]byte("Spam message received and discarded: "), msg.Payload...)
|
||||
cr.Messages <- msg
|
||||
}
|
||||
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(options.StaticNodes))
|
||||
for _, n := range options.StaticNodes {
|
||||
go func(addr multiaddr.Multiaddr) {
|
||||
defer wg.Done()
|
||||
ctx, cancel := context.WithTimeout(c.ctx, time.Duration(3)*time.Second)
|
||||
defer cancel()
|
||||
|
||||
peerID, err := addr.ValueForProtocol(multiaddr.P_P2P)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
return
|
||||
}
|
||||
|
||||
c.ui.InfoMessage(fmt.Sprintf("Connecting to %s", addr.String()))
|
||||
|
||||
err = c.node.DialPeerWithMultiAddress(ctx, addr)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
} else {
|
||||
c.ui.InfoMessage(fmt.Sprintf("Connected to %s", peerID))
|
||||
}
|
||||
}(n)
|
||||
}
|
||||
|
||||
wg.Wait()
|
||||
|
||||
}
|
||||
|
||||
func (cr *Chat) displayMessages(messages []*wpb.WakuMessage) {
|
||||
for _, msg := range messages {
|
||||
msg, err := DecodeMessage(cr.useV1Payload, cr.contentTopic, msg)
|
||||
if err == nil {
|
||||
// send valid messages onto the Messages channel
|
||||
cr.Messages <- msg
|
||||
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")
|
||||
|
||||
if !c.options.RLNRelay.Enable {
|
||||
return
|
||||
}
|
||||
|
||||
idKey := c.node.RLNRelay().MembershipKeyPair().IDKey
|
||||
idCommitment := c.node.RLNRelay().MembershipKeyPair().IDCommitment
|
||||
|
||||
rlnMessage := "RLN config:\n"
|
||||
rlnMessage += fmt.Sprintf("- Your membership index is: %d\n", uint(c.node.RLNRelay().MembershipIndex()))
|
||||
rlnMessage += fmt.Sprintf("- Your rln identity key is: 0x%s\n", hex.EncodeToString(idKey[:]))
|
||||
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 options.Fleet != fleetNone {
|
||||
if options.Fleet == fleetTest {
|
||||
dnsDiscoveryUrl = "enrtree://AOFTICU2XWDULNLZGRMQS4RIZPAZEHYMV4FYHAPW563HNRAOERP7C@test.waku.nodes.status.im"
|
||||
} else {
|
||||
// Connect to prod by default
|
||||
dnsDiscoveryUrl = "enrtree://ANTL4SLG2COUILKAPE7EF2BYNL2SHSHVCHLRD5J7ZJLN5R3PRJD2Y@prod.waku.nodes.status.im"
|
||||
}
|
||||
}
|
||||
|
||||
if options.DNSDiscovery.Enable && options.DNSDiscovery.URL != "" {
|
||||
dnsDiscoveryUrl = 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(options.DNSDiscovery.Nameserver))
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(errors.New(err.Error()))
|
||||
} else {
|
||||
var nodeList []multiaddr.Multiaddr
|
||||
for _, n := range nodes {
|
||||
nodeList = append(nodeList, n.Addresses...)
|
||||
}
|
||||
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, addr multiaddr.Multiaddr) {
|
||||
defer wg.Done()
|
||||
|
||||
peerID, err := addr.ValueForProtocol(multiaddr.P_P2P)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(err)
|
||||
return
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(ctx, time.Duration(3)*time.Second)
|
||||
defer cancel()
|
||||
err = c.node.DialPeerWithMultiAddress(ctx, addr)
|
||||
if err != nil {
|
||||
c.ui.ErrorMessage(fmt.Errorf("could not connect to %s: %w", peerID, err))
|
||||
} else {
|
||||
c.ui.InfoMessage(fmt.Sprintf("Connected to %s", peerID))
|
||||
}
|
||||
}(c.ctx, n)
|
||||
|
||||
}
|
||||
wg.Wait()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,139 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"net"
|
||||
|
||||
tea "github.com/charmbracelet/bubbletea"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
"github.com/multiformats/go-multiaddr"
|
||||
"github.com/status-im/go-waku/waku/v2/node"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/filter"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/lightpush"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/pb"
|
||||
"github.com/status-im/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
|
||||
}
|
||||
}
|
||||
|
||||
opts := []node.WakuNodeOption{
|
||||
node.WithPrivateKey(options.NodeKey),
|
||||
node.WithHostAddress(hostAddr),
|
||||
node.WithWakuStore(false, false),
|
||||
}
|
||||
|
||||
if options.Relay.Enable {
|
||||
opts = append(opts, node.WithWakuRelay())
|
||||
}
|
||||
|
||||
if options.RLNRelay.Enable {
|
||||
spamHandler := func(message *pb.WakuMessage) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
if options.RLNRelay.Dynamic {
|
||||
idKey, idCommitment, index, err := getMembershipCredentials(options.RLNRelay.CredentialsFile, options.RLNRelay.IDKey, options.RLNRelay.IDCommitment, options.RLNRelay.MembershipIndex)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
fmt.Println("Setting up dynamic rln")
|
||||
opts = append(opts, node.WithDynamicRLNRelay(
|
||||
options.RLNRelay.PubsubTopic,
|
||||
options.RLNRelay.ContentTopic,
|
||||
index,
|
||||
idKey,
|
||||
idCommitment,
|
||||
spamHandler,
|
||||
options.RLNRelay.ETHClientAddress,
|
||||
options.RLNRelay.ETHPrivateKey,
|
||||
options.RLNRelay.MembershipContractAddress,
|
||||
))
|
||||
} else {
|
||||
opts = append(opts, node.WithStaticRLNRelay(
|
||||
options.RLNRelay.PubsubTopic,
|
||||
options.RLNRelay.ContentTopic,
|
||||
uint(options.RLNRelay.MembershipIndex),
|
||||
spamHandler))
|
||||
}
|
||||
}
|
||||
|
||||
if options.Filter.Enable {
|
||||
opts = append(opts, node.WithWakuFilter(false))
|
||||
}
|
||||
|
||||
if options.LightPush.Enable {
|
||||
opts = append(opts, node.WithLightPush())
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
defer cancel()
|
||||
|
||||
wakuNode, err := node.New(ctx, opts...)
|
||||
if err != nil {
|
||||
fmt.Print(err)
|
||||
return
|
||||
}
|
||||
|
||||
err = addPeer(wakuNode, options.Store.Node, string(store.StoreID_v20beta4))
|
||||
if err != nil {
|
||||
fmt.Print(err)
|
||||
return
|
||||
}
|
||||
|
||||
err = addPeer(wakuNode, options.LightPush.Node, string(lightpush.LightPushID_v20beta1))
|
||||
if err != nil {
|
||||
fmt.Print(err)
|
||||
return
|
||||
}
|
||||
|
||||
err = addPeer(wakuNode, options.Filter.Node, string(filter.FilterID_v20beta1))
|
||||
if err != nil {
|
||||
fmt.Print(err)
|
||||
return
|
||||
}
|
||||
|
||||
if err := wakuNode.Start(); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if options.RLNRelay.Enable && options.RLNRelay.Dynamic {
|
||||
if options.RLNRelay.IDKey == "" && options.RLNRelay.IDCommitment == "" {
|
||||
// Write membership credentials file only if the idkey and commitment are not specified
|
||||
err := writeRLNMembershipCredentialsToFile(options.RLNRelay.CredentialsFile, wakuNode.RLNRelay().MembershipKeyPair(), wakuNode.RLNRelay().MembershipIndex())
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
chat := NewChat(ctx, wakuNode, options)
|
||||
p := tea.NewProgram(chat.ui)
|
||||
if err := p.Start(); err != nil {
|
||||
fmt.Println(err)
|
||||
}
|
||||
|
||||
cancel()
|
||||
|
||||
wakuNode.Stop()
|
||||
chat.Stop()
|
||||
}
|
||||
|
||||
func addPeer(wakuNode *node.WakuNode, addr *multiaddr.Multiaddr, protocols ...string) error {
|
||||
if addr == nil {
|
||||
return nil
|
||||
}
|
||||
_, err := wakuNode.AddPeer(*addr, protocols...)
|
||||
return err
|
||||
}
|
|
@ -0,0 +1,245 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
wcli "github.com/status-im/go-waku/waku/cliutils"
|
||||
"github.com/status-im/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
|
||||
|
||||
return []cli.Flag{
|
||||
&cli.IntFlag{
|
||||
Name: "tcp-port",
|
||||
Aliases: []string{"port", "p"},
|
||||
Value: 60000,
|
||||
Usage: "Libp2p TCP listening port (0 for random)",
|
||||
Destination: &options.Port,
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "address",
|
||||
Aliases: []string{"host", "listen-address"},
|
||||
Value: "0.0.0.0",
|
||||
Usage: "Listening address",
|
||||
Destination: &options.Address,
|
||||
},
|
||||
&cli.GenericFlag{
|
||||
Name: "nodekey",
|
||||
Usage: "P2P node private key as hex. (default random)",
|
||||
Value: &wcli.PrivateKeyValue{
|
||||
Value: &options.NodeKey,
|
||||
},
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "content-topic",
|
||||
Usage: "content topic to use for the chat",
|
||||
Value: protocol.NewContentTopic("toy-chat", 2, "luzhou", "proto").String(),
|
||||
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.BoolFlag{
|
||||
Name: "payloadV1",
|
||||
Value: false,
|
||||
Usage: "use Waku v1 payload encoding/encryption",
|
||||
Destination: &options.UsePayloadV1,
|
||||
},
|
||||
&cli.StringSliceFlag{
|
||||
Name: "topics",
|
||||
Usage: "List of topics to listen",
|
||||
Destination: &options.Relay.Topics,
|
||||
},
|
||||
&cli.BoolFlag{
|
||||
Name: "store",
|
||||
Usage: "Enable relay 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.IntFlag{
|
||||
Name: "discv5-udp-port",
|
||||
Value: 9000,
|
||||
Usage: "Listening UDP port for Node Discovery v5.",
|
||||
Destination: &options.DiscV5.Port,
|
||||
},
|
||||
&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.IntFlag{
|
||||
Name: "rln-relay-membership-index",
|
||||
Value: 0,
|
||||
Usage: "(experimental) the index of node in the rln-relay group: a value between 0-99 inclusive",
|
||||
Destination: &options.RLNRelay.MembershipIndex,
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "rln-relay-pubsub-topic",
|
||||
Value: "/waku/2/default-waku/proto",
|
||||
Usage: "the pubsub topic for which rln-relay gets enabled",
|
||||
Destination: &options.RLNRelay.PubsubTopic,
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "rln-relay-content-topic",
|
||||
Value: "/toy-chat/2/luzhou/proto",
|
||||
Usage: "the content topic for which rln-relay gets enabled",
|
||||
Destination: &options.RLNRelay.ContentTopic,
|
||||
},
|
||||
&cli.BoolFlag{
|
||||
Name: "rln-relay-dynamic",
|
||||
Usage: "Enable waku-rln-relay with on-chain dynamic group management",
|
||||
Destination: &options.RLNRelay.Dynamic,
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "rln-relay-id",
|
||||
Usage: "Rln relay identity secret key as a Hex string",
|
||||
Destination: &options.RLNRelay.IDKey,
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "rln-relay-id-commitment",
|
||||
Usage: "Rln relay identity commitment key as a Hex string",
|
||||
Destination: &options.RLNRelay.IDCommitment,
|
||||
},
|
||||
&cli.PathFlag{
|
||||
Name: "rln-relay-membership-credentials-file",
|
||||
Usage: "RLN relay membership credentials file",
|
||||
Value: "rlnCredentials.txt",
|
||||
Destination: &options.RLNRelay.CredentialsFile,
|
||||
},
|
||||
// TODO: this is a good candidate option for subcommands
|
||||
// TODO: consider accepting a private key file and passwd
|
||||
&cli.GenericFlag{
|
||||
Name: "eth-account-privatekey",
|
||||
Usage: "Ethereum Goerli testnet account private key used for registering in member contract",
|
||||
Value: &wcli.PrivateKeyValue{
|
||||
Value: &options.RLNRelay.ETHPrivateKey,
|
||||
},
|
||||
},
|
||||
&cli.StringFlag{
|
||||
Name: "eth-client-address",
|
||||
Usage: "Ethereum testnet client address",
|
||||
Value: "ws://localhost:8545",
|
||||
Destination: &options.RLNRelay.ETHClientAddress,
|
||||
},
|
||||
&cli.GenericFlag{
|
||||
Name: "eth-mem-contract-address",
|
||||
Usage: "Address of membership contract on an Ethereum testnet",
|
||||
Value: &wcli.AddressValue{
|
||||
Value: &options.RLNRelay.MembershipContractAddress,
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
|
@ -1,163 +0,0 @@
|
|||
{
|
||||
"fleets": {
|
||||
"eth.prod": {
|
||||
"boot": {
|
||||
"boot-01.ac-cn-hongkong-c.eth.prod": "enode://6e6554fb3034b211398fcd0f0082cbb6bd13619e1a7e76ba66e1809aaa0c5f1ac53c9ae79cf2fd4a7bacb10d12010899b370c75fed19b991d9c0cdd02891abad@47.75.99.169:443",
|
||||
"boot-01.do-ams3.eth.prod": "enode://436cc6f674928fdc9a9f7990f2944002b685d1c37f025c1be425185b5b1f0900feaf1ccc2a6130268f9901be4a7d252f37302c8335a2c1a62736e9232691cc3a@178.128.138.128:443",
|
||||
"boot-01.gc-us-central1-a.eth.prod": "enode://32ff6d88760b0947a3dee54ceff4d8d7f0b4c023c6dad34568615fcae89e26cc2753f28f12485a4116c977be937a72665116596265aa0736b53d46b27446296a@34.70.75.208:443",
|
||||
"boot-02.ac-cn-hongkong-c.eth.prod": "enode://23d0740b11919358625d79d4cac7d50a34d79e9c69e16831c5c70573757a1f5d7d884510bc595d7ee4da3c1508adf87bbc9e9260d804ef03f8c1e37f2fb2fc69@47.52.106.107:443",
|
||||
"boot-02.do-ams3.eth.prod": "enode://5395aab7833f1ecb671b59bf0521cf20224fe8162fc3d2675de4ee4d5636a75ec32d13268fc184df8d1ddfa803943906882da62a4df42d4fccf6d17808156a87@178.128.140.188:443",
|
||||
"boot-02.gc-us-central1-a.eth.prod": "enode://5405c509df683c962e7c9470b251bb679dd6978f82d5b469f1f6c64d11d50fbd5dd9f7801c6ad51f3b20a5f6c7ffe248cc9ab223f8bcbaeaf14bb1c0ef295fd0@35.223.215.156:443"
|
||||
},
|
||||
"mail": {
|
||||
"mail-01.ac-cn-hongkong-c.eth.prod": "enode://606ae04a71e5db868a722c77a21c8244ae38f1bd6e81687cc6cfe88a3063fa1c245692232f64f45bd5408fed5133eab8ed78049332b04f9c110eac7f71c1b429@47.75.247.214:443",
|
||||
"mail-01.do-ams3.eth.prod": "enode://c42f368a23fa98ee546fd247220759062323249ef657d26d357a777443aec04db1b29a3a22ef3e7c548e18493ddaf51a31b0aed6079bd6ebe5ae838fcfaf3a49@178.128.142.54:443",
|
||||
"mail-01.gc-us-central1-a.eth.prod": "enode://ee2b53b0ace9692167a410514bca3024695dbf0e1a68e1dff9716da620efb195f04a4b9e873fb9b74ac84de801106c465b8e2b6c4f0d93b8749d1578bfcaf03e@104.197.238.144:443",
|
||||
"mail-02.ac-cn-hongkong-c.eth.prod": "enode://2c8de3cbb27a3d30cbb5b3e003bc722b126f5aef82e2052aaef032ca94e0c7ad219e533ba88c70585ebd802de206693255335b100307645ab5170e88620d2a81@47.244.221.14:443",
|
||||
"mail-02.do-ams3.eth.prod": "enode://7aa648d6e855950b2e3d3bf220c496e0cae4adfddef3e1e6062e6b177aec93bc6cdcf1282cb40d1656932ebfdd565729da440368d7c4da7dbd4d004b1ac02bf8@178.128.142.26:443",
|
||||
"mail-02.gc-us-central1-a.eth.prod": "enode://30211cbd81c25f07b03a0196d56e6ce4604bb13db773ff1c0ea2253547fafd6c06eae6ad3533e2ba39d59564cfbdbb5e2ce7c137a5ebb85e99dcfc7a75f99f55@23.236.58.92:443",
|
||||
"mail-03.ac-cn-hongkong-c.eth.prod": "enode://e85f1d4209f2f99da801af18db8716e584a28ad0bdc47fbdcd8f26af74dbd97fc279144680553ec7cd9092afe683ddea1e0f9fc571ebcb4b1d857c03a088853d@47.244.129.82:443",
|
||||
"mail-03.do-ams3.eth.prod": "enode://8a64b3c349a2e0ef4a32ea49609ed6eb3364be1110253c20adc17a3cebbc39a219e5d3e13b151c0eee5d8e0f9a8ba2cd026014e67b41a4ab7d1d5dd67ca27427@178.128.142.94:443",
|
||||
"mail-03.gc-us-central1-a.eth.prod": "enode://44160e22e8b42bd32a06c1532165fa9e096eebedd7fa6d6e5f8bbef0440bc4a4591fe3651be68193a7ec029021cdb496cfe1d7f9f1dc69eb99226e6f39a7a5d4@35.225.221.245:443"
|
||||
},
|
||||
"rendezvous": {
|
||||
"boot-01.ac-cn-hongkong-c.eth.prod": "/ip4/47.75.99.169/tcp/30703/ethv4/16Uiu2HAmV8Hq9e3zm9TMVP4zrVHo3BjqW5D6bDVV6VQntQd687e4",
|
||||
"boot-01.do-ams3.eth.prod": "/ip4/178.128.138.128/tcp/30703/ethv4/16Uiu2HAmRHPzF3rQg55PgYPcQkyvPVH9n2hWsYPhUJBZ6kVjJgdV",
|
||||
"boot-01.gc-us-central1-a.eth.prod": "/ip4/34.70.75.208/tcp/30703/ethv4/16Uiu2HAm6ZsERLx2BwVD2UM9SVPnnMU6NBycG8XPtu8qKys5awsU",
|
||||
"boot-02.ac-cn-hongkong-c.eth.prod": "/ip4/47.52.106.107/tcp/30703/ethv4/16Uiu2HAmEHiptiDDd9gqNY8oQqo8hHUWMHJzfwt5aLRdD6W2zcXR",
|
||||
"boot-02.do-ams3.eth.prod": "/ip4/178.128.140.188/tcp/30703/ethv4/16Uiu2HAmLqTXuY4Sb6G28HNooaFUXUKzpzKXCcgyJxgaEE2i5vnf",
|
||||
"boot-02.gc-us-central1-a.eth.prod": "/ip4/35.223.215.156/tcp/30703/ethv4/16Uiu2HAmQEUFE2YaJohavWtHxPTEFv3sEGJtDqvtGEv78DFoEWQF"
|
||||
},
|
||||
"whisper": {
|
||||
"node-01.ac-cn-hongkong-c.eth.prod": "enode://b957e51f41e4abab8382e1ea7229e88c6e18f34672694c6eae389eac22dab8655622bbd4a08192c321416b9becffaab11c8e2b7a5d0813b922aa128b82990dab@47.75.222.178:443",
|
||||
"node-01.do-ams3.eth.prod": "enode://66ba15600cda86009689354c3a77bdf1a97f4f4fb3ab50ffe34dbc904fac561040496828397be18d9744c75881ffc6ac53729ddbd2cdbdadc5f45c400e2622f7@178.128.141.87:443",
|
||||
"node-01.gc-us-central1-a.eth.prod": "enode://182ed5d658d1a1a4382c9e9f7c9e5d8d9fec9db4c71ae346b9e23e1a589116aeffb3342299bdd00e0ab98dbf804f7b2d8ae564ed18da9f45650b444aed79d509@34.68.132.118:443",
|
||||
"node-02.ac-cn-hongkong-c.eth.prod": "enode://8bebe73ddf7cf09e77602c7d04c93a73f455b51f24ae0d572917a4792f1dec0bb4c562759b8830cc3615a658d38c1a4a38597a1d7ae3ba35111479fc42d65dec@47.75.85.212:443",
|
||||
"node-02.do-ams3.eth.prod": "enode://4ea35352702027984a13274f241a56a47854a7fd4b3ba674a596cff917d3c825506431cf149f9f2312a293bb7c2b1cca55db742027090916d01529fe0729643b@134.209.136.79:443",
|
||||
"node-02.gc-us-central1-a.eth.prod": "enode://fbeddac99d396b91d59f2c63a3cb5fc7e0f8a9f7ce6fe5f2eed5e787a0154161b7173a6a73124a4275ef338b8966dc70a611e9ae2192f0f2340395661fad81c0@34.67.230.193:443",
|
||||
"node-03.ac-cn-hongkong-c.eth.prod": "enode://ac3948b2c0786ada7d17b80cf869cf59b1909ea3accd45944aae35bf864cc069126da8b82dfef4ddf23f1d6d6b44b1565c4cf81c8b98022253c6aea1a89d3ce2@47.75.88.12:443",
|
||||
"node-03.do-ams3.eth.prod": "enode://ce559a37a9c344d7109bd4907802dd690008381d51f658c43056ec36ac043338bd92f1ac6043e645b64953b06f27202d679756a9c7cf62fdefa01b2e6ac5098e@134.209.136.123:443",
|
||||
"node-03.gc-us-central1-a.eth.prod": "enode://c07aa0deea3b7056c5d45a85bca42f0d8d3b1404eeb9577610f386e0a4744a0e7b2845ae328efc4aa4b28075af838b59b5b3985bffddeec0090b3b7669abc1f3@35.226.92.155:443",
|
||||
"node-04.ac-cn-hongkong-c.eth.prod": "enode://385579fc5b14e04d5b04af7eee835d426d3d40ccf11f99dbd95340405f37cf3bbbf830b3eb8f70924be0c2909790120682c9c3e791646e2d5413e7801545d353@47.244.221.249:443",
|
||||
"node-04.do-ams3.eth.prod": "enode://4e0a8db9b73403c9339a2077e911851750fc955db1fc1e09f81a4a56725946884dd5e4d11258eac961f9078a393c45bcab78dd0e3bc74e37ce773b3471d2e29c@134.209.136.101:443",
|
||||
"node-04.gc-us-central1-a.eth.prod": "enode://0624b4a90063923c5cc27d12624b6a49a86dfb3623fcb106801217fdbab95f7617b83fa2468b9ae3de593ff6c1cf556ccf9bc705bfae9cb4625999765127b423@35.222.158.246:443",
|
||||
"node-05.ac-cn-hongkong-c.eth.prod": "enode://b77bffc29e2592f30180311dd81204ab845e5f78953b5ba0587c6631be9c0862963dea5eb64c90617cf0efd75308e22a42e30bc4eb3cd1bbddbd1da38ff6483e@47.75.10.177:443",
|
||||
"node-05.do-ams3.eth.prod": "enode://a8bddfa24e1e92a82609b390766faa56cf7a5eef85b22a2b51e79b333c8aaeec84f7b4267e432edd1cf45b63a3ad0fc7d6c3a16f046aa6bc07ebe50e80b63b8c@178.128.141.249:443",
|
||||
"node-05.gc-us-central1-a.eth.prod": "enode://a5fe9c82ad1ffb16ae60cb5d4ffe746b9de4c5fbf20911992b7dd651b1c08ba17dd2c0b27ee6b03162c52d92f219961cc3eb14286aca8a90b75cf425826c3bd8@104.154.230.58:443",
|
||||
"node-06.ac-cn-hongkong-c.eth.prod": "enode://cf5f7a7e64e3b306d1bc16073fba45be3344cb6695b0b616ccc2da66ea35b9f35b3b231c6cf335fdfaba523519659a440752fc2e061d1e5bc4ef33864aac2f19@47.75.221.196:443",
|
||||
"node-06.do-ams3.eth.prod": "enode://887cbd92d95afc2c5f1e227356314a53d3d18855880ac0509e0c0870362aee03939d4074e6ad31365915af41d34320b5094bfcc12a67c381788cd7298d06c875@178.128.141.0:443",
|
||||
"node-06.gc-us-central1-a.eth.prod": "enode://282e009967f9f132a5c2dd366a76319f0d22d60d0c51f7e99795a1e40f213c2705a2c10e4cc6f3890319f59da1a535b8835ed9b9c4b57c3aad342bf312fd7379@35.223.240.17:443",
|
||||
"node-07.ac-cn-hongkong-c.eth.prod": "enode://13d63a1f85ccdcbd2fb6861b9bd9d03f94bdba973608951f7c36e5df5114c91de2b8194d71288f24bfd17908c48468e89dd8f0fb8ccc2b2dedae84acdf65f62a@47.244.210.80:443",
|
||||
"node-07.do-ams3.eth.prod": "enode://2b01955d7e11e29dce07343b456e4e96c081760022d1652b1c4b641eaf320e3747871870fa682e9e9cfb85b819ce94ed2fee1ac458904d54fd0b97d33ba2c4a4@134.209.136.112:443",
|
||||
"node-07.gc-us-central1-a.eth.prod": "enode://b706a60572634760f18a27dd407b2b3582f7e065110dae10e3998498f1ae3f29ba04db198460d83ed6d2bfb254bb06b29aab3c91415d75d3b869cd0037f3853c@35.239.5.162:443",
|
||||
"node-08.ac-cn-hongkong-c.eth.prod": "enode://32915c8841faaef21a6b75ab6ed7c2b6f0790eb177ad0f4ea6d731bacc19b938624d220d937ebd95e0f6596b7232bbb672905ee12601747a12ee71a15bfdf31c@47.75.59.11:443",
|
||||
"node-08.do-ams3.eth.prod": "enode://0d9d65fcd5592df33ed4507ce862b9c748b6dbd1ea3a1deb94e3750052760b4850aa527265bbaf357021d64d5cc53c02b410458e732fafc5b53f257944247760@178.128.141.42:443",
|
||||
"node-08.gc-us-central1-a.eth.prod": "enode://e87f1d8093d304c3a9d6f1165b85d6b374f1c0cc907d39c0879eb67f0a39d779be7a85cbd52920b6f53a94da43099c58837034afa6a7be4b099bfcd79ad13999@35.238.106.101:443"
|
||||
}
|
||||
},
|
||||
"eth.staging": {
|
||||
"boot": {
|
||||
"boot-01.ac-cn-hongkong-c.eth.staging": "enode://630b0342ca4e9552f50714b6c8e28d6955bc0fd14e7950f93bc3b2b8cc8c1f3b6d103df66f51a13d773b5db0f130661fb5c7b8fa21c48890c64c79b41a56a490@47.91.229.44:443",
|
||||
"boot-01.do-ams3.eth.staging": "enode://f79fb3919f72ca560ad0434dcc387abfe41e0666201ebdada8ede0462454a13deb05cda15f287d2c4bd85da81f0eb25d0a486bbbc8df427b971ac51533bd00fe@174.138.107.239:443",
|
||||
"boot-01.gc-us-central1-a.eth.staging": "enode://10a78c17929a7019ef4aa2249d7302f76ae8a06f40b2dc88b7b31ebff4a623fbb44b4a627acba296c1ced3775d91fbe18463c15097a6a36fdb2c804ff3fc5b35@35.238.97.234:443"
|
||||
},
|
||||
"mail": {
|
||||
"mail-01.ac-cn-hongkong-c.eth.staging": "enode://b74859176c9751d314aeeffc26ec9f866a412752e7ddec91b19018a18e7cca8d637cfe2cedcb972f8eb64d816fbd5b4e89c7e8c7fd7df8a1329fa43db80b0bfe@47.52.90.156:443",
|
||||
"mail-01.do-ams3.eth.staging": "enode://69f72baa7f1722d111a8c9c68c39a31430e9d567695f6108f31ccb6cd8f0adff4991e7fdca8fa770e75bc8a511a87d24690cbc80e008175f40c157d6f6788d48@206.189.240.16:443",
|
||||
"mail-01.gc-us-central1-a.eth.staging": "enode://e4fc10c1f65c8aed83ac26bc1bfb21a45cc1a8550a58077c8d2de2a0e0cd18e40fd40f7e6f7d02dc6cd06982b014ce88d6e468725ffe2c138e958788d0002a7f@35.239.193.41:443"
|
||||
},
|
||||
"rendezvous": {
|
||||
"boot-01.ac-cn-hongkong-c.eth.staging": "/ip4/47.91.229.44/tcp/30703/ethv4/16Uiu2HAmRnt2Eyoknh3auxh4fJwkRgqkH1gqrWGes8Pk1k3MV4xu",
|
||||
"boot-01.do-ams3.eth.staging": "/ip4/174.138.107.239/tcp/30703/ethv4/16Uiu2HAm8UZXUHEPZrpJbcQ3yVFH6UtKrwsG6jH4ai72PsbLfVFb",
|
||||
"boot-01.gc-us-central1-a.eth.staging": "/ip4/35.238.97.234/tcp/30703/ethv4/16Uiu2HAm6G9sDMkrB4Xa5EH3Zx2dysCxFgBTSRzghic3Z9tRFRNE"
|
||||
},
|
||||
"whisper": {
|
||||
"node-01.ac-cn-hongkong-c.eth.staging": "enode://088cf5a93c576fae52f6f075178467b8ff98bacf72f59e88efb16dfba5b30f80a4db78f8e3cb3d87f2f6521746ef4a8768465ef2896c6af24fd77a425e95b6dd@47.52.226.137:443",
|
||||
"node-01.do-ams3.eth.staging": "enode://914c0b30f27bab30c1dfd31dad7652a46fda9370542aee1b062498b1345ee0913614b8b9e3e84622e84a7203c5858ae1d9819f63aece13ee668e4f6668063989@167.99.19.148:443",
|
||||
"node-01.gc-us-central1-a.eth.staging": "enode://d3878441652f010326889f28360e69f2d09d06540f934fada0e17b374ce5319de64279aba3c44a5bf807d9967c6d705b3b4c6b03fa70763240e2ee6af01a539e@35.192.0.86:443"
|
||||
}
|
||||
},
|
||||
"eth.test": {
|
||||
"boot": {
|
||||
"boot-01.ac-cn-hongkong-c.eth.test": "enode://daae2e72820e86e942fa2a8aa7d6e9954d4043a753483d8bd338e16be82cf962392d5c0e1ae57c3d793c3d3dddd8fd58339262e4234dc966f953cd73b535f5fa@47.52.188.149:443",
|
||||
"boot-01.do-ams3.eth.test": "enode://9e0988575eb7717c25dea72fd11c7b37767dc09c1a7686f7c2ec577d308d24b377ceb675de4317474a1a870e47882732967f4fa785b02ba95d669b31d464dec0@206.189.243.164:443",
|
||||
"boot-01.gc-us-central1-a.eth.test": "enode://c1e5018887c863d64e431b69bf617561087825430e4401733f5ba77c70db14236df381fefb0ebe1ac42294b9e261bbe233dbdb83e32c586c66ae26c8de70cb4c@35.188.168.137:443"
|
||||
},
|
||||
"mail": {
|
||||
"mail-01.ac-cn-hongkong-c.eth.test": "enode://619dbb5dda12e85bf0eb5db40fb3de625609043242737c0e975f7dfd659d85dc6d9a84f9461a728c5ab68c072fed38ca6a53917ca24b8e93cc27bdef3a1e79ac@47.52.188.196:443",
|
||||
"mail-01.do-ams3.eth.test": "enode://e4865fe6c2a9c1a563a6447990d8e9ce672644ae3e08277ce38ec1f1b690eef6320c07a5d60c3b629f5d4494f93d6b86a745a0bf64ab295bbf6579017adc6ed8@206.189.243.161:443",
|
||||
"mail-01.gc-us-central1-a.eth.test": "enode://707e57453acd3e488c44b9d0e17975371e2f8fb67525eae5baca9b9c8e06c86cde7c794a6c2e36203bf9f56cae8b0e50f3b33c4c2b694a7baeea1754464ce4e3@35.192.229.172:443"
|
||||
},
|
||||
"rendezvous": {
|
||||
"boot-01.ac-cn-hongkong-c.eth.test": "/ip4/47.52.188.149/tcp/30703/ethv4/16Uiu2HAm9Vatqr4GfVCqnyeaPtCF3q8fz8kDDUgqXVfFG7ZfSA7w",
|
||||
"boot-01.do-ams3.eth.test": "/ip4/206.189.243.164/tcp/30703/ethv4/16Uiu2HAmBCh5bgYr6V3fDuLqUzvtSAsFTQJCQ3TVHT8ta8bTu2Jm",
|
||||
"boot-01.gc-us-central1-a.eth.test": "/ip4/35.188.168.137/tcp/30703/ethv4/16Uiu2HAm3MUqtGjmetyZ9L4SN2R8oHDWvACUcec25LjtDD5euiRH"
|
||||
},
|
||||
"whisper": {
|
||||
"node-01.ac-cn-hongkong-c.eth.test": "enode://ad38f94030a846cc7005b7a1f3b6b01bf4ef59d34e8d3d6f4d12df23d14ba8656702a435d34cf4df3b412c0c1923df5adcce8461321a0d8ffb9435b26e572c2a@47.52.255.194:443",
|
||||
"node-01.do-ams3.eth.test": "enode://1d193635e015918fb85bbaf774863d12f65d70c6977506187ef04420d74ec06c9e8f0dcb57ea042f85df87433dab17a1260ed8dde1bdf9d6d5d2de4b7bf8e993@206.189.243.163:443",
|
||||
"node-01.gc-us-central1-a.eth.test": "enode://f593a27731bc0f8eb088e2d39222c2d59dfb9bf0b3950d7a828d51e8ab9e08fffbd9916a82fd993c1a080c57c2bd70ed6c36f489a969de697aff93088dbee1a9@35.194.31.108:443"
|
||||
}
|
||||
},
|
||||
"go-waku.test": {
|
||||
"libp2p": {
|
||||
"node-01.ac-cn-hongkong-c.go-waku.test": "/ip4/8.218.2.110/tcp/30303/p2p/16Uiu2HAmBDbMWFiG9ki8sDw6fYtraSxo4oHU9HbuN43S2HVyq1FD",
|
||||
"node-01.do-ams3.go-waku.test": "/ip4/134.209.134.63/tcp/30303/p2p/16Uiu2HAm9vnvCQgCDrynDK1h7GJoEZVGvnuzq84RyDQ3DEdXmcX7",
|
||||
"node-01.gc-us-central1-a.go-waku.test": "/ip4/35.223.183.91/tcp/30303/p2p/16Uiu2HAmPz63Xc6AuVkDeujz7YeZta18rcdau3Y1BzaxKAfDrBqz"
|
||||
},
|
||||
"websocket": {
|
||||
"node-01.ac-cn-hongkong-c.go-waku.test": "/dns4/node-01.ac-cn-hongkong-c.go-waku.test.statusim.net/tcp/443/wss/p2p/16Uiu2HAmBDbMWFiG9ki8sDw6fYtraSxo4oHU9HbuN43S2HVyq1FD",
|
||||
"node-01.do-ams3.go-waku.test": "/dns4/node-01.do-ams3.go-waku.test.statusim.net/tcp/443/wss/p2p/16Uiu2HAm9vnvCQgCDrynDK1h7GJoEZVGvnuzq84RyDQ3DEdXmcX7",
|
||||
"node-01.gc-us-central1-a.go-waku.test": "/dns4/node-01.gc-us-central1-a.go-waku.test.statusim.net/tcp/443/wss/p2p/16Uiu2HAmPz63Xc6AuVkDeujz7YeZta18rcdau3Y1BzaxKAfDrBqz"
|
||||
}
|
||||
},
|
||||
"status.test": {
|
||||
"tcp/p2p/waku": {
|
||||
"node-01.ac-cn-hongkong-c.status.test": "/ip4/47.242.233.36/tcp/30303/p2p/16Uiu2HAm2BjXxCp1sYFJQKpLLbPbwd5juxbsYofu3TsS3auvT9Yi",
|
||||
"node-01.do-ams3.status.test": "/ip4/64.225.81.237/tcp/30303/p2p/16Uiu2HAkukebeXjTQ9QDBeNDWuGfbaSg79wkkhK4vPocLgR6QFDf",
|
||||
"node-01.gc-us-central1-a.status.test": "/ip4/34.122.252.118/tcp/30303/p2p/16Uiu2HAmGDX3iAFox93PupVYaHa88kULGqMpJ7AEHGwj3jbMtt76"
|
||||
}
|
||||
},
|
||||
"waku.connect": {
|
||||
"tcp/p2p/waku": {
|
||||
"nim-01.ac-cn-hongkong-c.waku.connect": "/ip4/47.242.185.35/tcp/30303/p2p/16Uiu2HAm75XUMGev2Ti74G3wUzhyxCtbaDKVWzNwbq3tn5WfzRd4",
|
||||
"nim-01.do-ams3.waku.connect": "/ip4/206.189.242.0/tcp/30303/p2p/16Uiu2HAm9VLETt1xBwDAwfKxj2XvAZDw73Bn4HQf11U26JGDxqZD",
|
||||
"nim-01.gc-us-central1-a.waku.connect": "/ip4/35.193.87.35/tcp/30303/p2p/16Uiu2HAmMi8xaj9W22a67shGg5wtw1nZDNtfrTPHkgKA5Uhvnvbn"
|
||||
},
|
||||
"wss/p2p/waku": {
|
||||
"nim-01.ac-cn-hongkong-c.waku.connect": "/dns4/nim-01.ac-cn-hongkong-c.waku.connect.statusim.net/tcp/443/wss/p2p/16Uiu2HAm75XUMGev2Ti74G3wUzhyxCtbaDKVWzNwbq3tn5WfzRd4",
|
||||
"nim-01.do-ams3.waku.connect": "/dns4/nim-01.do-ams3.waku.connect.statusim.net/tcp/443/wss/p2p/16Uiu2HAm9VLETt1xBwDAwfKxj2XvAZDw73Bn4HQf11U26JGDxqZD",
|
||||
"nim-01.gc-us-central1-a.waku.connect": "/dns4/nim-01.gc-us-central1-a.waku.connect.statusim.net/tcp/443/wss/p2p/16Uiu2HAmMi8xaj9W22a67shGg5wtw1nZDNtfrTPHkgKA5Uhvnvbn"
|
||||
}
|
||||
},
|
||||
"wakuv2.prod": {
|
||||
"waku": {
|
||||
"node-01.ac-cn-hongkong-c.wakuv2.prod": "/ip4/8.210.222.231/tcp/30303/p2p/16Uiu2HAm4v86W3bmT1BiH6oSPzcsSr24iDQpSN5Qa992BCjjwgrD",
|
||||
"node-01.do-ams3.wakuv2.prod": "/ip4/188.166.135.145/tcp/30303/p2p/16Uiu2HAmL5okWopX7NqZWBUKVqW8iUxCEmd5GMHLVPwCgzYzQv3e",
|
||||
"node-01.gc-us-central1-a.wakuv2.prod": "/ip4/34.121.100.108/tcp/30303/p2p/16Uiu2HAmVkKntsECaYfefR1V2yCR79CegLATuTPE6B9TxgxBiiiA"
|
||||
},
|
||||
"waku-websocket": {
|
||||
"node-01.ac-cn-hongkong-c.wakuv2.prod": "/dns4/node-01.ac-cn-hongkong-c.wakuv2.prod.statusim.net/tcp/443/wss/p2p/16Uiu2HAm4v86W3bmT1BiH6oSPzcsSr24iDQpSN5Qa992BCjjwgrD",
|
||||
"node-01.do-ams3.wakuv2.prod": "/dns4/node-01.do-ams3.wakuv2.prod.statusim.net/tcp/443/wss/p2p/16Uiu2HAmL5okWopX7NqZWBUKVqW8iUxCEmd5GMHLVPwCgzYzQv3e",
|
||||
"node-01.gc-us-central1-a.wakuv2.prod": "/dns4/node-01.gc-us-central1-a.wakuv2.prod.statusim.net/tcp/443/wss/p2p/16Uiu2HAmVkKntsECaYfefR1V2yCR79CegLATuTPE6B9TxgxBiiiA"
|
||||
}
|
||||
},
|
||||
"wakuv2.test": {
|
||||
"waku": {
|
||||
"node-01.ac-cn-hongkong-c.wakuv2.test": "/ip4/47.242.210.73/tcp/30303/p2p/16Uiu2HAkvWiyFsgRhuJEb9JfjYxEkoHLgnUQmr1N5mKWnYjxYRVm",
|
||||
"node-01.do-ams3.wakuv2.test": "/ip4/134.209.139.210/tcp/30303/p2p/16Uiu2HAmPLe7Mzm8TsYUubgCAW1aJoeFScxrLj8ppHFivPo97bUZ",
|
||||
"node-01.gc-us-central1-a.wakuv2.test": "/ip4/104.154.239.128/tcp/30303/p2p/16Uiu2HAmJb2e28qLXxT5kZxVUUoJt72EMzNGXB47Rxx5hw3q4YjS"
|
||||
},
|
||||
"waku-websocket": {
|
||||
"node-01.ac-cn-hongkong-c.wakuv2.test": "/dns4/node-01.ac-cn-hongkong-c.wakuv2.test.statusim.net/tcp/443/wss/p2p/16Uiu2HAkvWiyFsgRhuJEb9JfjYxEkoHLgnUQmr1N5mKWnYjxYRVm",
|
||||
"node-01.do-ams3.wakuv2.test": "/dns4/node-01.do-ams3.wakuv2.test.statusim.net/tcp/443/wss/p2p/16Uiu2HAmPLe7Mzm8TsYUubgCAW1aJoeFScxrLj8ppHFivPo97bUZ",
|
||||
"node-01.gc-us-central1-a.wakuv2.test": "/dns4/node-01.gc-us-central1-a.wakuv2.test.statusim.net/tcp/443/wss/p2p/16Uiu2HAmJb2e28qLXxT5kZxVUUoJt72EMzNGXB47Rxx5hw3q4YjS"
|
||||
}
|
||||
}
|
||||
},
|
||||
"meta": {
|
||||
"hostname": "node-01.do-ams3.sites.misc",
|
||||
"timestamp": "2022-02-21T14:40:19.809193",
|
||||
"warning": "This data should only be used at build time."
|
||||
}
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
module chat2
|
||||
module chat3
|
||||
|
||||
go 1.17
|
||||
|
||||
|
@ -9,22 +9,26 @@ replace github.com/ethereum/go-ethereum v1.10.18 => github.com/status-im/go-ethe
|
|||
replace github.com/flynn/noise v1.0.0 => github.com/status-im/noise v1.0.1-handshakeMessages
|
||||
|
||||
require (
|
||||
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.20
|
||||
github.com/gdamore/tcell/v2 v2.2.0
|
||||
github.com/golang/protobuf v1.5.2
|
||||
github.com/ipfs/go-log v1.0.5
|
||||
github.com/libp2p/go-libp2p-core v0.16.1
|
||||
github.com/muesli/reflow v0.3.0
|
||||
github.com/multiformats/go-multiaddr v0.5.0
|
||||
github.com/rivo/tview v0.0.0-20210312174852-ae9464cc3598
|
||||
github.com/status-im/go-waku v0.0.0-20211101194039-94e8b9cf86fc
|
||||
github.com/status-im/go-rln v0.0.9
|
||||
github.com/status-im/go-waku v0.0.0-00010101000000-000000000000
|
||||
github.com/urfave/cli/v2 v2.11.1
|
||||
golang.org/x/crypto v0.0.0-20220411220226-7b82a4e95df4
|
||||
golang.org/x/term v0.0.0-20210927222741-03fcf44c2211
|
||||
google.golang.org/protobuf v1.28.0
|
||||
)
|
||||
|
||||
require github.com/status-im/go-rln v0.0.7
|
||||
|
||||
require (
|
||||
github.com/StackExchange/wmi v0.0.0-20180116203802-5d049714c4a6 // indirect
|
||||
github.com/atotto/clipboard v0.1.4 // indirect
|
||||
github.com/benbjohnson/clock v1.3.0 // indirect
|
||||
github.com/beorn7/perks v1.0.1 // indirect
|
||||
github.com/btcsuite/btcd v0.22.1 // indirect
|
||||
|
@ -33,7 +37,9 @@ require (
|
|||
github.com/cespare/xxhash/v2 v2.1.2 // indirect
|
||||
github.com/cheekybits/genny v1.0.0 // indirect
|
||||
github.com/containerd/cgroups v1.0.3 // indirect
|
||||
github.com/containerd/console v1.0.3 // indirect
|
||||
github.com/coreos/go-systemd/v22 v22.3.2 // indirect
|
||||
github.com/cpuguy83/go-md2man/v2 v2.0.2 // indirect
|
||||
github.com/cruxic/go-hmac-drbg v0.0.0-20170206035330-84c46983886d // indirect
|
||||
github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c // indirect
|
||||
github.com/deckarep/golang-set v1.8.0 // indirect
|
||||
|
@ -43,7 +49,6 @@ require (
|
|||
github.com/flynn/noise v1.0.0 // indirect
|
||||
github.com/francoispqt/gojay v1.2.13 // indirect
|
||||
github.com/fsnotify/fsnotify v1.5.4 // indirect
|
||||
github.com/gdamore/encoding v1.0.0 // indirect
|
||||
github.com/go-ole/go-ole v1.2.1 // indirect
|
||||
github.com/go-stack/stack v1.8.0 // indirect
|
||||
github.com/go-task/slim-sprig v0.0.0-20210107165309-348f09dbbbc0 // indirect
|
||||
|
@ -90,7 +95,7 @@ require (
|
|||
github.com/marten-seemann/qtls-go1-18 v0.1.1 // indirect
|
||||
github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd // indirect
|
||||
github.com/mattn/go-isatty v0.0.14 // indirect
|
||||
github.com/mattn/go-runewidth v0.0.10 // indirect
|
||||
github.com/mattn/go-runewidth v0.0.13 // indirect
|
||||
github.com/mattn/go-sqlite3 v1.14.13 // indirect
|
||||
github.com/matttproud/golang_protobuf_extensions v1.0.2-0.20181231171920-c182affec369 // indirect
|
||||
github.com/miekg/dns v1.1.43 // indirect
|
||||
|
@ -99,6 +104,9 @@ require (
|
|||
github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 // indirect
|
||||
github.com/minio/sha256-simd v1.0.0 // 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.0.3 // indirect
|
||||
github.com/multiformats/go-base36 v0.1.0 // indirect
|
||||
github.com/multiformats/go-multiaddr-dns v0.3.1 // indirect
|
||||
|
@ -122,6 +130,7 @@ require (
|
|||
github.com/raulk/go-watchdog v1.2.0 // indirect
|
||||
github.com/rivo/uniseg v0.2.0 // indirect
|
||||
github.com/rjeczalik/notify v0.9.1 // indirect
|
||||
github.com/russross/blackfriday/v2 v2.1.0 // indirect
|
||||
github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible // indirect
|
||||
github.com/spacemonkeygo/spacelog v0.0.0-20180420211403-2296661a0572 // indirect
|
||||
github.com/spaolacci/murmur3 v1.1.0 // indirect
|
||||
|
@ -132,6 +141,7 @@ require (
|
|||
github.com/tklauser/numcpus v0.2.2 // indirect
|
||||
github.com/whyrusleeping/multiaddr-filter v0.0.0-20160516205228-e903e4adabd7 // indirect
|
||||
github.com/whyrusleeping/timecache v0.0.0-20160911033111-cfcb2f1abfee // indirect
|
||||
github.com/xrash/smetrics v0.0.0-20201216005158-039620a65673 // indirect
|
||||
go.opencensus.io v0.23.0 // indirect
|
||||
go.uber.org/atomic v1.9.0 // indirect
|
||||
go.uber.org/multierr v1.7.0 // indirect
|
||||
|
@ -140,8 +150,6 @@ require (
|
|||
golang.org/x/net v0.0.0-20220607020251-c690dde0001d // indirect
|
||||
golang.org/x/sync v0.0.0-20220513210516-0976fa681c29 // indirect
|
||||
golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a // indirect
|
||||
golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 // indirect
|
||||
golang.org/x/text v0.3.7 // indirect
|
||||
golang.org/x/time v0.0.0-20220224211638-0e9765cccd65 // indirect
|
||||
golang.org/x/tools v0.1.8-0.20211029000441-d6a9af8af023 // indirect
|
||||
golang.org/x/xerrors v0.0.0-20220517211312-f3a8303e98df // indirect
|
||||
|
|
|
@ -169,6 +169,8 @@ github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmV
|
|||
github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8=
|
||||
github.com/aryann/difflib v0.0.0-20170710044230-e206f873d14a/go.mod h1:DAHtR1m6lCRdSC2Tm3DSWRPvIPr6xNKyeHdqDQSQT+A=
|
||||
github.com/asaskevich/govalidator v0.0.0-20190424111038-f61b66f89f4a/go.mod h1:lB+ZfQJz7igIIfQNfa7Ml4HSf2uFQQRzpGGRXenZAgY=
|
||||
github.com/atotto/clipboard v0.1.4 h1:EH0zSVneZPSuFR11BlR9YppQTVDbh5+16AmcJi4g1z4=
|
||||
github.com/atotto/clipboard v0.1.4/go.mod h1:ZY9tmq7sm5xIbd9bOK4onWV4S6X0u6GY7Vn0Yu86PYI=
|
||||
github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU=
|
||||
github.com/aws/aws-sdk-go v1.15.11/go.mod h1:mFuSZ37Z9YOHbQEwBWztmVzqXrEkub65tZoCYDt7FT0=
|
||||
github.com/aws/aws-sdk-go v1.17.7/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo=
|
||||
|
@ -275,6 +277,14 @@ github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghf
|
|||
github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
|
||||
github.com/cespare/xxhash/v2 v2.1.2 h1:YRXhKfTDauu4ajMg1TPgFO5jnlC2HCbmLXMcTG5cbYE=
|
||||
github.com/cespare/xxhash/v2 v2.1.2/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs=
|
||||
github.com/charmbracelet/bubbles v0.13.0 h1:zP/ROH3wJEBqZWKIsD50ZKKlx3ydLInq3LdD/Nrlb8w=
|
||||
github.com/charmbracelet/bubbles v0.13.0/go.mod h1:bbeTiXwPww4M031aGi8UK2HT9RDWoiNibae+1yCMtcc=
|
||||
github.com/charmbracelet/bubbletea v0.21.0/go.mod h1:GgmJMec61d08zXsOhqRC/AiOx4K4pmz+VIcRIm1FKr4=
|
||||
github.com/charmbracelet/bubbletea v0.22.0 h1:E1BTNSE3iIrq0G0X6TjGAmrQ32cGCbFDPcIuImikrUc=
|
||||
github.com/charmbracelet/bubbletea v0.22.0/go.mod h1:aoVIwlNlr5wbCB26KhxfrqAn0bMp4YpJcoOelbxApjs=
|
||||
github.com/charmbracelet/harmonica v0.2.0/go.mod h1:KSri/1RMQOZLbw7AHqgcBycp8pgJnQMYYT8QZRqZ1Ao=
|
||||
github.com/charmbracelet/lipgloss v0.5.0 h1:lulQHuVeodSgDez+3rGiuxlPVXSnhth442DATR2/8t8=
|
||||
github.com/charmbracelet/lipgloss v0.5.0/go.mod h1:EZLha/HbzEt7cYqdFPovlqy5FZPj0xFhg5SaqxScmgs=
|
||||
github.com/checkpoint-restore/go-criu/v4 v4.1.0/go.mod h1:xUQBLp4RLc5zJtWY++yjOoMoB5lihDt7fai+75m+rGw=
|
||||
github.com/checkpoint-restore/go-criu/v5 v5.0.0/go.mod h1:cfwC0EG7HMUenopBsUf9d89JlCLQIfgVcNsNN0t6T2M=
|
||||
github.com/checkpoint-restore/go-criu/v5 v5.3.0/go.mod h1:E/eQpaFtUKGOOSEBZgmKAcn+zUUwWxqcaKZlF54wK8E=
|
||||
|
@ -334,6 +344,7 @@ github.com/containerd/console v0.0.0-20181022165439-0650fd9eeb50/go.mod h1:Tj/on
|
|||
github.com/containerd/console v0.0.0-20191206165004-02ecf6a7291e/go.mod h1:8Pf4gM6VEbTNRIT26AyyU7hxdQU3MvAvxVI0sc00XBE=
|
||||
github.com/containerd/console v1.0.1/go.mod h1:XUsP6YE/mKtz6bxc+I8UiKKTP04qjQL4qcS3XoQ5xkw=
|
||||
github.com/containerd/console v1.0.2/go.mod h1:ytZPjGgY2oeTkAONYafi2kSj0aYggsf8acV1PGKCbzQ=
|
||||
github.com/containerd/console v1.0.3 h1:lIr7SlA5PxZyMV30bDW0MGbiOPXwc63yRuCP0ARubLw=
|
||||
github.com/containerd/console v1.0.3/go.mod h1:7LqA/THxQ86k76b8c/EMSiaJ3h1eZkMkXar0TQ1gf3U=
|
||||
github.com/containerd/containerd v1.2.10/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA=
|
||||
github.com/containerd/containerd v1.3.0-beta.2.0.20190828155532-0293cbd26c69/go.mod h1:bC6axHOhabU15QhwfG7w5PipXdVtMXFTttgp+kVtyUA=
|
||||
|
@ -566,12 +577,6 @@ github.com/garslo/gogen v0.0.0-20170306192744-1d203ffc1f61/go.mod h1:Q0X6pkwTILD
|
|||
github.com/garyburd/redigo v0.0.0-20150301180006-535138d7bcd7/go.mod h1:NR3MbYisc3/PwhQ00EMzDiPmrwpPxAn5GI05/YaO1SY=
|
||||
github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff h1:tY80oXqGNY4FhTFhk+o9oFHGINQ/+vhlm8HFzi6znCI=
|
||||
github.com/gballet/go-libpcsclite v0.0.0-20190607065134-2772fd86a8ff/go.mod h1:x7DCsMOv1taUwEWCzT4cmDeAkigA5/QCwUodaVOe8Ww=
|
||||
github.com/gdamore/encoding v1.0.0 h1:+7OoQ1Bc6eTm5niUzBa0Ctsh6JbMW6Ra+YNuAtDBdko=
|
||||
github.com/gdamore/encoding v1.0.0/go.mod h1:alR0ol34c49FCSBLjhosxzcPHQbf2trDkoo5dl+VrEg=
|
||||
github.com/gdamore/tcell v1.4.0 h1:vUnHwJRvcPQa3tzi+0QI4U9JINXYJlOz9yiaiPQ2wMU=
|
||||
github.com/gdamore/tcell v1.4.0/go.mod h1:vxEiSDZdW3L+Uhjii9c3375IlDmR05bzxY404ZVSMo0=
|
||||
github.com/gdamore/tcell/v2 v2.2.0 h1:vSyEgKwraXPSOkvCk7IwOSyX+Pv3V2cV9CikJMXg4U4=
|
||||
github.com/gdamore/tcell/v2 v2.2.0/go.mod h1:cTTuF84Dlj/RqmaCIV5p4w8uG1zWdk0SF6oBpwHp4fU=
|
||||
github.com/getkin/kin-openapi v0.53.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW6yheSwk8Yi4=
|
||||
github.com/getkin/kin-openapi v0.61.0/go.mod h1:7Yn5whZr5kJi6t+kShccXS8ae1APpYTW6yheSwk8Yi4=
|
||||
github.com/getsentry/raven-go v0.2.0/go.mod h1:KungGk8q33+aIAZUIVWZDr2OfAEBsO49PX4NzFV5kcQ=
|
||||
|
@ -1201,7 +1206,6 @@ github.com/lucas-clemente/quic-go v0.25.0/go.mod h1:YtzP8bxRVCBlO77yRanE264+fY/T
|
|||
github.com/lucas-clemente/quic-go v0.27.0/go.mod h1:AzgQoPda7N+3IqMMMkywBKggIFo2KT6pfnlrQ2QieeI=
|
||||
github.com/lucas-clemente/quic-go v0.27.1 h1:sOw+4kFSVrdWOYmUjufQ9GBVPqZ+tu+jMtXxXNmRJyk=
|
||||
github.com/lucas-clemente/quic-go v0.27.1/go.mod h1:AzgQoPda7N+3IqMMMkywBKggIFo2KT6pfnlrQ2QieeI=
|
||||
github.com/lucasb-eyer/go-colorful v1.0.3/go.mod h1:R4dSotOR9KMtayYi1e77YzuveK+i7ruzyGqttikkLy0=
|
||||
github.com/lucasb-eyer/go-colorful v1.2.0 h1:1nnpGOrhyZZuNyfu1QjKiUICQ74+3FNCN69Aj6K7nkY=
|
||||
github.com/lucasb-eyer/go-colorful v1.2.0/go.mod h1:R4dSotOR9KMtayYi1e77YzuveK+i7ruzyGqttikkLy0=
|
||||
github.com/lunixbochs/vtclean v1.0.0/go.mod h1:pHhQNgMf3btfWnGBVipUOjRYhoOsdGqdm/+2c2E2WMI=
|
||||
|
@ -1256,10 +1260,11 @@ github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9
|
|||
github.com/mattn/go-isatty v0.0.14/go.mod h1:7GGIvUiUoEMVVmxf/4nioHXj79iQHKdU27kJ6hsGG94=
|
||||
github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU=
|
||||
github.com/mattn/go-runewidth v0.0.3/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU=
|
||||
github.com/mattn/go-runewidth v0.0.7/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI=
|
||||
github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI=
|
||||
github.com/mattn/go-runewidth v0.0.10 h1:CoZ3S2P7pvtP45xOtBw+/mDL2z0RKI576gSkzRRpdGg=
|
||||
github.com/mattn/go-runewidth v0.0.10/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk=
|
||||
github.com/mattn/go-runewidth v0.0.12/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk=
|
||||
github.com/mattn/go-runewidth v0.0.13 h1:lTGmDsbAYt5DmK6OnoV7EuIF1wEIFAcxld6ypU4OSgU=
|
||||
github.com/mattn/go-runewidth v0.0.13/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w=
|
||||
github.com/mattn/go-shellwords v1.0.3/go.mod h1:3xCvwCdWdlDJUrvuMn7Wuy9eWs4pE8vqg+NOMyg4B2o=
|
||||
github.com/mattn/go-shellwords v1.0.6/go.mod h1:3xCvwCdWdlDJUrvuMn7Wuy9eWs4pE8vqg+NOMyg4B2o=
|
||||
github.com/mattn/go-shellwords v1.0.12/go.mod h1:EZzvwXDESEeg03EKmM+RmDnNOPKG4lLtQsUlTZDWQ8Y=
|
||||
|
@ -1338,6 +1343,17 @@ github.com/mr-tron/base58 v1.2.0 h1:T/HDJBh4ZCPbU39/+c3rRvE0uKBQlU27+QI8LJ4t64o=
|
|||
github.com/mr-tron/base58 v1.2.0/go.mod h1:BinMc/sQntlIE1frQmRFPUoPA1Zkr8VRgBdjWI2mNwc=
|
||||
github.com/mrunalp/fileutils v0.5.0/go.mod h1:M1WthSahJixYnrXQl/DFQuteStB1weuxD2QJNHXfbSQ=
|
||||
github.com/mschoch/smat v0.0.0-20160514031455-90eadee771ae/go.mod h1:qAyveg+e4CE+eKJXWVjKXM4ck2QobLqTDytGJbLLhJg=
|
||||
github.com/muesli/ansi v0.0.0-20211018074035-2e021307bc4b h1:1XF24mVaiu7u+CFywTdcDo2ie1pzzhwjt6RHqzpMU34=
|
||||
github.com/muesli/ansi v0.0.0-20211018074035-2e021307bc4b/go.mod h1:fQuZ0gauxyBcmsdE3ZT4NasjaRdxmbCS0jRHsrWu3Ho=
|
||||
github.com/muesli/cancelreader v0.2.0/go.mod h1:3XuTXfFS2VjM+HTLZY9Ak0l6eUKfijIfMUZ4EgX0QYo=
|
||||
github.com/muesli/cancelreader v0.2.1 h1:Xzd1B4U5bWQOuSKuN398MyynIGTNT89dxzpEDsalXZs=
|
||||
github.com/muesli/cancelreader v0.2.1/go.mod h1:3XuTXfFS2VjM+HTLZY9Ak0l6eUKfijIfMUZ4EgX0QYo=
|
||||
github.com/muesli/reflow v0.2.1-0.20210115123740-9e1d0d53df68/go.mod h1:Xk+z4oIWdQqJzsxyjgl3P22oYZnHdZ8FFTHAQQt5BMQ=
|
||||
github.com/muesli/reflow v0.3.0 h1:IFsN6K9NfGtjeggFP+68I4chLZV2yIKsXJFNZ+eWh6s=
|
||||
github.com/muesli/reflow v0.3.0/go.mod h1:pbwTDkVPibjO2kyvBQRBxTWEEGDGq0FlB1BIKtnHY/8=
|
||||
github.com/muesli/termenv v0.11.1-0.20220204035834-5ac8409525e0/go.mod h1:Bd5NYQ7pd+SrtBSrSNoBBmXlcY8+Xj4BMJgh8qcZrvs=
|
||||
github.com/muesli/termenv v0.11.1-0.20220212125758-44cd13922739 h1:QANkGiGr39l1EESqrE0gZw0/AJNYzIvoGLhIoVYtluI=
|
||||
github.com/muesli/termenv v0.11.1-0.20220212125758-44cd13922739/go.mod h1:Bd5NYQ7pd+SrtBSrSNoBBmXlcY8+Xj4BMJgh8qcZrvs=
|
||||
github.com/multiformats/go-base32 v0.0.3 h1:tw5+NhuwaOjJCC5Pp82QuXbrmLzWg7uxlMFp8Nq/kkI=
|
||||
github.com/multiformats/go-base32 v0.0.3/go.mod h1:pLiuGC8y0QR3Ue4Zug5UzK9LjgbkL8NSQj0zQ5Nz/AA=
|
||||
github.com/multiformats/go-base36 v0.1.0 h1:JR6TyF7JjGd3m6FbLU2cOxhC0Li8z8dLNGQ89tUg4F4=
|
||||
|
@ -1584,8 +1600,6 @@ github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqn
|
|||
github.com/remyoudompheng/bigfft v0.0.0-20190728182440-6a916e37a237/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo=
|
||||
github.com/remyoudompheng/bigfft v0.0.0-20200410134404-eec4a21b6bb0/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo=
|
||||
github.com/retailnext/hllpp v1.0.1-0.20180308014038-101a6d2f8b52/go.mod h1:RDpi1RftBQPUCDRw6SmxeaREsAaRKnOclghuzp/WRzc=
|
||||
github.com/rivo/tview v0.0.0-20210312174852-ae9464cc3598 h1:AbRrGXhagPRDItERv7nauBUUPi7Ma3IGIj9FqkQKW6k=
|
||||
github.com/rivo/tview v0.0.0-20210312174852-ae9464cc3598/go.mod h1:VzCN9WX13RF88iH2CaGkmdHOlsy1ZZQcTmNwROqC+LI=
|
||||
github.com/rivo/uniseg v0.1.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc=
|
||||
github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY=
|
||||
github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc=
|
||||
|
@ -1610,6 +1624,7 @@ github.com/ruudk/golang-pdf417 v0.0.0-20181029194003-1af4ab5afa58/go.mod h1:6lfF
|
|||
github.com/ryanuber/columnize v0.0.0-20160712163229-9b3edd62028f/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts=
|
||||
github.com/safchain/ethtool v0.0.0-20190326074333-42ed695e3de8/go.mod h1:Z0q5wiBQGYcxhMZ6gUqHn6pYNLypFAvaL3UvgZLR0U4=
|
||||
github.com/safchain/ethtool v0.0.0-20210803160452-9aa261dae9b1/go.mod h1:Z0q5wiBQGYcxhMZ6gUqHn6pYNLypFAvaL3UvgZLR0U4=
|
||||
github.com/sahilm/fuzzy v0.1.0/go.mod h1:VFvziUEIMCrT6A6tw2RFIXPXXmzXbOsSHF0DOI8ZK9Y=
|
||||
github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E=
|
||||
github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0=
|
||||
github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw=
|
||||
|
@ -1695,8 +1710,8 @@ github.com/spf13/viper v1.7.0/go.mod h1:8WkrPz2fc9jxqZNCJI/76HCieCp4Q8HaLFoCha5q
|
|||
github.com/src-d/envconfig v1.0.0/go.mod h1:Q9YQZ7BKITldTBnoxsE5gOeB5y66RyPXeue/R4aaNBc=
|
||||
github.com/status-im/go-discover v0.0.0-20220406135310-85a2ce36f63e h1:fDm8hqKGFy8LMNV8zedT3W+QYVPVDfb0F9Fr7fVf9rQ=
|
||||
github.com/status-im/go-discover v0.0.0-20220406135310-85a2ce36f63e/go.mod h1:u1s0ACIlweIjmJrgXyljRPSOflZLaS6ezb044+92W3c=
|
||||
github.com/status-im/go-rln v0.0.7 h1:YhP0iCQIT9C6jGEE5VW2Z7ene1Du0XhQ6JAzgFC3u/8=
|
||||
github.com/status-im/go-rln v0.0.7/go.mod h1:t8Bf4nNAuQh9VlEiz/rgFVtEz7sjEfcd62MIuEnZn8U=
|
||||
github.com/status-im/go-rln v0.0.9 h1:qDmFJoFS/1PGNDB0pEjlRDRNZUpIFBAde2f3fIixCbA=
|
||||
github.com/status-im/go-rln v0.0.9/go.mod h1:t8Bf4nNAuQh9VlEiz/rgFVtEz7sjEfcd62MIuEnZn8U=
|
||||
github.com/status-im/go-waku-rendezvous v0.0.0-20211018070416-a93f3b70c432 h1:cbNFU38iimo9fY4B7CdF/fvIF6tNPJIZjBbpfmW2EY4=
|
||||
github.com/status-im/go-waku-rendezvous v0.0.0-20211018070416-a93f3b70c432/go.mod h1:A8t3i0CUGtXCA0aiLsP7iyikmk/KaD/2XVvNJqGCU20=
|
||||
github.com/status-im/keycard-go v0.0.0-20190316090335-8537d3370df4 h1:Gb2Tyox57NRNuZ2d3rmvB3pcmbu7O1RS3m8WRx7ilrg=
|
||||
|
@ -1754,8 +1769,9 @@ github.com/urfave/cli v1.22.2 h1:gsqYFH8bb9ekPA12kRo0hfjngWQjkJPlN9R0N78BoUo=
|
|||
github.com/urfave/cli v1.22.2/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0=
|
||||
github.com/urfave/cli/v2 v2.3.0/go.mod h1:LJmUH05zAU44vOAcrfzZQKsZbVcdbOG8rtL3/XcUArI=
|
||||
github.com/urfave/cli/v2 v2.10.2/go.mod h1:f8iq5LtQ/bLxafbdBSLPPNsgaW0l/2fYYEHhAyPlwvo=
|
||||
github.com/urfave/cli/v2 v2.10.3 h1:oi571Fxz5aHugfBAJd5nkwSk3fzATXtMlpxdLylSCMo=
|
||||
github.com/urfave/cli/v2 v2.10.3/go.mod h1:f8iq5LtQ/bLxafbdBSLPPNsgaW0l/2fYYEHhAyPlwvo=
|
||||
github.com/urfave/cli/v2 v2.11.1 h1:UKK6SP7fV3eKOefbS87iT9YHefv7iB/53ih6e+GNAsE=
|
||||
github.com/urfave/cli/v2 v2.11.1/go.mod h1:f8iq5LtQ/bLxafbdBSLPPNsgaW0l/2fYYEHhAyPlwvo=
|
||||
github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc=
|
||||
github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8=
|
||||
github.com/valyala/fasttemplate v1.2.1/go.mod h1:KHLXt3tVN2HBp8eijSv/kGJopbvo7S+qRAEEKiv+SiQ=
|
||||
|
@ -2126,7 +2142,6 @@ golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7w
|
|||
golang.org/x/sys v0.0.0-20190606203320-7fc4e5ec1444/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20190616124812-15dcb6c0061f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20190626150813-e07cf5db2756/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
golang.org/x/sys v0.0.0-20190801041406-cbf593c0f2f3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs=
|
||||
|
@ -2237,6 +2252,8 @@ golang.org/x/sys v0.0.0-20211205182925-97ca703d548d/go.mod h1:oPkhp1MJrh7nUepCBc
|
|||
golang.org/x/sys v0.0.0-20211216021012-1d35b9e2eb4e/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
golang.org/x/sys v0.0.0-20220111092808-5a964db01320/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
golang.org/x/sys v0.0.0-20220114195835-da31bd327af9/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
golang.org/x/sys v0.0.0-20220204135822-1c1b9b1eba6a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
golang.org/x/sys v0.0.0-20220209214540-3681064d5158/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
golang.org/x/sys v0.0.0-20220317061510-51cd9980dadf/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
golang.org/x/sys v0.0.0-20220412211240-33da011f77ad/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
golang.org/x/sys v0.0.0-20220422013727-9388b58f7150/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
|
@ -2244,7 +2261,6 @@ golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a h1:dGzPydgVsqGcTRVwiLJ1jVbuf
|
|||
golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg=
|
||||
golang.org/x/term v0.0.0-20201117132131-f5c789dd3221/go.mod h1:Nr5EML6q2oocZ2LXRh80K7BxOlk5/8JxuGnuhpl+muw=
|
||||
golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
|
||||
golang.org/x/term v0.0.0-20201210144234-2321bbc49cbf/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
|
||||
golang.org/x/term v0.0.0-20210220032956-6a3ed077a48d/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo=
|
||||
golang.org/x/term v0.0.0-20210615171337-6886f2dfbf5b/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8=
|
||||
golang.org/x/term v0.0.0-20210927222741-03fcf44c2211 h1:JGgROgKl9N8DuW20oFS5gxc+lE67/N3FcwmBPMe7ArY=
|
||||
|
|
|
@ -1,360 +1,40 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/rand"
|
||||
"encoding/hex"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"flag"
|
||||
"fmt"
|
||||
mrand "math/rand"
|
||||
"net"
|
||||
"os"
|
||||
"time"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/ethereum/go-ethereum/crypto"
|
||||
logging "github.com/ipfs/go-log"
|
||||
"github.com/libp2p/go-libp2p-core/peer"
|
||||
"github.com/libp2p/go-libp2p-core/protocol"
|
||||
"github.com/status-im/go-rln/rln"
|
||||
wakuprotocol "github.com/status-im/go-waku/waku/v2/protocol"
|
||||
"github.com/status-im/go-waku/waku/v2/utils"
|
||||
|
||||
"github.com/multiformats/go-multiaddr"
|
||||
"github.com/status-im/go-waku/waku/v2/dnsdisc"
|
||||
"github.com/status-im/go-waku/waku/v2/node"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/filter"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/lightpush"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/pb"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/store"
|
||||
"github.com/urfave/cli/v2"
|
||||
)
|
||||
|
||||
var DefaultContentTopic string = wakuprotocol.NewContentTopic("toy-chat", 2, "luzhou", "proto").String()
|
||||
var options Options
|
||||
|
||||
func main() {
|
||||
mrand.Seed(time.Now().UTC().UnixNano())
|
||||
app := &cli.App{
|
||||
Flags: getFlags(),
|
||||
Action: func(c *cli.Context) error {
|
||||
// for go-libp2p loggers
|
||||
logLevel := "panic" // to mute output from logs
|
||||
lvl, err := logging.LevelFromString(logLevel)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
logging.SetAllLoggers(lvl)
|
||||
|
||||
// Display panic level to reduce log noise
|
||||
lvl, err := logging.LevelFromString("panic")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
logging.SetAllLoggers(lvl)
|
||||
// go-waku logger
|
||||
err = utils.SetLogLevel(logLevel)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// go-waku logger
|
||||
err = utils.SetLogLevel("panic")
|
||||
if err != nil {
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
nickFlag := flag.String("nick", "", "nickname to use in chat. will be generated if empty")
|
||||
fleetFlag := flag.String("fleet", "wakuv2.prod", "Select the fleet to connect to. (wakuv2.prod, wakuv2.test)")
|
||||
contentTopicFlag := flag.String("contenttopic", DefaultContentTopic, "content topic to use for the chat")
|
||||
nodeKeyFlag := flag.String("nodekey", "", "private key for this node. will be generated if empty")
|
||||
staticNodeFlag := flag.String("staticnode", "", "connects to a node. will get a random node from fleets.status.im if empty")
|
||||
relayFlag := flag.Bool("relay", true, "enable relay protocol")
|
||||
storeNodeFlag := flag.String("storenode", "", "connects to a store node to retrieve messages. will get a random node from fleets.status.im if empty")
|
||||
port := flag.Int("port", 0, "port. Will be random if 0")
|
||||
payloadV1Flag := flag.Bool("payloadV1", false, "use Waku v1 payload encoding/encryption. default false")
|
||||
filterFlag := flag.Bool("filter", false, "enable filter protocol")
|
||||
filterNodeFlag := flag.String("filternode", "", "multiaddr of peer to to request content filtering of messages")
|
||||
lightPushFlag := flag.Bool("lightpush", false, "enable lightpush protocol")
|
||||
lightPushNodeFlag := flag.String("lightpushnode", "", "Multiaddr of peer to to request lightpush of published messages")
|
||||
keepAliveFlag := flag.Int64("keep-alive", 20, "interval in seconds for pinging peers to keep the connection alive.")
|
||||
|
||||
dnsDiscoveryFlag := flag.Bool("dns-discovery", false, "enable dns discovery")
|
||||
dnsDiscoveryUrlFlag := flag.String("dns-discovery-url", "", "URL for DNS node list in format 'enrtree://<key>@<fqdn>'")
|
||||
dnsDiscoveryNameServerFlag := flag.String("dns-discovery-nameserver", "", "DNS name server IP to query (empty to use system default)")
|
||||
|
||||
rlnRelayFlag := flag.Bool("rln-relay", false, "enable spam protection through rln-relay")
|
||||
rlnRelayMemIndexFlag := flag.Int("rln-relay-membership-index", 0, "(experimental) the index of node in the rln-relay group: a value between 0-99 inclusive")
|
||||
rlnRelayContentTopicFlag := flag.String("rln-relay-content-topic", "/toy-chat/2/luzhou/proto", "the content topic for which rln-relay gets enabled")
|
||||
rlnRelayPubsubTopicFlag := flag.String("rln-relay-pubsub-topic", "/waku/2/default-waku/proto", "the pubsub topic for which rln-relay gets enabled")
|
||||
|
||||
rlnRelayDynamicFlag := flag.Bool("rln-relay-dynamic", false, "Enable waku-rln-relay with on-chain dynamic group management")
|
||||
rlnRelayIdKeyFlag := flag.String("rln-relay-id", "", "Rln relay identity secret key as a Hex string")
|
||||
rlnRelayIdCommitmentKeyFlag := flag.String("rln-relay-id-commitment", "", "Rln relay identity commitment key as a Hex string")
|
||||
rlnRelayEthAccountPrivKeyFlag := flag.String("eth-account-privatekey", "", "Account private key for an Ethereum testnet")
|
||||
rlnRelayEthClientAddressFlag := flag.String("eth-client-address", "ws://localhost:8545/", "Ethereum testnet client address")
|
||||
rlnRelayEthMemContractAddressFlag := flag.String("eth-mem-contract-address", "", "Address of membership contract on an Ethereum testnet")
|
||||
rlnRelayCredentialsFile := flag.String("rln-relay-credentials-file", "rlnCredentials.txt", "RLN credentials file")
|
||||
|
||||
flag.Parse()
|
||||
|
||||
hostAddr, _ := net.ResolveTCPAddr("tcp", fmt.Sprintf("0.0.0.0:%d", *port))
|
||||
|
||||
if *fleetFlag != "wakuv2.prod" && *fleetFlag != "wakuv2.test" {
|
||||
fmt.Println("Invalid fleet. Valid values are wakuv2.prod and wakuv2.test")
|
||||
return
|
||||
}
|
||||
|
||||
// use the nickname from the cli flag, or a default if blank
|
||||
nodekey := *nodeKeyFlag
|
||||
if len(nodekey) == 0 {
|
||||
var err error
|
||||
nodekey, err = randomHex(32)
|
||||
if err != nil {
|
||||
fmt.Println("Could not generate random key")
|
||||
return
|
||||
}
|
||||
}
|
||||
prvKey, err := crypto.HexToECDSA(nodekey)
|
||||
|
||||
ctx := context.Background()
|
||||
|
||||
opts := []node.WakuNodeOption{
|
||||
node.WithPrivateKey(prvKey),
|
||||
node.WithHostAddress(hostAddr),
|
||||
node.WithWakuStore(false, false),
|
||||
node.WithKeepAlive(time.Duration(*keepAliveFlag) * time.Second),
|
||||
}
|
||||
|
||||
if *relayFlag {
|
||||
opts = append(opts, node.WithWakuRelay())
|
||||
}
|
||||
|
||||
spamChan := make(chan *pb.WakuMessage, 100)
|
||||
if *rlnRelayFlag {
|
||||
spamHandler := func(message *pb.WakuMessage) error {
|
||||
spamChan <- message
|
||||
execute(options)
|
||||
return nil
|
||||
}
|
||||
|
||||
if *rlnRelayDynamicFlag {
|
||||
key, err := crypto.ToECDSA(common.FromHex(*rlnRelayEthAccountPrivKeyFlag))
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
idKey, idCommitment, index, err := getMembershipCredentials(*rlnRelayCredentialsFile, *rlnRelayIdKeyFlag, *rlnRelayIdCommitmentKeyFlag, *rlnRelayMemIndexFlag)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
fmt.Println("Setting up dynamic rln")
|
||||
opts = append(opts, node.WithDynamicRLNRelay(
|
||||
*rlnRelayPubsubTopicFlag,
|
||||
*rlnRelayContentTopicFlag,
|
||||
index,
|
||||
idKey,
|
||||
idCommitment,
|
||||
spamHandler,
|
||||
*rlnRelayEthClientAddressFlag,
|
||||
key,
|
||||
common.HexToAddress(*rlnRelayEthMemContractAddressFlag),
|
||||
))
|
||||
} else {
|
||||
opts = append(opts, node.WithStaticRLNRelay(*rlnRelayPubsubTopicFlag, *rlnRelayContentTopicFlag, rln.MembershipIndex(*rlnRelayMemIndexFlag), spamHandler))
|
||||
}
|
||||
},
|
||||
}
|
||||
|
||||
if *filterFlag {
|
||||
opts = append(opts, node.WithWakuFilter(false))
|
||||
}
|
||||
|
||||
if *lightPushFlag || *lightPushNodeFlag != "" {
|
||||
*lightPushFlag = true // If a lightpushnode was set and lightpush flag was false
|
||||
opts = append(opts, node.WithLightPush())
|
||||
}
|
||||
|
||||
wakuNode, err := node.New(ctx, opts...)
|
||||
if err != nil {
|
||||
fmt.Print(err)
|
||||
return
|
||||
}
|
||||
|
||||
if *lightPushFlag {
|
||||
addPeer(wakuNode, *lightPushNodeFlag, lightpush.LightPushID_v20beta1)
|
||||
}
|
||||
|
||||
if *filterFlag {
|
||||
addPeer(wakuNode, *filterNodeFlag, filter.FilterID_v20beta1)
|
||||
}
|
||||
|
||||
if err := wakuNode.Start(); err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
if *rlnRelayFlag && *rlnRelayDynamicFlag {
|
||||
err := writeRLNMembershipCredentialsToFile(*rlnRelayCredentialsFile, wakuNode.RLNRelay().MembershipKeyPair(), wakuNode.RLNRelay().MembershipIndex())
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
fmt.Printf("Wrote credentials in file %s\n", *rlnRelayCredentialsFile)
|
||||
}
|
||||
|
||||
// use the nickname from the cli flag, or a default if blank
|
||||
nick := *nickFlag
|
||||
if len(nick) == 0 {
|
||||
nick = defaultNick(wakuNode.Host().ID())
|
||||
}
|
||||
|
||||
// join the chat
|
||||
chat, err := NewChat(ctx, wakuNode, wakuNode.Host().ID(), *contentTopicFlag, *payloadV1Flag, *lightPushFlag, nick, spamChan)
|
||||
err := app.Run(os.Args)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
ui := NewChatUI(ctx, chat)
|
||||
|
||||
// Connect to a static node or use random node from fleets.status.im
|
||||
go func() {
|
||||
time.Sleep(200 * time.Millisecond)
|
||||
|
||||
staticnode := *staticNodeFlag
|
||||
storenode := *storeNodeFlag
|
||||
|
||||
var fleetData []byte
|
||||
if len(staticnode) == 0 || len(storenode) == 0 {
|
||||
fleetData = getFleetData()
|
||||
}
|
||||
|
||||
if len(staticnode) == 0 {
|
||||
ui.displayMessage(fmt.Sprintf("No static peers configured. Choosing one at random from %s fleet...", *fleetFlag))
|
||||
staticnode = getRandomFleetNode(fleetData, *fleetFlag)
|
||||
}
|
||||
|
||||
ctx, cancel := context.WithTimeout(ctx, time.Duration(5)*time.Second)
|
||||
defer cancel()
|
||||
err = wakuNode.DialPeer(ctx, staticnode)
|
||||
if err != nil {
|
||||
ui.displayMessage("Could not connect to peer: " + err.Error())
|
||||
return
|
||||
} else {
|
||||
ui.displayMessage("Connected to peer: " + staticnode)
|
||||
}
|
||||
|
||||
enableDiscovery := *dnsDiscoveryFlag
|
||||
dnsDiscoveryUrl := *dnsDiscoveryUrlFlag
|
||||
dnsDiscoveryNameServer := *dnsDiscoveryNameServerFlag
|
||||
|
||||
if enableDiscovery && dnsDiscoveryUrl != "" {
|
||||
ui.displayMessage(fmt.Sprintf("attempting DNS discovery with %s", dnsDiscoveryUrl))
|
||||
nodes, err := dnsdisc.RetrieveNodes(ctx, dnsDiscoveryUrl, dnsdisc.WithNameserver(dnsDiscoveryNameServer))
|
||||
if err != nil {
|
||||
ui.displayMessage("DNS discovery error: " + err.Error())
|
||||
} else {
|
||||
for _, n := range nodes {
|
||||
for _, m := range n.Addresses {
|
||||
go func(ctx context.Context, m multiaddr.Multiaddr) {
|
||||
ctx, cancel := context.WithTimeout(ctx, time.Duration(3)*time.Second)
|
||||
defer cancel()
|
||||
err = wakuNode.DialPeerWithMultiAddress(ctx, m)
|
||||
if err != nil {
|
||||
ui.displayMessage("error dialing peer: " + err.Error())
|
||||
}
|
||||
}(ctx, m)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if len(storenode) == 0 {
|
||||
ui.displayMessage(fmt.Sprintf("No store node configured. Choosing one at random from %s fleet...", *fleetFlag))
|
||||
storenode = getRandomFleetNode(fleetData, *fleetFlag)
|
||||
}
|
||||
|
||||
storeNodeId, err := addPeer(wakuNode, storenode, store.StoreID_v20beta4)
|
||||
if err != nil {
|
||||
ui.displayMessage("Could not connect to storenode: " + err.Error())
|
||||
return
|
||||
} else {
|
||||
ui.displayMessage("Connected to storenode: " + storenode)
|
||||
}
|
||||
|
||||
time.Sleep(300 * time.Millisecond)
|
||||
ui.displayMessage("Querying historic messages")
|
||||
|
||||
tCtx, _ := context.WithTimeout(ctx, 5*time.Second)
|
||||
|
||||
q := store.Query{
|
||||
ContentTopics: []string{*contentTopicFlag},
|
||||
}
|
||||
response, err := wakuNode.Store().Query(tCtx, q,
|
||||
store.WithAutomaticRequestId(),
|
||||
store.WithPeer(*storeNodeId),
|
||||
store.WithPaging(true, 0))
|
||||
|
||||
if err != nil {
|
||||
ui.displayMessage("Could not query storenode: " + err.Error())
|
||||
} else {
|
||||
chat.displayMessages(response.Messages)
|
||||
}
|
||||
}()
|
||||
|
||||
//draw the UI
|
||||
if err = ui.Run(); err != nil {
|
||||
printErr("error running text UI: %s", err)
|
||||
}
|
||||
|
||||
wakuNode.Stop()
|
||||
// TODO: filter unsubscribeAll
|
||||
|
||||
}
|
||||
|
||||
// Generates a random hex string with a length of n
|
||||
func randomHex(n int) (string, error) {
|
||||
bytes := make([]byte, n)
|
||||
if _, err := rand.Read(bytes); err != nil {
|
||||
return "", err
|
||||
}
|
||||
return hex.EncodeToString(bytes), nil
|
||||
}
|
||||
|
||||
// printErr is like fmt.Printf, but writes to stderr.
|
||||
func printErr(m string, args ...interface{}) {
|
||||
fmt.Fprintf(os.Stderr, m, args...)
|
||||
}
|
||||
|
||||
// defaultNick generates a nickname based on the $USER environment variable and
|
||||
// the last 8 chars of a peer ID.
|
||||
func defaultNick(p peer.ID) string {
|
||||
return fmt.Sprintf("%s-%s", os.Getenv("USER"), shortID(p))
|
||||
}
|
||||
|
||||
// shortID returns the last 8 chars of a base58-encoded peer id.
|
||||
func shortID(p peer.ID) string {
|
||||
pretty := p.Pretty()
|
||||
return pretty[len(pretty)-8:]
|
||||
}
|
||||
|
||||
func getFleetData() []byte {
|
||||
b, err := os.ReadFile("fleets.json")
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
return b
|
||||
}
|
||||
|
||||
func getRandomFleetNode(data []byte, fleetId string) string {
|
||||
var result map[string]interface{}
|
||||
json.Unmarshal(data, &result)
|
||||
fleets := result["fleets"].(map[string]interface{})
|
||||
fleet := fleets[fleetId].(map[string]interface{})
|
||||
waku := fleet["waku"].(map[string]interface{})
|
||||
|
||||
var wakunodes []string
|
||||
for v := range waku {
|
||||
wakunodes = append(wakunodes, v)
|
||||
break
|
||||
}
|
||||
|
||||
randKey := wakunodes[mrand.Intn(len(wakunodes))]
|
||||
|
||||
return waku[randKey].(string)
|
||||
}
|
||||
|
||||
func addPeer(wakuNode *node.WakuNode, addr string, protocol protocol.ID) (*peer.ID, error) {
|
||||
if addr == "" {
|
||||
return nil, errors.New("invalid multiaddress")
|
||||
}
|
||||
|
||||
ma, err := multiaddr.NewMultiaddr(addr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return wakuNode.AddPeer(ma, string(protocol))
|
||||
}
|
||||
|
|
|
@ -0,0 +1,144 @@
|
|||
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
|
||||
CredentialsFile string
|
||||
MembershipIndex int
|
||||
PubsubTopic string
|
||||
ContentTopic string
|
||||
Dynamic bool
|
||||
IDKey string
|
||||
IDCommitment string
|
||||
ETHPrivateKey *ecdsa.PrivateKey
|
||||
ETHClientAddress string
|
||||
MembershipContractAddress common.Address
|
||||
}
|
||||
|
||||
// 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) {
|
||||
if f.Node == nil {
|
||||
return peer.ID(""), errors.New("node is nil")
|
||||
}
|
||||
|
||||
peerID, err := (*f.Node).ValueForProtocol(multiaddr.P_P2P)
|
||||
if err != nil {
|
||||
return peer.ID(""), err
|
||||
}
|
||||
|
||||
return peer.Decode(peerID)
|
||||
}
|
||||
|
||||
// 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) {
|
||||
if f.Node == nil {
|
||||
return peer.ID(""), errors.New("node is nil")
|
||||
}
|
||||
|
||||
peerID, err := (*f.Node).ValueForProtocol(multiaddr.P_P2P)
|
||||
if err != nil {
|
||||
return peer.ID(""), err
|
||||
}
|
||||
|
||||
return peer.Decode(peerID)
|
||||
}
|
||||
|
||||
// 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) {
|
||||
if f.Node == nil {
|
||||
return peer.ID(""), errors.New("node is nil")
|
||||
}
|
||||
|
||||
peerID, err := (*f.Node).ValueForProtocol(multiaddr.P_P2P)
|
||||
if err != nil {
|
||||
return peer.ID(""), err
|
||||
}
|
||||
|
||||
return peer.Decode(peerID)
|
||||
}
|
||||
|
||||
// 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
|
||||
UsePayloadV1 bool
|
||||
Nickname string
|
||||
StaticNodes []multiaddr.Multiaddr
|
||||
|
||||
Relay RelayOptions
|
||||
Store StoreOptions
|
||||
Filter FilterOptions
|
||||
LightPush LightpushOptions
|
||||
RLNRelay RLNRelayOptions
|
||||
DiscV5 DiscV5Options
|
||||
DNSDiscovery DNSDiscoveryOptions
|
||||
}
|
|
@ -11,9 +11,14 @@ import (
|
|||
"github.com/status-im/go-rln/rln"
|
||||
)
|
||||
|
||||
type membershipKeyPair struct {
|
||||
IDKey rln.IDKey `json:"idKey"`
|
||||
IDCommitment rln.IDCommitment `json:"idCommitment"`
|
||||
}
|
||||
|
||||
type membershipCredentials struct {
|
||||
Keypair rln.MembershipKeyPair `json:"keypair"`
|
||||
Index rln.MembershipIndex `json:"index"`
|
||||
Keypair membershipKeyPair `json:"membershipKeyPair"`
|
||||
Index rln.MembershipIndex `json:"rlnIndex"`
|
||||
}
|
||||
|
||||
func fileExists(path string) bool {
|
||||
|
@ -34,7 +39,7 @@ func writeRLNMembershipCredentialsToFile(path string, keyPair rln.MembershipKeyP
|
|||
}
|
||||
|
||||
credentialsJSON, err := json.Marshal(membershipCredentials{
|
||||
Keypair: keyPair,
|
||||
Keypair: membershipKeyPair(keyPair),
|
||||
Index: idx,
|
||||
})
|
||||
if err != nil {
|
||||
|
@ -56,19 +61,27 @@ func loadMembershipCredentialsFromFile(path string) (rln.MembershipKeyPair, rln.
|
|||
return rln.MembershipKeyPair{}, rln.MembershipIndex(0), err
|
||||
}
|
||||
|
||||
return credentials.Keypair, credentials.Index, err
|
||||
return rln.MembershipKeyPair(credentials.Keypair), credentials.Index, err
|
||||
}
|
||||
|
||||
func getMembershipCredentials(path string, rlnIDKey string, rlnIDCommitment string, rlnMembershipIndex int) (idKey *rln.IDKey, idCommitment *rln.IDCommitment, index rln.MembershipIndex, err error) {
|
||||
if _, err = os.Stat(path); err == nil {
|
||||
if keyPair, index, err := loadMembershipCredentialsFromFile(path); err != nil {
|
||||
return nil, nil, rln.MembershipIndex(0), fmt.Errorf("could not read membership credentials file: %w", err)
|
||||
} else {
|
||||
return &keyPair.IDKey, &keyPair.IDCommitment, index, nil
|
||||
valuesWereInput := false
|
||||
if rlnIDKey != "" || rlnIDCommitment != "" {
|
||||
valuesWereInput = true
|
||||
}
|
||||
|
||||
var osErr error
|
||||
if !valuesWereInput {
|
||||
if _, osErr = os.Stat(path); osErr == nil {
|
||||
if keyPair, index, err := loadMembershipCredentialsFromFile(path); err != nil {
|
||||
return nil, nil, rln.MembershipIndex(0), fmt.Errorf("could not read membership credentials file: %w", err)
|
||||
} else {
|
||||
return &keyPair.IDKey, &keyPair.IDCommitment, index, nil
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if os.IsNotExist(err) {
|
||||
if valuesWereInput || os.IsNotExist(osErr) {
|
||||
if rlnIDKey != "" {
|
||||
idKey = new(rln.IDKey)
|
||||
copy((*idKey)[:], common.FromHex(rlnIDKey))
|
||||
|
|
|
@ -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
|
||||
}
|
|
@ -1,227 +1,341 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"chat2/pb"
|
||||
"context"
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"io"
|
||||
"strings"
|
||||
"time"
|
||||
|
||||
"github.com/gdamore/tcell/v2"
|
||||
"github.com/rivo/tview"
|
||||
"github.com/status-im/go-waku/waku/v2/protocol/relay"
|
||||
"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"
|
||||
)
|
||||
|
||||
// ChatUI is a Text User Interface (TUI) for a ChatRoom.
|
||||
// The Run method will draw the UI to the terminal in "fullscreen"
|
||||
// mode. You can quit with Ctrl-C, or by typing "/quit" into the
|
||||
// chat prompt.
|
||||
type ChatUI struct {
|
||||
app *tview.Application
|
||||
chat *Chat
|
||||
const viewportMargin = 6
|
||||
|
||||
msgW io.Writer
|
||||
inputCh chan string
|
||||
doneCh chan struct{}
|
||||
var (
|
||||
appStyle = lipgloss.NewStyle().Padding(1, 2)
|
||||
|
||||
ctx context.Context
|
||||
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
|
||||
}
|
||||
|
||||
// NewChatUI returns a new ChatUI struct that controls the text UI.
|
||||
// It won't actually do anything until you call Run().
|
||||
func NewChatUI(ctx context.Context, chat *Chat) *ChatUI {
|
||||
chatUI := new(ChatUI)
|
||||
type UI struct {
|
||||
ready bool
|
||||
err error
|
||||
|
||||
app := tview.NewApplication()
|
||||
quitChan chan struct{}
|
||||
readyChan chan<- struct{}
|
||||
inputChan chan<- string
|
||||
|
||||
// make a NewChatUI text view to contain our chat messages
|
||||
msgBox := tview.NewTextView()
|
||||
msgBox.SetDynamicColors(true)
|
||||
msgBox.SetBorder(true)
|
||||
msgBox.SetTitle("chat2 example")
|
||||
messageChan chan message
|
||||
messages []message
|
||||
|
||||
// text views are io.Writers, but they don't automatically refresh.
|
||||
// this sets a change handler to force the app to redraw when we get
|
||||
// new messages to display.
|
||||
msgBox.SetChangedFunc(func() {
|
||||
app.Draw()
|
||||
})
|
||||
isSendingChan chan sending
|
||||
isSending bool
|
||||
|
||||
// an input field for typing messages into
|
||||
inputCh := make(chan string, 32)
|
||||
input := tview.NewInputField().
|
||||
SetLabel(chat.nick + " > ").
|
||||
SetFieldWidth(0).
|
||||
SetFieldBackgroundColor(tcell.ColorBlack)
|
||||
width int
|
||||
height int
|
||||
|
||||
// the done func is called when the user hits enter, or tabs out of the field
|
||||
input.SetDoneFunc(func(key tcell.Key) {
|
||||
if key != tcell.KeyEnter {
|
||||
// we don't want to do anything if they just tabbed away
|
||||
return
|
||||
}
|
||||
line := input.GetText()
|
||||
viewport viewport.Model
|
||||
textarea textarea.Model
|
||||
|
||||
if len(line) == 0 {
|
||||
// ignore blank lines
|
||||
return
|
||||
}
|
||||
spinner spinner.Model
|
||||
}
|
||||
|
||||
input.SetText("")
|
||||
func NewUIModel(readyChan chan<- struct{}, inputChan chan<- string) UI {
|
||||
width, height := GetTerminalDimensions()
|
||||
|
||||
// bail if requested
|
||||
if line == "/quit" {
|
||||
app.Stop()
|
||||
return
|
||||
}
|
||||
ta := textarea.New()
|
||||
ta.Placeholder = "Send a message..."
|
||||
ta.Focus()
|
||||
|
||||
// add peer
|
||||
if strings.HasPrefix(line, "/connect ") {
|
||||
peer := strings.TrimPrefix(line, "/connect ")
|
||||
go func(peer string) {
|
||||
chatUI.displayMessage("Connecting to peer...")
|
||||
ctx, cancel := context.WithTimeout(ctx, time.Duration(5)*time.Second)
|
||||
defer cancel()
|
||||
err := chat.node.DialPeer(ctx, peer)
|
||||
if err != nil {
|
||||
chatUI.displayMessage(err.Error())
|
||||
} else {
|
||||
chatUI.displayMessage("Peer connected successfully")
|
||||
}
|
||||
}(peer)
|
||||
return
|
||||
}
|
||||
ta.Prompt = "┃ "
|
||||
ta.CharLimit = 2000
|
||||
|
||||
// list peers
|
||||
if line == "/peers" {
|
||||
peers := chat.node.Relay().PubSub().ListPeers(string(relay.DefaultWakuTopic))
|
||||
if len(peers) == 0 {
|
||||
chatUI.displayMessage("No peers available")
|
||||
}
|
||||
for _, p := range peers {
|
||||
chatUI.displayMessage("- " + p.Pretty())
|
||||
}
|
||||
return
|
||||
}
|
||||
// Remove cursor line styling
|
||||
ta.FocusedStyle.CursorLine = lipgloss.NewStyle()
|
||||
ta.SetHeight(3)
|
||||
ta.SetWidth(width)
|
||||
ta.ShowLineNumbers = false
|
||||
|
||||
// change nick
|
||||
if strings.HasPrefix(line, "/nick ") {
|
||||
newNick := strings.TrimSpace(strings.TrimPrefix(line, "/nick "))
|
||||
chat.nick = newNick
|
||||
input.SetLabel(chat.nick + " > ")
|
||||
return
|
||||
}
|
||||
ta.KeyMap.InsertNewline.SetEnabled(false)
|
||||
|
||||
if line == "/help" {
|
||||
chatUI.displayMessage(`
|
||||
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
|
||||
/quit - closes the app
|
||||
`)
|
||||
return
|
||||
}
|
||||
s := spinner.New()
|
||||
s.Spinner = spinner.Jump
|
||||
s.Style = lipgloss.NewStyle().Foreground(lipgloss.Color("205"))
|
||||
|
||||
// send the line onto the input chan and reset the field text
|
||||
inputCh <- line
|
||||
})
|
||||
|
||||
chatPanel := tview.NewFlex().
|
||||
AddItem(msgBox, 0, 1, false)
|
||||
|
||||
// flex is a vertical box with the chatPanel on top and the input field at the bottom.
|
||||
flex := tview.NewFlex().
|
||||
SetDirection(tview.FlexRow).
|
||||
AddItem(chatPanel, 0, 1, false).
|
||||
AddItem(input, 1, 1, true)
|
||||
|
||||
app.SetRoot(flex, true)
|
||||
|
||||
chatUI.app = app
|
||||
chatUI.msgW = msgBox
|
||||
chatUI.chat = chat
|
||||
chatUI.ctx = ctx
|
||||
chatUI.inputCh = inputCh
|
||||
chatUI.doneCh = make(chan struct{}, 1)
|
||||
|
||||
for _, addr := range chat.node.ListenAddresses() {
|
||||
chatUI.displayMessage(fmt.Sprintf("Listening on %s", addr))
|
||||
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 chatUI
|
||||
return m
|
||||
}
|
||||
|
||||
// Run starts the chat event loop in the background, then starts
|
||||
// the event loop for the text UI.
|
||||
func (ui *ChatUI) Run() error {
|
||||
ui.displayMessage("\nWelcome, " + ui.chat.nick)
|
||||
ui.displayMessage("type /help to see available commands \n")
|
||||
func (m UI) Init() tea.Cmd {
|
||||
return tea.Batch(
|
||||
recvQuitSignal(m.quitChan),
|
||||
recvMessages(m.messageChan),
|
||||
recvSendingState(m.isSendingChan),
|
||||
textarea.Blink,
|
||||
spinner.Tick,
|
||||
)
|
||||
}
|
||||
|
||||
if ui.chat.node.RLNRelay() != nil {
|
||||
func (m UI) Update(msg tea.Msg) (tea.Model, tea.Cmd) {
|
||||
var (
|
||||
tiCmd tea.Cmd
|
||||
vpCmd tea.Cmd
|
||||
)
|
||||
|
||||
idKey := ui.chat.node.RLNRelay().MembershipKeyPair().IDKey
|
||||
idCommitment := ui.chat.node.RLNRelay().MembershipKeyPair().IDCommitment
|
||||
m.textarea, tiCmd = m.textarea.Update(msg)
|
||||
m.viewport, vpCmd = m.viewport.Update(msg)
|
||||
|
||||
ui.displayMessage("RLN config:")
|
||||
ui.displayMessage(fmt.Sprintf("- Your membership index is: %d", uint(ui.chat.node.RLNRelay().MembershipIndex())))
|
||||
ui.displayMessage(fmt.Sprintf("- Your rln identity key is: 0x%s", hex.EncodeToString(idKey[:])))
|
||||
ui.displayMessage(fmt.Sprintf("- Your rln identity commitment key is: 0x%s\n", hex.EncodeToString(idCommitment[:])))
|
||||
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:
|
||||
m.textarea.Placeholder = "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
|
||||
}
|
||||
|
||||
go ui.handleEvents()
|
||||
defer ui.end()
|
||||
if printMessages {
|
||||
var sb strings.Builder
|
||||
for i, msg := range m.messages {
|
||||
line := ""
|
||||
|
||||
return ui.app.Run()
|
||||
}
|
||||
|
||||
// end signals the event loop to exit gracefully
|
||||
func (ui *ChatUI) end() {
|
||||
ui.doneCh <- struct{}{}
|
||||
}
|
||||
|
||||
// displayChatMessage writes a ChatMessage from the room to the message window,
|
||||
// with the sender's nick highlighted in green.
|
||||
func (ui *ChatUI) displayChatMessage(cm *pb.Chat2Message) {
|
||||
t := time.Unix(int64(cm.Timestamp), 0)
|
||||
prompt := withColor("green", fmt.Sprintf("<%s> %s:", t.Format("Jan 02, 15:04"), cm.Nick))
|
||||
fmt.Fprintf(ui.msgW, "%s %s\n", prompt, cm.Payload)
|
||||
}
|
||||
|
||||
// displayMessage writes a blue message to output
|
||||
func (ui *ChatUI) displayMessage(msg string) {
|
||||
fmt.Fprintf(ui.msgW, "%s\n", withColor("grey", msg))
|
||||
}
|
||||
|
||||
// handleEvents runs an event loop that sends user input to the chat room
|
||||
// and displays messages received from the chat room. It also periodically
|
||||
// refreshes the list of peers in the UI.
|
||||
func (ui *ChatUI) handleEvents() {
|
||||
for {
|
||||
select {
|
||||
case input := <-ui.inputCh:
|
||||
err := ui.chat.Publish(ui.ctx, input)
|
||||
if err != nil {
|
||||
printErr("publish error: %s", err)
|
||||
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)
|
||||
case InfoMessageType:
|
||||
line += m.breaklineIfNeeded(i, InfoMessageType)
|
||||
line += wordwrap.String(infoStyle("INFO:")+" "+msg.content, m.width-10)
|
||||
}
|
||||
|
||||
case m := <-ui.chat.Messages:
|
||||
// when we receive a message from the chat room, print it to the message window
|
||||
ui.displayChatMessage(m)
|
||||
sb.WriteString(line + "\n")
|
||||
|
||||
case <-ui.ctx.Done():
|
||||
return
|
||||
|
||||
case <-ui.doneCh:
|
||||
return
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
}
|
||||
|
||||
// withColor wraps a string with color tags for display in the messages text box.
|
||||
func withColor(color, msg string) string {
|
||||
return fmt.Sprintf("[%s]%s[-]", color, msg)
|
||||
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"),
|
||||
),
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue