mirror of
https://github.com/status-im/status-go.git
synced 2025-01-09 22:26:30 +00:00
cec5985066
This avoids locking of the community until the end of reevaluation. There is no special handling for community changes while reevaluation is ongoing. If members are added or removed, the function will behave correctly. If permissions are changed, they will be accommodated in the next reevaluation. fixes: status-im/status-desktop#14775
6013 lines
182 KiB
Go
6013 lines
182 KiB
Go
package communities
|
|
|
|
import (
|
|
"bytes"
|
|
"context"
|
|
"crypto/ecdsa"
|
|
"database/sql"
|
|
"encoding/hex"
|
|
"fmt"
|
|
"io/ioutil"
|
|
"net"
|
|
"os"
|
|
"sort"
|
|
"strconv"
|
|
"strings"
|
|
"sync"
|
|
"time"
|
|
|
|
"github.com/anacrolix/torrent"
|
|
"github.com/anacrolix/torrent/bencode"
|
|
"github.com/anacrolix/torrent/metainfo"
|
|
"github.com/golang/protobuf/proto"
|
|
|
|
"github.com/google/uuid"
|
|
"github.com/pkg/errors"
|
|
"go.uber.org/zap"
|
|
|
|
gethcommon "github.com/ethereum/go-ethereum/common"
|
|
"github.com/ethereum/go-ethereum/common/hexutil"
|
|
"github.com/status-im/status-go/account"
|
|
utils "github.com/status-im/status-go/common"
|
|
"github.com/status-im/status-go/eth-node/crypto"
|
|
"github.com/status-im/status-go/eth-node/types"
|
|
"github.com/status-im/status-go/images"
|
|
multiaccountscommon "github.com/status-im/status-go/multiaccounts/common"
|
|
"github.com/status-im/status-go/params"
|
|
"github.com/status-im/status-go/protocol/common"
|
|
"github.com/status-im/status-go/protocol/common/shard"
|
|
community_token "github.com/status-im/status-go/protocol/communities/token"
|
|
"github.com/status-im/status-go/protocol/encryption"
|
|
"github.com/status-im/status-go/protocol/ens"
|
|
"github.com/status-im/status-go/protocol/protobuf"
|
|
"github.com/status-im/status-go/protocol/requests"
|
|
"github.com/status-im/status-go/protocol/transport"
|
|
"github.com/status-im/status-go/services/wallet/bigint"
|
|
walletcommon "github.com/status-im/status-go/services/wallet/common"
|
|
"github.com/status-im/status-go/services/wallet/thirdparty"
|
|
"github.com/status-im/status-go/services/wallet/token"
|
|
"github.com/status-im/status-go/signal"
|
|
"github.com/status-im/status-go/transactions"
|
|
)
|
|
|
|
var defaultAnnounceList = [][]string{
|
|
{"udp://tracker.opentrackr.org:1337/announce"},
|
|
{"udp://tracker.openbittorrent.com:6969/announce"},
|
|
}
|
|
var pieceLength = 100 * 1024
|
|
|
|
const maxArchiveSizeInBytes = 30000000
|
|
|
|
var maxNbMembers = 5000
|
|
var maxNbPendingRequestedMembers = 100
|
|
|
|
var memberPermissionsCheckInterval = 1 * time.Hour
|
|
var validateInterval = 2 * time.Minute
|
|
|
|
// Used for testing only
|
|
func SetValidateInterval(duration time.Duration) {
|
|
validateInterval = duration
|
|
}
|
|
func SetMaxNbMembers(maxNb int) {
|
|
maxNbMembers = maxNb
|
|
}
|
|
func SetMaxNbPendingRequestedMembers(maxNb int) {
|
|
maxNbPendingRequestedMembers = maxNb
|
|
}
|
|
|
|
// errors
|
|
var (
|
|
ErrTorrentTimedout = errors.New("torrent has timed out")
|
|
ErrCommunityRequestAlreadyRejected = errors.New("that user was already rejected from the community")
|
|
ErrInvalidClock = errors.New("invalid clock to cancel request to join")
|
|
)
|
|
|
|
type Manager struct {
|
|
persistence *Persistence
|
|
encryptor *encryption.Protocol
|
|
ensSubscription chan []*ens.VerificationRecord
|
|
subscriptions []chan *Subscription
|
|
ensVerifier *ens.Verifier
|
|
ownerVerifier OwnerVerifier
|
|
identity *ecdsa.PrivateKey
|
|
installationID string
|
|
accountsManager account.Manager
|
|
tokenManager TokenManager
|
|
collectiblesManager CollectiblesManager
|
|
logger *zap.Logger
|
|
stdoutLogger *zap.Logger
|
|
transport *transport.Transport
|
|
timesource common.TimeSource
|
|
quit chan struct{}
|
|
torrentConfig *params.TorrentConfig
|
|
torrentClient *torrent.Client
|
|
walletConfig *params.WalletConfig
|
|
communityTokensService CommunityTokensServiceInterface
|
|
historyArchiveTasksWaitGroup sync.WaitGroup
|
|
historyArchiveTasks sync.Map // stores `chan struct{}`
|
|
membersReevaluationTasks sync.Map // stores `membersReevaluationTask`
|
|
forceMembersReevaluation map[string]chan struct{}
|
|
torrentTasks map[string]metainfo.Hash
|
|
historyArchiveDownloadTasks map[string]*HistoryArchiveDownloadTask
|
|
stopped bool
|
|
RekeyInterval time.Duration
|
|
PermissionChecker PermissionChecker
|
|
keyDistributor KeyDistributor
|
|
communityLock *CommunityLock
|
|
}
|
|
|
|
type CommunityLock struct {
|
|
logger *zap.Logger
|
|
locks map[string]*sync.Mutex
|
|
mutex sync.Mutex
|
|
}
|
|
|
|
func NewCommunityLock(logger *zap.Logger) *CommunityLock {
|
|
return &CommunityLock{
|
|
logger: logger.Named("CommunityLock"),
|
|
locks: make(map[string]*sync.Mutex),
|
|
}
|
|
}
|
|
|
|
func (c *CommunityLock) Lock(communityID types.HexBytes) {
|
|
c.mutex.Lock()
|
|
communityIDStr := types.EncodeHex(communityID)
|
|
lock, ok := c.locks[communityIDStr]
|
|
if !ok {
|
|
lock = &sync.Mutex{}
|
|
c.locks[communityIDStr] = lock
|
|
}
|
|
c.mutex.Unlock()
|
|
|
|
lock.Lock()
|
|
}
|
|
|
|
func (c *CommunityLock) Unlock(communityID types.HexBytes) {
|
|
c.mutex.Lock()
|
|
communityIDStr := types.EncodeHex(communityID)
|
|
lock, ok := c.locks[communityIDStr]
|
|
c.mutex.Unlock()
|
|
|
|
if ok {
|
|
lock.Unlock()
|
|
} else {
|
|
c.logger.Warn("trying to unlock a non-existent lock", zap.String("communityID", communityIDStr))
|
|
}
|
|
}
|
|
|
|
func (c *CommunityLock) Init() {
|
|
c.locks = make(map[string]*sync.Mutex)
|
|
}
|
|
|
|
type HistoryArchiveDownloadTask struct {
|
|
CancelChan chan struct{}
|
|
Waiter sync.WaitGroup
|
|
m sync.RWMutex
|
|
Cancelled bool
|
|
}
|
|
|
|
func (t *HistoryArchiveDownloadTask) IsCancelled() bool {
|
|
t.m.RLock()
|
|
defer t.m.RUnlock()
|
|
return t.Cancelled
|
|
}
|
|
|
|
func (t *HistoryArchiveDownloadTask) Cancel() {
|
|
t.m.Lock()
|
|
defer t.m.Unlock()
|
|
t.Cancelled = true
|
|
close(t.CancelChan)
|
|
}
|
|
|
|
type membersReevaluationTask struct {
|
|
lastStartTime time.Time
|
|
lastSuccessTime time.Time
|
|
onDemandRequestTime time.Time
|
|
mutex sync.Mutex
|
|
}
|
|
|
|
type managerOptions struct {
|
|
accountsManager account.Manager
|
|
tokenManager TokenManager
|
|
collectiblesManager CollectiblesManager
|
|
walletConfig *params.WalletConfig
|
|
communityTokensService CommunityTokensServiceInterface
|
|
permissionChecker PermissionChecker
|
|
|
|
// allowForcingCommunityMembersReevaluation indicates whether we should allow forcing community members reevaluation.
|
|
// This will allow using `force` argument in ScheduleMembersReevaluation.
|
|
// Should only be used in tests.
|
|
allowForcingCommunityMembersReevaluation bool
|
|
}
|
|
|
|
type TokenManager interface {
|
|
GetBalancesByChain(ctx context.Context, accounts, tokens []gethcommon.Address, chainIDs []uint64) (map[uint64]map[gethcommon.Address]map[gethcommon.Address]*hexutil.Big, error)
|
|
FindOrCreateTokenByAddress(ctx context.Context, chainID uint64, address gethcommon.Address) *token.Token
|
|
GetAllChainIDs() ([]uint64, error)
|
|
}
|
|
|
|
type CollectibleContractData struct {
|
|
TotalSupply *bigint.BigInt
|
|
Transferable bool
|
|
RemoteBurnable bool
|
|
InfiniteSupply bool
|
|
}
|
|
|
|
type AssetContractData struct {
|
|
TotalSupply *bigint.BigInt
|
|
InfiniteSupply bool
|
|
}
|
|
|
|
type CommunityTokensServiceInterface interface {
|
|
GetCollectibleContractData(chainID uint64, contractAddress string) (*CollectibleContractData, error)
|
|
SetSignerPubKey(ctx context.Context, chainID uint64, contractAddress string, txArgs transactions.SendTxArgs, password string, newSignerPubKey string) (string, error)
|
|
GetAssetContractData(chainID uint64, contractAddress string) (*AssetContractData, error)
|
|
SafeGetSignerPubKey(ctx context.Context, chainID uint64, communityID string) (string, error)
|
|
DeploymentSignatureDigest(chainID uint64, addressFrom string, communityID string) ([]byte, error)
|
|
}
|
|
|
|
type DefaultTokenManager struct {
|
|
tokenManager *token.Manager
|
|
}
|
|
|
|
func NewDefaultTokenManager(tm *token.Manager) *DefaultTokenManager {
|
|
return &DefaultTokenManager{tokenManager: tm}
|
|
}
|
|
|
|
type BalancesByChain = map[uint64]map[gethcommon.Address]map[gethcommon.Address]*hexutil.Big
|
|
|
|
func (m *DefaultTokenManager) GetAllChainIDs() ([]uint64, error) {
|
|
networks, err := m.tokenManager.RPCClient.NetworkManager.Get(false)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
areTestNetworksEnabled, err := m.tokenManager.RPCClient.NetworkManager.GetTestNetworksEnabled()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
chainIDs := make([]uint64, 0)
|
|
for _, network := range networks {
|
|
if areTestNetworksEnabled == network.IsTest {
|
|
chainIDs = append(chainIDs, network.ChainID)
|
|
}
|
|
}
|
|
return chainIDs, nil
|
|
}
|
|
|
|
type CollectiblesManager interface {
|
|
FetchBalancesByOwnerAndContractAddress(ctx context.Context, chainID walletcommon.ChainID, ownerAddress gethcommon.Address, contractAddresses []gethcommon.Address) (thirdparty.TokenBalancesPerContractAddress, error)
|
|
GetCollectibleOwnership(id thirdparty.CollectibleUniqueID) ([]thirdparty.AccountBalance, error)
|
|
}
|
|
|
|
func (m *DefaultTokenManager) GetBalancesByChain(ctx context.Context, accounts, tokenAddresses []gethcommon.Address, chainIDs []uint64) (BalancesByChain, error) {
|
|
clients, err := m.tokenManager.RPCClient.EthClients(chainIDs)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
resp, err := m.tokenManager.GetBalancesByChain(context.Background(), clients, accounts, tokenAddresses)
|
|
return resp, err
|
|
}
|
|
|
|
func (m *DefaultTokenManager) FindOrCreateTokenByAddress(ctx context.Context, chainID uint64, address gethcommon.Address) *token.Token {
|
|
return m.tokenManager.FindOrCreateTokenByAddress(ctx, chainID, address)
|
|
}
|
|
|
|
type ManagerOption func(*managerOptions)
|
|
|
|
func WithAccountManager(accountsManager account.Manager) ManagerOption {
|
|
return func(opts *managerOptions) {
|
|
opts.accountsManager = accountsManager
|
|
}
|
|
}
|
|
|
|
func WithPermissionChecker(permissionChecker PermissionChecker) ManagerOption {
|
|
return func(opts *managerOptions) {
|
|
opts.permissionChecker = permissionChecker
|
|
}
|
|
}
|
|
|
|
func WithCollectiblesManager(collectiblesManager CollectiblesManager) ManagerOption {
|
|
return func(opts *managerOptions) {
|
|
opts.collectiblesManager = collectiblesManager
|
|
}
|
|
}
|
|
|
|
func WithTokenManager(tokenManager TokenManager) ManagerOption {
|
|
return func(opts *managerOptions) {
|
|
opts.tokenManager = tokenManager
|
|
}
|
|
}
|
|
|
|
func WithWalletConfig(walletConfig *params.WalletConfig) ManagerOption {
|
|
return func(opts *managerOptions) {
|
|
opts.walletConfig = walletConfig
|
|
}
|
|
}
|
|
|
|
func WithCommunityTokensService(communityTokensService CommunityTokensServiceInterface) ManagerOption {
|
|
return func(opts *managerOptions) {
|
|
opts.communityTokensService = communityTokensService
|
|
}
|
|
}
|
|
|
|
func WithAllowForcingCommunityMembersReevaluation(enabled bool) ManagerOption {
|
|
return func(opts *managerOptions) {
|
|
opts.allowForcingCommunityMembersReevaluation = enabled
|
|
}
|
|
}
|
|
|
|
type OwnerVerifier interface {
|
|
SafeGetSignerPubKey(ctx context.Context, chainID uint64, communityID string) (string, error)
|
|
}
|
|
|
|
func NewManager(identity *ecdsa.PrivateKey, installationID string, db *sql.DB, encryptor *encryption.Protocol, logger *zap.Logger, ensverifier *ens.Verifier, ownerVerifier OwnerVerifier, transport *transport.Transport, timesource common.TimeSource, keyDistributor KeyDistributor, torrentConfig *params.TorrentConfig, opts ...ManagerOption) (*Manager, error) {
|
|
if identity == nil {
|
|
return nil, errors.New("empty identity")
|
|
}
|
|
|
|
if timesource == nil {
|
|
return nil, errors.New("no timesource")
|
|
}
|
|
|
|
var err error
|
|
if logger == nil {
|
|
if logger, err = zap.NewDevelopment(); err != nil {
|
|
return nil, errors.Wrap(err, "failed to create a logger")
|
|
}
|
|
}
|
|
|
|
stdoutLogger, err := zap.NewDevelopment()
|
|
if err != nil {
|
|
return nil, errors.Wrap(err, "failed to create archive logger")
|
|
}
|
|
|
|
managerConfig := managerOptions{}
|
|
for _, opt := range opts {
|
|
opt(&managerConfig)
|
|
}
|
|
|
|
manager := &Manager{
|
|
logger: logger,
|
|
stdoutLogger: stdoutLogger,
|
|
encryptor: encryptor,
|
|
identity: identity,
|
|
installationID: installationID,
|
|
ownerVerifier: ownerVerifier,
|
|
quit: make(chan struct{}),
|
|
transport: transport,
|
|
timesource: timesource,
|
|
torrentConfig: torrentConfig,
|
|
torrentTasks: make(map[string]metainfo.Hash),
|
|
historyArchiveDownloadTasks: make(map[string]*HistoryArchiveDownloadTask),
|
|
keyDistributor: keyDistributor,
|
|
communityLock: NewCommunityLock(logger),
|
|
}
|
|
|
|
manager.persistence = &Persistence{
|
|
db: db,
|
|
recordBundleToCommunity: manager.dbRecordBundleToCommunity,
|
|
}
|
|
|
|
if managerConfig.accountsManager != nil {
|
|
manager.accountsManager = managerConfig.accountsManager
|
|
}
|
|
|
|
if managerConfig.collectiblesManager != nil {
|
|
manager.collectiblesManager = managerConfig.collectiblesManager
|
|
}
|
|
|
|
if managerConfig.tokenManager != nil {
|
|
manager.tokenManager = managerConfig.tokenManager
|
|
}
|
|
|
|
if managerConfig.walletConfig != nil {
|
|
manager.walletConfig = managerConfig.walletConfig
|
|
}
|
|
|
|
if managerConfig.communityTokensService != nil {
|
|
manager.communityTokensService = managerConfig.communityTokensService
|
|
}
|
|
|
|
if ensverifier != nil {
|
|
|
|
sub := ensverifier.Subscribe()
|
|
manager.ensSubscription = sub
|
|
manager.ensVerifier = ensverifier
|
|
}
|
|
|
|
if managerConfig.permissionChecker != nil {
|
|
manager.PermissionChecker = managerConfig.permissionChecker
|
|
} else {
|
|
manager.PermissionChecker = &DefaultPermissionChecker{
|
|
tokenManager: manager.tokenManager,
|
|
collectiblesManager: manager.collectiblesManager,
|
|
logger: logger,
|
|
ensVerifier: ensverifier,
|
|
}
|
|
}
|
|
|
|
if managerConfig.allowForcingCommunityMembersReevaluation {
|
|
manager.logger.Warn("allowing forcing community members reevaluation, this should only be used in test environment")
|
|
manager.forceMembersReevaluation = make(map[string]chan struct{}, 10)
|
|
}
|
|
|
|
return manager, nil
|
|
}
|
|
|
|
func (m *Manager) LogStdout(msg string, fields ...zap.Field) {
|
|
m.stdoutLogger.Info(msg, fields...)
|
|
m.logger.Debug(msg, fields...)
|
|
}
|
|
|
|
type archiveMDSlice []*archiveMetadata
|
|
|
|
type archiveMetadata struct {
|
|
hash string
|
|
from uint64
|
|
}
|
|
|
|
func (md archiveMDSlice) Len() int {
|
|
return len(md)
|
|
}
|
|
|
|
func (md archiveMDSlice) Swap(i, j int) {
|
|
md[i], md[j] = md[j], md[i]
|
|
}
|
|
|
|
func (md archiveMDSlice) Less(i, j int) bool {
|
|
return md[i].from > md[j].from
|
|
}
|
|
|
|
type Subscription struct {
|
|
Community *Community
|
|
CreatingHistoryArchivesSignal *signal.CreatingHistoryArchivesSignal
|
|
HistoryArchivesCreatedSignal *signal.HistoryArchivesCreatedSignal
|
|
NoHistoryArchivesCreatedSignal *signal.NoHistoryArchivesCreatedSignal
|
|
HistoryArchivesSeedingSignal *signal.HistoryArchivesSeedingSignal
|
|
HistoryArchivesUnseededSignal *signal.HistoryArchivesUnseededSignal
|
|
HistoryArchiveDownloadedSignal *signal.HistoryArchiveDownloadedSignal
|
|
DownloadingHistoryArchivesStartedSignal *signal.DownloadingHistoryArchivesStartedSignal
|
|
DownloadingHistoryArchivesFinishedSignal *signal.DownloadingHistoryArchivesFinishedSignal
|
|
ImportingHistoryArchiveMessagesSignal *signal.ImportingHistoryArchiveMessagesSignal
|
|
CommunityEventsMessage *CommunityEventsMessage
|
|
AcceptedRequestsToJoin []types.HexBytes
|
|
RejectedRequestsToJoin []types.HexBytes
|
|
CommunityPrivilegedMemberSyncMessage *CommunityPrivilegedMemberSyncMessage
|
|
TokenCommunityValidated *CommunityResponse
|
|
}
|
|
|
|
type CommunityResponse struct {
|
|
Community *Community `json:"community"`
|
|
Changes *CommunityChanges `json:"changes"`
|
|
RequestsToJoin []*RequestToJoin `json:"requestsToJoin"`
|
|
FailedToDecrypt []*CommunityPrivateDataFailedToDecrypt `json:"-"`
|
|
}
|
|
|
|
func (m *Manager) Subscribe() chan *Subscription {
|
|
subscription := make(chan *Subscription, 100)
|
|
m.subscriptions = append(m.subscriptions, subscription)
|
|
return subscription
|
|
}
|
|
|
|
func (m *Manager) Start() error {
|
|
m.stopped = false
|
|
m.communityLock.Init()
|
|
if m.ensVerifier != nil {
|
|
m.runENSVerificationLoop()
|
|
}
|
|
|
|
if m.ownerVerifier != nil {
|
|
m.runOwnerVerificationLoop()
|
|
}
|
|
|
|
go func() {
|
|
_ = m.fillMissingCommunityTokens()
|
|
}()
|
|
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) SetOnline(online bool) {
|
|
if online {
|
|
if m.torrentConfig != nil && m.torrentConfig.Enabled && !m.TorrentClientStarted() {
|
|
err := m.StartTorrentClient()
|
|
if err != nil {
|
|
m.LogStdout("couldn't start torrent client", zap.Error(err))
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
func (m *Manager) runENSVerificationLoop() {
|
|
go func() {
|
|
for {
|
|
select {
|
|
case <-m.quit:
|
|
m.logger.Debug("quitting ens verification loop")
|
|
return
|
|
case records, more := <-m.ensSubscription:
|
|
if !more {
|
|
m.logger.Debug("no more ens records, quitting")
|
|
return
|
|
}
|
|
m.logger.Info("received records", zap.Any("records", records))
|
|
}
|
|
}
|
|
}()
|
|
}
|
|
|
|
// This function is mostly a way to fix any community that is missing tokens in its description
|
|
func (m *Manager) fillMissingCommunityTokens() error {
|
|
controlledCommunities, err := m.Controlled()
|
|
if err != nil {
|
|
m.logger.Error("failed to retrieve orgs", zap.Error(err))
|
|
return err
|
|
}
|
|
|
|
unlock := func() {
|
|
for _, c := range controlledCommunities {
|
|
m.communityLock.Unlock(c.ID())
|
|
}
|
|
}
|
|
for _, c := range controlledCommunities {
|
|
m.communityLock.Lock(c.ID())
|
|
}
|
|
defer unlock()
|
|
|
|
for _, community := range controlledCommunities {
|
|
tokens, err := m.GetCommunityTokens(community.IDString())
|
|
if err != nil {
|
|
m.logger.Error("failed to retrieve community tokens", zap.Error(err))
|
|
return err
|
|
}
|
|
|
|
for _, token := range tokens {
|
|
if token.DeployState != community_token.Deployed {
|
|
continue
|
|
}
|
|
tokenMetadata := &protobuf.CommunityTokenMetadata{
|
|
ContractAddresses: map[uint64]string{uint64(token.ChainID): token.Address},
|
|
Description: token.Description,
|
|
Image: token.Base64Image,
|
|
Symbol: token.Symbol,
|
|
TokenType: token.TokenType,
|
|
Name: token.Name,
|
|
Decimals: uint32(token.Decimals),
|
|
}
|
|
modified, err := community.UpsertCommunityTokensMetadata(tokenMetadata)
|
|
if err != nil {
|
|
m.logger.Error("failed to add token metadata to the description", zap.Error(err))
|
|
return err
|
|
}
|
|
if modified {
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
m.logger.Error("failed to save the new community", zap.Error(err))
|
|
return err
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
// Only for testing
|
|
func (m *Manager) CommunitiesToValidate() (map[string][]communityToValidate, error) { // nolint: golint
|
|
return m.persistence.getCommunitiesToValidate()
|
|
}
|
|
|
|
func (m *Manager) runOwnerVerificationLoop() {
|
|
m.logger.Info("starting owner verification loop")
|
|
go func() {
|
|
for {
|
|
select {
|
|
case <-m.quit:
|
|
m.logger.Debug("quitting owner verification loop")
|
|
return
|
|
case <-time.After(validateInterval):
|
|
// If ownerverifier is nil, we skip, this is useful for testing
|
|
if m.ownerVerifier == nil {
|
|
continue
|
|
}
|
|
|
|
communitiesToValidate, err := m.persistence.getCommunitiesToValidate()
|
|
|
|
if err != nil {
|
|
m.logger.Error("failed to fetch communities to validate", zap.Error(err))
|
|
continue
|
|
}
|
|
for id, communities := range communitiesToValidate {
|
|
m.logger.Info("validating communities", zap.String("id", id), zap.Int("count", len(communities)))
|
|
|
|
_, _ = m.validateCommunity(communities)
|
|
}
|
|
}
|
|
}
|
|
}()
|
|
}
|
|
|
|
func (m *Manager) ValidateCommunityByID(communityID types.HexBytes) (*CommunityResponse, error) {
|
|
communitiesToValidate, err := m.persistence.getCommunityToValidateByID(communityID)
|
|
if err != nil {
|
|
m.logger.Error("failed to validate community by ID", zap.String("id", communityID.String()), zap.Error(err))
|
|
return nil, err
|
|
}
|
|
return m.validateCommunity(communitiesToValidate)
|
|
|
|
}
|
|
|
|
func (m *Manager) validateCommunity(communityToValidateData []communityToValidate) (*CommunityResponse, error) {
|
|
for _, community := range communityToValidateData {
|
|
signer, description, err := UnwrapCommunityDescriptionMessage(community.payload)
|
|
if err != nil {
|
|
m.logger.Error("failed to unwrap community", zap.Error(err))
|
|
continue
|
|
}
|
|
|
|
chainID := CommunityDescriptionTokenOwnerChainID(description)
|
|
if chainID == 0 {
|
|
// This should not happen
|
|
m.logger.Error("chain id is 0, ignoring")
|
|
continue
|
|
}
|
|
|
|
m.logger.Info("validating community", zap.String("id", types.EncodeHex(community.id)), zap.String("signer", common.PubkeyToHex(signer)))
|
|
|
|
ctx, cancel := context.WithTimeout(context.Background(), time.Second*3)
|
|
defer cancel()
|
|
|
|
owner, err := m.ownerVerifier.SafeGetSignerPubKey(ctx, chainID, types.EncodeHex(community.id))
|
|
if err != nil {
|
|
m.logger.Error("failed to get owner", zap.Error(err))
|
|
continue
|
|
}
|
|
|
|
ownerPK, err := common.HexToPubkey(owner)
|
|
if err != nil {
|
|
m.logger.Error("failed to convert pk string to ecdsa", zap.Error(err))
|
|
continue
|
|
}
|
|
|
|
// TODO: handle shards
|
|
response, err := m.HandleCommunityDescriptionMessage(signer, description, community.payload, ownerPK, nil)
|
|
if err != nil {
|
|
m.logger.Error("failed to handle community", zap.Error(err))
|
|
err = m.persistence.DeleteCommunityToValidate(community.id, community.clock)
|
|
if err != nil {
|
|
m.logger.Error("failed to delete community to validate", zap.Error(err))
|
|
}
|
|
continue
|
|
}
|
|
|
|
if response != nil {
|
|
|
|
m.logger.Info("community validated", zap.String("id", types.EncodeHex(community.id)), zap.String("signer", common.PubkeyToHex(signer)))
|
|
m.publish(&Subscription{TokenCommunityValidated: response})
|
|
err := m.persistence.DeleteCommunitiesToValidateByCommunityID(community.id)
|
|
if err != nil {
|
|
m.logger.Error("failed to delete communities to validate", zap.Error(err))
|
|
}
|
|
return response, nil
|
|
}
|
|
}
|
|
|
|
return nil, nil
|
|
}
|
|
|
|
func (m *Manager) Stop() error {
|
|
m.stopped = true
|
|
close(m.quit)
|
|
for _, c := range m.subscriptions {
|
|
close(c)
|
|
}
|
|
m.StopTorrentClient()
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) SetTorrentConfig(config *params.TorrentConfig) {
|
|
m.torrentConfig = config
|
|
}
|
|
|
|
// getTCPandUDPport will return the same port number given if != 0,
|
|
// otherwise, it will attempt to find a free random tcp and udp port using
|
|
// the same number for both protocols
|
|
func (m *Manager) getTCPandUDPport(portNumber int) (int, error) {
|
|
if portNumber != 0 {
|
|
return portNumber, nil
|
|
}
|
|
|
|
// Find free port
|
|
for i := 0; i < 10; i++ {
|
|
port := func() int {
|
|
tcpAddr, err := net.ResolveTCPAddr("tcp", net.JoinHostPort("localhost", "0"))
|
|
if err != nil {
|
|
m.logger.Warn("unable to resolve tcp addr: %v", zap.Error(err))
|
|
return 0
|
|
}
|
|
|
|
tcpListener, err := net.ListenTCP("tcp", tcpAddr)
|
|
if err != nil {
|
|
m.logger.Warn("unable to listen on addr", zap.Stringer("addr", tcpAddr), zap.Error(err))
|
|
return 0
|
|
}
|
|
defer tcpListener.Close()
|
|
|
|
port := tcpListener.Addr().(*net.TCPAddr).Port
|
|
|
|
udpAddr, err := net.ResolveUDPAddr("udp", net.JoinHostPort("localhost", fmt.Sprintf("%d", port)))
|
|
if err != nil {
|
|
m.logger.Warn("unable to resolve udp addr: %v", zap.Error(err))
|
|
return 0
|
|
}
|
|
|
|
udpListener, err := net.ListenUDP("udp", udpAddr)
|
|
if err != nil {
|
|
m.logger.Warn("unable to listen on addr", zap.Stringer("addr", udpAddr), zap.Error(err))
|
|
return 0
|
|
}
|
|
defer udpListener.Close()
|
|
|
|
return port
|
|
}()
|
|
|
|
if port != 0 {
|
|
return port, nil
|
|
}
|
|
}
|
|
|
|
return 0, fmt.Errorf("no free port found")
|
|
}
|
|
|
|
func (m *Manager) StartTorrentClient() error {
|
|
if m.torrentConfig == nil {
|
|
return fmt.Errorf("can't start torrent client: missing torrentConfig")
|
|
}
|
|
|
|
if m.TorrentClientStarted() {
|
|
return nil
|
|
}
|
|
|
|
port, err := m.getTCPandUDPport(m.torrentConfig.Port)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
config := torrent.NewDefaultClientConfig()
|
|
config.SetListenAddr(":" + fmt.Sprint(port))
|
|
config.Seed = true
|
|
|
|
config.DataDir = m.torrentConfig.DataDir
|
|
|
|
if _, err := os.Stat(m.torrentConfig.DataDir); os.IsNotExist(err) {
|
|
err := os.MkdirAll(m.torrentConfig.DataDir, 0700)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
|
|
m.logger.Info("Starting torrent client", zap.Any("port", port))
|
|
// Instantiating the client will make it bootstrap and listen eagerly,
|
|
// so no go routine is needed here
|
|
client, err := torrent.NewClient(config)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
m.torrentClient = client
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) StopTorrentClient() []error {
|
|
if m.TorrentClientStarted() {
|
|
m.StopHistoryArchiveTasksIntervals()
|
|
m.logger.Info("Stopping torrent client")
|
|
errs := m.torrentClient.Close()
|
|
if len(errs) > 0 {
|
|
return errs
|
|
}
|
|
m.torrentClient = nil
|
|
}
|
|
return make([]error, 0)
|
|
}
|
|
|
|
func (m *Manager) TorrentClientStarted() bool {
|
|
return m.torrentClient != nil
|
|
}
|
|
|
|
func (m *Manager) publish(subscription *Subscription) {
|
|
if m.stopped {
|
|
return
|
|
}
|
|
for _, s := range m.subscriptions {
|
|
select {
|
|
case s <- subscription:
|
|
default:
|
|
m.logger.Warn("subscription channel full, dropping message")
|
|
}
|
|
}
|
|
}
|
|
|
|
func (m *Manager) All() ([]*Community, error) {
|
|
return m.persistence.AllCommunities(&m.identity.PublicKey)
|
|
}
|
|
|
|
type CommunityShard struct {
|
|
CommunityID string `json:"communityID"`
|
|
Shard *shard.Shard `json:"shard"`
|
|
}
|
|
|
|
type CuratedCommunities struct {
|
|
ContractCommunities []string
|
|
ContractFeaturedCommunities []string
|
|
}
|
|
|
|
type KnownCommunitiesResponse struct {
|
|
ContractCommunities []string `json:"contractCommunities"`
|
|
ContractFeaturedCommunities []string `json:"contractFeaturedCommunities"`
|
|
Descriptions map[string]*Community `json:"communities"`
|
|
UnknownCommunities []string `json:"unknownCommunities"`
|
|
}
|
|
|
|
func (m *Manager) GetStoredDescriptionForCommunities(communityIDs []string) (*KnownCommunitiesResponse, error) {
|
|
response := &KnownCommunitiesResponse{
|
|
Descriptions: make(map[string]*Community),
|
|
}
|
|
|
|
for i := range communityIDs {
|
|
communityID := communityIDs[i]
|
|
communityIDBytes, err := types.DecodeHex(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
community, err := m.GetByID(types.HexBytes(communityIDBytes))
|
|
if err != nil && err != ErrOrgNotFound {
|
|
return nil, err
|
|
}
|
|
|
|
if community != nil {
|
|
response.Descriptions[community.IDString()] = community
|
|
} else {
|
|
response.UnknownCommunities = append(response.UnknownCommunities, communityID)
|
|
}
|
|
|
|
response.ContractCommunities = append(response.ContractCommunities, communityID)
|
|
}
|
|
|
|
return response, nil
|
|
}
|
|
|
|
func (m *Manager) Joined() ([]*Community, error) {
|
|
return m.persistence.JoinedCommunities(&m.identity.PublicKey)
|
|
}
|
|
|
|
func (m *Manager) Spectated() ([]*Community, error) {
|
|
return m.persistence.SpectatedCommunities(&m.identity.PublicKey)
|
|
}
|
|
|
|
func (m *Manager) CommunityUpdateLastOpenedAt(communityID types.HexBytes, timestamp int64) (*Community, error) {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.persistence.UpdateLastOpenedAt(community.ID(), timestamp)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
community.UpdateLastOpenedAt(timestamp)
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) JoinedAndPendingCommunitiesWithRequests() ([]*Community, error) {
|
|
return m.persistence.JoinedAndPendingCommunitiesWithRequests(&m.identity.PublicKey)
|
|
}
|
|
|
|
func (m *Manager) LeftCommunities() ([]*Community, error) {
|
|
return m.persistence.LeftCommunities(&m.identity.PublicKey)
|
|
}
|
|
|
|
func (m *Manager) DeletedCommunities() ([]*Community, error) {
|
|
return m.persistence.DeletedCommunities(&m.identity.PublicKey)
|
|
}
|
|
|
|
func (m *Manager) Controlled() ([]*Community, error) {
|
|
communities, err := m.persistence.CommunitiesWithPrivateKey(&m.identity.PublicKey)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
controlled := make([]*Community, 0, len(communities))
|
|
|
|
for _, c := range communities {
|
|
if c.IsControlNode() {
|
|
controlled = append(controlled, c)
|
|
}
|
|
}
|
|
|
|
return controlled, nil
|
|
}
|
|
|
|
// CreateCommunity takes a description, generates an ID for it, saves it and return it
|
|
func (m *Manager) CreateCommunity(request *requests.CreateCommunity, publish bool) (*Community, error) {
|
|
|
|
description, err := request.ToCommunityDescription()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
description.Members = make(map[string]*protobuf.CommunityMember)
|
|
description.Members[common.PubkeyToHex(&m.identity.PublicKey)] = &protobuf.CommunityMember{Roles: []protobuf.CommunityMember_Roles{protobuf.CommunityMember_ROLE_OWNER}}
|
|
|
|
err = ValidateCommunityDescription(description)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
description.Clock = 1
|
|
|
|
key, err := crypto.GenerateKey()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
description.ID = types.EncodeHex(crypto.CompressPubkey(&key.PublicKey))
|
|
|
|
config := Config{
|
|
ID: &key.PublicKey,
|
|
PrivateKey: key,
|
|
ControlNode: &key.PublicKey,
|
|
ControlDevice: true,
|
|
Logger: m.logger,
|
|
Joined: true,
|
|
JoinedAt: time.Now().Unix(),
|
|
MemberIdentity: &m.identity.PublicKey,
|
|
CommunityDescription: description,
|
|
Shard: nil,
|
|
LastOpenedAt: 0,
|
|
}
|
|
|
|
var descriptionEncryptor DescriptionEncryptor
|
|
if m.encryptor != nil {
|
|
descriptionEncryptor = m
|
|
}
|
|
community, err := New(config, m.timesource, descriptionEncryptor)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// We join any community we create
|
|
community.Join()
|
|
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// Save grant for own community
|
|
grant, err := community.BuildGrant(&m.identity.PublicKey, "")
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
err = m.persistence.SaveCommunityGrant(community.IDString(), grant, uint64(time.Now().UnixMilli()))
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// Mark this device as the control node
|
|
syncControlNode := &protobuf.SyncCommunityControlNode{
|
|
Clock: 1,
|
|
InstallationId: m.installationID,
|
|
}
|
|
err = m.SaveSyncControlNode(community.ID(), syncControlNode)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if publish {
|
|
m.publish(&Subscription{Community: community})
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) CreateCommunityTokenPermission(request *requests.CreateCommunityTokenPermission) (*Community, *CommunityChanges, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
community, err := m.GetByID(request.CommunityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
// ensure key is generated before marshaling,
|
|
// as it requires key to encrypt description
|
|
if community.IsControlNode() && m.encryptor != nil {
|
|
key, err := m.encryptor.GenerateHashRatchetKey(community.ID())
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
keyID, err := key.GetKeyID()
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
m.logger.Info("generate key for token", zap.String("group-id", types.Bytes2Hex(community.ID())), zap.String("key-id", types.Bytes2Hex(keyID)))
|
|
}
|
|
|
|
community, changes, err := m.createCommunityTokenPermission(request, community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, changes, nil
|
|
}
|
|
|
|
func (m *Manager) EditCommunityTokenPermission(request *requests.EditCommunityTokenPermission) (*Community, *CommunityChanges, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
community, err := m.GetByID(request.CommunityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
tokenPermission := request.ToCommunityTokenPermission()
|
|
|
|
changes, err := community.UpsertTokenPermission(&tokenPermission)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, changes, nil
|
|
}
|
|
|
|
type reevaluateMemberRole struct {
|
|
old protobuf.CommunityMember_Roles
|
|
new protobuf.CommunityMember_Roles
|
|
}
|
|
|
|
func (rmr reevaluateMemberRole) hasChanged() bool {
|
|
return rmr.old != rmr.new
|
|
}
|
|
|
|
func (rmr reevaluateMemberRole) isPrivileged() bool {
|
|
return rmr.new != protobuf.CommunityMember_ROLE_NONE
|
|
}
|
|
|
|
func (rmr reevaluateMemberRole) hasChangedToPrivileged() bool {
|
|
return rmr.hasChanged() && rmr.old == protobuf.CommunityMember_ROLE_NONE
|
|
}
|
|
|
|
type reevaluateMembersResult struct {
|
|
membersToRemove map[string]struct{}
|
|
membersRoles map[string]*reevaluateMemberRole
|
|
membersToRemoveFromChannels map[string]map[string]struct{}
|
|
membersToAddToChannels map[string]map[string]protobuf.CommunityMember_ChannelRole
|
|
}
|
|
|
|
func (rmr *reevaluateMembersResult) newPrivilegedRoles() (map[protobuf.CommunityMember_Roles][]*ecdsa.PublicKey, error) {
|
|
result := map[protobuf.CommunityMember_Roles][]*ecdsa.PublicKey{}
|
|
|
|
for memberKey, roles := range rmr.membersRoles {
|
|
if roles.hasChangedToPrivileged() {
|
|
memberPubKey, err := common.HexToPubkey(memberKey)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
if result[roles.new] == nil {
|
|
result[roles.new] = []*ecdsa.PublicKey{}
|
|
}
|
|
result[roles.new] = append(result[roles.new], memberPubKey)
|
|
}
|
|
}
|
|
|
|
return result, nil
|
|
}
|
|
|
|
// use it only for testing purposes
|
|
func (m *Manager) ReevaluateMembers(communityID types.HexBytes) (*Community, map[protobuf.CommunityMember_Roles][]*ecdsa.PublicKey, error) {
|
|
return m.reevaluateMembers(communityID)
|
|
}
|
|
|
|
// First, the community is read from the database,
|
|
// then the members are reevaluated, and only then
|
|
// the community is locked and changes are applied.
|
|
// NOTE: Changes made to the same community
|
|
// while reevaluation is ongoing are respected
|
|
// and do not affect the result of this function.
|
|
// If permissions are changed in the meantime,
|
|
// they will be accommodated with the next reevaluation.
|
|
func (m *Manager) reevaluateMembers(communityID types.HexBytes) (*Community, map[protobuf.CommunityMember_Roles][]*ecdsa.PublicKey, error) {
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
if !community.IsControlNode() {
|
|
return nil, nil, ErrNotEnoughPermissions
|
|
}
|
|
|
|
communityPermissionsPreParsedData, channelPermissionsPreParsedData := PreParsePermissionsData(community.tokenPermissions())
|
|
|
|
result := &reevaluateMembersResult{
|
|
membersToRemove: map[string]struct{}{},
|
|
membersRoles: map[string]*reevaluateMemberRole{},
|
|
membersToRemoveFromChannels: map[string]map[string]struct{}{},
|
|
membersToAddToChannels: map[string]map[string]protobuf.CommunityMember_ChannelRole{},
|
|
}
|
|
|
|
membersAccounts, err := m.persistence.GetCommunityRequestsToJoinRevealedAddresses(community.ID())
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
for memberKey := range community.Members() {
|
|
memberPubKey, err := common.HexToPubkey(memberKey)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
if memberKey == common.PubkeyToHex(&m.identity.PublicKey) || community.IsMemberOwner(memberPubKey) {
|
|
continue
|
|
}
|
|
|
|
revealedAccount, memberHasWallet := membersAccounts[memberKey]
|
|
if !memberHasWallet {
|
|
result.membersToRemove[memberKey] = struct{}{}
|
|
continue
|
|
}
|
|
|
|
accountsAndChainIDs := revealedAccountsToAccountsAndChainIDsCombination(revealedAccount)
|
|
|
|
result.membersRoles[memberKey] = &reevaluateMemberRole{
|
|
old: community.MemberRole(memberPubKey),
|
|
new: protobuf.CommunityMember_ROLE_NONE,
|
|
}
|
|
|
|
becomeTokenMasterPermissions := communityPermissionsPreParsedData[protobuf.CommunityTokenPermission_BECOME_TOKEN_MASTER]
|
|
if becomeTokenMasterPermissions != nil {
|
|
permissionResponse, err := m.PermissionChecker.CheckPermissions(becomeTokenMasterPermissions, accountsAndChainIDs, true)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
if permissionResponse.Satisfied {
|
|
result.membersRoles[memberKey].new = protobuf.CommunityMember_ROLE_TOKEN_MASTER
|
|
// Skip further validation if user has TokenMaster permissions
|
|
continue
|
|
}
|
|
}
|
|
|
|
becomeAdminPermissions := communityPermissionsPreParsedData[protobuf.CommunityTokenPermission_BECOME_ADMIN]
|
|
if becomeAdminPermissions != nil {
|
|
permissionResponse, err := m.PermissionChecker.CheckPermissions(becomeAdminPermissions, accountsAndChainIDs, true)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
if permissionResponse.Satisfied {
|
|
result.membersRoles[memberKey].new = protobuf.CommunityMember_ROLE_ADMIN
|
|
// Skip further validation if user has Admin permissions
|
|
continue
|
|
}
|
|
}
|
|
|
|
becomeMemberPermissions := communityPermissionsPreParsedData[protobuf.CommunityTokenPermission_BECOME_MEMBER]
|
|
if becomeMemberPermissions != nil {
|
|
permissionResponse, err := m.PermissionChecker.CheckPermissions(becomeMemberPermissions, accountsAndChainIDs, true)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
if !permissionResponse.Satisfied {
|
|
result.membersToRemove[memberKey] = struct{}{}
|
|
// Skip channels validation if user has been removed
|
|
continue
|
|
}
|
|
}
|
|
|
|
addToChannels, removeFromChannels, err := m.reevaluateMemberChannelsPermissions(community, memberPubKey, channelPermissionsPreParsedData, accountsAndChainIDs)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
result.membersToAddToChannels[memberKey] = addToChannels
|
|
result.membersToRemoveFromChannels[memberKey] = removeFromChannels
|
|
}
|
|
|
|
newPrivilegedRoles, err := result.newPrivilegedRoles()
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
// Note: community itself may have changed in the meantime of permissions reevaluation.
|
|
community, err = m.applyReevaluateMembersResult(communityID, result)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, newPrivilegedRoles, m.saveAndPublish(community)
|
|
}
|
|
|
|
// Apply results on the most up-to-date community.
|
|
func (m *Manager) applyReevaluateMembersResult(communityID types.HexBytes, result *reevaluateMembersResult) (*Community, error) {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if !community.IsControlNode() {
|
|
return nil, ErrNotEnoughPermissions
|
|
}
|
|
|
|
// Remove members.
|
|
for memberKey := range result.membersToRemove {
|
|
memberPubKey, err := common.HexToPubkey(memberKey)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
_, err = community.RemoveUserFromOrg(memberPubKey)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
// Ensure members have proper roles.
|
|
for memberKey, roles := range result.membersRoles {
|
|
memberPubKey, err := common.HexToPubkey(memberKey)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if !community.HasMember(memberPubKey) {
|
|
continue
|
|
}
|
|
|
|
_, err = community.SetRoleToMember(memberPubKey, roles.new)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// Ensure privileged members can post in all chats.
|
|
if roles.isPrivileged() {
|
|
for channelID := range community.Chats() {
|
|
_, err = community.AddMemberToChat(channelID, memberPubKey, []protobuf.CommunityMember_Roles{roles.new}, protobuf.CommunityMember_CHANNEL_ROLE_POSTER)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
// Remove members from channels.
|
|
for memberKey, channels := range result.membersToRemoveFromChannels {
|
|
memberPubKey, err := common.HexToPubkey(memberKey)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
for channelID := range channels {
|
|
_, err = community.RemoveUserFromChat(memberPubKey, channelID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
}
|
|
|
|
// Add unprivileged members to channels.
|
|
for memberKey, channels := range result.membersToAddToChannels {
|
|
memberPubKey, err := common.HexToPubkey(memberKey)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if !community.HasMember(memberPubKey) {
|
|
continue
|
|
}
|
|
|
|
for channelID, channelRole := range channels {
|
|
_, err = community.AddMemberToChat(channelID, memberPubKey, []protobuf.CommunityMember_Roles{protobuf.CommunityMember_ROLE_NONE}, channelRole)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) reevaluateMemberChannelsPermissions(community *Community, memberPubKey *ecdsa.PublicKey,
|
|
channelPermissionsPreParsedData map[string]*PreParsedCommunityPermissionsData, accountsAndChainIDs []*AccountChainIDsCombination) (map[string]protobuf.CommunityMember_ChannelRole, map[string]struct{}, error) {
|
|
|
|
addToChannels := map[string]protobuf.CommunityMember_ChannelRole{}
|
|
removeFromChannels := map[string]struct{}{}
|
|
|
|
// check which permissions we satisfy and which not
|
|
channelPermissionsCheckResult, err := m.checkChannelsPermissions(channelPermissionsPreParsedData, accountsAndChainIDs, true)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
for channelID := range community.Chats() {
|
|
channelPermissionsCheckResult, hasChannelPermission := channelPermissionsCheckResult[community.ChatID(channelID)]
|
|
|
|
// ensure member is added if channel has no permissions
|
|
if !hasChannelPermission {
|
|
addToChannels[channelID] = protobuf.CommunityMember_CHANNEL_ROLE_POSTER
|
|
continue
|
|
}
|
|
|
|
viewAndPostSatisfied, viewAndPostPermissionExists := channelPermissionsCheckResult[protobuf.CommunityTokenPermission_CAN_VIEW_AND_POST_CHANNEL]
|
|
viewOnlySatisfied, viewOnlyPermissionExists := channelPermissionsCheckResult[protobuf.CommunityTokenPermission_CAN_VIEW_CHANNEL]
|
|
|
|
satisfied := false
|
|
channelRole := protobuf.CommunityMember_CHANNEL_ROLE_VIEWER
|
|
if viewAndPostPermissionExists && viewAndPostSatisfied {
|
|
satisfied = viewAndPostSatisfied
|
|
channelRole = protobuf.CommunityMember_CHANNEL_ROLE_POSTER
|
|
} else if !satisfied && viewOnlyPermissionExists {
|
|
satisfied = viewOnlySatisfied
|
|
}
|
|
|
|
if satisfied {
|
|
addToChannels[channelID] = channelRole
|
|
} else {
|
|
removeFromChannels[channelID] = struct{}{}
|
|
}
|
|
}
|
|
|
|
return addToChannels, removeFromChannels, nil
|
|
}
|
|
|
|
func (m *Manager) checkChannelsPermissions(channelsPermissionsPreParsedData map[string]*PreParsedCommunityPermissionsData, accountsAndChainIDs []*AccountChainIDsCombination, shortcircuit bool) (map[string]map[protobuf.CommunityTokenPermission_Type]bool, error) {
|
|
channelPermissionsCheckResult := make(map[string]map[protobuf.CommunityTokenPermission_Type]bool)
|
|
for _, channelsPermissionPreParsedData := range channelsPermissionsPreParsedData {
|
|
permissionResponse, err := m.PermissionChecker.CheckPermissions(channelsPermissionPreParsedData, accountsAndChainIDs, true)
|
|
if err != nil {
|
|
return channelPermissionsCheckResult, err
|
|
}
|
|
// Note: in `PreParsedCommunityPermissionsData` for channels there will be only one permission
|
|
// no need to iterate over `Permissions`
|
|
for _, chatId := range channelsPermissionPreParsedData.Permissions[0].ChatIds {
|
|
if _, exists := channelPermissionsCheckResult[chatId]; !exists {
|
|
channelPermissionsCheckResult[chatId] = make(map[protobuf.CommunityTokenPermission_Type]bool)
|
|
}
|
|
satisfied, exists := channelPermissionsCheckResult[chatId][channelsPermissionPreParsedData.Permissions[0].Type]
|
|
if exists && satisfied {
|
|
continue
|
|
}
|
|
channelPermissionsCheckResult[chatId][channelsPermissionPreParsedData.Permissions[0].Type] = permissionResponse.Satisfied
|
|
}
|
|
}
|
|
return channelPermissionsCheckResult, nil
|
|
}
|
|
|
|
func (m *Manager) StartMembersReevaluationLoop(communityID types.HexBytes, reevaluateOnStart bool) {
|
|
go m.reevaluateMembersLoop(communityID, reevaluateOnStart)
|
|
}
|
|
|
|
func (m *Manager) reevaluateMembersLoop(communityID types.HexBytes, reevaluateOnStart bool) {
|
|
|
|
if _, exists := m.membersReevaluationTasks.Load(communityID.String()); exists {
|
|
return
|
|
}
|
|
|
|
m.membersReevaluationTasks.Store(communityID.String(), &membersReevaluationTask{})
|
|
defer m.membersReevaluationTasks.Delete(communityID.String())
|
|
|
|
var forceReevaluation chan struct{}
|
|
if m.forceMembersReevaluation != nil {
|
|
forceReevaluation = make(chan struct{}, 10)
|
|
m.forceMembersReevaluation[communityID.String()] = forceReevaluation
|
|
}
|
|
|
|
type criticalError struct {
|
|
error
|
|
}
|
|
|
|
shouldReevaluate := func(task *membersReevaluationTask, force bool) bool {
|
|
task.mutex.Lock()
|
|
defer task.mutex.Unlock()
|
|
|
|
// Ensure reevaluation is performed not more often than once per minute
|
|
if !force && task.lastSuccessTime.After(time.Now().Add(-1*time.Minute)) {
|
|
return false
|
|
}
|
|
|
|
if !task.lastSuccessTime.Before(time.Now().Add(-memberPermissionsCheckInterval)) &&
|
|
!task.lastStartTime.Before(task.onDemandRequestTime) {
|
|
return false
|
|
}
|
|
|
|
return true
|
|
}
|
|
|
|
reevaluateMembers := func(force bool) (err error) {
|
|
t, exists := m.membersReevaluationTasks.Load(communityID.String())
|
|
if !exists {
|
|
return criticalError{
|
|
error: errors.New("missing task"),
|
|
}
|
|
}
|
|
task, ok := t.(*membersReevaluationTask)
|
|
if !ok {
|
|
return criticalError{
|
|
error: errors.New("invalid task type"),
|
|
}
|
|
}
|
|
|
|
if !shouldReevaluate(task, force) {
|
|
return nil
|
|
}
|
|
|
|
task.mutex.Lock()
|
|
task.lastStartTime = time.Now()
|
|
task.mutex.Unlock()
|
|
|
|
err = m.reevaluateCommunityMembersPermissions(communityID)
|
|
if err != nil {
|
|
if errors.Is(err, ErrOrgNotFound) {
|
|
return criticalError{
|
|
error: err,
|
|
}
|
|
}
|
|
return err
|
|
}
|
|
|
|
task.mutex.Lock()
|
|
task.lastSuccessTime = time.Now()
|
|
task.mutex.Unlock()
|
|
|
|
m.logger.Info("reevaluation finished", zap.String("communityID", communityID.String()), zap.Duration("elapsed", task.lastSuccessTime.Sub(task.lastStartTime)))
|
|
return nil
|
|
}
|
|
|
|
ticker := time.NewTicker(10 * time.Second)
|
|
defer ticker.Stop()
|
|
|
|
reevaluate := reevaluateOnStart
|
|
force := false
|
|
|
|
for {
|
|
if reevaluate {
|
|
err := reevaluateMembers(force)
|
|
if err != nil {
|
|
var criticalError *criticalError
|
|
if errors.As(err, &criticalError) {
|
|
return
|
|
}
|
|
}
|
|
}
|
|
|
|
force = false
|
|
reevaluate = false
|
|
|
|
select {
|
|
case <-ticker.C:
|
|
reevaluate = true
|
|
continue
|
|
|
|
case <-forceReevaluation:
|
|
reevaluate = true
|
|
force = true
|
|
continue
|
|
|
|
case <-m.quit:
|
|
return
|
|
}
|
|
}
|
|
}
|
|
|
|
func (m *Manager) ForceMembersReevaluation(communityID types.HexBytes) error {
|
|
if m.forceMembersReevaluation == nil {
|
|
return errors.New("forcing members reevaluation is not allowed")
|
|
}
|
|
return m.scheduleMembersReevaluation(communityID, true)
|
|
}
|
|
|
|
func (m *Manager) ScheduleMembersReevaluation(communityID types.HexBytes) error {
|
|
return m.scheduleMembersReevaluation(communityID, false)
|
|
}
|
|
|
|
func (m *Manager) scheduleMembersReevaluation(communityID types.HexBytes, forceImmediateReevaluation bool) error {
|
|
t, exists := m.membersReevaluationTasks.Load(communityID.String())
|
|
if !exists {
|
|
return errors.New("reevaluation task doesn't exist")
|
|
}
|
|
|
|
task, ok := t.(*membersReevaluationTask)
|
|
if !ok {
|
|
return errors.New("invalid task type")
|
|
}
|
|
task.mutex.Lock()
|
|
defer task.mutex.Unlock()
|
|
task.onDemandRequestTime = time.Now()
|
|
|
|
if forceImmediateReevaluation {
|
|
m.forceMembersReevaluation[communityID.String()] <- struct{}{}
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) DeleteCommunityTokenPermission(request *requests.DeleteCommunityTokenPermission) (*Community, *CommunityChanges, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
community, err := m.GetByID(request.CommunityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
changes, err := community.DeleteTokenPermission(request.PermissionID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, changes, nil
|
|
}
|
|
|
|
func (m *Manager) reevaluateCommunityMembersPermissions(communityID types.HexBytes) error {
|
|
// Publish when the reevluation started since it can take a while
|
|
signal.SendCommunityMemberReevaluationStarted(types.EncodeHex(communityID))
|
|
|
|
community, newPrivilegedMembers, err := m.reevaluateMembers(communityID)
|
|
|
|
// Publish the reevaluation ending, even if it errored
|
|
// A possible improvement would be to pass the error here
|
|
signal.SendCommunityMemberReevaluationEnded(types.EncodeHex(communityID))
|
|
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
return m.shareRequestsToJoinWithNewPrivilegedMembers(community, newPrivilegedMembers)
|
|
}
|
|
|
|
func (m *Manager) DeleteCommunity(id types.HexBytes) error {
|
|
m.communityLock.Lock(id)
|
|
defer m.communityLock.Unlock(id)
|
|
|
|
err := m.persistence.DeleteCommunity(id)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
return m.persistence.DeleteCommunitySettings(id)
|
|
}
|
|
|
|
func (m *Manager) updateShard(community *Community, shard *shard.Shard, clock uint64) error {
|
|
community.config.Shard = shard
|
|
if shard == nil {
|
|
return m.persistence.DeleteCommunityShard(community.ID())
|
|
}
|
|
|
|
return m.persistence.SaveCommunityShard(community.ID(), shard, clock)
|
|
}
|
|
|
|
func (m *Manager) UpdateShard(community *Community, shard *shard.Shard, clock uint64) error {
|
|
m.communityLock.Lock(community.ID())
|
|
defer m.communityLock.Unlock(community.ID())
|
|
|
|
return m.updateShard(community, shard, clock)
|
|
}
|
|
|
|
// SetShard assigns a shard to a community
|
|
func (m *Manager) SetShard(communityID types.HexBytes, shard *shard.Shard) (*Community, error) {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
community.increaseClock()
|
|
|
|
err = m.updateShard(community, shard, community.Clock())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) UpdatePubsubTopicPrivateKey(topic string, privKey *ecdsa.PrivateKey) error {
|
|
if privKey != nil {
|
|
return m.transport.StorePubsubTopicKey(topic, privKey)
|
|
}
|
|
|
|
return m.transport.RemovePubsubTopicKey(topic)
|
|
}
|
|
|
|
// EditCommunity takes a description, updates the community with the description,
|
|
// saves it and returns it
|
|
func (m *Manager) EditCommunity(request *requests.EditCommunity) (*Community, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
community, err := m.GetByID(request.CommunityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
newDescription, err := request.ToCommunityDescription()
|
|
if err != nil {
|
|
return nil, fmt.Errorf("can't create community description: %v", err)
|
|
}
|
|
|
|
// If permissions weren't explicitly set on original request, use existing ones
|
|
if newDescription.Permissions.Access == protobuf.CommunityPermissions_UNKNOWN_ACCESS {
|
|
newDescription.Permissions.Access = community.config.CommunityDescription.Permissions.Access
|
|
}
|
|
// Use existing images for the entries that were not updated
|
|
// NOTE: This will NOT allow deletion of the community image; it will need to
|
|
// be handled separately.
|
|
for imageName := range community.config.CommunityDescription.Identity.Images {
|
|
_, exists := newDescription.Identity.Images[imageName]
|
|
if !exists {
|
|
// If no image was set in ToCommunityDescription then Images is nil.
|
|
if newDescription.Identity.Images == nil {
|
|
newDescription.Identity.Images = make(map[string]*protobuf.IdentityImage)
|
|
}
|
|
newDescription.Identity.Images[imageName] = community.config.CommunityDescription.Identity.Images[imageName]
|
|
}
|
|
}
|
|
// TODO: handle delete image (if needed)
|
|
|
|
err = ValidateCommunityDescription(newDescription)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if !(community.IsControlNode() || community.hasPermissionToSendCommunityEvent(protobuf.CommunityEvent_COMMUNITY_EDIT)) {
|
|
return nil, ErrNotAuthorized
|
|
}
|
|
|
|
// Edit the community values
|
|
community.Edit(newDescription)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if community.IsControlNode() {
|
|
community.increaseClock()
|
|
} else {
|
|
err := community.addNewCommunityEvent(community.ToCommunityEditCommunityEvent(newDescription))
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) RemovePrivateKey(id types.HexBytes) (*Community, error) {
|
|
m.communityLock.Lock(id)
|
|
defer m.communityLock.Unlock(id)
|
|
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return community, err
|
|
}
|
|
|
|
if !community.IsControlNode() {
|
|
return community, ErrNotControlNode
|
|
}
|
|
|
|
community.config.PrivateKey = nil
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return community, err
|
|
}
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) ExportCommunity(id types.HexBytes) (*ecdsa.PrivateKey, error) {
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if !community.IsControlNode() {
|
|
return nil, ErrNotControlNode
|
|
}
|
|
|
|
return community.config.PrivateKey, nil
|
|
}
|
|
|
|
func (m *Manager) ImportCommunity(key *ecdsa.PrivateKey, clock uint64) (*Community, error) {
|
|
communityID := crypto.CompressPubkey(&key.PublicKey)
|
|
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil && err != ErrOrgNotFound {
|
|
return nil, err
|
|
}
|
|
|
|
if community == nil {
|
|
createCommunityRequest := requests.CreateCommunity{
|
|
Membership: protobuf.CommunityPermissions_MANUAL_ACCEPT,
|
|
Name: "unknown imported",
|
|
}
|
|
|
|
description, err := createCommunityRequest.ToCommunityDescription()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = ValidateCommunityDescription(description)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
description.Clock = 1
|
|
description.ID = types.EncodeHex(communityID)
|
|
|
|
config := Config{
|
|
ID: &key.PublicKey,
|
|
PrivateKey: key,
|
|
ControlNode: &key.PublicKey,
|
|
ControlDevice: true,
|
|
Logger: m.logger,
|
|
Joined: true,
|
|
JoinedAt: time.Now().Unix(),
|
|
MemberIdentity: &m.identity.PublicKey,
|
|
CommunityDescription: description,
|
|
LastOpenedAt: 0,
|
|
}
|
|
|
|
var descriptionEncryptor DescriptionEncryptor
|
|
if m.encryptor != nil {
|
|
descriptionEncryptor = m
|
|
}
|
|
community, err = New(config, m.timesource, descriptionEncryptor)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
} else {
|
|
community.config.PrivateKey = key
|
|
community.config.ControlDevice = true
|
|
}
|
|
|
|
community.Join()
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// Save grant for own community
|
|
grant, err := community.BuildGrant(&m.identity.PublicKey, "")
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
err = m.persistence.SaveCommunityGrant(community.IDString(), grant, uint64(time.Now().UnixMilli()))
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// Mark this device as the control node
|
|
syncControlNode := &protobuf.SyncCommunityControlNode{
|
|
Clock: clock,
|
|
InstallationId: m.installationID,
|
|
}
|
|
err = m.SaveSyncControlNode(community.ID(), syncControlNode)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) CreateChat(communityID types.HexBytes, chat *protobuf.CommunityChat, publish bool, thirdPartyID string) (*CommunityChanges, error) {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
chatID := uuid.New().String()
|
|
if thirdPartyID != "" {
|
|
chatID = chatID + thirdPartyID
|
|
}
|
|
|
|
changes, err := community.CreateChat(chatID, chat)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return changes, nil
|
|
}
|
|
|
|
func (m *Manager) EditChat(communityID types.HexBytes, chatID string, chat *protobuf.CommunityChat) (*Community, *CommunityChanges, error) {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
// Remove communityID prefix from chatID if exists
|
|
if strings.HasPrefix(chatID, communityID.String()) {
|
|
chatID = strings.TrimPrefix(chatID, communityID.String())
|
|
}
|
|
|
|
oldChat, err := community.GetChat(chatID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
// We can't edit permissions and members with an Edit, so we set to what we had, otherwise they will be lost
|
|
chat.Permissions = oldChat.Permissions
|
|
chat.Members = oldChat.Members
|
|
|
|
changes, err := community.EditChat(chatID, chat)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, changes, nil
|
|
}
|
|
|
|
func (m *Manager) DeleteChat(communityID types.HexBytes, chatID string) (*Community, *CommunityChanges, error) {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
// Remove communityID prefix from chatID if exists
|
|
if strings.HasPrefix(chatID, communityID.String()) {
|
|
chatID = strings.TrimPrefix(chatID, communityID.String())
|
|
}
|
|
changes, err := community.DeleteChat(chatID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, changes, nil
|
|
}
|
|
|
|
func (m *Manager) CreateCategory(request *requests.CreateCommunityCategory, publish bool) (*Community, *CommunityChanges, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
community, err := m.GetByID(request.CommunityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
categoryID := uuid.New().String()
|
|
if request.ThirdPartyID != "" {
|
|
categoryID = categoryID + request.ThirdPartyID
|
|
}
|
|
|
|
// Remove communityID prefix from chatID if exists
|
|
for i, cid := range request.ChatIDs {
|
|
if strings.HasPrefix(cid, request.CommunityID.String()) {
|
|
request.ChatIDs[i] = strings.TrimPrefix(cid, request.CommunityID.String())
|
|
}
|
|
}
|
|
|
|
changes, err := community.CreateCategory(categoryID, request.CategoryName, request.ChatIDs)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, changes, nil
|
|
}
|
|
|
|
func (m *Manager) EditCategory(request *requests.EditCommunityCategory) (*Community, *CommunityChanges, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
community, err := m.GetByID(request.CommunityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
// Remove communityID prefix from chatID if exists
|
|
for i, cid := range request.ChatIDs {
|
|
if strings.HasPrefix(cid, request.CommunityID.String()) {
|
|
request.ChatIDs[i] = strings.TrimPrefix(cid, request.CommunityID.String())
|
|
}
|
|
}
|
|
|
|
changes, err := community.EditCategory(request.CategoryID, request.CategoryName, request.ChatIDs)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, changes, nil
|
|
}
|
|
|
|
func (m *Manager) EditChatFirstMessageTimestamp(communityID types.HexBytes, chatID string, timestamp uint32) (*Community, *CommunityChanges, error) {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
// Remove communityID prefix from chatID if exists
|
|
if strings.HasPrefix(chatID, communityID.String()) {
|
|
chatID = strings.TrimPrefix(chatID, communityID.String())
|
|
}
|
|
|
|
changes, err := community.UpdateChatFirstMessageTimestamp(chatID, timestamp)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
// Advertise changes
|
|
m.publish(&Subscription{Community: community})
|
|
|
|
return community, changes, nil
|
|
}
|
|
|
|
func (m *Manager) ReorderCategories(request *requests.ReorderCommunityCategories) (*Community, *CommunityChanges, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
community, err := m.GetByID(request.CommunityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
changes, err := community.ReorderCategories(request.CategoryID, request.Position)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, changes, nil
|
|
}
|
|
|
|
func (m *Manager) ReorderChat(request *requests.ReorderCommunityChat) (*Community, *CommunityChanges, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
community, err := m.GetByID(request.CommunityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
// Remove communityID prefix from chatID if exists
|
|
if strings.HasPrefix(request.ChatID, request.CommunityID.String()) {
|
|
request.ChatID = strings.TrimPrefix(request.ChatID, request.CommunityID.String())
|
|
}
|
|
|
|
changes, err := community.ReorderChat(request.CategoryID, request.ChatID, request.Position)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, changes, nil
|
|
}
|
|
|
|
func (m *Manager) DeleteCategory(request *requests.DeleteCommunityCategory) (*Community, *CommunityChanges, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
community, err := m.GetByID(request.CommunityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
changes, err := community.DeleteCategory(request.CategoryID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return changes.Community, changes, nil
|
|
}
|
|
|
|
func (m *Manager) GenerateRequestsToJoinForAutoApprovalOnNewOwnership(communityID types.HexBytes, kickedMembers map[string]*protobuf.CommunityMember) ([]*RequestToJoin, error) {
|
|
var requestsToJoin []*RequestToJoin
|
|
clock := uint64(time.Now().Unix())
|
|
for pubKeyStr := range kickedMembers {
|
|
requestToJoin := &RequestToJoin{
|
|
PublicKey: pubKeyStr,
|
|
Clock: clock,
|
|
CommunityID: communityID,
|
|
State: RequestToJoinStateAwaitingAddresses,
|
|
Our: true,
|
|
RevealedAccounts: make([]*protobuf.RevealedAccount, 0),
|
|
}
|
|
|
|
requestToJoin.CalculateID()
|
|
|
|
requestsToJoin = append(requestsToJoin, requestToJoin)
|
|
}
|
|
|
|
return requestsToJoin, m.persistence.SaveRequestsToJoin(requestsToJoin)
|
|
}
|
|
|
|
func (m *Manager) Queue(signer *ecdsa.PublicKey, community *Community, clock uint64, payload []byte) error {
|
|
|
|
m.logger.Info("queuing community", zap.String("id", community.IDString()), zap.String("signer", common.PubkeyToHex(signer)))
|
|
|
|
communityToValidate := communityToValidate{
|
|
id: community.ID(),
|
|
clock: clock,
|
|
payload: payload,
|
|
validateAt: uint64(time.Now().UnixNano()),
|
|
signer: crypto.CompressPubkey(signer),
|
|
}
|
|
err := m.persistence.SaveCommunityToValidate(communityToValidate)
|
|
if err != nil {
|
|
m.logger.Error("failed to save community", zap.Error(err))
|
|
return err
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) HandleCommunityDescriptionMessage(signer *ecdsa.PublicKey, description *protobuf.CommunityDescription, payload []byte, verifiedOwner *ecdsa.PublicKey, communityShard *protobuf.Shard) (*CommunityResponse, error) {
|
|
m.logger.Debug("HandleCommunityDescriptionMessage", zap.String("communityID", description.ID), zap.Uint64("clock", description.Clock))
|
|
|
|
if signer == nil {
|
|
return nil, errors.New("signer can't be nil")
|
|
}
|
|
|
|
var id []byte
|
|
var err error
|
|
if len(description.ID) != 0 {
|
|
id, err = types.DecodeHex(description.ID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
} else {
|
|
// Backward compatibility
|
|
id = crypto.CompressPubkey(signer)
|
|
}
|
|
|
|
failedToDecrypt, processedDescription, err := m.preprocessDescription(id, description)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
m.communityLock.Lock(id)
|
|
defer m.communityLock.Unlock(id)
|
|
community, err := m.GetByID(id)
|
|
if err != nil && err != ErrOrgNotFound {
|
|
return nil, err
|
|
}
|
|
|
|
// We don't process failed to decrypt if the whole metadata is encrypted
|
|
// and we joined the community already
|
|
if community != nil && community.Joined() && len(failedToDecrypt) != 0 && processedDescription != nil && len(processedDescription.Members) == 0 {
|
|
return &CommunityResponse{FailedToDecrypt: failedToDecrypt}, nil
|
|
}
|
|
|
|
// We should queue only if the community has a token owner, and the owner has been verified
|
|
hasTokenOwnership := HasTokenOwnership(processedDescription)
|
|
shouldQueue := hasTokenOwnership && verifiedOwner == nil
|
|
|
|
if community == nil {
|
|
pubKey, err := crypto.DecompressPubkey(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
config := Config{
|
|
CommunityDescription: processedDescription,
|
|
Logger: m.logger,
|
|
CommunityDescriptionProtocolMessage: payload,
|
|
MemberIdentity: &m.identity.PublicKey,
|
|
ID: pubKey,
|
|
ControlNode: signer,
|
|
Shard: shard.FromProtobuff(communityShard),
|
|
}
|
|
|
|
var descriptionEncryptor DescriptionEncryptor
|
|
if m.encryptor != nil {
|
|
descriptionEncryptor = m
|
|
}
|
|
community, err = New(config, m.timesource, descriptionEncryptor)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// A new community, we need to check if we need to validate async.
|
|
// That would be the case if it has a contract. We queue everything and process separately.
|
|
if shouldQueue {
|
|
return nil, m.Queue(signer, community, processedDescription.Clock, payload)
|
|
}
|
|
} else {
|
|
// only queue if already known control node is different than the signer
|
|
// and if the clock is greater
|
|
shouldQueue = shouldQueue && !common.IsPubKeyEqual(community.ControlNode(), signer) &&
|
|
community.config.CommunityDescription.Clock < processedDescription.Clock
|
|
if shouldQueue {
|
|
return nil, m.Queue(signer, community, processedDescription.Clock, payload)
|
|
}
|
|
}
|
|
|
|
if hasTokenOwnership && verifiedOwner != nil {
|
|
// Override verified owner
|
|
m.logger.Info("updating verified owner",
|
|
zap.String("communityID", community.IDString()),
|
|
zap.String("verifiedOwner", common.PubkeyToHex(verifiedOwner)),
|
|
zap.String("signer", common.PubkeyToHex(signer)),
|
|
zap.String("controlNode", common.PubkeyToHex(community.ControlNode())),
|
|
)
|
|
|
|
// If we are not the verified owner anymore, drop the private key
|
|
if !common.IsPubKeyEqual(verifiedOwner, &m.identity.PublicKey) {
|
|
community.config.PrivateKey = nil
|
|
}
|
|
|
|
// new control node will be set in the 'UpdateCommunityDescription'
|
|
if !common.IsPubKeyEqual(verifiedOwner, signer) {
|
|
return nil, ErrNotAuthorized
|
|
}
|
|
} else if !common.IsPubKeyEqual(community.ControlNode(), signer) {
|
|
return nil, ErrNotAuthorized
|
|
}
|
|
|
|
r, err := m.handleCommunityDescriptionMessageCommon(community, processedDescription, payload, verifiedOwner)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
r.FailedToDecrypt = failedToDecrypt
|
|
return r, nil
|
|
}
|
|
|
|
func (m *Manager) NewHashRatchetKeys(keys []*encryption.HashRatchetInfo) error {
|
|
return m.persistence.InvalidateDecryptedCommunityCacheForKeys(keys)
|
|
}
|
|
|
|
func (m *Manager) preprocessDescription(id types.HexBytes, description *protobuf.CommunityDescription) ([]*CommunityPrivateDataFailedToDecrypt, *protobuf.CommunityDescription, error) {
|
|
decryptedCommunity, err := m.persistence.GetDecryptedCommunityDescription(id, description.Clock)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
if decryptedCommunity != nil {
|
|
return nil, decryptedCommunity, nil
|
|
}
|
|
|
|
response, err := decryptDescription(id, m, description, m.logger)
|
|
if err != nil {
|
|
return response, description, err
|
|
}
|
|
|
|
upgradeTokenPermissions(description)
|
|
|
|
// Workaround for https://github.com/status-im/status-desktop/issues/12188
|
|
hydrateChannelsMembers(types.EncodeHex(id), description)
|
|
|
|
return response, description, m.persistence.SaveDecryptedCommunityDescription(id, response, description)
|
|
}
|
|
|
|
func (m *Manager) handleCommunityDescriptionMessageCommon(community *Community, description *protobuf.CommunityDescription, payload []byte, newControlNode *ecdsa.PublicKey) (*CommunityResponse, error) {
|
|
prevClock := community.config.CommunityDescription.Clock
|
|
changes, err := community.UpdateCommunityDescription(description, payload, newControlNode)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if err = m.handleCommunityTokensMetadata(community); err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
hasCommunityArchiveInfo, err := m.persistence.HasCommunityArchiveInfo(community.ID())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
cdMagnetlinkClock := community.config.CommunityDescription.ArchiveMagnetlinkClock
|
|
if !hasCommunityArchiveInfo {
|
|
err = m.persistence.SaveCommunityArchiveInfo(community.ID(), cdMagnetlinkClock, 0)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
} else {
|
|
magnetlinkClock, err := m.persistence.GetMagnetlinkMessageClock(community.ID())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
if cdMagnetlinkClock > magnetlinkClock {
|
|
err = m.persistence.UpdateMagnetlinkMessageClock(community.ID(), cdMagnetlinkClock)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
}
|
|
|
|
pkString := common.PubkeyToHex(&m.identity.PublicKey)
|
|
if m.tokenManager != nil && description.CommunityTokensMetadata != nil && len(description.CommunityTokensMetadata) > 0 {
|
|
for _, tokenMetadata := range description.CommunityTokensMetadata {
|
|
if tokenMetadata.TokenType != protobuf.CommunityTokenType_ERC20 {
|
|
continue
|
|
}
|
|
|
|
for chainID, address := range tokenMetadata.ContractAddresses {
|
|
_ = m.tokenManager.FindOrCreateTokenByAddress(context.Background(), chainID, gethcommon.HexToAddress(address))
|
|
}
|
|
}
|
|
}
|
|
|
|
// If the community require membership, we set whether we should leave/join the community after a state change
|
|
if community.ManualAccept() || community.AutoAccept() {
|
|
if changes.HasNewMember(pkString) {
|
|
hasPendingRequest, err := m.persistence.HasPendingRequestsToJoinForUserAndCommunity(pkString, changes.Community.ID())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
// If there's any pending request, we should join the community
|
|
// automatically
|
|
changes.ShouldMemberJoin = hasPendingRequest
|
|
}
|
|
|
|
if changes.HasMemberLeft(pkString) {
|
|
// If we joined previously the community, that means we have been kicked
|
|
changes.MemberKicked = community.Joined()
|
|
}
|
|
}
|
|
|
|
if description.Clock > prevClock {
|
|
err = m.persistence.DeleteCommunityEvents(community.ID())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
community.config.EventsData = nil
|
|
}
|
|
|
|
// Set Joined if we are part of the member list
|
|
if !community.Joined() && community.hasMember(&m.identity.PublicKey) {
|
|
changes.ShouldMemberJoin = true
|
|
}
|
|
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// We mark our requests as completed, though maybe we should mark
|
|
// any request for any user that has been added as completed
|
|
if err := m.markRequestToJoinAsAccepted(&m.identity.PublicKey, community); err != nil {
|
|
return nil, err
|
|
}
|
|
// Check if there's a change and we should be joining
|
|
|
|
return &CommunityResponse{
|
|
Community: community,
|
|
Changes: changes,
|
|
}, nil
|
|
}
|
|
|
|
func (m *Manager) signEvents(community *Community) error {
|
|
for i := range community.config.EventsData.Events {
|
|
communityEvent := &community.config.EventsData.Events[i]
|
|
if communityEvent.Signature == nil || len(communityEvent.Signature) == 0 {
|
|
err := communityEvent.Sign(m.identity)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) HandleCommunityEventsMessage(signer *ecdsa.PublicKey, message *protobuf.CommunityEventsMessage) (*CommunityResponse, error) {
|
|
if signer == nil {
|
|
return nil, errors.New("signer can't be nil")
|
|
}
|
|
|
|
eventsMessage, err := CommunityEventsMessageFromProtobuf(message)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
m.communityLock.Lock(eventsMessage.CommunityID)
|
|
defer m.communityLock.Unlock(eventsMessage.CommunityID)
|
|
|
|
community, err := m.GetByID(eventsMessage.CommunityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if !community.IsPrivilegedMember(signer) {
|
|
return nil, errors.New("user has not permissions to send events")
|
|
}
|
|
|
|
originCommunity := community.CreateDeepCopy()
|
|
|
|
var lastlyAppliedEvents map[string]uint64
|
|
if community.IsControlNode() {
|
|
lastlyAppliedEvents, err = m.persistence.GetAppliedCommunityEvents(community.ID())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
additionalCommunityResponse, err := m.handleCommunityEventsAndMetadata(community, eventsMessage, lastlyAppliedEvents)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// Control node applies events and publish updated CommunityDescription
|
|
if community.IsControlNode() {
|
|
appliedEvents := map[string]uint64{}
|
|
if community.config.EventsData != nil {
|
|
for _, event := range community.config.EventsData.Events {
|
|
appliedEvents[event.EventTypeID()] = event.CommunityEventClock
|
|
}
|
|
}
|
|
community.config.EventsData = nil // clear events, they are already applied
|
|
community.increaseClock()
|
|
|
|
if m.keyDistributor != nil {
|
|
encryptionKeyActions := EvaluateCommunityEncryptionKeyActions(originCommunity, community)
|
|
err := m.keyDistributor.Generate(community, encryptionKeyActions)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.persistence.UpsertAppliedCommunityEvents(community.ID(), appliedEvents)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
m.publish(&Subscription{Community: community})
|
|
} else {
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
err := m.persistence.SaveCommunityEvents(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
return &CommunityResponse{
|
|
Community: community,
|
|
Changes: EvaluateCommunityChanges(originCommunity, community),
|
|
RequestsToJoin: additionalCommunityResponse.RequestsToJoin,
|
|
}, nil
|
|
}
|
|
|
|
func (m *Manager) handleAdditionalAdminChanges(community *Community) (*CommunityResponse, error) {
|
|
communityResponse := CommunityResponse{
|
|
RequestsToJoin: make([]*RequestToJoin, 0),
|
|
}
|
|
|
|
if !(community.IsControlNode() || community.HasPermissionToSendCommunityEvents()) {
|
|
// we're a normal user/member node, so there's nothing for us to do here
|
|
return &communityResponse, nil
|
|
}
|
|
|
|
if community.config.EventsData == nil {
|
|
return &communityResponse, nil
|
|
}
|
|
|
|
handledMembers := map[string]struct{}{}
|
|
|
|
for i := len(community.config.EventsData.Events) - 1; i >= 0; i-- {
|
|
communityEvent := &community.config.EventsData.Events[i]
|
|
if _, handled := handledMembers[communityEvent.MemberToAction]; handled {
|
|
continue
|
|
}
|
|
switch communityEvent.Type {
|
|
case protobuf.CommunityEvent_COMMUNITY_REQUEST_TO_JOIN_ACCEPT:
|
|
handledMembers[communityEvent.MemberToAction] = struct{}{}
|
|
requestsToJoin, err := m.handleCommunityEventRequestAccepted(community, communityEvent)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
if requestsToJoin != nil {
|
|
communityResponse.RequestsToJoin = append(communityResponse.RequestsToJoin, requestsToJoin...)
|
|
}
|
|
|
|
case protobuf.CommunityEvent_COMMUNITY_REQUEST_TO_JOIN_REJECT:
|
|
handledMembers[communityEvent.MemberToAction] = struct{}{}
|
|
requestsToJoin, err := m.handleCommunityEventRequestRejected(community, communityEvent)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
if requestsToJoin != nil {
|
|
communityResponse.RequestsToJoin = append(communityResponse.RequestsToJoin, requestsToJoin...)
|
|
}
|
|
|
|
default:
|
|
}
|
|
}
|
|
return &communityResponse, nil
|
|
}
|
|
|
|
func (m *Manager) saveOrUpdateRequestToJoin(communityID types.HexBytes, requestToJoin *RequestToJoin) (bool, error) {
|
|
updated := false
|
|
|
|
existingRequestToJoin, err := m.persistence.GetRequestToJoin(requestToJoin.ID)
|
|
if err != nil && err != sql.ErrNoRows {
|
|
return updated, err
|
|
}
|
|
|
|
if existingRequestToJoin != nil {
|
|
// node already knows about this request to join, so let's compare clocks
|
|
// and update it if necessary
|
|
if existingRequestToJoin.Clock <= requestToJoin.Clock {
|
|
pk, err := common.HexToPubkey(existingRequestToJoin.PublicKey)
|
|
if err != nil {
|
|
return updated, err
|
|
}
|
|
err = m.persistence.SetRequestToJoinState(common.PubkeyToHex(pk), communityID, requestToJoin.State)
|
|
if err != nil {
|
|
return updated, err
|
|
}
|
|
updated = true
|
|
}
|
|
} else {
|
|
err := m.persistence.SaveRequestToJoin(requestToJoin)
|
|
if err != nil {
|
|
return updated, err
|
|
}
|
|
}
|
|
|
|
return updated, nil
|
|
}
|
|
|
|
func (m *Manager) handleCommunityEventRequestAccepted(community *Community, communityEvent *CommunityEvent) ([]*RequestToJoin, error) {
|
|
acceptedRequestsToJoin := make([]types.HexBytes, 0)
|
|
|
|
requestsToJoin := make([]*RequestToJoin, 0)
|
|
|
|
signer := communityEvent.MemberToAction
|
|
request := communityEvent.RequestToJoin
|
|
|
|
requestToJoin := &RequestToJoin{
|
|
PublicKey: signer,
|
|
Clock: request.Clock,
|
|
ENSName: request.EnsName,
|
|
CommunityID: request.CommunityId,
|
|
State: RequestToJoinStateAcceptedPending,
|
|
CustomizationColor: multiaccountscommon.IDToColorFallbackToBlue(request.CustomizationColor),
|
|
}
|
|
requestToJoin.CalculateID()
|
|
|
|
existingRequestToJoin, err := m.persistence.GetRequestToJoin(requestToJoin.ID)
|
|
if err != nil && err != sql.ErrNoRows {
|
|
return nil, err
|
|
}
|
|
|
|
if existingRequestToJoin != nil {
|
|
alreadyProcessedByControlNode := existingRequestToJoin.State == RequestToJoinStateAccepted
|
|
if alreadyProcessedByControlNode || existingRequestToJoin.State == RequestToJoinStateCanceled {
|
|
return requestsToJoin, nil
|
|
}
|
|
}
|
|
|
|
requestUpdated, err := m.saveOrUpdateRequestToJoin(community.ID(), requestToJoin)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// If request to join exists in control node, add request to acceptedRequestsToJoin.
|
|
// Otherwise keep the request as RequestToJoinStateAcceptedPending,
|
|
// as privileged users don't have revealed addresses. This can happen if control node received
|
|
// community event message before user request to join.
|
|
if community.IsControlNode() && requestUpdated {
|
|
acceptedRequestsToJoin = append(acceptedRequestsToJoin, requestToJoin.ID)
|
|
}
|
|
|
|
requestsToJoin = append(requestsToJoin, requestToJoin)
|
|
|
|
if community.IsControlNode() {
|
|
m.publish(&Subscription{AcceptedRequestsToJoin: acceptedRequestsToJoin})
|
|
}
|
|
return requestsToJoin, nil
|
|
}
|
|
|
|
func (m *Manager) handleCommunityEventRequestRejected(community *Community, communityEvent *CommunityEvent) ([]*RequestToJoin, error) {
|
|
rejectedRequestsToJoin := make([]types.HexBytes, 0)
|
|
|
|
requestsToJoin := make([]*RequestToJoin, 0)
|
|
|
|
signer := communityEvent.MemberToAction
|
|
request := communityEvent.RequestToJoin
|
|
|
|
requestToJoin := &RequestToJoin{
|
|
PublicKey: signer,
|
|
Clock: request.Clock,
|
|
ENSName: request.EnsName,
|
|
CommunityID: request.CommunityId,
|
|
State: RequestToJoinStateDeclinedPending,
|
|
CustomizationColor: multiaccountscommon.IDToColorFallbackToBlue(request.CustomizationColor),
|
|
}
|
|
requestToJoin.CalculateID()
|
|
|
|
existingRequestToJoin, err := m.persistence.GetRequestToJoin(requestToJoin.ID)
|
|
if err != nil && err != sql.ErrNoRows {
|
|
return nil, err
|
|
}
|
|
|
|
if existingRequestToJoin != nil {
|
|
alreadyProcessedByControlNode := existingRequestToJoin.State == RequestToJoinStateDeclined
|
|
if alreadyProcessedByControlNode || existingRequestToJoin.State == RequestToJoinStateCanceled {
|
|
return requestsToJoin, nil
|
|
}
|
|
}
|
|
|
|
requestUpdated, err := m.saveOrUpdateRequestToJoin(community.ID(), requestToJoin)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
// If request to join exists in control node, add request to rejectedRequestsToJoin.
|
|
// Otherwise keep the request as RequestToJoinStateDeclinedPending,
|
|
// as privileged users don't have revealed addresses. This can happen if control node received
|
|
// community event message before user request to join.
|
|
if community.IsControlNode() && requestUpdated {
|
|
rejectedRequestsToJoin = append(rejectedRequestsToJoin, requestToJoin.ID)
|
|
}
|
|
|
|
requestsToJoin = append(requestsToJoin, requestToJoin)
|
|
|
|
if community.IsControlNode() {
|
|
m.publish(&Subscription{RejectedRequestsToJoin: rejectedRequestsToJoin})
|
|
}
|
|
return requestsToJoin, nil
|
|
}
|
|
|
|
// markRequestToJoinAsAccepted marks all the pending requests to join as completed
|
|
// if we are members
|
|
func (m *Manager) markRequestToJoinAsAccepted(pk *ecdsa.PublicKey, community *Community) error {
|
|
if community.HasMember(pk) {
|
|
return m.persistence.SetRequestToJoinState(common.PubkeyToHex(pk), community.ID(), RequestToJoinStateAccepted)
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) markRequestToJoinAsCanceled(pk *ecdsa.PublicKey, community *Community) error {
|
|
return m.persistence.SetRequestToJoinState(common.PubkeyToHex(pk), community.ID(), RequestToJoinStateCanceled)
|
|
}
|
|
|
|
func (m *Manager) markRequestToJoinAsAcceptedPending(pk *ecdsa.PublicKey, community *Community) error {
|
|
return m.persistence.SetRequestToJoinState(common.PubkeyToHex(pk), community.ID(), RequestToJoinStateAcceptedPending)
|
|
}
|
|
|
|
func (m *Manager) DeletePendingRequestToJoin(request *RequestToJoin) error {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
community, err := m.GetByID(request.CommunityID)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
err = m.persistence.DeletePendingRequestToJoin(request.ID)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
// UpdateClockInRequestToJoin method is used for testing
|
|
func (m *Manager) UpdateClockInRequestToJoin(id types.HexBytes, clock uint64) error {
|
|
return m.persistence.UpdateClockInRequestToJoin(id, clock)
|
|
}
|
|
|
|
func (m *Manager) SetMuted(id types.HexBytes, muted bool) error {
|
|
m.communityLock.Lock(id)
|
|
defer m.communityLock.Unlock(id)
|
|
|
|
return m.persistence.SetMuted(id, muted)
|
|
}
|
|
|
|
func (m *Manager) MuteCommunityTill(communityID []byte, muteTill time.Time) error {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
return m.persistence.MuteCommunityTill(communityID, muteTill)
|
|
}
|
|
func (m *Manager) CancelRequestToJoin(request *requests.CancelRequestToJoinCommunity) (*RequestToJoin, *Community, error) {
|
|
dbRequest, err := m.persistence.GetRequestToJoin(request.ID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
community, err := m.GetByID(dbRequest.CommunityID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
pk, err := common.HexToPubkey(dbRequest.PublicKey)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
dbRequest.State = RequestToJoinStateCanceled
|
|
if err := m.markRequestToJoinAsCanceled(pk, community); err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return dbRequest, community, nil
|
|
}
|
|
|
|
func (m *Manager) CheckPermissionToJoin(id []byte, addresses []gethcommon.Address) (*CheckPermissionToJoinResponse, error) {
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return m.PermissionChecker.CheckPermissionToJoin(community, addresses)
|
|
}
|
|
|
|
// Light version of CheckPermissionToJoin, which does not use network requests.
|
|
// Instead of checking wallet balances it checks if there is an access to a member list of the community.
|
|
func (m *Manager) CheckPermissionToJoinLight(id []byte) (bool, error) {
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return false, err
|
|
}
|
|
meAsMember := community.GetMember(&m.identity.PublicKey)
|
|
return meAsMember != nil, nil
|
|
}
|
|
|
|
func (m *Manager) accountsSatisfyPermissionsToJoin(
|
|
communityPermissionsPreParsedData map[protobuf.CommunityTokenPermission_Type]*PreParsedCommunityPermissionsData,
|
|
accountsAndChainIDs []*AccountChainIDsCombination) (bool, protobuf.CommunityMember_Roles, error) {
|
|
|
|
if m.accountsHasPrivilegedPermission(communityPermissionsPreParsedData[protobuf.CommunityTokenPermission_BECOME_TOKEN_MASTER], accountsAndChainIDs) {
|
|
return true, protobuf.CommunityMember_ROLE_TOKEN_MASTER, nil
|
|
}
|
|
if m.accountsHasPrivilegedPermission(communityPermissionsPreParsedData[protobuf.CommunityTokenPermission_BECOME_ADMIN], accountsAndChainIDs) {
|
|
return true, protobuf.CommunityMember_ROLE_ADMIN, nil
|
|
}
|
|
|
|
preParsedBecomeMemberPermissions := communityPermissionsPreParsedData[protobuf.CommunityTokenPermission_BECOME_MEMBER]
|
|
if preParsedBecomeMemberPermissions != nil {
|
|
permissionResponse, err := m.PermissionChecker.CheckPermissions(preParsedBecomeMemberPermissions, accountsAndChainIDs, true)
|
|
if err != nil {
|
|
return false, protobuf.CommunityMember_ROLE_NONE, err
|
|
}
|
|
|
|
return permissionResponse.Satisfied, protobuf.CommunityMember_ROLE_NONE, nil
|
|
}
|
|
|
|
return true, protobuf.CommunityMember_ROLE_NONE, nil
|
|
}
|
|
|
|
func (m *Manager) accountsSatisfyPermissionsToJoinChannels(
|
|
community *Community,
|
|
channelPermissionsPreParsedData map[string]*PreParsedCommunityPermissionsData,
|
|
accountsAndChainIDs []*AccountChainIDsCombination) (map[string]*protobuf.CommunityChat, map[string]*protobuf.CommunityChat, error) {
|
|
|
|
viewChats := make(map[string]*protobuf.CommunityChat)
|
|
viewAndPostChats := make(map[string]*protobuf.CommunityChat)
|
|
|
|
if len(channelPermissionsPreParsedData) == 0 {
|
|
for channelID, channel := range community.config.CommunityDescription.Chats {
|
|
viewAndPostChats[channelID] = channel
|
|
}
|
|
|
|
return viewChats, viewAndPostChats, nil
|
|
}
|
|
|
|
// check which permissions we satisfy and which not
|
|
channelPermissionsCheckResult, err := m.checkChannelsPermissions(channelPermissionsPreParsedData, accountsAndChainIDs, true)
|
|
if err != nil {
|
|
m.logger.Warn("check channel permission failed: %v", zap.Error(err))
|
|
return viewChats, viewAndPostChats, err
|
|
}
|
|
|
|
for channelID, channel := range community.config.CommunityDescription.Chats {
|
|
chatID := community.ChatID(channelID)
|
|
channelPermissionsCheckResult, exists := channelPermissionsCheckResult[chatID]
|
|
|
|
if !exists {
|
|
viewAndPostChats[channelID] = channel
|
|
continue
|
|
}
|
|
|
|
viewAndPostSatisfied, exists := channelPermissionsCheckResult[protobuf.CommunityTokenPermission_CAN_VIEW_AND_POST_CHANNEL]
|
|
if exists && viewAndPostSatisfied {
|
|
delete(viewChats, channelID)
|
|
viewAndPostChats[channelID] = channel
|
|
continue
|
|
}
|
|
|
|
viewOnlySatisfied, exists := channelPermissionsCheckResult[protobuf.CommunityTokenPermission_CAN_VIEW_CHANNEL]
|
|
if exists && viewOnlySatisfied {
|
|
if _, exists := viewAndPostChats[channelID]; !exists {
|
|
viewChats[channelID] = channel
|
|
}
|
|
}
|
|
}
|
|
|
|
return viewChats, viewAndPostChats, nil
|
|
}
|
|
|
|
func (m *Manager) AcceptRequestToJoin(dbRequest *RequestToJoin) (*Community, error) {
|
|
m.communityLock.Lock(dbRequest.CommunityID)
|
|
defer m.communityLock.Unlock(dbRequest.CommunityID)
|
|
|
|
pk, err := common.HexToPubkey(dbRequest.PublicKey)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
community, err := m.GetByID(dbRequest.CommunityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if community.IsControlNode() {
|
|
revealedAccounts, err := m.persistence.GetRequestToJoinRevealedAddresses(dbRequest.ID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
accountsAndChainIDs := revealedAccountsToAccountsAndChainIDsCombination(revealedAccounts)
|
|
|
|
communityPermissionsPreParsedData, channelPermissionsPreParsedData := PreParsePermissionsData(community.tokenPermissions())
|
|
|
|
permissionsSatisfied, role, err := m.accountsSatisfyPermissionsToJoin(communityPermissionsPreParsedData, accountsAndChainIDs)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if !permissionsSatisfied {
|
|
return community, ErrNoPermissionToJoin
|
|
}
|
|
|
|
memberRoles := []protobuf.CommunityMember_Roles{}
|
|
if role != protobuf.CommunityMember_ROLE_NONE {
|
|
memberRoles = []protobuf.CommunityMember_Roles{role}
|
|
}
|
|
|
|
_, err = community.AddMember(pk, memberRoles)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
viewChannels, postChannels, err := m.accountsSatisfyPermissionsToJoinChannels(community, channelPermissionsPreParsedData, accountsAndChainIDs)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
for channelID := range viewChannels {
|
|
_, err = community.AddMemberToChat(channelID, pk, memberRoles, protobuf.CommunityMember_CHANNEL_ROLE_VIEWER)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
for channelID := range postChannels {
|
|
_, err = community.AddMemberToChat(channelID, pk, memberRoles, protobuf.CommunityMember_CHANNEL_ROLE_POSTER)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
dbRequest.State = RequestToJoinStateAccepted
|
|
if err := m.markRequestToJoinAsAccepted(pk, community); err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
dbRequest.RevealedAccounts = revealedAccounts
|
|
if err = m.shareAcceptedRequestToJoinWithPrivilegedMembers(community, dbRequest); err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// if accepted member has a privilege role, share with him requests to join
|
|
memberRole := community.MemberRole(pk)
|
|
if memberRole == protobuf.CommunityMember_ROLE_OWNER || memberRole == protobuf.CommunityMember_ROLE_ADMIN ||
|
|
memberRole == protobuf.CommunityMember_ROLE_TOKEN_MASTER {
|
|
|
|
newPrivilegedMember := make(map[protobuf.CommunityMember_Roles][]*ecdsa.PublicKey)
|
|
newPrivilegedMember[memberRole] = []*ecdsa.PublicKey{pk}
|
|
if err = m.shareRequestsToJoinWithNewPrivilegedMembers(community, newPrivilegedMember); err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
} else if community.hasPermissionToSendCommunityEvent(protobuf.CommunityEvent_COMMUNITY_REQUEST_TO_JOIN_ACCEPT) {
|
|
err := community.addNewCommunityEvent(community.ToCommunityRequestToJoinAcceptCommunityEvent(dbRequest.PublicKey, dbRequest.ToCommunityRequestToJoinProtobuf()))
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
dbRequest.State = RequestToJoinStateAcceptedPending
|
|
if err := m.markRequestToJoinAsAcceptedPending(pk, community); err != nil {
|
|
return nil, err
|
|
}
|
|
} else {
|
|
return nil, ErrNotAuthorized
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) GetRequestToJoin(ID types.HexBytes) (*RequestToJoin, error) {
|
|
return m.persistence.GetRequestToJoin(ID)
|
|
}
|
|
|
|
func (m *Manager) DeclineRequestToJoin(dbRequest *RequestToJoin) (*Community, error) {
|
|
m.communityLock.Lock(dbRequest.CommunityID)
|
|
defer m.communityLock.Unlock(dbRequest.CommunityID)
|
|
|
|
community, err := m.GetByID(dbRequest.CommunityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
adminEventCreated, err := community.DeclineRequestToJoin(dbRequest)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
requestToJoinState := RequestToJoinStateDeclined
|
|
if adminEventCreated {
|
|
requestToJoinState = RequestToJoinStateDeclinedPending // can only be declined by control node
|
|
}
|
|
|
|
dbRequest.State = requestToJoinState
|
|
err = m.persistence.SetRequestToJoinState(dbRequest.PublicKey, dbRequest.CommunityID, requestToJoinState)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) shouldUserRetainDeclined(signer *ecdsa.PublicKey, community *Community, requestClock uint64) (bool, error) {
|
|
requestID := CalculateRequestID(common.PubkeyToHex(signer), types.HexBytes(community.IDString()))
|
|
request, err := m.persistence.GetRequestToJoin(requestID)
|
|
if err != nil {
|
|
if err == sql.ErrNoRows {
|
|
return false, nil
|
|
}
|
|
return false, err
|
|
}
|
|
|
|
return request.ShouldRetainDeclined(requestClock)
|
|
}
|
|
|
|
func (m *Manager) HandleCommunityCancelRequestToJoin(signer *ecdsa.PublicKey, request *protobuf.CommunityCancelRequestToJoin) (*RequestToJoin, error) {
|
|
m.communityLock.Lock(request.CommunityId)
|
|
defer m.communityLock.Unlock(request.CommunityId)
|
|
|
|
community, err := m.GetByID(request.CommunityId)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
previousRequestToJoin, err := m.GetRequestToJoinByPkAndCommunityID(signer, community.ID())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if request.Clock <= previousRequestToJoin.Clock {
|
|
return nil, ErrInvalidClock
|
|
}
|
|
|
|
retainDeclined, err := m.shouldUserRetainDeclined(signer, community, request.Clock)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
if retainDeclined {
|
|
return nil, ErrCommunityRequestAlreadyRejected
|
|
}
|
|
|
|
err = m.markRequestToJoinAsCanceled(signer, community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
requestToJoin, err := m.persistence.GetRequestToJoinByPk(common.PubkeyToHex(signer), community.ID(), RequestToJoinStateCanceled)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if community.HasMember(signer) {
|
|
_, err = community.RemoveUserFromOrg(signer)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
return requestToJoin, nil
|
|
}
|
|
|
|
func (m *Manager) HandleCommunityRequestToJoin(signer *ecdsa.PublicKey, receiver *ecdsa.PublicKey, request *protobuf.CommunityRequestToJoin) (*Community, *RequestToJoin, error) {
|
|
community, err := m.GetByID(request.CommunityId)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
err = community.ValidateRequestToJoin(signer, request)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
nbPendingRequestsToJoin, err := m.persistence.GetNumberOfPendingRequestsToJoin(community.ID())
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
if nbPendingRequestsToJoin >= maxNbPendingRequestedMembers {
|
|
return nil, nil, errors.New("max number of requests to join reached")
|
|
}
|
|
|
|
requestToJoin := &RequestToJoin{
|
|
PublicKey: common.PubkeyToHex(signer),
|
|
Clock: request.Clock,
|
|
ENSName: request.EnsName,
|
|
CommunityID: request.CommunityId,
|
|
State: RequestToJoinStatePending,
|
|
RevealedAccounts: request.RevealedAccounts,
|
|
CustomizationColor: multiaccountscommon.IDToColorFallbackToBlue(request.CustomizationColor),
|
|
}
|
|
requestToJoin.CalculateID()
|
|
|
|
existingRequestToJoin, err := m.persistence.GetRequestToJoin(requestToJoin.ID)
|
|
if err != nil && err != sql.ErrNoRows {
|
|
return nil, nil, err
|
|
}
|
|
|
|
if existingRequestToJoin == nil {
|
|
err = m.SaveRequestToJoin(requestToJoin)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
} else {
|
|
retainDeclined, err := existingRequestToJoin.ShouldRetainDeclined(request.Clock)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
if retainDeclined {
|
|
return nil, nil, ErrCommunityRequestAlreadyRejected
|
|
}
|
|
|
|
switch existingRequestToJoin.State {
|
|
case RequestToJoinStatePending, RequestToJoinStateDeclined, RequestToJoinStateCanceled:
|
|
// Another request have been received, save request back to pending state
|
|
err = m.SaveRequestToJoin(requestToJoin)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
case RequestToJoinStateAccepted:
|
|
// if member leaved the community and tries to request to join again
|
|
if !community.HasMember(signer) {
|
|
err = m.SaveRequestToJoin(requestToJoin)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
if community.IsControlNode() {
|
|
// verify if revealed addresses indeed belong to requester
|
|
for _, revealedAccount := range request.RevealedAccounts {
|
|
recoverParams := account.RecoverParams{
|
|
Message: types.EncodeHex(crypto.Keccak256(crypto.CompressPubkey(signer), community.ID(), requestToJoin.ID)),
|
|
Signature: types.EncodeHex(revealedAccount.Signature),
|
|
}
|
|
|
|
matching, err := m.accountsManager.CanRecover(recoverParams, types.HexToAddress(revealedAccount.Address))
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
if !matching {
|
|
// if ownership of only one wallet address cannot be verified,
|
|
// we mark the request as cancelled and stop
|
|
requestToJoin.State = RequestToJoinStateDeclined
|
|
return community, requestToJoin, nil
|
|
}
|
|
}
|
|
|
|
// Save revealed addresses + signatures so they can later be added
|
|
// to the control node's local table of known revealed addresses
|
|
err = m.persistence.SaveRequestToJoinRevealedAddresses(requestToJoin.ID, requestToJoin.RevealedAccounts)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
if existingRequestToJoin != nil {
|
|
// request to join was already processed by privileged user
|
|
// and waits to get confirmation for its decision
|
|
if existingRequestToJoin.State == RequestToJoinStateDeclinedPending {
|
|
requestToJoin.State = RequestToJoinStateDeclined
|
|
return community, requestToJoin, nil
|
|
} else if existingRequestToJoin.State == RequestToJoinStateAcceptedPending {
|
|
requestToJoin.State = RequestToJoinStateAccepted
|
|
return community, requestToJoin, nil
|
|
|
|
} else if existingRequestToJoin.State == RequestToJoinStateAwaitingAddresses {
|
|
// community ownership changed, accept request automatically
|
|
requestToJoin.State = RequestToJoinStateAccepted
|
|
return community, requestToJoin, nil
|
|
}
|
|
}
|
|
|
|
// Check if we reached the limit, if we did, change the community setting to be On Request
|
|
if community.AutoAccept() && community.MembersCount() >= maxNbMembers {
|
|
community.EditPermissionAccess(protobuf.CommunityPermissions_MANUAL_ACCEPT)
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
}
|
|
|
|
// If user is already a member, then accept request automatically
|
|
// It may happen when member removes itself from community and then tries to rejoin
|
|
// More specifically, CommunityRequestToLeave may be delivered later than CommunityRequestToJoin, or not delivered at all
|
|
acceptAutomatically := community.AutoAccept() || community.HasMember(signer)
|
|
if acceptAutomatically {
|
|
// Don't check permissions here,
|
|
// it will be done further in the processing pipeline.
|
|
requestToJoin.State = RequestToJoinStateAccepted
|
|
return community, requestToJoin, nil
|
|
}
|
|
}
|
|
|
|
return community, requestToJoin, nil
|
|
}
|
|
|
|
func (m *Manager) HandleCommunityEditSharedAddresses(signer *ecdsa.PublicKey, request *protobuf.CommunityEditSharedAddresses) error {
|
|
m.communityLock.Lock(request.CommunityId)
|
|
defer m.communityLock.Unlock(request.CommunityId)
|
|
|
|
community, err := m.GetByID(request.CommunityId)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
if err := community.ValidateEditSharedAddresses(signer, request); err != nil {
|
|
return err
|
|
}
|
|
|
|
// verify if revealed addresses indeed belong to requester
|
|
for _, revealedAccount := range request.RevealedAccounts {
|
|
recoverParams := account.RecoverParams{
|
|
Message: types.EncodeHex(crypto.Keccak256(crypto.CompressPubkey(signer), community.ID())),
|
|
Signature: types.EncodeHex(revealedAccount.Signature),
|
|
}
|
|
|
|
matching, err := m.accountsManager.CanRecover(recoverParams, types.HexToAddress(revealedAccount.Address))
|
|
if err != nil {
|
|
return err
|
|
}
|
|
if !matching {
|
|
// if ownership of only one wallet address cannot be verified we stop
|
|
return errors.New("wrong wallet address used")
|
|
}
|
|
}
|
|
|
|
requestToJoin := &RequestToJoin{
|
|
PublicKey: common.PubkeyToHex(signer),
|
|
CommunityID: community.ID(),
|
|
RevealedAccounts: request.RevealedAccounts,
|
|
}
|
|
requestToJoin.CalculateID()
|
|
|
|
err = m.persistence.RemoveRequestToJoinRevealedAddresses(requestToJoin.ID)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
err = m.persistence.SaveRequestToJoinRevealedAddresses(requestToJoin.ID, requestToJoin.RevealedAccounts)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
if community.IsControlNode() {
|
|
m.publish(&Subscription{Community: community})
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func calculateChainIDsSet(accountsAndChainIDs []*AccountChainIDsCombination, requirementsChainIDs map[uint64]bool) []uint64 {
|
|
|
|
revealedAccountsChainIDs := make([]uint64, 0)
|
|
revealedAccountsChainIDsMap := make(map[uint64]bool)
|
|
|
|
// we want all chainIDs provided by revealed addresses that also exist
|
|
// in the token requirements
|
|
for _, accountAndChainIDs := range accountsAndChainIDs {
|
|
for _, chainID := range accountAndChainIDs.ChainIDs {
|
|
if requirementsChainIDs[chainID] && !revealedAccountsChainIDsMap[chainID] {
|
|
revealedAccountsChainIDsMap[chainID] = true
|
|
revealedAccountsChainIDs = append(revealedAccountsChainIDs, chainID)
|
|
}
|
|
}
|
|
}
|
|
return revealedAccountsChainIDs
|
|
}
|
|
|
|
type CollectiblesByChain = map[uint64]map[gethcommon.Address]thirdparty.TokenBalancesPerContractAddress
|
|
|
|
func (m *Manager) GetOwnedERC721Tokens(walletAddresses []gethcommon.Address, tokenRequirements map[uint64]map[string]*protobuf.TokenCriteria, chainIDs []uint64) (CollectiblesByChain, error) {
|
|
if m.collectiblesManager == nil {
|
|
return nil, errors.New("no collectibles manager")
|
|
}
|
|
|
|
ctx := context.Background()
|
|
|
|
ownedERC721Tokens := make(CollectiblesByChain)
|
|
|
|
for chainID, erc721Tokens := range tokenRequirements {
|
|
|
|
skipChain := true
|
|
for _, cID := range chainIDs {
|
|
if chainID == cID {
|
|
skipChain = false
|
|
}
|
|
}
|
|
|
|
if skipChain {
|
|
continue
|
|
}
|
|
|
|
contractAddresses := make([]gethcommon.Address, 0)
|
|
for contractAddress := range erc721Tokens {
|
|
contractAddresses = append(contractAddresses, gethcommon.HexToAddress(contractAddress))
|
|
}
|
|
|
|
if _, exists := ownedERC721Tokens[chainID]; !exists {
|
|
ownedERC721Tokens[chainID] = make(map[gethcommon.Address]thirdparty.TokenBalancesPerContractAddress)
|
|
}
|
|
|
|
for _, owner := range walletAddresses {
|
|
balances, err := m.collectiblesManager.FetchBalancesByOwnerAndContractAddress(ctx, walletcommon.ChainID(chainID), owner, contractAddresses)
|
|
if err != nil {
|
|
m.logger.Info("couldn't fetch owner assets", zap.Error(err))
|
|
return nil, err
|
|
}
|
|
ownedERC721Tokens[chainID][owner] = balances
|
|
}
|
|
}
|
|
return ownedERC721Tokens, nil
|
|
}
|
|
|
|
func (m *Manager) CheckChannelPermissions(communityID types.HexBytes, chatID string, addresses []gethcommon.Address) (*CheckChannelPermissionsResponse, error) {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if chatID == "" {
|
|
return nil, errors.New(fmt.Sprintf("couldn't check channel permissions, invalid chat id: %s", chatID))
|
|
}
|
|
|
|
viewOnlyPermissions := community.ChannelTokenPermissionsByType(chatID, protobuf.CommunityTokenPermission_CAN_VIEW_CHANNEL)
|
|
viewAndPostPermissions := community.ChannelTokenPermissionsByType(chatID, protobuf.CommunityTokenPermission_CAN_VIEW_AND_POST_CHANNEL)
|
|
viewOnlyPreParsedPermissions := preParsedCommunityPermissionsData(viewOnlyPermissions)
|
|
viewAndPostPreParsedPermissions := preParsedCommunityPermissionsData(viewAndPostPermissions)
|
|
|
|
allChainIDs, err := m.tokenManager.GetAllChainIDs()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
accountsAndChainIDs := combineAddressesAndChainIDs(addresses, allChainIDs)
|
|
|
|
response, err := m.checkChannelPermissions(viewOnlyPreParsedPermissions, viewAndPostPreParsedPermissions, accountsAndChainIDs, false)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.persistence.SaveCheckChannelPermissionResponse(communityID.String(), chatID, response)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return response, nil
|
|
}
|
|
|
|
func (m *Manager) checkChannelPermissionsLight(community *Community, communityChat *protobuf.CommunityChat, channelID string) *CheckChannelPermissionsResponse {
|
|
|
|
viewOnlyPermissions := community.ChannelTokenPermissionsByType(community.IDString()+channelID, protobuf.CommunityTokenPermission_CAN_VIEW_CHANNEL)
|
|
viewAndPostPermissions := community.ChannelTokenPermissionsByType(community.IDString()+channelID, protobuf.CommunityTokenPermission_CAN_VIEW_AND_POST_CHANNEL)
|
|
|
|
hasViewOnlyPermissions := len(viewOnlyPermissions) > 0
|
|
hasViewAndPostPermissions := len(viewAndPostPermissions) > 0
|
|
|
|
meAsMember := communityChat.Members[common.PubkeyToHex(&m.identity.PublicKey)]
|
|
|
|
viewSatisfied := !hasViewOnlyPermissions || (meAsMember != nil && meAsMember.GetChannelRole() == protobuf.CommunityMember_CHANNEL_ROLE_VIEWER)
|
|
postSatisfied := !hasViewAndPostPermissions || (meAsMember != nil && meAsMember.GetChannelRole() == protobuf.CommunityMember_CHANNEL_ROLE_POSTER)
|
|
|
|
finalViewSatisfied := m.computeViewOnlySatisfied(hasViewOnlyPermissions, hasViewAndPostPermissions, viewSatisfied, postSatisfied)
|
|
finalPostSatisfied := m.computeViewAndPostSatisfied(hasViewOnlyPermissions, hasViewAndPostPermissions, postSatisfied)
|
|
|
|
return &CheckChannelPermissionsResponse{
|
|
ViewOnlyPermissions: &CheckChannelViewOnlyPermissionsResult{
|
|
Satisfied: finalViewSatisfied,
|
|
Permissions: make(map[string]*PermissionTokenCriteriaResult),
|
|
},
|
|
ViewAndPostPermissions: &CheckChannelViewAndPostPermissionsResult{
|
|
Satisfied: finalPostSatisfied,
|
|
Permissions: make(map[string]*PermissionTokenCriteriaResult),
|
|
},
|
|
}
|
|
}
|
|
|
|
// Light version of CheckChannelPermissions, which does not use network requests.
|
|
// Instead of checking wallet balances it checks if there is an access to a member list of the chat.
|
|
func (m *Manager) CheckChannelPermissionsLight(communityID types.HexBytes, chatID string) (*CheckChannelPermissionsResponse, error) {
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// Remove communityID prefix from chatID if exists
|
|
if strings.HasPrefix(chatID, communityID.String()) {
|
|
chatID = strings.TrimPrefix(chatID, communityID.String())
|
|
}
|
|
|
|
if chatID == "" {
|
|
return nil, errors.New(fmt.Sprintf("couldn't check channel permissions, invalid chat id: %s", chatID))
|
|
}
|
|
|
|
communityChat, err := community.GetChat(chatID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return m.checkChannelPermissionsLight(community, communityChat, chatID), nil
|
|
}
|
|
|
|
type CheckChannelPermissionsResponse struct {
|
|
ViewOnlyPermissions *CheckChannelViewOnlyPermissionsResult `json:"viewOnlyPermissions"`
|
|
ViewAndPostPermissions *CheckChannelViewAndPostPermissionsResult `json:"viewAndPostPermissions"`
|
|
}
|
|
|
|
type CheckChannelViewOnlyPermissionsResult struct {
|
|
Satisfied bool `json:"satisfied"`
|
|
Permissions map[string]*PermissionTokenCriteriaResult `json:"permissions"`
|
|
}
|
|
|
|
type CheckChannelViewAndPostPermissionsResult struct {
|
|
Satisfied bool `json:"satisfied"`
|
|
Permissions map[string]*PermissionTokenCriteriaResult `json:"permissions"`
|
|
}
|
|
|
|
func (m *Manager) computeViewOnlySatisfied(hasViewOnlyPermissions bool, hasViewAndPostPermissions bool, checkedViewOnlySatisfied bool, checkedViewAndPostSatisified bool) bool {
|
|
if (hasViewAndPostPermissions && !hasViewOnlyPermissions) || (hasViewOnlyPermissions && hasViewAndPostPermissions && checkedViewAndPostSatisified) {
|
|
return checkedViewAndPostSatisified
|
|
} else {
|
|
return checkedViewOnlySatisfied
|
|
}
|
|
}
|
|
|
|
func (m *Manager) computeViewAndPostSatisfied(hasViewOnlyPermissions bool, hasViewAndPostPermissions bool, checkedViewAndPostSatisified bool) bool {
|
|
if hasViewOnlyPermissions && !hasViewAndPostPermissions {
|
|
return false
|
|
} else {
|
|
return checkedViewAndPostSatisified
|
|
}
|
|
}
|
|
|
|
func (m *Manager) checkChannelPermissions(viewOnlyPreParsedPermissions *PreParsedCommunityPermissionsData, viewAndPostPreParsedPermissions *PreParsedCommunityPermissionsData, accountsAndChainIDs []*AccountChainIDsCombination, shortcircuit bool) (*CheckChannelPermissionsResponse, error) {
|
|
|
|
response := &CheckChannelPermissionsResponse{
|
|
ViewOnlyPermissions: &CheckChannelViewOnlyPermissionsResult{
|
|
Satisfied: false,
|
|
Permissions: make(map[string]*PermissionTokenCriteriaResult),
|
|
},
|
|
ViewAndPostPermissions: &CheckChannelViewAndPostPermissionsResult{
|
|
Satisfied: false,
|
|
Permissions: make(map[string]*PermissionTokenCriteriaResult),
|
|
},
|
|
}
|
|
|
|
viewOnlyPermissionsResponse, err := m.PermissionChecker.CheckPermissions(viewOnlyPreParsedPermissions, accountsAndChainIDs, shortcircuit)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
viewAndPostPermissionsResponse, err := m.PermissionChecker.CheckPermissions(viewAndPostPreParsedPermissions, accountsAndChainIDs, shortcircuit)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
hasViewOnlyPermissions := viewOnlyPreParsedPermissions != nil
|
|
hasViewAndPostPermissions := viewAndPostPreParsedPermissions != nil
|
|
|
|
response.ViewOnlyPermissions.Satisfied = m.computeViewOnlySatisfied(hasViewOnlyPermissions, hasViewAndPostPermissions, viewOnlyPermissionsResponse.Satisfied, viewAndPostPermissionsResponse.Satisfied)
|
|
response.ViewOnlyPermissions.Permissions = viewOnlyPermissionsResponse.Permissions
|
|
|
|
response.ViewAndPostPermissions.Satisfied = m.computeViewAndPostSatisfied(hasViewOnlyPermissions, hasViewAndPostPermissions, viewAndPostPermissionsResponse.Satisfied)
|
|
response.ViewAndPostPermissions.Permissions = viewAndPostPermissionsResponse.Permissions
|
|
|
|
return response, nil
|
|
}
|
|
|
|
func (m *Manager) CheckAllChannelsPermissions(communityID types.HexBytes, addresses []gethcommon.Address) (*CheckAllChannelsPermissionsResponse, error) {
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
channels := community.Chats()
|
|
|
|
allChainIDs, err := m.tokenManager.GetAllChainIDs()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
accountsAndChainIDs := combineAddressesAndChainIDs(addresses, allChainIDs)
|
|
|
|
response := &CheckAllChannelsPermissionsResponse{
|
|
Channels: make(map[string]*CheckChannelPermissionsResponse),
|
|
}
|
|
|
|
// TODO: optimize
|
|
for channelID := range channels {
|
|
viewOnlyPermissions := community.ChannelTokenPermissionsByType(community.IDString()+channelID, protobuf.CommunityTokenPermission_CAN_VIEW_CHANNEL)
|
|
viewAndPostPermissions := community.ChannelTokenPermissionsByType(community.IDString()+channelID, protobuf.CommunityTokenPermission_CAN_VIEW_AND_POST_CHANNEL)
|
|
viewOnlyPreParsedPermissions := preParsedCommunityPermissionsData(viewOnlyPermissions)
|
|
viewAndPostPreParsedPermissions := preParsedCommunityPermissionsData(viewAndPostPermissions)
|
|
checkChannelPermissionsResponse, err := m.checkChannelPermissions(viewOnlyPreParsedPermissions, viewAndPostPreParsedPermissions, accountsAndChainIDs, false)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
err = m.persistence.SaveCheckChannelPermissionResponse(community.IDString(), community.IDString()+channelID, checkChannelPermissionsResponse)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
response.Channels[community.IDString()+channelID] = checkChannelPermissionsResponse
|
|
}
|
|
return response, nil
|
|
}
|
|
|
|
// Light version of CheckAllChannelsPermissionsLight, which does not use network requests.
|
|
// Instead of checking wallet balances it checks if there is an access to a member list of chats.
|
|
func (m *Manager) CheckAllChannelsPermissionsLight(communityID types.HexBytes) (*CheckAllChannelsPermissionsResponse, error) {
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
response := &CheckAllChannelsPermissionsResponse{
|
|
Channels: make(map[string]*CheckChannelPermissionsResponse),
|
|
}
|
|
|
|
channels := community.Chats()
|
|
|
|
for channelID, channel := range channels {
|
|
response.Channels[community.IDString()+channelID] = m.checkChannelPermissionsLight(community, channel, channelID)
|
|
}
|
|
return response, nil
|
|
}
|
|
|
|
func (m *Manager) GetCheckChannelPermissionResponses(communityID types.HexBytes) (*CheckAllChannelsPermissionsResponse, error) {
|
|
|
|
response, err := m.persistence.GetCheckChannelPermissionResponses(communityID.String())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return &CheckAllChannelsPermissionsResponse{Channels: response}, nil
|
|
}
|
|
|
|
type CheckAllChannelsPermissionsResponse struct {
|
|
Channels map[string]*CheckChannelPermissionsResponse `json:"channels"`
|
|
}
|
|
|
|
func (m *Manager) HandleCommunityRequestToJoinResponse(signer *ecdsa.PublicKey, request *protobuf.CommunityRequestToJoinResponse) (*RequestToJoin, error) {
|
|
m.communityLock.Lock(request.CommunityId)
|
|
defer m.communityLock.Unlock(request.CommunityId)
|
|
|
|
pkString := common.PubkeyToHex(&m.identity.PublicKey)
|
|
|
|
community, err := m.GetByID(request.CommunityId)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
communityDescriptionBytes, err := proto.Marshal(request.Community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// We need to wrap `request.Community` in an `ApplicationMetadataMessage`
|
|
// of type `CommunityDescription` because `UpdateCommunityDescription` expects this.
|
|
//
|
|
// This is merely for marsheling/unmarsheling, hence we attaching a `Signature`
|
|
// is not needed.
|
|
metadataMessage := &protobuf.ApplicationMetadataMessage{
|
|
Payload: communityDescriptionBytes,
|
|
Type: protobuf.ApplicationMetadataMessage_COMMUNITY_DESCRIPTION,
|
|
}
|
|
|
|
appMetadataMsg, err := proto.Marshal(metadataMessage)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
isControlNodeSigner := common.IsPubKeyEqual(community.ControlNode(), signer)
|
|
if !isControlNodeSigner {
|
|
return nil, ErrNotAuthorized
|
|
}
|
|
|
|
_, processedDescription, err := m.preprocessDescription(community.ID(), request.Community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
_, err = community.UpdateCommunityDescription(processedDescription, appMetadataMsg, nil)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if err = m.handleCommunityTokensMetadata(community); err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if community.Encrypted() && len(request.Grant) > 0 {
|
|
_, err = m.HandleCommunityGrant(community, request.Grant, request.Clock)
|
|
if err != nil && err != ErrGrantOlder && err != ErrGrantExpired {
|
|
m.logger.Error("Error handling a community grant", zap.Error(err))
|
|
}
|
|
}
|
|
|
|
err = m.persistence.SaveCommunity(community)
|
|
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if request.Accepted {
|
|
err = m.markRequestToJoinAsAccepted(&m.identity.PublicKey, community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
} else {
|
|
|
|
err = m.persistence.SetRequestToJoinState(pkString, community.ID(), RequestToJoinStateDeclined)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
return m.persistence.GetRequestToJoinByPkAndCommunityID(pkString, community.ID())
|
|
}
|
|
|
|
func (m *Manager) HandleCommunityRequestToLeave(signer *ecdsa.PublicKey, proto *protobuf.CommunityRequestToLeave) error {
|
|
requestToLeave := NewRequestToLeave(common.PubkeyToHex(signer), proto)
|
|
if err := m.persistence.SaveRequestToLeave(requestToLeave); err != nil {
|
|
return err
|
|
}
|
|
|
|
// Ensure corresponding requestToJoin clock is older than requestToLeave
|
|
requestToJoin, err := m.persistence.GetRequestToJoin(requestToLeave.ID)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
if requestToJoin.Clock > requestToLeave.Clock {
|
|
return ErrOldRequestToLeave
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func UnwrapCommunityDescriptionMessage(payload []byte) (*ecdsa.PublicKey, *protobuf.CommunityDescription, error) {
|
|
|
|
applicationMetadataMessage := &protobuf.ApplicationMetadataMessage{}
|
|
err := proto.Unmarshal(payload, applicationMetadataMessage)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
if applicationMetadataMessage.Type != protobuf.ApplicationMetadataMessage_COMMUNITY_DESCRIPTION {
|
|
return nil, nil, ErrInvalidMessage
|
|
}
|
|
signer, err := utils.RecoverKey(applicationMetadataMessage)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
description := &protobuf.CommunityDescription{}
|
|
|
|
err = proto.Unmarshal(applicationMetadataMessage.Payload, description)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return signer, description, nil
|
|
}
|
|
|
|
func (m *Manager) JoinCommunity(id types.HexBytes, forceJoin bool) (*Community, error) {
|
|
m.communityLock.Lock(id)
|
|
defer m.communityLock.Unlock(id)
|
|
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
if !forceJoin && community.Joined() {
|
|
// Nothing to do, we are already joined
|
|
return community, ErrOrgAlreadyJoined
|
|
}
|
|
community.Join()
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) SpectateCommunity(id types.HexBytes) (*Community, error) {
|
|
m.communityLock.Lock(id)
|
|
defer m.communityLock.Unlock(id)
|
|
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
community.Spectate()
|
|
if err = m.persistence.SaveCommunity(community); err != nil {
|
|
return nil, err
|
|
}
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) GetMagnetlinkMessageClock(communityID types.HexBytes) (uint64, error) {
|
|
return m.persistence.GetMagnetlinkMessageClock(communityID)
|
|
}
|
|
|
|
func (m *Manager) GetRequestToJoinIDByPkAndCommunityID(pk *ecdsa.PublicKey, communityID []byte) ([]byte, error) {
|
|
return m.persistence.GetRequestToJoinIDByPkAndCommunityID(common.PubkeyToHex(pk), communityID)
|
|
}
|
|
|
|
func (m *Manager) GetCommunityRequestToJoinClock(pk *ecdsa.PublicKey, communityID string) (uint64, error) {
|
|
request, err := m.persistence.GetRequestToJoinByPkAndCommunityID(common.PubkeyToHex(pk), []byte(communityID))
|
|
if errors.Is(err, sql.ErrNoRows) {
|
|
return 0, nil
|
|
} else if err != nil {
|
|
return 0, err
|
|
}
|
|
|
|
if request == nil || request.State != RequestToJoinStateAccepted {
|
|
return 0, nil
|
|
}
|
|
return request.Clock, nil
|
|
}
|
|
|
|
func (m *Manager) GetRequestToJoinByPkAndCommunityID(pk *ecdsa.PublicKey, communityID []byte) (*RequestToJoin, error) {
|
|
return m.persistence.GetRequestToJoinByPkAndCommunityID(common.PubkeyToHex(pk), communityID)
|
|
}
|
|
|
|
func (m *Manager) UpdateCommunityDescriptionMagnetlinkMessageClock(communityID types.HexBytes, clock uint64) error {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByIDString(communityID.String())
|
|
if err != nil {
|
|
return err
|
|
}
|
|
community.config.CommunityDescription.ArchiveMagnetlinkClock = clock
|
|
return m.persistence.SaveCommunity(community)
|
|
}
|
|
|
|
func (m *Manager) UpdateMagnetlinkMessageClock(communityID types.HexBytes, clock uint64) error {
|
|
return m.persistence.UpdateMagnetlinkMessageClock(communityID, clock)
|
|
}
|
|
|
|
func (m *Manager) UpdateLastSeenMagnetlink(communityID types.HexBytes, magnetlinkURI string) error {
|
|
return m.persistence.UpdateLastSeenMagnetlink(communityID, magnetlinkURI)
|
|
}
|
|
|
|
func (m *Manager) GetLastSeenMagnetlink(communityID types.HexBytes) (string, error) {
|
|
return m.persistence.GetLastSeenMagnetlink(communityID)
|
|
}
|
|
|
|
func (m *Manager) LeaveCommunity(id types.HexBytes) (*Community, error) {
|
|
m.communityLock.Lock(id)
|
|
defer m.communityLock.Unlock(id)
|
|
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
community.RemoveOurselvesFromOrg(&m.identity.PublicKey)
|
|
community.Leave()
|
|
|
|
if err = m.persistence.SaveCommunity(community); err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
// Same as LeaveCommunity, but we have an option to stay spectating
|
|
func (m *Manager) KickedOutOfCommunity(id types.HexBytes, spectateMode bool) (*Community, error) {
|
|
m.communityLock.Lock(id)
|
|
defer m.communityLock.Unlock(id)
|
|
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
community.RemoveOurselvesFromOrg(&m.identity.PublicKey)
|
|
community.Leave()
|
|
if spectateMode {
|
|
community.Spectate()
|
|
}
|
|
|
|
if err = m.persistence.SaveCommunity(community); err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) AddMemberOwnerToCommunity(communityID types.HexBytes, pk *ecdsa.PublicKey) (*Community, error) {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
_, err = community.AddMember(pk, []protobuf.CommunityMember_Roles{protobuf.CommunityMember_ROLE_OWNER})
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
m.publish(&Subscription{Community: community})
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) RemoveUserFromCommunity(id types.HexBytes, pk *ecdsa.PublicKey) (*Community, error) {
|
|
m.communityLock.Lock(id)
|
|
defer m.communityLock.Unlock(id)
|
|
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
_, err = community.RemoveUserFromOrg(pk)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) UnbanUserFromCommunity(request *requests.UnbanUserFromCommunity) (*Community, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
id := request.CommunityID
|
|
publicKey, err := common.HexToPubkey(request.User.String())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
_, err = community.UnbanUserFromCommunity(publicKey)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) AddRoleToMember(request *requests.AddRoleToMember) (*Community, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
id := request.CommunityID
|
|
publicKey, err := common.HexToPubkey(request.User.String())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if !community.hasMember(publicKey) {
|
|
return nil, ErrMemberNotFound
|
|
}
|
|
|
|
_, err = community.AddRoleToMember(publicKey, request.Role)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
m.publish(&Subscription{Community: community})
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) RemoveRoleFromMember(request *requests.RemoveRoleFromMember) (*Community, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
id := request.CommunityID
|
|
publicKey, err := common.HexToPubkey(request.User.String())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if !community.hasMember(publicKey) {
|
|
return nil, ErrMemberNotFound
|
|
}
|
|
|
|
_, err = community.RemoveRoleFromMember(publicKey, request.Role)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
m.publish(&Subscription{Community: community})
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) BanUserFromCommunity(request *requests.BanUserFromCommunity) (*Community, error) {
|
|
m.communityLock.Lock(request.CommunityID)
|
|
defer m.communityLock.Unlock(request.CommunityID)
|
|
|
|
id := request.CommunityID
|
|
|
|
publicKey, err := common.HexToPubkey(request.User.String())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
_, err = community.BanUserFromCommunity(publicKey, &protobuf.CommunityBanInfo{DeleteAllMessages: request.DeleteAllMessages})
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = m.saveAndPublish(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) dbRecordBundleToCommunity(r *CommunityRecordBundle) (*Community, error) {
|
|
var descriptionEncryptor DescriptionEncryptor
|
|
if m.encryptor != nil {
|
|
descriptionEncryptor = m
|
|
}
|
|
|
|
return recordBundleToCommunity(r, &m.identity.PublicKey, m.installationID, m.logger, m.timesource, descriptionEncryptor, func(community *Community) error {
|
|
_, description, err := m.preprocessDescription(community.ID(), community.config.CommunityDescription)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
community.config.CommunityDescription = description
|
|
|
|
if community.config.EventsData != nil {
|
|
eventsDescription, err := validateAndGetEventsMessageCommunityDescription(community.config.EventsData.EventsBaseCommunityDescription, community.ControlNode())
|
|
if err != nil {
|
|
m.logger.Error("invalid EventsBaseCommunityDescription", zap.Error(err))
|
|
}
|
|
if eventsDescription != nil && eventsDescription.Clock == community.Clock() {
|
|
community.applyEvents()
|
|
}
|
|
}
|
|
|
|
if m.transport != nil && m.transport.WakuVersion() == 2 {
|
|
topic := community.PubsubTopic()
|
|
privKey, err := m.transport.RetrievePubsubTopicKey(topic)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
community.config.PubsubTopicPrivateKey = privKey
|
|
}
|
|
|
|
return nil
|
|
})
|
|
}
|
|
|
|
func (m *Manager) GetByID(id []byte) (*Community, error) {
|
|
community, err := m.persistence.GetByID(&m.identity.PublicKey, id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
if community == nil {
|
|
return nil, ErrOrgNotFound
|
|
}
|
|
return community, nil
|
|
}
|
|
|
|
func (m *Manager) GetByIDString(idString string) (*Community, error) {
|
|
id, err := types.DecodeHex(idString)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return m.GetByID(id)
|
|
}
|
|
|
|
func (m *Manager) GetCommunityShard(communityID types.HexBytes) (*shard.Shard, error) {
|
|
return m.persistence.GetCommunityShard(communityID)
|
|
}
|
|
|
|
func (m *Manager) SaveCommunityShard(communityID types.HexBytes, shard *shard.Shard, clock uint64) error {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
return m.persistence.SaveCommunityShard(communityID, shard, clock)
|
|
}
|
|
|
|
func (m *Manager) DeleteCommunityShard(communityID types.HexBytes) error {
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
return m.persistence.DeleteCommunityShard(communityID)
|
|
}
|
|
|
|
func (m *Manager) SaveRequestToJoinRevealedAddresses(requestID types.HexBytes, revealedAccounts []*protobuf.RevealedAccount) error {
|
|
return m.persistence.SaveRequestToJoinRevealedAddresses(requestID, revealedAccounts)
|
|
}
|
|
|
|
func (m *Manager) RemoveRequestToJoinRevealedAddresses(requestID types.HexBytes) error {
|
|
return m.persistence.RemoveRequestToJoinRevealedAddresses(requestID)
|
|
}
|
|
|
|
func (m *Manager) SaveRequestToJoinAndCommunity(requestToJoin *RequestToJoin, community *Community) (*Community, *RequestToJoin, error) {
|
|
if err := m.persistence.SaveRequestToJoin(requestToJoin); err != nil {
|
|
return nil, nil, err
|
|
}
|
|
community.config.RequestedToJoinAt = uint64(time.Now().Unix())
|
|
community.AddRequestToJoin(requestToJoin)
|
|
|
|
// Save revealed addresses to our own table so that we can retrieve them later when editing
|
|
if err := m.SaveRequestToJoinRevealedAddresses(requestToJoin.ID, requestToJoin.RevealedAccounts); err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, requestToJoin, nil
|
|
}
|
|
|
|
func (m *Manager) CreateRequestToJoin(request *requests.RequestToJoinCommunity, customizationColor multiaccountscommon.CustomizationColor) *RequestToJoin {
|
|
clock := uint64(time.Now().Unix())
|
|
requestToJoin := &RequestToJoin{
|
|
PublicKey: common.PubkeyToHex(&m.identity.PublicKey),
|
|
Clock: clock,
|
|
ENSName: request.ENSName,
|
|
CommunityID: request.CommunityID,
|
|
State: RequestToJoinStatePending,
|
|
Our: true,
|
|
RevealedAccounts: make([]*protobuf.RevealedAccount, 0),
|
|
CustomizationColor: customizationColor,
|
|
}
|
|
|
|
requestToJoin.CalculateID()
|
|
|
|
addSignature := len(request.Signatures) == len(request.AddressesToReveal)
|
|
for i := range request.AddressesToReveal {
|
|
revealedAcc := &protobuf.RevealedAccount{
|
|
Address: request.AddressesToReveal[i],
|
|
IsAirdropAddress: types.HexToAddress(request.AddressesToReveal[i]) == types.HexToAddress(request.AirdropAddress),
|
|
}
|
|
|
|
if addSignature {
|
|
revealedAcc.Signature = request.Signatures[i]
|
|
}
|
|
|
|
requestToJoin.RevealedAccounts = append(requestToJoin.RevealedAccounts, revealedAcc)
|
|
}
|
|
|
|
return requestToJoin
|
|
}
|
|
|
|
func (m *Manager) SaveRequestToJoin(request *RequestToJoin) error {
|
|
return m.persistence.SaveRequestToJoin(request)
|
|
}
|
|
|
|
func (m *Manager) CanceledRequestsToJoinForUser(pk *ecdsa.PublicKey) ([]*RequestToJoin, error) {
|
|
return m.persistence.CanceledRequestsToJoinForUser(common.PubkeyToHex(pk))
|
|
}
|
|
|
|
func (m *Manager) CanceledRequestToJoinForUserForCommunityID(pk *ecdsa.PublicKey, communityID []byte) (*RequestToJoin, error) {
|
|
return m.persistence.CanceledRequestToJoinForUserForCommunityID(common.PubkeyToHex(pk), communityID)
|
|
}
|
|
|
|
func (m *Manager) PendingRequestsToJoin() ([]*RequestToJoin, error) {
|
|
return m.persistence.PendingRequestsToJoin()
|
|
}
|
|
|
|
func (m *Manager) PendingRequestsToJoinForUser(pk *ecdsa.PublicKey) ([]*RequestToJoin, error) {
|
|
return m.persistence.RequestsToJoinForUserByState(common.PubkeyToHex(pk), RequestToJoinStatePending)
|
|
}
|
|
|
|
func (m *Manager) PendingRequestsToJoinForCommunity(id types.HexBytes) ([]*RequestToJoin, error) {
|
|
m.logger.Info("fetching pending invitations", zap.String("community-id", id.String()))
|
|
return m.persistence.PendingRequestsToJoinForCommunity(id)
|
|
}
|
|
|
|
func (m *Manager) DeclinedRequestsToJoinForCommunity(id types.HexBytes) ([]*RequestToJoin, error) {
|
|
m.logger.Info("fetching declined invitations", zap.String("community-id", id.String()))
|
|
return m.persistence.DeclinedRequestsToJoinForCommunity(id)
|
|
}
|
|
|
|
func (m *Manager) CanceledRequestsToJoinForCommunity(id types.HexBytes) ([]*RequestToJoin, error) {
|
|
m.logger.Info("fetching canceled invitations", zap.String("community-id", id.String()))
|
|
return m.persistence.CanceledRequestsToJoinForCommunity(id)
|
|
}
|
|
|
|
func (m *Manager) AcceptedRequestsToJoinForCommunity(id types.HexBytes) ([]*RequestToJoin, error) {
|
|
m.logger.Info("fetching canceled invitations", zap.String("community-id", id.String()))
|
|
return m.persistence.AcceptedRequestsToJoinForCommunity(id)
|
|
}
|
|
|
|
func (m *Manager) AcceptedPendingRequestsToJoinForCommunity(id types.HexBytes) ([]*RequestToJoin, error) {
|
|
return m.persistence.AcceptedPendingRequestsToJoinForCommunity(id)
|
|
}
|
|
|
|
func (m *Manager) DeclinedPendingRequestsToJoinForCommunity(id types.HexBytes) ([]*RequestToJoin, error) {
|
|
return m.persistence.DeclinedPendingRequestsToJoinForCommunity(id)
|
|
}
|
|
|
|
func (m *Manager) AllNonApprovedCommunitiesRequestsToJoin() ([]*RequestToJoin, error) {
|
|
m.logger.Info("fetching all non-approved invitations for all communities")
|
|
return m.persistence.AllNonApprovedCommunitiesRequestsToJoin()
|
|
}
|
|
|
|
func (m *Manager) RequestsToJoinForCommunityAwaitingAddresses(id types.HexBytes) ([]*RequestToJoin, error) {
|
|
m.logger.Info("fetching ownership changed invitations", zap.String("community-id", id.String()))
|
|
return m.persistence.RequestsToJoinForCommunityAwaitingAddresses(id)
|
|
}
|
|
|
|
func (m *Manager) CanPost(pk *ecdsa.PublicKey, communityID string, chatID string, messageType protobuf.ApplicationMetadataMessage_Type) (bool, error) {
|
|
community, err := m.GetByIDString(communityID)
|
|
if err != nil {
|
|
return false, err
|
|
}
|
|
return community.CanPost(pk, chatID, messageType)
|
|
}
|
|
|
|
func (m *Manager) IsEncrypted(communityID string) (bool, error) {
|
|
community, err := m.GetByIDString(communityID)
|
|
if err != nil {
|
|
return false, err
|
|
}
|
|
|
|
return community.Encrypted(), nil
|
|
}
|
|
|
|
func (m *Manager) IsChannelEncrypted(communityID string, chatID string) (bool, error) {
|
|
community, err := m.GetByIDString(communityID)
|
|
if err != nil {
|
|
return false, err
|
|
}
|
|
|
|
channelID := strings.TrimPrefix(chatID, communityID)
|
|
return community.ChannelEncrypted(channelID), nil
|
|
}
|
|
|
|
func (m *Manager) ShouldHandleSyncCommunity(community *protobuf.SyncInstallationCommunity) (bool, error) {
|
|
return m.persistence.ShouldHandleSyncCommunity(community)
|
|
}
|
|
|
|
func (m *Manager) ShouldHandleSyncCommunitySettings(communitySettings *protobuf.SyncCommunitySettings) (bool, error) {
|
|
return m.persistence.ShouldHandleSyncCommunitySettings(communitySettings)
|
|
}
|
|
|
|
func (m *Manager) HandleSyncCommunitySettings(syncCommunitySettings *protobuf.SyncCommunitySettings) (*CommunitySettings, error) {
|
|
id, err := types.DecodeHex(syncCommunitySettings.CommunityId)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
settings, err := m.persistence.GetCommunitySettingsByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if settings == nil {
|
|
settings = &CommunitySettings{
|
|
CommunityID: syncCommunitySettings.CommunityId,
|
|
HistoryArchiveSupportEnabled: syncCommunitySettings.HistoryArchiveSupportEnabled,
|
|
Clock: syncCommunitySettings.Clock,
|
|
}
|
|
}
|
|
|
|
if syncCommunitySettings.Clock > settings.Clock {
|
|
settings.CommunityID = syncCommunitySettings.CommunityId
|
|
settings.HistoryArchiveSupportEnabled = syncCommunitySettings.HistoryArchiveSupportEnabled
|
|
settings.Clock = syncCommunitySettings.Clock
|
|
}
|
|
|
|
err = m.persistence.SaveCommunitySettings(*settings)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return settings, nil
|
|
}
|
|
|
|
func (m *Manager) SetSyncClock(id []byte, clock uint64) error {
|
|
return m.persistence.SetSyncClock(id, clock)
|
|
}
|
|
|
|
func (m *Manager) SetPrivateKey(id []byte, privKey *ecdsa.PrivateKey) error {
|
|
return m.persistence.SetPrivateKey(id, privKey)
|
|
}
|
|
|
|
func (m *Manager) GetSyncedRawCommunity(id []byte) (*RawCommunityRow, error) {
|
|
return m.persistence.getSyncedRawCommunity(id)
|
|
}
|
|
|
|
func (m *Manager) GetCommunitySettingsByID(id types.HexBytes) (*CommunitySettings, error) {
|
|
return m.persistence.GetCommunitySettingsByID(id)
|
|
}
|
|
|
|
func (m *Manager) GetCommunitiesSettings() ([]CommunitySettings, error) {
|
|
return m.persistence.GetCommunitiesSettings()
|
|
}
|
|
|
|
func (m *Manager) SaveCommunitySettings(settings CommunitySettings) error {
|
|
return m.persistence.SaveCommunitySettings(settings)
|
|
}
|
|
|
|
func (m *Manager) CommunitySettingsExist(id types.HexBytes) (bool, error) {
|
|
return m.persistence.CommunitySettingsExist(id)
|
|
}
|
|
|
|
func (m *Manager) DeleteCommunitySettings(id types.HexBytes) error {
|
|
return m.persistence.DeleteCommunitySettings(id)
|
|
}
|
|
|
|
func (m *Manager) UpdateCommunitySettings(settings CommunitySettings) error {
|
|
return m.persistence.UpdateCommunitySettings(settings)
|
|
}
|
|
|
|
func (m *Manager) GetOwnedCommunitiesChatIDs() (map[string]bool, error) {
|
|
ownedCommunities, err := m.Controlled()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
chatIDs := make(map[string]bool)
|
|
for _, c := range ownedCommunities {
|
|
if c.Joined() {
|
|
for _, id := range c.ChatIDs() {
|
|
chatIDs[id] = true
|
|
}
|
|
}
|
|
}
|
|
return chatIDs, nil
|
|
}
|
|
|
|
func (m *Manager) GetCommunityChatsFilters(communityID types.HexBytes) ([]*transport.Filter, error) {
|
|
chatIDs, err := m.persistence.GetCommunityChatIDs(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
filters := []*transport.Filter{}
|
|
for _, cid := range chatIDs {
|
|
filters = append(filters, m.transport.FilterByChatID(cid))
|
|
}
|
|
return filters, nil
|
|
}
|
|
|
|
func (m *Manager) GetCommunityChatsTopics(communityID types.HexBytes) ([]types.TopicType, error) {
|
|
filters, err := m.GetCommunityChatsFilters(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
topics := []types.TopicType{}
|
|
for _, filter := range filters {
|
|
topics = append(topics, filter.ContentTopic)
|
|
}
|
|
|
|
return topics, nil
|
|
}
|
|
|
|
func (m *Manager) StoreWakuMessage(message *types.Message) error {
|
|
return m.persistence.SaveWakuMessage(message)
|
|
}
|
|
|
|
func (m *Manager) StoreWakuMessages(messages []*types.Message) error {
|
|
return m.persistence.SaveWakuMessages(messages)
|
|
}
|
|
|
|
func (m *Manager) GetLatestWakuMessageTimestamp(topics []types.TopicType) (uint64, error) {
|
|
return m.persistence.GetLatestWakuMessageTimestamp(topics)
|
|
}
|
|
|
|
func (m *Manager) GetOldestWakuMessageTimestamp(topics []types.TopicType) (uint64, error) {
|
|
return m.persistence.GetOldestWakuMessageTimestamp(topics)
|
|
}
|
|
|
|
func (m *Manager) GetLastMessageArchiveEndDate(communityID types.HexBytes) (uint64, error) {
|
|
return m.persistence.GetLastMessageArchiveEndDate(communityID)
|
|
}
|
|
|
|
func (m *Manager) GetHistoryArchivePartitionStartTimestamp(communityID types.HexBytes) (uint64, error) {
|
|
filters, err := m.GetCommunityChatsFilters(communityID)
|
|
if err != nil {
|
|
m.LogStdout("failed to get community chats filters", zap.Error(err))
|
|
return 0, err
|
|
}
|
|
|
|
if len(filters) == 0 {
|
|
// If we don't have chat filters, we likely don't have any chats
|
|
// associated to this community, which means there's nothing more
|
|
// to do here
|
|
return 0, nil
|
|
}
|
|
|
|
topics := []types.TopicType{}
|
|
|
|
for _, filter := range filters {
|
|
topics = append(topics, filter.ContentTopic)
|
|
}
|
|
|
|
lastArchiveEndDateTimestamp, err := m.GetLastMessageArchiveEndDate(communityID)
|
|
if err != nil {
|
|
m.LogStdout("failed to get last archive end date", zap.Error(err))
|
|
return 0, err
|
|
}
|
|
|
|
if lastArchiveEndDateTimestamp == 0 {
|
|
// If we don't have a tracked last message archive end date, it
|
|
// means we haven't created an archive before, which means
|
|
// the next thing to look at is the oldest waku message timestamp for
|
|
// this community
|
|
lastArchiveEndDateTimestamp, err = m.GetOldestWakuMessageTimestamp(topics)
|
|
if err != nil {
|
|
m.LogStdout("failed to get oldest waku message timestamp", zap.Error(err))
|
|
return 0, err
|
|
}
|
|
if lastArchiveEndDateTimestamp == 0 {
|
|
// This means there's no waku message stored for this community so far
|
|
// (even after requesting possibly missed messages), so no messages exist yet that can be archived
|
|
m.LogStdout("can't find valid `lastArchiveEndTimestamp`")
|
|
return 0, nil
|
|
}
|
|
}
|
|
|
|
return lastArchiveEndDateTimestamp, nil
|
|
}
|
|
|
|
func (m *Manager) CreateAndSeedHistoryArchive(communityID types.HexBytes, topics []types.TopicType, startDate time.Time, endDate time.Time, partition time.Duration, encrypt bool) error {
|
|
m.UnseedHistoryArchiveTorrent(communityID)
|
|
_, err := m.CreateHistoryArchiveTorrentFromDB(communityID, topics, startDate, endDate, partition, encrypt)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
return m.SeedHistoryArchiveTorrent(communityID)
|
|
}
|
|
|
|
func (m *Manager) StartHistoryArchiveTasksInterval(community *Community, interval time.Duration) {
|
|
id := community.IDString()
|
|
if _, exists := m.historyArchiveTasks.Load(id); exists {
|
|
m.LogStdout("history archive tasks interval already in progres", zap.String("id", id))
|
|
return
|
|
}
|
|
|
|
cancel := make(chan struct{})
|
|
m.historyArchiveTasks.Store(id, cancel)
|
|
m.historyArchiveTasksWaitGroup.Add(1)
|
|
|
|
ticker := time.NewTicker(interval)
|
|
defer ticker.Stop()
|
|
|
|
m.LogStdout("starting history archive tasks interval", zap.String("id", id))
|
|
for {
|
|
select {
|
|
case <-ticker.C:
|
|
m.LogStdout("starting archive task...", zap.String("id", id))
|
|
lastArchiveEndDateTimestamp, err := m.GetHistoryArchivePartitionStartTimestamp(community.ID())
|
|
if err != nil {
|
|
m.LogStdout("failed to get last archive end date", zap.Error(err))
|
|
continue
|
|
}
|
|
|
|
if lastArchiveEndDateTimestamp == 0 {
|
|
// This means there are no waku messages for this community,
|
|
// so nothing to do here
|
|
m.LogStdout("couldn't determine archive start date - skipping")
|
|
continue
|
|
}
|
|
|
|
topics, err := m.GetCommunityChatsTopics(community.ID())
|
|
if err != nil {
|
|
m.LogStdout("failed to get community chat topics ", zap.Error(err))
|
|
continue
|
|
}
|
|
|
|
ts := time.Now().Unix()
|
|
to := time.Unix(ts, 0)
|
|
lastArchiveEndDate := time.Unix(int64(lastArchiveEndDateTimestamp), 0)
|
|
|
|
err = m.CreateAndSeedHistoryArchive(community.ID(), topics, lastArchiveEndDate, to, interval, community.Encrypted())
|
|
if err != nil {
|
|
m.LogStdout("failed to create and seed history archive", zap.Error(err))
|
|
continue
|
|
}
|
|
case <-cancel:
|
|
m.UnseedHistoryArchiveTorrent(community.ID())
|
|
m.historyArchiveTasks.Delete(id)
|
|
m.historyArchiveTasksWaitGroup.Done()
|
|
return
|
|
}
|
|
}
|
|
}
|
|
|
|
func (m *Manager) StopHistoryArchiveTasksIntervals() {
|
|
m.historyArchiveTasks.Range(func(_, task interface{}) bool {
|
|
close(task.(chan struct{})) // Need to cast to the chan
|
|
return true
|
|
})
|
|
// Stoping archive interval tasks is async, so we need
|
|
// to wait for all of them to be closed before we shutdown
|
|
// the torrent client
|
|
m.historyArchiveTasksWaitGroup.Wait()
|
|
}
|
|
|
|
func (m *Manager) StopHistoryArchiveTasksInterval(communityID types.HexBytes) {
|
|
task, exists := m.historyArchiveTasks.Load(communityID.String())
|
|
if exists {
|
|
m.logger.Info("Stopping history archive tasks interval", zap.Any("id", communityID.String()))
|
|
close(task.(chan struct{})) // Need to cast to the chan
|
|
}
|
|
}
|
|
|
|
type EncodedArchiveData struct {
|
|
padding int
|
|
bytes []byte
|
|
}
|
|
|
|
func (m *Manager) CreateHistoryArchiveTorrentFromMessages(communityID types.HexBytes, messages []*types.Message, topics []types.TopicType, startDate time.Time, endDate time.Time, partition time.Duration, encrypt bool) ([]string, error) {
|
|
return m.CreateHistoryArchiveTorrent(communityID, messages, topics, startDate, endDate, partition, encrypt)
|
|
}
|
|
|
|
func (m *Manager) CreateHistoryArchiveTorrentFromDB(communityID types.HexBytes, topics []types.TopicType, startDate time.Time, endDate time.Time, partition time.Duration, encrypt bool) ([]string, error) {
|
|
|
|
return m.CreateHistoryArchiveTorrent(communityID, make([]*types.Message, 0), topics, startDate, endDate, partition, encrypt)
|
|
}
|
|
func (m *Manager) CreateHistoryArchiveTorrent(communityID types.HexBytes, msgs []*types.Message, topics []types.TopicType, startDate time.Time, endDate time.Time, partition time.Duration, encrypt bool) ([]string, error) {
|
|
|
|
loadFromDB := len(msgs) == 0
|
|
|
|
from := startDate
|
|
to := from.Add(partition)
|
|
if to.After(endDate) {
|
|
to = endDate
|
|
}
|
|
|
|
archiveDir := m.torrentConfig.DataDir + "/" + communityID.String()
|
|
torrentDir := m.torrentConfig.TorrentDir
|
|
indexPath := archiveDir + "/index"
|
|
dataPath := archiveDir + "/data"
|
|
|
|
wakuMessageArchiveIndexProto := &protobuf.WakuMessageArchiveIndex{}
|
|
wakuMessageArchiveIndex := make(map[string]*protobuf.WakuMessageArchiveIndexMetadata)
|
|
archiveIDs := make([]string, 0)
|
|
|
|
if _, err := os.Stat(archiveDir); os.IsNotExist(err) {
|
|
err := os.MkdirAll(archiveDir, 0700)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
}
|
|
if _, err := os.Stat(torrentDir); os.IsNotExist(err) {
|
|
err := os.MkdirAll(torrentDir, 0700)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
}
|
|
|
|
_, err := os.Stat(indexPath)
|
|
if err == nil {
|
|
wakuMessageArchiveIndexProto, err = m.LoadHistoryArchiveIndexFromFile(m.identity, communityID)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
}
|
|
|
|
var offset uint64 = 0
|
|
|
|
for hash, metadata := range wakuMessageArchiveIndexProto.Archives {
|
|
offset = offset + metadata.Size
|
|
wakuMessageArchiveIndex[hash] = metadata
|
|
}
|
|
|
|
var encodedArchives []*EncodedArchiveData
|
|
topicsAsByteArrays := topicsAsByteArrays(topics)
|
|
|
|
m.publish(&Subscription{CreatingHistoryArchivesSignal: &signal.CreatingHistoryArchivesSignal{
|
|
CommunityID: communityID.String(),
|
|
}})
|
|
|
|
m.LogStdout("creating archives",
|
|
zap.Any("startDate", startDate),
|
|
zap.Any("endDate", endDate),
|
|
zap.Duration("partition", partition),
|
|
)
|
|
for {
|
|
if from.Equal(endDate) || from.After(endDate) {
|
|
break
|
|
}
|
|
m.LogStdout("creating message archive",
|
|
zap.Any("from", from),
|
|
zap.Any("to", to),
|
|
)
|
|
|
|
var messages []types.Message
|
|
if loadFromDB {
|
|
messages, err = m.persistence.GetWakuMessagesByFilterTopic(topics, uint64(from.Unix()), uint64(to.Unix()))
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
} else {
|
|
for _, msg := range msgs {
|
|
if int64(msg.Timestamp) >= from.Unix() && int64(msg.Timestamp) < to.Unix() {
|
|
messages = append(messages, *msg)
|
|
}
|
|
}
|
|
|
|
}
|
|
|
|
if len(messages) == 0 {
|
|
// No need to create an archive with zero messages
|
|
m.LogStdout("no messages in this partition")
|
|
from = to
|
|
to = to.Add(partition)
|
|
if to.After(endDate) {
|
|
to = endDate
|
|
}
|
|
continue
|
|
}
|
|
|
|
// Not only do we partition messages, we also chunk them
|
|
// roughly by size, such that each chunk will not exceed a given
|
|
// size and archive data doesn't get too big
|
|
messageChunks := make([][]types.Message, 0)
|
|
currentChunkSize := 0
|
|
currentChunk := make([]types.Message, 0)
|
|
|
|
for _, msg := range messages {
|
|
msgSize := len(msg.Payload) + len(msg.Sig)
|
|
if msgSize > maxArchiveSizeInBytes {
|
|
// we drop messages this big
|
|
continue
|
|
}
|
|
|
|
if currentChunkSize+msgSize > maxArchiveSizeInBytes {
|
|
messageChunks = append(messageChunks, currentChunk)
|
|
currentChunk = make([]types.Message, 0)
|
|
currentChunkSize = 0
|
|
}
|
|
currentChunk = append(currentChunk, msg)
|
|
currentChunkSize = currentChunkSize + msgSize
|
|
}
|
|
messageChunks = append(messageChunks, currentChunk)
|
|
|
|
for _, messages := range messageChunks {
|
|
wakuMessageArchive := m.createWakuMessageArchive(from, to, messages, topicsAsByteArrays)
|
|
encodedArchive, err := proto.Marshal(wakuMessageArchive)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
|
|
if encrypt {
|
|
messageSpec, err := m.encryptor.BuildHashRatchetMessage(communityID, encodedArchive)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
|
|
encodedArchive, err = proto.Marshal(messageSpec.Message)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
}
|
|
|
|
rawSize := len(encodedArchive)
|
|
padding := 0
|
|
size := 0
|
|
|
|
if rawSize > pieceLength {
|
|
size = rawSize + pieceLength - (rawSize % pieceLength)
|
|
padding = size - rawSize
|
|
} else {
|
|
padding = pieceLength - rawSize
|
|
size = rawSize + padding
|
|
}
|
|
|
|
wakuMessageArchiveIndexMetadata := &protobuf.WakuMessageArchiveIndexMetadata{
|
|
Metadata: wakuMessageArchive.Metadata,
|
|
Offset: offset,
|
|
Size: uint64(size),
|
|
Padding: uint64(padding),
|
|
}
|
|
|
|
wakuMessageArchiveIndexMetadataBytes, err := proto.Marshal(wakuMessageArchiveIndexMetadata)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
|
|
archiveID := crypto.Keccak256Hash(wakuMessageArchiveIndexMetadataBytes).String()
|
|
archiveIDs = append(archiveIDs, archiveID)
|
|
wakuMessageArchiveIndex[archiveID] = wakuMessageArchiveIndexMetadata
|
|
encodedArchives = append(encodedArchives, &EncodedArchiveData{bytes: encodedArchive, padding: padding})
|
|
offset = offset + uint64(rawSize) + uint64(padding)
|
|
}
|
|
|
|
from = to
|
|
to = to.Add(partition)
|
|
if to.After(endDate) {
|
|
to = endDate
|
|
}
|
|
}
|
|
|
|
if len(encodedArchives) > 0 {
|
|
|
|
dataBytes := make([]byte, 0)
|
|
|
|
for _, encodedArchiveData := range encodedArchives {
|
|
dataBytes = append(dataBytes, encodedArchiveData.bytes...)
|
|
dataBytes = append(dataBytes, make([]byte, encodedArchiveData.padding)...)
|
|
}
|
|
|
|
wakuMessageArchiveIndexProto.Archives = wakuMessageArchiveIndex
|
|
indexBytes, err := proto.Marshal(wakuMessageArchiveIndexProto)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
|
|
if encrypt {
|
|
messageSpec, err := m.encryptor.BuildHashRatchetMessage(communityID, indexBytes)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
indexBytes, err = proto.Marshal(messageSpec.Message)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
}
|
|
|
|
err = os.WriteFile(indexPath, indexBytes, 0644) // nolint: gosec
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
|
|
file, err := os.OpenFile(dataPath, os.O_WRONLY|os.O_CREATE|os.O_APPEND, 0644)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
defer file.Close()
|
|
|
|
_, err = file.Write(dataBytes)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
|
|
metaInfo := metainfo.MetaInfo{
|
|
AnnounceList: defaultAnnounceList,
|
|
}
|
|
metaInfo.SetDefaults()
|
|
metaInfo.CreatedBy = common.PubkeyToHex(&m.identity.PublicKey)
|
|
|
|
info := metainfo.Info{
|
|
PieceLength: int64(pieceLength),
|
|
}
|
|
|
|
err = info.BuildFromFilePath(archiveDir)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
|
|
metaInfo.InfoBytes, err = bencode.Marshal(info)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
|
|
metaInfoBytes, err := bencode.Marshal(metaInfo)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
|
|
err = os.WriteFile(m.torrentFile(communityID.String()), metaInfoBytes, 0644) // nolint: gosec
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
|
|
m.LogStdout("torrent created", zap.Any("from", startDate.Unix()), zap.Any("to", endDate.Unix()))
|
|
|
|
m.publish(&Subscription{
|
|
HistoryArchivesCreatedSignal: &signal.HistoryArchivesCreatedSignal{
|
|
CommunityID: communityID.String(),
|
|
From: int(startDate.Unix()),
|
|
To: int(endDate.Unix()),
|
|
},
|
|
})
|
|
} else {
|
|
m.LogStdout("no archives created")
|
|
m.publish(&Subscription{
|
|
NoHistoryArchivesCreatedSignal: &signal.NoHistoryArchivesCreatedSignal{
|
|
CommunityID: communityID.String(),
|
|
From: int(startDate.Unix()),
|
|
To: int(endDate.Unix()),
|
|
},
|
|
})
|
|
}
|
|
|
|
lastMessageArchiveEndDate, err := m.persistence.GetLastMessageArchiveEndDate(communityID)
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
|
|
if lastMessageArchiveEndDate > 0 {
|
|
err = m.persistence.UpdateLastMessageArchiveEndDate(communityID, uint64(from.Unix()))
|
|
} else {
|
|
err = m.persistence.SaveLastMessageArchiveEndDate(communityID, uint64(from.Unix()))
|
|
}
|
|
if err != nil {
|
|
return archiveIDs, err
|
|
}
|
|
return archiveIDs, nil
|
|
}
|
|
|
|
func (m *Manager) SeedHistoryArchiveTorrent(communityID types.HexBytes) error {
|
|
m.UnseedHistoryArchiveTorrent(communityID)
|
|
|
|
id := communityID.String()
|
|
torrentFile := m.torrentFile(id)
|
|
|
|
metaInfo, err := metainfo.LoadFromFile(torrentFile)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
info, err := metaInfo.UnmarshalInfo()
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
hash := metaInfo.HashInfoBytes()
|
|
m.torrentTasks[id] = hash
|
|
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
torrent, err := m.torrentClient.AddTorrent(metaInfo)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
torrent.DownloadAll()
|
|
|
|
m.publish(&Subscription{
|
|
HistoryArchivesSeedingSignal: &signal.HistoryArchivesSeedingSignal{
|
|
CommunityID: communityID.String(),
|
|
},
|
|
})
|
|
|
|
magnetLink := metaInfo.Magnet(nil, &info).String()
|
|
|
|
m.LogStdout("seeding torrent", zap.String("id", id), zap.String("magnetLink", magnetLink))
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) UnseedHistoryArchiveTorrent(communityID types.HexBytes) {
|
|
id := communityID.String()
|
|
|
|
hash, exists := m.torrentTasks[id]
|
|
|
|
if exists {
|
|
torrent, ok := m.torrentClient.Torrent(hash)
|
|
if ok {
|
|
m.logger.Debug("Unseeding and dropping torrent for community: ", zap.Any("id", id))
|
|
torrent.Drop()
|
|
delete(m.torrentTasks, id)
|
|
|
|
m.publish(&Subscription{
|
|
HistoryArchivesUnseededSignal: &signal.HistoryArchivesUnseededSignal{
|
|
CommunityID: id,
|
|
},
|
|
})
|
|
}
|
|
}
|
|
}
|
|
|
|
func (m *Manager) IsSeedingHistoryArchiveTorrent(communityID types.HexBytes) bool {
|
|
id := communityID.String()
|
|
hash := m.torrentTasks[id]
|
|
torrent, ok := m.torrentClient.Torrent(hash)
|
|
return ok && torrent.Seeding()
|
|
}
|
|
|
|
func (m *Manager) GetHistoryArchiveDownloadTask(communityID string) *HistoryArchiveDownloadTask {
|
|
return m.historyArchiveDownloadTasks[communityID]
|
|
}
|
|
|
|
func (m *Manager) DeleteHistoryArchiveDownloadTask(communityID string) {
|
|
delete(m.historyArchiveDownloadTasks, communityID)
|
|
}
|
|
|
|
func (m *Manager) AddHistoryArchiveDownloadTask(communityID string, task *HistoryArchiveDownloadTask) {
|
|
m.historyArchiveDownloadTasks[communityID] = task
|
|
}
|
|
|
|
type HistoryArchiveDownloadTaskInfo struct {
|
|
TotalDownloadedArchivesCount int
|
|
TotalArchivesCount int
|
|
Cancelled bool
|
|
}
|
|
|
|
func (m *Manager) DownloadHistoryArchivesByMagnetlink(communityID types.HexBytes, magnetlink string, cancelTask chan struct{}) (*HistoryArchiveDownloadTaskInfo, error) {
|
|
|
|
id := communityID.String()
|
|
|
|
ml, err := metainfo.ParseMagnetUri(magnetlink)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
m.logger.Debug("adding torrent via magnetlink for community", zap.String("id", id), zap.String("magnetlink", magnetlink))
|
|
torrent, err := m.torrentClient.AddMagnet(magnetlink)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
downloadTaskInfo := &HistoryArchiveDownloadTaskInfo{
|
|
TotalDownloadedArchivesCount: 0,
|
|
TotalArchivesCount: 0,
|
|
Cancelled: false,
|
|
}
|
|
|
|
m.torrentTasks[id] = ml.InfoHash
|
|
timeout := time.After(20 * time.Second)
|
|
|
|
m.LogStdout("fetching torrent info", zap.String("magnetlink", magnetlink))
|
|
select {
|
|
case <-timeout:
|
|
return nil, ErrTorrentTimedout
|
|
case <-cancelTask:
|
|
m.LogStdout("cancelled fetching torrent info")
|
|
downloadTaskInfo.Cancelled = true
|
|
return downloadTaskInfo, nil
|
|
case <-torrent.GotInfo():
|
|
|
|
files := torrent.Files()
|
|
|
|
i, ok := findIndexFile(files)
|
|
if !ok {
|
|
// We're dealing with a malformed torrent, so don't do anything
|
|
return nil, errors.New("malformed torrent data")
|
|
}
|
|
|
|
indexFile := files[i]
|
|
indexFile.Download()
|
|
|
|
m.LogStdout("downloading history archive index")
|
|
ticker := time.NewTicker(100 * time.Millisecond)
|
|
defer ticker.Stop()
|
|
|
|
for {
|
|
select {
|
|
case <-cancelTask:
|
|
m.LogStdout("cancelled downloading archive index")
|
|
downloadTaskInfo.Cancelled = true
|
|
return downloadTaskInfo, nil
|
|
case <-ticker.C:
|
|
if indexFile.BytesCompleted() == indexFile.Length() {
|
|
|
|
index, err := m.LoadHistoryArchiveIndexFromFile(m.identity, communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
existingArchiveIDs, err := m.persistence.GetDownloadedMessageArchiveIDs(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if len(existingArchiveIDs) == len(index.Archives) {
|
|
m.LogStdout("download cancelled, no new archives")
|
|
return downloadTaskInfo, nil
|
|
}
|
|
|
|
downloadTaskInfo.TotalDownloadedArchivesCount = len(existingArchiveIDs)
|
|
downloadTaskInfo.TotalArchivesCount = len(index.Archives)
|
|
|
|
archiveHashes := make(archiveMDSlice, 0, downloadTaskInfo.TotalArchivesCount)
|
|
|
|
for hash, metadata := range index.Archives {
|
|
archiveHashes = append(archiveHashes, &archiveMetadata{hash: hash, from: metadata.Metadata.From})
|
|
}
|
|
|
|
sort.Sort(sort.Reverse(archiveHashes))
|
|
|
|
m.publish(&Subscription{
|
|
DownloadingHistoryArchivesStartedSignal: &signal.DownloadingHistoryArchivesStartedSignal{
|
|
CommunityID: communityID.String(),
|
|
},
|
|
})
|
|
|
|
for _, hd := range archiveHashes {
|
|
|
|
hash := hd.hash
|
|
hasArchive := false
|
|
|
|
for _, existingHash := range existingArchiveIDs {
|
|
if existingHash == hash {
|
|
hasArchive = true
|
|
break
|
|
}
|
|
}
|
|
if hasArchive {
|
|
continue
|
|
}
|
|
|
|
metadata := index.Archives[hash]
|
|
startIndex := int(metadata.Offset) / pieceLength
|
|
endIndex := startIndex + int(metadata.Size)/pieceLength
|
|
|
|
downloadMsg := fmt.Sprintf("downloading data for message archive (%d/%d)", downloadTaskInfo.TotalDownloadedArchivesCount+1, downloadTaskInfo.TotalArchivesCount)
|
|
m.LogStdout(downloadMsg, zap.String("hash", hash))
|
|
m.LogStdout("pieces (start, end)", zap.Any("startIndex", startIndex), zap.Any("endIndex", endIndex-1))
|
|
torrent.DownloadPieces(startIndex, endIndex)
|
|
|
|
piecesCompleted := make(map[int]bool)
|
|
for i = startIndex; i < endIndex; i++ {
|
|
piecesCompleted[i] = false
|
|
}
|
|
|
|
psc := torrent.SubscribePieceStateChanges()
|
|
downloadTicker := time.NewTicker(1 * time.Second)
|
|
defer downloadTicker.Stop()
|
|
|
|
downloadLoop:
|
|
for {
|
|
select {
|
|
case <-downloadTicker.C:
|
|
done := true
|
|
for i = startIndex; i < endIndex; i++ {
|
|
piecesCompleted[i] = torrent.PieceState(i).Complete
|
|
if !piecesCompleted[i] {
|
|
done = false
|
|
}
|
|
}
|
|
if done {
|
|
psc.Close()
|
|
break downloadLoop
|
|
}
|
|
case <-cancelTask:
|
|
m.LogStdout("downloading archive data interrupted")
|
|
downloadTaskInfo.Cancelled = true
|
|
return downloadTaskInfo, nil
|
|
}
|
|
}
|
|
downloadTaskInfo.TotalDownloadedArchivesCount++
|
|
err = m.persistence.SaveMessageArchiveID(communityID, hash)
|
|
if err != nil {
|
|
m.LogStdout("couldn't save message archive ID", zap.Error(err))
|
|
continue
|
|
}
|
|
m.publish(&Subscription{
|
|
HistoryArchiveDownloadedSignal: &signal.HistoryArchiveDownloadedSignal{
|
|
CommunityID: communityID.String(),
|
|
From: int(metadata.Metadata.From),
|
|
To: int(metadata.Metadata.To),
|
|
},
|
|
})
|
|
}
|
|
m.publish(&Subscription{
|
|
HistoryArchivesSeedingSignal: &signal.HistoryArchivesSeedingSignal{
|
|
CommunityID: communityID.String(),
|
|
},
|
|
})
|
|
m.LogStdout("finished downloading archives")
|
|
return downloadTaskInfo, nil
|
|
}
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
func (m *Manager) GetMessageArchiveIDsToImport(communityID types.HexBytes) ([]string, error) {
|
|
return m.persistence.GetMessageArchiveIDsToImport(communityID)
|
|
}
|
|
|
|
func (m *Manager) ExtractMessagesFromHistoryArchive(communityID types.HexBytes, archiveID string) ([]*protobuf.WakuMessage, error) {
|
|
id := communityID.String()
|
|
|
|
index, err := m.LoadHistoryArchiveIndexFromFile(m.identity, communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
dataFile, err := os.Open(m.archiveDataFile(id))
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
defer dataFile.Close()
|
|
|
|
m.LogStdout("extracting messages from history archive", zap.String("archive id", archiveID))
|
|
metadata := index.Archives[archiveID]
|
|
|
|
_, err = dataFile.Seek(int64(metadata.Offset), 0)
|
|
if err != nil {
|
|
m.LogStdout("failed to seek archive data file", zap.Error(err))
|
|
return nil, err
|
|
}
|
|
|
|
data := make([]byte, metadata.Size-metadata.Padding)
|
|
m.LogStdout("loading history archive data into memory", zap.Float64("data_size_MB", float64(metadata.Size-metadata.Padding)/1024.0/1024.0))
|
|
_, err = dataFile.Read(data)
|
|
if err != nil {
|
|
m.LogStdout("failed failed to read archive data", zap.Error(err))
|
|
return nil, err
|
|
}
|
|
|
|
archive := &protobuf.WakuMessageArchive{}
|
|
|
|
err = proto.Unmarshal(data, archive)
|
|
if err != nil {
|
|
// The archive data might eb encrypted so we try to decrypt instead first
|
|
var protocolMessage encryption.ProtocolMessage
|
|
err := proto.Unmarshal(data, &protocolMessage)
|
|
if err != nil {
|
|
m.LogStdout("failed to unmarshal protocol message", zap.Error(err))
|
|
return nil, err
|
|
}
|
|
|
|
pk, err := crypto.DecompressPubkey(communityID)
|
|
if err != nil {
|
|
m.logger.Debug("failed to decompress community pubkey", zap.Error(err))
|
|
return nil, err
|
|
}
|
|
decryptedBytes, err := m.encryptor.HandleMessage(m.identity, pk, &protocolMessage, make([]byte, 0))
|
|
if err != nil {
|
|
m.LogStdout("failed to decrypt message archive", zap.Error(err))
|
|
return nil, err
|
|
}
|
|
err = proto.Unmarshal(decryptedBytes.DecryptedMessage, archive)
|
|
if err != nil {
|
|
m.LogStdout("failed to unmarshal message archive data", zap.Error(err))
|
|
return nil, err
|
|
}
|
|
}
|
|
return archive.Messages, nil
|
|
}
|
|
|
|
func (m *Manager) SetMessageArchiveIDImported(communityID types.HexBytes, hash string, imported bool) error {
|
|
return m.persistence.SetMessageArchiveIDImported(communityID, hash, imported)
|
|
}
|
|
|
|
func (m *Manager) GetHistoryArchiveMagnetlink(communityID types.HexBytes) (string, error) {
|
|
id := communityID.String()
|
|
torrentFile := m.torrentFile(id)
|
|
|
|
metaInfo, err := metainfo.LoadFromFile(torrentFile)
|
|
if err != nil {
|
|
return "", err
|
|
}
|
|
|
|
info, err := metaInfo.UnmarshalInfo()
|
|
if err != nil {
|
|
return "", err
|
|
}
|
|
|
|
return metaInfo.Magnet(nil, &info).String(), nil
|
|
}
|
|
|
|
func (m *Manager) createWakuMessageArchive(from time.Time, to time.Time, messages []types.Message, topics [][]byte) *protobuf.WakuMessageArchive {
|
|
var wakuMessages []*protobuf.WakuMessage
|
|
|
|
for _, msg := range messages {
|
|
topic := types.TopicTypeToByteArray(msg.Topic)
|
|
wakuMessage := &protobuf.WakuMessage{
|
|
Sig: msg.Sig,
|
|
Timestamp: uint64(msg.Timestamp),
|
|
Topic: topic,
|
|
Payload: msg.Payload,
|
|
Padding: msg.Padding,
|
|
Hash: msg.Hash,
|
|
ThirdPartyId: msg.ThirdPartyID,
|
|
}
|
|
wakuMessages = append(wakuMessages, wakuMessage)
|
|
}
|
|
|
|
metadata := protobuf.WakuMessageArchiveMetadata{
|
|
From: uint64(from.Unix()),
|
|
To: uint64(to.Unix()),
|
|
ContentTopic: topics,
|
|
}
|
|
|
|
wakuMessageArchive := &protobuf.WakuMessageArchive{
|
|
Metadata: &metadata,
|
|
Messages: wakuMessages,
|
|
}
|
|
return wakuMessageArchive
|
|
}
|
|
|
|
func (m *Manager) LoadHistoryArchiveIndexFromFile(myKey *ecdsa.PrivateKey, communityID types.HexBytes) (*protobuf.WakuMessageArchiveIndex, error) {
|
|
wakuMessageArchiveIndexProto := &protobuf.WakuMessageArchiveIndex{}
|
|
|
|
indexPath := m.archiveIndexFile(communityID.String())
|
|
indexData, err := os.ReadFile(indexPath)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
err = proto.Unmarshal(indexData, wakuMessageArchiveIndexProto)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if len(wakuMessageArchiveIndexProto.Archives) == 0 && len(indexData) > 0 {
|
|
// This means we're dealing with an encrypted index file, so we have to decrypt it first
|
|
var protocolMessage encryption.ProtocolMessage
|
|
err := proto.Unmarshal(indexData, &protocolMessage)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
pk, err := crypto.DecompressPubkey(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
decryptedBytes, err := m.encryptor.HandleMessage(myKey, pk, &protocolMessage, make([]byte, 0))
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
err = proto.Unmarshal(decryptedBytes.DecryptedMessage, wakuMessageArchiveIndexProto)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
|
|
return wakuMessageArchiveIndexProto, nil
|
|
}
|
|
|
|
func (m *Manager) TorrentFileExists(communityID string) bool {
|
|
_, err := os.Stat(m.torrentFile(communityID))
|
|
return err == nil
|
|
}
|
|
|
|
func (m *Manager) torrentFile(communityID string) string {
|
|
return m.torrentConfig.TorrentDir + "/" + communityID + ".torrent"
|
|
}
|
|
|
|
func (m *Manager) archiveIndexFile(communityID string) string {
|
|
return m.torrentConfig.DataDir + "/" + communityID + "/index"
|
|
}
|
|
|
|
func (m *Manager) archiveDataFile(communityID string) string {
|
|
return m.torrentConfig.DataDir + "/" + communityID + "/data"
|
|
}
|
|
|
|
func topicsAsByteArrays(topics []types.TopicType) [][]byte {
|
|
var topicsAsByteArrays [][]byte
|
|
for _, t := range topics {
|
|
topic := types.TopicTypeToByteArray(t)
|
|
topicsAsByteArrays = append(topicsAsByteArrays, topic)
|
|
}
|
|
return topicsAsByteArrays
|
|
}
|
|
|
|
func findIndexFile(files []*torrent.File) (index int, ok bool) {
|
|
for i, f := range files {
|
|
if f.DisplayPath() == "index" {
|
|
return i, true
|
|
}
|
|
}
|
|
return 0, false
|
|
}
|
|
|
|
func (m *Manager) GetCommunityToken(communityID string, chainID int, address string) (*community_token.CommunityToken, error) {
|
|
return m.persistence.GetCommunityToken(communityID, chainID, address)
|
|
}
|
|
|
|
func (m *Manager) GetCommunityTokenByChainAndAddress(chainID int, address string) (*community_token.CommunityToken, error) {
|
|
return m.persistence.GetCommunityTokenByChainAndAddress(chainID, address)
|
|
}
|
|
|
|
func (m *Manager) GetCommunityTokens(communityID string) ([]*community_token.CommunityToken, error) {
|
|
return m.persistence.GetCommunityTokens(communityID)
|
|
}
|
|
|
|
func (m *Manager) GetAllCommunityTokens() ([]*community_token.CommunityToken, error) {
|
|
return m.persistence.GetAllCommunityTokens()
|
|
}
|
|
|
|
func (m *Manager) GetCommunityGrant(communityID string) ([]byte, uint64, error) {
|
|
return m.persistence.GetCommunityGrant(communityID)
|
|
}
|
|
|
|
func (m *Manager) ImageToBase64(uri string) string {
|
|
if uri == "" {
|
|
return ""
|
|
}
|
|
file, err := os.Open(uri)
|
|
if err != nil {
|
|
m.logger.Error(err.Error())
|
|
return ""
|
|
}
|
|
defer file.Close()
|
|
|
|
payload, err := ioutil.ReadAll(file)
|
|
if err != nil {
|
|
m.logger.Error(err.Error())
|
|
return ""
|
|
}
|
|
base64img, err := images.GetPayloadDataURI(payload)
|
|
if err != nil {
|
|
m.logger.Error(err.Error())
|
|
return ""
|
|
}
|
|
return base64img
|
|
}
|
|
|
|
func (m *Manager) SaveCommunityToken(token *community_token.CommunityToken, croppedImage *images.CroppedImage) (*community_token.CommunityToken, error) {
|
|
|
|
_, err := m.GetByIDString(token.CommunityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if croppedImage != nil && croppedImage.ImagePath != "" {
|
|
bytes, err := images.OpenAndAdjustImage(*croppedImage, true)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
base64img, err := images.GetPayloadDataURI(bytes)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
token.Base64Image = base64img
|
|
} else if !images.IsPayloadDataURI(token.Base64Image) {
|
|
// if image is already base64 do not convert (owner and master tokens have already base64 image)
|
|
token.Base64Image = m.ImageToBase64(token.Base64Image)
|
|
}
|
|
|
|
return token, m.persistence.AddCommunityToken(token)
|
|
}
|
|
|
|
func (m *Manager) AddCommunityToken(token *community_token.CommunityToken, clock uint64) (*Community, error) {
|
|
if token == nil {
|
|
return nil, errors.New("Token is absent in database")
|
|
}
|
|
|
|
communityID, err := types.DecodeHex(token.CommunityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
m.communityLock.Lock(communityID)
|
|
defer m.communityLock.Unlock(communityID)
|
|
|
|
community, err := m.GetByID(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if !community.MemberCanManageToken(&m.identity.PublicKey, token) {
|
|
return nil, ErrInvalidManageTokensPermission
|
|
}
|
|
|
|
tokenMetadata := &protobuf.CommunityTokenMetadata{
|
|
ContractAddresses: map[uint64]string{uint64(token.ChainID): token.Address},
|
|
Description: token.Description,
|
|
Image: token.Base64Image,
|
|
Symbol: token.Symbol,
|
|
TokenType: token.TokenType,
|
|
Name: token.Name,
|
|
Decimals: uint32(token.Decimals),
|
|
}
|
|
_, err = community.AddCommunityTokensMetadata(tokenMetadata)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if community.IsControlNode() && (token.PrivilegesLevel == community_token.MasterLevel || token.PrivilegesLevel == community_token.OwnerLevel) {
|
|
permissionType := protobuf.CommunityTokenPermission_BECOME_TOKEN_OWNER
|
|
if token.PrivilegesLevel == community_token.MasterLevel {
|
|
permissionType = protobuf.CommunityTokenPermission_BECOME_TOKEN_MASTER
|
|
}
|
|
|
|
contractAddresses := make(map[uint64]string)
|
|
contractAddresses[uint64(token.ChainID)] = token.Address
|
|
|
|
tokenCriteria := &protobuf.TokenCriteria{
|
|
ContractAddresses: contractAddresses,
|
|
Type: protobuf.CommunityTokenType_ERC721,
|
|
Symbol: token.Symbol,
|
|
Name: token.Name,
|
|
Amount: "1",
|
|
AmountInWei: "1",
|
|
Decimals: uint64(0),
|
|
}
|
|
|
|
request := &requests.CreateCommunityTokenPermission{
|
|
CommunityID: community.ID(),
|
|
Type: permissionType,
|
|
TokenCriteria: []*protobuf.TokenCriteria{tokenCriteria},
|
|
IsPrivate: true,
|
|
ChatIds: []string{},
|
|
}
|
|
|
|
community, _, err = m.createCommunityTokenPermission(request, community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if token.PrivilegesLevel == community_token.OwnerLevel {
|
|
_, err = m.promoteSelfToControlNode(community, clock)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
}
|
|
}
|
|
|
|
return community, m.saveAndPublish(community)
|
|
}
|
|
|
|
func (m *Manager) UpdateCommunityTokenState(chainID int, contractAddress string, deployState community_token.DeployState) error {
|
|
return m.persistence.UpdateCommunityTokenState(chainID, contractAddress, deployState)
|
|
}
|
|
|
|
func (m *Manager) UpdateCommunityTokenAddress(chainID int, oldContractAddress string, newContractAddress string) error {
|
|
return m.persistence.UpdateCommunityTokenAddress(chainID, oldContractAddress, newContractAddress)
|
|
}
|
|
|
|
func (m *Manager) UpdateCommunityTokenSupply(chainID int, contractAddress string, supply *bigint.BigInt) error {
|
|
return m.persistence.UpdateCommunityTokenSupply(chainID, contractAddress, supply)
|
|
}
|
|
|
|
func (m *Manager) RemoveCommunityToken(chainID int, contractAddress string) error {
|
|
return m.persistence.RemoveCommunityToken(chainID, contractAddress)
|
|
}
|
|
|
|
func (m *Manager) SetCommunityActiveMembersCount(communityID string, activeMembersCount uint64) error {
|
|
id, err := types.DecodeHex(communityID)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
m.communityLock.Lock(id)
|
|
defer m.communityLock.Unlock(id)
|
|
|
|
community, err := m.GetByID(id)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
updated, err := community.SetActiveMembersCount(activeMembersCount)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
if updated {
|
|
if err = m.persistence.SaveCommunity(community); err != nil {
|
|
return err
|
|
}
|
|
|
|
m.publish(&Subscription{Community: community})
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func combineAddressesAndChainIDs(addresses []gethcommon.Address, chainIDs []uint64) []*AccountChainIDsCombination {
|
|
combinations := make([]*AccountChainIDsCombination, 0)
|
|
for _, address := range addresses {
|
|
combinations = append(combinations, &AccountChainIDsCombination{
|
|
Address: address,
|
|
ChainIDs: chainIDs,
|
|
})
|
|
}
|
|
return combinations
|
|
}
|
|
|
|
func revealedAccountsToAccountsAndChainIDsCombination(revealedAccounts []*protobuf.RevealedAccount) []*AccountChainIDsCombination {
|
|
accountsAndChainIDs := make([]*AccountChainIDsCombination, 0)
|
|
for _, revealedAccount := range revealedAccounts {
|
|
accountsAndChainIDs = append(accountsAndChainIDs, &AccountChainIDsCombination{
|
|
Address: gethcommon.HexToAddress(revealedAccount.Address),
|
|
ChainIDs: revealedAccount.ChainIds,
|
|
})
|
|
}
|
|
return accountsAndChainIDs
|
|
}
|
|
|
|
func (m *Manager) accountsHasPrivilegedPermission(preParsedCommunityPermissionData *PreParsedCommunityPermissionsData, accounts []*AccountChainIDsCombination) bool {
|
|
if preParsedCommunityPermissionData != nil {
|
|
permissionResponse, err := m.PermissionChecker.CheckPermissions(preParsedCommunityPermissionData, accounts, true)
|
|
if err != nil {
|
|
m.logger.Warn("check privileged permission failed: %v", zap.Error(err))
|
|
return false
|
|
}
|
|
return permissionResponse.Satisfied
|
|
}
|
|
return false
|
|
}
|
|
|
|
func (m *Manager) saveAndPublish(community *Community) error {
|
|
err := m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
if community.IsControlNode() {
|
|
m.publish(&Subscription{Community: community})
|
|
return nil
|
|
}
|
|
|
|
if community.HasPermissionToSendCommunityEvents() {
|
|
err := m.signEvents(community)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
err = m.persistence.SaveCommunityEvents(community)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
m.publish(&Subscription{CommunityEventsMessage: community.toCommunityEventsMessage()})
|
|
return nil
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) GetRevealedAddresses(communityID types.HexBytes, memberPk string) ([]*protobuf.RevealedAccount, error) {
|
|
logger := m.logger.Named("GetRevealedAddresses")
|
|
|
|
requestID := CalculateRequestID(memberPk, communityID)
|
|
response, err := m.persistence.GetRequestToJoinRevealedAddresses(requestID)
|
|
|
|
revealedAddresses := make([]string, len(response))
|
|
for i, acc := range response {
|
|
revealedAddresses[i] = acc.Address
|
|
}
|
|
logger.Debug("Revealed addresses", zap.Any("Addresses:", revealedAddresses))
|
|
|
|
return response, err
|
|
}
|
|
|
|
func (m *Manager) handleCommunityTokensMetadata(community *Community) error {
|
|
communityID := community.IDString()
|
|
communityTokens := community.CommunityTokensMetadata()
|
|
|
|
if len(communityTokens) == 0 {
|
|
return nil
|
|
}
|
|
for _, tokenMetadata := range communityTokens {
|
|
for chainID, address := range tokenMetadata.ContractAddresses {
|
|
exists, err := m.persistence.HasCommunityToken(communityID, address, int(chainID))
|
|
if err != nil {
|
|
return err
|
|
}
|
|
if !exists {
|
|
// Fetch community token to make sure it's stored in the DB, discard result
|
|
communityToken, err := m.FetchCommunityToken(community, tokenMetadata, chainID, address)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
err = m.persistence.AddCommunityToken(communityToken)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) HandleCommunityGrant(community *Community, grant []byte, clock uint64) (uint64, error) {
|
|
_, oldClock, err := m.GetCommunityGrant(community.IDString())
|
|
if err != nil {
|
|
return 0, err
|
|
}
|
|
|
|
if oldClock >= clock {
|
|
return 0, ErrGrantOlder
|
|
}
|
|
|
|
verifiedGrant, err := community.VerifyGrantSignature(grant)
|
|
if err != nil {
|
|
return 0, err
|
|
}
|
|
|
|
if !bytes.Equal(verifiedGrant.MemberId, crypto.CompressPubkey(&m.identity.PublicKey)) {
|
|
return 0, ErrGrantMemberPublicKeyIsDifferent
|
|
}
|
|
|
|
return clock - oldClock, m.persistence.SaveCommunityGrant(community.IDString(), grant, clock)
|
|
}
|
|
|
|
func (m *Manager) FetchCommunityToken(community *Community, tokenMetadata *protobuf.CommunityTokenMetadata, chainID uint64, contractAddress string) (*community_token.CommunityToken, error) {
|
|
communityID := community.IDString()
|
|
|
|
communityToken := &community_token.CommunityToken{
|
|
CommunityID: communityID,
|
|
Address: contractAddress,
|
|
TokenType: tokenMetadata.TokenType,
|
|
Name: tokenMetadata.Name,
|
|
Symbol: tokenMetadata.Symbol,
|
|
Description: tokenMetadata.Description,
|
|
Transferable: true,
|
|
RemoteSelfDestruct: false,
|
|
ChainID: int(chainID),
|
|
DeployState: community_token.Deployed,
|
|
Base64Image: tokenMetadata.Image,
|
|
Decimals: int(tokenMetadata.Decimals),
|
|
}
|
|
|
|
switch tokenMetadata.TokenType {
|
|
case protobuf.CommunityTokenType_ERC721:
|
|
contractData, err := m.communityTokensService.GetCollectibleContractData(chainID, contractAddress)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
communityToken.Supply = contractData.TotalSupply
|
|
communityToken.Transferable = contractData.Transferable
|
|
communityToken.RemoteSelfDestruct = contractData.RemoteBurnable
|
|
communityToken.InfiniteSupply = contractData.InfiniteSupply
|
|
|
|
case protobuf.CommunityTokenType_ERC20:
|
|
contractData, err := m.communityTokensService.GetAssetContractData(chainID, contractAddress)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
communityToken.Supply = contractData.TotalSupply
|
|
communityToken.InfiniteSupply = contractData.InfiniteSupply
|
|
}
|
|
|
|
communityToken.PrivilegesLevel = getPrivilegesLevel(chainID, contractAddress, community.TokenPermissions())
|
|
|
|
return communityToken, nil
|
|
}
|
|
|
|
func getPrivilegesLevel(chainID uint64, tokenAddress string, tokenPermissions map[string]*CommunityTokenPermission) community_token.PrivilegesLevel {
|
|
for _, permission := range tokenPermissions {
|
|
if permission.Type == protobuf.CommunityTokenPermission_BECOME_TOKEN_MASTER || permission.Type == protobuf.CommunityTokenPermission_BECOME_TOKEN_OWNER {
|
|
for _, tokenCriteria := range permission.TokenCriteria {
|
|
value, exist := tokenCriteria.ContractAddresses[chainID]
|
|
if exist && value == tokenAddress {
|
|
if permission.Type == protobuf.CommunityTokenPermission_BECOME_TOKEN_OWNER {
|
|
return community_token.OwnerLevel
|
|
}
|
|
return community_token.MasterLevel
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return community_token.CommunityLevel
|
|
}
|
|
|
|
func (m *Manager) ValidateCommunityPrivilegedUserSyncMessage(message *protobuf.CommunityPrivilegedUserSyncMessage) error {
|
|
if message == nil {
|
|
return errors.New("invalid CommunityPrivilegedUserSyncMessage message")
|
|
}
|
|
|
|
if message.CommunityId == nil || len(message.CommunityId) == 0 {
|
|
return errors.New("invalid CommunityId in CommunityPrivilegedUserSyncMessage message")
|
|
}
|
|
|
|
switch message.Type {
|
|
case protobuf.CommunityPrivilegedUserSyncMessage_CONTROL_NODE_ACCEPT_REQUEST_TO_JOIN:
|
|
fallthrough
|
|
case protobuf.CommunityPrivilegedUserSyncMessage_CONTROL_NODE_REJECT_REQUEST_TO_JOIN:
|
|
if message.RequestToJoin == nil || len(message.RequestToJoin) == 0 {
|
|
return errors.New("invalid request to join in CommunityPrivilegedUserSyncMessage message")
|
|
}
|
|
|
|
for _, requestToJoinProto := range message.RequestToJoin {
|
|
if len(requestToJoinProto.CommunityId) == 0 {
|
|
return errors.New("no communityId in request to join in CommunityPrivilegedUserSyncMessage message")
|
|
}
|
|
}
|
|
case protobuf.CommunityPrivilegedUserSyncMessage_CONTROL_NODE_ALL_SYNC_REQUESTS_TO_JOIN:
|
|
if message.SyncRequestsToJoin == nil || len(message.SyncRequestsToJoin) == 0 {
|
|
return errors.New("invalid sync requests to join in CommunityPrivilegedUserSyncMessage message")
|
|
}
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) createCommunityTokenPermission(request *requests.CreateCommunityTokenPermission, community *Community) (*Community, *CommunityChanges, error) {
|
|
if community == nil {
|
|
return nil, nil, ErrOrgNotFound
|
|
}
|
|
|
|
tokenPermission := request.ToCommunityTokenPermission()
|
|
tokenPermission.Id = uuid.New().String()
|
|
changes, err := community.UpsertTokenPermission(&tokenPermission)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
|
|
return community, changes, nil
|
|
|
|
}
|
|
|
|
func (m *Manager) PromoteSelfToControlNode(community *Community, clock uint64) (*CommunityChanges, error) {
|
|
if community == nil {
|
|
return nil, ErrOrgNotFound
|
|
}
|
|
|
|
m.communityLock.Lock(community.ID())
|
|
defer m.communityLock.Unlock(community.ID())
|
|
|
|
ownerChanged, err := m.promoteSelfToControlNode(community, clock)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if ownerChanged {
|
|
return community.RemoveAllUsersFromOrg(), m.saveAndPublish(community)
|
|
}
|
|
|
|
return community.emptyCommunityChanges(), m.saveAndPublish(community)
|
|
}
|
|
|
|
func (m *Manager) promoteSelfToControlNode(community *Community, clock uint64) (bool, error) {
|
|
ownerChanged := false
|
|
community.setPrivateKey(m.identity)
|
|
if !community.ControlNode().Equal(&m.identity.PublicKey) {
|
|
ownerChanged = true
|
|
community.setControlNode(&m.identity.PublicKey)
|
|
}
|
|
|
|
// Mark this device as the control node
|
|
syncControlNode := &protobuf.SyncCommunityControlNode{
|
|
Clock: clock,
|
|
InstallationId: m.installationID,
|
|
}
|
|
|
|
err := m.SaveSyncControlNode(community.ID(), syncControlNode)
|
|
if err != nil {
|
|
return false, err
|
|
}
|
|
community.config.ControlDevice = true
|
|
|
|
if exists := community.HasMember(&m.identity.PublicKey); !exists {
|
|
ownerRole := []protobuf.CommunityMember_Roles{protobuf.CommunityMember_ROLE_OWNER}
|
|
_, err = community.AddMember(&m.identity.PublicKey, ownerRole)
|
|
if err != nil {
|
|
return false, err
|
|
}
|
|
|
|
for channelID := range community.Chats() {
|
|
_, err = community.AddMemberToChat(channelID, &m.identity.PublicKey, ownerRole, protobuf.CommunityMember_CHANNEL_ROLE_POSTER)
|
|
if err != nil {
|
|
return false, err
|
|
}
|
|
}
|
|
} else {
|
|
_, err = community.AddRoleToMember(&m.identity.PublicKey, protobuf.CommunityMember_ROLE_OWNER)
|
|
}
|
|
|
|
if err != nil {
|
|
return false, err
|
|
}
|
|
|
|
err = m.handleCommunityEvents(community)
|
|
if err != nil {
|
|
return false, err
|
|
}
|
|
|
|
community.increaseClock()
|
|
|
|
return ownerChanged, nil
|
|
}
|
|
|
|
func (m *Manager) handleCommunityEventsAndMetadata(community *Community, eventsMessage *CommunityEventsMessage,
|
|
lastlyAppliedEvents map[string]uint64) (*CommunityResponse, error) {
|
|
err := community.processEvents(eventsMessage, lastlyAppliedEvents)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
additionalCommunityResponse, err := m.handleAdditionalAdminChanges(community)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
if err = m.handleCommunityTokensMetadata(community); err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return additionalCommunityResponse, err
|
|
}
|
|
|
|
func (m *Manager) handleCommunityEvents(community *Community) error {
|
|
if community.config.EventsData == nil {
|
|
return nil
|
|
}
|
|
|
|
lastlyAppliedEvents, err := m.persistence.GetAppliedCommunityEvents(community.ID())
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
_, err = m.handleCommunityEventsAndMetadata(community, community.toCommunityEventsMessage(), lastlyAppliedEvents)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
appliedEvents := map[string]uint64{}
|
|
if community.config.EventsData != nil {
|
|
for _, event := range community.config.EventsData.Events {
|
|
appliedEvents[event.EventTypeID()] = event.CommunityEventClock
|
|
}
|
|
}
|
|
|
|
community.config.EventsData = nil // clear events, they are already applied
|
|
community.increaseClock()
|
|
|
|
err = m.persistence.SaveCommunity(community)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
err = m.persistence.UpsertAppliedCommunityEvents(community.ID(), appliedEvents)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
m.publish(&Subscription{Community: community})
|
|
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) shareRequestsToJoinWithNewPrivilegedMembers(community *Community, newPrivilegedMembers map[protobuf.CommunityMember_Roles][]*ecdsa.PublicKey) error {
|
|
requestsToJoin, err := m.GetCommunityRequestsToJoinWithRevealedAddresses(community.ID())
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
var syncRequestsWithoutRevealedAccounts []*protobuf.SyncCommunityRequestsToJoin
|
|
var syncRequestsWithRevealedAccounts []*protobuf.SyncCommunityRequestsToJoin
|
|
for _, request := range requestsToJoin {
|
|
syncRequestsWithRevealedAccounts = append(syncRequestsWithRevealedAccounts, request.ToSyncProtobuf())
|
|
requestProtoWithoutAccounts := request.ToSyncProtobuf()
|
|
requestProtoWithoutAccounts.RevealedAccounts = []*protobuf.RevealedAccount{}
|
|
syncRequestsWithoutRevealedAccounts = append(syncRequestsWithoutRevealedAccounts, requestProtoWithoutAccounts)
|
|
}
|
|
|
|
syncMsgWithoutRevealedAccounts := &protobuf.CommunityPrivilegedUserSyncMessage{
|
|
Type: protobuf.CommunityPrivilegedUserSyncMessage_CONTROL_NODE_ALL_SYNC_REQUESTS_TO_JOIN,
|
|
CommunityId: community.ID(),
|
|
SyncRequestsToJoin: syncRequestsWithoutRevealedAccounts,
|
|
}
|
|
|
|
syncMsgWitRevealedAccounts := &protobuf.CommunityPrivilegedUserSyncMessage{
|
|
Type: protobuf.CommunityPrivilegedUserSyncMessage_CONTROL_NODE_ALL_SYNC_REQUESTS_TO_JOIN,
|
|
CommunityId: community.ID(),
|
|
SyncRequestsToJoin: syncRequestsWithRevealedAccounts,
|
|
}
|
|
|
|
subscriptionMsg := &CommunityPrivilegedMemberSyncMessage{
|
|
CommunityPrivateKey: community.PrivateKey(),
|
|
}
|
|
|
|
for role, members := range newPrivilegedMembers {
|
|
if len(members) == 0 {
|
|
continue
|
|
}
|
|
|
|
subscriptionMsg.Receivers = members
|
|
|
|
switch role {
|
|
case protobuf.CommunityMember_ROLE_ADMIN:
|
|
subscriptionMsg.CommunityPrivilegedUserSyncMessage = syncMsgWithoutRevealedAccounts
|
|
case protobuf.CommunityMember_ROLE_OWNER:
|
|
continue
|
|
case protobuf.CommunityMember_ROLE_TOKEN_MASTER:
|
|
subscriptionMsg.CommunityPrivilegedUserSyncMessage = syncMsgWitRevealedAccounts
|
|
}
|
|
|
|
m.publish(&Subscription{CommunityPrivilegedMemberSyncMessage: subscriptionMsg})
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) shareAcceptedRequestToJoinWithPrivilegedMembers(community *Community, requestsToJoin *RequestToJoin) error {
|
|
pk, err := common.HexToPubkey(requestsToJoin.PublicKey)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
acceptedRequestsToJoinWithoutRevealedAccounts := make(map[string]*protobuf.CommunityRequestToJoin)
|
|
acceptedRequestsToJoinWithRevealedAccounts := make(map[string]*protobuf.CommunityRequestToJoin)
|
|
|
|
acceptedRequestsToJoinWithRevealedAccounts[requestsToJoin.PublicKey] = requestsToJoin.ToCommunityRequestToJoinProtobuf()
|
|
requestsToJoin.RevealedAccounts = make([]*protobuf.RevealedAccount, 0)
|
|
acceptedRequestsToJoinWithoutRevealedAccounts[requestsToJoin.PublicKey] = requestsToJoin.ToCommunityRequestToJoinProtobuf()
|
|
|
|
msgWithRevealedAccounts := &protobuf.CommunityPrivilegedUserSyncMessage{
|
|
Type: protobuf.CommunityPrivilegedUserSyncMessage_CONTROL_NODE_ACCEPT_REQUEST_TO_JOIN,
|
|
CommunityId: community.ID(),
|
|
RequestToJoin: acceptedRequestsToJoinWithRevealedAccounts,
|
|
}
|
|
|
|
msgWithoutRevealedAccounts := &protobuf.CommunityPrivilegedUserSyncMessage{
|
|
Type: protobuf.CommunityPrivilegedUserSyncMessage_CONTROL_NODE_ACCEPT_REQUEST_TO_JOIN,
|
|
CommunityId: community.ID(),
|
|
RequestToJoin: acceptedRequestsToJoinWithoutRevealedAccounts,
|
|
}
|
|
|
|
// do not sent to ourself and to the accepted user
|
|
skipMembers := make(map[string]struct{})
|
|
skipMembers[common.PubkeyToHex(&m.identity.PublicKey)] = struct{}{}
|
|
skipMembers[common.PubkeyToHex(pk)] = struct{}{}
|
|
|
|
subscriptionMsg := &CommunityPrivilegedMemberSyncMessage{
|
|
CommunityPrivateKey: community.PrivateKey(),
|
|
}
|
|
|
|
fileredPrivilegedMembers := community.GetFilteredPrivilegedMembers(skipMembers)
|
|
for role, members := range fileredPrivilegedMembers {
|
|
if len(members) == 0 {
|
|
continue
|
|
}
|
|
|
|
subscriptionMsg.Receivers = members
|
|
|
|
switch role {
|
|
case protobuf.CommunityMember_ROLE_ADMIN:
|
|
subscriptionMsg.CommunityPrivilegedUserSyncMessage = msgWithoutRevealedAccounts
|
|
case protobuf.CommunityMember_ROLE_OWNER:
|
|
fallthrough
|
|
case protobuf.CommunityMember_ROLE_TOKEN_MASTER:
|
|
subscriptionMsg.CommunityPrivilegedUserSyncMessage = msgWithRevealedAccounts
|
|
}
|
|
|
|
m.publish(&Subscription{CommunityPrivilegedMemberSyncMessage: subscriptionMsg})
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) GetCommunityRequestsToJoinWithRevealedAddresses(communityID types.HexBytes) ([]*RequestToJoin, error) {
|
|
return m.persistence.GetCommunityRequestsToJoinWithRevealedAddresses(communityID)
|
|
}
|
|
|
|
func (m *Manager) SaveCommunity(community *Community) error {
|
|
return m.persistence.SaveCommunity(community)
|
|
}
|
|
|
|
func (m *Manager) CreateCommunityTokenDeploymentSignature(ctx context.Context, chainID uint64, addressFrom string, communityID string) ([]byte, error) {
|
|
community, err := m.GetByIDString(communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
if !community.IsControlNode() {
|
|
return nil, ErrNotControlNode
|
|
}
|
|
digest, err := m.communityTokensService.DeploymentSignatureDigest(chainID, addressFrom, communityID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return crypto.Sign(digest, community.PrivateKey())
|
|
}
|
|
|
|
func (m *Manager) GetSyncControlNode(id types.HexBytes) (*protobuf.SyncCommunityControlNode, error) {
|
|
return m.persistence.GetSyncControlNode(id)
|
|
}
|
|
|
|
func (m *Manager) SaveSyncControlNode(id types.HexBytes, syncControlNode *protobuf.SyncCommunityControlNode) error {
|
|
return m.persistence.SaveSyncControlNode(id, syncControlNode.Clock, syncControlNode.InstallationId)
|
|
}
|
|
|
|
func (m *Manager) SetSyncControlNode(id types.HexBytes, syncControlNode *protobuf.SyncCommunityControlNode) error {
|
|
existingSyncControlNode, err := m.GetSyncControlNode(id)
|
|
if err != nil {
|
|
return err
|
|
}
|
|
|
|
if existingSyncControlNode == nil || existingSyncControlNode.Clock < syncControlNode.Clock {
|
|
return m.SaveSyncControlNode(id, syncControlNode)
|
|
}
|
|
|
|
return nil
|
|
}
|
|
|
|
func (m *Manager) GetCommunityRequestToJoinWithRevealedAddresses(pubKey string, communityID types.HexBytes) (*RequestToJoin, error) {
|
|
return m.persistence.GetCommunityRequestToJoinWithRevealedAddresses(pubKey, communityID)
|
|
}
|
|
|
|
func (m *Manager) SafeGetSignerPubKey(chainID uint64, communityID string) (string, error) {
|
|
ctx, cancel := context.WithTimeout(context.Background(), time.Second*3)
|
|
defer cancel()
|
|
|
|
return m.ownerVerifier.SafeGetSignerPubKey(ctx, chainID, communityID)
|
|
}
|
|
|
|
func (m *Manager) GetCuratedCommunities() (*CuratedCommunities, error) {
|
|
return m.persistence.GetCuratedCommunities()
|
|
}
|
|
|
|
func (m *Manager) SetCuratedCommunities(communities *CuratedCommunities) error {
|
|
return m.persistence.SetCuratedCommunities(communities)
|
|
}
|
|
|
|
func (m *Manager) encryptCommunityDescriptionImpl(groupID []byte, d *protobuf.CommunityDescription) (string, []byte, error) {
|
|
payload, err := proto.Marshal(d)
|
|
if err != nil {
|
|
return "", nil, err
|
|
}
|
|
|
|
encryptedPayload, ratchet, newSeqNo, err := m.encryptor.EncryptWithHashRatchet(groupID, payload)
|
|
if err == encryption.ErrNoEncryptionKey {
|
|
_, err := m.encryptor.GenerateHashRatchetKey(groupID)
|
|
if err != nil {
|
|
return "", nil, err
|
|
}
|
|
encryptedPayload, ratchet, newSeqNo, err = m.encryptor.EncryptWithHashRatchet(groupID, payload)
|
|
if err != nil {
|
|
return "", nil, err
|
|
}
|
|
|
|
} else if err != nil {
|
|
return "", nil, err
|
|
}
|
|
|
|
keyID, err := ratchet.GetKeyID()
|
|
if err != nil {
|
|
return "", nil, err
|
|
}
|
|
|
|
m.logger.Debug("encrypting community description",
|
|
zap.Any("community", d),
|
|
zap.String("groupID", types.Bytes2Hex(groupID)),
|
|
zap.String("keyID", types.Bytes2Hex(keyID)))
|
|
|
|
keyIDSeqNo := fmt.Sprintf("%s%d", hex.EncodeToString(keyID), newSeqNo)
|
|
|
|
return keyIDSeqNo, encryptedPayload, nil
|
|
}
|
|
|
|
func (m *Manager) encryptCommunityDescription(community *Community, d *protobuf.CommunityDescription) (string, []byte, error) {
|
|
return m.encryptCommunityDescriptionImpl(community.ID(), d)
|
|
}
|
|
|
|
func (m *Manager) encryptCommunityDescriptionChannel(community *Community, channelID string, d *protobuf.CommunityDescription) (string, []byte, error) {
|
|
return m.encryptCommunityDescriptionImpl([]byte(community.IDString()+channelID), d)
|
|
}
|
|
|
|
// TODO: add collectiblesManager to messenger intance
|
|
func (m *Manager) GetCollectiblesManager() CollectiblesManager {
|
|
return m.collectiblesManager
|
|
}
|
|
|
|
type DecryptCommunityResponse struct {
|
|
Decrypted bool
|
|
Description *protobuf.CommunityDescription
|
|
KeyID []byte
|
|
GroupID []byte
|
|
}
|
|
|
|
func (m *Manager) decryptCommunityDescription(keyIDSeqNo string, d []byte) (*DecryptCommunityResponse, error) {
|
|
const hashHexLength = 64
|
|
if len(keyIDSeqNo) <= hashHexLength {
|
|
return nil, errors.New("invalid keyIDSeqNo")
|
|
}
|
|
|
|
keyID, err := hex.DecodeString(keyIDSeqNo[:hashHexLength])
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
seqNo, err := strconv.ParseUint(keyIDSeqNo[hashHexLength:], 10, 32)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
decryptedPayload, err := m.encryptor.DecryptWithHashRatchet(keyID, uint32(seqNo), d)
|
|
if err == encryption.ErrNoRatchetKey {
|
|
return &DecryptCommunityResponse{
|
|
KeyID: keyID,
|
|
}, err
|
|
|
|
}
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
var description protobuf.CommunityDescription
|
|
err = proto.Unmarshal(decryptedPayload, &description)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
decryptCommunityResponse := &DecryptCommunityResponse{
|
|
Decrypted: true,
|
|
KeyID: keyID,
|
|
Description: &description,
|
|
}
|
|
return decryptCommunityResponse, nil
|
|
}
|
|
|
|
func ToLinkPreveiwThumbnail(image images.IdentityImage) (*common.LinkPreviewThumbnail, error) {
|
|
thumbnail := &common.LinkPreviewThumbnail{}
|
|
|
|
if image.IsEmpty() {
|
|
return nil, nil
|
|
}
|
|
|
|
width, height, err := images.GetImageDimensions(image.Payload)
|
|
if err != nil {
|
|
return nil, fmt.Errorf("failed to get image dimensions: %w", err)
|
|
}
|
|
|
|
dataURI, err := image.GetDataURI()
|
|
if err != nil {
|
|
return nil, fmt.Errorf("failed to get data uri: %w", err)
|
|
}
|
|
|
|
thumbnail.Width = width
|
|
thumbnail.Height = height
|
|
thumbnail.DataURI = dataURI
|
|
return thumbnail, nil
|
|
}
|
|
|
|
func (c *Community) ToStatusLinkPreview() (*common.StatusCommunityLinkPreview, error) {
|
|
communityLinkPreview := &common.StatusCommunityLinkPreview{}
|
|
if image, ok := c.Images()[images.SmallDimName]; ok {
|
|
thumbnail, err := ToLinkPreveiwThumbnail(images.IdentityImage{Payload: image.Payload})
|
|
if err != nil {
|
|
c.config.Logger.Warn("unfurling status link: failed to set community thumbnail", zap.Error(err))
|
|
}
|
|
communityLinkPreview.Icon = *thumbnail
|
|
}
|
|
|
|
if image, ok := c.Images()[images.BannerIdentityName]; ok {
|
|
thumbnail, err := ToLinkPreveiwThumbnail(images.IdentityImage{Payload: image.Payload})
|
|
if err != nil {
|
|
c.config.Logger.Warn("unfurling status link: failed to set community thumbnail", zap.Error(err))
|
|
}
|
|
communityLinkPreview.Banner = *thumbnail
|
|
}
|
|
|
|
communityLinkPreview.CommunityID = c.IDString()
|
|
communityLinkPreview.DisplayName = c.Name()
|
|
communityLinkPreview.Description = c.DescriptionText()
|
|
communityLinkPreview.MembersCount = uint32(c.MembersCount())
|
|
communityLinkPreview.Color = c.Color()
|
|
|
|
return communityLinkPreview, nil
|
|
}
|