mirror of
https://github.com/anyproto/any-sync.git
synced 2025-06-07 21:47:02 +09:00
Merge pull request #464 from anyproto/GO-5609-acl-space-pull
GO-5609: Add space pull with acls
This commit is contained in:
commit
f2b41f6274
9 changed files with 787 additions and 133 deletions
|
@ -25,9 +25,13 @@ func (t *TreeStatsCollector) Register(tree *syncTree) {
|
|||
|
||||
func (t *TreeStatsCollector) Collect() []TreeStats {
|
||||
t.mutex.Lock()
|
||||
defer t.mutex.Unlock()
|
||||
stats := make([]TreeStats, 0, len(t.trees))
|
||||
trees := make([]*syncTree, 0, len(t.trees))
|
||||
for _, tree := range t.trees {
|
||||
trees = append(trees, tree)
|
||||
}
|
||||
t.mutex.Unlock()
|
||||
stats := make([]TreeStats, 0, len(t.trees))
|
||||
for _, tree := range trees {
|
||||
tree.Lock()
|
||||
stats = append(stats, TreeStats{
|
||||
TreeLen: tree.Len(),
|
||||
|
|
|
@ -37,6 +37,7 @@ type SpaceDescription struct {
|
|||
AclPayload []byte
|
||||
SpaceSettingsId string
|
||||
SpaceSettingsPayload []byte
|
||||
AclRecords []*spacesyncproto.AclRecord
|
||||
}
|
||||
|
||||
func NewSpaceId(id string, repKey uint64) string {
|
||||
|
@ -95,6 +96,19 @@ func (s *space) Description(ctx context.Context) (desc SpaceDescription, err err
|
|||
if err != nil {
|
||||
return
|
||||
}
|
||||
s.aclList.RLock()
|
||||
defer s.aclList.RUnlock()
|
||||
recs, err := s.aclList.RecordsAfter(ctx, "")
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
aclRecs := make([]*spacesyncproto.AclRecord, 0, len(recs))
|
||||
for _, rec := range recs {
|
||||
aclRecs = append(aclRecs, &spacesyncproto.AclRecord{
|
||||
Id: rec.Id,
|
||||
AclPayload: rec.Payload,
|
||||
})
|
||||
}
|
||||
root := s.aclList.Root()
|
||||
settingsStorage, err := s.storage.TreeStorage(ctx, state.SettingsId)
|
||||
if err != nil {
|
||||
|
@ -114,6 +128,7 @@ func (s *space) Description(ctx context.Context) (desc SpaceDescription, err err
|
|||
AclPayload: root.Payload,
|
||||
SpaceSettingsId: settingsRoot.Id,
|
||||
SpaceSettingsPayload: settingsRoot.RawChange,
|
||||
AclRecords: aclRecs,
|
||||
}
|
||||
return
|
||||
}
|
||||
|
|
296
commonspace/spacepull_test.go
Normal file
296
commonspace/spacepull_test.go
Normal file
|
@ -0,0 +1,296 @@
|
|||
package commonspace
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/require"
|
||||
"go.uber.org/mock/gomock"
|
||||
|
||||
"github.com/anyproto/any-sync/app"
|
||||
"github.com/anyproto/any-sync/commonspace/config"
|
||||
"github.com/anyproto/any-sync/commonspace/credentialprovider"
|
||||
"github.com/anyproto/any-sync/commonspace/object/acl/list"
|
||||
"github.com/anyproto/any-sync/commonspace/object/acl/recordverifier"
|
||||
"github.com/anyproto/any-sync/commonspace/object/tree/objecttree"
|
||||
"github.com/anyproto/any-sync/commonspace/object/tree/treestorage"
|
||||
"github.com/anyproto/any-sync/commonspace/object/treemanager"
|
||||
"github.com/anyproto/any-sync/commonspace/spacepayloads"
|
||||
"github.com/anyproto/any-sync/commonspace/spacestorage"
|
||||
"github.com/anyproto/any-sync/commonspace/spacesyncproto"
|
||||
"github.com/anyproto/any-sync/commonspace/syncstatus"
|
||||
"github.com/anyproto/any-sync/net/peer"
|
||||
"github.com/anyproto/any-sync/net/rpc/rpctest"
|
||||
"github.com/anyproto/any-sync/net/streampool"
|
||||
"github.com/anyproto/any-sync/nodeconf/testconf"
|
||||
"github.com/anyproto/any-sync/testutil/accounttest"
|
||||
"github.com/anyproto/any-sync/util/crypto"
|
||||
"github.com/anyproto/any-sync/util/syncqueues"
|
||||
)
|
||||
|
||||
func TestSpaceService_SpacePull(t *testing.T) {
|
||||
var makeClientServer = func(t *testing.T) (fxC, fxS *spacePullFixture, peerId string) {
|
||||
fxC = newSpacePullFixture(t)
|
||||
fxS = newSpacePullFixture(t)
|
||||
peerId = "peer"
|
||||
mcS, mcC := rpctest.MultiConnPair(peerId, peerId+"client")
|
||||
pS, err := peer.NewPeer(mcS, fxC.ts)
|
||||
require.NoError(t, err)
|
||||
fxC.tp.AddPeer(ctx, pS)
|
||||
_, err = peer.NewPeer(mcC, fxS.ts)
|
||||
fxC.managerProvider.peer = pS
|
||||
require.NoError(t, err)
|
||||
return
|
||||
}
|
||||
|
||||
t.Run("successful space pull", func(t *testing.T) {
|
||||
fxC, fxS, _ := makeClientServer(t)
|
||||
defer fxC.Finish(t)
|
||||
defer fxS.Finish(t)
|
||||
|
||||
spaceId, payload := fxS.createTestSpace(t)
|
||||
|
||||
space, err := fxC.spaceService.NewSpace(ctx, spaceId, Deps{
|
||||
SyncStatus: syncstatus.NewNoOpSyncStatus(),
|
||||
TreeSyncer: &mockTreeSyncer{},
|
||||
AccountService: fxC.account,
|
||||
})
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, space.Init(ctx))
|
||||
require.NotNil(t, space)
|
||||
require.Equal(t, spaceId, space.Id())
|
||||
|
||||
storage := space.Storage()
|
||||
state, err := storage.StateStorage().GetState(ctx)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, spaceId, state.SpaceId)
|
||||
require.Equal(t, payload.SpaceHeaderWithId.Id, state.SpaceId)
|
||||
})
|
||||
|
||||
t.Run("space pull with acl records", func(t *testing.T) {
|
||||
fxC, fxS, _ := makeClientServer(t)
|
||||
defer fxC.Finish(t)
|
||||
defer fxS.Finish(t)
|
||||
|
||||
spaceId, recLen, payload := fxS.createTestSpaceWithAclRecords(t)
|
||||
|
||||
space, err := fxC.spaceService.NewSpace(ctx, spaceId, Deps{
|
||||
SyncStatus: syncstatus.NewNoOpSyncStatus(),
|
||||
TreeSyncer: &mockTreeSyncer{},
|
||||
AccountService: fxC.account,
|
||||
recordVerifier: recordverifier.NewValidateFull(),
|
||||
})
|
||||
require.NoError(t, err)
|
||||
require.NotNil(t, space)
|
||||
require.NoError(t, space.Init(ctx))
|
||||
records, err := space.Acl().RecordsAfter(ctx, "")
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, recLen, len(records))
|
||||
storage := space.Storage()
|
||||
state, err := storage.StateStorage().GetState(ctx)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, spaceId, state.SpaceId)
|
||||
require.Equal(t, payload.SpaceHeaderWithId.Id, state.SpaceId)
|
||||
})
|
||||
}
|
||||
|
||||
type spacePullFixture struct {
|
||||
*spaceService
|
||||
app *app.App
|
||||
ctrl *gomock.Controller
|
||||
ts *rpctest.TestServer
|
||||
tp *rpctest.TestPool
|
||||
tmpDir string
|
||||
account *accounttest.AccountTestService
|
||||
configService *testconf.StubConf
|
||||
storage spacestorage.SpaceStorageProvider
|
||||
managerProvider *mockPeerManagerProvider
|
||||
}
|
||||
|
||||
func newSpacePullFixture(t *testing.T) (fx *spacePullFixture) {
|
||||
ts := rpctest.NewTestServer()
|
||||
fx = &spacePullFixture{
|
||||
spaceService: New().(*spaceService),
|
||||
ctrl: gomock.NewController(t),
|
||||
app: new(app.App),
|
||||
ts: ts,
|
||||
tp: rpctest.NewTestPool(),
|
||||
tmpDir: t.TempDir(),
|
||||
account: &accounttest.AccountTestService{},
|
||||
configService: &testconf.StubConf{},
|
||||
storage: &spaceStorageProvider{rootPath: t.TempDir()},
|
||||
managerProvider: &mockPeerManagerProvider{},
|
||||
}
|
||||
|
||||
configGetter := &mockSpaceConfigGetter{}
|
||||
|
||||
fx.app.Register(configGetter).
|
||||
Register(mockCoordinatorClient{}).
|
||||
Register(mockNodeClient{}).
|
||||
Register(credentialprovider.NewNoOp()).
|
||||
Register(streampool.New()).
|
||||
Register(newStreamOpener("spaceId")).
|
||||
Register(fx.account).
|
||||
Register(fx.storage).
|
||||
Register(fx.managerProvider).
|
||||
Register(syncqueues.New()).
|
||||
Register(&mockTreeManager{}).
|
||||
Register(fx.spaceService).
|
||||
Register(fx.tp).
|
||||
Register(fx.ts).
|
||||
Register(fx.configService)
|
||||
|
||||
require.NoError(t, spacesyncproto.DRPCRegisterSpaceSync(ts, &testSpaceSyncServer{spaceService: fx.spaceService}))
|
||||
require.NoError(t, fx.app.Start(ctx))
|
||||
|
||||
return fx
|
||||
}
|
||||
|
||||
func (fx *spacePullFixture) Finish(t *testing.T) {
|
||||
fx.ctrl.Finish()
|
||||
}
|
||||
|
||||
func (fx *spacePullFixture) createTestSpace(t *testing.T) (string, spacestorage.SpaceStorageCreatePayload) {
|
||||
keys := fx.account.Account()
|
||||
masterKey, _, err := crypto.GenerateRandomEd25519KeyPair()
|
||||
require.NoError(t, err)
|
||||
metaKey, _, err := crypto.GenerateRandomEd25519KeyPair()
|
||||
require.NoError(t, err)
|
||||
readKey := crypto.NewAES()
|
||||
metadata := []byte("metadata")
|
||||
|
||||
payload := spacepayloads.SpaceCreatePayload{
|
||||
SigningKey: keys.SignKey,
|
||||
SpaceType: "test",
|
||||
ReplicationKey: 1,
|
||||
SpacePayload: nil,
|
||||
MasterKey: masterKey,
|
||||
ReadKey: readKey,
|
||||
MetadataKey: metaKey,
|
||||
Metadata: metadata,
|
||||
}
|
||||
|
||||
createPayload, err := spacepayloads.StoragePayloadForSpaceCreate(payload)
|
||||
require.NoError(t, err)
|
||||
storage, err := fx.storage.CreateSpaceStorage(ctx, createPayload)
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, storage.Close(ctx))
|
||||
|
||||
return createPayload.SpaceHeaderWithId.Id, createPayload
|
||||
}
|
||||
|
||||
func (fx *spacePullFixture) createTestSpaceWithAclRecords(t *testing.T) (string, int, spacestorage.SpaceStorageCreatePayload) {
|
||||
spaceId, payload := fx.createTestSpace(t)
|
||||
keys := fx.account.Account()
|
||||
|
||||
executor := list.NewExternalKeysAclExecutor(spaceId, keys, []byte("metadata"), payload.AclWithId)
|
||||
cmds := []string{
|
||||
"0.init::0",
|
||||
"0.invite::invId1",
|
||||
"0.invite::invId2",
|
||||
}
|
||||
|
||||
for _, cmd := range cmds {
|
||||
err := executor.Execute(cmd)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
allRecords, err := executor.ActualAccounts()["0"].Acl.RecordsAfter(ctx, "")
|
||||
require.NoError(t, err)
|
||||
|
||||
storage, err := fx.storage.WaitSpaceStorage(ctx, spaceId)
|
||||
require.NoError(t, err)
|
||||
defer storage.Close(ctx)
|
||||
|
||||
aclStorage, err := storage.AclStorage()
|
||||
require.NoError(t, err)
|
||||
|
||||
for i, rec := range allRecords[1:] { // Skip first record as it's already there
|
||||
err := aclStorage.AddAll(ctx, []list.StorageRecord{
|
||||
{
|
||||
RawRecord: rec.Payload,
|
||||
Id: rec.Id,
|
||||
PrevId: allRecords[i].Id,
|
||||
Order: i + 2,
|
||||
ChangeSize: len(rec.Payload),
|
||||
},
|
||||
})
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
return spaceId, len(allRecords), payload
|
||||
}
|
||||
|
||||
type testSpaceSyncServer struct {
|
||||
spacesyncproto.DRPCSpaceSyncUnimplementedServer
|
||||
spaceService SpaceService
|
||||
}
|
||||
|
||||
func (t *testSpaceSyncServer) SpacePull(ctx context.Context, req *spacesyncproto.SpacePullRequest) (*spacesyncproto.SpacePullResponse, error) {
|
||||
sp, err := t.spaceService.NewSpace(ctx, req.Id, Deps{
|
||||
SyncStatus: syncstatus.NewNoOpSyncStatus(),
|
||||
TreeSyncer: mockTreeSyncer{},
|
||||
recordVerifier: recordverifier.NewValidateFull(),
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = sp.Init(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
spaceDesc, err := sp.Description(ctx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &spacesyncproto.SpacePullResponse{
|
||||
Payload: &spacesyncproto.SpacePayload{
|
||||
SpaceHeader: spaceDesc.SpaceHeader,
|
||||
AclPayloadId: spaceDesc.AclId,
|
||||
AclPayload: spaceDesc.AclPayload,
|
||||
SpaceSettingsPayload: spaceDesc.SpaceSettingsPayload,
|
||||
SpaceSettingsPayloadId: spaceDesc.SpaceSettingsId,
|
||||
},
|
||||
AclRecords: spaceDesc.AclRecords,
|
||||
}, nil
|
||||
}
|
||||
|
||||
type mockSpaceConfigGetter struct{}
|
||||
|
||||
func (m *mockSpaceConfigGetter) GetStreamConfig() streampool.StreamConfig {
|
||||
return streampool.StreamConfig{}
|
||||
}
|
||||
|
||||
func (m *mockSpaceConfigGetter) Init(a *app.App) error { return nil }
|
||||
func (m *mockSpaceConfigGetter) Name() string { return "config" }
|
||||
func (m *mockSpaceConfigGetter) GetSpace() config.Config {
|
||||
return config.Config{
|
||||
GCTTL: 60,
|
||||
SyncPeriod: 5,
|
||||
KeepTreeDataInMemory: true,
|
||||
}
|
||||
}
|
||||
|
||||
type mockTreeManager struct{}
|
||||
|
||||
func (m *mockTreeManager) Init(a *app.App) error { return nil }
|
||||
func (m *mockTreeManager) Name() string { return treemanager.CName }
|
||||
func (m *mockTreeManager) Run(ctx context.Context) error { return nil }
|
||||
func (m *mockTreeManager) Close(ctx context.Context) error { return nil }
|
||||
func (m *mockTreeManager) GetTree(ctx context.Context, spaceId, treeId string) (objecttree.ObjectTree, error) {
|
||||
return nil, nil
|
||||
}
|
||||
func (m *mockTreeManager) CreateTree(ctx context.Context, spaceId string) (objecttree.ObjectTree, error) {
|
||||
return nil, nil
|
||||
}
|
||||
func (m *mockTreeManager) PutTree(ctx context.Context, payload treestorage.TreeStorageCreatePayload) (objecttree.ObjectTree, error) {
|
||||
return nil, nil
|
||||
}
|
||||
func (m *mockTreeManager) DeleteTree(ctx context.Context, spaceId, treeId string) error { return nil }
|
||||
func (m *mockTreeManager) MarkTreeDeleted(ctx context.Context, spaceId, treeId string) error {
|
||||
return nil
|
||||
}
|
||||
func (m *mockTreeManager) ValidateAndPutTree(ctx context.Context, spaceId string, payload treestorage.TreeStorageCreatePayload) error {
|
||||
return nil
|
||||
}
|
|
@ -13,6 +13,7 @@ import (
|
|||
|
||||
"github.com/anyproto/any-sync/commonspace/acl/aclclient"
|
||||
"github.com/anyproto/any-sync/commonspace/deletionmanager"
|
||||
"github.com/anyproto/any-sync/commonspace/object/acl/list"
|
||||
"github.com/anyproto/any-sync/commonspace/object/acl/recordverifier"
|
||||
"github.com/anyproto/any-sync/commonspace/object/keyvalue"
|
||||
"github.com/anyproto/any-sync/commonspace/object/keyvalue/keyvaluestorage"
|
||||
|
@ -161,7 +162,7 @@ func (s *spaceService) NewSpace(ctx context.Context, id string, deps Deps) (Spac
|
|||
return nil, err
|
||||
}
|
||||
} else {
|
||||
st, err = s.getSpaceStorageFromRemote(ctx, id)
|
||||
st, err = s.getSpaceStorageFromRemote(ctx, id, deps)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -243,7 +244,7 @@ func (s *spaceService) addSpaceStorage(ctx context.Context, spaceDescription Spa
|
|||
return
|
||||
}
|
||||
|
||||
func (s *spaceService) getSpaceStorageFromRemote(ctx context.Context, id string) (st spacestorage.SpaceStorage, err error) {
|
||||
func (s *spaceService) getSpaceStorageFromRemote(ctx context.Context, id string, deps Deps) (st spacestorage.SpaceStorage, err error) {
|
||||
// we can't connect to client if it is a node
|
||||
if s.configurationService.IsResponsible(id) {
|
||||
err = spacesyncproto.ErrSpaceMissing
|
||||
|
@ -282,7 +283,7 @@ func (s *spaceService) getSpaceStorageFromRemote(ctx context.Context, id string)
|
|||
}
|
||||
}
|
||||
for i, p := range peers {
|
||||
if st, err = s.spacePullWithPeer(ctx, p, id); err != nil {
|
||||
if st, err = s.spacePullWithPeer(ctx, p, id, deps); err != nil {
|
||||
if i+1 == len(peers) {
|
||||
return
|
||||
} else {
|
||||
|
@ -295,7 +296,7 @@ func (s *spaceService) getSpaceStorageFromRemote(ctx context.Context, id string)
|
|||
return nil, net.ErrUnableToConnect
|
||||
}
|
||||
|
||||
func (s *spaceService) spacePullWithPeer(ctx context.Context, p peer.Peer, id string) (st spacestorage.SpaceStorage, err error) {
|
||||
func (s *spaceService) spacePullWithPeer(ctx context.Context, p peer.Peer, id string, deps Deps) (st spacestorage.SpaceStorage, err error) {
|
||||
var res *spacesyncproto.SpacePullResponse
|
||||
err = p.DoDrpc(ctx, func(conn drpc.Conn) error {
|
||||
cl := spacesyncproto.NewDRPCSpaceSyncClient(conn)
|
||||
|
@ -307,7 +308,7 @@ func (s *spaceService) spacePullWithPeer(ctx context.Context, p peer.Peer, id st
|
|||
return
|
||||
}
|
||||
|
||||
return s.createSpaceStorage(ctx, spacestorage.SpaceStorageCreatePayload{
|
||||
st, err = s.createSpaceStorage(ctx, spacestorage.SpaceStorageCreatePayload{
|
||||
AclWithId: &consensusproto.RawRecordWithId{
|
||||
Payload: res.Payload.AclPayload,
|
||||
Id: res.Payload.AclPayloadId,
|
||||
|
@ -318,6 +319,35 @@ func (s *spaceService) spacePullWithPeer(ctx context.Context, p peer.Peer, id st
|
|||
},
|
||||
SpaceHeaderWithId: res.Payload.SpaceHeader,
|
||||
})
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
if res.AclRecords != nil {
|
||||
aclSt, err := st.AclStorage()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
recordVerifier := recordverifier.New()
|
||||
if deps.recordVerifier != nil {
|
||||
recordVerifier = deps.recordVerifier
|
||||
}
|
||||
acl, err := list.BuildAclListWithIdentity(s.account.Account(), aclSt, recordVerifier)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
consRecs := make([]*consensusproto.RawRecordWithId, 0, len(res.AclRecords))
|
||||
for _, rec := range res.AclRecords {
|
||||
consRecs = append(consRecs, &consensusproto.RawRecordWithId{
|
||||
Id: rec.Id,
|
||||
Payload: rec.AclPayload,
|
||||
})
|
||||
}
|
||||
err = acl.AddRawRecords(consRecs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
return st, nil
|
||||
}
|
||||
|
||||
func (s *spaceService) createSpaceStorage(ctx context.Context, payload spacestorage.SpaceStorageCreatePayload) (spacestorage.SpaceStorage, error) {
|
||||
|
|
|
@ -103,6 +103,12 @@ message SpacePullRequest {
|
|||
// SpacePullResponse is a response with header and acl root
|
||||
message SpacePullResponse {
|
||||
SpacePayload payload = 1;
|
||||
repeated AclRecord aclRecords = 2;
|
||||
}
|
||||
|
||||
message AclRecord {
|
||||
bytes aclPayload = 1;
|
||||
string id = 2;
|
||||
}
|
||||
|
||||
// SpacePayload is a payload for pushing a space
|
||||
|
|
|
@ -749,7 +749,8 @@ func (m *SpacePullRequest) GetId() string {
|
|||
|
||||
// SpacePullResponse is a response with header and acl root
|
||||
type SpacePullResponse struct {
|
||||
Payload *SpacePayload `protobuf:"bytes,1,opt,name=payload,proto3" json:"payload,omitempty"`
|
||||
Payload *SpacePayload `protobuf:"bytes,1,opt,name=payload,proto3" json:"payload,omitempty"`
|
||||
AclRecords []*AclRecord `protobuf:"bytes,2,rep,name=aclRecords,proto3" json:"aclRecords,omitempty"`
|
||||
}
|
||||
|
||||
func (m *SpacePullResponse) Reset() { *m = SpacePullResponse{} }
|
||||
|
@ -800,6 +801,73 @@ func (m *SpacePullResponse) GetPayload() *SpacePayload {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (m *SpacePullResponse) GetAclRecords() []*AclRecord {
|
||||
if m != nil {
|
||||
return m.AclRecords
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type AclRecord struct {
|
||||
AclPayload []byte `protobuf:"bytes,1,opt,name=aclPayload,proto3" json:"aclPayload,omitempty"`
|
||||
Id string `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"`
|
||||
}
|
||||
|
||||
func (m *AclRecord) Reset() { *m = AclRecord{} }
|
||||
func (m *AclRecord) String() string { return proto.CompactTextString(m) }
|
||||
func (*AclRecord) ProtoMessage() {}
|
||||
func (*AclRecord) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{10}
|
||||
}
|
||||
func (m *AclRecord) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
}
|
||||
func (m *AclRecord) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
if deterministic {
|
||||
return xxx_messageInfo_AclRecord.Marshal(b, m, deterministic)
|
||||
} else {
|
||||
b = b[:cap(b)]
|
||||
n, err := m.MarshalToSizedBuffer(b)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return b[:n], nil
|
||||
}
|
||||
}
|
||||
func (m *AclRecord) XXX_MarshalAppend(b []byte, newLen int) ([]byte, error) {
|
||||
b = b[:newLen]
|
||||
_, err := m.MarshalToSizedBuffer(b)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return b, nil
|
||||
}
|
||||
func (m *AclRecord) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_AclRecord.Merge(m, src)
|
||||
}
|
||||
func (m *AclRecord) XXX_Size() int {
|
||||
return m.Size()
|
||||
}
|
||||
func (m *AclRecord) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_AclRecord.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_AclRecord proto.InternalMessageInfo
|
||||
|
||||
func (m *AclRecord) GetAclPayload() []byte {
|
||||
if m != nil {
|
||||
return m.AclPayload
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *AclRecord) GetId() string {
|
||||
if m != nil {
|
||||
return m.Id
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
// SpacePayload is a payload for pushing a space
|
||||
type SpacePayload struct {
|
||||
SpaceHeader *RawSpaceHeaderWithId `protobuf:"bytes,1,opt,name=spaceHeader,proto3" json:"spaceHeader,omitempty"`
|
||||
|
@ -813,7 +881,7 @@ func (m *SpacePayload) Reset() { *m = SpacePayload{} }
|
|||
func (m *SpacePayload) String() string { return proto.CompactTextString(m) }
|
||||
func (*SpacePayload) ProtoMessage() {}
|
||||
func (*SpacePayload) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{10}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{11}
|
||||
}
|
||||
func (m *SpacePayload) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -899,7 +967,7 @@ func (m *SpaceHeader) Reset() { *m = SpaceHeader{} }
|
|||
func (m *SpaceHeader) String() string { return proto.CompactTextString(m) }
|
||||
func (*SpaceHeader) ProtoMessage() {}
|
||||
func (*SpaceHeader) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{11}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{12}
|
||||
}
|
||||
func (m *SpaceHeader) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -988,7 +1056,7 @@ func (m *RawSpaceHeader) Reset() { *m = RawSpaceHeader{} }
|
|||
func (m *RawSpaceHeader) String() string { return proto.CompactTextString(m) }
|
||||
func (*RawSpaceHeader) ProtoMessage() {}
|
||||
func (*RawSpaceHeader) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{12}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{13}
|
||||
}
|
||||
func (m *RawSpaceHeader) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1049,7 +1117,7 @@ func (m *RawSpaceHeaderWithId) Reset() { *m = RawSpaceHeaderWithId{} }
|
|||
func (m *RawSpaceHeaderWithId) String() string { return proto.CompactTextString(m) }
|
||||
func (*RawSpaceHeaderWithId) ProtoMessage() {}
|
||||
func (*RawSpaceHeaderWithId) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{13}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{14}
|
||||
}
|
||||
func (m *RawSpaceHeaderWithId) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1113,7 +1181,7 @@ func (m *SpaceSettingsContent) Reset() { *m = SpaceSettingsContent{} }
|
|||
func (m *SpaceSettingsContent) String() string { return proto.CompactTextString(m) }
|
||||
func (*SpaceSettingsContent) ProtoMessage() {}
|
||||
func (*SpaceSettingsContent) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{14}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{15}
|
||||
}
|
||||
func (m *SpaceSettingsContent) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1204,7 +1272,7 @@ func (m *ObjectDelete) Reset() { *m = ObjectDelete{} }
|
|||
func (m *ObjectDelete) String() string { return proto.CompactTextString(m) }
|
||||
func (*ObjectDelete) ProtoMessage() {}
|
||||
func (*ObjectDelete) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{15}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{16}
|
||||
}
|
||||
func (m *ObjectDelete) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1258,7 +1326,7 @@ func (m *StoreHeader) Reset() { *m = StoreHeader{} }
|
|||
func (m *StoreHeader) String() string { return proto.CompactTextString(m) }
|
||||
func (*StoreHeader) ProtoMessage() {}
|
||||
func (*StoreHeader) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{16}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{17}
|
||||
}
|
||||
func (m *StoreHeader) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1318,7 +1386,7 @@ func (m *SpaceDelete) Reset() { *m = SpaceDelete{} }
|
|||
func (m *SpaceDelete) String() string { return proto.CompactTextString(m) }
|
||||
func (*SpaceDelete) ProtoMessage() {}
|
||||
func (*SpaceDelete) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{17}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{18}
|
||||
}
|
||||
func (m *SpaceDelete) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1372,7 +1440,7 @@ func (m *SpaceSettingsSnapshot) Reset() { *m = SpaceSettingsSnapshot{} }
|
|||
func (m *SpaceSettingsSnapshot) String() string { return proto.CompactTextString(m) }
|
||||
func (*SpaceSettingsSnapshot) ProtoMessage() {}
|
||||
func (*SpaceSettingsSnapshot) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{18}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{19}
|
||||
}
|
||||
func (m *SpaceSettingsSnapshot) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1433,7 +1501,7 @@ func (m *SettingsData) Reset() { *m = SettingsData{} }
|
|||
func (m *SettingsData) String() string { return proto.CompactTextString(m) }
|
||||
func (*SettingsData) ProtoMessage() {}
|
||||
func (*SettingsData) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{19}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{20}
|
||||
}
|
||||
func (m *SettingsData) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1493,7 +1561,7 @@ func (m *SpaceSubscription) Reset() { *m = SpaceSubscription{} }
|
|||
func (m *SpaceSubscription) String() string { return proto.CompactTextString(m) }
|
||||
func (*SpaceSubscription) ProtoMessage() {}
|
||||
func (*SpaceSubscription) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{20}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{21}
|
||||
}
|
||||
func (m *SpaceSubscription) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1554,7 +1622,7 @@ func (m *AclAddRecordRequest) Reset() { *m = AclAddRecordRequest{} }
|
|||
func (m *AclAddRecordRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*AclAddRecordRequest) ProtoMessage() {}
|
||||
func (*AclAddRecordRequest) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{21}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{22}
|
||||
}
|
||||
func (m *AclAddRecordRequest) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1615,7 +1683,7 @@ func (m *AclAddRecordResponse) Reset() { *m = AclAddRecordResponse{} }
|
|||
func (m *AclAddRecordResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*AclAddRecordResponse) ProtoMessage() {}
|
||||
func (*AclAddRecordResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{22}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{23}
|
||||
}
|
||||
func (m *AclAddRecordResponse) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1677,7 +1745,7 @@ func (m *AclGetRecordsRequest) Reset() { *m = AclGetRecordsRequest{} }
|
|||
func (m *AclGetRecordsRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*AclGetRecordsRequest) ProtoMessage() {}
|
||||
func (*AclGetRecordsRequest) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{23}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{24}
|
||||
}
|
||||
func (m *AclGetRecordsRequest) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1737,7 +1805,7 @@ func (m *AclGetRecordsResponse) Reset() { *m = AclGetRecordsResponse{} }
|
|||
func (m *AclGetRecordsResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*AclGetRecordsResponse) ProtoMessage() {}
|
||||
func (*AclGetRecordsResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{24}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{25}
|
||||
}
|
||||
func (m *AclGetRecordsResponse) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1790,7 +1858,7 @@ func (m *StoreDiffRequest) Reset() { *m = StoreDiffRequest{} }
|
|||
func (m *StoreDiffRequest) String() string { return proto.CompactTextString(m) }
|
||||
func (*StoreDiffRequest) ProtoMessage() {}
|
||||
func (*StoreDiffRequest) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{25}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{26}
|
||||
}
|
||||
func (m *StoreDiffRequest) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1849,7 +1917,7 @@ func (m *StoreDiffResponse) Reset() { *m = StoreDiffResponse{} }
|
|||
func (m *StoreDiffResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*StoreDiffResponse) ProtoMessage() {}
|
||||
func (*StoreDiffResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{26}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{27}
|
||||
}
|
||||
func (m *StoreDiffResponse) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1905,7 +1973,7 @@ func (m *StoreKeyValue) Reset() { *m = StoreKeyValue{} }
|
|||
func (m *StoreKeyValue) String() string { return proto.CompactTextString(m) }
|
||||
func (*StoreKeyValue) ProtoMessage() {}
|
||||
func (*StoreKeyValue) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{27}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{28}
|
||||
}
|
||||
func (m *StoreKeyValue) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -1985,7 +2053,7 @@ func (m *StoreKeyValues) Reset() { *m = StoreKeyValues{} }
|
|||
func (m *StoreKeyValues) String() string { return proto.CompactTextString(m) }
|
||||
func (*StoreKeyValues) ProtoMessage() {}
|
||||
func (*StoreKeyValues) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{28}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{29}
|
||||
}
|
||||
func (m *StoreKeyValues) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -2042,7 +2110,7 @@ func (m *StoreKeyInner) Reset() { *m = StoreKeyInner{} }
|
|||
func (m *StoreKeyInner) String() string { return proto.CompactTextString(m) }
|
||||
func (*StoreKeyInner) ProtoMessage() {}
|
||||
func (*StoreKeyInner) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{29}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{30}
|
||||
}
|
||||
func (m *StoreKeyInner) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -2130,7 +2198,7 @@ func (m *StorageHeader) Reset() { *m = StorageHeader{} }
|
|||
func (m *StorageHeader) String() string { return proto.CompactTextString(m) }
|
||||
func (*StorageHeader) ProtoMessage() {}
|
||||
func (*StorageHeader) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{30}
|
||||
return fileDescriptor_80e49f1f4ac27799, []int{31}
|
||||
}
|
||||
func (m *StorageHeader) XXX_Unmarshal(b []byte) error {
|
||||
return m.Unmarshal(b)
|
||||
|
@ -2196,6 +2264,7 @@ func init() {
|
|||
proto.RegisterType((*SpacePushResponse)(nil), "spacesync.SpacePushResponse")
|
||||
proto.RegisterType((*SpacePullRequest)(nil), "spacesync.SpacePullRequest")
|
||||
proto.RegisterType((*SpacePullResponse)(nil), "spacesync.SpacePullResponse")
|
||||
proto.RegisterType((*AclRecord)(nil), "spacesync.AclRecord")
|
||||
proto.RegisterType((*SpacePayload)(nil), "spacesync.SpacePayload")
|
||||
proto.RegisterType((*SpaceHeader)(nil), "spacesync.SpaceHeader")
|
||||
proto.RegisterType((*RawSpaceHeader)(nil), "spacesync.RawSpaceHeader")
|
||||
|
@ -2224,106 +2293,108 @@ func init() {
|
|||
}
|
||||
|
||||
var fileDescriptor_80e49f1f4ac27799 = []byte{
|
||||
// 1572 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x18, 0x4d, 0x6f, 0x1b, 0x45,
|
||||
0x3b, 0xbb, 0x4e, 0x1c, 0xfb, 0x89, 0xe3, 0x6e, 0x26, 0x49, 0xe3, 0xd7, 0xad, 0x5c, 0x6b, 0xf4,
|
||||
0xaa, 0x6f, 0x14, 0xbd, 0xb4, 0x4d, 0x0a, 0x95, 0x5a, 0xe0, 0x90, 0x26, 0x69, 0x63, 0x4a, 0x9a,
|
||||
0x68, 0xdc, 0x0f, 0x09, 0x09, 0xa4, 0xcd, 0xee, 0x24, 0x59, 0xba, 0xde, 0x35, 0x3b, 0xe3, 0x36,
|
||||
0x3e, 0x72, 0xe2, 0x04, 0xe2, 0xcc, 0x7f, 0xe0, 0xc0, 0xbf, 0xe0, 0x58, 0x38, 0x71, 0x44, 0xed,
|
||||
0x9d, 0xdf, 0x80, 0x66, 0x76, 0x76, 0x76, 0xd6, 0x1f, 0xa5, 0xa8, 0x70, 0x89, 0xe7, 0x79, 0xe6,
|
||||
0xf9, 0xfe, 0x9c, 0x0d, 0x6c, 0x7a, 0x71, 0xaf, 0x17, 0x47, 0xac, 0xef, 0x7a, 0xf4, 0xba, 0xfc,
|
||||
0xcb, 0x86, 0x91, 0xd7, 0x4f, 0x62, 0x1e, 0x5f, 0x97, 0x7f, 0x59, 0x8e, 0xbd, 0x26, 0x11, 0xa8,
|
||||
0xaa, 0x11, 0x98, 0xc2, 0xe2, 0x3e, 0x75, 0xfd, 0xee, 0x30, 0xf2, 0x88, 0x1b, 0x9d, 0x52, 0x84,
|
||||
0x60, 0xf6, 0x24, 0x89, 0x7b, 0x0d, 0xab, 0x6d, 0xad, 0xcf, 0x12, 0x79, 0x46, 0x75, 0xb0, 0x79,
|
||||
0xdc, 0xb0, 0x25, 0xc6, 0xe6, 0x31, 0x5a, 0x81, 0xb9, 0x30, 0xe8, 0x05, 0xbc, 0x51, 0x6a, 0x5b,
|
||||
0xeb, 0x8b, 0x24, 0x05, 0x50, 0x13, 0x2a, 0x34, 0xa4, 0x3d, 0x1a, 0x71, 0xd6, 0x98, 0x6d, 0x5b,
|
||||
0xeb, 0x15, 0xa2, 0x61, 0x7c, 0x0e, 0x75, 0xad, 0x86, 0xb2, 0x41, 0xc8, 0x85, 0x9e, 0x33, 0x97,
|
||||
0x9d, 0x49, 0x3d, 0x35, 0x22, 0xcf, 0xe8, 0x23, 0x43, 0x82, 0xdd, 0x2e, 0xad, 0x2f, 0x6c, 0xb5,
|
||||
0xaf, 0xe5, 0xb6, 0x17, 0x05, 0xec, 0xa5, 0x84, 0xb9, 0x0e, 0x61, 0x95, 0x17, 0x0f, 0x22, 0x6d,
|
||||
0x95, 0x04, 0xf0, 0x87, 0xb0, 0x3a, 0x91, 0x51, 0x38, 0x15, 0xf8, 0x52, 0x7d, 0x95, 0xd8, 0x81,
|
||||
0x2f, 0x0d, 0xa2, 0xae, 0x2f, 0xdd, 0xac, 0x12, 0x79, 0xc6, 0xdf, 0x59, 0x70, 0x21, 0xe7, 0xfe,
|
||||
0x6a, 0x40, 0x19, 0x47, 0x0d, 0x98, 0x97, 0x36, 0x75, 0x32, 0xe6, 0x0c, 0x44, 0x37, 0xa0, 0x9c,
|
||||
0x88, 0x18, 0x66, 0xc6, 0x37, 0x26, 0x19, 0x2f, 0x08, 0x88, 0xa2, 0x43, 0xd7, 0xa1, 0xe2, 0x07,
|
||||
0x27, 0x27, 0x8f, 0x86, 0x7d, 0x2a, 0xad, 0xae, 0x6f, 0x2d, 0x1b, 0x3c, 0xbb, 0xea, 0x8a, 0x68,
|
||||
0x22, 0x7c, 0x0e, 0x8e, 0xe1, 0x4d, 0x3f, 0x8e, 0x18, 0x45, 0x37, 0x61, 0x3e, 0x91, 0x9e, 0xb1,
|
||||
0x86, 0x25, 0xf5, 0xfe, 0x67, 0x6a, 0xd0, 0x48, 0x46, 0x59, 0xd0, 0x6c, 0xbf, 0x8d, 0xe6, 0x5f,
|
||||
0x2d, 0x58, 0x3a, 0x3c, 0xfe, 0x92, 0x7a, 0x5c, 0x88, 0x3b, 0xa0, 0x8c, 0xb9, 0xa7, 0xf4, 0x0d,
|
||||
0xc1, 0xb8, 0x0c, 0xd5, 0x24, 0x8d, 0x58, 0x27, 0x8b, 0x69, 0x8e, 0x10, 0x7c, 0x09, 0xed, 0x87,
|
||||
0xc3, 0x8e, 0x2f, 0xfd, 0xae, 0x92, 0x0c, 0x14, 0x37, 0x7d, 0x77, 0x18, 0xc6, 0xae, 0x2f, 0x8b,
|
||||
0xa8, 0x46, 0x32, 0x50, 0xd4, 0x57, 0x2c, 0x0d, 0xe8, 0xf8, 0x8d, 0x39, 0xc9, 0xa4, 0x61, 0xf4,
|
||||
0x01, 0x40, 0x7a, 0x96, 0x0e, 0x95, 0xa5, 0x43, 0xab, 0x86, 0x43, 0x87, 0xfa, 0x92, 0x18, 0x84,
|
||||
0x98, 0x82, 0xd3, 0x15, 0x34, 0x47, 0x03, 0x76, 0x96, 0xe5, 0x77, 0x33, 0x37, 0x40, 0xb8, 0xb4,
|
||||
0xb0, 0xb5, 0x66, 0xc8, 0x49, 0xa9, 0xd3, 0xeb, 0xdc, 0xb2, 0x16, 0xc0, 0x4e, 0x42, 0x7d, 0x1a,
|
||||
0xf1, 0xc0, 0x0d, 0xa5, 0xb3, 0x35, 0x62, 0x60, 0xf0, 0x32, 0x2c, 0x19, 0x6a, 0xd2, 0xb4, 0x61,
|
||||
0xac, 0x75, 0x87, 0x61, 0xa6, 0x7b, 0xa4, 0x26, 0xf1, 0x3d, 0xcd, 0x28, 0x68, 0x54, 0xbe, 0xff,
|
||||
0xbe, 0x81, 0xf8, 0x6b, 0x1b, 0x6a, 0xe6, 0x0d, 0xda, 0x86, 0x05, 0xc9, 0x23, 0xca, 0x83, 0x26,
|
||||
0x4a, 0xce, 0x15, 0x43, 0x0e, 0x71, 0x5f, 0x74, 0x73, 0x82, 0xa7, 0x01, 0x3f, 0xeb, 0xf8, 0xc4,
|
||||
0xe4, 0x11, 0x4e, 0xbb, 0x5e, 0xa8, 0x04, 0x66, 0x4e, 0xe7, 0x18, 0x84, 0xa1, 0x96, 0x43, 0x3a,
|
||||
0xcf, 0x05, 0x1c, 0xda, 0x82, 0x15, 0x29, 0xb2, 0x4b, 0x39, 0x0f, 0xa2, 0x53, 0x76, 0x54, 0xc8,
|
||||
0xfc, 0xc4, 0x3b, 0x74, 0x0b, 0x2e, 0x4e, 0xc2, 0xeb, 0xa2, 0x98, 0x72, 0x8b, 0x7f, 0xb1, 0x60,
|
||||
0xc1, 0x70, 0x49, 0x94, 0x53, 0x20, 0x13, 0xc4, 0x87, 0x6a, 0x08, 0x69, 0x58, 0x14, 0x2f, 0x0f,
|
||||
0x7a, 0x94, 0x71, 0xb7, 0xd7, 0x97, 0xae, 0x95, 0x48, 0x8e, 0x10, 0xb7, 0x52, 0x87, 0x6e, 0xdb,
|
||||
0x2a, 0xc9, 0x11, 0xe8, 0x2a, 0xd4, 0x45, 0x2d, 0x07, 0x9e, 0xcb, 0x83, 0x38, 0x7a, 0x40, 0x87,
|
||||
0xd2, 0x9b, 0x59, 0x32, 0x82, 0x15, 0xf3, 0x86, 0x51, 0x9a, 0x5a, 0x5d, 0x23, 0xf2, 0x8c, 0xae,
|
||||
0x01, 0x32, 0x42, 0x9c, 0x45, 0xa3, 0x2c, 0x29, 0x26, 0xdc, 0xe0, 0x23, 0xa8, 0x17, 0x13, 0x85,
|
||||
0xda, 0xe3, 0x89, 0xad, 0x15, 0xf3, 0x26, 0xac, 0x0f, 0x4e, 0x23, 0x97, 0x0f, 0x12, 0xaa, 0xd2,
|
||||
0x96, 0x23, 0xf0, 0x2e, 0xac, 0x4c, 0x4a, 0xbd, 0x6c, 0x67, 0xf7, 0x45, 0x41, 0x6a, 0x8e, 0x50,
|
||||
0x75, 0x6b, 0xeb, 0xba, 0xfd, 0xc1, 0x82, 0x95, 0xae, 0x99, 0x86, 0x9d, 0x38, 0xe2, 0x62, 0xe8,
|
||||
0x7e, 0x0c, 0xb5, 0xb4, 0xfd, 0x76, 0x69, 0x48, 0x39, 0x9d, 0x50, 0xc0, 0x87, 0xc6, 0xf5, 0xfe,
|
||||
0x0c, 0x29, 0x90, 0xa3, 0x3b, 0xca, 0x3b, 0xc5, 0x6d, 0x4b, 0xee, 0x8b, 0xa3, 0xe5, 0xaf, 0x99,
|
||||
0x4d, 0xe2, 0xbb, 0xf3, 0x30, 0xf7, 0xdc, 0x0d, 0x07, 0x14, 0xb7, 0xa0, 0x66, 0x2a, 0x19, 0x6b,
|
||||
0xba, 0x0e, 0x2c, 0x74, 0x79, 0x9c, 0x64, 0xf1, 0x9a, 0x3e, 0xe2, 0x44, 0xac, 0x79, 0x9c, 0xb8,
|
||||
0xa7, 0xf4, 0xa1, 0xdb, 0xa3, 0xca, 0x7d, 0x13, 0x85, 0x6f, 0xaa, 0x92, 0x53, 0x9a, 0xfe, 0x0b,
|
||||
0x8b, 0xbe, 0x3c, 0x25, 0x47, 0x94, 0x26, 0x5a, 0x60, 0x11, 0x89, 0x3f, 0x87, 0xd5, 0x42, 0xec,
|
||||
0xba, 0x91, 0xdb, 0x67, 0x67, 0x31, 0x17, 0x1d, 0x97, 0x52, 0xfa, 0x1d, 0x3f, 0x9d, 0xf5, 0x55,
|
||||
0x62, 0x60, 0xc6, 0xc5, 0xdb, 0x93, 0xc4, 0x7f, 0x63, 0x41, 0x2d, 0x13, 0xbd, 0xeb, 0x72, 0x17,
|
||||
0xdd, 0x86, 0x79, 0x2f, 0x4d, 0x8f, 0xda, 0x1f, 0x57, 0x46, 0x03, 0x3a, 0x92, 0x45, 0x92, 0xd1,
|
||||
0x8b, 0x85, 0xcd, 0x94, 0x75, 0x2a, 0x19, 0xed, 0x69, 0xbc, 0x99, 0x17, 0x44, 0x73, 0xe0, 0x67,
|
||||
0x6a, 0xba, 0x75, 0x07, 0xc7, 0xcc, 0x4b, 0x82, 0xbe, 0xe8, 0x0c, 0xd1, 0x96, 0x2a, 0xbe, 0x99,
|
||||
0x8b, 0x1a, 0x46, 0x77, 0xa0, 0xec, 0x7a, 0x82, 0x4a, 0xad, 0x2c, 0x3c, 0xa6, 0xcc, 0x90, 0xb4,
|
||||
0x2d, 0x29, 0x89, 0xe2, 0xc0, 0x1d, 0x58, 0xde, 0xf6, 0xc2, 0x6d, 0xdf, 0x27, 0xd4, 0x8b, 0x13,
|
||||
0xff, 0xaf, 0xb7, 0xb9, 0xb1, 0x88, 0xec, 0xc2, 0x22, 0xc2, 0x9f, 0xc2, 0x4a, 0x51, 0x94, 0x1a,
|
||||
0xcc, 0x4d, 0xa8, 0x24, 0x12, 0xa3, 0x85, 0x69, 0xf8, 0x0d, 0xd2, 0x3e, 0x91, 0xd2, 0xee, 0x53,
|
||||
0x9e, 0x4a, 0x63, 0x6f, 0x65, 0x99, 0xeb, 0x85, 0xfb, 0xf9, 0x63, 0x25, 0x03, 0xf1, 0x26, 0xac,
|
||||
0x8e, 0xc8, 0x52, 0xa6, 0xc9, 0x7d, 0x2b, 0x51, 0x32, 0xa8, 0x35, 0x92, 0x81, 0xf8, 0x0b, 0x70,
|
||||
0x64, 0xb5, 0x8b, 0x95, 0xff, 0x2f, 0x3c, 0x71, 0xf0, 0x3e, 0x2c, 0x19, 0xf2, 0xdf, 0xe1, 0xc9,
|
||||
0x82, 0x7f, 0xb2, 0x60, 0x51, 0x8a, 0x7a, 0x40, 0x87, 0x4f, 0x44, 0x27, 0x8b, 0xa1, 0xf4, 0x8c,
|
||||
0x0e, 0x0b, 0xbd, 0x94, 0x23, 0xc4, 0x7b, 0x50, 0x36, 0xbc, 0x0a, 0x78, 0x0a, 0xa0, 0xff, 0xc3,
|
||||
0x52, 0x36, 0xe6, 0xbb, 0x7a, 0x0c, 0x96, 0x24, 0xc5, 0xf8, 0x85, 0x68, 0xa9, 0x3e, 0xa5, 0x49,
|
||||
0x4e, 0x99, 0x6e, 0xa6, 0x22, 0xd2, 0x8c, 0xd7, 0x5c, 0x21, 0x5e, 0x78, 0x1f, 0xea, 0x05, 0x93,
|
||||
0x19, 0xba, 0x25, 0x6d, 0x4e, 0x01, 0xe5, 0xbc, 0x19, 0xc4, 0x02, 0x35, 0xc9, 0x49, 0xf1, 0x8f,
|
||||
0x86, 0xf7, 0x9d, 0x28, 0xa2, 0x89, 0x58, 0x20, 0xc2, 0x8c, 0xec, 0x05, 0x2d, 0xce, 0x85, 0xa5,
|
||||
0x66, 0x8f, 0x2c, 0x35, 0x1d, 0x8f, 0x92, 0x19, 0x8f, 0xab, 0x50, 0xd7, 0x9b, 0xed, 0x20, 0xf0,
|
||||
0x92, 0x58, 0xba, 0x58, 0x22, 0x23, 0x58, 0x11, 0x6b, 0x55, 0x65, 0xda, 0xcb, 0x1c, 0x81, 0x1c,
|
||||
0x28, 0x3d, 0xa3, 0x43, 0xb9, 0xa9, 0xaa, 0x44, 0x1c, 0xf1, 0x83, 0xd4, 0x5c, 0xf7, 0xf4, 0x1f,
|
||||
0x98, 0xa3, 0x1b, 0x7f, 0x58, 0x50, 0xd9, 0x4b, 0x92, 0x9d, 0xd8, 0xa7, 0x0c, 0xd5, 0x01, 0x1e,
|
||||
0x47, 0xf4, 0xbc, 0x4f, 0x3d, 0x4e, 0x7d, 0x67, 0x06, 0x39, 0xea, 0x6d, 0x73, 0x10, 0x30, 0x16,
|
||||
0x44, 0xa7, 0x8e, 0x85, 0x2e, 0xa8, 0xb1, 0xbb, 0x77, 0x1e, 0x30, 0xce, 0x1c, 0x1b, 0x2d, 0xc3,
|
||||
0x05, 0x89, 0x78, 0x18, 0xf3, 0x4e, 0xb4, 0xe3, 0x7a, 0x67, 0xd4, 0x29, 0x21, 0x04, 0x75, 0x89,
|
||||
0xec, 0xb0, 0x74, 0x3c, 0xfb, 0xce, 0x2c, 0x6a, 0xc0, 0x8a, 0xac, 0x1e, 0xf6, 0x30, 0xe6, 0xaa,
|
||||
0x5a, 0x83, 0xe3, 0x90, 0x3a, 0x73, 0x68, 0x05, 0x1c, 0x42, 0x3d, 0x1a, 0xf4, 0x79, 0x87, 0x75,
|
||||
0xa2, 0xe7, 0x6e, 0x18, 0xf8, 0x4e, 0x59, 0xc8, 0x50, 0x80, 0x5a, 0xc9, 0xce, 0xbc, 0xa0, 0xdc,
|
||||
0x1d, 0xa4, 0xab, 0x9e, 0xaa, 0x8e, 0x72, 0x2a, 0xe8, 0x12, 0xac, 0x3d, 0x8a, 0xe3, 0x03, 0x37,
|
||||
0x1a, 0x2a, 0x1c, 0xbb, 0x97, 0xc4, 0x3d, 0xa1, 0xcc, 0xa9, 0x0a, 0x83, 0xf7, 0x92, 0x24, 0x4e,
|
||||
0x0e, 0x4f, 0x4e, 0x18, 0xe5, 0x8e, 0xbf, 0x71, 0x1b, 0xd6, 0xa6, 0x0c, 0x34, 0xb4, 0x08, 0x55,
|
||||
0x85, 0x3d, 0xa6, 0xce, 0x8c, 0x60, 0x7d, 0x1c, 0x31, 0x8d, 0xb0, 0x36, 0xfe, 0x07, 0x95, 0xec,
|
||||
0xf9, 0x8e, 0x16, 0x60, 0xbe, 0x13, 0x05, 0xe2, 0x0d, 0xea, 0xcc, 0xa0, 0x32, 0xd8, 0x4f, 0x36,
|
||||
0x1d, 0x4b, 0xfe, 0x6e, 0x39, 0xf6, 0xc6, 0x7b, 0x00, 0xf9, 0xb3, 0x18, 0x55, 0x60, 0xf6, 0x51,
|
||||
0x42, 0x85, 0xc4, 0x79, 0x28, 0x6d, 0x7b, 0xa1, 0x63, 0xa1, 0x1a, 0x54, 0xb2, 0x4a, 0x74, 0xec,
|
||||
0xad, 0x6f, 0xcb, 0x50, 0x4d, 0x6d, 0x1a, 0x46, 0x1e, 0xda, 0x81, 0x4a, 0xd6, 0xa7, 0xa8, 0x39,
|
||||
0xb1, 0x79, 0xa5, 0x93, 0xcd, 0x4b, 0x93, 0x1b, 0x3b, 0x1d, 0x03, 0xf7, 0xa0, 0xaa, 0x67, 0x03,
|
||||
0xba, 0x34, 0xda, 0x05, 0xc6, 0x44, 0x6a, 0x5e, 0x9e, 0x7c, 0xa9, 0xe4, 0xdc, 0x57, 0xad, 0xb1,
|
||||
0x97, 0x7d, 0x0a, 0x4e, 0xed, 0xa8, 0xe6, 0xd4, 0x9b, 0x75, 0xeb, 0x86, 0x25, 0x0d, 0xca, 0x1e,
|
||||
0xea, 0x45, 0x83, 0x46, 0xbe, 0x12, 0x8a, 0x06, 0x8d, 0xbe, 0xed, 0x0d, 0x39, 0x61, 0x38, 0x49,
|
||||
0x8e, 0x7e, 0xf1, 0x4f, 0x92, 0x63, 0x3c, 0xf5, 0x09, 0x38, 0xf9, 0x37, 0x57, 0x97, 0x27, 0xd4,
|
||||
0xed, 0xa1, 0xcb, 0x63, 0x8f, 0x25, 0xe3, 0x83, 0xac, 0xf9, 0xc6, 0x5b, 0xe9, 0xe3, 0x7e, 0x96,
|
||||
0x76, 0x99, 0xbb, 0x77, 0x90, 0x86, 0x9e, 0xc2, 0x5a, 0x8e, 0x54, 0x0e, 0xbd, 0xbb, 0x91, 0x37,
|
||||
0x2c, 0x74, 0x08, 0x35, 0x73, 0xc1, 0xa2, 0x96, 0x41, 0x3f, 0x61, 0x89, 0x37, 0xaf, 0x4c, 0xbd,
|
||||
0xd7, 0x71, 0x5c, 0x2c, 0xec, 0x45, 0x34, 0xc2, 0x31, 0xb6, 0x7d, 0x9b, 0xed, 0xe9, 0x04, 0xa9,
|
||||
0xcc, 0xbb, 0xef, 0xff, 0xfc, 0xaa, 0x65, 0xbd, 0x7c, 0xd5, 0xb2, 0x7e, 0x7f, 0xd5, 0xb2, 0xbe,
|
||||
0x7f, 0xdd, 0x9a, 0x79, 0xf9, 0xba, 0x35, 0xf3, 0xdb, 0xeb, 0xd6, 0xcc, 0x67, 0xcd, 0xe9, 0xff,
|
||||
0x91, 0x39, 0x2e, 0xcb, 0x9f, 0x9b, 0x7f, 0x06, 0x00, 0x00, 0xff, 0xff, 0xdc, 0x8f, 0x50, 0x36,
|
||||
0xb6, 0x11, 0x00, 0x00,
|
||||
// 1609 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0xcf, 0x6f, 0x1b, 0xc5,
|
||||
0x17, 0xcf, 0xae, 0x13, 0xc7, 0x7e, 0x71, 0xdc, 0xcd, 0xc4, 0x69, 0xfc, 0x75, 0x2b, 0xd7, 0x5a,
|
||||
0x7d, 0x55, 0xa2, 0x08, 0xda, 0x26, 0x2d, 0x95, 0xda, 0xc2, 0x21, 0x4d, 0xd2, 0xc6, 0x94, 0x34,
|
||||
0xd1, 0xb8, 0x3f, 0x24, 0x24, 0x90, 0x36, 0xbb, 0x93, 0x64, 0xe9, 0x7a, 0xd7, 0xec, 0x8c, 0xdb,
|
||||
0x58, 0xe2, 0xc2, 0x89, 0x13, 0x88, 0x33, 0xff, 0x03, 0x07, 0xfe, 0x0b, 0x8e, 0x85, 0x13, 0x47,
|
||||
0xd4, 0xde, 0xf9, 0x1b, 0xd0, 0xcc, 0xce, 0xce, 0xce, 0xae, 0xed, 0x52, 0x51, 0xb8, 0xc4, 0x3b,
|
||||
0x6f, 0xde, 0x8f, 0xcf, 0x7b, 0xf3, 0x7e, 0xcc, 0x04, 0x36, 0xdc, 0xa8, 0xdf, 0x8f, 0x42, 0x3a,
|
||||
0x70, 0x5c, 0x72, 0x55, 0xfc, 0xa5, 0xa3, 0xd0, 0x1d, 0xc4, 0x11, 0x8b, 0xae, 0x8a, 0xbf, 0x34,
|
||||
0xa3, 0x5e, 0x11, 0x04, 0x54, 0x55, 0x04, 0x9b, 0xc0, 0xe2, 0x1e, 0x71, 0xbc, 0xde, 0x28, 0x74,
|
||||
0xb1, 0x13, 0x9e, 0x10, 0x84, 0x60, 0xf6, 0x38, 0x8e, 0xfa, 0x4d, 0xa3, 0x63, 0xac, 0xcd, 0x62,
|
||||
0xf1, 0x8d, 0xea, 0x60, 0xb2, 0xa8, 0x69, 0x0a, 0x8a, 0xc9, 0x22, 0xd4, 0x80, 0xb9, 0xc0, 0xef,
|
||||
0xfb, 0xac, 0x59, 0xea, 0x18, 0x6b, 0x8b, 0x38, 0x59, 0xa0, 0x16, 0x54, 0x48, 0x40, 0xfa, 0x24,
|
||||
0x64, 0xb4, 0x39, 0xdb, 0x31, 0xd6, 0x2a, 0x58, 0xad, 0xed, 0x33, 0xa8, 0x2b, 0x33, 0x84, 0x0e,
|
||||
0x03, 0xc6, 0xed, 0x9c, 0x3a, 0xf4, 0x54, 0xd8, 0xa9, 0x61, 0xf1, 0x8d, 0x3e, 0xd2, 0x34, 0x98,
|
||||
0x9d, 0xd2, 0xda, 0xc2, 0x66, 0xe7, 0x4a, 0x86, 0x3d, 0xaf, 0x60, 0x37, 0x61, 0xcc, 0x6c, 0x70,
|
||||
0x54, 0x6e, 0x34, 0x0c, 0x15, 0x2a, 0xb1, 0xb0, 0xef, 0xc0, 0xca, 0x44, 0x41, 0xee, 0x94, 0xef,
|
||||
0x09, 0xf3, 0x55, 0x6c, 0xfa, 0x9e, 0x00, 0x44, 0x1c, 0x4f, 0xb8, 0x59, 0xc5, 0xe2, 0xdb, 0xfe,
|
||||
0xde, 0x80, 0x73, 0x99, 0xf4, 0x57, 0x43, 0x42, 0x19, 0x6a, 0xc2, 0xbc, 0xc0, 0xd4, 0x4d, 0x85,
|
||||
0xd3, 0x25, 0xba, 0x06, 0xe5, 0x98, 0xc7, 0x30, 0x05, 0xdf, 0x9c, 0x04, 0x9e, 0x33, 0x60, 0xc9,
|
||||
0x87, 0xae, 0x42, 0xc5, 0xf3, 0x8f, 0x8f, 0x1f, 0x8d, 0x06, 0x44, 0xa0, 0xae, 0x6f, 0x2e, 0x6b,
|
||||
0x32, 0x3b, 0x72, 0x0b, 0x2b, 0x26, 0xfb, 0x0c, 0x2c, 0xcd, 0x9b, 0x41, 0x14, 0x52, 0x82, 0xae,
|
||||
0xc3, 0x7c, 0x2c, 0x3c, 0xa3, 0x4d, 0x43, 0xd8, 0xfd, 0xdf, 0xd4, 0xa0, 0xe1, 0x94, 0x33, 0x67,
|
||||
0xd9, 0x7c, 0x1b, 0xcb, 0xbf, 0x19, 0xb0, 0x74, 0x70, 0xf4, 0x25, 0x71, 0x19, 0x57, 0xb7, 0x4f,
|
||||
0x28, 0x75, 0x4e, 0xc8, 0x1b, 0x82, 0x71, 0x11, 0xaa, 0x71, 0x12, 0xb1, 0x6e, 0x1a, 0xd3, 0x8c,
|
||||
0xc0, 0xe5, 0x62, 0x32, 0x08, 0x46, 0x5d, 0x4f, 0xf8, 0x5d, 0xc5, 0xe9, 0x92, 0xef, 0x0c, 0x9c,
|
||||
0x51, 0x10, 0x39, 0x9e, 0x48, 0xa2, 0x1a, 0x4e, 0x97, 0x3c, 0xbf, 0x22, 0x01, 0xa0, 0xeb, 0x35,
|
||||
0xe7, 0x84, 0x90, 0x5a, 0xa3, 0x0f, 0x01, 0x92, 0x6f, 0xe1, 0x50, 0x59, 0x38, 0xb4, 0xa2, 0x39,
|
||||
0x74, 0xa0, 0x36, 0xb1, 0xc6, 0x68, 0x13, 0xb0, 0x7a, 0x9c, 0xe7, 0x70, 0x48, 0x4f, 0xd3, 0xf3,
|
||||
0xdd, 0xc8, 0x00, 0x70, 0x97, 0x16, 0x36, 0x57, 0x35, 0x3d, 0x09, 0x77, 0xb2, 0x9d, 0x21, 0x6b,
|
||||
0x03, 0x6c, 0xc7, 0xc4, 0x23, 0x21, 0xf3, 0x9d, 0x40, 0x38, 0x5b, 0xc3, 0x1a, 0xc5, 0x5e, 0x86,
|
||||
0x25, 0xcd, 0x4c, 0x72, 0x6c, 0xb6, 0xad, 0x6c, 0x07, 0x41, 0x6a, 0xbb, 0x90, 0x93, 0xf6, 0xd7,
|
||||
0x4a, 0x90, 0xf3, 0xc8, 0xf3, 0xfe, 0x07, 0x00, 0x6f, 0x00, 0x38, 0x6e, 0x80, 0x89, 0x1b, 0xc5,
|
||||
0x5e, 0x9a, 0x9d, 0x0d, 0x4d, 0x6a, 0x2b, 0xdd, 0xc4, 0x1a, 0x9f, 0x7d, 0x07, 0xaa, 0x6a, 0x83,
|
||||
0xfb, 0xe8, 0xb8, 0xc1, 0xa1, 0x66, 0xb8, 0x86, 0x35, 0x8a, 0x84, 0x6e, 0x2a, 0xe8, 0xdf, 0x98,
|
||||
0x50, 0xd3, 0xc1, 0xa0, 0x2d, 0x58, 0x10, 0x06, 0x79, 0x46, 0x92, 0x58, 0x42, 0xbf, 0xa4, 0x81,
|
||||
0xc0, 0xce, 0x8b, 0x5e, 0xc6, 0xf0, 0xd4, 0x67, 0xa7, 0x5d, 0x0f, 0xeb, 0x32, 0x05, 0x0c, 0xe6,
|
||||
0x18, 0x06, 0x1b, 0x6a, 0xd9, 0x4a, 0xa5, 0x56, 0x8e, 0x86, 0x36, 0xa1, 0x21, 0x54, 0xf6, 0x08,
|
||||
0x63, 0x7e, 0x78, 0x42, 0x0f, 0x73, 0xc9, 0x36, 0x71, 0x0f, 0xdd, 0x84, 0xf3, 0x93, 0xe8, 0x2a,
|
||||
0x0f, 0xa7, 0xec, 0xda, 0xbf, 0x1a, 0xb0, 0xa0, 0xb9, 0xc4, 0x33, 0xd8, 0x17, 0x39, 0xc1, 0x46,
|
||||
0x32, 0x82, 0x6a, 0xcd, 0xeb, 0x85, 0xf9, 0x7d, 0x42, 0x99, 0xd3, 0x1f, 0x08, 0xd7, 0x4a, 0x38,
|
||||
0x23, 0xf0, 0x5d, 0x61, 0x43, 0x75, 0x8a, 0x2a, 0xce, 0x08, 0xe8, 0x32, 0xd4, 0x79, 0xf9, 0xf8,
|
||||
0xae, 0xc3, 0xfc, 0x28, 0x7c, 0x40, 0x46, 0xc2, 0x9b, 0x59, 0x5c, 0xa0, 0xf2, 0x16, 0x47, 0x09,
|
||||
0x49, 0x50, 0xd7, 0xb0, 0xf8, 0x46, 0x57, 0x00, 0x69, 0x21, 0x4e, 0xa3, 0x51, 0x16, 0x1c, 0x13,
|
||||
0x76, 0xec, 0x43, 0xa8, 0xe7, 0x0f, 0x0a, 0x75, 0xc6, 0x0f, 0xb6, 0x96, 0x3f, 0x37, 0x8e, 0xde,
|
||||
0x3f, 0x09, 0x1d, 0x36, 0x8c, 0x89, 0x3c, 0xb6, 0x8c, 0x60, 0xef, 0x40, 0x63, 0xd2, 0xd1, 0x8b,
|
||||
0x0e, 0xe2, 0xbc, 0xc8, 0x69, 0xcd, 0x08, 0x63, 0xf9, 0xf6, 0xa3, 0x01, 0x8d, 0x9e, 0x7e, 0x0c,
|
||||
0xdb, 0x51, 0xc8, 0x78, 0x9f, 0xff, 0x18, 0x6a, 0x49, 0xc5, 0xef, 0x90, 0x80, 0x30, 0x32, 0xa1,
|
||||
0x66, 0x0e, 0xb4, 0xed, 0xbd, 0x19, 0x9c, 0x63, 0x47, 0xb7, 0xa5, 0x77, 0x52, 0xda, 0x14, 0xd2,
|
||||
0xe7, 0x8b, 0x15, 0xa7, 0x84, 0x75, 0xe6, 0xbb, 0xf3, 0x30, 0xf7, 0xdc, 0x09, 0x86, 0xc4, 0x6e,
|
||||
0x43, 0x4d, 0x37, 0x32, 0x56, 0xe7, 0x5d, 0x58, 0xe8, 0xb1, 0x28, 0x4e, 0xe3, 0x35, 0xbd, 0xab,
|
||||
0xf2, 0x58, 0xb3, 0x28, 0x76, 0x4e, 0xc8, 0x43, 0xa7, 0x4f, 0xa4, 0xfb, 0x3a, 0xc9, 0xbe, 0x2e,
|
||||
0x53, 0x4e, 0x5a, 0xfa, 0x3f, 0x2c, 0x7a, 0xe2, 0x2b, 0x3e, 0x24, 0x24, 0x56, 0x0a, 0xf3, 0x44,
|
||||
0xfb, 0x73, 0x58, 0xc9, 0xc5, 0xae, 0x17, 0x3a, 0x03, 0x7a, 0x1a, 0x31, 0x5e, 0x71, 0x09, 0xa7,
|
||||
0xd7, 0xf5, 0x92, 0xf1, 0x52, 0xc5, 0x1a, 0x65, 0x5c, 0xbd, 0x39, 0x49, 0xfd, 0xb7, 0x06, 0xd4,
|
||||
0x52, 0xd5, 0x3b, 0x0e, 0x73, 0xd0, 0x2d, 0x98, 0x77, 0x93, 0xe3, 0x91, 0x23, 0xeb, 0x52, 0x31,
|
||||
0xa0, 0x85, 0x53, 0xc4, 0x29, 0x3f, 0xbf, 0x23, 0x50, 0x89, 0x4e, 0x1e, 0x46, 0x67, 0x9a, 0x6c,
|
||||
0xea, 0x05, 0x56, 0x12, 0xf6, 0x33, 0xd9, 0x50, 0x7b, 0xc3, 0x23, 0xea, 0xc6, 0xfe, 0x80, 0x57,
|
||||
0x06, 0x2f, 0x4b, 0x19, 0xdf, 0xd4, 0x45, 0xb5, 0x46, 0xb7, 0xa1, 0xec, 0xb8, 0x9c, 0x4b, 0x4e,
|
||||
0x49, 0x7b, 0xcc, 0x98, 0xa6, 0x69, 0x4b, 0x70, 0x62, 0x29, 0x61, 0x77, 0x61, 0x79, 0xcb, 0x0d,
|
||||
0xb6, 0x3c, 0x4f, 0xf6, 0xd6, 0xbf, 0xbd, 0x40, 0x68, 0xb3, 0xcf, 0xcc, 0xcd, 0x3e, 0xfb, 0x53,
|
||||
0x68, 0xe4, 0x55, 0xc9, 0x59, 0xd0, 0x82, 0x4a, 0x2c, 0x28, 0x4a, 0x99, 0x5a, 0xbf, 0x41, 0xdb,
|
||||
0x27, 0x42, 0xdb, 0x7d, 0xc2, 0x64, 0xa7, 0x7f, 0x2b, 0x64, 0x8e, 0x1b, 0xec, 0x65, 0xf7, 0xa3,
|
||||
0x74, 0x69, 0x6f, 0xc0, 0x4a, 0x41, 0x97, 0x84, 0x26, 0x46, 0x7c, 0x32, 0x70, 0x78, 0x50, 0x6b,
|
||||
0x38, 0x5d, 0xda, 0x5f, 0x80, 0x25, 0xb2, 0x9d, 0xdf, 0x32, 0xfe, 0x83, 0x5b, 0x95, 0xbd, 0x07,
|
||||
0x4b, 0x9a, 0xfe, 0x77, 0xb8, 0x25, 0xd9, 0x3f, 0x1b, 0xb0, 0x28, 0x54, 0x3d, 0x20, 0xa3, 0x27,
|
||||
0xbc, 0x92, 0x79, 0x53, 0x7a, 0x46, 0x46, 0xb9, 0x5a, 0xca, 0x08, 0xfc, 0x0a, 0x2a, 0x0a, 0x5e,
|
||||
0x06, 0x3c, 0x59, 0xa0, 0xf7, 0x61, 0x29, 0x6d, 0xf3, 0x3d, 0xd5, 0x06, 0x4b, 0x82, 0x63, 0x7c,
|
||||
0x83, 0x97, 0xd4, 0x80, 0x90, 0x38, 0xe3, 0x4c, 0x26, 0x53, 0x9e, 0xa8, 0xc7, 0x6b, 0x2e, 0x17,
|
||||
0x2f, 0x7b, 0x0f, 0xea, 0x39, 0xc8, 0x14, 0xdd, 0x14, 0x98, 0x93, 0x85, 0x74, 0x5e, 0x0f, 0x62,
|
||||
0x8e, 0x1b, 0x67, 0xac, 0xf6, 0x4f, 0x9a, 0xf7, 0xdd, 0x30, 0x24, 0x31, 0x1f, 0x20, 0x1c, 0x46,
|
||||
0x7a, 0x69, 0xe7, 0xdf, 0xb9, 0xa1, 0x66, 0x16, 0x86, 0x9a, 0x8a, 0x47, 0x49, 0x8f, 0xc7, 0x65,
|
||||
0xa8, 0xab, 0xc9, 0xb6, 0xef, 0xbb, 0x71, 0x24, 0x5c, 0x2c, 0xe1, 0x02, 0x95, 0xc7, 0x5a, 0x66,
|
||||
0x99, 0xf2, 0x32, 0x23, 0x20, 0x0b, 0x4a, 0xcf, 0xc8, 0x48, 0x4c, 0xaa, 0x2a, 0xe6, 0x9f, 0xf6,
|
||||
0x83, 0x04, 0xae, 0x73, 0xf2, 0x2f, 0xf4, 0xd1, 0xf5, 0x3f, 0x0d, 0xa8, 0xec, 0xc6, 0xf1, 0x76,
|
||||
0xe4, 0x11, 0x8a, 0xea, 0x00, 0x8f, 0x43, 0x72, 0x36, 0x20, 0x2e, 0x23, 0x9e, 0x35, 0x83, 0x2c,
|
||||
0x79, 0xb7, 0xd9, 0xf7, 0x29, 0xf5, 0xc3, 0x13, 0xcb, 0x40, 0xe7, 0x64, 0xdb, 0xdd, 0x3d, 0xf3,
|
||||
0x29, 0xa3, 0x96, 0x89, 0x96, 0xe1, 0x9c, 0x20, 0x3c, 0x8c, 0x58, 0x37, 0xdc, 0x76, 0xdc, 0x53,
|
||||
0x62, 0x95, 0x10, 0x82, 0xba, 0x20, 0x76, 0x69, 0xd2, 0x9e, 0x3d, 0x6b, 0x16, 0x35, 0xa1, 0x21,
|
||||
0xb2, 0x87, 0x3e, 0x8c, 0x98, 0xcc, 0x56, 0xff, 0x28, 0x20, 0xd6, 0x1c, 0x6a, 0x80, 0x85, 0x89,
|
||||
0x4b, 0xfc, 0x01, 0xeb, 0xd2, 0x6e, 0xf8, 0xdc, 0x09, 0x7c, 0xcf, 0x2a, 0x73, 0x1d, 0x72, 0x21,
|
||||
0x47, 0xb2, 0x35, 0xcf, 0x39, 0x77, 0x86, 0xc9, 0xa8, 0x27, 0xb2, 0xa2, 0xac, 0x0a, 0xba, 0x00,
|
||||
0xab, 0x8f, 0xa2, 0x68, 0xdf, 0x09, 0x47, 0x92, 0x46, 0xef, 0xc5, 0x51, 0x9f, 0x1b, 0xb3, 0xaa,
|
||||
0x1c, 0xf0, 0x6e, 0x1c, 0x47, 0xf1, 0xc1, 0xf1, 0x31, 0x25, 0xcc, 0xf2, 0xd6, 0x6f, 0xc1, 0xea,
|
||||
0x94, 0x86, 0x86, 0x16, 0xa1, 0x2a, 0xa9, 0x47, 0xc4, 0x9a, 0xe1, 0xa2, 0x8f, 0x43, 0xaa, 0x08,
|
||||
0xc6, 0xfa, 0x7b, 0x50, 0x49, 0x5f, 0x0c, 0x68, 0x01, 0xe6, 0xbb, 0xa1, 0xcf, 0xaf, 0xbd, 0xd6,
|
||||
0x0c, 0x2a, 0x83, 0xf9, 0x64, 0xc3, 0x32, 0xc4, 0xef, 0xa6, 0x65, 0xae, 0x7f, 0x00, 0x90, 0xdd,
|
||||
0xc4, 0x51, 0x05, 0x66, 0x1f, 0xc5, 0x84, 0x6b, 0x9c, 0x87, 0xd2, 0x96, 0x1b, 0x58, 0x06, 0xaa,
|
||||
0x41, 0x25, 0xcd, 0x44, 0xcb, 0xdc, 0xfc, 0xae, 0x0c, 0xd5, 0x04, 0xd3, 0x28, 0x74, 0xd1, 0x36,
|
||||
0x54, 0xd2, 0x3a, 0x45, 0xad, 0x89, 0xc5, 0x2b, 0x9c, 0x6c, 0x5d, 0x98, 0x5c, 0xd8, 0x49, 0x1b,
|
||||
0xb8, 0x07, 0x55, 0xd5, 0x1b, 0xd0, 0x85, 0x62, 0x15, 0x68, 0x1d, 0xa9, 0x75, 0x71, 0xf2, 0xa6,
|
||||
0xd4, 0x73, 0x5f, 0x96, 0xc6, 0x6e, 0xfa, 0xfa, 0x9c, 0x5a, 0x51, 0xad, 0xa9, 0x3b, 0x6b, 0xc6,
|
||||
0x35, 0x43, 0x00, 0x4a, 0xdf, 0x06, 0x79, 0x40, 0x85, 0x87, 0x49, 0x1e, 0x50, 0xf1, 0x39, 0xa1,
|
||||
0xe9, 0x09, 0x82, 0x49, 0x7a, 0xd4, 0x23, 0x63, 0x92, 0x1e, 0xed, 0x75, 0x81, 0xc1, 0xca, 0x9e,
|
||||
0x79, 0x3d, 0x16, 0x13, 0xa7, 0x8f, 0x2e, 0x8e, 0x5d, 0x96, 0xb4, 0x37, 0x60, 0xeb, 0x8d, 0xbb,
|
||||
0xc2, 0xc7, 0xbd, 0xf4, 0xd8, 0xc5, 0xd9, 0xbd, 0x83, 0x36, 0xf4, 0x14, 0x56, 0x33, 0xa2, 0x74,
|
||||
0xe8, 0xdd, 0x41, 0x5e, 0x33, 0xd0, 0x01, 0xd4, 0xf4, 0x01, 0x8b, 0xda, 0xf9, 0xd7, 0x51, 0x71,
|
||||
0x88, 0xb7, 0x2e, 0x4d, 0xdd, 0x57, 0x71, 0x5c, 0xcc, 0xcd, 0x45, 0x54, 0x90, 0x18, 0x9b, 0xbe,
|
||||
0xad, 0xce, 0x74, 0x86, 0x44, 0xe7, 0xdd, 0x1b, 0xbf, 0xbc, 0x6a, 0x1b, 0x2f, 0x5f, 0xb5, 0x8d,
|
||||
0x3f, 0x5e, 0xb5, 0x8d, 0x1f, 0x5e, 0xb7, 0x67, 0x5e, 0xbe, 0x6e, 0xcf, 0xfc, 0xfe, 0xba, 0x3d,
|
||||
0xf3, 0x59, 0x6b, 0xfa, 0x3f, 0x81, 0x8e, 0xca, 0xe2, 0xe7, 0xfa, 0x5f, 0x01, 0x00, 0x00, 0xff,
|
||||
0xff, 0x9d, 0x2c, 0x6b, 0x14, 0x29, 0x12, 0x00, 0x00,
|
||||
}
|
||||
|
||||
func (m *HeadSyncRange) Marshal() (dAtA []byte, err error) {
|
||||
|
@ -2729,6 +2800,20 @@ func (m *SpacePullResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
|
|||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if len(m.AclRecords) > 0 {
|
||||
for iNdEx := len(m.AclRecords) - 1; iNdEx >= 0; iNdEx-- {
|
||||
{
|
||||
size, err := m.AclRecords[iNdEx].MarshalToSizedBuffer(dAtA[:i])
|
||||
if err != nil {
|
||||
return 0, err
|
||||
}
|
||||
i -= size
|
||||
i = encodeVarintSpacesync(dAtA, i, uint64(size))
|
||||
}
|
||||
i--
|
||||
dAtA[i] = 0x12
|
||||
}
|
||||
}
|
||||
if m.Payload != nil {
|
||||
{
|
||||
size, err := m.Payload.MarshalToSizedBuffer(dAtA[:i])
|
||||
|
@ -2744,6 +2829,43 @@ func (m *SpacePullResponse) MarshalToSizedBuffer(dAtA []byte) (int, error) {
|
|||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func (m *AclRecord) Marshal() (dAtA []byte, err error) {
|
||||
size := m.Size()
|
||||
dAtA = make([]byte, size)
|
||||
n, err := m.MarshalToSizedBuffer(dAtA[:size])
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return dAtA[:n], nil
|
||||
}
|
||||
|
||||
func (m *AclRecord) MarshalTo(dAtA []byte) (int, error) {
|
||||
size := m.Size()
|
||||
return m.MarshalToSizedBuffer(dAtA[:size])
|
||||
}
|
||||
|
||||
func (m *AclRecord) MarshalToSizedBuffer(dAtA []byte) (int, error) {
|
||||
i := len(dAtA)
|
||||
_ = i
|
||||
var l int
|
||||
_ = l
|
||||
if len(m.Id) > 0 {
|
||||
i -= len(m.Id)
|
||||
copy(dAtA[i:], m.Id)
|
||||
i = encodeVarintSpacesync(dAtA, i, uint64(len(m.Id)))
|
||||
i--
|
||||
dAtA[i] = 0x12
|
||||
}
|
||||
if len(m.AclPayload) > 0 {
|
||||
i -= len(m.AclPayload)
|
||||
copy(dAtA[i:], m.AclPayload)
|
||||
i = encodeVarintSpacesync(dAtA, i, uint64(len(m.AclPayload)))
|
||||
i--
|
||||
dAtA[i] = 0xa
|
||||
}
|
||||
return len(dAtA) - i, nil
|
||||
}
|
||||
|
||||
func (m *SpacePayload) Marshal() (dAtA []byte, err error) {
|
||||
size := m.Size()
|
||||
dAtA = make([]byte, size)
|
||||
|
@ -3849,6 +3971,29 @@ func (m *SpacePullResponse) Size() (n int) {
|
|||
l = m.Payload.Size()
|
||||
n += 1 + l + sovSpacesync(uint64(l))
|
||||
}
|
||||
if len(m.AclRecords) > 0 {
|
||||
for _, e := range m.AclRecords {
|
||||
l = e.Size()
|
||||
n += 1 + l + sovSpacesync(uint64(l))
|
||||
}
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
func (m *AclRecord) Size() (n int) {
|
||||
if m == nil {
|
||||
return 0
|
||||
}
|
||||
var l int
|
||||
_ = l
|
||||
l = len(m.AclPayload)
|
||||
if l > 0 {
|
||||
n += 1 + l + sovSpacesync(uint64(l))
|
||||
}
|
||||
l = len(m.Id)
|
||||
if l > 0 {
|
||||
n += 1 + l + sovSpacesync(uint64(l))
|
||||
}
|
||||
return n
|
||||
}
|
||||
|
||||
|
@ -5444,6 +5589,156 @@ func (m *SpacePullResponse) Unmarshal(dAtA []byte) error {
|
|||
return err
|
||||
}
|
||||
iNdEx = postIndex
|
||||
case 2:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field AclRecords", wireType)
|
||||
}
|
||||
var msglen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowSpacesync
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
msglen |= int(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
if msglen < 0 {
|
||||
return ErrInvalidLengthSpacesync
|
||||
}
|
||||
postIndex := iNdEx + msglen
|
||||
if postIndex < 0 {
|
||||
return ErrInvalidLengthSpacesync
|
||||
}
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.AclRecords = append(m.AclRecords, &AclRecord{})
|
||||
if err := m.AclRecords[len(m.AclRecords)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil {
|
||||
return err
|
||||
}
|
||||
iNdEx = postIndex
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skipSpacesync(dAtA[iNdEx:])
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if (skippy < 0) || (iNdEx+skippy) < 0 {
|
||||
return ErrInvalidLengthSpacesync
|
||||
}
|
||||
if (iNdEx + skippy) > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
iNdEx += skippy
|
||||
}
|
||||
}
|
||||
|
||||
if iNdEx > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
return nil
|
||||
}
|
||||
func (m *AclRecord) Unmarshal(dAtA []byte) error {
|
||||
l := len(dAtA)
|
||||
iNdEx := 0
|
||||
for iNdEx < l {
|
||||
preIndex := iNdEx
|
||||
var wire uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowSpacesync
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
wire |= uint64(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
fieldNum := int32(wire >> 3)
|
||||
wireType := int(wire & 0x7)
|
||||
if wireType == 4 {
|
||||
return fmt.Errorf("proto: AclRecord: wiretype end group for non-group")
|
||||
}
|
||||
if fieldNum <= 0 {
|
||||
return fmt.Errorf("proto: AclRecord: illegal tag %d (wire type %d)", fieldNum, wire)
|
||||
}
|
||||
switch fieldNum {
|
||||
case 1:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field AclPayload", wireType)
|
||||
}
|
||||
var byteLen int
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowSpacesync
|
||||
}
|
||||
if iNdEx >= l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
b := dAtA[iNdEx]
|
||||
iNdEx++
|
||||
byteLen |= int(b&0x7F) << shift
|
||||
if b < 0x80 {
|
||||
break
|
||||
}
|
||||
}
|
||||
if byteLen < 0 {
|
||||
return ErrInvalidLengthSpacesync
|
||||
}
|
||||
postIndex := iNdEx + byteLen
|
||||
if postIndex < 0 {
|
||||
return ErrInvalidLengthSpacesync
|
||||
}
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.AclPayload = append(m.AclPayload[:0], dAtA[iNdEx:postIndex]...)
|
||||
if m.AclPayload == nil {
|
||||
m.AclPayload = []byte{}
|
||||
}
|
||||
iNdEx = postIndex
|
||||
case 2:
|
||||
if wireType != 2 {
|
||||
return fmt.Errorf("proto: wrong wireType = %d for field Id", wireType)
|
||||
}
|
||||
var stringLen uint64
|
||||
for shift := uint(0); ; shift += 7 {
|
||||
if shift >= 64 {
|
||||
return ErrIntOverflowSpacesync
|
||||
}
|
||||
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 ErrInvalidLengthSpacesync
|
||||
}
|
||||
postIndex := iNdEx + intStringLen
|
||||
if postIndex < 0 {
|
||||
return ErrInvalidLengthSpacesync
|
||||
}
|
||||
if postIndex > l {
|
||||
return io.ErrUnexpectedEOF
|
||||
}
|
||||
m.Id = string(dAtA[iNdEx:postIndex])
|
||||
iNdEx = postIndex
|
||||
default:
|
||||
iNdEx = preIndex
|
||||
skippy, err := skipSpacesync(dAtA[iNdEx:])
|
||||
|
|
|
@ -71,6 +71,7 @@ func (m mockNodeClient) AclAddRecord(ctx context.Context, spaceId string, rec *c
|
|||
}
|
||||
|
||||
type mockPeerManager struct {
|
||||
peer peer.Peer
|
||||
}
|
||||
|
||||
func (p *mockPeerManager) BroadcastMessage(ctx context.Context, msg drpc.Message) error {
|
||||
|
@ -90,6 +91,9 @@ func (p *mockPeerManager) Name() (name string) {
|
|||
}
|
||||
|
||||
func (p *mockPeerManager) GetResponsiblePeers(ctx context.Context) (peers []peer.Peer, err error) {
|
||||
if p.peer != nil {
|
||||
return []peer.Peer{p.peer}, nil
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
|
@ -115,6 +119,7 @@ func (m *testPeerManagerProvider) NewPeerManager(ctx context.Context, spaceId st
|
|||
}
|
||||
|
||||
type mockPeerManagerProvider struct {
|
||||
peer peer.Peer
|
||||
}
|
||||
|
||||
func (m *mockPeerManagerProvider) Init(a *app.App) (err error) {
|
||||
|
@ -126,7 +131,7 @@ func (m *mockPeerManagerProvider) Name() (name string) {
|
|||
}
|
||||
|
||||
func (m *mockPeerManagerProvider) NewPeerManager(ctx context.Context, spaceId string) (sm peermanager.PeerManager, err error) {
|
||||
return &mockPeerManager{}, nil
|
||||
return &mockPeerManager{m.peer}, nil
|
||||
}
|
||||
|
||||
type mockPool struct {
|
||||
|
@ -491,7 +496,10 @@ func (s *streamOpener) NewReadMessage() drpc.Message {
|
|||
}
|
||||
|
||||
func (s *streamOpener) Init(a *app.App) (err error) {
|
||||
s.spaceGetter = a.MustComponent(RpcName).(*RpcServer)
|
||||
sp := a.Component(RpcName)
|
||||
if sp != nil {
|
||||
s.spaceGetter = sp.(*RpcServer)
|
||||
}
|
||||
s.streamPool = a.MustComponent(streampool.CName).(streampool.StreamPool)
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -43,7 +43,7 @@ func (s *spaceStorageProvider) WaitSpaceStorage(ctx context.Context, id string)
|
|||
}
|
||||
dbPath := path.Join(s.rootPath, id)
|
||||
if _, err := os.Stat(dbPath); err != nil {
|
||||
return nil, err
|
||||
return nil, spacestorage.ErrSpaceStorageMissing
|
||||
}
|
||||
db, err := anystore.Open(ctx, dbPath, nil)
|
||||
if err != nil {
|
||||
|
|
|
@ -69,7 +69,7 @@ func (m *StubConf) NodeIds(spaceId string) []string {
|
|||
}
|
||||
|
||||
func (m *StubConf) IsResponsible(spaceId string) bool {
|
||||
return true
|
||||
return false
|
||||
}
|
||||
|
||||
func (m *StubConf) FilePeers() []string {
|
||||
|
|
Loading…
Add table
Add a link
Reference in a new issue