Squashed commit of the following:

commit 4827425146
Author: chrislu <chris.lu@gmail.com>
Date:   Sat Sep 16 15:05:38 2023 -0700

    balancer works

commit 3b50139f68
Author: chrislu <chris.lu@gmail.com>
Date:   Fri Sep 15 22:22:32 2023 -0700

    comments

commit 7f685ce7ba
Author: chrislu <chris.lu@gmail.com>
Date:   Fri Sep 15 22:20:05 2023 -0700

    adjust APIs

commit 436d99443b
Author: chrislu <chris.lu@gmail.com>
Date:   Thu Sep 14 23:49:05 2023 -0700

    receive broker stats

commit b771fefa37
Merge: 0a851ec00 890881037
Author: chrislu <chris.lu@gmail.com>
Date:   Wed Sep 13 00:03:47 2023 -0700

    Merge branch 'master' into sub

commit 0a851ec00b
Author: chrislu <chris.lu@gmail.com>
Date:   Sun Sep 10 22:01:25 2023 -0700

    Create balancer.go

commit 39941edc0b
Author: chrislu <chris.lu@gmail.com>
Date:   Thu Sep 7 23:55:19 2023 -0700

    add publisher shutdown

commit 875f562779
Author: chrislu <chris.lu@gmail.com>
Date:   Wed Sep 6 23:16:41 2023 -0700

    server side send response at least once per second

commit 984b6c54cf
Author: chrislu <chris.lu@gmail.com>
Date:   Wed Sep 6 23:15:29 2023 -0700

    ack interval 128

commit 2492a45499
Author: chrislu <chris.lu@gmail.com>
Date:   Wed Sep 6 22:39:46 2023 -0700

    ack interval

commit ba67e6ca29
Author: chrislu <chris.lu@gmail.com>
Date:   Mon Sep 4 21:43:50 2023 -0700

    api for sub

commit 9e4f985698
Author: chrislu <chris.lu@gmail.com>
Date:   Mon Sep 4 21:43:30 2023 -0700

    publish, benchmark

commit cb470d44df
Author: chrislu <chris.lu@gmail.com>
Date:   Fri Sep 1 00:36:51 2023 -0700

    can pub and sub

commit 1eb2da46d5
Author: chrislu <chris.lu@gmail.com>
Date:   Mon Aug 28 09:02:12 2023 -0700

    connect and publish

commit 504ae8383a
Author: chrislu <chris.lu@gmail.com>
Date:   Mon Aug 28 09:01:25 2023 -0700

    protoc version

commit dbcba75271
Author: chrislu <chris.lu@gmail.com>
Date:   Sun Aug 27 18:59:04 2023 -0700

    rename to lookup

commit c9caf33119
Author: chrislu <chris.lu@gmail.com>
Date:   Sun Aug 27 18:33:46 2023 -0700

    move functions

commit 4d6c18d86f
Author: chrislu <chris.lu@gmail.com>
Date:   Sun Aug 27 17:50:59 2023 -0700

    pub sub initial tests

commit 4eb8e8624d
Author: chrislu <chris.lu@gmail.com>
Date:   Sun Aug 27 13:14:39 2023 -0700

    rename

commit 1990456670
Author: chrislu <chris.lu@gmail.com>
Date:   Sun Aug 27 13:13:14 2023 -0700

    sub

commit 905911853d
Author: chrislu <chris.lu@gmail.com>
Date:   Sat Aug 26 13:39:21 2023 -0700

    adjust proto
This commit is contained in:
chrislu 2023-09-16 15:06:16 -07:00
parent 8908810376
commit 89a1fd1751
36 changed files with 2874 additions and 804 deletions

1
.gitignore vendored
View File

@ -87,3 +87,4 @@ other/java/hdfs/dependency-reduced-pom.xml
# binary file
weed/weed
weed/mq/client/cmd/weed_pub/weed_pub

1
go.mod
View File

@ -266,6 +266,7 @@ require (
github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect
github.com/putdotio/go-putio/putio v0.0.0-20200123120452-16d982cac2b8 // indirect
github.com/rclone/ftp v0.0.0-20230327202000-dadc1f64e87d // indirect
github.com/rdleal/intervalst v0.0.0-20221028215511-a098aa0d2cb8 // indirect
github.com/rfjakob/eme v1.1.2 // indirect
github.com/rivo/uniseg v0.4.4 // indirect
github.com/shirou/gopsutil/v3 v3.23.5 // indirect

2
go.sum
View File

@ -752,6 +752,8 @@ github.com/rclone/rclone v1.63.1 h1:iITCUNBfAXnguHjRPFq+w/gGIW0L0las78h4H5CH2Ms=
github.com/rclone/rclone v1.63.1/go.mod h1:eUQaKsf1wJfHKB0RDoM8RaPAeRB2eI/Qw+Vc9Ho5FGM=
github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5XpJzTSTfLsJV/mx9Q9g7kxmchpfZyxgzM=
github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
github.com/rdleal/intervalst v0.0.0-20221028215511-a098aa0d2cb8 h1:5jSBlCYQYquRF8Zch4QrimJcX7/H1qbWQEAzYNxMubc=
github.com/rdleal/intervalst v0.0.0-20221028215511-a098aa0d2cb8/go.mod h1:xO89Z6BC+LQDH+IPQQw/OESt5UADgFD41tYMUINGpxQ=
github.com/redis/go-redis/v9 v9.0.2 h1:BA426Zqe/7r56kCcvxYLWe1mkaz71LKF77GwgFzSxfE=
github.com/redis/go-redis/v9 v9.0.2/go.mod h1:/xDTe9EF1LM61hek62Poq2nzQSGj0xSrEtEHbBQevps=
github.com/rekby/fixenv v0.3.2/go.mod h1:/b5LRc06BYJtslRtHKxsPWFT/ySpHV+rWvzTg+XWk4c=

View File

@ -69,10 +69,13 @@ service SeaweedFiler {
rpc CacheRemoteObjectToLocalCluster (CacheRemoteObjectToLocalClusterRequest) returns (CacheRemoteObjectToLocalClusterResponse) {
}
rpc Lock(LockRequest) returns (LockResponse) {
rpc DistributedLock(LockRequest) returns (LockResponse) {
}
rpc Unlock(UnlockRequest) returns (UnlockResponse) {
rpc DistributedUnlock(UnlockRequest) returns (UnlockResponse) {
}
rpc FindLockOwner(FindLockOwnerRequest) returns (FindLockOwnerResponse) {
}
// distributed lock management internal use only
rpc TransferLocks(TransferLocksRequest) returns (TransferLocksResponse) {
}
}
@ -444,6 +447,7 @@ message LockRequest {
int64 seconds_to_lock = 2;
string renew_token = 3;
bool is_moved = 4;
string owner = 5;
}
message LockResponse {
string renew_token = 1;
@ -459,10 +463,18 @@ message UnlockResponse {
string error = 1;
string moved_to = 2;
}
message FindLockOwnerRequest {
string name = 1;
bool is_moved = 2;
}
message FindLockOwnerResponse {
string owner = 1;
}
message Lock {
string name = 1;
string renew_token = 2;
int64 expired_at_ns = 3;
string owner = 4;
}
message TransferLocksRequest {
repeated Lock locks = 1;

View File

@ -37,6 +37,10 @@ debug_s3:
go build -gcflags="all=-N -l"
dlv --listen=:2345 --headless=true --api-version=2 --accept-multiclient exec ./weed -- -v=4 s3
debug_mq:
go build -gcflags="all=-N -l"
dlv --listen=:2345 --headless=true --api-version=2 --accept-multiclient exec ./weed -- -v=4 mq.broker
debug_filer_copy:
go build -gcflags="all=-N -l"
dlv --listen=:2345 --headless=true --api-version=2 --accept-multiclient exec ./weed -- -v=4 filer.backup -filer=localhost:8888 -filerProxy -timeAgo=10h

View File

@ -16,13 +16,15 @@ type LockClient struct {
grpcDialOption grpc.DialOption
maxLockDuration time.Duration
sleepDuration time.Duration
seedFiler pb.ServerAddress
}
func NewLockClient(grpcDialOption grpc.DialOption) *LockClient {
func NewLockClient(grpcDialOption grpc.DialOption, seedFiler pb.ServerAddress) *LockClient {
return &LockClient{
grpcDialOption: grpcDialOption,
maxLockDuration: 5 * time.Second,
sleepDuration: 4 * time.Millisecond,
sleepDuration: 2473 * time.Millisecond,
seedFiler: seedFiler,
}
}
@ -34,25 +36,62 @@ type LiveLock struct {
cancelCh chan struct{}
grpcDialOption grpc.DialOption
isLocked bool
}
// NewLockWithTimeout locks the key with the given duration
func (lc *LockClient) NewLockWithTimeout(filer pb.ServerAddress, key string, lockDuration time.Duration) (lock *LiveLock) {
return lc.doNewLock(filer, key, lockDuration)
owner string
lc *LockClient
}
// NewLock creates a lock with a very long duration
func (lc *LockClient) NewLock(filer pb.ServerAddress, key string) (lock *LiveLock) {
return lc.doNewLock(filer, key, lock_manager.MaxDuration)
func (lc *LockClient) NewLock(key string, owner string) (lock *LiveLock) {
return lc.doNewLock(key, lock_manager.MaxDuration, owner)
}
func (lc *LockClient) doNewLock(filer pb.ServerAddress, key string, lockDuration time.Duration) (lock *LiveLock) {
// StartLock starts a goroutine to lock the key and returns immediately.
func (lc *LockClient) StartLock(key string, owner string) (lock *LiveLock) {
lock = &LiveLock{
key: key,
filer: filer,
filer: lc.seedFiler,
cancelCh: make(chan struct{}),
expireAtNs: time.Now().Add(lock_manager.MaxDuration).UnixNano(),
grpcDialOption: lc.grpcDialOption,
owner: owner,
lc: lc,
}
go func() {
util.RetryForever("create lock:"+key, func() error {
errorMessage, err := lock.doLock(lock_manager.MaxDuration)
if err != nil {
glog.Infof("create lock %s: %s", key, err)
time.Sleep(time.Second)
return err
}
if errorMessage != "" {
glog.Infof("create lock %s: %s", key, errorMessage)
time.Sleep(time.Second)
return fmt.Errorf("%v", errorMessage)
}
lock.isLocked = true
return nil
}, func(err error) (shouldContinue bool) {
if err != nil {
glog.Warningf("create lock %s: %s", key, err)
time.Sleep(time.Second)
}
return lock.renewToken == ""
})
lc.keepLock(lock)
}()
return
}
func (lc *LockClient) doNewLock(key string, lockDuration time.Duration, owner string) (lock *LiveLock) {
lock = &LiveLock{
key: key,
filer: lc.seedFiler,
cancelCh: make(chan struct{}),
expireAtNs: time.Now().Add(lockDuration).UnixNano(),
grpcDialOption: lc.grpcDialOption,
owner: owner,
lc: lc,
}
var needRenewal bool
if lockDuration > lc.maxLockDuration {
@ -62,11 +101,14 @@ func (lc *LockClient) doNewLock(filer pb.ServerAddress, key string, lockDuration
util.RetryForever("create lock:"+key, func() error {
errorMessage, err := lock.doLock(lockDuration)
if err != nil {
time.Sleep(time.Second)
return err
}
if errorMessage != "" {
time.Sleep(time.Second)
return fmt.Errorf("%v", errorMessage)
}
lock.isLocked = true
return nil
}, func(err error) (shouldContinue bool) {
if err != nil {
@ -75,8 +117,6 @@ func (lc *LockClient) doNewLock(filer pb.ServerAddress, key string, lockDuration
return lock.renewToken == ""
})
lock.isLocked = true
if needRenewal {
go lc.keepLock(lock)
}
@ -88,10 +128,13 @@ func (lock *LiveLock) IsLocked() bool {
return lock.isLocked
}
func (lock *LiveLock) Unlock() error {
func (lock *LiveLock) StopLock() error {
close(lock.cancelCh)
if !lock.isLocked {
return nil
}
return pb.WithFilerClient(false, 0, lock.filer, lock.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
_, err := client.Unlock(context.Background(), &filer_pb.UnlockRequest{
_, err := client.DistributedUnlock(context.Background(), &filer_pb.UnlockRequest{
Name: lock.key,
RenewToken: lock.renewToken,
})
@ -100,9 +143,10 @@ func (lock *LiveLock) Unlock() error {
}
func (lc *LockClient) keepLock(lock *LiveLock) {
ticker := time.Tick(lc.sleepDuration)
for {
select {
case <-time.After(lc.sleepDuration):
case <-ticker:
// renew the lock if lock.expireAtNs is still greater than now
util.RetryForever("keep lock:"+lock.key, func() error {
lockDuration := time.Duration(lock.expireAtNs-time.Now().UnixNano()) * time.Nanosecond
@ -116,10 +160,12 @@ func (lc *LockClient) keepLock(lock *LiveLock) {
errorMessage, err := lock.doLock(lockDuration)
if err != nil {
lock.isLocked = false
time.Sleep(time.Second)
return err
}
if errorMessage != "" {
lock.isLocked = false
time.Sleep(time.Second)
return fmt.Errorf("%v", errorMessage)
}
return nil
@ -141,11 +187,12 @@ func (lc *LockClient) keepLock(lock *LiveLock) {
func (lock *LiveLock) doLock(lockDuration time.Duration) (errorMessage string, err error) {
err = pb.WithFilerClient(false, 0, lock.filer, lock.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
resp, err := client.Lock(context.Background(), &filer_pb.LockRequest{
resp, err := client.DistributedLock(context.Background(), &filer_pb.LockRequest{
Name: lock.key,
SecondsToLock: int64(lockDuration.Seconds()),
RenewToken: lock.renewToken,
IsMoved: false,
Owner: lock.owner,
})
if err == nil {
lock.renewToken = resp.RenewToken
@ -154,6 +201,7 @@ func (lock *LiveLock) doLock(lockDuration time.Duration) (errorMessage string, e
errorMessage = resp.Error
if resp.MovedTo != "" {
lock.filer = pb.ServerAddress(resp.MovedTo)
lock.lc.seedFiler = lock.filer
}
}
return err

View File

@ -24,23 +24,38 @@ func NewDistributedLockManager(host pb.ServerAddress) *DistributedLockManager {
}
}
func (dlm *DistributedLockManager) Lock(key string, token string) (renewToken string, movedTo pb.ServerAddress, err error) {
return dlm.LockWithTimeout(key, MaxDuration, token)
func (dlm *DistributedLockManager) LockWithTimeout(key string, expiredAtNs int64, token string, owner string) (renewToken string, movedTo pb.ServerAddress, err error) {
movedTo, err = dlm.findLockOwningFiler(key)
if err != nil {
return
}
if movedTo != dlm.Host {
return
}
renewToken, err = dlm.lockManager.Lock(key, expiredAtNs, token, owner)
return
}
func (dlm *DistributedLockManager) LockWithTimeout(key string, expiredAtNs int64, token string) (renewToken string, movedTo pb.ServerAddress, err error) {
func (dlm *DistributedLockManager) findLockOwningFiler(key string) (movedTo pb.ServerAddress, err error) {
servers := dlm.LockRing.GetSnapshot()
if servers == nil {
err = NoLockServerError
return
}
server := hashKeyToServer(key, servers)
if server != dlm.Host {
movedTo = server
movedTo = hashKeyToServer(key, servers)
return
}
func (dlm *DistributedLockManager) FindLockOwner(key string) (owner string, movedTo pb.ServerAddress, err error) {
movedTo, err = dlm.findLockOwningFiler(key)
if err != nil {
return
}
renewToken, err = dlm.lockManager.Lock(key, expiredAtNs, token)
if movedTo != dlm.Host {
return
}
owner, err = dlm.lockManager.GetLockOwner(key)
return
}
@ -62,8 +77,8 @@ func (dlm *DistributedLockManager) Unlock(key string, token string) (movedTo pb.
// InsertLock is used to insert a lock to a server unconditionally
// It is used when a server is down and the lock is moved to another server
func (dlm *DistributedLockManager) InsertLock(key string, expiredAtNs int64, token string) {
dlm.lockManager.InsertLock(key, expiredAtNs, token)
func (dlm *DistributedLockManager) InsertLock(key string, expiredAtNs int64, token string, owner string) {
dlm.lockManager.InsertLock(key, expiredAtNs, token, owner)
}
func (dlm *DistributedLockManager) SelectNotOwnedLocks(servers []pb.ServerAddress) (locks []*Lock) {
return dlm.lockManager.SelectLocks(func(key string) bool {

View File

@ -12,7 +12,7 @@ var LockErrorNonEmptyTokenOnExpiredLock = fmt.Errorf("lock: non-empty token on a
var LockErrorTokenMismatch = fmt.Errorf("lock: token mismatch")
var UnlockErrorTokenMismatch = fmt.Errorf("unlock: token mismatch")
// LockManager lock manager
// LockManager local lock manager, used by distributed lock manager
type LockManager struct {
locks *xsync.MapOf[string, *Lock]
}
@ -20,6 +20,7 @@ type Lock struct {
Token string
ExpiredAtNs int64
Key string // only used for moving locks
Owner string
}
func NewLockManager() *LockManager {
@ -30,7 +31,7 @@ func NewLockManager() *LockManager {
return t
}
func (lm *LockManager) Lock(path string, expiredAtNs int64, token string) (renewToken string, err error) {
func (lm *LockManager) Lock(path string, expiredAtNs int64, token string, owner string) (renewToken string, err error) {
lm.locks.Compute(path, func(oldValue *Lock, loaded bool) (newValue *Lock, delete bool) {
if oldValue != nil {
if oldValue.ExpiredAtNs > 0 && oldValue.ExpiredAtNs < time.Now().UnixNano() {
@ -41,14 +42,14 @@ func (lm *LockManager) Lock(path string, expiredAtNs int64, token string) (renew
} else {
// new lock
renewToken = uuid.New().String()
return &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs}, false
return &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}, false
}
}
// not expired
if oldValue.Token == token {
// token matches, renew the lock
renewToken = uuid.New().String()
return &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs}, false
return &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}, false
} else {
err = LockErrorTokenMismatch
return oldValue, false
@ -57,7 +58,7 @@ func (lm *LockManager) Lock(path string, expiredAtNs int64, token string) (renew
if token == "" {
// new lock
renewToken = uuid.New().String()
return &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs}, false
return &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}, false
} else {
err = LockErrorNonEmptyTokenOnNewLock
return nil, false
@ -132,6 +133,18 @@ func (lm *LockManager) SelectLocks(selectFn func(key string) bool) (locks []*Loc
}
// InsertLock inserts a lock unconditionally
func (lm *LockManager) InsertLock(path string, expiredAtNs int64, token string) {
lm.locks.Store(path, &Lock{Token: token, ExpiredAtNs: expiredAtNs})
func (lm *LockManager) InsertLock(path string, expiredAtNs int64, token string, owner string) {
lm.locks.Store(path, &Lock{Token: token, ExpiredAtNs: expiredAtNs, Owner: owner})
}
func (lm *LockManager) GetLockOwner(key string) (owner string, err error) {
lm.locks.Range(func(k string, lock *Lock) bool {
if k == key {
owner = lock.Owner
return false
}
return true
})
return
}

View File

@ -0,0 +1,24 @@
package balancer
import (
cmap "github.com/orcaman/concurrent-map/v2"
)
type Balancer struct {
Brokers cmap.ConcurrentMap[string, *BrokerStats]
}
type BrokerStats struct {
TopicPartitionCount int32
ConsumerCount int32
CpuUsagePercent int32
}
func NewBalancer() *Balancer {
return &Balancer{
Brokers: cmap.New[*BrokerStats](),
}
}
func NewBrokerStats() *BrokerStats {
return &BrokerStats{}
}

View File

@ -88,69 +88,38 @@ func (broker *MessageQueueBroker) CheckBrokerLoad(c context.Context, request *mq
return ret, nil
}
// FindTopicBrokers returns the brokers that are serving the topic
//
// 1. lock the topic
//
// 2. find the topic partitions on the filer
// 2.1 if the topic is not found, return error
// 2.2 if the request is_for_publish, create the topic
// 2.2.1 if the request is_for_subscribe, return error not found
// 2.2.2 if the request is_for_publish, create the topic
// 2.2 if the topic is found, return the brokers
//
// 3. unlock the topic
func (broker *MessageQueueBroker) FindTopicBrokers(c context.Context, request *mq_pb.FindTopicBrokersRequest) (*mq_pb.FindTopicBrokersResponse, error) {
ret := &mq_pb.FindTopicBrokersResponse{}
// lock the topic
// find the topic partitions on the filer
// if the topic is not found
// if the request is_for_publish
// create the topic
// if the request is_for_subscribe
// return error not found
return ret, nil
}
// CheckTopicPartitionsStatus check the topic partitions on the broker
func (broker *MessageQueueBroker) CheckTopicPartitionsStatus(c context.Context, request *mq_pb.CheckTopicPartitionsStatusRequest) (*mq_pb.CheckTopicPartitionsStatusResponse, error) {
ret := &mq_pb.CheckTopicPartitionsStatusResponse{}
return ret, nil
}
// createOrUpdateTopicPartitions creates the topic partitions on the broker
// 1. check
func (broker *MessageQueueBroker) createOrUpdateTopicPartitions(topic *topic.Topic, prevAssignment *mq_pb.TopicPartitionsAssignment) (err error) {
func (broker *MessageQueueBroker) createOrUpdateTopicPartitions(topic *topic.Topic, prevAssignments []*mq_pb.BrokerPartitionAssignment) (err error) {
// create or update each partition
if prevAssignment == nil {
if prevAssignments == nil {
broker.createOrUpdateTopicPartition(topic, nil)
} else {
for _, partitionAssignment := range prevAssignment.BrokerPartitions {
broker.createOrUpdateTopicPartition(topic, partitionAssignment)
for _, brokerPartitionAssignment := range prevAssignments {
broker.createOrUpdateTopicPartition(topic, brokerPartitionAssignment)
}
}
return nil
}
func (broker *MessageQueueBroker) createOrUpdateTopicPartition(topic *topic.Topic, oldAssignment *mq_pb.BrokerPartitionsAssignment) (newAssignment *mq_pb.BrokerPartitionsAssignment) {
func (broker *MessageQueueBroker) createOrUpdateTopicPartition(topic *topic.Topic, oldAssignment *mq_pb.BrokerPartitionAssignment) (newAssignment *mq_pb.BrokerPartitionAssignment) {
shouldCreate := broker.confirmBrokerPartitionAssignment(topic, oldAssignment)
if !shouldCreate {
}
return
}
func (broker *MessageQueueBroker) confirmBrokerPartitionAssignment(topic *topic.Topic, oldAssignment *mq_pb.BrokerPartitionsAssignment) (shouldCreate bool) {
func (broker *MessageQueueBroker) confirmBrokerPartitionAssignment(topic *topic.Topic, oldAssignment *mq_pb.BrokerPartitionAssignment) (shouldCreate bool) {
if oldAssignment == nil {
return true
}
for _, b := range oldAssignment.FollowerBrokers {
pb.WithBrokerClient(false, pb.ServerAddress(b), broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
pb.WithBrokerGrpcClient(false, b, broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.CheckTopicPartitionsStatus(context.Background(), &mq_pb.CheckTopicPartitionsStatusRequest{
Namespace: string(topic.Namespace),
Topic: topic.Name,
BrokerPartitionsAssignment: oldAssignment,
ShouldCancelIfNotMatch: true,
Namespace: string(topic.Namespace),
Topic: topic.Name,
BrokerPartitionAssignment: oldAssignment,
ShouldCancelIfNotMatch: true,
})
if err != nil {
shouldCreate = true

View File

@ -0,0 +1,52 @@
package broker
import (
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/balancer"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
// BrokerConnectToBalancer receives connections from brokers and collects stats
func (broker *MessageQueueBroker) ConnectToBalancer(stream mq_pb.SeaweedMessaging_ConnectToBalancerServer) error {
if !broker.lockAsBalancer.IsLocked() {
return status.Errorf(codes.Unavailable, "not current broker balancer")
}
req, err := stream.Recv()
if err != nil {
return err
}
// process init message
initMessage := req.GetInit()
brokerStats := balancer.NewBrokerStats()
if initMessage != nil {
broker.Balancer.Brokers.Set(initMessage.Broker, brokerStats)
} else {
return status.Errorf(codes.InvalidArgument, "balancer init message is empty")
}
defer func() {
broker.Balancer.Brokers.Remove(initMessage.Broker)
}()
// process stats message
for {
req, err := stream.Recv()
if err != nil {
return err
}
if !broker.lockAsBalancer.IsLocked() {
return status.Errorf(codes.Unavailable, "not current broker balancer")
}
if receivedStats := req.GetStats(); receivedStats != nil {
brokerStats.TopicPartitionCount = receivedStats.TopicPartitionCount
brokerStats.ConsumerCount = receivedStats.ConsumerCount
brokerStats.CpuUsagePercent = receivedStats.CpuUsagePercent
glog.V(3).Infof("broker %s stats: %+v", initMessage.Broker, brokerStats)
}
}
return nil
}

View File

@ -0,0 +1,50 @@
package broker
import (
"context"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
)
// FindTopicBrokers returns the brokers that are serving the topic
//
// 1. lock the topic
//
// 2. find the topic partitions on the filer
// 2.1 if the topic is not found, return error
// 2.2 if the request is_for_publish, create the topic
// 2.2.1 if the request is_for_subscribe, return error not found
// 2.2.2 if the request is_for_publish, create the topic
// 2.2 if the topic is found, return the brokers
//
// 3. unlock the topic
func (broker *MessageQueueBroker) LookupTopicBrokers(ctx context.Context, request *mq_pb.LookupTopicBrokersRequest) (*mq_pb.LookupTopicBrokersResponse, error) {
ret := &mq_pb.LookupTopicBrokersResponse{}
// TODO lock the topic
// find the topic partitions on the filer
// if the topic is not found
// if the request is_for_publish
// create the topic
// if the request is_for_subscribe
// return error not found
// t := topic.FromPbTopic(request.Topic)
ret.Topic = request.Topic
ret.BrokerPartitionAssignments = []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:17777",
FollowerBrokers: []string{"localhost:17777"},
Partition: &mq_pb.Partition{
RingSize: MaxPartitionCount,
RangeStart: 0,
RangeStop: MaxPartitionCount,
},
},
}
return ret, nil
}
// CheckTopicPartitionsStatus check the topic partitions on the broker
func (broker *MessageQueueBroker) CheckTopicPartitionsStatus(c context.Context, request *mq_pb.CheckTopicPartitionsStatusRequest) (*mq_pb.CheckTopicPartitionsStatusResponse, error) {
ret := &mq_pb.CheckTopicPartitionsStatusResponse{}
return ret, nil
}

View File

@ -7,6 +7,8 @@ import (
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"sync/atomic"
"time"
)
// For a new or re-configured topic, or one of the broker went offline,
@ -73,39 +75,89 @@ func (broker *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishS
// 3. write to the filer
var localTopicPartition *topic.LocalPartition
req, err := stream.Recv()
if err != nil {
return err
}
response := &mq_pb.PublishResponse{}
// TODO check whether current broker should be the leader for the topic partition
ackInterval := 1
initMessage := req.GetInit()
if initMessage != nil {
t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition)
localTopicPartition = broker.localTopicManager.GetTopicPartition(t, p)
if localTopicPartition == nil {
localTopicPartition = topic.NewLocalPartition(t, p, true, nil)
broker.localTopicManager.AddTopicPartition(t, localTopicPartition)
}
ackInterval = int(initMessage.AckInterval)
stream.Send(response)
} else {
response.Error = fmt.Sprintf("topic %v partition %v not found", initMessage.Topic, initMessage.Partition)
glog.Errorf("topic %v partition %v not found", initMessage.Topic, initMessage.Partition)
return stream.Send(response)
}
ackCounter := 0
var ackSequence int64
var isStopping int32
respChan := make(chan *mq_pb.PublishResponse, 128)
defer func() {
atomic.StoreInt32(&isStopping, 1)
close(respChan)
}()
go func() {
ticker := time.NewTicker(1 * time.Second)
for {
select {
case resp := <-respChan:
if resp != nil {
if err := stream.Send(resp); err != nil {
glog.Errorf("Error sending response %v: %v", resp, err)
}
} else {
return
}
case <-ticker.C:
if atomic.LoadInt32(&isStopping) == 0 {
response := &mq_pb.PublishResponse{
AckSequence: ackSequence,
}
respChan <- response
} else {
return
}
}
}
}()
// process each published messages
for {
// receive a message
req, err := stream.Recv()
if err != nil {
return err
}
// Process the received message
sequence := req.GetSequence()
response := &mq_pb.PublishResponse{
AckSequence: sequence,
}
if dataMessage := req.GetData(); dataMessage != nil {
if localTopicPartition == nil {
response.Error = "topic partition not initialized"
glog.Errorf("topic partition not found")
} else {
localTopicPartition.Publish(dataMessage)
}
} else if initMessage := req.GetInit(); initMessage != nil {
localTopicPartition = broker.localTopicManager.GetTopicPartition(
topic.NewTopic(topic.Namespace(initMessage.Segment.Namespace), initMessage.Segment.Topic),
topic.FromPbPartition(initMessage.Segment.Partition),
)
if localTopicPartition == nil {
response.Error = fmt.Sprintf("topic partition %v not found", initMessage.Segment)
glog.Errorf("topic partition %v not found", initMessage.Segment)
}
localTopicPartition.Publish(dataMessage)
}
if err := stream.Send(response); err != nil {
glog.Errorf("Error sending setup response: %v", err)
ackCounter++
ackSequence++
if ackCounter >= ackInterval {
ackCounter = 0
// send back the ack
response := &mq_pb.PublishResponse{
AckSequence: ackSequence,
}
respChan <- response
}
}
glog.Infof("publish stream closed")
return nil
}
@ -114,14 +166,14 @@ func (broker *MessageQueueBroker) AssignTopicPartitions(c context.Context, reque
ret := &mq_pb.AssignTopicPartitionsResponse{}
self := pb.ServerAddress(fmt.Sprintf("%s:%d", broker.option.Ip, broker.option.Port))
for _, partition := range request.TopicPartitionsAssignment.BrokerPartitions {
localPartiton := topic.FromPbBrokerPartitionsAssignment(self, partition)
for _, brokerPartition := range request.BrokerPartitionAssignments {
localPartiton := topic.FromPbBrokerPartitionAssignment(self, brokerPartition)
broker.localTopicManager.AddTopicPartition(
topic.FromPbTopic(request.Topic),
localPartiton)
if request.IsLeader {
for _, follower := range localPartiton.FollowerBrokers {
err := pb.WithBrokerClient(false, follower, broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
err := pb.WithBrokerGrpcClient(false, follower.String(), broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.AssignTopicPartitions(context.Background(), request)
return err
})

View File

@ -0,0 +1,44 @@
package broker
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"time"
)
func (broker *MessageQueueBroker) Subscribe(req *mq_pb.SubscribeRequest, stream mq_pb.SeaweedMessaging_SubscribeServer) error {
localTopicPartition := broker.localTopicManager.GetTopicPartition(topic.FromPbTopic(req.Cursor.Topic),
topic.FromPbPartition(req.Cursor.Partition))
if localTopicPartition == nil {
stream.Send(&mq_pb.SubscribeResponse{
Message: &mq_pb.SubscribeResponse_Ctrl{
Ctrl: &mq_pb.SubscribeResponse_CtrlMessage{
Error: "not initialized",
},
},
})
return nil
}
clientName := fmt.Sprintf("%s/%s-%s", req.Consumer.ConsumerGroup, req.Consumer.ConsumerId, req.Consumer.ClientId)
localTopicPartition.Subscribe(clientName, time.Now(), func(logEntry *filer_pb.LogEntry) error {
value := logEntry.GetData()
if err := stream.Send(&mq_pb.SubscribeResponse{Message: &mq_pb.SubscribeResponse_Data{
Data: &mq_pb.DataMessage{
Key: []byte(fmt.Sprintf("key-%d", logEntry.PartitionKeyHash)),
Value: value,
},
}}); err != nil {
glog.Errorf("Error sending setup response: %v", err)
return err
}
return nil
})
return nil
}

View File

@ -1,6 +1,9 @@
package broker
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"time"
@ -34,6 +37,8 @@ type MessageQueueBroker struct {
filers map[pb.ServerAddress]struct{}
currentFiler pb.ServerAddress
localTopicManager *topic.LocalTopicManager
Balancer *balancer.Balancer
lockAsBalancer *cluster.LiveLock
}
func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.DialOption) (mqBroker *MessageQueueBroker, err error) {
@ -41,9 +46,10 @@ func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.Dial
mqBroker = &MessageQueueBroker{
option: option,
grpcDialOption: grpcDialOption,
MasterClient: wdclient.NewMasterClient(grpcDialOption, option.FilerGroup, cluster.BrokerType, pb.NewServerAddress(option.Ip, option.Port, 0), option.DataCenter, option.Rack, *pb.NewServiceDiscoveryFromMap(option.Masters)),
MasterClient: wdclient.NewMasterClient(grpcDialOption, option.FilerGroup, cluster.BrokerType, pb.NewServerAddress(option.Ip, option.Port, 0), option.DataCenter, option.Rack, *pb.NewServiceDiscoveryFromMap(option.Masters)),
filers: make(map[pb.ServerAddress]struct{}),
localTopicManager: topic.NewLocalTopicManager(),
Balancer: balancer.NewBalancer(),
}
mqBroker.MasterClient.SetOnPeerUpdateFn(mqBroker.OnBrokerUpdate)
@ -54,6 +60,25 @@ func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.Dial
mqBroker.OnBrokerUpdate(newNode, time.Now())
}
// keep connecting to balancer
go func() {
for mqBroker.currentFiler == "" {
time.Sleep(time.Millisecond * 237)
}
self := fmt.Sprintf("%s:%d", option.Ip, option.Port)
glog.V(1).Infof("broker %s found filer %s", self, mqBroker.currentFiler)
lockClient := cluster.NewLockClient(grpcDialOption, mqBroker.currentFiler)
mqBroker.lockAsBalancer = lockClient.StartLock(LockBrokerBalancer, self)
for {
err := mqBroker.BrokerConnectToBalancer(self)
if err != nil {
fmt.Printf("BrokerConnectToBalancer: %v\n", err)
}
time.Sleep(time.Second)
}
}()
return mqBroker, nil
}
@ -112,7 +137,7 @@ func (broker *MessageQueueBroker) withMasterClient(streamingMode bool, master pb
func (broker *MessageQueueBroker) withBrokerClient(streamingMode bool, server pb.ServerAddress, fn func(client mq_pb.SeaweedMessagingClient) error) error {
return pb.WithBrokerClient(streamingMode, server, broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
return pb.WithBrokerGrpcClient(streamingMode, server.String(), broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
return fn(client)
})

View File

@ -0,0 +1,74 @@
package broker
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"math/rand"
"time"
)
const (
LockBrokerBalancer = "broker_balancer"
)
// BrokerConnectToBalancer connects to the broker balancer and sends stats
func (broker *MessageQueueBroker) BrokerConnectToBalancer(self string) error {
// find the lock owner
var brokerBalancer string
err := broker.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
resp, err := client.FindLockOwner(context.Background(), &filer_pb.FindLockOwnerRequest{
Name: LockBrokerBalancer,
})
if err != nil {
return err
}
brokerBalancer = resp.Owner
return nil
})
if err != nil {
return err
}
glog.V(1).Infof("broker %s found balancer %s", self, brokerBalancer)
// connect to the lock owner
err = pb.WithBrokerGrpcClient(false, brokerBalancer, broker.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
stream, err := client.ConnectToBalancer(context.Background())
if err != nil {
return fmt.Errorf("connect to balancer %v: %v", brokerBalancer, err)
}
defer stream.CloseSend()
err = stream.Send(&mq_pb.ConnectToBalancerRequest{
Message: &mq_pb.ConnectToBalancerRequest_Init{
Init: &mq_pb.ConnectToBalancerRequest_InitMessage{
Broker: self,
},
},
})
if err != nil {
return fmt.Errorf("send init message: %v", err)
}
for {
stats := broker.localTopicManager.CollectStats(time.Second * 5)
err = stream.Send(&mq_pb.ConnectToBalancerRequest{
Message: &mq_pb.ConnectToBalancerRequest_Stats{
Stats: stats,
},
})
if err != nil {
return fmt.Errorf("send stats message: %v", err)
}
time.Sleep(time.Millisecond*5000 + time.Duration(rand.Intn(1000))*time.Millisecond)
}
return nil
})
return err
}

View File

@ -0,0 +1,58 @@
package main
import (
"flag"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client"
"log"
"sync"
"time"
)
var (
messageCount = flag.Int("n", 1000, "message count")
concurrency = flag.Int("c", 4, "concurrency count")
)
func doPublish(publisher *pub_client.TopicPublisher, id int) {
startTime := time.Now()
for i := 0; i < *messageCount / *concurrency; i++ {
// Simulate publishing a message
key := []byte(fmt.Sprintf("key-%d-%d", id, i))
value := []byte(fmt.Sprintf("value-%d-%d", id, i))
publisher.Publish(key, value) // Call your publisher function here
// println("Published", string(key), string(value))
}
elapsed := time.Since(startTime)
log.Printf("Publisher %d finished in %s", id, elapsed)
}
func main() {
flag.Parse()
publisher := pub_client.NewTopicPublisher(
"test", "test")
if err := publisher.Connect("localhost:17777"); err != nil {
fmt.Println(err)
return
}
startTime := time.Now()
// Start multiple publishers
var wg sync.WaitGroup
for i := 0; i < *concurrency; i++ {
wg.Add(1)
go func(id int) {
defer wg.Done()
doPublish(publisher, id)
}(i)
}
// Wait for all publishers to finish
wg.Wait()
elapsed := time.Since(startTime)
publisher.Shutdown()
log.Printf("Published %d messages in %s (%.2f msg/s)", *messageCount, elapsed, float64(*messageCount)/elapsed.Seconds())
}

View File

@ -0,0 +1,44 @@
package main
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
)
func main() {
subscriberConfig := &sub_client.SubscriberConfiguration{
ClientId: "testSubscriber",
GroupId: "test",
GroupInstanceId: "test",
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
}
contentConfig := &sub_client.ContentConfiguration{
Namespace: "test",
Topic: "test",
Filter: "",
}
subscriber := sub_client.NewTopicSubscriber(subscriberConfig, contentConfig)
if err := subscriber.Connect("localhost:17777"); err != nil {
fmt.Println(err)
return
}
subscriber.SetEachMessageFunc(func(key, value []byte) bool {
println(string(key), "=>", string(value))
return true
})
subscriber.SetCompletionFunc(func() {
println("done subscribing")
})
if err := subscriber.Subscribe(); err != nil {
fmt.Println(err)
}
}

View File

@ -0,0 +1,116 @@
package pub_client
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
func (p *TopicPublisher) doLookup(brokerAddress string) error {
err := pb.WithBrokerGrpcClient(true,
brokerAddress,
p.grpcDialOption,
func(client mq_pb.SeaweedMessagingClient) error {
lookupResp, err := client.LookupTopicBrokers(context.Background(),
&mq_pb.LookupTopicBrokersRequest{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
IsForPublish: true,
})
if err != nil {
return err
}
for _, brokerPartitionAssignment := range lookupResp.BrokerPartitionAssignments {
// partition => publishClient
publishClient, redirectTo, err := p.doConnect(brokerPartitionAssignment.Partition, brokerPartitionAssignment.LeaderBroker)
if err != nil {
return err
}
for redirectTo != "" {
publishClient, redirectTo, err = p.doConnect(brokerPartitionAssignment.Partition, redirectTo)
if err != nil {
return err
}
}
p.partition2Broker.Insert(
brokerPartitionAssignment.Partition.RangeStart,
brokerPartitionAssignment.Partition.RangeStop,
publishClient)
}
return nil
})
if err != nil {
return fmt.Errorf("lookup topic %s/%s: %v", p.namespace, p.topic, err)
}
return nil
}
// broker => publish client
// send init message
// save the publishing client
func (p *TopicPublisher) doConnect(partition *mq_pb.Partition, brokerAddress string) (publishClient *PublishClient, redirectTo string, err error) {
grpcConnection, err := pb.GrpcDial(context.Background(), brokerAddress, true, p.grpcDialOption)
if err != nil {
return publishClient, redirectTo, fmt.Errorf("dial broker %s: %v", brokerAddress, err)
}
brokerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection)
stream, err := brokerClient.Publish(context.Background())
if err != nil {
return publishClient, redirectTo, fmt.Errorf("create publish client: %v", err)
}
publishClient = &PublishClient{
SeaweedMessaging_PublishClient: stream,
Broker: brokerAddress,
}
if err = publishClient.Send(&mq_pb.PublishRequest{
Message: &mq_pb.PublishRequest_Init{
Init: &mq_pb.PublishRequest_InitMessage{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
Partition: &mq_pb.Partition{
RingSize: partition.RingSize,
RangeStart: partition.RangeStart,
RangeStop: partition.RangeStop,
},
AckInterval: 128,
},
},
}); err != nil {
return publishClient, redirectTo, fmt.Errorf("send init message: %v", err)
}
resp, err := stream.Recv()
if err != nil {
return publishClient, redirectTo, fmt.Errorf("recv init response: %v", err)
}
if resp.Error != "" {
return publishClient, redirectTo, fmt.Errorf("init response error: %v", resp.Error)
}
if resp.RedirectToBroker != "" {
redirectTo = resp.RedirectToBroker
return publishClient, redirectTo, nil
}
go func() {
for {
_, err := publishClient.Recv()
if err != nil {
e, ok := status.FromError(err)
if ok && e.Code() == codes.Unknown && e.Message() == "EOF" {
return
}
publishClient.Err = err
fmt.Printf("publish to %s error: %v\n", publishClient.Broker, err)
return
}
}
}()
return publishClient, redirectTo, nil
}

View File

@ -0,0 +1,41 @@
package pub_client
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/mq/broker"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
)
func (p *TopicPublisher) Publish(key, value []byte) error {
hashKey := util.HashToInt32(key) % broker.MaxPartitionCount
if hashKey < 0 {
hashKey = -hashKey
}
publishClient, found := p.partition2Broker.Floor(hashKey, hashKey)
if !found {
return fmt.Errorf("no broker found for key %d", hashKey)
}
p.Lock()
defer p.Unlock()
// dead lock here
//google.golang.org/grpc/internal/transport.(*writeQuota).get(flowcontrol.go:59)
//google.golang.org/grpc/internal/transport.(*http2Client).Write(http2_client.go:1047)
//google.golang.org/grpc.(*csAttempt).sendMsg(stream.go:1040)
//google.golang.org/grpc.(*clientStream).SendMsg.func2(stream.go:892)
//google.golang.org/grpc.(*clientStream).withRetry(stream.go:752)
//google.golang.org/grpc.(*clientStream).SendMsg(stream.go:894)
//github.com/seaweedfs/seaweedfs/weed/pb/mq_pb.(*seaweedMessagingPublishClient).Send(mq_grpc.pb.go:141)
//github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client.(*TopicPublisher).Publish(publish.go:19)
if err := publishClient.Send(&mq_pb.PublishRequest{
Message: &mq_pb.PublishRequest_Data{
Data: &mq_pb.DataMessage{
Key: key,
Value: value,
},
},
}); err != nil {
return fmt.Errorf("send publish request: %v", err)
}
return nil
}

View File

@ -0,0 +1,57 @@
package pub_client
import (
"github.com/rdleal/intervalst/interval"
"github.com/seaweedfs/seaweedfs/weed/mq/broker"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"sync"
"time"
)
type PublisherConfiguration struct {
}
type PublishClient struct {
mq_pb.SeaweedMessaging_PublishClient
Broker string
Err error
}
type TopicPublisher struct {
namespace string
topic string
partition2Broker *interval.SearchTree[*PublishClient, int32]
grpcDialOption grpc.DialOption
sync.Mutex // protects grpc
}
func NewTopicPublisher(namespace, topic string) *TopicPublisher {
return &TopicPublisher{
namespace: namespace,
topic: topic,
partition2Broker: interval.NewSearchTree[*PublishClient](func(a, b int32) int {
return int(a - b)
}),
grpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
}
}
func (p *TopicPublisher) Connect(bootstrapBroker string) error {
if err := p.doLookup(bootstrapBroker); err != nil {
return err
}
return nil
}
func (p *TopicPublisher) Shutdown() error {
if clients, found := p.partition2Broker.AllIntersections(0, broker.MaxPartitionCount); found {
for _, client := range clients {
client.CloseSend()
}
}
time.Sleep(1100 * time.Millisecond)
return nil
}

View File

@ -0,0 +1,34 @@
package sub_client
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
)
func (sub *TopicSubscriber) doLookup(brokerAddress string) error {
err := pb.WithBrokerGrpcClient(true,
brokerAddress,
sub.SubscriberConfig.GrpcDialOption,
func(client mq_pb.SeaweedMessagingClient) error {
lookupResp, err := client.LookupTopicBrokers(context.Background(),
&mq_pb.LookupTopicBrokersRequest{
Topic: &mq_pb.Topic{
Namespace: sub.ContentConfig.Namespace,
Name: sub.ContentConfig.Topic,
},
IsForPublish: false,
})
if err != nil {
return err
}
sub.brokerPartitionAssignments = lookupResp.BrokerPartitionAssignments
return nil
})
if err != nil {
return fmt.Errorf("lookup topic %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
}
return nil
}

View File

@ -0,0 +1,72 @@
package sub_client
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"sync"
)
// Subscribe subscribes to a topic's specified partitions.
// If a partition is moved to another broker, the subscriber will automatically reconnect to the new broker.
func (sub *TopicSubscriber) Subscribe() error {
var wg sync.WaitGroup
for _, brokerPartitionAssignment := range sub.brokerPartitionAssignments {
brokerAddress := brokerPartitionAssignment.LeaderBroker
grpcConnection, err := pb.GrpcDial(context.Background(), brokerAddress, true, sub.SubscriberConfig.GrpcDialOption)
if err != nil {
return fmt.Errorf("dial broker %s: %v", brokerAddress, err)
}
brokerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection)
subscribeClient, err := brokerClient.Subscribe(context.Background(), &mq_pb.SubscribeRequest{
Consumer: &mq_pb.SubscribeRequest_Consumer{
ConsumerGroup: sub.SubscriberConfig.GroupId,
ConsumerId: sub.SubscriberConfig.GroupInstanceId,
},
Cursor: &mq_pb.SubscribeRequest_Cursor{
Topic: &mq_pb.Topic{
Namespace: sub.ContentConfig.Namespace,
Name: sub.ContentConfig.Topic,
},
Partition: &mq_pb.Partition{
RingSize: brokerPartitionAssignment.Partition.RingSize,
RangeStart: brokerPartitionAssignment.Partition.RangeStart,
RangeStop: brokerPartitionAssignment.Partition.RangeStop,
},
Filter: sub.ContentConfig.Filter,
},
})
if err != nil {
return fmt.Errorf("create subscribe client: %v", err)
}
wg.Add(1)
go func() {
defer wg.Done()
if sub.OnCompletionFunc != nil {
defer sub.OnCompletionFunc()
}
for {
resp, err := subscribeClient.Recv()
if err != nil {
fmt.Printf("subscribe error: %v\n", err)
return
}
if resp.Message == nil {
continue
}
switch m := resp.Message.(type) {
case *mq_pb.SubscribeResponse_Data:
if !sub.OnEachMessageFunc(m.Data.Key, m.Data.Value) {
return
}
case *mq_pb.SubscribeResponse_Ctrl:
// ignore
}
}
}()
}
wg.Wait()
return nil
}

View File

@ -0,0 +1,53 @@
package sub_client
import (
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc"
)
type SubscriberConfiguration struct {
ClientId string
GroupId string
GroupInstanceId string
BootstrapServers []string
GrpcDialOption grpc.DialOption
}
type ContentConfiguration struct {
Namespace string
Topic string
Filter string
}
type OnEachMessageFunc func(key, value []byte) (shouldContinue bool)
type OnCompletionFunc func()
type TopicSubscriber struct {
SubscriberConfig *SubscriberConfiguration
ContentConfig *ContentConfiguration
brokerPartitionAssignments []*mq_pb.BrokerPartitionAssignment
OnEachMessageFunc OnEachMessageFunc
OnCompletionFunc OnCompletionFunc
}
func NewTopicSubscriber(subscriber *SubscriberConfiguration, content *ContentConfiguration) *TopicSubscriber {
return &TopicSubscriber{
SubscriberConfig: subscriber,
ContentConfig: content,
}
}
func (sub *TopicSubscriber) Connect(bootstrapBroker string) error {
if err := sub.doLookup(bootstrapBroker); err != nil {
return err
}
return nil
}
func (sub *TopicSubscriber) SetEachMessageFunc(onEachMessageFn OnEachMessageFunc) {
sub.OnEachMessageFunc = onEachMessageFn
}
func (sub *TopicSubscriber) SetCompletionFunc(onCompeletionFn OnCompletionFunc) {
sub.OnCompletionFunc = onCompeletionFn
}

View File

@ -2,6 +2,9 @@ package topic
import (
cmap "github.com/orcaman/concurrent-map/v2"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/shirou/gopsutil/v3/cpu"
"time"
)
// LocalTopicManager manages topics on local broker
@ -25,6 +28,7 @@ func (manager *LocalTopicManager) AddTopicPartition(topic Topic, localPartition
Partitions: make([]*LocalPartition, 0),
}
}
manager.topics.SetIfAbsent(topic.String(), localTopic)
if localTopic.findPartition(localPartition.Partition) != nil {
return
}
@ -52,3 +56,22 @@ func (manager *LocalTopicManager) RemoveTopicPartition(topic Topic, partition Pa
}
return localTopic.removePartition(partition)
}
func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.BrokerStats {
stats := &mq_pb.BrokerStats{}
manager.topics.IterCb(func(topic string, localTopic *LocalTopic) {
for _, localPartition := range localTopic.Partitions {
stats.TopicPartitionCount++
stats.ConsumerCount += localPartition.ConsumerCount
}
})
// collect current broker's cpu usage
usages, err := cpu.Percent(duration, false)
if err == nil && len(usages) > 0 {
stats.CpuUsagePercent = int32(usages[0])
}
return stats
}

View File

@ -1,7 +1,9 @@
package topic
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"time"
@ -12,21 +14,44 @@ type LocalPartition struct {
isLeader bool
FollowerBrokers []pb.ServerAddress
logBuffer *log_buffer.LogBuffer
ConsumerCount int32
}
func (p LocalPartition) Publish(message *mq_pb.PublishRequest_DataMessage) {
func NewLocalPartition(topic Topic, partition Partition, isLeader bool, followerBrokers []pb.ServerAddress) *LocalPartition {
return &LocalPartition{
Partition: partition,
isLeader: isLeader,
FollowerBrokers: followerBrokers,
logBuffer: log_buffer.NewLogBuffer(
fmt.Sprintf("%s/%s/%4d-%4d", topic.Namespace, topic.Name, partition.RangeStart, partition.RangeStop),
2*time.Minute,
func(startTime, stopTime time.Time, buf []byte) {
},
func() {
},
),
}
}
type OnEachMessageFn func(logEntry *filer_pb.LogEntry) error
func (p LocalPartition) Publish(message *mq_pb.DataMessage) {
p.logBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano())
}
func FromPbBrokerPartitionsAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionsAssignment) *LocalPartition {
func (p LocalPartition) Subscribe(clientName string, startReadTime time.Time, eachMessageFn OnEachMessageFn) {
p.logBuffer.LoopProcessLogData(clientName, startReadTime, 0, func() bool {
return true
}, eachMessageFn)
}
func FromPbBrokerPartitionAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionAssignment) *LocalPartition {
isLeaer := assignment.LeaderBroker == string(self)
localPartition := &LocalPartition{
Partition: Partition{
RangeStart: assignment.PartitionStart,
RangeStop: assignment.PartitionStop,
RingSize: PartitionCount,
},
isLeader: isLeaer,
Partition: FromPbPartition(assignment.Partition),
isLeader: isLeaer,
}
if !isLeaer {
return localPartition

View File

@ -69,10 +69,13 @@ service SeaweedFiler {
rpc CacheRemoteObjectToLocalCluster (CacheRemoteObjectToLocalClusterRequest) returns (CacheRemoteObjectToLocalClusterResponse) {
}
rpc Lock(LockRequest) returns (LockResponse) {
rpc DistributedLock(LockRequest) returns (LockResponse) {
}
rpc Unlock(UnlockRequest) returns (UnlockResponse) {
rpc DistributedUnlock(UnlockRequest) returns (UnlockResponse) {
}
rpc FindLockOwner(FindLockOwnerRequest) returns (FindLockOwnerResponse) {
}
// distributed lock management internal use only
rpc TransferLocks(TransferLocksRequest) returns (TransferLocksResponse) {
}
}
@ -444,6 +447,7 @@ message LockRequest {
int64 seconds_to_lock = 2;
string renew_token = 3;
bool is_moved = 4;
string owner = 5;
}
message LockResponse {
string renew_token = 1;
@ -459,10 +463,18 @@ message UnlockResponse {
string error = 1;
string moved_to = 2;
}
message FindLockOwnerRequest {
string name = 1;
bool is_moved = 2;
}
message FindLockOwnerResponse {
string owner = 1;
}
message Lock {
string name = 1;
string renew_token = 2;
int64 expired_at_ns = 3;
string owner = 4;
}
message TransferLocksRequest {
repeated Lock locks = 1;

View File

@ -3621,6 +3621,7 @@ type LockRequest struct {
SecondsToLock int64 `protobuf:"varint,2,opt,name=seconds_to_lock,json=secondsToLock,proto3" json:"seconds_to_lock,omitempty"`
RenewToken string `protobuf:"bytes,3,opt,name=renew_token,json=renewToken,proto3" json:"renew_token,omitempty"`
IsMoved bool `protobuf:"varint,4,opt,name=is_moved,json=isMoved,proto3" json:"is_moved,omitempty"`
Owner string `protobuf:"bytes,5,opt,name=owner,proto3" json:"owner,omitempty"`
}
func (x *LockRequest) Reset() {
@ -3683,6 +3684,13 @@ func (x *LockRequest) GetIsMoved() bool {
return false
}
func (x *LockRequest) GetOwner() string {
if x != nil {
return x.Owner
}
return ""
}
type LockResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -3864,6 +3872,108 @@ func (x *UnlockResponse) GetMovedTo() string {
return ""
}
type FindLockOwnerRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
IsMoved bool `protobuf:"varint,2,opt,name=is_moved,json=isMoved,proto3" json:"is_moved,omitempty"`
}
func (x *FindLockOwnerRequest) Reset() {
*x = FindLockOwnerRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_filer_proto_msgTypes[59]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *FindLockOwnerRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*FindLockOwnerRequest) ProtoMessage() {}
func (x *FindLockOwnerRequest) ProtoReflect() protoreflect.Message {
mi := &file_filer_proto_msgTypes[59]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use FindLockOwnerRequest.ProtoReflect.Descriptor instead.
func (*FindLockOwnerRequest) Descriptor() ([]byte, []int) {
return file_filer_proto_rawDescGZIP(), []int{59}
}
func (x *FindLockOwnerRequest) GetName() string {
if x != nil {
return x.Name
}
return ""
}
func (x *FindLockOwnerRequest) GetIsMoved() bool {
if x != nil {
return x.IsMoved
}
return false
}
type FindLockOwnerResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
Owner string `protobuf:"bytes,1,opt,name=owner,proto3" json:"owner,omitempty"`
}
func (x *FindLockOwnerResponse) Reset() {
*x = FindLockOwnerResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_filer_proto_msgTypes[60]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *FindLockOwnerResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*FindLockOwnerResponse) ProtoMessage() {}
func (x *FindLockOwnerResponse) ProtoReflect() protoreflect.Message {
mi := &file_filer_proto_msgTypes[60]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
ms.StoreMessageInfo(mi)
}
return ms
}
return mi.MessageOf(x)
}
// Deprecated: Use FindLockOwnerResponse.ProtoReflect.Descriptor instead.
func (*FindLockOwnerResponse) Descriptor() ([]byte, []int) {
return file_filer_proto_rawDescGZIP(), []int{60}
}
func (x *FindLockOwnerResponse) GetOwner() string {
if x != nil {
return x.Owner
}
return ""
}
type Lock struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -3872,12 +3982,13 @@ type Lock struct {
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
RenewToken string `protobuf:"bytes,2,opt,name=renew_token,json=renewToken,proto3" json:"renew_token,omitempty"`
ExpiredAtNs int64 `protobuf:"varint,3,opt,name=expired_at_ns,json=expiredAtNs,proto3" json:"expired_at_ns,omitempty"`
Owner string `protobuf:"bytes,4,opt,name=owner,proto3" json:"owner,omitempty"`
}
func (x *Lock) Reset() {
*x = Lock{}
if protoimpl.UnsafeEnabled {
mi := &file_filer_proto_msgTypes[59]
mi := &file_filer_proto_msgTypes[61]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -3890,7 +4001,7 @@ func (x *Lock) String() string {
func (*Lock) ProtoMessage() {}
func (x *Lock) ProtoReflect() protoreflect.Message {
mi := &file_filer_proto_msgTypes[59]
mi := &file_filer_proto_msgTypes[61]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -3903,7 +4014,7 @@ func (x *Lock) ProtoReflect() protoreflect.Message {
// Deprecated: Use Lock.ProtoReflect.Descriptor instead.
func (*Lock) Descriptor() ([]byte, []int) {
return file_filer_proto_rawDescGZIP(), []int{59}
return file_filer_proto_rawDescGZIP(), []int{61}
}
func (x *Lock) GetName() string {
@ -3927,6 +4038,13 @@ func (x *Lock) GetExpiredAtNs() int64 {
return 0
}
func (x *Lock) GetOwner() string {
if x != nil {
return x.Owner
}
return ""
}
type TransferLocksRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -3938,7 +4056,7 @@ type TransferLocksRequest struct {
func (x *TransferLocksRequest) Reset() {
*x = TransferLocksRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_filer_proto_msgTypes[60]
mi := &file_filer_proto_msgTypes[62]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -3951,7 +4069,7 @@ func (x *TransferLocksRequest) String() string {
func (*TransferLocksRequest) ProtoMessage() {}
func (x *TransferLocksRequest) ProtoReflect() protoreflect.Message {
mi := &file_filer_proto_msgTypes[60]
mi := &file_filer_proto_msgTypes[62]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -3964,7 +4082,7 @@ func (x *TransferLocksRequest) ProtoReflect() protoreflect.Message {
// Deprecated: Use TransferLocksRequest.ProtoReflect.Descriptor instead.
func (*TransferLocksRequest) Descriptor() ([]byte, []int) {
return file_filer_proto_rawDescGZIP(), []int{60}
return file_filer_proto_rawDescGZIP(), []int{62}
}
func (x *TransferLocksRequest) GetLocks() []*Lock {
@ -3983,7 +4101,7 @@ type TransferLocksResponse struct {
func (x *TransferLocksResponse) Reset() {
*x = TransferLocksResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_filer_proto_msgTypes[61]
mi := &file_filer_proto_msgTypes[63]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -3996,7 +4114,7 @@ func (x *TransferLocksResponse) String() string {
func (*TransferLocksResponse) ProtoMessage() {}
func (x *TransferLocksResponse) ProtoReflect() protoreflect.Message {
mi := &file_filer_proto_msgTypes[61]
mi := &file_filer_proto_msgTypes[63]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -4009,7 +4127,7 @@ func (x *TransferLocksResponse) ProtoReflect() protoreflect.Message {
// Deprecated: Use TransferLocksResponse.ProtoReflect.Descriptor instead.
func (*TransferLocksResponse) Descriptor() ([]byte, []int) {
return file_filer_proto_rawDescGZIP(), []int{61}
return file_filer_proto_rawDescGZIP(), []int{63}
}
// if found, send the exact address
@ -4026,7 +4144,7 @@ type LocateBrokerResponse_Resource struct {
func (x *LocateBrokerResponse_Resource) Reset() {
*x = LocateBrokerResponse_Resource{}
if protoimpl.UnsafeEnabled {
mi := &file_filer_proto_msgTypes[64]
mi := &file_filer_proto_msgTypes[66]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -4039,7 +4157,7 @@ func (x *LocateBrokerResponse_Resource) String() string {
func (*LocateBrokerResponse_Resource) ProtoMessage() {}
func (x *LocateBrokerResponse_Resource) ProtoReflect() protoreflect.Message {
mi := &file_filer_proto_msgTypes[64]
mi := &file_filer_proto_msgTypes[66]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -4090,7 +4208,7 @@ type FilerConf_PathConf struct {
func (x *FilerConf_PathConf) Reset() {
*x = FilerConf_PathConf{}
if protoimpl.UnsafeEnabled {
mi := &file_filer_proto_msgTypes[65]
mi := &file_filer_proto_msgTypes[67]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -4103,7 +4221,7 @@ func (x *FilerConf_PathConf) String() string {
func (*FilerConf_PathConf) ProtoMessage() {}
func (x *FilerConf_PathConf) ProtoReflect() protoreflect.Message {
mi := &file_filer_proto_msgTypes[65]
mi := &file_filer_proto_msgTypes[67]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -4687,7 +4805,7 @@ var file_filer_proto_rawDesc = []byte{
0x54, 0x6f, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x18,
0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62,
0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x22, 0x85, 0x01,
0x2e, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x22, 0x9b, 0x01,
0x0a, 0x0b, 0x4c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a,
0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d,
0x65, 0x12, 0x26, 0x0a, 0x0f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x5f, 0x74, 0x6f, 0x5f,
@ -4696,163 +4814,179 @@ var file_filer_proto_rawDesc = []byte{
0x65, 0x77, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a,
0x72, 0x65, 0x6e, 0x65, 0x77, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73,
0x5f, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73,
0x4d, 0x6f, 0x76, 0x65, 0x64, 0x22, 0x60, 0x0a, 0x0c, 0x4c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73,
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x6e, 0x65, 0x77, 0x5f, 0x74,
0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x6e, 0x65,
0x77, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x5f,
0x4d, 0x6f, 0x76, 0x65, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x18, 0x05,
0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x22, 0x60, 0x0a, 0x0c, 0x4c,
0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72,
0x65, 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
0x52, 0x0a, 0x72, 0x65, 0x6e, 0x65, 0x77, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x19, 0x0a, 0x08,
0x6d, 0x6f, 0x76, 0x65, 0x64, 0x5f, 0x74, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07,
0x6d, 0x6f, 0x76, 0x65, 0x64, 0x54, 0x6f, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72,
0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x5f, 0x0a,
0x0d, 0x55, 0x6e, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12,
0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61,
0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x6b, 0x65,
0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x6e, 0x65, 0x77, 0x54, 0x6f,
0x6b, 0x65, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x69, 0x73, 0x5f, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x18,
0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x69, 0x73, 0x4d, 0x6f, 0x76, 0x65, 0x64, 0x22, 0x41,
0x0a, 0x0e, 0x55, 0x6e, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x5f,
0x74, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x54,
0x6f, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09,
0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x5f, 0x0a, 0x0d, 0x55, 0x6e, 0x6c, 0x6f, 0x63,
0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65,
0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b,
0x72, 0x65, 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28,
0x09, 0x52, 0x0a, 0x72, 0x65, 0x6e, 0x65, 0x77, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x19, 0x0a,
0x08, 0x69, 0x73, 0x5f, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52,
0x07, 0x69, 0x73, 0x4d, 0x6f, 0x76, 0x65, 0x64, 0x22, 0x41, 0x0a, 0x0e, 0x55, 0x6e, 0x6c, 0x6f,
0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72,
0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72,
0x12, 0x19, 0x0a, 0x08, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x5f, 0x74, 0x6f, 0x18, 0x02, 0x20, 0x01,
0x28, 0x09, 0x52, 0x07, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x54, 0x6f, 0x22, 0x5f, 0x0a, 0x04, 0x4c,
0x6f, 0x63, 0x6b, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x6e, 0x65, 0x77,
0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x65,
0x6e, 0x65, 0x77, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x22, 0x0a, 0x0d, 0x65, 0x78, 0x70, 0x69,
0x72, 0x65, 0x64, 0x5f, 0x61, 0x74, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52,
0x0b, 0x65, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x41, 0x74, 0x4e, 0x73, 0x22, 0x3c, 0x0a, 0x14,
0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x6f, 0x63, 0x6b, 0x73, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x12, 0x24, 0x0a, 0x05, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x18, 0x01, 0x20,
0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c,
0x6f, 0x63, 0x6b, 0x52, 0x05, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x54, 0x72,
0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x6f, 0x63, 0x6b, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x32, 0xa5, 0x0f, 0x0a, 0x0c, 0x53, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x46,
0x69, 0x6c, 0x65, 0x72, 0x12, 0x67, 0x0a, 0x14, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x69,
0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x25, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x69,
0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c,
0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e,
0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4e, 0x0a,
0x0b, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x12, 0x1c, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72,
0x69, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65,
0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x4c, 0x0a,
0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1c, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e,
0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72,
0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4c, 0x0a, 0x0b, 0x55,
0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1c, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72,
0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72,
0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0d, 0x41, 0x70, 0x70,
0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1e, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e,
0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e,
0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4c, 0x0a,
0x0b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1c, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e,
0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72,
0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x11, 0x41,
0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79,
0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x74, 0x6f, 0x6d,
0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e,
0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72,
0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x53,
0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79,
0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x72, 0x65,
0x61, 0x6d, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e,
0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72,
0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x4f, 0x0a,
0x0c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x1d, 0x2e,
0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x56,
0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x56, 0x6f,
0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4f,
0x0a, 0x0c, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12, 0x1d,
0x6f, 0x22, 0x45, 0x0a, 0x14, 0x46, 0x69, 0x6e, 0x64, 0x4c, 0x6f, 0x63, 0x6b, 0x4f, 0x77, 0x6e,
0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d,
0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x19, 0x0a,
0x08, 0x69, 0x73, 0x5f, 0x6d, 0x6f, 0x76, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52,
0x07, 0x69, 0x73, 0x4d, 0x6f, 0x76, 0x65, 0x64, 0x22, 0x2d, 0x0a, 0x15, 0x46, 0x69, 0x6e, 0x64,
0x4c, 0x6f, 0x63, 0x6b, 0x4f, 0x77, 0x6e, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x12, 0x14, 0x0a, 0x05, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
0x52, 0x05, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x22, 0x75, 0x0a, 0x04, 0x4c, 0x6f, 0x63, 0x6b, 0x12,
0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e,
0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x6e, 0x65, 0x77, 0x5f, 0x74, 0x6f, 0x6b,
0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x6e, 0x65, 0x77, 0x54,
0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x22, 0x0a, 0x0d, 0x65, 0x78, 0x70, 0x69, 0x72, 0x65, 0x64, 0x5f,
0x61, 0x74, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x65, 0x78, 0x70,
0x69, 0x72, 0x65, 0x64, 0x41, 0x74, 0x4e, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x6f, 0x77, 0x6e, 0x65,
0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x22, 0x3c,
0x0a, 0x14, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x6f, 0x63, 0x6b, 0x73, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x24, 0x0a, 0x05, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x18,
0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62,
0x2e, 0x4c, 0x6f, 0x63, 0x6b, 0x52, 0x05, 0x6c, 0x6f, 0x63, 0x6b, 0x73, 0x22, 0x17, 0x0a, 0x15,
0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x6f, 0x63, 0x6b, 0x73, 0x52, 0x65, 0x73,
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, 0x8f, 0x10, 0x0a, 0x0c, 0x53, 0x65, 0x61, 0x77, 0x65, 0x65,
0x64, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x12, 0x67, 0x0a, 0x14, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70,
0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x25,
0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70,
0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e,
0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56,
0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
0x55, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73,
0x74, 0x12, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f, 0x6c,
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x1a, 0x20, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x6f,
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65,
0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c,
0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e,
0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43,
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x00, 0x12, 0x49, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63,
0x73, 0x12, 0x1b, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61,
0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c,
0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73,
0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x37,
0x0a, 0x04, 0x50, 0x69, 0x6e, 0x67, 0x12, 0x15, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70,
0x62, 0x2e, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e,
0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73,
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6a, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x46, 0x69,
0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e,
0x12, 0x26, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x46,
0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f,
0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72,
0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66,
0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65,
0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72,
0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74,
0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62,
0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x65, 0x0a, 0x16, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69,
0x62, 0x65, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12,
0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63,
0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53,
0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x3a, 0x0a, 0x05,
0x4b, 0x76, 0x47, 0x65, 0x74, 0x12, 0x16, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62,
0x2e, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e,
0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x4b, 0x76, 0x50, 0x75,
0x74, 0x12, 0x16, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x50,
0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x66, 0x69, 0x6c, 0x65,
0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 0x22, 0x00, 0x12, 0x88, 0x01, 0x0a, 0x1f, 0x43, 0x61, 0x63, 0x68, 0x65, 0x52, 0x65,
0x6d, 0x6f, 0x74, 0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x6f, 0x4c, 0x6f, 0x63, 0x61,
0x6c, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x30, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72,
0x5f, 0x70, 0x62, 0x2e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x4f,
0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x6f, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x43, 0x6c, 0x75, 0x73,
0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x66, 0x69, 0x6c,
0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62,
0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79,
0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
0x4e, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x12, 0x1c,
0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e,
0x74, 0x72, 0x69, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72,
0x69, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12,
0x4c, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1c,
0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65,
0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x6e,
0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4c, 0x0a,
0x0b, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1c, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e,
0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72,
0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0d, 0x41,
0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1e, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f,
0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x54, 0x6f,
0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
0x4c, 0x0a, 0x0b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x1c,
0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65,
0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, 0x6e,
0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a,
0x11, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74,
0x72, 0x79, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x74,
0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70,
0x62, 0x2e, 0x41, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e,
0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a,
0x11, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74,
0x72, 0x79, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74,
0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70,
0x62, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x45, 0x6e,
0x74, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12,
0x4f, 0x0a, 0x0c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x12,
0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67,
0x6e, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e,
0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e,
0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
0x12, 0x4f, 0x0a, 0x0c, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65,
0x12, 0x1d, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b,
0x75, 0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75,
0x70, 0x56, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
0x00, 0x12, 0x55, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c,
0x69, 0x73, 0x74, 0x12, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43,
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e,
0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65,
0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x6f,
0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74,
0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x49, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74,
0x69, 0x63, 0x73, 0x12, 0x1b, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53,
0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x1a, 0x1c, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, 0x74,
0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00,
0x12, 0x37, 0x0a, 0x04, 0x50, 0x69, 0x6e, 0x67, 0x12, 0x15, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72,
0x5f, 0x70, 0x62, 0x2e, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x16, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x69, 0x6e, 0x67, 0x52,
0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6a, 0x0a, 0x15, 0x47, 0x65, 0x74,
0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69,
0x6f, 0x6e, 0x12, 0x26, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65,
0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74,
0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f,
0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69,
0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d,
0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23,
0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72,
0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x65, 0x0a, 0x16, 0x53, 0x75, 0x62, 0x73, 0x63,
0x72, 0x69, 0x62, 0x65, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74,
0x61, 0x12, 0x22, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62,
0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62,
0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61,
0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x3a,
0x0a, 0x05, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x12, 0x16, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f,
0x70, 0x62, 0x2e, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a,
0x17, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x47, 0x65, 0x74,
0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3a, 0x0a, 0x05, 0x4b, 0x76,
0x50, 0x75, 0x74, 0x12, 0x16, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b,
0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x17, 0x2e, 0x66, 0x69,
0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x88, 0x01, 0x0a, 0x1f, 0x43, 0x61, 0x63, 0x68, 0x65,
0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x6f, 0x4c, 0x6f,
0x63, 0x61, 0x6c, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x12, 0x30, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74,
0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x6f, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x43, 0x6c,
0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
0x37, 0x0a, 0x04, 0x4c, 0x6f, 0x63, 0x6b, 0x12, 0x15, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f,
0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16,
0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x6b, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x3d, 0x0a, 0x06, 0x55, 0x6e, 0x6c, 0x6f,
0x63, 0x6b, 0x12, 0x17, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x6e,
0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x66, 0x69,
0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x6e, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73,
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0d, 0x54, 0x72, 0x61, 0x6e, 0x73,
0x66, 0x65, 0x72, 0x4c, 0x6f, 0x63, 0x6b, 0x73, 0x12, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72,
0x5f, 0x70, 0x62, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x6f, 0x63, 0x6b,
0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72,
0x5f, 0x70, 0x62, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x6f, 0x63, 0x6b,
0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x4f, 0x0a, 0x10, 0x73,
0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2e, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x42,
0x0a, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2f, 0x67, 0x69, 0x74,
0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66,
0x73, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64,
0x2f, 0x70, 0x62, 0x2f, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x33,
0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x52, 0x65, 0x6d,
0x6f, 0x74, 0x65, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x54, 0x6f, 0x4c, 0x6f, 0x63, 0x61, 0x6c,
0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
0x00, 0x12, 0x42, 0x0a, 0x0f, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x64,
0x4c, 0x6f, 0x63, 0x6b, 0x12, 0x15, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e,
0x4c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x66, 0x69,
0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x48, 0x0a, 0x11, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62,
0x75, 0x74, 0x65, 0x64, 0x55, 0x6e, 0x6c, 0x6f, 0x63, 0x6b, 0x12, 0x17, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55, 0x6e, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x1a, 0x18, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x55,
0x6e, 0x6c, 0x6f, 0x63, 0x6b, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
0x52, 0x0a, 0x0d, 0x46, 0x69, 0x6e, 0x64, 0x4c, 0x6f, 0x63, 0x6b, 0x4f, 0x77, 0x6e, 0x65, 0x72,
0x12, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64,
0x4c, 0x6f, 0x63, 0x6b, 0x4f, 0x77, 0x6e, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x1a, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64,
0x4c, 0x6f, 0x63, 0x6b, 0x4f, 0x77, 0x6e, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
0x65, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0d, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c,
0x6f, 0x63, 0x6b, 0x73, 0x12, 0x1e, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e,
0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x6f, 0x63, 0x6b, 0x73, 0x52, 0x65, 0x71,
0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e,
0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4c, 0x6f, 0x63, 0x6b, 0x73, 0x52, 0x65, 0x73,
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x42, 0x4f, 0x0a, 0x10, 0x73, 0x65, 0x61, 0x77, 0x65,
0x65, 0x64, 0x66, 0x73, 0x2e, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x42, 0x0a, 0x46, 0x69, 0x6c,
0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e,
0x63, 0x6f, 0x6d, 0x2f, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x73, 0x65,
0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2f, 0x77, 0x65, 0x65, 0x64, 0x2f, 0x70, 0x62, 0x2f,
0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@ -4867,7 +5001,7 @@ func file_filer_proto_rawDescGZIP() []byte {
return file_filer_proto_rawDescData
}
var file_filer_proto_msgTypes = make([]protoimpl.MessageInfo, 66)
var file_filer_proto_msgTypes = make([]protoimpl.MessageInfo, 68)
var file_filer_proto_goTypes = []interface{}{
(*LookupDirectoryEntryRequest)(nil), // 0: filer_pb.LookupDirectoryEntryRequest
(*LookupDirectoryEntryResponse)(nil), // 1: filer_pb.LookupDirectoryEntryResponse
@ -4928,20 +5062,22 @@ var file_filer_proto_goTypes = []interface{}{
(*LockResponse)(nil), // 56: filer_pb.LockResponse
(*UnlockRequest)(nil), // 57: filer_pb.UnlockRequest
(*UnlockResponse)(nil), // 58: filer_pb.UnlockResponse
(*Lock)(nil), // 59: filer_pb.Lock
(*TransferLocksRequest)(nil), // 60: filer_pb.TransferLocksRequest
(*TransferLocksResponse)(nil), // 61: filer_pb.TransferLocksResponse
nil, // 62: filer_pb.Entry.ExtendedEntry
nil, // 63: filer_pb.LookupVolumeResponse.LocationsMapEntry
(*LocateBrokerResponse_Resource)(nil), // 64: filer_pb.LocateBrokerResponse.Resource
(*FilerConf_PathConf)(nil), // 65: filer_pb.FilerConf.PathConf
(*FindLockOwnerRequest)(nil), // 59: filer_pb.FindLockOwnerRequest
(*FindLockOwnerResponse)(nil), // 60: filer_pb.FindLockOwnerResponse
(*Lock)(nil), // 61: filer_pb.Lock
(*TransferLocksRequest)(nil), // 62: filer_pb.TransferLocksRequest
(*TransferLocksResponse)(nil), // 63: filer_pb.TransferLocksResponse
nil, // 64: filer_pb.Entry.ExtendedEntry
nil, // 65: filer_pb.LookupVolumeResponse.LocationsMapEntry
(*LocateBrokerResponse_Resource)(nil), // 66: filer_pb.LocateBrokerResponse.Resource
(*FilerConf_PathConf)(nil), // 67: filer_pb.FilerConf.PathConf
}
var file_filer_proto_depIdxs = []int32{
5, // 0: filer_pb.LookupDirectoryEntryResponse.entry:type_name -> filer_pb.Entry
5, // 1: filer_pb.ListEntriesResponse.entry:type_name -> filer_pb.Entry
8, // 2: filer_pb.Entry.chunks:type_name -> filer_pb.FileChunk
11, // 3: filer_pb.Entry.attributes:type_name -> filer_pb.FuseAttributes
62, // 4: filer_pb.Entry.extended:type_name -> filer_pb.Entry.ExtendedEntry
64, // 4: filer_pb.Entry.extended:type_name -> filer_pb.Entry.ExtendedEntry
4, // 5: filer_pb.Entry.remote_entry:type_name -> filer_pb.RemoteEntry
5, // 6: filer_pb.FullEntry.entry:type_name -> filer_pb.Entry
5, // 7: filer_pb.EventNotification.old_entry:type_name -> filer_pb.Entry
@ -4955,13 +5091,13 @@ var file_filer_proto_depIdxs = []int32{
7, // 15: filer_pb.StreamRenameEntryResponse.event_notification:type_name -> filer_pb.EventNotification
28, // 16: filer_pb.AssignVolumeResponse.location:type_name -> filer_pb.Location
28, // 17: filer_pb.Locations.locations:type_name -> filer_pb.Location
63, // 18: filer_pb.LookupVolumeResponse.locations_map:type_name -> filer_pb.LookupVolumeResponse.LocationsMapEntry
65, // 18: filer_pb.LookupVolumeResponse.locations_map:type_name -> filer_pb.LookupVolumeResponse.LocationsMapEntry
30, // 19: filer_pb.CollectionListResponse.collections:type_name -> filer_pb.Collection
7, // 20: filer_pb.SubscribeMetadataResponse.event_notification:type_name -> filer_pb.EventNotification
64, // 21: filer_pb.LocateBrokerResponse.resources:type_name -> filer_pb.LocateBrokerResponse.Resource
65, // 22: filer_pb.FilerConf.locations:type_name -> filer_pb.FilerConf.PathConf
66, // 21: filer_pb.LocateBrokerResponse.resources:type_name -> filer_pb.LocateBrokerResponse.Resource
67, // 22: filer_pb.FilerConf.locations:type_name -> filer_pb.FilerConf.PathConf
5, // 23: filer_pb.CacheRemoteObjectToLocalClusterResponse.entry:type_name -> filer_pb.Entry
59, // 24: filer_pb.TransferLocksRequest.locks:type_name -> filer_pb.Lock
61, // 24: filer_pb.TransferLocksRequest.locks:type_name -> filer_pb.Lock
27, // 25: filer_pb.LookupVolumeResponse.LocationsMapEntry.value:type_name -> filer_pb.Locations
0, // 26: filer_pb.SeaweedFiler.LookupDirectoryEntry:input_type -> filer_pb.LookupDirectoryEntryRequest
2, // 27: filer_pb.SeaweedFiler.ListEntries:input_type -> filer_pb.ListEntriesRequest
@ -4983,34 +5119,36 @@ var file_filer_proto_depIdxs = []int32{
48, // 43: filer_pb.SeaweedFiler.KvGet:input_type -> filer_pb.KvGetRequest
50, // 44: filer_pb.SeaweedFiler.KvPut:input_type -> filer_pb.KvPutRequest
53, // 45: filer_pb.SeaweedFiler.CacheRemoteObjectToLocalCluster:input_type -> filer_pb.CacheRemoteObjectToLocalClusterRequest
55, // 46: filer_pb.SeaweedFiler.Lock:input_type -> filer_pb.LockRequest
57, // 47: filer_pb.SeaweedFiler.Unlock:input_type -> filer_pb.UnlockRequest
60, // 48: filer_pb.SeaweedFiler.TransferLocks:input_type -> filer_pb.TransferLocksRequest
1, // 49: filer_pb.SeaweedFiler.LookupDirectoryEntry:output_type -> filer_pb.LookupDirectoryEntryResponse
3, // 50: filer_pb.SeaweedFiler.ListEntries:output_type -> filer_pb.ListEntriesResponse
13, // 51: filer_pb.SeaweedFiler.CreateEntry:output_type -> filer_pb.CreateEntryResponse
15, // 52: filer_pb.SeaweedFiler.UpdateEntry:output_type -> filer_pb.UpdateEntryResponse
17, // 53: filer_pb.SeaweedFiler.AppendToEntry:output_type -> filer_pb.AppendToEntryResponse
19, // 54: filer_pb.SeaweedFiler.DeleteEntry:output_type -> filer_pb.DeleteEntryResponse
21, // 55: filer_pb.SeaweedFiler.AtomicRenameEntry:output_type -> filer_pb.AtomicRenameEntryResponse
23, // 56: filer_pb.SeaweedFiler.StreamRenameEntry:output_type -> filer_pb.StreamRenameEntryResponse
25, // 57: filer_pb.SeaweedFiler.AssignVolume:output_type -> filer_pb.AssignVolumeResponse
29, // 58: filer_pb.SeaweedFiler.LookupVolume:output_type -> filer_pb.LookupVolumeResponse
32, // 59: filer_pb.SeaweedFiler.CollectionList:output_type -> filer_pb.CollectionListResponse
34, // 60: filer_pb.SeaweedFiler.DeleteCollection:output_type -> filer_pb.DeleteCollectionResponse
36, // 61: filer_pb.SeaweedFiler.Statistics:output_type -> filer_pb.StatisticsResponse
38, // 62: filer_pb.SeaweedFiler.Ping:output_type -> filer_pb.PingResponse
40, // 63: filer_pb.SeaweedFiler.GetFilerConfiguration:output_type -> filer_pb.GetFilerConfigurationResponse
42, // 64: filer_pb.SeaweedFiler.SubscribeMetadata:output_type -> filer_pb.SubscribeMetadataResponse
42, // 65: filer_pb.SeaweedFiler.SubscribeLocalMetadata:output_type -> filer_pb.SubscribeMetadataResponse
49, // 66: filer_pb.SeaweedFiler.KvGet:output_type -> filer_pb.KvGetResponse
51, // 67: filer_pb.SeaweedFiler.KvPut:output_type -> filer_pb.KvPutResponse
54, // 68: filer_pb.SeaweedFiler.CacheRemoteObjectToLocalCluster:output_type -> filer_pb.CacheRemoteObjectToLocalClusterResponse
56, // 69: filer_pb.SeaweedFiler.Lock:output_type -> filer_pb.LockResponse
58, // 70: filer_pb.SeaweedFiler.Unlock:output_type -> filer_pb.UnlockResponse
61, // 71: filer_pb.SeaweedFiler.TransferLocks:output_type -> filer_pb.TransferLocksResponse
49, // [49:72] is the sub-list for method output_type
26, // [26:49] is the sub-list for method input_type
55, // 46: filer_pb.SeaweedFiler.DistributedLock:input_type -> filer_pb.LockRequest
57, // 47: filer_pb.SeaweedFiler.DistributedUnlock:input_type -> filer_pb.UnlockRequest
59, // 48: filer_pb.SeaweedFiler.FindLockOwner:input_type -> filer_pb.FindLockOwnerRequest
62, // 49: filer_pb.SeaweedFiler.TransferLocks:input_type -> filer_pb.TransferLocksRequest
1, // 50: filer_pb.SeaweedFiler.LookupDirectoryEntry:output_type -> filer_pb.LookupDirectoryEntryResponse
3, // 51: filer_pb.SeaweedFiler.ListEntries:output_type -> filer_pb.ListEntriesResponse
13, // 52: filer_pb.SeaweedFiler.CreateEntry:output_type -> filer_pb.CreateEntryResponse
15, // 53: filer_pb.SeaweedFiler.UpdateEntry:output_type -> filer_pb.UpdateEntryResponse
17, // 54: filer_pb.SeaweedFiler.AppendToEntry:output_type -> filer_pb.AppendToEntryResponse
19, // 55: filer_pb.SeaweedFiler.DeleteEntry:output_type -> filer_pb.DeleteEntryResponse
21, // 56: filer_pb.SeaweedFiler.AtomicRenameEntry:output_type -> filer_pb.AtomicRenameEntryResponse
23, // 57: filer_pb.SeaweedFiler.StreamRenameEntry:output_type -> filer_pb.StreamRenameEntryResponse
25, // 58: filer_pb.SeaweedFiler.AssignVolume:output_type -> filer_pb.AssignVolumeResponse
29, // 59: filer_pb.SeaweedFiler.LookupVolume:output_type -> filer_pb.LookupVolumeResponse
32, // 60: filer_pb.SeaweedFiler.CollectionList:output_type -> filer_pb.CollectionListResponse
34, // 61: filer_pb.SeaweedFiler.DeleteCollection:output_type -> filer_pb.DeleteCollectionResponse
36, // 62: filer_pb.SeaweedFiler.Statistics:output_type -> filer_pb.StatisticsResponse
38, // 63: filer_pb.SeaweedFiler.Ping:output_type -> filer_pb.PingResponse
40, // 64: filer_pb.SeaweedFiler.GetFilerConfiguration:output_type -> filer_pb.GetFilerConfigurationResponse
42, // 65: filer_pb.SeaweedFiler.SubscribeMetadata:output_type -> filer_pb.SubscribeMetadataResponse
42, // 66: filer_pb.SeaweedFiler.SubscribeLocalMetadata:output_type -> filer_pb.SubscribeMetadataResponse
49, // 67: filer_pb.SeaweedFiler.KvGet:output_type -> filer_pb.KvGetResponse
51, // 68: filer_pb.SeaweedFiler.KvPut:output_type -> filer_pb.KvPutResponse
54, // 69: filer_pb.SeaweedFiler.CacheRemoteObjectToLocalCluster:output_type -> filer_pb.CacheRemoteObjectToLocalClusterResponse
56, // 70: filer_pb.SeaweedFiler.DistributedLock:output_type -> filer_pb.LockResponse
58, // 71: filer_pb.SeaweedFiler.DistributedUnlock:output_type -> filer_pb.UnlockResponse
60, // 72: filer_pb.SeaweedFiler.FindLockOwner:output_type -> filer_pb.FindLockOwnerResponse
63, // 73: filer_pb.SeaweedFiler.TransferLocks:output_type -> filer_pb.TransferLocksResponse
50, // [50:74] is the sub-list for method output_type
26, // [26:50] is the sub-list for method input_type
26, // [26:26] is the sub-list for extension type_name
26, // [26:26] is the sub-list for extension extendee
0, // [0:26] is the sub-list for field type_name
@ -5731,7 +5869,7 @@ func file_filer_proto_init() {
}
}
file_filer_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*Lock); i {
switch v := v.(*FindLockOwnerRequest); i {
case 0:
return &v.state
case 1:
@ -5743,7 +5881,7 @@ func file_filer_proto_init() {
}
}
file_filer_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*TransferLocksRequest); i {
switch v := v.(*FindLockOwnerResponse); i {
case 0:
return &v.state
case 1:
@ -5755,6 +5893,30 @@ func file_filer_proto_init() {
}
}
file_filer_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*Lock); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_filer_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*TransferLocksRequest); i {
case 0:
return &v.state
case 1:
return &v.sizeCache
case 2:
return &v.unknownFields
default:
return nil
}
}
file_filer_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*TransferLocksResponse); i {
case 0:
return &v.state
@ -5766,7 +5928,7 @@ func file_filer_proto_init() {
return nil
}
}
file_filer_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} {
file_filer_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*LocateBrokerResponse_Resource); i {
case 0:
return &v.state
@ -5778,7 +5940,7 @@ func file_filer_proto_init() {
return nil
}
}
file_filer_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} {
file_filer_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*FilerConf_PathConf); i {
case 0:
return &v.state
@ -5797,7 +5959,7 @@ func file_filer_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_filer_proto_rawDesc,
NumEnums: 0,
NumMessages: 66,
NumMessages: 68,
NumExtensions: 0,
NumServices: 1,
},

View File

@ -42,8 +42,10 @@ type SeaweedFilerClient interface {
KvGet(ctx context.Context, in *KvGetRequest, opts ...grpc.CallOption) (*KvGetResponse, error)
KvPut(ctx context.Context, in *KvPutRequest, opts ...grpc.CallOption) (*KvPutResponse, error)
CacheRemoteObjectToLocalCluster(ctx context.Context, in *CacheRemoteObjectToLocalClusterRequest, opts ...grpc.CallOption) (*CacheRemoteObjectToLocalClusterResponse, error)
Lock(ctx context.Context, in *LockRequest, opts ...grpc.CallOption) (*LockResponse, error)
Unlock(ctx context.Context, in *UnlockRequest, opts ...grpc.CallOption) (*UnlockResponse, error)
DistributedLock(ctx context.Context, in *LockRequest, opts ...grpc.CallOption) (*LockResponse, error)
DistributedUnlock(ctx context.Context, in *UnlockRequest, opts ...grpc.CallOption) (*UnlockResponse, error)
FindLockOwner(ctx context.Context, in *FindLockOwnerRequest, opts ...grpc.CallOption) (*FindLockOwnerResponse, error)
// distributed lock management internal use only
TransferLocks(ctx context.Context, in *TransferLocksRequest, opts ...grpc.CallOption) (*TransferLocksResponse, error)
}
@ -327,18 +329,27 @@ func (c *seaweedFilerClient) CacheRemoteObjectToLocalCluster(ctx context.Context
return out, nil
}
func (c *seaweedFilerClient) Lock(ctx context.Context, in *LockRequest, opts ...grpc.CallOption) (*LockResponse, error) {
func (c *seaweedFilerClient) DistributedLock(ctx context.Context, in *LockRequest, opts ...grpc.CallOption) (*LockResponse, error) {
out := new(LockResponse)
err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/Lock", in, out, opts...)
err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/DistributedLock", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *seaweedFilerClient) Unlock(ctx context.Context, in *UnlockRequest, opts ...grpc.CallOption) (*UnlockResponse, error) {
func (c *seaweedFilerClient) DistributedUnlock(ctx context.Context, in *UnlockRequest, opts ...grpc.CallOption) (*UnlockResponse, error) {
out := new(UnlockResponse)
err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/Unlock", in, out, opts...)
err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/DistributedUnlock", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *seaweedFilerClient) FindLockOwner(ctx context.Context, in *FindLockOwnerRequest, opts ...grpc.CallOption) (*FindLockOwnerResponse, error) {
out := new(FindLockOwnerResponse)
err := c.cc.Invoke(ctx, "/filer_pb.SeaweedFiler/FindLockOwner", in, out, opts...)
if err != nil {
return nil, err
}
@ -378,8 +389,10 @@ type SeaweedFilerServer interface {
KvGet(context.Context, *KvGetRequest) (*KvGetResponse, error)
KvPut(context.Context, *KvPutRequest) (*KvPutResponse, error)
CacheRemoteObjectToLocalCluster(context.Context, *CacheRemoteObjectToLocalClusterRequest) (*CacheRemoteObjectToLocalClusterResponse, error)
Lock(context.Context, *LockRequest) (*LockResponse, error)
Unlock(context.Context, *UnlockRequest) (*UnlockResponse, error)
DistributedLock(context.Context, *LockRequest) (*LockResponse, error)
DistributedUnlock(context.Context, *UnlockRequest) (*UnlockResponse, error)
FindLockOwner(context.Context, *FindLockOwnerRequest) (*FindLockOwnerResponse, error)
// distributed lock management internal use only
TransferLocks(context.Context, *TransferLocksRequest) (*TransferLocksResponse, error)
mustEmbedUnimplementedSeaweedFilerServer()
}
@ -448,11 +461,14 @@ func (UnimplementedSeaweedFilerServer) KvPut(context.Context, *KvPutRequest) (*K
func (UnimplementedSeaweedFilerServer) CacheRemoteObjectToLocalCluster(context.Context, *CacheRemoteObjectToLocalClusterRequest) (*CacheRemoteObjectToLocalClusterResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method CacheRemoteObjectToLocalCluster not implemented")
}
func (UnimplementedSeaweedFilerServer) Lock(context.Context, *LockRequest) (*LockResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method Lock not implemented")
func (UnimplementedSeaweedFilerServer) DistributedLock(context.Context, *LockRequest) (*LockResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method DistributedLock not implemented")
}
func (UnimplementedSeaweedFilerServer) Unlock(context.Context, *UnlockRequest) (*UnlockResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method Unlock not implemented")
func (UnimplementedSeaweedFilerServer) DistributedUnlock(context.Context, *UnlockRequest) (*UnlockResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method DistributedUnlock not implemented")
}
func (UnimplementedSeaweedFilerServer) FindLockOwner(context.Context, *FindLockOwnerRequest) (*FindLockOwnerResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method FindLockOwner not implemented")
}
func (UnimplementedSeaweedFilerServer) TransferLocks(context.Context, *TransferLocksRequest) (*TransferLocksResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method TransferLocks not implemented")
@ -842,38 +858,56 @@ func _SeaweedFiler_CacheRemoteObjectToLocalCluster_Handler(srv interface{}, ctx
return interceptor(ctx, in, info, handler)
}
func _SeaweedFiler_Lock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
func _SeaweedFiler_DistributedLock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(LockRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(SeaweedFilerServer).Lock(ctx, in)
return srv.(SeaweedFilerServer).DistributedLock(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/filer_pb.SeaweedFiler/Lock",
FullMethod: "/filer_pb.SeaweedFiler/DistributedLock",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(SeaweedFilerServer).Lock(ctx, req.(*LockRequest))
return srv.(SeaweedFilerServer).DistributedLock(ctx, req.(*LockRequest))
}
return interceptor(ctx, in, info, handler)
}
func _SeaweedFiler_Unlock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
func _SeaweedFiler_DistributedUnlock_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(UnlockRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(SeaweedFilerServer).Unlock(ctx, in)
return srv.(SeaweedFilerServer).DistributedUnlock(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/filer_pb.SeaweedFiler/Unlock",
FullMethod: "/filer_pb.SeaweedFiler/DistributedUnlock",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(SeaweedFilerServer).Unlock(ctx, req.(*UnlockRequest))
return srv.(SeaweedFilerServer).DistributedUnlock(ctx, req.(*UnlockRequest))
}
return interceptor(ctx, in, info, handler)
}
func _SeaweedFiler_FindLockOwner_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(FindLockOwnerRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(SeaweedFilerServer).FindLockOwner(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/filer_pb.SeaweedFiler/FindLockOwner",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(SeaweedFilerServer).FindLockOwner(ctx, req.(*FindLockOwnerRequest))
}
return interceptor(ctx, in, info, handler)
}
@ -968,12 +1002,16 @@ var SeaweedFiler_ServiceDesc = grpc.ServiceDesc{
Handler: _SeaweedFiler_CacheRemoteObjectToLocalCluster_Handler,
},
{
MethodName: "Lock",
Handler: _SeaweedFiler_Lock_Handler,
MethodName: "DistributedLock",
Handler: _SeaweedFiler_DistributedLock_Handler,
},
{
MethodName: "Unlock",
Handler: _SeaweedFiler_Unlock_Handler,
MethodName: "DistributedUnlock",
Handler: _SeaweedFiler_DistributedUnlock_Handler,
},
{
MethodName: "FindLockOwner",
Handler: _SeaweedFiler_FindLockOwner_Handler,
},
{
MethodName: "TransferLocks",

View File

@ -225,14 +225,6 @@ func WithVolumeServerClient(streamingMode bool, volumeServer ServerAddress, grpc
}
func WithBrokerClient(streamingMode bool, broker ServerAddress, grpcDialOption grpc.DialOption, fn func(client mq_pb.SeaweedMessagingClient) error) error {
return WithGrpcClient(streamingMode, 0, func(grpcConnection *grpc.ClientConn) error {
client := mq_pb.NewSeaweedMessagingClient(grpcConnection)
return fn(client)
}, broker.ToGrpcAddress(), false, grpcDialOption)
}
func WithOneOfGrpcMasterClients(streamingMode bool, masterGrpcAddresses map[string]ServerAddress, grpcDialOption grpc.DialOption, fn func(client master_pb.SeaweedClient) error) (err error) {
for _, masterGrpcAddress := range masterGrpcAddresses {

View File

@ -20,8 +20,11 @@ service SeaweedMessaging {
rpc CheckBrokerLoad (CheckBrokerLoadRequest) returns (CheckBrokerLoadResponse) {
}
// control plane for balancer
rpc ConnectToBalancer (stream ConnectToBalancerRequest) returns (stream ConnectToBalancerResponse) {
}
// control plane for topic partitions
rpc FindTopicBrokers (FindTopicBrokersRequest) returns (FindTopicBrokersResponse) {
rpc LookupTopicBrokers (LookupTopicBrokersRequest) returns (LookupTopicBrokersResponse) {
}
// a pub client will call this to get the topic partitions assignment
rpc RequestTopicPartitions (RequestTopicPartitionsRequest) returns (RequestTopicPartitionsResponse) {
@ -34,6 +37,8 @@ service SeaweedMessaging {
// data plane
rpc Publish (stream PublishRequest) returns (stream PublishResponse) {
}
rpc Subscribe (SubscribeRequest) returns (stream SubscribeResponse) {
}
}
//////////////////////////////////////////////////
@ -98,23 +103,36 @@ message CheckBrokerLoadResponse {
}
message FindTopicBrokersRequest {
//////////////////////////////////////////////////
message BrokerStats {
int32 topic_partition_count = 1;
int32 consumer_count = 2;
int32 cpu_usage_percent = 3;
}
message ConnectToBalancerRequest {
message InitMessage {
string broker = 1;
}
oneof message {
InitMessage init = 1;
BrokerStats stats = 2;
}
}
message ConnectToBalancerResponse {
}
//////////////////////////////////////////////////
message LookupTopicBrokersRequest {
Topic topic = 1;
bool is_for_publish = 2;
}
message FindTopicBrokersResponse {
message LookupTopicBrokersResponse {
Topic topic = 1;
TopicPartitionsAssignment topic_partitions_assignment = 2;
repeated BrokerPartitionAssignment broker_partition_assignments = 2;
}
message BrokerPartitionsAssignment {
int32 partition_start = 1;
int32 partition_stop = 2;
string leader_broker = 3;
repeated string follower_brokers = 4;
}
message TopicPartitionsAssignment {
int32 partition_count = 1; // over-sharded partitions, usually 1024
repeated BrokerPartitionsAssignment broker_partitions = 2;
message BrokerPartitionAssignment {
Partition partition = 1;
string leader_broker = 2;
repeated string follower_brokers = 3;
}
message RequestTopicPartitionsRequest {
@ -122,12 +140,12 @@ message RequestTopicPartitionsRequest {
int32 partition_count = 2;
}
message RequestTopicPartitionsResponse {
TopicPartitionsAssignment topic_partitions_assignment = 1;
repeated BrokerPartitionAssignment broker_partition_assignments = 1;
}
message AssignTopicPartitionsRequest {
Topic topic = 1;
TopicPartitionsAssignment topic_partitions_assignment = 2;
repeated BrokerPartitionAssignment broker_partition_assignments = 2;
bool is_leader = 3;
}
message AssignTopicPartitionsResponse {
@ -136,21 +154,23 @@ message AssignTopicPartitionsResponse {
message CheckTopicPartitionsStatusRequest {
string namespace = 1;
string topic = 2;
BrokerPartitionsAssignment broker_partitions_assignment = 3;
BrokerPartitionAssignment broker_partition_assignment = 3;
bool should_cancel_if_not_match = 4;
}
message CheckTopicPartitionsStatusResponse {
TopicPartitionsAssignment topic_partitions_assignment = 1;
repeated BrokerPartitionAssignment broker_partition_assignments = 1;
}
//////////////////////////////////////////////////
message DataMessage {
bytes key = 1;
bytes value = 2;
}
message PublishRequest {
message InitMessage {
Segment segment = 1;
}
message DataMessage {
bytes key = 1;
bytes value = 2;
Topic topic = 1;
Partition partition = 2;
int32 ack_interval = 3;
}
oneof message {
InitMessage init = 1;
@ -161,5 +181,33 @@ message PublishRequest {
message PublishResponse {
int64 ack_sequence = 1;
string error = 2;
bool is_closed = 3;
string redirect_to_broker = 3;
}
message SubscribeRequest {
message Consumer {
string consumer_group = 1;
string consumer_id = 2;
string client_id = 3;
}
message Cursor {
Topic topic = 1;
Partition partition = 2;
oneof offset {
int64 start_offset = 3;
int64 start_timestamp_ns = 4;
}
string filter = 5;
}
Consumer consumer = 1;
Cursor cursor = 2;
}
message SubscribeResponse {
message CtrlMessage {
string error = 1;
string redirect_to_broker = 2;
}
oneof message {
CtrlMessage ctrl = 1;
DataMessage data = 2;
}
}

File diff suppressed because it is too large Load Diff

View File

@ -27,14 +27,17 @@ type SeaweedMessagingClient interface {
AssignSegmentBrokers(ctx context.Context, in *AssignSegmentBrokersRequest, opts ...grpc.CallOption) (*AssignSegmentBrokersResponse, error)
CheckSegmentStatus(ctx context.Context, in *CheckSegmentStatusRequest, opts ...grpc.CallOption) (*CheckSegmentStatusResponse, error)
CheckBrokerLoad(ctx context.Context, in *CheckBrokerLoadRequest, opts ...grpc.CallOption) (*CheckBrokerLoadResponse, error)
// control plane for balancer
ConnectToBalancer(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_ConnectToBalancerClient, error)
// control plane for topic partitions
FindTopicBrokers(ctx context.Context, in *FindTopicBrokersRequest, opts ...grpc.CallOption) (*FindTopicBrokersResponse, error)
LookupTopicBrokers(ctx context.Context, in *LookupTopicBrokersRequest, opts ...grpc.CallOption) (*LookupTopicBrokersResponse, error)
// a pub client will call this to get the topic partitions assignment
RequestTopicPartitions(ctx context.Context, in *RequestTopicPartitionsRequest, opts ...grpc.CallOption) (*RequestTopicPartitionsResponse, error)
AssignTopicPartitions(ctx context.Context, in *AssignTopicPartitionsRequest, opts ...grpc.CallOption) (*AssignTopicPartitionsResponse, error)
CheckTopicPartitionsStatus(ctx context.Context, in *CheckTopicPartitionsStatusRequest, opts ...grpc.CallOption) (*CheckTopicPartitionsStatusResponse, error)
// data plane
Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error)
Subscribe(ctx context.Context, in *SubscribeRequest, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error)
}
type seaweedMessagingClient struct {
@ -81,9 +84,40 @@ func (c *seaweedMessagingClient) CheckBrokerLoad(ctx context.Context, in *CheckB
return out, nil
}
func (c *seaweedMessagingClient) FindTopicBrokers(ctx context.Context, in *FindTopicBrokersRequest, opts ...grpc.CallOption) (*FindTopicBrokersResponse, error) {
out := new(FindTopicBrokersResponse)
err := c.cc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/FindTopicBrokers", in, out, opts...)
func (c *seaweedMessagingClient) ConnectToBalancer(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_ConnectToBalancerClient, error) {
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[0], "/messaging_pb.SeaweedMessaging/ConnectToBalancer", opts...)
if err != nil {
return nil, err
}
x := &seaweedMessagingConnectToBalancerClient{stream}
return x, nil
}
type SeaweedMessaging_ConnectToBalancerClient interface {
Send(*ConnectToBalancerRequest) error
Recv() (*ConnectToBalancerResponse, error)
grpc.ClientStream
}
type seaweedMessagingConnectToBalancerClient struct {
grpc.ClientStream
}
func (x *seaweedMessagingConnectToBalancerClient) Send(m *ConnectToBalancerRequest) error {
return x.ClientStream.SendMsg(m)
}
func (x *seaweedMessagingConnectToBalancerClient) Recv() (*ConnectToBalancerResponse, error) {
m := new(ConnectToBalancerResponse)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func (c *seaweedMessagingClient) LookupTopicBrokers(ctx context.Context, in *LookupTopicBrokersRequest, opts ...grpc.CallOption) (*LookupTopicBrokersResponse, error) {
out := new(LookupTopicBrokersResponse)
err := c.cc.Invoke(ctx, "/messaging_pb.SeaweedMessaging/LookupTopicBrokers", in, out, opts...)
if err != nil {
return nil, err
}
@ -118,7 +152,7 @@ func (c *seaweedMessagingClient) CheckTopicPartitionsStatus(ctx context.Context,
}
func (c *seaweedMessagingClient) Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error) {
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[0], "/messaging_pb.SeaweedMessaging/Publish", opts...)
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[1], "/messaging_pb.SeaweedMessaging/Publish", opts...)
if err != nil {
return nil, err
}
@ -148,6 +182,38 @@ func (x *seaweedMessagingPublishClient) Recv() (*PublishResponse, error) {
return m, nil
}
func (c *seaweedMessagingClient) Subscribe(ctx context.Context, in *SubscribeRequest, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error) {
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[2], "/messaging_pb.SeaweedMessaging/Subscribe", opts...)
if err != nil {
return nil, err
}
x := &seaweedMessagingSubscribeClient{stream}
if err := x.ClientStream.SendMsg(in); err != nil {
return nil, err
}
if err := x.ClientStream.CloseSend(); err != nil {
return nil, err
}
return x, nil
}
type SeaweedMessaging_SubscribeClient interface {
Recv() (*SubscribeResponse, error)
grpc.ClientStream
}
type seaweedMessagingSubscribeClient struct {
grpc.ClientStream
}
func (x *seaweedMessagingSubscribeClient) Recv() (*SubscribeResponse, error) {
m := new(SubscribeResponse)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
// SeaweedMessagingServer is the server API for SeaweedMessaging service.
// All implementations must embed UnimplementedSeaweedMessagingServer
// for forward compatibility
@ -157,14 +223,17 @@ type SeaweedMessagingServer interface {
AssignSegmentBrokers(context.Context, *AssignSegmentBrokersRequest) (*AssignSegmentBrokersResponse, error)
CheckSegmentStatus(context.Context, *CheckSegmentStatusRequest) (*CheckSegmentStatusResponse, error)
CheckBrokerLoad(context.Context, *CheckBrokerLoadRequest) (*CheckBrokerLoadResponse, error)
// control plane for balancer
ConnectToBalancer(SeaweedMessaging_ConnectToBalancerServer) error
// control plane for topic partitions
FindTopicBrokers(context.Context, *FindTopicBrokersRequest) (*FindTopicBrokersResponse, error)
LookupTopicBrokers(context.Context, *LookupTopicBrokersRequest) (*LookupTopicBrokersResponse, error)
// a pub client will call this to get the topic partitions assignment
RequestTopicPartitions(context.Context, *RequestTopicPartitionsRequest) (*RequestTopicPartitionsResponse, error)
AssignTopicPartitions(context.Context, *AssignTopicPartitionsRequest) (*AssignTopicPartitionsResponse, error)
CheckTopicPartitionsStatus(context.Context, *CheckTopicPartitionsStatusRequest) (*CheckTopicPartitionsStatusResponse, error)
// data plane
Publish(SeaweedMessaging_PublishServer) error
Subscribe(*SubscribeRequest, SeaweedMessaging_SubscribeServer) error
mustEmbedUnimplementedSeaweedMessagingServer()
}
@ -184,8 +253,11 @@ func (UnimplementedSeaweedMessagingServer) CheckSegmentStatus(context.Context, *
func (UnimplementedSeaweedMessagingServer) CheckBrokerLoad(context.Context, *CheckBrokerLoadRequest) (*CheckBrokerLoadResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method CheckBrokerLoad not implemented")
}
func (UnimplementedSeaweedMessagingServer) FindTopicBrokers(context.Context, *FindTopicBrokersRequest) (*FindTopicBrokersResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method FindTopicBrokers not implemented")
func (UnimplementedSeaweedMessagingServer) ConnectToBalancer(SeaweedMessaging_ConnectToBalancerServer) error {
return status.Errorf(codes.Unimplemented, "method ConnectToBalancer not implemented")
}
func (UnimplementedSeaweedMessagingServer) LookupTopicBrokers(context.Context, *LookupTopicBrokersRequest) (*LookupTopicBrokersResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method LookupTopicBrokers not implemented")
}
func (UnimplementedSeaweedMessagingServer) RequestTopicPartitions(context.Context, *RequestTopicPartitionsRequest) (*RequestTopicPartitionsResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method RequestTopicPartitions not implemented")
@ -199,6 +271,9 @@ func (UnimplementedSeaweedMessagingServer) CheckTopicPartitionsStatus(context.Co
func (UnimplementedSeaweedMessagingServer) Publish(SeaweedMessaging_PublishServer) error {
return status.Errorf(codes.Unimplemented, "method Publish not implemented")
}
func (UnimplementedSeaweedMessagingServer) Subscribe(*SubscribeRequest, SeaweedMessaging_SubscribeServer) error {
return status.Errorf(codes.Unimplemented, "method Subscribe not implemented")
}
func (UnimplementedSeaweedMessagingServer) mustEmbedUnimplementedSeaweedMessagingServer() {}
// UnsafeSeaweedMessagingServer may be embedded to opt out of forward compatibility for this service.
@ -284,20 +359,46 @@ func _SeaweedMessaging_CheckBrokerLoad_Handler(srv interface{}, ctx context.Cont
return interceptor(ctx, in, info, handler)
}
func _SeaweedMessaging_FindTopicBrokers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(FindTopicBrokersRequest)
func _SeaweedMessaging_ConnectToBalancer_Handler(srv interface{}, stream grpc.ServerStream) error {
return srv.(SeaweedMessagingServer).ConnectToBalancer(&seaweedMessagingConnectToBalancerServer{stream})
}
type SeaweedMessaging_ConnectToBalancerServer interface {
Send(*ConnectToBalancerResponse) error
Recv() (*ConnectToBalancerRequest, error)
grpc.ServerStream
}
type seaweedMessagingConnectToBalancerServer struct {
grpc.ServerStream
}
func (x *seaweedMessagingConnectToBalancerServer) Send(m *ConnectToBalancerResponse) error {
return x.ServerStream.SendMsg(m)
}
func (x *seaweedMessagingConnectToBalancerServer) Recv() (*ConnectToBalancerRequest, error) {
m := new(ConnectToBalancerRequest)
if err := x.ServerStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func _SeaweedMessaging_LookupTopicBrokers_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(LookupTopicBrokersRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(SeaweedMessagingServer).FindTopicBrokers(ctx, in)
return srv.(SeaweedMessagingServer).LookupTopicBrokers(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/messaging_pb.SeaweedMessaging/FindTopicBrokers",
FullMethod: "/messaging_pb.SeaweedMessaging/LookupTopicBrokers",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(SeaweedMessagingServer).FindTopicBrokers(ctx, req.(*FindTopicBrokersRequest))
return srv.(SeaweedMessagingServer).LookupTopicBrokers(ctx, req.(*LookupTopicBrokersRequest))
}
return interceptor(ctx, in, info, handler)
}
@ -382,6 +483,27 @@ func (x *seaweedMessagingPublishServer) Recv() (*PublishRequest, error) {
return m, nil
}
func _SeaweedMessaging_Subscribe_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(SubscribeRequest)
if err := stream.RecvMsg(m); err != nil {
return err
}
return srv.(SeaweedMessagingServer).Subscribe(m, &seaweedMessagingSubscribeServer{stream})
}
type SeaweedMessaging_SubscribeServer interface {
Send(*SubscribeResponse) error
grpc.ServerStream
}
type seaweedMessagingSubscribeServer struct {
grpc.ServerStream
}
func (x *seaweedMessagingSubscribeServer) Send(m *SubscribeResponse) error {
return x.ServerStream.SendMsg(m)
}
// SeaweedMessaging_ServiceDesc is the grpc.ServiceDesc for SeaweedMessaging service.
// It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy)
@ -406,8 +528,8 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
Handler: _SeaweedMessaging_CheckBrokerLoad_Handler,
},
{
MethodName: "FindTopicBrokers",
Handler: _SeaweedMessaging_FindTopicBrokers_Handler,
MethodName: "LookupTopicBrokers",
Handler: _SeaweedMessaging_LookupTopicBrokers_Handler,
},
{
MethodName: "RequestTopicPartitions",
@ -423,12 +545,23 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
},
},
Streams: []grpc.StreamDesc{
{
StreamName: "ConnectToBalancer",
Handler: _SeaweedMessaging_ConnectToBalancer_Handler,
ServerStreams: true,
ClientStreams: true,
},
{
StreamName: "Publish",
Handler: _SeaweedMessaging_Publish_Handler,
ServerStreams: true,
ClientStreams: true,
},
{
StreamName: "Subscribe",
Handler: _SeaweedMessaging_Subscribe_Handler,
ServerStreams: true,
},
},
Metadata: "mq.proto",
}

View File

@ -244,9 +244,9 @@ func (fs *FilerServer) AppendToEntry(ctx context.Context, req *filer_pb.AppendTo
glog.V(4).Infof("AppendToEntry %v", req)
fullpath := util.NewFullPath(req.Directory, req.EntryName)
lockClient := cluster.NewLockClient(fs.grpcDialOption)
lock := lockClient.NewLock(fs.option.Host, string(fullpath))
defer lock.Unlock()
lockClient := cluster.NewLockClient(fs.grpcDialOption, fs.option.Host)
lock := lockClient.NewLock(string(fullpath), string(fs.option.Host))
defer lock.StopLock()
var offset int64 = 0
entry, err := fs.filer.FindEntry(ctx, fullpath)

View File

@ -6,24 +6,28 @@ import (
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"time"
)
// Lock is a grpc handler to handle FilerServer's LockRequest
func (fs *FilerServer) Lock(ctx context.Context, req *filer_pb.LockRequest) (resp *filer_pb.LockResponse, err error) {
// DistributedLock is a grpc handler to handle FilerServer's LockRequest
func (fs *FilerServer) DistributedLock(ctx context.Context, req *filer_pb.LockRequest) (resp *filer_pb.LockResponse, err error) {
resp = &filer_pb.LockResponse{}
var movedTo pb.ServerAddress
expiredAtNs := time.Now().Add(time.Duration(req.SecondsToLock) * time.Second).UnixNano()
resp.RenewToken, movedTo, err = fs.filer.Dlm.LockWithTimeout(req.Name, expiredAtNs, req.RenewToken)
if !req.IsMoved && movedTo != "" {
resp.RenewToken, movedTo, err = fs.filer.Dlm.LockWithTimeout(req.Name, expiredAtNs, req.RenewToken, req.Owner)
glog.Infof("lock %s %v %v %v, isMoved=%v %v", req.Name, req.SecondsToLock, req.RenewToken, req.Owner, req.IsMoved, movedTo)
if movedTo != "" && movedTo != fs.option.Host && !req.IsMoved {
err = pb.WithFilerClient(false, 0, movedTo, fs.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
secondResp, err := client.Lock(context.Background(), &filer_pb.LockRequest{
secondResp, err := client.DistributedLock(context.Background(), &filer_pb.LockRequest{
Name: req.Name,
SecondsToLock: req.SecondsToLock,
RenewToken: req.RenewToken,
IsMoved: true,
Owner: req.Owner,
})
if err == nil {
resp.RenewToken = secondResp.RenewToken
@ -45,7 +49,7 @@ func (fs *FilerServer) Lock(ctx context.Context, req *filer_pb.LockRequest) (res
}
// Unlock is a grpc handler to handle FilerServer's UnlockRequest
func (fs *FilerServer) Unlock(ctx context.Context, req *filer_pb.UnlockRequest) (resp *filer_pb.UnlockResponse, err error) {
func (fs *FilerServer) DistributedUnlock(ctx context.Context, req *filer_pb.UnlockRequest) (resp *filer_pb.UnlockResponse, err error) {
resp = &filer_pb.UnlockResponse{}
@ -54,7 +58,7 @@ func (fs *FilerServer) Unlock(ctx context.Context, req *filer_pb.UnlockRequest)
if !req.IsMoved && movedTo != "" {
err = pb.WithFilerClient(false, 0, movedTo, fs.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
secondResp, err := client.Unlock(context.Background(), &filer_pb.UnlockRequest{
secondResp, err := client.DistributedUnlock(context.Background(), &filer_pb.UnlockRequest{
Name: req.Name,
RenewToken: req.RenewToken,
IsMoved: true,
@ -75,11 +79,34 @@ func (fs *FilerServer) Unlock(ctx context.Context, req *filer_pb.UnlockRequest)
}
func (fs *FilerServer) FindLockOwner(ctx context.Context, req *filer_pb.FindLockOwnerRequest) (*filer_pb.FindLockOwnerResponse, error) {
owner, movedTo, err := fs.filer.Dlm.FindLockOwner(req.Name)
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
if !req.IsMoved && movedTo != "" {
err = pb.WithFilerClient(false, 0, movedTo, fs.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
secondResp, err := client.FindLockOwner(context.Background(), &filer_pb.FindLockOwnerRequest{
Name: req.Name,
IsMoved: true,
})
owner = secondResp.Owner
return err
})
if err != nil {
return nil, err
}
}
return &filer_pb.FindLockOwnerResponse{
Owner: owner,
}, nil
}
// TransferLocks is a grpc handler to handle FilerServer's TransferLocksRequest
func (fs *FilerServer) TransferLocks(ctx context.Context, req *filer_pb.TransferLocksRequest) (*filer_pb.TransferLocksResponse, error) {
for _, lock := range req.Locks {
fs.filer.Dlm.InsertLock(lock.Name, lock.ExpiredAtNs, lock.RenewToken)
fs.filer.Dlm.InsertLock(lock.Name, lock.ExpiredAtNs, lock.RenewToken, lock.Owner)
}
return &filer_pb.TransferLocksResponse{}, nil
@ -101,6 +128,7 @@ func (fs *FilerServer) OnDlmChangeSnapshot(snapshot []pb.ServerAddress) {
Name: lock.Key,
RenewToken: lock.Token,
ExpiredAtNs: lock.ExpiredAtNs,
Owner: lock.Owner,
},
},
})

View File

@ -25,7 +25,7 @@ type MasterClient struct {
currentMaster pb.ServerAddress
currentMasterLock sync.RWMutex
masters pb.ServerDiscovery
grpcDialOption grpc.DialOption
grpcDialOption grpc.DialOption
*vidMap
vidMapCacheSize int
@ -244,9 +244,9 @@ func (mc *MasterClient) tryConnectToMaster(master pb.ServerAddress) (nextHintedL
if mc.OnPeerUpdate != nil {
if update.FilerGroup == mc.FilerGroup {
if update.IsAdd {
glog.V(0).Infof("+ %s.%s %s\n", update.FilerGroup, update.NodeType, update.Address)
glog.V(0).Infof("+ %s@%s noticed %s.%s %s\n", mc.clientType, mc.clientHost, update.FilerGroup, update.NodeType, update.Address)
} else {
glog.V(0).Infof("- %s.%s %s\n", update.FilerGroup, update.NodeType, update.Address)
glog.V(0).Infof("- %s@%s noticed %s.%s %s\n", mc.clientType, mc.clientHost, update.FilerGroup, update.NodeType, update.Address)
}
stats.MasterClientConnectCounter.WithLabelValues(stats.OnPeerUpdate).Inc()
mc.OnPeerUpdate(update, time.Now())