mirror of
https://github.com/anyproto/any-sync.git
synced 2025-06-08 05:57:03 +09:00
Merge branch 'main' of github.com:anyproto/any-sync into GO-3125-shared-spaces-limit
This commit is contained in:
commit
3a3be11416
26 changed files with 1304 additions and 404 deletions
1
Makefile
1
Makefile
|
@ -23,6 +23,7 @@ proto:
|
|||
protoc --gogofaster_out=$(PKGMAP):. --go-drpc_out=protolib=github.com/gogo/protobuf:. commonfile/fileproto/protos/*.proto
|
||||
protoc --gogofaster_out=$(PKGMAP):. --go-drpc_out=protolib=github.com/gogo/protobuf:. net/streampool/testservice/protos/*.proto
|
||||
protoc --gogofaster_out=:. net/secureservice/handshake/handshakeproto/protos/*.proto
|
||||
protoc --gogofaster_out=:. net/rpc/limiter/limiterproto/protos/*.proto
|
||||
protoc --gogofaster_out=$(PKGMAP):. --go-drpc_out=protolib=github.com/gogo/protobuf:. coordinator/coordinatorproto/protos/*.proto
|
||||
protoc --gogofaster_out=:. --go-drpc_out=protolib=github.com/gogo/protobuf:. consensus/consensusproto/protos/*.proto
|
||||
protoc --gogofaster_out=:. --go-drpc_out=protolib=github.com/gogo/protobuf:. identityrepo/identityrepoproto/protos/*.proto
|
||||
|
|
10
app/app.go
10
app/app.go
|
@ -4,14 +4,16 @@ import (
|
|||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"github.com/anyproto/any-sync/app/logger"
|
||||
"go.uber.org/zap"
|
||||
"os"
|
||||
"runtime"
|
||||
"runtime/debug"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/anyproto/any-sync/app/logger"
|
||||
)
|
||||
|
||||
var (
|
||||
|
@ -227,7 +229,7 @@ func (app *App) Start(ctx context.Context) (err error) {
|
|||
for i := idx; i >= 0; i-- {
|
||||
if serviceClose, ok := app.components[i].(ComponentRunnable); ok {
|
||||
if e := serviceClose.Close(ctx); e != nil {
|
||||
log.Info("close error", zap.String("component", serviceClose.Name()), zap.Error(e))
|
||||
log.Error("close error", zap.String("component", serviceClose.Name()), zap.Error(e))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -235,6 +237,7 @@ func (app *App) Start(ctx context.Context) (err error) {
|
|||
|
||||
for i, s := range app.components {
|
||||
if err = s.Init(app); err != nil {
|
||||
log.Error("can't init service", zap.String("service", s.Name()), zap.Error(err))
|
||||
closeServices(i)
|
||||
return fmt.Errorf("can't init service '%s': %w", s.Name(), err)
|
||||
}
|
||||
|
@ -244,6 +247,7 @@ func (app *App) Start(ctx context.Context) (err error) {
|
|||
if serviceRun, ok := s.(ComponentRunnable); ok {
|
||||
start := time.Now()
|
||||
if err = serviceRun.Run(ctx); err != nil {
|
||||
log.Error("can't run service", zap.String("service", serviceRun.Name()), zap.Error(err))
|
||||
closeServices(i)
|
||||
return fmt.Errorf("can't run service '%s': %w", serviceRun.Name(), err)
|
||||
}
|
||||
|
|
|
@ -275,7 +275,6 @@ func (c *oCache) GC() {
|
|||
var toClose []*entry
|
||||
for _, e := range c.data {
|
||||
if e.isActive() && e.lastUsage.Before(deadline) {
|
||||
e.close = make(chan struct{})
|
||||
toClose = append(toClose, e)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -11,7 +11,7 @@ import (
|
|||
"github.com/anyproto/any-sync/commonspace/object/acl/list"
|
||||
"github.com/anyproto/any-sync/commonspace/object/acl/liststorage"
|
||||
"github.com/anyproto/any-sync/consensus/consensusproto"
|
||||
"github.com/anyproto/any-sync/coordinator/coordinatorclient"
|
||||
"github.com/anyproto/any-sync/node/nodeclient"
|
||||
)
|
||||
|
||||
const CName = "common.acl.aclclient"
|
||||
|
@ -25,8 +25,8 @@ type AclJoiningClient interface {
|
|||
}
|
||||
|
||||
type aclJoiningClient struct {
|
||||
coordinatorClient coordinatorclient.CoordinatorClient
|
||||
keys *accountdata.AccountKeys
|
||||
nodeClient nodeclient.NodeClient
|
||||
keys *accountdata.AccountKeys
|
||||
}
|
||||
|
||||
func NewAclJoiningClient() AclJoiningClient {
|
||||
|
@ -38,13 +38,13 @@ func (c *aclJoiningClient) Name() (name string) {
|
|||
}
|
||||
|
||||
func (c *aclJoiningClient) Init(a *app.App) (err error) {
|
||||
c.coordinatorClient = a.MustComponent(coordinatorclient.CName).(coordinatorclient.CoordinatorClient)
|
||||
c.nodeClient = a.MustComponent(nodeclient.CName).(nodeclient.NodeClient)
|
||||
c.keys = a.MustComponent(accountservice.CName).(accountservice.Service).Account()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c *aclJoiningClient) AclGetRecords(ctx context.Context, spaceId, aclHead string) (recs []*consensusproto.RawRecordWithId, err error) {
|
||||
return c.coordinatorClient.AclGetRecords(ctx, spaceId, aclHead)
|
||||
return c.nodeClient.AclGetRecords(ctx, spaceId, aclHead)
|
||||
}
|
||||
|
||||
func (c *aclJoiningClient) getAcl(ctx context.Context, spaceId string) (l list.AclList, err error) {
|
||||
|
@ -76,7 +76,7 @@ func (c *aclJoiningClient) CancelJoin(ctx context.Context, spaceId string) (err
|
|||
if err != nil {
|
||||
return
|
||||
}
|
||||
_, err = c.coordinatorClient.AclAddRecord(ctx, spaceId, res)
|
||||
_, err = c.nodeClient.AclAddRecord(ctx, spaceId, res)
|
||||
return
|
||||
}
|
||||
|
||||
|
@ -99,7 +99,7 @@ func (c *aclJoiningClient) RequestJoin(ctx context.Context, spaceId string, payl
|
|||
if err != nil {
|
||||
return
|
||||
}
|
||||
recWithId, err := c.coordinatorClient.AclAddRecord(ctx, spaceId, rec)
|
||||
recWithId, err := c.nodeClient.AclAddRecord(ctx, spaceId, rec)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
@ -126,6 +126,6 @@ func (c *aclJoiningClient) CancelRemoveSelf(ctx context.Context, spaceId string)
|
|||
if err != nil {
|
||||
return
|
||||
}
|
||||
_, err = c.coordinatorClient.AclAddRecord(ctx, spaceId, newRec)
|
||||
_, err = c.nodeClient.AclAddRecord(ctx, spaceId, newRec)
|
||||
return
|
||||
}
|
||||
|
|
|
@ -9,7 +9,7 @@ import (
|
|||
"github.com/anyproto/any-sync/commonspace/object/acl/syncacl"
|
||||
"github.com/anyproto/any-sync/commonspace/spacestate"
|
||||
"github.com/anyproto/any-sync/consensus/consensusproto"
|
||||
"github.com/anyproto/any-sync/coordinator/coordinatorclient"
|
||||
"github.com/anyproto/any-sync/node/nodeclient"
|
||||
"github.com/anyproto/any-sync/util/crypto"
|
||||
)
|
||||
|
||||
|
@ -45,13 +45,13 @@ func NewAclSpaceClient() AclSpaceClient {
|
|||
}
|
||||
|
||||
type aclSpaceClient struct {
|
||||
coordinatorClient coordinatorclient.CoordinatorClient
|
||||
acl list.AclList
|
||||
spaceId string
|
||||
nodeClient nodeclient.NodeClient
|
||||
acl list.AclList
|
||||
spaceId string
|
||||
}
|
||||
|
||||
func (c *aclSpaceClient) Init(a *app.App) (err error) {
|
||||
c.coordinatorClient = a.MustComponent(coordinatorclient.CName).(coordinatorclient.CoordinatorClient)
|
||||
c.nodeClient = a.MustComponent(nodeclient.CName).(nodeclient.NodeClient)
|
||||
c.acl = a.MustComponent(syncacl.CName).(list.AclList)
|
||||
c.spaceId = a.MustComponent(spacestate.CName).(*spacestate.SpaceState).SpaceId
|
||||
return nil
|
||||
|
@ -217,7 +217,7 @@ func (c *aclSpaceClient) AddRecord(ctx context.Context, consRec *consensusproto.
|
|||
}
|
||||
|
||||
func (c *aclSpaceClient) sendRecordAndUpdate(ctx context.Context, spaceId string, rec *consensusproto.RawRecord) (err error) {
|
||||
res, err := c.coordinatorClient.AclAddRecord(ctx, spaceId, rec)
|
||||
res, err := c.nodeClient.AclAddRecord(ctx, spaceId, rec)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
|
|
@ -28,6 +28,7 @@ import (
|
|||
"github.com/anyproto/any-sync/identityrepo/identityrepoproto"
|
||||
"github.com/anyproto/any-sync/net/peer"
|
||||
"github.com/anyproto/any-sync/net/pool"
|
||||
"github.com/anyproto/any-sync/node/nodeclient"
|
||||
"github.com/anyproto/any-sync/nodeconf"
|
||||
"github.com/anyproto/any-sync/testutil/accounttest"
|
||||
)
|
||||
|
@ -447,6 +448,27 @@ func (m mockCoordinatorClient) Name() (name string) {
|
|||
return coordinatorclient.CName
|
||||
}
|
||||
|
||||
var _ nodeclient.NodeClient = (*mockNodeClient)(nil)
|
||||
|
||||
type mockNodeClient struct {
|
||||
}
|
||||
|
||||
func (m mockNodeClient) Init(a *app.App) (err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (m mockNodeClient) Name() (name string) {
|
||||
return nodeclient.CName
|
||||
}
|
||||
|
||||
func (m mockNodeClient) AclGetRecords(ctx context.Context, spaceId, aclHead string) (recs []*consensusproto.RawRecordWithId, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
func (m mockNodeClient) AclAddRecord(ctx context.Context, spaceId string, rec *consensusproto.RawRecord) (recWithId *consensusproto.RawRecordWithId, err error) {
|
||||
return
|
||||
}
|
||||
|
||||
//
|
||||
// Space fixture
|
||||
//
|
||||
|
@ -484,6 +506,7 @@ func newFixture(t *testing.T) *spaceFixture {
|
|||
Register(credentialprovider.NewNoOp()).
|
||||
Register(&mockStatusServiceProvider{}).
|
||||
Register(mockCoordinatorClient{}).
|
||||
Register(mockNodeClient{}).
|
||||
Register(fx.configurationService).
|
||||
Register(fx.storageProvider).
|
||||
Register(fx.peermanagerProvider).
|
||||
|
|
1
go.mod
1
go.mod
|
@ -106,6 +106,7 @@ require (
|
|||
golang.org/x/mod v0.15.0 // indirect
|
||||
golang.org/x/sync v0.6.0 // indirect
|
||||
golang.org/x/sys v0.18.0 // indirect
|
||||
golang.org/x/time v0.5.0 // indirect
|
||||
golang.org/x/tools v0.17.0 // indirect
|
||||
google.golang.org/protobuf v1.32.0 // indirect
|
||||
lukechampine.com/blake3 v1.2.1 // indirect
|
||||
|
|
2
go.sum
2
go.sum
|
@ -369,6 +369,8 @@ golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9sn
|
|||
golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ=
|
||||
golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ=
|
||||
golang.org/x/text v0.14.0 h1:ScX5w1eTa3QqT8oi6+ziP7dTV1S2+ALU0bI+0zXKWiQ=
|
||||
golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk=
|
||||
golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM=
|
||||
golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ=
|
||||
golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||
golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs=
|
||||
|
|
15
net/rpc/limiter/config.go
Normal file
15
net/rpc/limiter/config.go
Normal file
|
@ -0,0 +1,15 @@
|
|||
package limiter
|
||||
|
||||
type ConfigGetter interface {
|
||||
GetLimiterConf() Config
|
||||
}
|
||||
|
||||
type Tokens struct {
|
||||
TokensPerSecond int `yaml:"rps"`
|
||||
MaxTokens int `yaml:"burst"`
|
||||
}
|
||||
|
||||
type Config struct {
|
||||
DefaultTokens Tokens `yaml:"default"`
|
||||
ResponseTokens map[string]Tokens `yaml:"rpc"`
|
||||
}
|
130
net/rpc/limiter/limiter.go
Normal file
130
net/rpc/limiter/limiter.go
Normal file
|
@ -0,0 +1,130 @@
|
|||
//go:generate mockgen -destination mock_limiter/mock_limiter.go github.com/anyproto/any-sync/net/rpc/limiter RpcLimiter
|
||||
package limiter
|
||||
|
||||
import (
|
||||
"context"
|
||||
"strings"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"golang.org/x/time/rate"
|
||||
"storj.io/drpc"
|
||||
|
||||
"github.com/anyproto/any-sync/app"
|
||||
"github.com/anyproto/any-sync/app/logger"
|
||||
"github.com/anyproto/any-sync/net/peer"
|
||||
"github.com/anyproto/any-sync/net/rpc/limiter/limiterproto"
|
||||
"github.com/anyproto/any-sync/util/periodicsync"
|
||||
)
|
||||
|
||||
const (
|
||||
peerCheckInterval = 10 * time.Second
|
||||
checkTimeout = 2 * time.Second
|
||||
)
|
||||
|
||||
var log = logger.NewNamed(CName)
|
||||
|
||||
const CName = "common.rpc.limiter"
|
||||
|
||||
type RpcLimiter interface {
|
||||
app.ComponentRunnable
|
||||
// WrapDRPCHandler wraps the given drpc.Handler with additional functionality
|
||||
WrapDRPCHandler(handler drpc.Handler) drpc.Handler
|
||||
}
|
||||
|
||||
func New() RpcLimiter {
|
||||
return &limiter{
|
||||
limiters: make(map[string]*peerLimiter),
|
||||
peerCheckInterval: peerCheckInterval,
|
||||
checkTimeout: checkTimeout,
|
||||
}
|
||||
}
|
||||
|
||||
type peerLimiter struct {
|
||||
*rate.Limiter
|
||||
lastUsage time.Time
|
||||
}
|
||||
|
||||
type limiter struct {
|
||||
drpc.Handler
|
||||
limiters map[string]*peerLimiter
|
||||
periodicLoop periodicsync.PeriodicSync
|
||||
peerCheckInterval time.Duration
|
||||
checkTimeout time.Duration
|
||||
cfg Config
|
||||
mx sync.Mutex
|
||||
}
|
||||
|
||||
func (h *limiter) Run(ctx context.Context) (err error) {
|
||||
h.periodicLoop.Run()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *limiter) Close(ctx context.Context) (err error) {
|
||||
h.periodicLoop.Close()
|
||||
return
|
||||
}
|
||||
|
||||
func (h *limiter) Init(a *app.App) (err error) {
|
||||
h.periodicLoop = periodicsync.NewPeriodicSyncDuration(h.peerCheckInterval, h.checkTimeout, h.peerLoop, log)
|
||||
h.cfg = a.MustComponent("config").(ConfigGetter).GetLimiterConf()
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *limiter) Name() (name string) {
|
||||
return CName
|
||||
}
|
||||
|
||||
func (h *limiter) peerLoop(ctx context.Context) error {
|
||||
h.mx.Lock()
|
||||
defer h.mx.Unlock()
|
||||
for rpcPeer, lim := range h.limiters {
|
||||
if time.Since(lim.lastUsage) > h.peerCheckInterval {
|
||||
delete(h.limiters, rpcPeer)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (h *limiter) WrapDRPCHandler(handler drpc.Handler) drpc.Handler {
|
||||
h.mx.Lock()
|
||||
defer h.mx.Unlock()
|
||||
h.Handler = handler
|
||||
return h
|
||||
}
|
||||
|
||||
func (h *limiter) HandleRPC(stream drpc.Stream, rpc string) (err error) {
|
||||
peerId, err := peer.CtxPeerId(stream.Context())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
lim := h.getPeerLimiter(peerId, rpc)
|
||||
if !lim.Allow() {
|
||||
return limiterproto.ErrLimitExceeded
|
||||
}
|
||||
return h.Handler.HandleRPC(stream, rpc)
|
||||
}
|
||||
|
||||
func (h *limiter) getLimits(rpc string) Tokens {
|
||||
if tokens, exists := h.cfg.ResponseTokens[rpc]; exists {
|
||||
return tokens
|
||||
}
|
||||
return h.cfg.DefaultTokens
|
||||
}
|
||||
|
||||
func (h *limiter) getPeerLimiter(peerId string, rpc string) *peerLimiter {
|
||||
// rpc looks like this /anyNodeSync.NodeSync/PartitionSync
|
||||
rpcPeer := strings.Join([]string{peerId, rpc}, "-")
|
||||
h.mx.Lock()
|
||||
defer h.mx.Unlock()
|
||||
lim, ok := h.limiters[rpcPeer]
|
||||
if !ok {
|
||||
limits := h.getLimits(rpc)
|
||||
lim = &peerLimiter{
|
||||
Limiter: rate.NewLimiter(rate.Limit(limits.TokensPerSecond), limits.MaxTokens),
|
||||
}
|
||||
h.limiters[rpcPeer] = lim
|
||||
}
|
||||
lim.lastUsage = time.Now()
|
||||
return lim
|
||||
}
|
181
net/rpc/limiter/limiter_test.go
Normal file
181
net/rpc/limiter/limiter_test.go
Normal file
|
@ -0,0 +1,181 @@
|
|||
package limiter
|
||||
|
||||
import (
|
||||
"context"
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
"storj.io/drpc"
|
||||
|
||||
"github.com/anyproto/any-sync/net/peer"
|
||||
"github.com/anyproto/any-sync/net/rpc/limiter/limiterproto"
|
||||
)
|
||||
|
||||
var ctx = context.Background()
|
||||
|
||||
type mockHandler struct {
|
||||
calls atomic.Int32
|
||||
}
|
||||
|
||||
type mockStream struct {
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
func (m mockStream) Context() context.Context {
|
||||
return m.ctx
|
||||
}
|
||||
|
||||
func (m mockStream) MsgSend(msg drpc.Message, enc drpc.Encoding) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m mockStream) MsgRecv(msg drpc.Message, enc drpc.Encoding) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m mockStream) CloseSend() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m mockStream) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *mockHandler) HandleRPC(stream drpc.Stream, rpc string) (err error) {
|
||||
m.calls.Add(1)
|
||||
return nil
|
||||
}
|
||||
|
||||
func TestLimiter_Synchronous(t *testing.T) {
|
||||
lim := New().(*limiter)
|
||||
handler := &mockHandler{}
|
||||
lim.cfg = Config{
|
||||
DefaultTokens: Tokens{
|
||||
TokensPerSecond: 100,
|
||||
MaxTokens: 100,
|
||||
},
|
||||
ResponseTokens: map[string]Tokens{
|
||||
"rpc": {
|
||||
TokensPerSecond: 10,
|
||||
MaxTokens: 1,
|
||||
},
|
||||
},
|
||||
}
|
||||
lim.peerCheckInterval = 10 * time.Millisecond
|
||||
wrapped := lim.WrapDRPCHandler(handler)
|
||||
// rpc call allows only one token max, so it should let only first call
|
||||
// for second one we should wait 100 ms
|
||||
firstStream := mockStream{ctx: peer.CtxWithPeerId(ctx, "peer1")}
|
||||
// check that we are using specific timeout
|
||||
err := wrapped.HandleRPC(firstStream, "rpc")
|
||||
require.NoError(t, err)
|
||||
err = wrapped.HandleRPC(firstStream, "rpc")
|
||||
require.Equal(t, limiterproto.ErrLimitExceeded, err)
|
||||
// second stream should not affect the first one
|
||||
secondStream := mockStream{ctx: peer.CtxWithPeerId(ctx, "peer2")}
|
||||
err = wrapped.HandleRPC(secondStream, "rpc")
|
||||
require.NoError(t, err)
|
||||
// after 100 ms new token has been generated
|
||||
time.Sleep(100 * time.Millisecond)
|
||||
err = wrapped.HandleRPC(firstStream, "rpc")
|
||||
require.NoError(t, err)
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
// checking that peer loop cleaned the map
|
||||
err = lim.peerLoop(ctx)
|
||||
require.NoError(t, err)
|
||||
// now we should be able to call again, because we cleared the map
|
||||
err = wrapped.HandleRPC(firstStream, "rpc")
|
||||
require.NoError(t, err)
|
||||
// but limit of 1 sec is not enough to clean the map
|
||||
time.Sleep(1 * time.Millisecond)
|
||||
err = wrapped.HandleRPC(firstStream, "rpc")
|
||||
require.Equal(t, limiterproto.ErrLimitExceeded, err)
|
||||
}
|
||||
|
||||
func TestLimiter_Concurrent_NoBursts(t *testing.T) {
|
||||
lim := New().(*limiter)
|
||||
handler := &mockHandler{}
|
||||
var (
|
||||
targetRps = 10
|
||||
// peerRps should be greater than targetRps
|
||||
peerRps = 100
|
||||
reqDelay = time.Duration(1000/peerRps) * time.Millisecond
|
||||
)
|
||||
lim.cfg = Config{
|
||||
DefaultTokens: Tokens{
|
||||
TokensPerSecond: targetRps,
|
||||
MaxTokens: 1,
|
||||
},
|
||||
}
|
||||
wrapped := lim.WrapDRPCHandler(handler)
|
||||
firstStream := mockStream{ctx: peer.CtxWithPeerId(ctx, "peer1")}
|
||||
secondStream := mockStream{ctx: peer.CtxWithPeerId(ctx, "peer2")}
|
||||
waitFirst := make(chan struct{})
|
||||
waitSecond := make(chan struct{})
|
||||
go func() {
|
||||
for i := 0; i < peerRps; i++ {
|
||||
time.Sleep(reqDelay)
|
||||
_ = wrapped.HandleRPC(firstStream, "rpc")
|
||||
}
|
||||
close(waitFirst)
|
||||
}()
|
||||
go func() {
|
||||
for i := 0; i < peerRps; i++ {
|
||||
time.Sleep(reqDelay)
|
||||
_ = wrapped.HandleRPC(secondStream, "rpc")
|
||||
}
|
||||
close(waitSecond)
|
||||
}()
|
||||
<-waitFirst
|
||||
<-waitSecond
|
||||
// 2 for number of peers and 2 for error margin (delays etc)
|
||||
maxCalls := 2 * 2 * targetRps
|
||||
require.Greater(t, maxCalls, int(handler.calls.Load()))
|
||||
}
|
||||
|
||||
func TestLimiter_Concurrent_Bursts(t *testing.T) {
|
||||
lim := New().(*limiter)
|
||||
handler := &mockHandler{}
|
||||
var (
|
||||
targetRps = 10
|
||||
// bursts are not affected by rps limit
|
||||
burst = 10
|
||||
// peerRps should be greater than targetRps + burst
|
||||
peerRps = 100
|
||||
reqDelay = time.Duration(1000/peerRps) * time.Millisecond
|
||||
)
|
||||
lim.cfg = Config{
|
||||
DefaultTokens: Tokens{
|
||||
TokensPerSecond: targetRps,
|
||||
MaxTokens: burst,
|
||||
},
|
||||
}
|
||||
wrapped := lim.WrapDRPCHandler(handler)
|
||||
firstStream := mockStream{ctx: peer.CtxWithPeerId(ctx, "peer1")}
|
||||
secondStream := mockStream{ctx: peer.CtxWithPeerId(ctx, "peer2")}
|
||||
waitFirst := make(chan struct{})
|
||||
waitSecond := make(chan struct{})
|
||||
go func() {
|
||||
for i := 0; i < peerRps; i++ {
|
||||
time.Sleep(reqDelay)
|
||||
_ = wrapped.HandleRPC(firstStream, "rpc")
|
||||
}
|
||||
close(waitFirst)
|
||||
}()
|
||||
go func() {
|
||||
for i := 0; i < peerRps; i++ {
|
||||
time.Sleep(reqDelay)
|
||||
_ = wrapped.HandleRPC(secondStream, "rpc")
|
||||
}
|
||||
close(waitSecond)
|
||||
}()
|
||||
<-waitFirst
|
||||
<-waitSecond
|
||||
// 2 for number of peers and 2 for error margin (delays etc)
|
||||
maxCalls := 2 * 2 * (targetRps + burst)
|
||||
minCalls := 2 * (targetRps + burst)
|
||||
require.Greater(t, maxCalls, int(handler.calls.Load()))
|
||||
require.LessOrEqual(t, minCalls, int(handler.calls.Load()))
|
||||
}
|
13
net/rpc/limiter/limiterproto/errors.go
Normal file
13
net/rpc/limiter/limiterproto/errors.go
Normal file
|
@ -0,0 +1,13 @@
|
|||
package limiterproto
|
||||
|
||||
import (
|
||||
"errors"
|
||||
|
||||
"github.com/anyproto/any-sync/net/rpc/rpcerr"
|
||||
)
|
||||
|
||||
var (
|
||||
errGroup = rpcerr.ErrGroup(ErrCodes_ErrorOffset)
|
||||
|
||||
ErrLimitExceeded = errGroup.Register(errors.New("rate limit exceeded"), uint64(ErrCodes_RateLimitExceeded))
|
||||
)
|
68
net/rpc/limiter/limiterproto/limiter.pb.go
Normal file
68
net/rpc/limiter/limiterproto/limiter.pb.go
Normal file
|
@ -0,0 +1,68 @@
|
|||
// Code generated by protoc-gen-gogo. DO NOT EDIT.
|
||||
// source: net/rpc/limiter/limiterproto/protos/limiter.proto
|
||||
|
||||
package limiterproto
|
||||
|
||||
import (
|
||||
fmt "fmt"
|
||||
proto "github.com/gogo/protobuf/proto"
|
||||
math "math"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
var _ = fmt.Errorf
|
||||
var _ = math.Inf
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the proto package it is being compiled against.
|
||||
// A compilation error at this line likely means your copy of the
|
||||
// proto package needs to be updated.
|
||||
const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package
|
||||
|
||||
type ErrCodes int32
|
||||
|
||||
const (
|
||||
ErrCodes_RateLimitExceeded ErrCodes = 0
|
||||
ErrCodes_ErrorOffset ErrCodes = 700
|
||||
)
|
||||
|
||||
var ErrCodes_name = map[int32]string{
|
||||
0: "RateLimitExceeded",
|
||||
700: "ErrorOffset",
|
||||
}
|
||||
|
||||
var ErrCodes_value = map[string]int32{
|
||||
"RateLimitExceeded": 0,
|
||||
"ErrorOffset": 700,
|
||||
}
|
||||
|
||||
func (x ErrCodes) String() string {
|
||||
return proto.EnumName(ErrCodes_name, int32(x))
|
||||
}
|
||||
|
||||
func (ErrCodes) EnumDescriptor() ([]byte, []int) {
|
||||
return fileDescriptor_43f29163996a95d8, []int{0}
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterEnum("limiter.ErrCodes", ErrCodes_name, ErrCodes_value)
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterFile("net/rpc/limiter/limiterproto/protos/limiter.proto", fileDescriptor_43f29163996a95d8)
|
||||
}
|
||||
|
||||
var fileDescriptor_43f29163996a95d8 = []byte{
|
||||
// 147 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x32, 0xcc, 0x4b, 0x2d, 0xd1,
|
||||
0x2f, 0x2a, 0x48, 0xd6, 0xcf, 0xc9, 0xcc, 0xcd, 0x2c, 0x49, 0x2d, 0x82, 0xd1, 0x05, 0x45, 0xf9,
|
||||
0x25, 0xf9, 0xfa, 0x60, 0xb2, 0x18, 0x26, 0xa6, 0x07, 0xe6, 0x0a, 0xb1, 0x43, 0xb9, 0x5a, 0xc6,
|
||||
0x5c, 0x1c, 0xae, 0x45, 0x45, 0xce, 0xf9, 0x29, 0xa9, 0xc5, 0x42, 0xa2, 0x5c, 0x82, 0x41, 0x89,
|
||||
0x25, 0xa9, 0x3e, 0x20, 0x29, 0xd7, 0x8a, 0xe4, 0xd4, 0xd4, 0x94, 0xd4, 0x14, 0x01, 0x06, 0x21,
|
||||
0x01, 0x2e, 0x6e, 0xd7, 0xa2, 0xa2, 0xfc, 0x22, 0xff, 0xb4, 0xb4, 0xe2, 0xd4, 0x12, 0x81, 0x3d,
|
||||
0xac, 0x4e, 0x66, 0x27, 0x1e, 0xc9, 0x31, 0x5e, 0x78, 0x24, 0xc7, 0xf8, 0xe0, 0x91, 0x1c, 0xe3,
|
||||
0x84, 0xc7, 0x72, 0x0c, 0x17, 0x1e, 0xcb, 0x31, 0xdc, 0x78, 0x2c, 0xc7, 0x10, 0x25, 0x83, 0xcf,
|
||||
0x29, 0x49, 0x6c, 0x60, 0xca, 0x18, 0x10, 0x00, 0x00, 0xff, 0xff, 0x1e, 0x6b, 0xb3, 0x13, 0xb1,
|
||||
0x00, 0x00, 0x00,
|
||||
}
|
8
net/rpc/limiter/limiterproto/protos/limiter.proto
Normal file
8
net/rpc/limiter/limiterproto/protos/limiter.proto
Normal file
|
@ -0,0 +1,8 @@
|
|||
syntax = "proto3";
|
||||
package limiter;
|
||||
option go_package = "net/rpc/limiter/limiterproto";
|
||||
|
||||
enum ErrCodes {
|
||||
RateLimitExceeded = 0;
|
||||
ErrorOffset = 700;
|
||||
}
|
111
net/rpc/limiter/mock_limiter/mock_limiter.go
Normal file
111
net/rpc/limiter/mock_limiter/mock_limiter.go
Normal file
|
@ -0,0 +1,111 @@
|
|||
// Code generated by MockGen. DO NOT EDIT.
|
||||
// Source: github.com/anyproto/any-sync/net/rpc/limiter (interfaces: RpcLimiter)
|
||||
//
|
||||
// Generated by this command:
|
||||
//
|
||||
// mockgen -destination mock_limiter/mock_limiter.go github.com/anyproto/any-sync/net/rpc/limiter RpcLimiter
|
||||
//
|
||||
// Package mock_limiter is a generated GoMock package.
|
||||
package mock_limiter
|
||||
|
||||
import (
|
||||
context "context"
|
||||
reflect "reflect"
|
||||
|
||||
app "github.com/anyproto/any-sync/app"
|
||||
gomock "go.uber.org/mock/gomock"
|
||||
drpc "storj.io/drpc"
|
||||
)
|
||||
|
||||
// MockRpcLimiter is a mock of RpcLimiter interface.
|
||||
type MockRpcLimiter struct {
|
||||
ctrl *gomock.Controller
|
||||
recorder *MockRpcLimiterMockRecorder
|
||||
}
|
||||
|
||||
// MockRpcLimiterMockRecorder is the mock recorder for MockRpcLimiter.
|
||||
type MockRpcLimiterMockRecorder struct {
|
||||
mock *MockRpcLimiter
|
||||
}
|
||||
|
||||
// NewMockRpcLimiter creates a new mock instance.
|
||||
func NewMockRpcLimiter(ctrl *gomock.Controller) *MockRpcLimiter {
|
||||
mock := &MockRpcLimiter{ctrl: ctrl}
|
||||
mock.recorder = &MockRpcLimiterMockRecorder{mock}
|
||||
return mock
|
||||
}
|
||||
|
||||
// EXPECT returns an object that allows the caller to indicate expected use.
|
||||
func (m *MockRpcLimiter) EXPECT() *MockRpcLimiterMockRecorder {
|
||||
return m.recorder
|
||||
}
|
||||
|
||||
// Close mocks base method.
|
||||
func (m *MockRpcLimiter) Close(arg0 context.Context) error {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "Close", arg0)
|
||||
ret0, _ := ret[0].(error)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// Close indicates an expected call of Close.
|
||||
func (mr *MockRpcLimiterMockRecorder) Close(arg0 any) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockRpcLimiter)(nil).Close), arg0)
|
||||
}
|
||||
|
||||
// Init mocks base method.
|
||||
func (m *MockRpcLimiter) Init(arg0 *app.App) error {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "Init", arg0)
|
||||
ret0, _ := ret[0].(error)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// Init indicates an expected call of Init.
|
||||
func (mr *MockRpcLimiterMockRecorder) Init(arg0 any) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Init", reflect.TypeOf((*MockRpcLimiter)(nil).Init), arg0)
|
||||
}
|
||||
|
||||
// Name mocks base method.
|
||||
func (m *MockRpcLimiter) Name() string {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "Name")
|
||||
ret0, _ := ret[0].(string)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// Name indicates an expected call of Name.
|
||||
func (mr *MockRpcLimiterMockRecorder) Name() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Name", reflect.TypeOf((*MockRpcLimiter)(nil).Name))
|
||||
}
|
||||
|
||||
// Run mocks base method.
|
||||
func (m *MockRpcLimiter) Run(arg0 context.Context) error {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "Run", arg0)
|
||||
ret0, _ := ret[0].(error)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// Run indicates an expected call of Run.
|
||||
func (mr *MockRpcLimiterMockRecorder) Run(arg0 any) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Run", reflect.TypeOf((*MockRpcLimiter)(nil).Run), arg0)
|
||||
}
|
||||
|
||||
// WrapDRPCHandler mocks base method.
|
||||
func (m *MockRpcLimiter) WrapDRPCHandler(arg0 drpc.Handler) drpc.Handler {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "WrapDRPCHandler", arg0)
|
||||
ret0, _ := ret[0].(drpc.Handler)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// WrapDRPCHandler indicates an expected call of WrapDRPCHandler.
|
||||
func (mr *MockRpcLimiterMockRecorder) WrapDRPCHandler(arg0 any) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WrapDRPCHandler", reflect.TypeOf((*MockRpcLimiter)(nil).WrapDRPCHandler), arg0)
|
||||
}
|
|
@ -43,7 +43,7 @@ func Unwrap(e error) error {
|
|||
}
|
||||
err, ok := errsMap[code]
|
||||
if !ok {
|
||||
return drpcerr.WithCode(fmt.Errorf("unexpected error: %v; code: %d", err, code), code)
|
||||
return drpcerr.WithCode(fmt.Errorf("unexpected error: %w; code: %d", e, code), code)
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -2,12 +2,15 @@ package server
|
|||
|
||||
import (
|
||||
"context"
|
||||
"net"
|
||||
|
||||
"github.com/anyproto/any-sync/app"
|
||||
"github.com/anyproto/any-sync/app/logger"
|
||||
"github.com/anyproto/any-sync/metric"
|
||||
"github.com/anyproto/any-sync/net/rpc"
|
||||
"github.com/anyproto/any-sync/net/rpc/limiter"
|
||||
|
||||
"go.uber.org/zap"
|
||||
"net"
|
||||
"storj.io/drpc"
|
||||
"storj.io/drpc/drpcmanager"
|
||||
"storj.io/drpc/drpcmux"
|
||||
|
@ -34,8 +37,9 @@ type DRPCServer interface {
|
|||
type drpcServer struct {
|
||||
drpcServer *drpcserver.Server
|
||||
*drpcmux.Mux
|
||||
config rpc.Config
|
||||
metric metric.Metric
|
||||
config rpc.Config
|
||||
metric metric.Metric
|
||||
limiter limiter.RpcLimiter
|
||||
}
|
||||
|
||||
type DRPCHandlerWrapper func(handler drpc.Handler) drpc.Handler
|
||||
|
@ -47,12 +51,16 @@ func (s *drpcServer) Name() (name string) {
|
|||
func (s *drpcServer) Init(a *app.App) (err error) {
|
||||
s.config = a.MustComponent("config").(rpc.ConfigGetter).GetDrpc()
|
||||
s.metric, _ = a.Component(metric.CName).(metric.Metric)
|
||||
s.limiter, _ = a.Component(limiter.CName).(limiter.RpcLimiter)
|
||||
s.Mux = drpcmux.New()
|
||||
|
||||
var handler drpc.Handler
|
||||
handler = s
|
||||
if s.limiter != nil {
|
||||
handler = s.limiter.WrapDRPCHandler(handler)
|
||||
}
|
||||
if s.metric != nil {
|
||||
handler = s.metric.WrapDRPCHandler(s)
|
||||
handler = s.metric.WrapDRPCHandler(handler)
|
||||
}
|
||||
bufSize := s.config.Stream.MaxMsgSizeMb * (1 << 20)
|
||||
s.drpcServer = drpcserver.NewWithOptions(handler, drpcserver.Options{Manager: drpcmanager.Options{
|
||||
|
|
99
node/nodeclient/mock_nodeclient/mock_nodeclient.go
Normal file
99
node/nodeclient/mock_nodeclient/mock_nodeclient.go
Normal file
|
@ -0,0 +1,99 @@
|
|||
// Code generated by MockGen. DO NOT EDIT.
|
||||
// Source: github.com/anyproto/any-sync/node/nodeclient (interfaces: NodeClient)
|
||||
//
|
||||
// Generated by this command:
|
||||
//
|
||||
// mockgen -destination mock_nodeclient/mock_nodeclient.go github.com/anyproto/any-sync/node/nodeclient NodeClient
|
||||
//
|
||||
// Package mock_nodeclient is a generated GoMock package.
|
||||
package mock_nodeclient
|
||||
|
||||
import (
|
||||
context "context"
|
||||
reflect "reflect"
|
||||
|
||||
app "github.com/anyproto/any-sync/app"
|
||||
consensusproto "github.com/anyproto/any-sync/consensus/consensusproto"
|
||||
gomock "go.uber.org/mock/gomock"
|
||||
)
|
||||
|
||||
// MockNodeClient is a mock of NodeClient interface.
|
||||
type MockNodeClient struct {
|
||||
ctrl *gomock.Controller
|
||||
recorder *MockNodeClientMockRecorder
|
||||
}
|
||||
|
||||
// MockNodeClientMockRecorder is the mock recorder for MockNodeClient.
|
||||
type MockNodeClientMockRecorder struct {
|
||||
mock *MockNodeClient
|
||||
}
|
||||
|
||||
// NewMockNodeClient creates a new mock instance.
|
||||
func NewMockNodeClient(ctrl *gomock.Controller) *MockNodeClient {
|
||||
mock := &MockNodeClient{ctrl: ctrl}
|
||||
mock.recorder = &MockNodeClientMockRecorder{mock}
|
||||
return mock
|
||||
}
|
||||
|
||||
// EXPECT returns an object that allows the caller to indicate expected use.
|
||||
func (m *MockNodeClient) EXPECT() *MockNodeClientMockRecorder {
|
||||
return m.recorder
|
||||
}
|
||||
|
||||
// AclAddRecord mocks base method.
|
||||
func (m *MockNodeClient) AclAddRecord(arg0 context.Context, arg1 string, arg2 *consensusproto.RawRecord) (*consensusproto.RawRecordWithId, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "AclAddRecord", arg0, arg1, arg2)
|
||||
ret0, _ := ret[0].(*consensusproto.RawRecordWithId)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// AclAddRecord indicates an expected call of AclAddRecord.
|
||||
func (mr *MockNodeClientMockRecorder) AclAddRecord(arg0, arg1, arg2 any) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AclAddRecord", reflect.TypeOf((*MockNodeClient)(nil).AclAddRecord), arg0, arg1, arg2)
|
||||
}
|
||||
|
||||
// AclGetRecords mocks base method.
|
||||
func (m *MockNodeClient) AclGetRecords(arg0 context.Context, arg1, arg2 string) ([]*consensusproto.RawRecordWithId, error) {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "AclGetRecords", arg0, arg1, arg2)
|
||||
ret0, _ := ret[0].([]*consensusproto.RawRecordWithId)
|
||||
ret1, _ := ret[1].(error)
|
||||
return ret0, ret1
|
||||
}
|
||||
|
||||
// AclGetRecords indicates an expected call of AclGetRecords.
|
||||
func (mr *MockNodeClientMockRecorder) AclGetRecords(arg0, arg1, arg2 any) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AclGetRecords", reflect.TypeOf((*MockNodeClient)(nil).AclGetRecords), arg0, arg1, arg2)
|
||||
}
|
||||
|
||||
// Init mocks base method.
|
||||
func (m *MockNodeClient) Init(arg0 *app.App) error {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "Init", arg0)
|
||||
ret0, _ := ret[0].(error)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// Init indicates an expected call of Init.
|
||||
func (mr *MockNodeClientMockRecorder) Init(arg0 any) *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Init", reflect.TypeOf((*MockNodeClient)(nil).Init), arg0)
|
||||
}
|
||||
|
||||
// Name mocks base method.
|
||||
func (m *MockNodeClient) Name() string {
|
||||
m.ctrl.T.Helper()
|
||||
ret := m.ctrl.Call(m, "Name")
|
||||
ret0, _ := ret[0].(string)
|
||||
return ret0
|
||||
}
|
||||
|
||||
// Name indicates an expected call of Name.
|
||||
func (mr *MockNodeClientMockRecorder) Name() *gomock.Call {
|
||||
mr.mock.ctrl.T.Helper()
|
||||
return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Name", reflect.TypeOf((*MockNodeClient)(nil).Name))
|
||||
}
|
98
node/nodeclient/nodeclient.go
Normal file
98
node/nodeclient/nodeclient.go
Normal file
|
@ -0,0 +1,98 @@
|
|||
//go:generate mockgen -destination mock_nodeclient/mock_nodeclient.go github.com/anyproto/any-sync/node/nodeclient NodeClient
|
||||
package nodeclient
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"storj.io/drpc"
|
||||
|
||||
"github.com/anyproto/any-sync/app"
|
||||
"github.com/anyproto/any-sync/commonspace/spacesyncproto"
|
||||
"github.com/anyproto/any-sync/consensus/consensusproto"
|
||||
"github.com/anyproto/any-sync/net/pool"
|
||||
"github.com/anyproto/any-sync/net/rpc/rpcerr"
|
||||
"github.com/anyproto/any-sync/nodeconf"
|
||||
)
|
||||
|
||||
const CName = "common.node.nodeclient"
|
||||
|
||||
func New() NodeClient {
|
||||
return &nodeClient{}
|
||||
}
|
||||
|
||||
type NodeClient interface {
|
||||
app.Component
|
||||
AclGetRecords(ctx context.Context, spaceId, aclHead string) (recs []*consensusproto.RawRecordWithId, err error)
|
||||
AclAddRecord(ctx context.Context, spaceId string, rec *consensusproto.RawRecord) (recWithId *consensusproto.RawRecordWithId, err error)
|
||||
}
|
||||
|
||||
type nodeClient struct {
|
||||
pool pool.Service
|
||||
nodeConf nodeconf.Service
|
||||
}
|
||||
|
||||
func (c *nodeClient) Init(a *app.App) (err error) {
|
||||
c.pool = a.MustComponent(pool.CName).(pool.Service)
|
||||
c.nodeConf = a.MustComponent(nodeconf.CName).(nodeconf.Service)
|
||||
return
|
||||
}
|
||||
|
||||
func (c *nodeClient) Name() (name string) {
|
||||
return CName
|
||||
}
|
||||
|
||||
func (c *nodeClient) AclGetRecords(ctx context.Context, spaceId, aclHead string) (recs []*consensusproto.RawRecordWithId, err error) {
|
||||
err = clientDo(c, ctx, spaceId, func(cl spacesyncproto.DRPCSpaceSyncClient) error {
|
||||
resp, err := cl.AclGetRecords(ctx, &spacesyncproto.AclGetRecordsRequest{
|
||||
SpaceId: spaceId,
|
||||
AclHead: aclHead,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
recs = make([]*consensusproto.RawRecordWithId, len(resp.Records))
|
||||
for i, rec := range resp.Records {
|
||||
recs[i] = &consensusproto.RawRecordWithId{}
|
||||
if err = recs[i].Unmarshal(rec); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
})
|
||||
return
|
||||
}
|
||||
|
||||
func (c *nodeClient) AclAddRecord(ctx context.Context, spaceId string, rec *consensusproto.RawRecord) (recWithId *consensusproto.RawRecordWithId, err error) {
|
||||
data, err := rec.Marshal()
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
err = clientDo(c, ctx, spaceId, func(cl spacesyncproto.DRPCSpaceSyncClient) error {
|
||||
res, err := cl.AclAddRecord(ctx, &spacesyncproto.AclAddRecordRequest{
|
||||
SpaceId: spaceId,
|
||||
Payload: data,
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
recWithId = &consensusproto.RawRecordWithId{
|
||||
Payload: res.Payload,
|
||||
Id: res.RecordId,
|
||||
}
|
||||
return nil
|
||||
})
|
||||
return
|
||||
}
|
||||
|
||||
var clientDo = (*nodeClient).doClient
|
||||
|
||||
func (c *nodeClient) doClient(ctx context.Context, spaceId string, f func(cl spacesyncproto.DRPCSpaceSyncClient) error) error {
|
||||
p, err := c.pool.GetOneOf(ctx, c.nodeConf.NodeIds(spaceId))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return p.DoDrpc(ctx, func(conn drpc.Conn) error {
|
||||
err := f(spacesyncproto.NewDRPCSpaceSyncClient(conn))
|
||||
return rpcerr.Unwrap(err)
|
||||
})
|
||||
}
|
92
node/nodeclient/nodeclient_test.go
Normal file
92
node/nodeclient/nodeclient_test.go
Normal file
|
@ -0,0 +1,92 @@
|
|||
package nodeclient
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
"go.uber.org/mock/gomock"
|
||||
|
||||
"github.com/anyproto/any-sync/commonspace/spacesyncproto"
|
||||
"github.com/anyproto/any-sync/commonspace/spacesyncproto/mock_spacesyncproto"
|
||||
"github.com/anyproto/any-sync/consensus/consensusproto"
|
||||
)
|
||||
|
||||
var ctx = context.Background()
|
||||
|
||||
type fixture struct {
|
||||
*nodeClient
|
||||
ctrl *gomock.Controller
|
||||
}
|
||||
|
||||
func (f *fixture) finish() {
|
||||
f.ctrl.Finish()
|
||||
}
|
||||
|
||||
func newFixture(t *testing.T) *fixture {
|
||||
ctrl := gomock.NewController(t)
|
||||
return &fixture{
|
||||
nodeClient: New().(*nodeClient),
|
||||
ctrl: ctrl,
|
||||
}
|
||||
}
|
||||
|
||||
func TestNodeClient_AclGetRecords(t *testing.T) {
|
||||
f := newFixture(t)
|
||||
defer f.finish()
|
||||
|
||||
spaceId := "spaceId"
|
||||
aclHead := "aclHead"
|
||||
cl := mock_spacesyncproto.NewMockDRPCSpaceSyncClient(f.ctrl)
|
||||
clientDo = func(client *nodeClient, ctx context.Context, s string, f func(cl spacesyncproto.DRPCSpaceSyncClient) error) error {
|
||||
return f(cl)
|
||||
}
|
||||
var (
|
||||
expectedRecs []*consensusproto.RawRecordWithId
|
||||
expectedByteRecs [][]byte
|
||||
total = 5
|
||||
)
|
||||
for i := 0; i < total; i++ {
|
||||
expectedRecs = append(expectedRecs, &consensusproto.RawRecordWithId{
|
||||
Id: fmt.Sprint(i),
|
||||
})
|
||||
marshalled, err := expectedRecs[i].Marshal()
|
||||
require.NoError(t, err)
|
||||
expectedByteRecs = append(expectedByteRecs, marshalled)
|
||||
}
|
||||
cl.EXPECT().AclGetRecords(ctx, &spacesyncproto.AclGetRecordsRequest{
|
||||
SpaceId: spaceId,
|
||||
AclHead: aclHead,
|
||||
}).Return(&spacesyncproto.AclGetRecordsResponse{Records: expectedByteRecs}, nil)
|
||||
recs, err := f.AclGetRecords(ctx, spaceId, aclHead)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, expectedRecs, recs)
|
||||
}
|
||||
|
||||
func TestNodeClient_AclAddRecords(t *testing.T) {
|
||||
f := newFixture(t)
|
||||
defer f.finish()
|
||||
|
||||
spaceId := "spaceId"
|
||||
cl := mock_spacesyncproto.NewMockDRPCSpaceSyncClient(f.ctrl)
|
||||
clientDo = func(client *nodeClient, ctx context.Context, s string, f func(cl spacesyncproto.DRPCSpaceSyncClient) error) error {
|
||||
return f(cl)
|
||||
}
|
||||
sendRec := &consensusproto.RawRecord{
|
||||
AcceptorTimestamp: 10,
|
||||
}
|
||||
data, err := sendRec.Marshal()
|
||||
require.NoError(t, err)
|
||||
expectedRec := &consensusproto.RawRecordWithId{
|
||||
Id: "recId",
|
||||
Payload: data,
|
||||
}
|
||||
cl.EXPECT().AclAddRecord(ctx, &spacesyncproto.AclAddRecordRequest{
|
||||
SpaceId: spaceId,
|
||||
Payload: data,
|
||||
}).Return(&spacesyncproto.AclAddRecordResponse{RecordId: expectedRec.Id, Payload: expectedRec.Payload}, nil)
|
||||
rec, err := f.AclAddRecord(ctx, spaceId, sendRec)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, expectedRec, rec)
|
||||
}
|
29
paymentservice/paymentserviceproto/errors.go
Normal file
29
paymentservice/paymentserviceproto/errors.go
Normal file
|
@ -0,0 +1,29 @@
|
|||
package paymentserviceproto
|
||||
|
||||
import (
|
||||
"errors"
|
||||
|
||||
"github.com/anyproto/any-sync/net/rpc/rpcerr"
|
||||
)
|
||||
|
||||
var (
|
||||
errGroup = rpcerr.ErrGroup(ErrorCodes_ErrorOffset)
|
||||
|
||||
ErrEthAddressEmpty = errGroup.Register(errors.New("owner eth address is empty"), uint64(ErrorCodes_EthAddressEmpty))
|
||||
ErrInvalidSignature = errGroup.Register(errors.New("invalid signature"), uint64(ErrorCodes_InvalidSignature))
|
||||
ErrTierWrong = errGroup.Register(errors.New("wrong tier specified"), uint64(ErrorCodes_TierWrong))
|
||||
ErrTierNotFound = errGroup.Register(errors.New("requested tier not found"), uint64(ErrorCodes_TierNotFound))
|
||||
ErrTierInactive = errGroup.Register(errors.New("requested tier is not active"), uint64(ErrorCodes_TierInactive))
|
||||
ErrPaymentMethodWrong = errGroup.Register(errors.New("wrong payment method specified"), uint64(ErrorCodes_PaymentMethodWrong))
|
||||
ErrBadAnyName = errGroup.Register(errors.New("bad any name specified"), uint64(ErrorCodes_BadAnyName))
|
||||
ErrUnknown = errGroup.Register(errors.New("unknown error"), uint64(ErrorCodes_Unknown))
|
||||
ErrSubsAlreadyActive = errGroup.Register(errors.New("user already has an active subscription"), uint64(ErrorCodes_SubsAlreadyActive))
|
||||
ErrSubsNotFound = errGroup.Register(errors.New("no subscription for user"), uint64(ErrorCodes_SubsNotFound))
|
||||
ErrSubsWrongState = errGroup.Register(errors.New("subscription is not in PendingRequiresFinalization state"), uint64(ErrorCodes_SubsWrongState))
|
||||
ErrEmailWrongFormat = errGroup.Register(errors.New("wrong email format"), uint64(ErrorCodes_EmailWrongFormat))
|
||||
ErrEmailAlreadyVerified = errGroup.Register(errors.New("email already verified"), uint64(ErrorCodes_EmailAlreadyVerified))
|
||||
ErrEmailAlreadySent = errGroup.Register(errors.New("email verification request already sent. wait for the email or try again later"), uint64(ErrorCodes_EmailAlreadySent))
|
||||
ErrEmailFailedToSend = errGroup.Register(errors.New("failed to send email"), uint64(ErrorCodes_EmailFailedToSend))
|
||||
ErrEmailExpired = errGroup.Register(errors.New("email verification request expired. try getting new code"), uint64(ErrorCodes_EmailExpired))
|
||||
ErrEmailWrongCode = errGroup.Register(errors.New("wrong verification code"), uint64(ErrorCodes_EmailWrongCode))
|
||||
)
|
|
@ -146,6 +146,82 @@ func (PaymentMethod) EnumDescriptor() ([]byte, []int) {
|
|||
return fileDescriptor_4feb29dcc5ba50f6, []int{2}
|
||||
}
|
||||
|
||||
type ErrorCodes int32
|
||||
|
||||
const (
|
||||
ErrorCodes_Unexpected ErrorCodes = 0
|
||||
ErrorCodes_EthAddressEmpty ErrorCodes = 1
|
||||
ErrorCodes_InvalidSignature ErrorCodes = 2
|
||||
ErrorCodes_TierWrong ErrorCodes = 3
|
||||
ErrorCodes_TierNotFound ErrorCodes = 4
|
||||
ErrorCodes_TierInactive ErrorCodes = 5
|
||||
ErrorCodes_PaymentMethodWrong ErrorCodes = 6
|
||||
ErrorCodes_BadAnyName ErrorCodes = 7
|
||||
ErrorCodes_Unknown ErrorCodes = 8
|
||||
ErrorCodes_SubsAlreadyActive ErrorCodes = 9
|
||||
ErrorCodes_SubsNotFound ErrorCodes = 10
|
||||
ErrorCodes_SubsWrongState ErrorCodes = 11
|
||||
ErrorCodes_EmailWrongFormat ErrorCodes = 12
|
||||
ErrorCodes_EmailAlreadyVerified ErrorCodes = 13
|
||||
ErrorCodes_EmailAlreadySent ErrorCodes = 14
|
||||
ErrorCodes_EmailFailedToSend ErrorCodes = 15
|
||||
ErrorCodes_EmailExpired ErrorCodes = 16
|
||||
ErrorCodes_EmailWrongCode ErrorCodes = 17
|
||||
ErrorCodes_ErrorOffset ErrorCodes = 600
|
||||
)
|
||||
|
||||
var ErrorCodes_name = map[int32]string{
|
||||
0: "Unexpected",
|
||||
1: "EthAddressEmpty",
|
||||
2: "InvalidSignature",
|
||||
3: "TierWrong",
|
||||
4: "TierNotFound",
|
||||
5: "TierInactive",
|
||||
6: "PaymentMethodWrong",
|
||||
7: "BadAnyName",
|
||||
8: "Unknown",
|
||||
9: "SubsAlreadyActive",
|
||||
10: "SubsNotFound",
|
||||
11: "SubsWrongState",
|
||||
12: "EmailWrongFormat",
|
||||
13: "EmailAlreadyVerified",
|
||||
14: "EmailAlreadySent",
|
||||
15: "EmailFailedToSend",
|
||||
16: "EmailExpired",
|
||||
17: "EmailWrongCode",
|
||||
600: "ErrorOffset",
|
||||
}
|
||||
|
||||
var ErrorCodes_value = map[string]int32{
|
||||
"Unexpected": 0,
|
||||
"EthAddressEmpty": 1,
|
||||
"InvalidSignature": 2,
|
||||
"TierWrong": 3,
|
||||
"TierNotFound": 4,
|
||||
"TierInactive": 5,
|
||||
"PaymentMethodWrong": 6,
|
||||
"BadAnyName": 7,
|
||||
"Unknown": 8,
|
||||
"SubsAlreadyActive": 9,
|
||||
"SubsNotFound": 10,
|
||||
"SubsWrongState": 11,
|
||||
"EmailWrongFormat": 12,
|
||||
"EmailAlreadyVerified": 13,
|
||||
"EmailAlreadySent": 14,
|
||||
"EmailFailedToSend": 15,
|
||||
"EmailExpired": 16,
|
||||
"EmailWrongCode": 17,
|
||||
"ErrorOffset": 600,
|
||||
}
|
||||
|
||||
func (x ErrorCodes) String() string {
|
||||
return proto.EnumName(ErrorCodes_name, int32(x))
|
||||
}
|
||||
|
||||
func (ErrorCodes) EnumDescriptor() ([]byte, []int) {
|
||||
return fileDescriptor_4feb29dcc5ba50f6, []int{3}
|
||||
}
|
||||
|
||||
type IsNameValidResponse_Code int32
|
||||
|
||||
const (
|
||||
|
@ -284,8 +360,8 @@ func (m *GetSubscriptionRequestSigned) GetSignature() []byte {
|
|||
}
|
||||
|
||||
type GetSubscriptionResponse struct {
|
||||
// was SubscriptionTier before, changed to int32 to allow us to use dynamic tiers
|
||||
Tier int32 `protobuf:"varint,1,opt,name=tier,proto3" json:"tier,omitempty"`
|
||||
// was SubscriptionTier before, changed to uint32 to allow us to use dynamic tiers
|
||||
Tier uint32 `protobuf:"varint,1,opt,name=tier,proto3" json:"tier,omitempty"`
|
||||
Status SubscriptionStatus `protobuf:"varint,2,opt,name=status,proto3,enum=SubscriptionStatus" json:"status,omitempty"`
|
||||
DateStarted uint64 `protobuf:"varint,3,opt,name=dateStarted,proto3" json:"dateStarted,omitempty"`
|
||||
DateEnds uint64 `protobuf:"varint,4,opt,name=dateEnds,proto3" json:"dateEnds,omitempty"`
|
||||
|
@ -330,7 +406,7 @@ func (m *GetSubscriptionResponse) XXX_DiscardUnknown() {
|
|||
|
||||
var xxx_messageInfo_GetSubscriptionResponse proto.InternalMessageInfo
|
||||
|
||||
func (m *GetSubscriptionResponse) GetTier() int32 {
|
||||
func (m *GetSubscriptionResponse) GetTier() uint32 {
|
||||
if m != nil {
|
||||
return m.Tier
|
||||
}
|
||||
|
@ -402,8 +478,8 @@ type BuySubscriptionRequest struct {
|
|||
// this is required to reserve a name for the owner (later that is done by user)
|
||||
// in the following format: "0x7a250d5630b4cf539739df2c5dacb4c659f2488d"
|
||||
OwnerEthAddress string `protobuf:"bytes,2,opt,name=ownerEthAddress,proto3" json:"ownerEthAddress,omitempty"`
|
||||
// was SubscriptionTier before, changed to int32 to allow us to use dynamic tiers
|
||||
RequestedTier int32 `protobuf:"varint,3,opt,name=requestedTier,proto3" json:"requestedTier,omitempty"`
|
||||
// was SubscriptionTier before, changed to uint32 to allow us to use dynamic tiers
|
||||
RequestedTier uint32 `protobuf:"varint,3,opt,name=requestedTier,proto3" json:"requestedTier,omitempty"`
|
||||
PaymentMethod PaymentMethod `protobuf:"varint,4,opt,name=paymentMethod,proto3,enum=PaymentMethod" json:"paymentMethod,omitempty"`
|
||||
// if empty - then no name requested
|
||||
// if non-empty - PP node will register that name on behalf of the user
|
||||
|
@ -457,7 +533,7 @@ func (m *BuySubscriptionRequest) GetOwnerEthAddress() string {
|
|||
return ""
|
||||
}
|
||||
|
||||
func (m *BuySubscriptionRequest) GetRequestedTier() int32 {
|
||||
func (m *BuySubscriptionRequest) GetRequestedTier() uint32 {
|
||||
if m != nil {
|
||||
return m.RequestedTier
|
||||
}
|
||||
|
@ -536,6 +612,8 @@ type BuySubscriptionResponse struct {
|
|||
// will feature current billing ID
|
||||
// stripe.com/?client_reference_id=1234
|
||||
PaymentUrl string `protobuf:"bytes,1,opt,name=paymentUrl,proto3" json:"paymentUrl,omitempty"`
|
||||
// billingID is passed via mobile client to payment platform
|
||||
BillingID string `protobuf:"bytes,2,opt,name=billingID,proto3" json:"billingID,omitempty"`
|
||||
}
|
||||
|
||||
func (m *BuySubscriptionResponse) Reset() { *m = BuySubscriptionResponse{} }
|
||||
|
@ -578,6 +656,13 @@ func (m *BuySubscriptionResponse) GetPaymentUrl() string {
|
|||
return ""
|
||||
}
|
||||
|
||||
func (m *BuySubscriptionResponse) GetBillingID() string {
|
||||
if m != nil {
|
||||
return m.BillingID
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type FinalizeSubscriptionRequest struct {
|
||||
// in the following format: "A5k2d9sFZw84yisTxRnz2bPRd1YPfVfhxqymZ6yESprFTG65"
|
||||
// you can get it with Account().SignKey.GetPublic().Account()
|
||||
|
@ -1199,7 +1284,7 @@ func (m *VerifyEmailRequestSigned) GetSignature() []byte {
|
|||
}
|
||||
|
||||
type IsNameValidRequest struct {
|
||||
RequestedTier int32 `protobuf:"varint,1,opt,name=requestedTier,proto3" json:"requestedTier,omitempty"`
|
||||
RequestedTier uint32 `protobuf:"varint,1,opt,name=requestedTier,proto3" json:"requestedTier,omitempty"`
|
||||
RequestedAnyName string `protobuf:"bytes,2,opt,name=requestedAnyName,proto3" json:"requestedAnyName,omitempty"`
|
||||
}
|
||||
|
||||
|
@ -1236,7 +1321,7 @@ func (m *IsNameValidRequest) XXX_DiscardUnknown() {
|
|||
|
||||
var xxx_messageInfo_IsNameValidRequest proto.InternalMessageInfo
|
||||
|
||||
func (m *IsNameValidRequest) GetRequestedTier() int32 {
|
||||
func (m *IsNameValidRequest) GetRequestedTier() uint32 {
|
||||
if m != nil {
|
||||
return m.RequestedTier
|
||||
}
|
||||
|
@ -1306,6 +1391,7 @@ func init() {
|
|||
proto.RegisterEnum("SubscriptionTier", SubscriptionTier_name, SubscriptionTier_value)
|
||||
proto.RegisterEnum("SubscriptionStatus", SubscriptionStatus_name, SubscriptionStatus_value)
|
||||
proto.RegisterEnum("PaymentMethod", PaymentMethod_name, PaymentMethod_value)
|
||||
proto.RegisterEnum("ErrorCodes", ErrorCodes_name, ErrorCodes_value)
|
||||
proto.RegisterEnum("IsNameValidResponse_Code", IsNameValidResponse_Code_name, IsNameValidResponse_Code_value)
|
||||
proto.RegisterType((*GetSubscriptionRequest)(nil), "GetSubscriptionRequest")
|
||||
proto.RegisterType((*GetSubscriptionRequestSigned)(nil), "GetSubscriptionRequestSigned")
|
||||
|
@ -1334,80 +1420,93 @@ func init() {
|
|||
}
|
||||
|
||||
var fileDescriptor_4feb29dcc5ba50f6 = []byte{
|
||||
// 1157 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x57, 0x4f, 0x6f, 0x1b, 0x45,
|
||||
0x14, 0xf7, 0xfa, 0x4f, 0x12, 0xbf, 0x34, 0xc9, 0xf6, 0xc5, 0x4d, 0xb7, 0x6e, 0x62, 0xa5, 0x2b,
|
||||
0xfe, 0x44, 0x41, 0x6c, 0x45, 0xe9, 0x01, 0x10, 0x42, 0x38, 0x69, 0x1a, 0x22, 0x95, 0xc8, 0x5a,
|
||||
0xbb, 0x45, 0xd0, 0x03, 0x6c, 0xbd, 0x53, 0x67, 0xc9, 0x66, 0x66, 0x99, 0x19, 0x27, 0x2c, 0x5f,
|
||||
0x00, 0x71, 0x43, 0xe2, 0xc2, 0xf7, 0xe1, 0xc2, 0x09, 0xf5, 0xc8, 0x11, 0x25, 0x37, 0x4e, 0x7c,
|
||||
0x04, 0x34, 0xb3, 0xeb, 0x78, 0x1d, 0xaf, 0x9d, 0xd0, 0x88, 0x4b, 0xbc, 0xf3, 0x9b, 0x79, 0x6f,
|
||||
0xde, 0xfb, 0xbd, 0x7f, 0x13, 0xf8, 0x24, 0xf2, 0xe2, 0x23, 0x42, 0xa5, 0x20, 0xfc, 0x38, 0xe8,
|
||||
0x92, 0xfb, 0xa3, 0xcb, 0x88, 0x33, 0xc9, 0xee, 0xeb, 0xbf, 0xe2, 0xc2, 0x96, 0xa3, 0xd1, 0xfa,
|
||||
0xa3, 0xd7, 0x95, 0xff, 0x5a, 0x06, 0x84, 0x8b, 0x44, 0x8b, 0xfd, 0x01, 0xac, 0xec, 0x12, 0xd9,
|
||||
0xee, 0xbf, 0x10, 0x5d, 0x1e, 0x44, 0x32, 0x60, 0xd4, 0x25, 0xdf, 0xf5, 0x89, 0x90, 0xd8, 0x00,
|
||||
0x60, 0x27, 0x94, 0xf0, 0x26, 0x8d, 0xf7, 0x1e, 0x59, 0xc6, 0xba, 0xb1, 0x51, 0x75, 0x33, 0x88,
|
||||
0xfd, 0x0c, 0x56, 0xf3, 0x25, 0xdb, 0x41, 0x8f, 0x12, 0x1f, 0x2d, 0x98, 0x8d, 0xbc, 0x38, 0x64,
|
||||
0x9e, 0xaf, 0x85, 0x6f, 0xb8, 0x83, 0x25, 0xae, 0x42, 0x55, 0x04, 0x3d, 0xea, 0xc9, 0x3e, 0x27,
|
||||
0x56, 0x51, 0xef, 0x0d, 0x01, 0xfb, 0x9f, 0x22, 0xdc, 0x1e, 0x53, 0x2c, 0x22, 0x46, 0x05, 0x41,
|
||||
0x84, 0xb2, 0x32, 0x5e, 0x2b, 0xac, 0xb8, 0xfa, 0x1b, 0xdf, 0x81, 0x19, 0x21, 0x3d, 0xd9, 0x17,
|
||||
0x5a, 0xd5, 0xe2, 0x83, 0x65, 0x27, 0x2b, 0xda, 0xd6, 0x5b, 0x6e, 0x7a, 0x04, 0xd7, 0x61, 0xde,
|
||||
0xf7, 0x24, 0x69, 0x4b, 0x8f, 0x4b, 0xe2, 0x5b, 0xa5, 0x75, 0x63, 0xa3, 0xec, 0x66, 0x21, 0xac,
|
||||
0xc3, 0x9c, 0x5a, 0xee, 0x50, 0x5f, 0x58, 0x65, 0xbd, 0x7d, 0xbe, 0x56, 0xd2, 0x81, 0x68, 0xf6,
|
||||
0x25, 0x73, 0x09, 0x25, 0x27, 0x56, 0x65, 0xdd, 0xd8, 0x98, 0x73, 0xb3, 0x10, 0x3e, 0x84, 0x85,
|
||||
0x94, 0xec, 0xcf, 0x89, 0x3c, 0x60, 0xbe, 0x35, 0xa3, 0x6d, 0x5a, 0x74, 0x5a, 0x59, 0xd4, 0x1d,
|
||||
0x3d, 0x84, 0x9b, 0x60, 0xf2, 0x84, 0x3b, 0xe2, 0x37, 0x69, 0xbc, 0xef, 0x1d, 0x11, 0x6b, 0x56,
|
||||
0x13, 0x3e, 0x86, 0x2b, 0xf2, 0xfa, 0x82, 0xf0, 0x9d, 0x23, 0x2f, 0x08, 0xad, 0x39, 0x7d, 0x68,
|
||||
0x08, 0xe0, 0x43, 0xb8, 0x25, 0x12, 0xef, 0x5f, 0x90, 0x0e, 0xdb, 0x27, 0x27, 0x22, 0x24, 0x52,
|
||||
0x12, 0x6e, 0x55, 0xb5, 0xad, 0xf9, 0x9b, 0xf6, 0xdf, 0x06, 0xac, 0x6c, 0xf5, 0xe3, 0xcb, 0xb2,
|
||||
0xc0, 0x1f, 0xcb, 0x02, 0x1f, 0x37, 0x60, 0x49, 0xaf, 0x76, 0xe4, 0x41, 0xd3, 0xf7, 0x39, 0x11,
|
||||
0x49, 0x18, 0xaa, 0xee, 0x45, 0x18, 0xdf, 0x80, 0x85, 0x73, 0x67, 0x3a, 0x2a, 0x88, 0x25, 0x1d,
|
||||
0xc4, 0x51, 0x70, 0x9c, 0xc0, 0xf2, 0xeb, 0x12, 0x58, 0xc9, 0x27, 0x50, 0xe5, 0x6d, 0xbe, 0xaf,
|
||||
0xd7, 0xcc, 0xdb, 0x0f, 0xe1, 0xf6, 0x98, 0xde, 0x34, 0x6d, 0x1b, 0x00, 0xa9, 0xbd, 0x4f, 0x79,
|
||||
0x38, 0x20, 0x71, 0x88, 0xd8, 0xbf, 0x18, 0x70, 0xf7, 0x71, 0x40, 0xbd, 0x30, 0xf8, 0x81, 0xfc,
|
||||
0xbf, 0x41, 0xc8, 0x23, 0xaa, 0x34, 0x81, 0xa8, 0x06, 0xac, 0xe6, 0x1b, 0x95, 0x78, 0x65, 0x3f,
|
||||
0x87, 0x7b, 0x53, 0x8c, 0xbe, 0x26, 0x9b, 0x5b, 0xb0, 0x7e, 0xa1, 0x09, 0xb4, 0x18, 0x97, 0x5e,
|
||||
0xf8, 0x24, 0xa0, 0x87, 0x57, 0xa4, 0xc5, 0xfe, 0x06, 0xde, 0xba, 0x4c, 0xc7, 0x35, 0xad, 0x6c,
|
||||
0xc2, 0xbd, 0x29, 0x37, 0xa4, 0xd1, 0x5f, 0x85, 0x6a, 0xa4, 0xd1, 0x61, 0xf0, 0x87, 0x80, 0xfd,
|
||||
0x93, 0x01, 0x77, 0x77, 0x89, 0x7c, 0x46, 0x78, 0xf0, 0x32, 0xe8, 0x7a, 0x4a, 0x87, 0x2e, 0xe5,
|
||||
0xab, 0xc6, 0xbe, 0x06, 0x15, 0xa2, 0x7b, 0x41, 0x12, 0xf1, 0x64, 0x31, 0xb9, 0x0f, 0x94, 0xa6,
|
||||
0xf5, 0x81, 0x86, 0x6e, 0xe9, 0x39, 0xa6, 0x0c, 0x23, 0x3e, 0xc5, 0xd4, 0x6b, 0x72, 0xc9, 0x01,
|
||||
0xb5, 0xe6, 0xf8, 0x3f, 0xb9, 0x7f, 0xf5, 0xd4, 0x47, 0x28, 0x77, 0x99, 0x3f, 0x48, 0x77, 0xfd,
|
||||
0x6d, 0xdf, 0x87, 0xe5, 0x91, 0x3b, 0xd3, 0x88, 0x59, 0x30, 0x2b, 0xfa, 0xdd, 0xae, 0x52, 0x66,
|
||||
0x68, 0xbe, 0x06, 0x4b, 0xdb, 0x05, 0x6b, 0xdc, 0xc8, 0x6b, 0x3a, 0xfe, 0x12, 0x70, 0x4f, 0xa8,
|
||||
0x8a, 0x7b, 0xe6, 0x85, 0x81, 0x3f, 0x70, 0x7c, 0xac, 0x5d, 0x1a, 0x79, 0xed, 0x32, 0xaf, 0x9e,
|
||||
0x8b, 0x13, 0xea, 0xf9, 0x0f, 0x03, 0x96, 0x47, 0x2e, 0x4a, 0xbd, 0x7d, 0x37, 0x25, 0xc6, 0xd0,
|
||||
0x9d, 0xf6, 0x8e, 0x93, 0x73, 0xc6, 0xd9, 0x66, 0x3e, 0x49, 0x38, 0xd3, 0x23, 0x94, 0x9c, 0xe7,
|
||||
0x7b, 0x7a, 0x5b, 0x16, 0xb2, 0x5f, 0x42, 0x59, 0x9d, 0xc7, 0x2a, 0x54, 0xb4, 0x16, 0xb3, 0x80,
|
||||
0x37, 0x60, 0x6e, 0x9f, 0x3d, 0x62, 0xb2, 0x49, 0x63, 0xd3, 0x50, 0xab, 0x0e, 0x63, 0xed, 0x03,
|
||||
0xc6, 0xa5, 0x59, 0xc4, 0x79, 0x98, 0xed, 0x30, 0xf6, 0x84, 0xd1, 0x9e, 0x59, 0xc2, 0x65, 0x58,
|
||||
0xfa, 0xcc, 0x13, 0x7b, 0xf4, 0x58, 0x09, 0x6e, 0x1f, 0x78, 0x5c, 0x98, 0x65, 0xbc, 0x05, 0x37,
|
||||
0x95, 0xb7, 0x8f, 0x89, 0x26, 0x6c, 0x9f, 0x29, 0xfb, 0xcc, 0xca, 0xe6, 0x6f, 0x06, 0x98, 0xd9,
|
||||
0xda, 0xd3, 0x8c, 0x2c, 0xc1, 0xbc, 0xfa, 0x7d, 0x4a, 0x0f, 0x29, 0x3b, 0xa1, 0x66, 0x01, 0x4d,
|
||||
0xb8, 0xa1, 0x80, 0x9d, 0xef, 0xa3, 0x90, 0x71, 0xc2, 0x4d, 0x03, 0x2d, 0xa8, 0x29, 0x64, 0xab,
|
||||
0x1f, 0x84, 0x3e, 0xe1, 0xef, 0x7d, 0x41, 0xc8, 0x61, 0x67, 0xa7, 0xdd, 0x31, 0x8b, 0x58, 0x87,
|
||||
0x15, 0xb5, 0xb3, 0xcd, 0xb6, 0x39, 0xf1, 0x24, 0xcb, 0xec, 0x95, 0xb0, 0x06, 0x66, 0x56, 0xea,
|
||||
0x4b, 0xe2, 0x71, 0xb3, 0x8c, 0x2b, 0x80, 0xa3, 0x12, 0x1a, 0xaf, 0x28, 0x3f, 0x32, 0xa7, 0x5b,
|
||||
0x61, 0x5f, 0x98, 0x33, 0x03, 0xb0, 0x49, 0x63, 0x19, 0x47, 0xa4, 0x43, 0xbc, 0x23, 0x73, 0x76,
|
||||
0x53, 0x00, 0x8e, 0x3f, 0x58, 0xf0, 0x26, 0x2c, 0x24, 0x5f, 0x43, 0x47, 0xce, 0xa1, 0x16, 0xa1,
|
||||
0x7e, 0x40, 0x7b, 0xa6, 0xa1, 0x7c, 0x4b, 0xa0, 0x66, 0x57, 0x06, 0xc7, 0xc4, 0x2c, 0xe2, 0x9b,
|
||||
0x70, 0x6f, 0xe4, 0x90, 0x4a, 0xa7, 0x80, 0x13, 0x91, 0x76, 0x6a, 0x5d, 0xb4, 0x66, 0x69, 0xf3,
|
||||
0x47, 0x03, 0x16, 0x46, 0x26, 0x2a, 0x2e, 0x02, 0x24, 0x5f, 0xdb, 0x1e, 0xf7, 0x13, 0xda, 0xd2,
|
||||
0x35, 0x8f, 0x23, 0xc9, 0x4c, 0x03, 0x11, 0x16, 0x13, 0xa4, 0x19, 0x45, 0x21, 0x69, 0x79, 0xb1,
|
||||
0x59, 0x54, 0x1e, 0x25, 0xd8, 0x2e, 0x63, 0xbd, 0x04, 0xd4, 0x4c, 0x65, 0x0e, 0xee, 0x51, 0x2f,
|
||||
0x8a, 0x92, 0x20, 0x66, 0x8f, 0x26, 0x70, 0xe5, 0xc1, 0xaf, 0x15, 0xa8, 0x35, 0x69, 0x9c, 0x1a,
|
||||
0xd3, 0xe2, 0x4c, 0xd5, 0x59, 0x40, 0x7b, 0xe8, 0xc2, 0xad, 0x0b, 0xbd, 0x35, 0xa5, 0x66, 0xcd,
|
||||
0x99, 0xf6, 0xee, 0xac, 0x5b, 0xce, 0x84, 0xd7, 0xa3, 0x5d, 0xc0, 0x8f, 0x60, 0x3e, 0x93, 0xdd,
|
||||
0xb8, 0xec, 0x8c, 0x17, 0x5e, 0xbd, 0x96, 0x57, 0x00, 0x76, 0x01, 0x9f, 0xc0, 0xd2, 0x85, 0xf9,
|
||||
0x8e, 0x6b, 0xce, 0xb4, 0x97, 0x44, 0xdd, 0x72, 0x26, 0x3c, 0x08, 0xec, 0x02, 0x3e, 0x87, 0x5a,
|
||||
0xde, 0xf0, 0x44, 0xdb, 0xb9, 0x74, 0xa6, 0xd6, 0xd7, 0x9c, 0xa9, 0x73, 0xb9, 0x80, 0xdf, 0xc2,
|
||||
0x9d, 0x89, 0x63, 0x09, 0xdf, 0x76, 0xae, 0x36, 0x14, 0xeb, 0xb6, 0x73, 0xe9, 0x6c, 0x4b, 0x1c,
|
||||
0xc9, 0x9b, 0x09, 0xa8, 0xa5, 0xa7, 0x8f, 0x8a, 0xfa, 0x9a, 0x33, 0x75, 0xdc, 0x14, 0xf0, 0x53,
|
||||
0x98, 0xcf, 0xb4, 0x5b, 0xbc, 0xe3, 0x4c, 0x6a, 0xbe, 0xf5, 0x9a, 0x93, 0xd3, 0xc8, 0x93, 0x88,
|
||||
0xef, 0x12, 0xd9, 0x0c, 0x43, 0x55, 0x78, 0x02, 0x57, 0xd4, 0x8d, 0xfa, 0x73, 0x54, 0xfc, 0x66,
|
||||
0x06, 0x1f, 0xc8, 0x6e, 0x7d, 0xfc, 0xfb, 0x69, 0xc3, 0x78, 0x75, 0xda, 0x30, 0xfe, 0x3a, 0x6d,
|
||||
0x18, 0x3f, 0x9f, 0x35, 0x0a, 0xaf, 0xce, 0x1a, 0x85, 0x3f, 0xcf, 0x1a, 0x85, 0xaf, 0xec, 0xcb,
|
||||
0xff, 0xf7, 0x7a, 0x31, 0xa3, 0x7f, 0xde, 0xff, 0x37, 0x00, 0x00, 0xff, 0xff, 0x95, 0xf9, 0xbc,
|
||||
0xe2, 0xe8, 0x0d, 0x00, 0x00,
|
||||
// 1363 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x57, 0xcd, 0x6f, 0x1b, 0x45,
|
||||
0x14, 0xf7, 0xfa, 0x23, 0x89, 0x9f, 0xe3, 0x64, 0x32, 0x71, 0xd3, 0xad, 0x9b, 0x58, 0xe9, 0x8a,
|
||||
0x8f, 0x28, 0x88, 0xad, 0x28, 0x3d, 0x20, 0x84, 0x10, 0x4e, 0xe2, 0x84, 0x48, 0x25, 0x44, 0xb6,
|
||||
0xdb, 0x0a, 0x7a, 0x80, 0x8d, 0xf7, 0xc5, 0x59, 0xba, 0x99, 0x59, 0x66, 0xc7, 0x4d, 0xcd, 0x8d,
|
||||
0x13, 0xe2, 0x86, 0xc4, 0x85, 0xff, 0x87, 0x0b, 0x27, 0xd4, 0x63, 0x8f, 0xa8, 0xbd, 0x71, 0xe2,
|
||||
0x4f, 0x40, 0x33, 0xbb, 0x8e, 0xd7, 0xf1, 0xda, 0x09, 0x8d, 0xb8, 0x24, 0x3b, 0xbf, 0x99, 0xf7,
|
||||
0xe6, 0xbd, 0xdf, 0xfb, 0x98, 0x67, 0xf8, 0x34, 0x70, 0xfa, 0xa7, 0xc8, 0x64, 0x88, 0xe2, 0x99,
|
||||
0xd7, 0xc1, 0xbb, 0xa3, 0xcb, 0x40, 0x70, 0xc9, 0xef, 0xea, 0xbf, 0xe1, 0x85, 0x2d, 0x5b, 0xa3,
|
||||
0xd5, 0x9d, 0x37, 0x95, 0xff, 0x46, 0x7a, 0x28, 0xc2, 0x48, 0x8b, 0xf5, 0x11, 0xac, 0xec, 0xa1,
|
||||
0x6c, 0xf5, 0x8e, 0xc2, 0x8e, 0xf0, 0x02, 0xe9, 0x71, 0xd6, 0xc4, 0xef, 0x7b, 0x18, 0x4a, 0x5a,
|
||||
0x03, 0xe0, 0x67, 0x0c, 0x45, 0x9d, 0xf5, 0xf7, 0x77, 0x4c, 0x63, 0xdd, 0xd8, 0x28, 0x36, 0x13,
|
||||
0x88, 0xf5, 0x08, 0x56, 0xd3, 0x25, 0x5b, 0x5e, 0x97, 0xa1, 0x4b, 0x4d, 0x98, 0x0d, 0x9c, 0xbe,
|
||||
0xcf, 0x1d, 0x57, 0x0b, 0xcf, 0x37, 0x07, 0x4b, 0xba, 0x0a, 0xc5, 0xd0, 0xeb, 0x32, 0x47, 0xf6,
|
||||
0x04, 0x9a, 0x59, 0xbd, 0x37, 0x04, 0xac, 0x7f, 0xb2, 0x70, 0x73, 0x4c, 0x71, 0x18, 0x70, 0x16,
|
||||
0x22, 0xa5, 0x90, 0x57, 0xc6, 0x6b, 0x85, 0xe5, 0xa6, 0xfe, 0xa6, 0xef, 0xc1, 0x4c, 0x28, 0x1d,
|
||||
0xd9, 0x0b, 0xb5, 0xaa, 0x85, 0x7b, 0xcb, 0x76, 0x52, 0xb4, 0xa5, 0xb7, 0x9a, 0xf1, 0x11, 0xba,
|
||||
0x0e, 0x25, 0xd7, 0x91, 0xd8, 0x92, 0x8e, 0x90, 0xe8, 0x9a, 0xb9, 0x75, 0x63, 0x23, 0xdf, 0x4c,
|
||||
0x42, 0xb4, 0x0a, 0x73, 0x6a, 0xd9, 0x60, 0x6e, 0x68, 0xe6, 0xf5, 0xf6, 0xf9, 0x5a, 0x49, 0x7b,
|
||||
0x61, 0xbd, 0x27, 0x79, 0x13, 0x19, 0x9e, 0x99, 0x85, 0x75, 0x63, 0x63, 0xae, 0x99, 0x84, 0xe8,
|
||||
0x7d, 0x28, 0xc7, 0x64, 0x7f, 0x81, 0xf2, 0x84, 0xbb, 0xe6, 0x8c, 0xb6, 0x69, 0xc1, 0x3e, 0x4c,
|
||||
0xa2, 0xcd, 0xd1, 0x43, 0x74, 0x13, 0x88, 0x88, 0xb8, 0x43, 0xb7, 0xce, 0xfa, 0x07, 0xce, 0x29,
|
||||
0x9a, 0xb3, 0x9a, 0xf0, 0x31, 0x5c, 0x91, 0xd7, 0x0b, 0x51, 0x34, 0x4e, 0x1d, 0xcf, 0x37, 0xe7,
|
||||
0xf4, 0xa1, 0x21, 0x40, 0xef, 0xc3, 0x8d, 0x30, 0xf2, 0xfe, 0x08, 0xdb, 0xfc, 0x00, 0xcf, 0x42,
|
||||
0x1f, 0xa5, 0x44, 0x61, 0x16, 0xb5, 0xad, 0xe9, 0x9b, 0xd6, 0xdf, 0x06, 0xac, 0x6c, 0xf5, 0xfa,
|
||||
0x97, 0x65, 0x81, 0x3b, 0x96, 0x05, 0x2e, 0xdd, 0x80, 0x45, 0xbd, 0x6a, 0xc8, 0x93, 0xba, 0xeb,
|
||||
0x0a, 0x0c, 0xa3, 0x30, 0x14, 0x9b, 0x17, 0x61, 0xfa, 0x16, 0x94, 0xcf, 0x9d, 0x69, 0xab, 0x20,
|
||||
0xe6, 0x74, 0x10, 0x47, 0xc1, 0x71, 0x02, 0xf3, 0x6f, 0x4a, 0x60, 0x21, 0x9d, 0x40, 0x95, 0xb7,
|
||||
0xe9, 0xbe, 0x5e, 0x33, 0x6f, 0x1f, 0xc3, 0xcd, 0x31, 0xbd, 0x71, 0xda, 0xd6, 0x00, 0x62, 0x7b,
|
||||
0x1f, 0x0a, 0x7f, 0x40, 0xe2, 0x10, 0x51, 0x8a, 0x8f, 0x3c, 0xdf, 0xf7, 0x58, 0x77, 0x7f, 0x27,
|
||||
0xa6, 0x6f, 0x08, 0x58, 0xbf, 0x1a, 0x70, 0x7b, 0xd7, 0x63, 0x8e, 0xef, 0xfd, 0x80, 0xff, 0x6f,
|
||||
0x88, 0xd2, 0x68, 0xcc, 0x4d, 0xa0, 0xb1, 0x06, 0xab, 0xe9, 0x46, 0x45, 0x3e, 0x5b, 0x4f, 0xe0,
|
||||
0xce, 0x14, 0xa3, 0xaf, 0xc9, 0xf5, 0x16, 0xac, 0x5f, 0x68, 0x11, 0x87, 0x5c, 0x48, 0xc7, 0x7f,
|
||||
0xe0, 0xb1, 0xa7, 0x57, 0xa4, 0xc5, 0xfa, 0x16, 0xde, 0xb9, 0x4c, 0xc7, 0x35, 0xad, 0xac, 0xc3,
|
||||
0x9d, 0x29, 0x37, 0xc4, 0xb9, 0xb1, 0x0a, 0xc5, 0x40, 0xa3, 0xc3, 0xd4, 0x18, 0x02, 0xd6, 0xcf,
|
||||
0x06, 0xdc, 0xde, 0x43, 0xf9, 0x08, 0x85, 0x77, 0xec, 0x75, 0x1c, 0xa5, 0x43, 0x17, 0xfa, 0x55,
|
||||
0x63, 0x5f, 0x81, 0x02, 0xea, 0x4e, 0x11, 0x45, 0x3c, 0x5a, 0x4c, 0xee, 0x12, 0xb9, 0x69, 0x5d,
|
||||
0xa2, 0xa6, 0x1b, 0x7e, 0x8a, 0x29, 0xc3, 0x88, 0x4f, 0x31, 0xf5, 0x9a, 0x5c, 0x0a, 0xa0, 0x5a,
|
||||
0x73, 0xff, 0x3f, 0xb9, 0x7f, 0xf5, 0xd4, 0xa7, 0x90, 0xef, 0x70, 0x77, 0x90, 0xee, 0xfa, 0xdb,
|
||||
0xba, 0x0b, 0xcb, 0x23, 0x77, 0xc6, 0x11, 0x33, 0x61, 0x36, 0xec, 0x75, 0x3a, 0x4a, 0x99, 0xa1,
|
||||
0xf9, 0x1a, 0x2c, 0xad, 0x26, 0x98, 0xe3, 0x46, 0x5e, 0xd3, 0xf1, 0x63, 0xa0, 0xfb, 0xa1, 0xaa,
|
||||
0xb8, 0x47, 0x8e, 0xef, 0xb9, 0x03, 0xc7, 0xc7, 0x9a, 0xa9, 0x91, 0xd6, 0x4c, 0xd3, 0xea, 0x39,
|
||||
0x3b, 0xa1, 0x9e, 0xff, 0x34, 0x60, 0x79, 0xe4, 0xa2, 0xd8, 0xdb, 0xf7, 0x63, 0x62, 0x0c, 0xdd,
|
||||
0x87, 0x6f, 0xd9, 0x29, 0x67, 0xec, 0x6d, 0xee, 0x62, 0xc4, 0x99, 0x7e, 0x60, 0xf1, 0x3c, 0xdf,
|
||||
0xe3, 0xdb, 0x92, 0x90, 0x75, 0x0c, 0x79, 0x75, 0x9e, 0x16, 0xa1, 0xa0, 0xb5, 0x90, 0x0c, 0x9d,
|
||||
0x87, 0xb9, 0x03, 0xbe, 0xc3, 0x65, 0x9d, 0xf5, 0x89, 0xa1, 0x56, 0x6d, 0xce, 0x5b, 0x27, 0x5c,
|
||||
0x48, 0x92, 0xa5, 0x25, 0x98, 0x6d, 0x73, 0xfe, 0x80, 0xb3, 0x2e, 0xc9, 0xd1, 0x65, 0x58, 0xfc,
|
||||
0xdc, 0x09, 0xf7, 0xd9, 0x33, 0x25, 0xb8, 0x7d, 0xe2, 0x88, 0x90, 0xe4, 0xe9, 0x0d, 0x58, 0x52,
|
||||
0xde, 0xee, 0xa2, 0x26, 0xec, 0x80, 0x2b, 0xfb, 0x48, 0x61, 0xf3, 0x77, 0x03, 0x48, 0xb2, 0xf6,
|
||||
0x34, 0x23, 0x8b, 0x50, 0x52, 0xff, 0x1f, 0xb2, 0xa7, 0x8c, 0x9f, 0x31, 0x92, 0xa1, 0x04, 0xe6,
|
||||
0x15, 0xd0, 0x78, 0x1e, 0xf8, 0x5c, 0xa0, 0x20, 0x06, 0x35, 0xa1, 0xa2, 0x90, 0xad, 0x9e, 0xe7,
|
||||
0xbb, 0x28, 0x3e, 0x78, 0x8c, 0xf8, 0xb4, 0xdd, 0x68, 0xb5, 0x49, 0x96, 0x56, 0x61, 0x45, 0xed,
|
||||
0x6c, 0xf3, 0x6d, 0x81, 0x8e, 0xe4, 0x89, 0xbd, 0x1c, 0xad, 0x00, 0x49, 0x4a, 0x7d, 0x85, 0x8e,
|
||||
0x20, 0x79, 0xba, 0x02, 0x74, 0x54, 0x42, 0xe3, 0x05, 0xe5, 0x47, 0xe2, 0xf4, 0xa1, 0xdf, 0x0b,
|
||||
0xc9, 0xcc, 0x00, 0xac, 0xb3, 0xbe, 0xec, 0x07, 0xd8, 0x46, 0xe7, 0x94, 0xcc, 0x6e, 0x86, 0x40,
|
||||
0xc7, 0xc7, 0x19, 0xba, 0x04, 0xe5, 0xe8, 0x6b, 0xe8, 0xc8, 0x39, 0x74, 0x88, 0xcc, 0xf5, 0x58,
|
||||
0x97, 0x18, 0xca, 0xb7, 0x08, 0xaa, 0x77, 0xa4, 0xf7, 0x0c, 0x49, 0x96, 0xbe, 0x0d, 0x77, 0x46,
|
||||
0x0e, 0xa9, 0x74, 0xf2, 0x04, 0x86, 0x71, 0xa7, 0xd6, 0x45, 0x4b, 0x72, 0x9b, 0x3f, 0x19, 0x50,
|
||||
0x1e, 0x79, 0x6f, 0xe9, 0x02, 0x40, 0xf4, 0xb5, 0xed, 0x08, 0x37, 0xa2, 0x2d, 0x5e, 0x8b, 0x7e,
|
||||
0x20, 0x39, 0x31, 0x28, 0x85, 0x85, 0x08, 0xa9, 0x07, 0x81, 0x8f, 0x87, 0x4e, 0x9f, 0x64, 0x95,
|
||||
0x47, 0x11, 0xb6, 0xc7, 0x79, 0x37, 0x02, 0x35, 0x53, 0x89, 0x83, 0xfb, 0xcc, 0x09, 0x82, 0x28,
|
||||
0x88, 0xc9, 0xa3, 0x11, 0x5c, 0xd8, 0xfc, 0x31, 0x07, 0xd0, 0x10, 0x82, 0x0b, 0x95, 0x32, 0xa1,
|
||||
0x32, 0xe3, 0x21, 0xc3, 0xe7, 0x01, 0x76, 0x24, 0x2a, 0x33, 0x96, 0x61, 0x71, 0x58, 0xc3, 0x8d,
|
||||
0xd3, 0x40, 0xaa, 0xfc, 0xa9, 0x00, 0x89, 0x33, 0xa4, 0x35, 0xa8, 0x22, 0x92, 0xa5, 0x65, 0x28,
|
||||
0x2a, 0x76, 0x1f, 0x8b, 0x28, 0x93, 0xe2, 0xb8, 0x1f, 0x70, 0xb9, 0xcb, 0x7b, 0xcc, 0x25, 0xf9,
|
||||
0x01, 0xb2, 0xcf, 0x9c, 0x88, 0xad, 0x82, 0x8a, 0xde, 0x08, 0x0b, 0x91, 0xec, 0x8c, 0xb2, 0x62,
|
||||
0xcb, 0x19, 0x14, 0x0e, 0x99, 0x55, 0x29, 0x3a, 0x88, 0xc3, 0x9c, 0x72, 0x44, 0x05, 0xac, 0xee,
|
||||
0x0b, 0x74, 0xdc, 0x7e, 0xcc, 0x7c, 0x51, 0xc7, 0xa2, 0x77, 0x14, 0x9e, 0xdf, 0x07, 0x8a, 0x30,
|
||||
0x85, 0x68, 0xa5, 0x2a, 0x28, 0x48, 0x4a, 0xca, 0x74, 0xdd, 0x3a, 0x34, 0xb8, 0xcb, 0xc5, 0xa9,
|
||||
0x23, 0xc9, 0xbc, 0xca, 0x48, 0x8d, 0xc6, 0x3a, 0xa3, 0x0e, 0x8b, 0x2e, 0x29, 0x9f, 0x9f, 0x8f,
|
||||
0x77, 0x5a, 0xc8, 0x24, 0x59, 0x50, 0x26, 0x68, 0x74, 0xd7, 0xf1, 0x7c, 0x74, 0xdb, 0xbc, 0x85,
|
||||
0xcc, 0x25, 0x8b, 0xca, 0x04, 0x0d, 0x37, 0x9e, 0x07, 0x9e, 0x40, 0x97, 0x10, 0x65, 0xc2, 0xf0,
|
||||
0x3a, 0xc5, 0x30, 0x59, 0xa2, 0x04, 0x4a, 0x9a, 0xf0, 0x2f, 0x8f, 0x8f, 0x43, 0x94, 0xe4, 0x65,
|
||||
0xfe, 0xde, 0x6f, 0x05, 0xa8, 0xd4, 0x59, 0x3f, 0xa6, 0xe2, 0x50, 0x70, 0xd5, 0xeb, 0x3c, 0xd6,
|
||||
0xa5, 0x4d, 0xb8, 0x71, 0xe1, 0x7d, 0x8b, 0xd3, 0x73, 0xcd, 0x9e, 0xf6, 0xcb, 0xa0, 0x6a, 0xda,
|
||||
0x13, 0xe6, 0x7b, 0x2b, 0x43, 0x3f, 0x86, 0x52, 0xa2, 0xc3, 0xd0, 0x65, 0x7b, 0xbc, 0xf9, 0x55,
|
||||
0x2b, 0x69, 0x4d, 0xc8, 0xca, 0xd0, 0x07, 0xb0, 0x78, 0x61, 0x02, 0xa3, 0x6b, 0xf6, 0xb4, 0x59,
|
||||
0xaf, 0x6a, 0xda, 0x13, 0x46, 0x36, 0x2b, 0x43, 0x9f, 0x40, 0x25, 0x6d, 0x80, 0xa1, 0x96, 0x7d,
|
||||
0xe9, 0x5c, 0x53, 0x5d, 0xb3, 0xa7, 0xce, 0x46, 0x19, 0xfa, 0x1d, 0xdc, 0x9a, 0x38, 0x1a, 0xd0,
|
||||
0x77, 0xed, 0xab, 0x0d, 0x26, 0x55, 0xcb, 0xbe, 0x74, 0xbe, 0x88, 0x1c, 0x49, 0x7b, 0x97, 0xa9,
|
||||
0x96, 0x9e, 0xfe, 0x5c, 0x57, 0xd7, 0xec, 0xa9, 0x4f, 0x7e, 0x86, 0x7e, 0x06, 0xa5, 0xc4, 0x93,
|
||||
0x47, 0x6f, 0xd9, 0x93, 0x1e, 0xc0, 0x6a, 0xc5, 0x4e, 0x79, 0x4c, 0xa3, 0x88, 0xef, 0xa1, 0xac,
|
||||
0xfb, 0xbe, 0xaa, 0xbe, 0x90, 0xae, 0xa8, 0x1b, 0xf5, 0xe7, 0xa8, 0xf8, 0x52, 0x02, 0x1f, 0xc8,
|
||||
0x6e, 0x7d, 0xf2, 0xc7, 0xab, 0x9a, 0xf1, 0xe2, 0x55, 0xcd, 0xf8, 0xeb, 0x55, 0xcd, 0xf8, 0xe5,
|
||||
0x75, 0x2d, 0xf3, 0xe2, 0x75, 0x2d, 0xf3, 0xf2, 0x75, 0x2d, 0xf3, 0xb5, 0x75, 0xf9, 0xaf, 0xe3,
|
||||
0xa3, 0x19, 0xfd, 0xef, 0xc3, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x38, 0x9b, 0x47, 0x11, 0x8a,
|
||||
0x0f, 0x00, 0x00,
|
||||
}
|
||||
|
||||
func (m *GetSubscriptionRequest) Marshal() (dAtA []byte, err error) {
|
||||
|
@ -1670,6 +1769,13 @@ func (m *BuySubscriptionResponse) MarshalToSizedBuffer(dAtA []byte) (int, error)
|
|||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if len(m.BillingID) > 0 {
|
||||
i -= len(m.BillingID)
|
||||
copy(dAtA[i:], m.BillingID)
|
||||
i = encodeVarintPaymentservice(dAtA, i, uint64(len(m.BillingID)))
|
||||
i--
|
||||
dAtA[i] = 0x12
|
||||
}
|
||||
if len(m.PaymentUrl) > 0 {
|
||||
i -= len(m.PaymentUrl)
|
||||
copy(dAtA[i:], m.PaymentUrl)
|
||||
|
@ -2305,6 +2411,10 @@ func (m *BuySubscriptionResponse) Size() (n int) {
|
|||
if l > 0 {
|
||||
n += 1 + l + sovPaymentservice(uint64(l))
|
||||
}
|
||||
l = len(m.BillingID)
|
||||
if l > 0 {
|
||||
n += 1 + l + sovPaymentservice(uint64(l))
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
|
@ -2775,7 +2885,7 @@ func (m *GetSubscriptionResponse) Unmarshal(dAtA []byte) error {
|
|||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
m.Tier |= int32(b&0x7F) << shift
|
||||
m.Tier |= uint32(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
|
@ -3088,7 +3198,7 @@ func (m *BuySubscriptionRequest) Unmarshal(dAtA []byte) error {
|
|||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
m.RequestedTier |= int32(b&0x7F) << shift
|
||||
m.RequestedTier |= uint32(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
|
@ -3344,6 +3454,38 @@ func (m *BuySubscriptionResponse) Unmarshal(dAtA []byte) error {
|
|||
}
|
||||
m.PaymentUrl = string(dAtA[iNdEx:postIndex])
|
||||
iNdEx = postIndex
|
||||
case 2:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field BillingID", wireType)
|
||||
}
|
||||
var stringLen uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowPaymentservice
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
stringLen |= uint64(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
intStringLen := int(stringLen)
|
||||
if intStringLen < 0 {
|
||||
return ErrInvalidLengthPaymentservice
|
||||
}
|
||||
postIndex := iNdEx + intStringLen
|
||||
if postIndex < 0 {
|
||||
return ErrInvalidLengthPaymentservice
|
||||
}
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.BillingID = string(dAtA[iNdEx:postIndex])
|
||||
iNdEx = postIndex
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skipPaymentservice(dAtA[iNdEx:])
|
||||
|
@ -4640,7 +4782,7 @@ func (m *IsNameValidRequest) Unmarshal(dAtA []byte) error {
|
|||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
m.RequestedTier |= int32(b&0x7F) << shift
|
||||
m.RequestedTier |= uint32(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
|
|
|
@ -60,13 +60,7 @@ func (PeriodType) EnumDescriptor() ([]byte, []int) {
|
|||
}
|
||||
|
||||
type Feature struct {
|
||||
// "invites"
|
||||
// "GBs"
|
||||
// "spaces"
|
||||
// ...
|
||||
ValueStr string `protobuf:"bytes,1,opt,name=valueStr,proto3" json:"valueStr,omitempty"`
|
||||
// each uint is a value of the feature
|
||||
ValueUint uint32 `protobuf:"varint,2,opt,name=valueUint,proto3" json:"valueUint,omitempty"`
|
||||
Description string `protobuf:"bytes,1,opt,name=description,proto3" json:"description,omitempty"`
|
||||
}
|
||||
|
||||
func (m *Feature) Reset() { *m = Feature{} }
|
||||
|
@ -102,27 +96,18 @@ func (m *Feature) XXX_DiscardUnknown() {
|
|||
|
||||
var xxx_messageInfo_Feature proto.InternalMessageInfo
|
||||
|
||||
func (m *Feature) GetValueStr() string {
|
||||
func (m *Feature) GetDescription() string {
|
||||
if m != nil {
|
||||
return m.ValueStr
|
||||
return m.Description
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *Feature) GetValueUint() uint32 {
|
||||
if m != nil {
|
||||
return m.ValueUint
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
type GetTiersRequest struct {
|
||||
// in the following format: "A5k2d9sFZw84yisTxRnz2bPRd1YPfVfhxqymZ6yESprFTG65"
|
||||
// you can get it with Account().SignKey.GetPublic().Account()
|
||||
OwnerAnyId string `protobuf:"bytes,1,opt,name=ownerAnyId,proto3" json:"ownerAnyId,omitempty"`
|
||||
Locale string `protobuf:"bytes,2,opt,name=locale,proto3" json:"locale,omitempty"`
|
||||
// see PaymentMethod enum
|
||||
PaymentMethod uint32 `protobuf:"varint,3,opt,name=paymentMethod,proto3" json:"paymentMethod,omitempty"`
|
||||
}
|
||||
|
||||
func (m *GetTiersRequest) Reset() { *m = GetTiersRequest{} }
|
||||
|
@ -172,13 +157,6 @@ func (m *GetTiersRequest) GetLocale() string {
|
|||
return ""
|
||||
}
|
||||
|
||||
func (m *GetTiersRequest) GetPaymentMethod() uint32 {
|
||||
if m != nil {
|
||||
return m.PaymentMethod
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
type GetTiersRequestSigned struct {
|
||||
// GetTiersRequest struct
|
||||
Payload []byte `protobuf:"bytes,1,opt,name=payload,proto3" json:"payload,omitempty"`
|
||||
|
@ -261,9 +239,9 @@ type TierData struct {
|
|||
// somename.any - len of 8
|
||||
AnyNameMinLength uint32 `protobuf:"varint,11,opt,name=anyNameMinLength,proto3" json:"anyNameMinLength,omitempty"`
|
||||
// each tier has a set of features
|
||||
// each feature has a unique key: "storage", "invites", etc
|
||||
// not using enum here to provide dynamic feature list
|
||||
Features map[string]*Feature `protobuf:"bytes,12,rep,name=features,proto3" json:"features,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
|
||||
Features []*Feature `protobuf:"bytes,12,rep,name=features,proto3" json:"features,omitempty"`
|
||||
// green, blue, red, purple or custom color in string format #ff00ff
|
||||
ColorStr string `protobuf:"bytes,13,opt,name=colorStr,proto3" json:"colorStr,omitempty"`
|
||||
}
|
||||
|
||||
func (m *TierData) Reset() { *m = TierData{} }
|
||||
|
@ -376,13 +354,20 @@ func (m *TierData) GetAnyNameMinLength() uint32 {
|
|||
return 0
|
||||
}
|
||||
|
||||
func (m *TierData) GetFeatures() map[string]*Feature {
|
||||
func (m *TierData) GetFeatures() []*Feature {
|
||||
if m != nil {
|
||||
return m.Features
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *TierData) GetColorStr() string {
|
||||
if m != nil {
|
||||
return m.ColorStr
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type GetTiersResponse struct {
|
||||
// list of all available tiers
|
||||
Tiers []*TierData `protobuf:"bytes,1,rep,name=tiers,proto3" json:"tiers,omitempty"`
|
||||
|
@ -434,7 +419,6 @@ func init() {
|
|||
proto.RegisterType((*GetTiersRequest)(nil), "GetTiersRequest")
|
||||
proto.RegisterType((*GetTiersRequestSigned)(nil), "GetTiersRequestSigned")
|
||||
proto.RegisterType((*TierData)(nil), "TierData")
|
||||
proto.RegisterMapType((map[string]*Feature)(nil), "TierData.FeaturesEntry")
|
||||
proto.RegisterType((*GetTiersResponse)(nil), "GetTiersResponse")
|
||||
}
|
||||
|
||||
|
@ -443,46 +427,42 @@ func init() {
|
|||
}
|
||||
|
||||
var fileDescriptor_597ac3048c641f44 = []byte{
|
||||
// 615 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x94, 0xcf, 0x4f, 0xdb, 0x4a,
|
||||
0x10, 0xc7, 0xe3, 0xfc, 0x80, 0x64, 0x92, 0x80, 0xdf, 0xf0, 0x78, 0xac, 0xd0, 0x93, 0x5f, 0x64,
|
||||
0xbd, 0x43, 0x44, 0xa5, 0x50, 0x41, 0x0f, 0x55, 0xd5, 0x0b, 0x05, 0xda, 0x22, 0x95, 0xb6, 0x5a,
|
||||
0xa0, 0x55, 0x7b, 0xa9, 0xb6, 0xf1, 0x14, 0x56, 0x24, 0xbb, 0xae, 0x77, 0x03, 0xf2, 0x5f, 0xd0,
|
||||
0x2b, 0x7f, 0x56, 0x8f, 0x1c, 0x7b, 0xac, 0xe0, 0x1f, 0xa9, 0xbc, 0x71, 0xe2, 0x84, 0x22, 0xf5,
|
||||
0x92, 0xec, 0xf7, 0xf3, 0xdd, 0x9d, 0x9d, 0xf1, 0x8c, 0x0d, 0x7b, 0xb1, 0x48, 0x87, 0xa4, 0xac,
|
||||
0xa1, 0xe4, 0x42, 0xf6, 0x69, 0x73, 0x5e, 0xc6, 0x89, 0xb6, 0x7a, 0xd3, 0xfd, 0x9a, 0x3b, 0xd6,
|
||||
0x27, 0x2b, 0x29, 0x31, 0x3d, 0xe7, 0x85, 0xbb, 0xb0, 0xf8, 0x9c, 0x84, 0x1d, 0x25, 0x84, 0xeb,
|
||||
0x50, 0xbf, 0x10, 0x83, 0x11, 0x1d, 0xd9, 0x84, 0x79, 0x1d, 0xaf, 0xdb, 0xe0, 0x53, 0x8d, 0xff,
|
||||
0x42, 0xc3, 0xad, 0x4f, 0xa4, 0xb2, 0xac, 0xdc, 0xf1, 0xba, 0x6d, 0x5e, 0x80, 0x50, 0xc3, 0xf2,
|
||||
0x0b, 0xb2, 0xc7, 0x59, 0x58, 0x4e, 0x5f, 0x47, 0x64, 0x2c, 0x06, 0x00, 0xfa, 0x52, 0x51, 0xb2,
|
||||
0xa3, 0xd2, 0x83, 0x28, 0x0f, 0x37, 0x43, 0xf0, 0x1f, 0x58, 0x18, 0xe8, 0xbe, 0x18, 0x90, 0x8b,
|
||||
0xd6, 0xe0, 0xb9, 0xc2, 0xff, 0xa1, 0x9d, 0x67, 0x7b, 0x48, 0xf6, 0x4c, 0x47, 0xac, 0xe2, 0x2e,
|
||||
0x9b, 0x87, 0xe1, 0x1b, 0x58, 0xbd, 0x73, 0xe1, 0x91, 0x3c, 0x55, 0x14, 0x21, 0x83, 0xc5, 0x58,
|
||||
0xa4, 0x03, 0x2d, 0xc6, 0x77, 0xb6, 0xf8, 0x44, 0x66, 0x15, 0x18, 0x79, 0xaa, 0x5c, 0xa9, 0xee,
|
||||
0xce, 0x16, 0x2f, 0x40, 0xf8, 0xad, 0x0a, 0xf5, 0x2c, 0xdc, 0x9e, 0xb0, 0x02, 0x97, 0xa0, 0x2c,
|
||||
0xc7, 0xe7, 0xdb, 0xbc, 0x2c, 0x23, 0x44, 0xa8, 0x2a, 0x31, 0x9c, 0x64, 0xea, 0xd6, 0xd8, 0x81,
|
||||
0x66, 0x44, 0xa6, 0x9f, 0xc8, 0xd8, 0x4a, 0xad, 0x5c, 0x96, 0x0d, 0x3e, 0x8b, 0xb2, 0xc7, 0x29,
|
||||
0xcd, 0x4e, 0xdf, 0xca, 0x0b, 0x62, 0xd5, 0x8e, 0xd7, 0xad, 0xf3, 0xa9, 0xce, 0xaa, 0x97, 0xe6,
|
||||
0x98, 0x8c, 0x65, 0x35, 0xe7, 0xe4, 0x0a, 0x43, 0x68, 0x49, 0xf3, 0x52, 0x46, 0x11, 0xa9, 0x2c,
|
||||
0x1b, 0xb6, 0xe0, 0xdc, 0x39, 0x86, 0x0f, 0x00, 0x62, 0x4a, 0xa4, 0x8e, 0x8e, 0xd3, 0x98, 0xd8,
|
||||
0x62, 0xc7, 0xeb, 0x2e, 0x6d, 0x35, 0x7b, 0x6f, 0xa7, 0x88, 0xcf, 0xd8, 0x59, 0x9a, 0x63, 0xf5,
|
||||
0x2e, 0x6b, 0x16, 0xab, 0xbb, 0x9a, 0x66, 0x11, 0x3e, 0x84, 0x95, 0x38, 0x91, 0xfd, 0xac, 0xcb,
|
||||
0x32, 0xa6, 0x13, 0x13, 0xed, 0x66, 0x93, 0xc2, 0x1a, 0x6e, 0xe7, 0x7d, 0x16, 0x3e, 0x82, 0x55,
|
||||
0xa1, 0xd2, 0xd7, 0x62, 0x48, 0x66, 0x57, 0x8f, 0x94, 0x3d, 0x50, 0xfd, 0xc1, 0x28, 0xa2, 0x88,
|
||||
0x81, 0x3b, 0x73, 0xbf, 0x89, 0x1b, 0xe0, 0xe7, 0xc6, 0xa1, 0x54, 0xaf, 0x48, 0x9d, 0xda, 0x33,
|
||||
0xd6, 0x74, 0x07, 0x7e, 0xe3, 0xb8, 0x0d, 0xf5, 0x2f, 0xe3, 0xa1, 0x34, 0xac, 0xd5, 0xa9, 0x74,
|
||||
0x9b, 0x5b, 0x6b, 0xbd, 0x49, 0x77, 0x7a, 0xf9, 0xb8, 0x9a, 0x7d, 0x65, 0x93, 0x94, 0x4f, 0x37,
|
||||
0xae, 0xef, 0x43, 0x7b, 0xce, 0x42, 0x1f, 0x2a, 0xe7, 0x94, 0xe6, 0xb3, 0x97, 0x2d, 0x31, 0x80,
|
||||
0x9a, 0x1b, 0x5a, 0xd7, 0xc9, 0xe6, 0x56, 0x7d, 0x12, 0x8b, 0x8f, 0xf1, 0x93, 0xf2, 0x63, 0x2f,
|
||||
0xdc, 0x06, 0xbf, 0x18, 0x2d, 0x13, 0x6b, 0x65, 0x08, 0xff, 0x83, 0x9a, 0x7b, 0x67, 0x98, 0xe7,
|
||||
0x92, 0x69, 0x4c, 0x93, 0xe1, 0x63, 0xbe, 0x71, 0xe5, 0x01, 0x14, 0x1d, 0xc0, 0x55, 0xf8, 0xab,
|
||||
0x50, 0x27, 0xea, 0x5c, 0xe9, 0x4b, 0xe5, 0x97, 0x70, 0x0d, 0x56, 0x66, 0xf1, 0x40, 0x0e, 0xa5,
|
||||
0xa5, 0xc8, 0xf7, 0x10, 0x61, 0xa9, 0x30, 0xf6, 0x44, 0x6a, 0xfc, 0x32, 0xae, 0xc0, 0x72, 0xc1,
|
||||
0xde, 0x13, 0x9d, 0x1b, 0xbf, 0x82, 0x7f, 0x83, 0x5f, 0xc0, 0x43, 0xad, 0xec, 0x99, 0xf1, 0xab,
|
||||
0xf3, 0x5b, 0x3f, 0x90, 0x48, 0x8c, 0x5f, 0x7b, 0xf6, 0xf4, 0xfb, 0x4d, 0xe0, 0x5d, 0xdf, 0x04,
|
||||
0xde, 0xcf, 0x9b, 0xc0, 0xbb, 0xba, 0x0d, 0x4a, 0xd7, 0xb7, 0x41, 0xe9, 0xc7, 0x6d, 0x50, 0xfa,
|
||||
0x18, 0xfe, 0xf9, 0xc3, 0xf1, 0x79, 0xc1, 0xfd, 0x6d, 0xff, 0x0a, 0x00, 0x00, 0xff, 0xff, 0x0a,
|
||||
0xee, 0x4f, 0x49, 0x65, 0x04, 0x00, 0x00,
|
||||
// 549 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x93, 0x4d, 0x6f, 0xd3, 0x4c,
|
||||
0x10, 0xc7, 0xe3, 0xa4, 0x2f, 0xce, 0x24, 0x6d, 0xfd, 0x6c, 0x9f, 0xc2, 0x0a, 0x21, 0x13, 0x59,
|
||||
0x1c, 0xa2, 0x56, 0x4a, 0x51, 0xcb, 0x91, 0x4b, 0x69, 0x04, 0x44, 0xa2, 0x80, 0x9c, 0x14, 0x04,
|
||||
0x17, 0xb4, 0xd8, 0x43, 0xba, 0xaa, 0xb3, 0x6b, 0x76, 0x37, 0xad, 0xfc, 0x2d, 0xfa, 0xb1, 0xb8,
|
||||
0x20, 0xf5, 0xc8, 0x11, 0x25, 0x5f, 0x04, 0x79, 0xf3, 0xe2, 0x24, 0xad, 0xc4, 0xc5, 0xf6, 0xff,
|
||||
0xff, 0xdb, 0x19, 0xcd, 0xcc, 0x8e, 0xa1, 0x9d, 0xb2, 0x6c, 0x80, 0xc2, 0x68, 0x54, 0x57, 0x3c,
|
||||
0xc2, 0xc3, 0x65, 0x99, 0x2a, 0x69, 0xe4, 0xa1, 0x7d, 0xea, 0x15, 0xf4, 0xd5, 0x70, 0x54, 0xba,
|
||||
0x65, 0x59, 0x70, 0x00, 0x9b, 0xaf, 0x90, 0x99, 0xa1, 0x42, 0xd2, 0x80, 0x5a, 0x8c, 0x3a, 0x52,
|
||||
0x3c, 0x35, 0x5c, 0x0a, 0xea, 0x34, 0x9c, 0x66, 0x35, 0x5c, 0xb4, 0x82, 0x0e, 0xec, 0xbc, 0x46,
|
||||
0xd3, 0xcb, 0xc3, 0x43, 0xfc, 0x31, 0x44, 0x6d, 0x88, 0x0f, 0x20, 0xaf, 0x05, 0xaa, 0x13, 0x91,
|
||||
0x75, 0xe2, 0x69, 0xcc, 0x82, 0x43, 0x1e, 0xc0, 0x46, 0x22, 0x23, 0x96, 0x20, 0x2d, 0x5b, 0x36,
|
||||
0x55, 0xc1, 0x7b, 0xd8, 0x5b, 0x49, 0xd5, 0xe5, 0x7d, 0x81, 0x31, 0xa1, 0xb0, 0x99, 0xb2, 0x2c,
|
||||
0x91, 0x6c, 0x92, 0xad, 0x1e, 0xce, 0x24, 0x79, 0x0c, 0x55, 0xcd, 0xfb, 0xc2, 0x16, 0x6b, 0xb3,
|
||||
0xd5, 0xc3, 0xc2, 0x08, 0x7e, 0x55, 0xc0, 0xcd, 0xd3, 0xb5, 0x99, 0x61, 0x64, 0x1b, 0xca, 0x7c,
|
||||
0x12, 0xbf, 0x15, 0x96, 0x79, 0x4c, 0x08, 0xac, 0x09, 0x36, 0x98, 0xd5, 0x60, 0xbf, 0x57, 0xdb,
|
||||
0xad, 0xdc, 0x69, 0x97, 0x3c, 0x02, 0x97, 0xeb, 0x93, 0xc8, 0xf0, 0x2b, 0xa4, 0x6b, 0x0d, 0xa7,
|
||||
0xe9, 0x86, 0x73, 0x9d, 0xf7, 0xc5, 0x75, 0x0f, 0xb5, 0xa1, 0xeb, 0x96, 0x4c, 0x15, 0x09, 0xa0,
|
||||
0xce, 0xf5, 0x1b, 0x1e, 0xc7, 0x28, 0xf2, 0x6a, 0xe8, 0x86, 0xa5, 0x4b, 0x1e, 0x39, 0x00, 0x48,
|
||||
0x51, 0x71, 0x19, 0xf7, 0xb2, 0x14, 0xe9, 0x66, 0xc3, 0x69, 0x6e, 0x1f, 0xd5, 0x5a, 0x1f, 0xe6,
|
||||
0x56, 0xb8, 0x80, 0xf3, 0x32, 0x27, 0xea, 0x23, 0x4b, 0x86, 0x48, 0x5d, 0xdb, 0xd3, 0xa2, 0x45,
|
||||
0x9e, 0xc1, 0x6e, 0xaa, 0x78, 0x84, 0x5d, 0xa3, 0x78, 0x8a, 0xe7, 0x3a, 0x3e, 0xcd, 0xef, 0x9a,
|
||||
0x56, 0xed, 0xc9, 0xfb, 0x10, 0x79, 0x0e, 0x7b, 0x4c, 0x64, 0xef, 0xd8, 0x00, 0xf5, 0xa9, 0x1c,
|
||||
0x0a, 0xd3, 0x11, 0x51, 0x32, 0x8c, 0x31, 0xa6, 0x60, 0x63, 0xee, 0x87, 0x64, 0x1f, 0xbc, 0x29,
|
||||
0x38, 0xe3, 0xe2, 0x2d, 0x8a, 0xbe, 0xb9, 0xa0, 0x35, 0x1b, 0x70, 0xc7, 0x27, 0x4f, 0xc1, 0xfd,
|
||||
0x3e, 0x59, 0x2b, 0x4d, 0xeb, 0x8d, 0x4a, 0xb3, 0x76, 0xe4, 0xb6, 0xa6, 0x7b, 0x16, 0xce, 0x49,
|
||||
0x3e, 0xe0, 0x48, 0x26, 0x52, 0x75, 0x8d, 0xa2, 0x5b, 0x76, 0xfe, 0x73, 0x1d, 0x1c, 0x83, 0x57,
|
||||
0x2c, 0x88, 0x4e, 0xa5, 0xd0, 0x48, 0x9e, 0xc0, 0xba, 0xdd, 0x5d, 0xea, 0xd8, 0x94, 0xd5, 0xd6,
|
||||
0xec, 0xc2, 0xc3, 0x89, 0xbf, 0x7f, 0xe3, 0x00, 0x14, 0x73, 0x24, 0x7b, 0xf0, 0x5f, 0xa1, 0xce,
|
||||
0xc5, 0xa5, 0x90, 0xd7, 0xc2, 0x2b, 0x91, 0x87, 0xb0, 0xbb, 0x68, 0x27, 0x7c, 0xc0, 0x0d, 0xc6,
|
||||
0x9e, 0x43, 0x08, 0x6c, 0x17, 0xa0, 0xcd, 0x32, 0xed, 0x95, 0xc9, 0x2e, 0xec, 0x14, 0xde, 0x27,
|
||||
0xc4, 0x4b, 0xed, 0x55, 0xc8, 0xff, 0xe0, 0x15, 0xe6, 0x99, 0x14, 0xe6, 0x42, 0x7b, 0x6b, 0xcb,
|
||||
0x47, 0x3f, 0x23, 0x53, 0xda, 0x5b, 0x7f, 0xf9, 0xe2, 0xe7, 0xc8, 0x77, 0x6e, 0x47, 0xbe, 0xf3,
|
||||
0x67, 0xe4, 0x3b, 0x37, 0x63, 0xbf, 0x74, 0x3b, 0xf6, 0x4b, 0xbf, 0xc7, 0x7e, 0xe9, 0x4b, 0xf0,
|
||||
0xef, 0x1f, 0xf8, 0xdb, 0x86, 0x7d, 0x1d, 0xff, 0x0d, 0x00, 0x00, 0xff, 0xff, 0xfe, 0xc4, 0x11,
|
||||
0x59, 0xed, 0x03, 0x00, 0x00,
|
||||
}
|
||||
|
||||
func (m *Feature) Marshal() (dAtA []byte, err error) {
|
||||
|
@ -505,15 +485,10 @@ func (m *Feature) MarshalToSizedBuffer(dAtA []byte) (int, error) {
|
|||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if m.ValueUint != 0 {
|
||||
i = encodeVarintPaymentserviceTiers(dAtA, i, uint64(m.ValueUint))
|
||||
i--
|
||||
dAtA[i] = 0x10
|
||||
}
|
||||
if len(m.ValueStr) > 0 {
|
||||
i -= len(m.ValueStr)
|
||||
copy(dAtA[i:], m.ValueStr)
|
||||
i = encodeVarintPaymentserviceTiers(dAtA, i, uint64(len(m.ValueStr)))
|
||||
if len(m.Description) > 0 {
|
||||
i -= len(m.Description)
|
||||
copy(dAtA[i:], m.Description)
|
||||
i = encodeVarintPaymentserviceTiers(dAtA, i, uint64(len(m.Description)))
|
||||
i--
|
||||
dAtA[i] = 0xa
|
||||
}
|
||||
|
@ -540,11 +515,6 @@ func (m *GetTiersRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) {
|
|||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if m.PaymentMethod != 0 {
|
||||
i = encodeVarintPaymentserviceTiers(dAtA, i, uint64(m.PaymentMethod))
|
||||
i--
|
||||
dAtA[i] = 0x18
|
||||
}
|
||||
if len(m.Locale) > 0 {
|
||||
i -= len(m.Locale)
|
||||
copy(dAtA[i:], m.Locale)
|
||||
|
@ -619,28 +589,23 @@ func (m *TierData) MarshalToSizedBuffer(dAtA []byte) (int, error) {
|
|||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if len(m.ColorStr) > 0 {
|
||||
i -= len(m.ColorStr)
|
||||
copy(dAtA[i:], m.ColorStr)
|
||||
i = encodeVarintPaymentserviceTiers(dAtA, i, uint64(len(m.ColorStr)))
|
||||
i--
|
||||
dAtA[i] = 0x6a
|
||||
}
|
||||
if len(m.Features) > 0 {
|
||||
for k := range m.Features {
|
||||
v := m.Features[k]
|
||||
baseI := i
|
||||
if v != nil {
|
||||
{
|
||||
size, err := v.MarshalToSizedBuffer(dAtA[:i])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i -= size
|
||||
i = encodeVarintPaymentserviceTiers(dAtA, i, uint64(size))
|
||||
for iNdEx := len(m.Features) - 1; iNdEx >= 0; iNdEx-- {
|
||||
{
|
||||
size, err := m.Features[iNdEx].MarshalToSizedBuffer(dAtA[:i])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i--
|
||||
dAtA[i] = 0x12
|
||||
i -= size
|
||||
i = encodeVarintPaymentserviceTiers(dAtA, i, uint64(size))
|
||||
}
|
||||
i -= len(k)
|
||||
copy(dAtA[i:], k)
|
||||
i = encodeVarintPaymentserviceTiers(dAtA, i, uint64(len(k)))
|
||||
i--
|
||||
dAtA[i] = 0xa
|
||||
i = encodeVarintPaymentserviceTiers(dAtA, i, uint64(baseI-i))
|
||||
i--
|
||||
dAtA[i] = 0x62
|
||||
}
|
||||
|
@ -776,13 +741,10 @@ func (m *Feature) Size() (n int) {
|
|||
}
|
||||
var l int
|
||||
_ = l
|
||||
l = len(m.ValueStr)
|
||||
l = len(m.Description)
|
||||
if l > 0 {
|
||||
n += 1 + l + sovPaymentserviceTiers(uint64(l))
|
||||
}
|
||||
if m.ValueUint != 0 {
|
||||
n += 1 + sovPaymentserviceTiers(uint64(m.ValueUint))
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
|
@ -800,9 +762,6 @@ func (m *GetTiersRequest) Size() (n int) {
|
|||
if l > 0 {
|
||||
n += 1 + l + sovPaymentserviceTiers(uint64(l))
|
||||
}
|
||||
if m.PaymentMethod != 0 {
|
||||
n += 1 + sovPaymentserviceTiers(uint64(m.PaymentMethod))
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
|
@ -865,18 +824,15 @@ func (m *TierData) Size() (n int) {
|
|||
n += 1 + sovPaymentserviceTiers(uint64(m.AnyNameMinLength))
|
||||
}
|
||||
if len(m.Features) > 0 {
|
||||
for k, v := range m.Features {
|
||||
_ = k
|
||||
_ = v
|
||||
l = 0
|
||||
if v != nil {
|
||||
l = v.Size()
|
||||
l += 1 + sovPaymentserviceTiers(uint64(l))
|
||||
}
|
||||
mapEntrySize := 1 + len(k) + sovPaymentserviceTiers(uint64(len(k))) + l
|
||||
n += mapEntrySize + 1 + sovPaymentserviceTiers(uint64(mapEntrySize))
|
||||
for _, e := range m.Features {
|
||||
l = e.Size()
|
||||
n += 1 + l + sovPaymentserviceTiers(uint64(l))
|
||||
}
|
||||
}
|
||||
l = len(m.ColorStr)
|
||||
if l > 0 {
|
||||
n += 1 + l + sovPaymentserviceTiers(uint64(l))
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
|
@ -932,7 +888,7 @@ func (m *Feature) Unmarshal(dAtA []byte) error {
|
|||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field ValueStr", wireType)
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Description", wireType)
|
||||
}
|
||||
var stringLen uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
|
@ -960,27 +916,8 @@ func (m *Feature) Unmarshal(dAtA []byte) error {
|
|||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.ValueStr = string(dAtA[iNdEx:postIndex])
|
||||
m.Description = string(dAtA[iNdEx:postIndex])
|
||||
iNdEx = postIndex
|
||||
case 2:
|
||||
if wireType != 0 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field ValueUint", wireType)
|
||||
}
|
||||
m.ValueUint = 0
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowPaymentserviceTiers
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
m.ValueUint |= uint32(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skipPaymentserviceTiers(dAtA[iNdEx:])
|
||||
|
@ -1095,25 +1032,6 @@ func (m *GetTiersRequest) Unmarshal(dAtA []byte) error {
|
|||
}
|
||||
m.Locale = string(dAtA[iNdEx:postIndex])
|
||||
iNdEx = postIndex
|
||||
case 3:
|
||||
if wireType != 0 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field PaymentMethod", wireType)
|
||||
}
|
||||
m.PaymentMethod = 0
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowPaymentserviceTiers
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
m.PaymentMethod |= uint32(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skipPaymentserviceTiers(dAtA[iNdEx:])
|
||||
|
@ -1549,105 +1467,42 @@ func (m *TierData) Unmarshal(dAtA []byte) error {
|
|||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
if m.Features == nil {
|
||||
m.Features = make(map[string]*Feature)
|
||||
m.Features = append(m.Features, &Feature{})
|
||||
if err := m.Features[len(m.Features)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
var mapkey string
|
||||
var mapvalue *Feature
|
||||
for iNdEx < postIndex {
|
||||
entryPreIndex := iNdEx
|
||||
var wire uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowPaymentserviceTiers
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
wire |= uint64(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
iNdEx = postIndex
|
||||
case 13:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field ColorStr", wireType)
|
||||
}
|
||||
var stringLen uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowPaymentserviceTiers
|
||||
}
|
||||
fieldNum := int32(wire >> 3)
|
||||
if fieldNum == 1 {
|
||||
var stringLenmapkey uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowPaymentserviceTiers
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
stringLenmapkey |= uint64(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
intStringLenmapkey := int(stringLenmapkey)
|
||||
if intStringLenmapkey < 0 {
|
||||
return ErrInvalidLengthPaymentserviceTiers
|
||||
}
|
||||
postStringIndexmapkey := iNdEx + intStringLenmapkey
|
||||
if postStringIndexmapkey < 0 {
|
||||
return ErrInvalidLengthPaymentserviceTiers
|
||||
}
|
||||
if postStringIndexmapkey > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
mapkey = string(dAtA[iNdEx:postStringIndexmapkey])
|
||||
iNdEx = postStringIndexmapkey
|
||||
} else if fieldNum == 2 {
|
||||
var mapmsglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowPaymentserviceTiers
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
mapmsglen |= int(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
if mapmsglen < 0 {
|
||||
return ErrInvalidLengthPaymentserviceTiers
|
||||
}
|
||||
postmsgIndex := iNdEx + mapmsglen
|
||||
if postmsgIndex < 0 {
|
||||
return ErrInvalidLengthPaymentserviceTiers
|
||||
}
|
||||
if postmsgIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
mapvalue = &Feature{}
|
||||
if err := mapvalue.Unmarshal(dAtA[iNdEx:postmsgIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
iNdEx = postmsgIndex
|
||||
} else {
|
||||
iNdEx = entryPreIndex
|
||||
skippy, err := skipPaymentserviceTiers(dAtA[iNdEx:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if (skippy < 0) || (iNdEx+skippy) < 0 {
|
||||
return ErrInvalidLengthPaymentserviceTiers
|
||||
}
|
||||
if (iNdEx + skippy) > postIndex {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
iNdEx += skippy
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
stringLen |= uint64(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
m.Features[mapkey] = mapvalue
|
||||
intStringLen := int(stringLen)
|
||||
if intStringLen < 0 {
|
||||
return ErrInvalidLengthPaymentserviceTiers
|
||||
}
|
||||
postIndex := iNdEx + intStringLen
|
||||
if postIndex < 0 {
|
||||
return ErrInvalidLengthPaymentserviceTiers
|
||||
}
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.ColorStr = string(dAtA[iNdEx:postIndex])
|
||||
iNdEx = postIndex
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
|
|
|
@ -60,8 +60,8 @@ message GetSubscriptionRequestSigned {
|
|||
}
|
||||
|
||||
message GetSubscriptionResponse {
|
||||
// was SubscriptionTier before, changed to int32 to allow us to use dynamic tiers
|
||||
int32 tier = 1;
|
||||
// was SubscriptionTier before, changed to uint32 to allow us to use dynamic tiers
|
||||
uint32 tier = 1;
|
||||
SubscriptionStatus status = 2;
|
||||
uint64 dateStarted = 3;
|
||||
uint64 dateEnds = 4;
|
||||
|
@ -82,8 +82,8 @@ message BuySubscriptionRequest {
|
|||
// this is required to reserve a name for the owner (later that is done by user)
|
||||
// in the following format: "0x7a250d5630b4cf539739df2c5dacb4c659f2488d"
|
||||
string ownerEthAddress = 2;
|
||||
// was SubscriptionTier before, changed to int32 to allow us to use dynamic tiers
|
||||
int32 requestedTier = 3;
|
||||
// was SubscriptionTier before, changed to uint32 to allow us to use dynamic tiers
|
||||
uint32 requestedTier = 3;
|
||||
PaymentMethod paymentMethod = 4;
|
||||
// if empty - then no name requested
|
||||
// if non-empty - PP node will register that name on behalf of the user
|
||||
|
@ -101,6 +101,8 @@ message BuySubscriptionResponse {
|
|||
// will feature current billing ID
|
||||
// stripe.com/?client_reference_id=1234
|
||||
string paymentUrl = 1;
|
||||
// billingID is passed via mobile client to payment platform
|
||||
string billingID = 2;
|
||||
}
|
||||
|
||||
message FinalizeSubscriptionRequest {
|
||||
|
@ -188,7 +190,7 @@ message VerifyEmailRequestSigned {
|
|||
}
|
||||
|
||||
message IsNameValidRequest {
|
||||
int32 requestedTier = 1;
|
||||
uint32 requestedTier = 1;
|
||||
string requestedAnyName = 2;
|
||||
}
|
||||
|
||||
|
@ -208,6 +210,32 @@ message IsNameValidResponse {
|
|||
}
|
||||
}
|
||||
|
||||
enum ErrorCodes {
|
||||
Unexpected = 0;
|
||||
|
||||
EthAddressEmpty = 1;
|
||||
InvalidSignature = 2;
|
||||
TierWrong = 3;
|
||||
TierNotFound = 4;
|
||||
TierInactive = 5;
|
||||
PaymentMethodWrong = 6;
|
||||
BadAnyName = 7;
|
||||
Unknown = 8;
|
||||
|
||||
SubsAlreadyActive = 9;
|
||||
SubsNotFound = 10;
|
||||
SubsWrongState = 11;
|
||||
|
||||
EmailWrongFormat = 12;
|
||||
EmailAlreadyVerified = 13;
|
||||
EmailAlreadySent = 14;
|
||||
EmailFailedToSend = 15;
|
||||
EmailExpired = 16;
|
||||
EmailWrongCode = 17;
|
||||
|
||||
ErrorOffset = 600;
|
||||
}
|
||||
|
||||
// NOTICE:
|
||||
// 1 - User can not ask for a payment/other links on behalf of another user (a signature is required)
|
||||
// 2 - Admin can do that on behalf of any user
|
||||
|
|
|
@ -12,14 +12,7 @@ enum PeriodType {
|
|||
}
|
||||
|
||||
message Feature {
|
||||
// "invites"
|
||||
// "GBs"
|
||||
// "spaces"
|
||||
// ...
|
||||
string valueStr = 1;
|
||||
|
||||
// each uint is a value of the feature
|
||||
uint32 valueUint = 2;
|
||||
string description = 1;
|
||||
}
|
||||
|
||||
message GetTiersRequest {
|
||||
|
@ -28,9 +21,6 @@ message GetTiersRequest {
|
|||
string ownerAnyId = 1;
|
||||
|
||||
string locale = 2;
|
||||
|
||||
// see PaymentMethod enum
|
||||
uint32 paymentMethod = 3;
|
||||
}
|
||||
|
||||
message GetTiersRequestSigned {
|
||||
|
@ -68,9 +58,9 @@ message TierData {
|
|||
// somename.any - len of 8
|
||||
uint32 anyNameMinLength = 11;
|
||||
// each tier has a set of features
|
||||
// each feature has a unique key: "storage", "invites", etc
|
||||
// not using enum here to provide dynamic feature list
|
||||
map<string, Feature> features = 12;
|
||||
repeated Feature features = 12;
|
||||
// green, blue, red, purple or custom color in string format #ff00ff
|
||||
string colorStr = 13;
|
||||
}
|
||||
|
||||
message GetTiersResponse {
|
||||
|
|
|
@ -3,10 +3,12 @@ package periodicsync
|
|||
|
||||
import (
|
||||
"context"
|
||||
"github.com/anyproto/any-sync/app/logger"
|
||||
"go.uber.org/zap"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/anyproto/any-sync/app/logger"
|
||||
)
|
||||
|
||||
type PeriodicSync interface {
|
||||
|
@ -17,39 +19,40 @@ type PeriodicSync interface {
|
|||
type SyncerFunc func(ctx context.Context) error
|
||||
|
||||
func NewPeriodicSync(periodSeconds int, timeout time.Duration, caller SyncerFunc, l logger.CtxLogger) PeriodicSync {
|
||||
// TODO: rename to PeriodicCall (including folders) and do PRs in all repos where we are using this
|
||||
// https://linear.app/anytype/issue/GO-1241/change-periodicsync-component-to-periodiccall
|
||||
return NewPeriodicSyncDuration(time.Duration(periodSeconds)*time.Second, timeout, caller, l)
|
||||
}
|
||||
|
||||
func NewPeriodicSyncDuration(periodicLoopInterval, timeout time.Duration, caller SyncerFunc, l logger.CtxLogger) PeriodicSync {
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
ctx = logger.CtxWithFields(ctx, zap.String("rootOp", "periodicCall"))
|
||||
return &periodicCall{
|
||||
caller: caller,
|
||||
log: l,
|
||||
loopCtx: ctx,
|
||||
loopCancel: cancel,
|
||||
loopDone: make(chan struct{}),
|
||||
periodSeconds: periodSeconds,
|
||||
timeout: timeout,
|
||||
caller: caller,
|
||||
log: l,
|
||||
loopCtx: ctx,
|
||||
loopCancel: cancel,
|
||||
loopDone: make(chan struct{}),
|
||||
period: periodicLoopInterval,
|
||||
timeout: timeout,
|
||||
}
|
||||
}
|
||||
|
||||
type periodicCall struct {
|
||||
log logger.CtxLogger
|
||||
caller SyncerFunc
|
||||
loopCtx context.Context
|
||||
loopCancel context.CancelFunc
|
||||
loopDone chan struct{}
|
||||
periodSeconds int
|
||||
timeout time.Duration
|
||||
isRunning atomic.Bool
|
||||
log logger.CtxLogger
|
||||
caller SyncerFunc
|
||||
loopCtx context.Context
|
||||
loopCancel context.CancelFunc
|
||||
loopDone chan struct{}
|
||||
period time.Duration
|
||||
timeout time.Duration
|
||||
isRunning atomic.Bool
|
||||
}
|
||||
|
||||
func (p *periodicCall) Run() {
|
||||
p.isRunning.Store(true)
|
||||
go p.loop(p.periodSeconds)
|
||||
go p.loop(p.period)
|
||||
}
|
||||
|
||||
func (p *periodicCall) loop(periodSeconds int) {
|
||||
period := time.Duration(periodSeconds) * time.Second
|
||||
func (p *periodicCall) loop(period time.Duration) {
|
||||
defer close(p.loopDone)
|
||||
doCall := func() {
|
||||
ctx := p.loopCtx
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue