Browse Source

Merge branch 'mq-subscribe'

pull/5637/head
chrislu 9 months ago
parent
commit
6a61b54f29
  1. 4
      .gitignore
  2. 2
      docker/Dockerfile.local
  3. 5
      docker/Makefile
  4. 126
      docker/compose/local-brokers-compose.yml
  5. 6
      other/java/client/src/main/proto/filer.proto
  6. 200
      weed/cluster/lock_client.go
  7. 9
      weed/cluster/lock_manager/distributed_lock_manager.go
  8. 180
      weed/cluster/lock_manager/lock_manager.go
  9. 41
      weed/filer/filechunk_manifest.go
  10. 4
      weed/filer/filer.go
  11. 16
      weed/filer/filer_notify.go
  12. 5
      weed/filer/meta_aggregator.go
  13. 3
      weed/filer/reader_cache.go
  14. 2
      weed/filer/stream.go
  15. 72
      weed/mq/broker/broker_connect.go
  16. 16
      weed/mq/broker/broker_grpc_admin.go
  17. 100
      weed/mq/broker/broker_grpc_assign.go
  18. 10
      weed/mq/broker/broker_grpc_balance.go
  19. 105
      weed/mq/broker/broker_grpc_configure.go
  20. 45
      weed/mq/broker/broker_grpc_lookup.go
  21. 73
      weed/mq/broker/broker_grpc_pub.go
  22. 12
      weed/mq/broker/broker_grpc_pub_balancer.go
  23. 96
      weed/mq/broker/broker_grpc_pub_follow.go
  24. 274
      weed/mq/broker/broker_grpc_sub.go
  25. 39
      weed/mq/broker/broker_grpc_sub_coordinator.go
  26. 33
      weed/mq/broker/broker_server.go
  27. 101
      weed/mq/broker/broker_topic_conf_read_write.go
  28. 168
      weed/mq/broker/broker_topic_partition_read_write.go
  29. 82
      weed/mq/broker/broker_write.go
  30. 27
      weed/mq/client/cmd/weed_pub/publisher.go
  31. 36
      weed/mq/client/cmd/weed_sub/subscriber.go
  32. 73
      weed/mq/client/pub_client/connect.go
  33. 84
      weed/mq/client/pub_client/lookup.go
  34. 31
      weed/mq/client/pub_client/publish.go
  35. 56
      weed/mq/client/pub_client/publisher.go
  36. 273
      weed/mq/client/pub_client/scheduler.go
  37. 180
      weed/mq/client/sub_client/connect_to_sub_coordinator.go
  38. 34
      weed/mq/client/sub_client/lookup.go
  39. 85
      weed/mq/client/sub_client/process.go
  40. 41
      weed/mq/client/sub_client/subscribe.go
  41. 28
      weed/mq/client/sub_client/subscriber.go
  42. 132
      weed/mq/pub_balancer/allocate.go
  43. 187
      weed/mq/pub_balancer/allocate_test.go
  44. 37
      weed/mq/pub_balancer/balancer.go
  45. 26
      weed/mq/pub_balancer/broker_stats.go
  46. 24
      weed/mq/pub_balancer/lookup.go
  47. 10
      weed/mq/pub_balancer/partition_list_broker.go
  48. 91
      weed/mq/sub_coordinator/consumer_group.go
  49. 42
      weed/mq/sub_coordinator/coordinator.go
  50. 31
      weed/mq/sub_coordinator/partition_consumer_mapping.go
  51. 20
      weed/mq/sub_coordinator/partition_consumer_mapping_test.go
  52. 14
      weed/mq/sub_coordinator/partition_list.go
  53. 16
      weed/mq/topic/local_manager.go
  54. 104
      weed/mq/topic/local_partition.go
  55. 7
      weed/mq/topic/local_partition_publishers.go
  56. 14
      weed/mq/topic/local_partition_subscribers.go
  57. 1
      weed/mq/topic/local_topic.go
  58. 9
      weed/mq/topic/partition.go
  59. 6
      weed/pb/filer.proto
  60. 548
      weed/pb/filer_pb/filer.pb.go
  61. 2
      weed/pb/filer_pb/filer_grpc.pb.go
  62. 4
      weed/pb/iam_pb/iam.pb.go
  63. 2
      weed/pb/iam_pb/iam_grpc.pb.go
  64. 4
      weed/pb/master_pb/master.pb.go
  65. 2
      weed/pb/master_pb/master_grpc.pb.go
  66. 4
      weed/pb/mount_pb/mount.pb.go
  67. 2
      weed/pb/mount_pb/mount_grpc.pb.go
  68. 82
      weed/pb/mq.proto
  69. 2372
      weed/pb/mq_pb/mq.pb.go
  70. 201
      weed/pb/mq_pb/mq_grpc.pb.go
  71. 4
      weed/pb/remote_pb/remote.pb.go
  72. 4
      weed/pb/s3_pb/s3.pb.go
  73. 2
      weed/pb/s3_pb/s3_grpc.pb.go
  74. 4
      weed/pb/volume_server_pb/volume_server.pb.go
  75. 2
      weed/pb/volume_server_pb/volume_server_grpc.pb.go
  76. 4
      weed/server/filer_grpc_server.go
  77. 21
      weed/server/filer_grpc_server_dlm.go
  78. 21
      weed/server/filer_grpc_server_sub_meta.go
  79. 17
      weed/server/filer_server.go
  80. 1
      weed/shell/command_mq_topic_desc.go
  81. 2
      weed/shell/command_mq_topic_list.go
  82. 137
      weed/util/buffered_queue/buffered_queue.go
  83. 128
      weed/util/buffered_queue/buffered_queue_test.go
  84. 38
      weed/util/http_util.go
  85. 257
      weed/util/log_buffer/log_buffer.go
  86. 2
      weed/util/log_buffer/log_buffer_test.go
  87. 66
      weed/util/log_buffer/log_read.go
  88. 13
      weed/util/log_buffer/sealed_buffer.go

4
.gitignore

@ -91,4 +91,6 @@ weed/mq/client/cmd/weed_pub/weed_pub
docker/weed
# test generated files
weed/*/*.jpg
weed/*/*.jpg
docker/weed_sub
docker/weed_pub

2
docker/Dockerfile.local

@ -1,6 +1,8 @@
FROM alpine AS final
LABEL author="Chris Lu"
COPY ./weed /usr/bin/
COPY ./weed_pub /usr/bin/
COPY ./weed_sub /usr/bin/
RUN mkdir -p /etc/seaweedfs
COPY ./filer.toml /etc/seaweedfs/filer.toml
COPY ./entrypoint.sh /entrypoint.sh

5
docker/Makefile

@ -9,6 +9,8 @@ binary:
export SWCOMMIT=$(shell git rev-parse --short HEAD)
export SWLDFLAGS="-X github.com/seaweedfs/seaweedfs/weed/util.COMMIT=$(SWCOMMIT)"
cd ../weed && CGO_ENABLED=$(cgo) GOOS=linux go build $(options) -tags "$(tags)" -ldflags "-s -w -extldflags -static $(SWLDFLAGS)" && mv weed ../docker/
cd ../weed/mq/client/cmd/weed_pub && CGO_ENABLED=$(cgo) GOOS=linux go build && mv weed_pub ../../../../../docker/
cd ../weed/mq/client/cmd/weed_sub && CGO_ENABLED=$(cgo) GOOS=linux go build && mv weed_sub ../../../../../docker/
binary_race: options = -race
binary_race: cgo = 1
@ -90,6 +92,9 @@ hashicorp_raft: build
s3tests: build s3tests_build
docker compose -f compose/local-s3tests-compose.yml -p seaweedfs up
brokers: build
docker compose -f compose/local-brokers-compose.yml -p seaweedfs up
filer_etcd: build
docker stack deploy -c compose/swarm-etcd.yml fs

126
docker/compose/local-brokers-compose.yml

@ -0,0 +1,126 @@
version: '3.9'
services:
master0:
image: chrislusf/seaweedfs:local
ports:
- 9333:9333
- 19333:19333
command: "-v=1 master -volumeSizeLimitMB 100 -resumeState=false -ip=master0 -port=9333 -peers=master0:9333,master1:9334,master2:9335 -mdir=/tmp"
environment:
WEED_MASTER_VOLUME_GROWTH_COPY_1: 1
WEED_MASTER_VOLUME_GROWTH_COPY_2: 2
WEED_MASTER_VOLUME_GROWTH_COPY_OTHER: 1
master1:
image: chrislusf/seaweedfs:local
ports:
- 9334:9334
- 19334:19334
command: "-v=1 master -volumeSizeLimitMB 100 -resumeState=false -ip=master1 -port=9334 -peers=master0:9333,master1:9334,master2:9335 -mdir=/tmp"
environment:
WEED_MASTER_VOLUME_GROWTH_COPY_1: 1
WEED_MASTER_VOLUME_GROWTH_COPY_2: 2
WEED_MASTER_VOLUME_GROWTH_COPY_OTHER: 1
master2:
image: chrislusf/seaweedfs:local
ports:
- 9335:9335
- 19335:19335
command: "-v=1 master -volumeSizeLimitMB 100 -resumeState=false -ip=master2 -port=9335 -peers=master0:9333,master1:9334,master2:9335 -mdir=/tmp"
environment:
WEED_MASTER_VOLUME_GROWTH_COPY_1: 1
WEED_MASTER_VOLUME_GROWTH_COPY_2: 2
WEED_MASTER_VOLUME_GROWTH_COPY_OTHER: 1
volume1:
image: chrislusf/seaweedfs:local
ports:
- 8080:8080
- 18080:18080
command: 'volume -dataCenter=dc1 -rack=v1 -mserver="master0:9333,master1:9334,master2:9335" -port=8080 -ip=volume1 -publicUrl=localhost:8080 -preStopSeconds=1'
depends_on:
- master0
- master1
- master2
volume2:
image: chrislusf/seaweedfs:local
ports:
- 8082:8082
- 18082:18082
command: 'volume -dataCenter=dc2 -rack=v2 -mserver="master0:9333,master1:9334,master2:9335" -port=8082 -ip=volume2 -publicUrl=localhost:8082 -preStopSeconds=1'
depends_on:
- master0
- master1
- master2
volume3:
image: chrislusf/seaweedfs:local
ports:
- 8083:8083
- 18083:18083
command: 'volume -dataCenter=dc3 -rack=v3 -mserver="master0:9333,master1:9334,master2:9335" -port=8083 -ip=volume3 -publicUrl=localhost:8083 -preStopSeconds=1'
depends_on:
- master0
- master1
- master2
filer1:
image: chrislusf/seaweedfs:local
ports:
- 8888:8888
- 18888:18888
command: 'filer -defaultReplicaPlacement=100 -iam -master="master0:9333,master1:9334,master2:9335" -port=8888 -ip=filer1'
depends_on:
- master0
- master1
- master2
- volume1
- volume2
filer2:
image: chrislusf/seaweedfs:local
ports:
- 8889:8889
- 18889:18889
command: 'filer -defaultReplicaPlacement=100 -iam -master="master0:9333,master1:9334,master2:9335" -port=8889 -ip=filer2'
depends_on:
- master0
- master1
- master2
- volume1
- volume2
broker1:
image: chrislusf/seaweedfs:local
ports:
- 17777:17777
command: 'mq.broker -master="master0:9333,master1:9334,master2:9335" -port=17777 -ip=broker1'
depends_on:
- master0
- master1
- master2
- volume1
- volume2
- filer1
- filer2
broker2:
image: chrislusf/seaweedfs:local
ports:
- 17778:17778
command: 'mq.broker -master="master0:9333,master1:9334,master2:9335" -port=17778 -ip=broker2'
depends_on:
- master0
- master1
- master2
- volume1
- volume2
- filer1
- filer2
broker3:
image: chrislusf/seaweedfs:local
ports:
- 17779:17779
command: 'mq.broker -master="master0:9333,master1:9334,master2:9335" -port=17779 -ip=broker3'
depends_on:
- master0
- master1
- master2
- volume1
- volume2
- filer1
- filer2

6
other/java/client/src/main/proto/filer.proto

@ -364,6 +364,7 @@ message LogEntry {
int64 ts_ns = 1;
int32 partition_key_hash = 2;
bytes data = 3;
bytes key = 4;
}
message KeepConnectedRequest {
@ -452,8 +453,9 @@ message LockRequest {
}
message LockResponse {
string renew_token = 1;
string moved_to = 2;
string error = 3;
string lock_owner = 2;
string lock_host_moved_to = 3;
string error = 4;
}
message UnlockRequest {
string name = 1;

200
weed/cluster/lock_client.go

@ -32,107 +32,104 @@ type LiveLock struct {
key string
renewToken string
expireAtNs int64
filer pb.ServerAddress
hostFiler pb.ServerAddress
cancelCh chan struct{}
grpcDialOption grpc.DialOption
isLocked bool
owner string
self string
lc *LockClient
owner string
}
// NewLock creates a lock with a very long duration
func (lc *LockClient) NewLock(key string, owner string) (lock *LiveLock) {
return lc.doNewLock(key, lock_manager.MaxDuration, owner)
}
// StartLock starts a goroutine to lock the key and returns immediately.
func (lc *LockClient) StartLock(key string, owner string) (lock *LiveLock) {
// NewShortLivedLock creates a lock with a 5-second duration
func (lc *LockClient) NewShortLivedLock(key string, owner string) (lock *LiveLock) {
lock = &LiveLock{
key: key,
filer: lc.seedFiler,
hostFiler: lc.seedFiler,
cancelCh: make(chan struct{}),
expireAtNs: time.Now().Add(lock_manager.MaxDuration).UnixNano(),
expireAtNs: time.Now().Add(5 * time.Second).UnixNano(),
grpcDialOption: lc.grpcDialOption,
owner: owner,
self: owner,
lc: lc,
}
go func() {
util.RetryUntil("create lock:"+key, func() error {
errorMessage, err := lock.doLock(lock_manager.MaxDuration)
if err != nil {
glog.V(0).Infof("create lock %s: %s", key, err)
time.Sleep(time.Second)
return err
}
if errorMessage != "" {
glog.V(4).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 {
time.Sleep(time.Second)
}
return lock.renewToken == ""
})
lc.keepLock(lock)
}()
lock.retryUntilLocked(5 * time.Second)
return
}
func (lc *LockClient) doNewLock(key string, lockDuration time.Duration, owner string) (lock *LiveLock) {
// StartLongLivedLock starts a goroutine to lock the key and returns immediately.
func (lc *LockClient) StartLongLivedLock(key string, owner string, onLockOwnerChange func(newLockOwner string)) (lock *LiveLock) {
lock = &LiveLock{
key: key,
filer: lc.seedFiler,
hostFiler: lc.seedFiler,
cancelCh: make(chan struct{}),
expireAtNs: time.Now().Add(lockDuration).UnixNano(),
expireAtNs: time.Now().Add(lock_manager.LiveLockTTL).UnixNano(),
grpcDialOption: lc.grpcDialOption,
owner: owner,
self: owner,
lc: lc,
}
var needRenewal bool
if lockDuration > lc.maxLockDuration {
lockDuration = lc.maxLockDuration
needRenewal = true
}
util.RetryUntil("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)
go func() {
isLocked := false
lockOwner := ""
for {
if isLocked {
if err := lock.AttemptToLock(lock_manager.LiveLockTTL); err != nil {
glog.V(0).Infof("Lost lock %s: %v", key, err)
isLocked = false
}
} else {
if err := lock.AttemptToLock(lock_manager.LiveLockTTL); err == nil {
isLocked = true
}
}
if lockOwner != lock.LockOwner() && lock.LockOwner() != "" {
glog.V(0).Infof("Lock owner changed from %s to %s", lockOwner, lock.LockOwner())
onLockOwnerChange(lock.LockOwner())
lockOwner = lock.LockOwner()
}
select {
case <-lock.cancelCh:
return
default:
time.Sleep(lock_manager.RenewInterval)
}
}
lock.isLocked = true
return nil
}()
return
}
func (lock *LiveLock) retryUntilLocked(lockDuration time.Duration) {
util.RetryUntil("create lock:"+lock.key, func() error {
return lock.AttemptToLock(lockDuration)
}, func(err error) (shouldContinue bool) {
if err != nil {
glog.Warningf("create lock %s: %s", key, err)
glog.Warningf("create lock %s: %s", lock.key, err)
}
return lock.renewToken == ""
})
if needRenewal {
go lc.keepLock(lock)
}
return
}
func (lock *LiveLock) IsLocked() bool {
return lock.isLocked
func (lock *LiveLock) AttemptToLock(lockDuration time.Duration) 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 (lock *LiveLock) StopLock() error {
close(lock.cancelCh)
func (lock *LiveLock) StopShortLivedLock() error {
if !lock.isLocked {
return nil
}
return pb.WithFilerClient(false, 0, lock.filer, lock.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
defer func() {
lock.isLocked = false
}()
return pb.WithFilerClient(false, 0, lock.hostFiler, lock.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
_, err := client.DistributedUnlock(context.Background(), &filer_pb.UnlockRequest{
Name: lock.key,
RenewToken: lock.renewToken,
@ -141,69 +138,40 @@ func (lock *LiveLock) StopLock() error {
})
}
func (lc *LockClient) keepLock(lock *LiveLock) {
ticker := time.Tick(lc.sleepDuration)
for {
select {
case <-ticker:
// renew the lock if lock.expireAtNs is still greater than now
util.RetryUntil("keep lock:"+lock.key, func() error {
lockDuration := time.Duration(lock.expireAtNs-time.Now().UnixNano()) * time.Nanosecond
if lockDuration > lc.maxLockDuration {
lockDuration = lc.maxLockDuration
}
if lockDuration <= 0 {
return nil
}
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
}, func(err error) (shouldContinue bool) {
if err == nil {
return false
}
glog.Warningf("keep lock %s: %v", lock.key, err)
return true
})
if !lock.isLocked {
return
}
case <-lock.cancelCh:
return
}
}
}
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 {
err = pb.WithFilerClient(false, 0, lock.hostFiler, lock.grpcDialOption, func(client filer_pb.SeaweedFilerClient) error {
resp, err := client.DistributedLock(context.Background(), &filer_pb.LockRequest{
Name: lock.key,
SecondsToLock: int64(lockDuration.Seconds()),
RenewToken: lock.renewToken,
IsMoved: false,
Owner: lock.owner,
Owner: lock.self,
})
if err == nil {
if err == nil && resp != nil {
lock.renewToken = resp.RenewToken
} else {
//this can be retried. Need to remember the last valid renewToken
lock.renewToken = ""
}
if resp != nil {
errorMessage = resp.Error
if resp.MovedTo != "" {
lock.filer = pb.ServerAddress(resp.MovedTo)
lock.lc.seedFiler = lock.filer
if resp.LockHostMovedTo != "" {
lock.hostFiler = pb.ServerAddress(resp.LockHostMovedTo)
lock.lc.seedFiler = lock.hostFiler
}
if resp.LockOwner != "" {
lock.owner = resp.LockOwner
// fmt.Printf("lock %s owner: %s\n", lock.key, lock.owner)
} else {
// fmt.Printf("lock %s has no owner\n", lock.key)
lock.owner = ""
}
}
return err
})
return
}
func (lock *LiveLock) LockOwner() string {
return lock.owner
}

9
weed/cluster/lock_manager/distributed_lock_manager.go

@ -2,11 +2,13 @@ package lock_manager
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"time"
)
const MaxDuration = 1<<63 - 1
const RenewInterval = time.Second * 3
const LiveLockTTL = time.Second * 7
var NoLockServerError = fmt.Errorf("no lock server found")
@ -24,7 +26,7 @@ func NewDistributedLockManager(host pb.ServerAddress) *DistributedLockManager {
}
}
func (dlm *DistributedLockManager) LockWithTimeout(key string, expiredAtNs int64, token string, owner string) (renewToken string, movedTo pb.ServerAddress, err error) {
func (dlm *DistributedLockManager) LockWithTimeout(key string, expiredAtNs int64, token string, owner string) (lockOwner string, renewToken string, movedTo pb.ServerAddress, err error) {
movedTo, err = dlm.findLockOwningFiler(key)
if err != nil {
return
@ -32,7 +34,7 @@ func (dlm *DistributedLockManager) LockWithTimeout(key string, expiredAtNs int64
if movedTo != dlm.Host {
return
}
renewToken, err = dlm.lockManager.Lock(key, expiredAtNs, token, owner)
lockOwner, renewToken, err = dlm.lockManager.Lock(key, expiredAtNs, token, owner)
return
}
@ -53,6 +55,7 @@ func (dlm *DistributedLockManager) FindLockOwner(key string) (owner string, move
return
}
if movedTo != dlm.Host {
glog.V(0).Infof("lock %s not on current %s but on %s", key, dlm.Host, movedTo)
return
}
owner, err = dlm.lockManager.GetLockOwner(key)

180
weed/cluster/lock_manager/lock_manager.go

@ -3,7 +3,8 @@ package lock_manager
import (
"fmt"
"github.com/google/uuid"
"github.com/puzpuzpuz/xsync/v2"
"github.com/seaweedfs/seaweedfs/weed/glog"
"sync"
"time"
)
@ -11,10 +12,12 @@ var LockErrorNonEmptyTokenOnNewLock = fmt.Errorf("lock: non-empty token on a new
var LockErrorNonEmptyTokenOnExpiredLock = fmt.Errorf("lock: non-empty token on an expired lock")
var LockErrorTokenMismatch = fmt.Errorf("lock: token mismatch")
var UnlockErrorTokenMismatch = fmt.Errorf("unlock: token mismatch")
var LockNotFound = fmt.Errorf("lock not found")
// LockManager local lock manager, used by distributed lock manager
type LockManager struct {
locks *xsync.MapOf[string, *Lock]
locks map[string]*Lock
accessLock sync.RWMutex
}
type Lock struct {
Token string
@ -25,126 +28,155 @@ type Lock struct {
func NewLockManager() *LockManager {
t := &LockManager{
locks: xsync.NewMapOf[*Lock](),
locks: make(map[string]*Lock),
}
go t.CleanUp()
return t
}
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() {
// lock is expired, set to a new lock
if token != "" {
err = LockErrorNonEmptyTokenOnExpiredLock
return nil, false
} else {
// new lock
renewToken = uuid.New().String()
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, Owner: owner}, false
func (lm *LockManager) Lock(path string, expiredAtNs int64, token string, owner string) (lockOwner, renewToken string, err error) {
lm.accessLock.Lock()
defer lm.accessLock.Unlock()
glog.V(4).Infof("lock %s %v %v %v", path, time.Unix(0, expiredAtNs), token, owner)
if oldValue, found := lm.locks[path]; found {
if oldValue.ExpiredAtNs > 0 && oldValue.ExpiredAtNs < time.Now().UnixNano() {
// lock is expired, set to a new lock
if token != "" {
glog.V(4).Infof("lock expired key %s non-empty token %v owner %v ts %s", path, token, owner, time.Unix(0, oldValue.ExpiredAtNs))
err = LockErrorNonEmptyTokenOnExpiredLock
return
} else {
err = LockErrorTokenMismatch
return oldValue, false
// new lock
renewToken = uuid.New().String()
glog.V(4).Infof("key %s new token %v owner %v", path, renewToken, owner)
lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}
return
}
}
// not expired
lockOwner = oldValue.Owner
if oldValue.Token == token {
// token matches, renew the lock
renewToken = uuid.New().String()
glog.V(4).Infof("key %s old token %v owner %v => %v owner %v", path, oldValue.Token, oldValue.Owner, renewToken, owner)
lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}
return
} else {
if token == "" {
// new lock
renewToken = uuid.New().String()
return &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}, false
} else {
err = LockErrorNonEmptyTokenOnNewLock
return nil, false
glog.V(4).Infof("key %s locked by %v", path, oldValue.Owner)
err = fmt.Errorf("lock already owned by %v", oldValue.Owner)
return
}
glog.V(4).Infof("key %s expected token %v owner %v received %v from %v", path, oldValue.Token, oldValue.Owner, token, owner)
err = fmt.Errorf("lock: token mismatch")
return
}
})
return
} else {
glog.V(4).Infof("key %s no lock owner %v", path, owner)
if token == "" {
// new lock
glog.V(4).Infof("key %s new token %v owner %v", path, token, owner)
renewToken = uuid.New().String()
lm.locks[path] = &Lock{Token: renewToken, ExpiredAtNs: expiredAtNs, Owner: owner}
return
} else {
glog.V(4).Infof("key %s non-empty token %v owner %v", path, token, owner)
err = LockErrorNonEmptyTokenOnNewLock
return
}
}
}
func (lm *LockManager) Unlock(path string, token string) (isUnlocked bool, err error) {
lm.locks.Compute(path, func(oldValue *Lock, loaded bool) (newValue *Lock, delete bool) {
if oldValue != nil {
now := time.Now()
if oldValue.ExpiredAtNs > 0 && oldValue.ExpiredAtNs < now.UnixNano() {
// lock is expired, delete it
isUnlocked = true
return nil, true
}
if oldValue.Token == token {
if oldValue.ExpiredAtNs <= now.UnixNano() {
isUnlocked = true
return nil, true
}
return oldValue, false
} else {
isUnlocked = false
err = UnlockErrorTokenMismatch
return oldValue, false
}
} else {
lm.accessLock.Lock()
defer lm.accessLock.Unlock()
if oldValue, found := lm.locks[path]; found {
now := time.Now()
if oldValue.ExpiredAtNs > 0 && oldValue.ExpiredAtNs < now.UnixNano() {
// lock is expired, delete it
isUnlocked = true
glog.V(4).Infof("key %s expired at %v", path, time.Unix(0, oldValue.ExpiredAtNs))
delete(lm.locks, path)
return
}
if oldValue.Token == token {
isUnlocked = true
return nil, true
glog.V(4).Infof("key %s unlocked with %v", path, token)
delete(lm.locks, path)
return
} else {
isUnlocked = false
err = UnlockErrorTokenMismatch
return
}
})
}
err = LockNotFound
return
}
func (lm *LockManager) CleanUp() {
for {
time.Sleep(1 * time.Minute)
now := time.Now().UnixNano()
lm.locks.Range(func(key string, value *Lock) bool {
lm.accessLock.Lock()
for key, value := range lm.locks {
if value == nil {
return true
continue
}
if now > value.ExpiredAtNs {
lm.locks.Delete(key)
return true
glog.V(4).Infof("key %s expired at %v", key, time.Unix(0, value.ExpiredAtNs))
delete(lm.locks, key)
}
return true
})
}
lm.accessLock.Unlock()
}
}
// SelectLocks takes out locks by key
// if keyFn return true, the lock will be taken out
func (lm *LockManager) SelectLocks(selectFn func(key string) bool) (locks []*Lock) {
lm.accessLock.RLock()
defer lm.accessLock.RUnlock()
now := time.Now().UnixNano()
lm.locks.Range(func(key string, lock *Lock) bool {
for key, lock := range lm.locks {
if now > lock.ExpiredAtNs {
lm.locks.Delete(key)
return true
glog.V(4).Infof("key %s expired at %v", key, time.Unix(0, lock.ExpiredAtNs))
delete(lm.locks, key)
continue
}
if selectFn(key) {
lm.locks.Delete(key)
glog.V(4).Infof("key %s selected and deleted", key)
delete(lm.locks, key)
lock.Key = key
locks = append(locks, lock)
}
return true
})
}
return
}
// InsertLock inserts a lock unconditionally
func (lm *LockManager) InsertLock(path string, expiredAtNs int64, token string, owner string) {
lm.locks.Store(path, &Lock{Token: token, ExpiredAtNs: expiredAtNs, Owner: owner})
lm.accessLock.Lock()
defer lm.accessLock.Unlock()
lm.locks[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 && lock != nil {
owner = lock.Owner
return false
}
return true
})
return
lm.accessLock.RLock()
defer lm.accessLock.RUnlock()
if lock, found := lm.locks[key]; found {
return lock.Owner, nil
}
err = LockNotFound
return
}

41
weed/filer/filechunk_manifest.go

@ -5,8 +5,6 @@ import (
"fmt"
"io"
"math"
"net/url"
"strings"
"sync"
"time"
@ -122,44 +120,7 @@ func fetchChunkRange(buffer []byte, lookupFileIdFn wdclient.LookupFileIdFunction
glog.Errorf("operation LookupFileId %s failed, err: %v", fileId, err)
return 0, err
}
return retriedFetchChunkData(buffer, urlStrings, cipherKey, isGzipped, false, offset)
}
func retriedFetchChunkData(buffer []byte, urlStrings []string, cipherKey []byte, isGzipped bool, isFullChunk bool, offset int64) (n int, err error) {
var shouldRetry bool
for waitTime := time.Second; waitTime < util.RetryWaitTime; waitTime += waitTime / 2 {
for _, urlString := range urlStrings {
n = 0
if strings.Contains(urlString, "%") {
urlString = url.PathEscape(urlString)
}
shouldRetry, err = util.ReadUrlAsStream(urlString+"?readDeleted=true", cipherKey, isGzipped, isFullChunk, offset, len(buffer), func(data []byte) {
if n < len(buffer) {
x := copy(buffer[n:], data)
n += x
}
})
if !shouldRetry {
break
}
if err != nil {
glog.V(0).Infof("read %s failed, err: %v", urlString, err)
} else {
break
}
}
if err != nil && shouldRetry {
glog.V(0).Infof("retry reading in %v", waitTime)
time.Sleep(waitTime)
} else {
break
}
}
return n, err
return util.RetriedFetchChunkData(buffer, urlStrings, cipherKey, isGzipped, false, offset)
}
func retriedStreamFetchChunkData(writer io.Writer, urlStrings []string, cipherKey []byte, isGzipped bool, isFullChunk bool, offset int64, size int) (err error) {

4
weed/filer/filer.go

@ -69,7 +69,7 @@ func NewFiler(masters pb.ServerDiscovery, grpcDialOption grpc.DialOption, filerH
f.UniqueFilerId = -f.UniqueFilerId
}
f.LocalMetaLogBuffer = log_buffer.NewLogBuffer("local", LogFlushInterval, f.logFlushFunc, notifyFn)
f.LocalMetaLogBuffer = log_buffer.NewLogBuffer("local", LogFlushInterval, f.logFlushFunc, nil, notifyFn)
f.metaLogCollection = collection
f.metaLogReplication = replication
@ -377,6 +377,6 @@ func (f *Filer) doListDirectoryEntries(ctx context.Context, p util.FullPath, sta
}
func (f *Filer) Shutdown() {
f.LocalMetaLogBuffer.Shutdown()
f.LocalMetaLogBuffer.ShutdownLogBuffer()
f.Store.Shutdown()
}

16
weed/filer/filer_notify.go

@ -3,6 +3,7 @@ package filer
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"io"
"math"
"regexp"
@ -86,7 +87,7 @@ func (f *Filer) logMetaEvent(ctx context.Context, fullpath string, eventNotifica
}
func (f *Filer) logFlushFunc(startTime, stopTime time.Time, buf []byte) {
func (f *Filer) logFlushFunc(logBuffer *log_buffer.LogBuffer, startTime, stopTime time.Time, buf []byte) {
if len(buf) == 0 {
return
@ -113,11 +114,10 @@ var (
VolumeNotFoundPattern = regexp.MustCompile(`volume \d+? not found`)
)
func (f *Filer) ReadPersistedLogBuffer(startTime time.Time, stopTsNs int64, eachLogEntryFn func(logEntry *filer_pb.LogEntry) error) (lastTsNs int64, isDone bool, err error) {
func (f *Filer) ReadPersistedLogBuffer(startPosition log_buffer.MessagePosition, stopTsNs int64, eachLogEntryFn log_buffer.EachLogEntryFuncType) (lastTsNs int64, isDone bool, err error) {
startTime = startTime.UTC()
startDate := fmt.Sprintf("%04d-%02d-%02d", startTime.Year(), startTime.Month(), startTime.Day())
startHourMinute := fmt.Sprintf("%02d-%02d", startTime.Hour(), startTime.Minute())
startDate := fmt.Sprintf("%04d-%02d-%02d", startPosition.Year(), startPosition.Month(), startPosition.Day())
startHourMinute := fmt.Sprintf("%02d-%02d", startPosition.Hour(), startPosition.Minute())
var stopDate, stopHourMinute string
if stopTsNs != 0 {
stopTime := time.Unix(0, stopTsNs+24*60*60*int64(time.Nanosecond)).UTC()
@ -126,7 +126,7 @@ func (f *Filer) ReadPersistedLogBuffer(startTime time.Time, stopTsNs int64, each
}
sizeBuf := make([]byte, 4)
startTsNs := startTime.UnixNano()
startTsNs := startPosition.UnixNano()
dayEntries, _, listDayErr := f.ListDirectoryEntries(context.Background(), SystemLogDir, startDate, true, math.MaxInt32, "", "", "")
if listDayErr != nil {
@ -177,7 +177,7 @@ func (f *Filer) ReadPersistedLogBuffer(startTime time.Time, stopTsNs int64, each
return lastTsNs, isDone, nil
}
func ReadEachLogEntry(r io.Reader, sizeBuf []byte, startTsNs, stopTsNs int64, eachLogEntryFn func(logEntry *filer_pb.LogEntry) error) (lastTsNs int64, err error) {
func ReadEachLogEntry(r io.Reader, sizeBuf []byte, startTsNs, stopTsNs int64, eachLogEntryFn log_buffer.EachLogEntryFuncType) (lastTsNs int64, err error) {
for {
n, err := r.Read(sizeBuf)
if err != nil {
@ -207,7 +207,7 @@ func ReadEachLogEntry(r io.Reader, sizeBuf []byte, startTsNs, stopTsNs int64, ea
return lastTsNs, err
}
// println("each log: ", logEntry.TsNs)
if err := eachLogEntryFn(logEntry); err != nil {
if _, err := eachLogEntryFn(logEntry); err != nil {
return lastTsNs, err
} else {
lastTsNs = logEntry.TsNs

5
weed/filer/meta_aggregator.go

@ -43,7 +43,7 @@ func NewMetaAggregator(filer *Filer, self pb.ServerAddress, grpcDialOption grpc.
peerChans: make(map[pb.ServerAddress]chan struct{}),
}
t.ListenersCond = sync.NewCond(&t.ListenersLock)
t.MetaLogBuffer = log_buffer.NewLogBuffer("aggr", LogFlushInterval, nil, func() {
t.MetaLogBuffer = log_buffer.NewLogBuffer("aggr", LogFlushInterval, nil, nil, func() {
t.ListenersCond.Broadcast()
})
return t
@ -188,6 +188,7 @@ func (ma *MetaAggregator) doSubscribeToOneFiler(f *Filer, self pb.ServerAddress,
ClientEpoch: atomic.LoadInt32(&ma.filer.UniqueFilerEpoch),
})
if err != nil {
glog.V(0).Infof("SubscribeLocalMetadata %v: %v", peer, err)
return fmt.Errorf("subscribe: %v", err)
}
@ -197,10 +198,12 @@ func (ma *MetaAggregator) doSubscribeToOneFiler(f *Filer, self pb.ServerAddress,
return nil
}
if listenErr != nil {
glog.V(0).Infof("SubscribeLocalMetadata stream %v: %v", peer, listenErr)
return listenErr
}
if err := processEventFn(resp); err != nil {
glog.V(0).Infof("SubscribeLocalMetadata process %v: %v", resp, err)
return fmt.Errorf("process %v: %v", resp, err)
}

3
weed/filer/reader_cache.go

@ -2,6 +2,7 @@ package filer
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/util"
"sync"
"sync/atomic"
"time"
@ -170,7 +171,7 @@ func (s *SingleChunkCacher) startCaching() {
s.data = mem.Allocate(s.chunkSize)
_, s.err = retriedFetchChunkData(s.data, urlStrings, s.cipherKey, s.isGzipped, true, 0)
_, s.err = util.RetriedFetchChunkData(s.data, urlStrings, s.cipherKey, s.isGzipped, true, 0)
if s.err != nil {
mem.Free(s.data)
s.data = nil

2
weed/filer/stream.go

@ -187,7 +187,7 @@ func ReadAll(buffer []byte, masterClient *wdclient.MasterClient, chunks []*filer
return err
}
n, err := retriedFetchChunkData(buffer[idx:idx+int(chunkView.ViewSize)], urlStrings, chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.OffsetInChunk)
n, err := util.RetriedFetchChunkData(buffer[idx:idx+int(chunkView.ViewSize)], urlStrings, chunkView.CipherKey, chunkView.IsGzipped, chunkView.IsFullChunk(), chunkView.OffsetInChunk)
if err != nil {
return err
}

72
weed/mq/broker/broker_stats.go → weed/mq/broker/broker_connect.go

@ -4,9 +4,7 @@ import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"io"
"math/rand"
@ -14,31 +12,17 @@ import (
)
// BrokerConnectToBalancer connects to the broker balancer and sends stats
func (b *MessageQueueBroker) BrokerConnectToBalancer(self string) error {
// find the lock owner
var brokerBalancer string
err := b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
resp, err := client.FindLockOwner(context.Background(), &filer_pb.FindLockOwnerRequest{
Name: pub_balancer.LockBrokerBalancer,
})
if err != nil {
return err
}
brokerBalancer = resp.Owner
return nil
})
if err != nil {
return err
}
b.currentBalancer = pb.ServerAddress(brokerBalancer)
func (b *MessageQueueBroker) BrokerConnectToBalancer(brokerBalancer string, stopCh chan struct{}) error {
self := string(b.option.BrokerAddress())
glog.V(0).Infof("broker %s found balancer %s", self, brokerBalancer)
glog.V(0).Infof("broker %s connects to balancer %s", self, brokerBalancer)
if brokerBalancer == "" {
return fmt.Errorf("no balancer found")
}
// connect to the lock owner
err = pb.WithBrokerGrpcClient(false, brokerBalancer, b.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
return pb.WithBrokerGrpcClient(true, brokerBalancer, b.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
stream, err := client.PublisherToPubBalancer(context.Background())
if err != nil {
return fmt.Errorf("connect to balancer %v: %v", brokerBalancer, err)
@ -56,6 +40,13 @@ func (b *MessageQueueBroker) BrokerConnectToBalancer(self string) error {
}
for {
// check if the broker is stopping
select {
case <-stopCh:
return nil
default:
}
stats := b.localTopicManager.CollectStats(time.Second * 5)
err = stream.Send(&mq_pb.PublisherToPubBalancerRequest{
Message: &mq_pb.PublisherToPubBalancerRequest_Stats{
@ -68,13 +59,44 @@ func (b *MessageQueueBroker) BrokerConnectToBalancer(self string) error {
}
return fmt.Errorf("send stats message: %v", err)
}
glog.V(3).Infof("sent stats: %+v", stats)
// glog.V(3).Infof("sent stats: %+v", stats)
time.Sleep(time.Millisecond*5000 + time.Duration(rand.Intn(1000))*time.Millisecond)
}
return nil
})
}
func (b *MessageQueueBroker) KeepConnectedToBrokerBalancer(newBrokerBalancerCh chan string) {
var stopPrevRunChan chan struct{}
for {
select {
case newBrokerBalancer := <-newBrokerBalancerCh:
if stopPrevRunChan != nil {
close(stopPrevRunChan)
stopPrevRunChan = nil
}
thisRunStopChan := make(chan struct{})
if newBrokerBalancer != "" {
stopPrevRunChan = thisRunStopChan
go func() {
for {
err := b.BrokerConnectToBalancer(newBrokerBalancer, thisRunStopChan)
if err != nil {
glog.V(0).Infof("connect to balancer %s: %v", newBrokerBalancer, err)
time.Sleep(time.Second)
} else {
break
}
return err
select {
case <-thisRunStopChan:
return
default:
}
}
}()
}
}
}
}

16
weed/mq/broker/broker_grpc_admin.go

@ -2,25 +2,21 @@ package broker
import (
"context"
"github.com/seaweedfs/seaweedfs/weed/cluster"
"github.com/seaweedfs/seaweedfs/weed/pb/master_pb"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
)
func (b *MessageQueueBroker) FindBrokerLeader(c context.Context, request *mq_pb.FindBrokerLeaderRequest) (*mq_pb.FindBrokerLeaderResponse, error) {
ret := &mq_pb.FindBrokerLeaderResponse{}
err := b.withMasterClient(false, b.MasterClient.GetMaster(), func(client master_pb.SeaweedClient) error {
resp, err := client.ListClusterNodes(context.Background(), &master_pb.ListClusterNodesRequest{
ClientType: cluster.BrokerType,
FilerGroup: request.FilerGroup,
err := b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
resp, err := client.FindLockOwner(context.Background(), &filer_pb.FindLockOwnerRequest{
Name: pub_balancer.LockBrokerBalancer,
})
if err != nil {
return err
}
if len(resp.ClusterNodes) == 0 {
return nil
}
ret.Broker = resp.ClusterNodes[0].Address
ret.Broker = resp.Owner
return nil
})
return ret, err

100
weed/mq/broker/broker_grpc_assign.go

@ -0,0 +1,100 @@
package broker
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"sync"
)
// AssignTopicPartitions Runs on the assigned broker, to execute the topic partition assignment
func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *mq_pb.AssignTopicPartitionsRequest) (*mq_pb.AssignTopicPartitionsResponse, error) {
ret := &mq_pb.AssignTopicPartitionsResponse{}
self := pb.ServerAddress(fmt.Sprintf("%s:%d", b.option.Ip, b.option.Port))
// drain existing topic partition subscriptions
for _, assignment := range request.BrokerPartitionAssignments {
t := topic.FromPbTopic(request.Topic)
partition := topic.FromPbPartition(assignment.Partition)
b.accessLock.Lock()
if request.IsDraining {
// TODO drain existing topic partition subscriptions
b.localTopicManager.RemoveTopicPartition(t, partition)
} else {
var localPartition *topic.LocalPartition
if localPartition = b.localTopicManager.GetTopicPartition(t, partition); localPartition == nil {
localPartition = topic.FromPbBrokerPartitionAssignment(self, partition, assignment, b.genLogFlushFunc(t, assignment.Partition), b.genLogOnDiskReadFunc(t, assignment.Partition))
b.localTopicManager.AddTopicPartition(t, localPartition)
}
}
b.accessLock.Unlock()
}
// if is leader, notify the followers to drain existing topic partition subscriptions
if request.IsLeader {
for _, brokerPartition := range request.BrokerPartitionAssignments {
for _, follower := range brokerPartition.FollowerBrokers {
err := pb.WithBrokerGrpcClient(false, follower, b.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.AssignTopicPartitions(context.Background(), request)
return err
})
if err != nil {
return ret, err
}
}
}
}
glog.V(0).Infof("AssignTopicPartitions: topic %s partition assignments: %v", request.Topic, request.BrokerPartitionAssignments)
return ret, nil
}
// called by broker leader to drain existing partitions.
// new/updated partitions will be detected by broker from the filer
func (b *MessageQueueBroker) assignTopicPartitionsToBrokers(ctx context.Context, t *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment, isAdd bool) error {
// notify the brokers to create the topic partitions in parallel
var wg sync.WaitGroup
for _, bpa := range assignments {
wg.Add(1)
go func(bpa *mq_pb.BrokerPartitionAssignment) {
defer wg.Done()
if doCreateErr := b.withBrokerClient(false, pb.ServerAddress(bpa.LeaderBroker), func(client mq_pb.SeaweedMessagingClient) error {
_, doCreateErr := client.AssignTopicPartitions(ctx, &mq_pb.AssignTopicPartitionsRequest{
Topic: t,
BrokerPartitionAssignments: []*mq_pb.BrokerPartitionAssignment{
{
Partition: bpa.Partition,
},
},
IsLeader: true,
IsDraining: !isAdd,
})
if doCreateErr != nil {
if !isAdd {
return fmt.Errorf("drain topic %s %v on %s: %v", t, bpa.LeaderBroker, bpa.Partition, doCreateErr)
} else {
return fmt.Errorf("create topic %s %v on %s: %v", t, bpa.LeaderBroker, bpa.Partition, doCreateErr)
}
}
brokerStats, found := b.Balancer.Brokers.Get(bpa.LeaderBroker)
if !found {
brokerStats = pub_balancer.NewBrokerStats()
if !b.Balancer.Brokers.SetIfAbsent(bpa.LeaderBroker, brokerStats) {
brokerStats, _ = b.Balancer.Brokers.Get(bpa.LeaderBroker)
}
}
brokerStats.RegisterAssignment(t, bpa.Partition, isAdd)
return nil
}); doCreateErr != nil {
glog.Errorf("create topic %s partition %+v on %s: %v", t, bpa.Partition, bpa.LeaderBroker, doCreateErr)
}
}(bpa)
}
wg.Wait()
return nil
}

10
weed/mq/broker/broker_grpc_balance.go

@ -2,17 +2,13 @@ package broker
import (
"context"
"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 (b *MessageQueueBroker) BalanceTopics(ctx context.Context, request *mq_pb.BalanceTopicsRequest) (resp *mq_pb.BalanceTopicsResponse, err error) {
if b.currentBalancer == "" {
return nil, status.Errorf(codes.Unavailable, "no balancer")
}
if !b.lockAsBalancer.IsLocked() {
proxyErr := b.withBrokerClient(false, b.currentBalancer, func(client mq_pb.SeaweedMessagingClient) error {
if !b.isLockOwner() {
proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error {
resp, err = client.BalanceTopics(ctx, request)
return nil
})

105
weed/mq/broker/broker_grpc_configure.go

@ -3,6 +3,7 @@ package broker
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb"
@ -15,11 +16,8 @@ import (
// It generates an assignments based on existing allocations,
// and then assign the partitions to the brokers.
func (b *MessageQueueBroker) ConfigureTopic(ctx context.Context, request *mq_pb.ConfigureTopicRequest) (resp *mq_pb.ConfigureTopicResponse, err error) {
if b.currentBalancer == "" {
return nil, status.Errorf(codes.Unavailable, "no balancer")
}
if !b.lockAsBalancer.IsLocked() {
proxyErr := b.withBrokerClient(false, b.currentBalancer, func(client mq_pb.SeaweedMessagingClient) error {
if !b.isLockOwner() {
proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error {
resp, err = client.ConfigureTopic(ctx, request)
return nil
})
@ -29,79 +27,44 @@ func (b *MessageQueueBroker) ConfigureTopic(ctx context.Context, request *mq_pb.
return resp, err
}
ret := &mq_pb.ConfigureTopicResponse{}
ret.BrokerPartitionAssignments, err = b.Balancer.LookupOrAllocateTopicPartitions(request.Topic, true, request.PartitionCount)
for _, bpa := range ret.BrokerPartitionAssignments {
// fmt.Printf("create topic %s on %s\n", request.Topic, bpa.LeaderBroker)
if doCreateErr := b.withBrokerClient(false, pb.ServerAddress(bpa.LeaderBroker), func(client mq_pb.SeaweedMessagingClient) error {
_, doCreateErr := client.AssignTopicPartitions(ctx, &mq_pb.AssignTopicPartitionsRequest{
Topic: request.Topic,
BrokerPartitionAssignments: []*mq_pb.BrokerPartitionAssignment{
{
Partition: bpa.Partition,
},
},
IsLeader: true,
IsDraining: false,
})
if doCreateErr != nil {
return fmt.Errorf("do create topic %s on %s: %v", request.Topic, bpa.LeaderBroker, doCreateErr)
}
brokerStats, found := b.Balancer.Brokers.Get(bpa.LeaderBroker)
if !found {
brokerStats = pub_balancer.NewBrokerStats()
if !b.Balancer.Brokers.SetIfAbsent(bpa.LeaderBroker, brokerStats) {
brokerStats, _ = b.Balancer.Brokers.Get(bpa.LeaderBroker)
}
}
brokerStats.RegisterAssignment(request.Topic, bpa.Partition)
return nil
}); doCreateErr != nil {
return nil, doCreateErr
}
t := topic.FromPbTopic(request.Topic)
var readErr, assignErr error
resp, readErr = b.readTopicConfFromFiler(t)
if readErr != nil {
glog.V(0).Infof("read topic %s conf: %v", request.Topic, readErr)
}
// TODO revert if some error happens in the middle of the assignments
return ret, err
}
// AssignTopicPartitions Runs on the assigned broker, to execute the topic partition assignment
func (b *MessageQueueBroker) AssignTopicPartitions(c context.Context, request *mq_pb.AssignTopicPartitionsRequest) (*mq_pb.AssignTopicPartitionsResponse, error) {
ret := &mq_pb.AssignTopicPartitionsResponse{}
self := pb.ServerAddress(fmt.Sprintf("%s:%d", b.option.Ip, b.option.Port))
if resp != nil {
assignErr = b.ensureTopicActiveAssignments(t, resp)
// no need to assign directly.
// The added or updated assignees will read from filer directly.
// The gone assignees will die by themselves.
}
// drain existing topic partition subscriptions
for _, brokerPartition := range request.BrokerPartitionAssignments {
localPartition := topic.FromPbBrokerPartitionAssignment(self, brokerPartition)
if request.IsDraining {
// TODO drain existing topic partition subscriptions
if readErr == nil && assignErr == nil && len(resp.BrokerPartitionAssignments) == int(request.PartitionCount) {
glog.V(0).Infof("existing topic partitions %d: %+v", len(resp.BrokerPartitionAssignments), resp.BrokerPartitionAssignments)
return
}
b.localTopicManager.RemoveTopicPartition(
topic.FromPbTopic(request.Topic),
localPartition.Partition)
} else {
b.localTopicManager.AddTopicPartition(
topic.FromPbTopic(request.Topic),
localPartition)
if resp != nil && len(resp.BrokerPartitionAssignments) > 0 {
if cancelErr := b.assignTopicPartitionsToBrokers(ctx, request.Topic, resp.BrokerPartitionAssignments, false); cancelErr != nil {
glog.V(1).Infof("cancel old topic %s partitions assignments %v : %v", request.Topic, resp.BrokerPartitionAssignments, cancelErr)
}
}
resp = &mq_pb.ConfigureTopicResponse{}
if b.Balancer.Brokers.IsEmpty() {
return nil, status.Errorf(codes.Unavailable, pub_balancer.ErrNoBroker.Error())
}
resp.BrokerPartitionAssignments = pub_balancer.AllocateTopicPartitions(b.Balancer.Brokers, request.PartitionCount)
// if is leader, notify the followers to drain existing topic partition subscriptions
if request.IsLeader {
for _, brokerPartition := range request.BrokerPartitionAssignments {
for _, follower := range brokerPartition.FollowerBrokers {
err := pb.WithBrokerGrpcClient(false, follower, b.grpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.AssignTopicPartitions(context.Background(), request)
return err
})
if err != nil {
return ret, err
}
}
}
// save the topic configuration on filer
if err := b.saveTopicConfToFiler(request.Topic, resp); err != nil {
return nil, fmt.Errorf("configure topic: %v", err)
}
return ret, nil
b.Balancer.OnPartitionChange(request.Topic, resp.BrokerPartitionAssignments)
glog.V(0).Infof("ConfigureTopic: topic %s partition assignments: %v", request.Topic, resp.BrokerPartitionAssignments)
return resp, err
}

45
weed/mq/broker/broker_grpc_lookup.go

@ -3,28 +3,16 @@ package broker
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
// 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.1.1 if the request is_for_subscribe, return error not found
// 2.1.2 if the request is_for_publish, create the topic
// 2.2 if the topic is found, return the brokers
//
// 3. unlock the topic
// LookupTopicBrokers returns the brokers that are serving the topic
func (b *MessageQueueBroker) LookupTopicBrokers(ctx context.Context, request *mq_pb.LookupTopicBrokersRequest) (resp *mq_pb.LookupTopicBrokersResponse, err error) {
if b.currentBalancer == "" {
return nil, status.Errorf(codes.Unavailable, "no balancer")
}
if !b.lockAsBalancer.IsLocked() {
proxyErr := b.withBrokerClient(false, b.currentBalancer, func(client mq_pb.SeaweedMessagingClient) error {
if !b.isLockOwner() {
proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error {
resp, err = client.LookupTopicBrokers(ctx, request)
return nil
})
@ -34,18 +22,23 @@ func (b *MessageQueueBroker) LookupTopicBrokers(ctx context.Context, request *mq
return resp, err
}
t := topic.FromPbTopic(request.Topic)
ret := &mq_pb.LookupTopicBrokersResponse{}
conf := &mq_pb.ConfigureTopicResponse{}
ret.Topic = request.Topic
ret.BrokerPartitionAssignments, err = b.Balancer.LookupOrAllocateTopicPartitions(ret.Topic, request.IsForPublish, 6)
if conf, err = b.readTopicConfFromFiler(t); err != nil {
glog.V(0).Infof("lookup topic %s conf: %v", request.Topic, err)
} else {
err = b.ensureTopicActiveAssignments(t, conf)
ret.BrokerPartitionAssignments = conf.BrokerPartitionAssignments
}
return ret, err
}
func (b *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb.ListTopicsRequest) (resp *mq_pb.ListTopicsResponse, err error) {
if b.currentBalancer == "" {
return nil, status.Errorf(codes.Unavailable, "no balancer")
}
if !b.lockAsBalancer.IsLocked() {
proxyErr := b.withBrokerClient(false, b.currentBalancer, func(client mq_pb.SeaweedMessagingClient) error {
if !b.isLockOwner() {
proxyErr := b.withBrokerClient(false, pb.ServerAddress(b.lockAsBalancer.LockOwner()), func(client mq_pb.SeaweedMessagingClient) error {
resp, err = client.ListTopics(ctx, request)
return nil
})
@ -76,3 +69,7 @@ func (b *MessageQueueBroker) ListTopics(ctx context.Context, request *mq_pb.List
return ret, nil
}
func (b *MessageQueueBroker) isLockOwner() bool {
return b.lockAsBalancer.LockOwner() == b.option.BrokerAddress().String()
}

73
weed/mq/broker/broker_grpc_pub.go

@ -5,12 +5,13 @@ import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc/peer"
"io"
"math/rand"
"net"
"sync/atomic"
"time"
)
// PUB
@ -34,7 +35,7 @@ import (
// Subscribers needs to listen for new partitions and connect to the brokers.
// Each subscription may not get data. It can act as a backup.
func (b *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishServer) error {
func (b *MessageQueueBroker) PublishMessage(stream mq_pb.SeaweedMessaging_PublishMessageServer) error {
// 1. write to the volume server
// 2. find the topic metadata owning filer
// 3. write to the filer
@ -44,19 +45,36 @@ func (b *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishServer
if err != nil {
return err
}
response := &mq_pb.PublishResponse{}
response := &mq_pb.PublishMessageResponse{}
// TODO check whether current broker should be the leader for the topic partition
ackInterval := 1
initMessage := req.GetInit()
var t topic.Topic
var p topic.Partition
if initMessage != nil {
t, p := topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition)
localTopicPartition = b.localTopicManager.GetTopicPartition(t, p)
if localTopicPartition == nil {
t, p = topic.FromPbTopic(initMessage.Topic), topic.FromPbPartition(initMessage.Partition)
localTopicPartition, _, err = b.GetOrGenLocalPartition(t, p)
if err != nil {
response.Error = fmt.Sprintf("topic %v partition %v not setup", initMessage.Topic, initMessage.Partition)
glog.Errorf("topic %v partition %v not setup", initMessage.Topic, initMessage.Partition)
return stream.Send(response)
}
ackInterval = int(initMessage.AckInterval)
for _, follower := range initMessage.FollowerBrokers {
followErr := b.withBrokerClient(false, pb.ServerAddress(follower), func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.PublishFollowMe(context.Background(), &mq_pb.PublishFollowMeRequest{
Topic: initMessage.Topic,
Partition: initMessage.Partition,
BrokerSelf: string(b.option.BrokerAddress()),
})
return err
})
if followErr != nil {
response.Error = fmt.Sprintf("follower %v failed: %v", follower, followErr)
glog.Errorf("follower %v failed: %v", follower, followErr)
return stream.Send(response)
}
}
stream.Send(response)
} else {
response.Error = fmt.Sprintf("missing init message")
@ -70,38 +88,23 @@ func (b *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishServer
ackCounter := 0
var ackSequence int64
var isStopping int32
respChan := make(chan *mq_pb.PublishResponse, 128)
respChan := make(chan *mq_pb.PublishMessageResponse, 128)
defer func() {
atomic.StoreInt32(&isStopping, 1)
respChan <- &mq_pb.PublishMessageResponse{
AckSequence: ackSequence,
}
close(respChan)
localTopicPartition.Publishers.RemovePublisher(clientName)
glog.V(0).Infof("topic %v partition %v published %d messges Publisher:%d Subscriber:%d", initMessage.Topic, initMessage.Partition, ackSequence, localTopicPartition.Publishers.Size(), localTopicPartition.Subscribers.Size())
if localTopicPartition.MaybeShutdownLocalPartition() {
b.localTopicManager.RemoveTopicPartition(t, p)
}
}()
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
}
case <-localTopicPartition.StopPublishersCh:
respChan <- &mq_pb.PublishResponse{
AckSequence: ackSequence,
ShouldClose: true,
}
for resp := range respChan {
if err := stream.Send(resp); err != nil {
glog.Errorf("Error sending response %v: %v", resp, err)
}
}
}()
@ -111,6 +114,10 @@ func (b *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishServer
// receive a message
req, err := stream.Recv()
if err != nil {
if err == io.EOF {
break
}
glog.V(0).Infof("topic %v partition %v publish stream error: %v", initMessage.Topic, initMessage.Partition, err)
return err
}
@ -124,7 +131,7 @@ func (b *MessageQueueBroker) Publish(stream mq_pb.SeaweedMessaging_PublishServer
if ackCounter >= ackInterval {
ackCounter = 0
// send back the ack
response := &mq_pb.PublishResponse{
response := &mq_pb.PublishMessageResponse{
AckSequence: ackSequence,
}
respChan <- response

12
weed/mq/broker/broker_grpc_pub_balancer.go

@ -1,7 +1,6 @@
package broker
import (
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc/codes"
@ -10,7 +9,7 @@ import (
// PublisherToPubBalancer receives connections from brokers and collects stats
func (b *MessageQueueBroker) PublisherToPubBalancer(stream mq_pb.SeaweedMessaging_PublisherToPubBalancerServer) error {
if !b.lockAsBalancer.IsLocked() {
if !b.isLockOwner() {
return status.Errorf(codes.Unavailable, "not current broker balancer")
}
req, err := stream.Recv()
@ -22,12 +21,12 @@ func (b *MessageQueueBroker) PublisherToPubBalancer(stream mq_pb.SeaweedMessagin
initMessage := req.GetInit()
var brokerStats *pub_balancer.BrokerStats
if initMessage != nil {
brokerStats = b.Balancer.OnBrokerConnected(initMessage.Broker)
brokerStats = b.Balancer.AddBroker(initMessage.Broker)
} else {
return status.Errorf(codes.InvalidArgument, "balancer init message is empty")
}
defer func() {
b.Balancer.OnBrokerDisconnected(initMessage.Broker, brokerStats)
b.Balancer.RemoveBroker(initMessage.Broker, brokerStats)
}()
// process stats message
@ -36,13 +35,12 @@ func (b *MessageQueueBroker) PublisherToPubBalancer(stream mq_pb.SeaweedMessagin
if err != nil {
return err
}
if !b.lockAsBalancer.IsLocked() {
if !b.isLockOwner() {
return status.Errorf(codes.Unavailable, "not current broker balancer")
}
if receivedStats := req.GetStats(); receivedStats != nil {
b.Balancer.OnBrokerStatsUpdated(initMessage.Broker, brokerStats, receivedStats)
glog.V(4).Infof("broker %s stats: %+v", initMessage.Broker, brokerStats)
glog.V(4).Infof("received stats: %+v", receivedStats)
// glog.V(4).Infof("received from %v: %+v", initMessage.Broker, receivedStats)
}
}

96
weed/mq/broker/broker_grpc_pub_follow.go

@ -0,0 +1,96 @@
package broker
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"io"
"math/rand"
"sync"
"time"
)
func (b *MessageQueueBroker) PublishFollowMe(c context.Context, request *mq_pb.PublishFollowMeRequest) (*mq_pb.PublishFollowMeResponse, error) {
glog.V(0).Infof("PublishFollowMe %v", request)
var wg sync.WaitGroup
wg.Add(1)
var ret error
go b.withBrokerClient(true, pb.ServerAddress(request.BrokerSelf), func(client mq_pb.SeaweedMessagingClient) error {
followerId := rand.Int31()
subscribeClient, err := client.FollowInMemoryMessages(context.Background(), &mq_pb.FollowInMemoryMessagesRequest{
Message: &mq_pb.FollowInMemoryMessagesRequest_Init{
Init: &mq_pb.FollowInMemoryMessagesRequest_InitMessage{
ConsumerGroup: string(b.option.BrokerAddress()),
ConsumerId: fmt.Sprintf("followMe-%d", followerId),
FollowerId: followerId,
Topic: request.Topic,
PartitionOffset: &mq_pb.PartitionOffset{
Partition: request.Partition,
StartTsNs: 0,
StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
},
},
},
})
if err != nil {
glog.Errorf("FollowInMemoryMessages error: %v", err)
ret = err
return err
}
// receive first hello message
resp, err := subscribeClient.Recv()
if err != nil {
return fmt.Errorf("FollowInMemoryMessages recv first message error: %v", err)
}
if resp == nil {
glog.V(0).Infof("doFollowInMemoryMessage recv first message nil response")
return io.ErrUnexpectedEOF
}
wg.Done()
b.doFollowInMemoryMessage(context.Background(), subscribeClient)
return nil
})
wg.Wait()
return &mq_pb.PublishFollowMeResponse{}, ret
}
func (b *MessageQueueBroker) doFollowInMemoryMessage(c context.Context, client mq_pb.SeaweedMessaging_FollowInMemoryMessagesClient) {
for {
resp, err := client.Recv()
if err != nil {
if err != io.EOF {
glog.V(0).Infof("doFollowInMemoryMessage error: %v", err)
}
return
}
if resp == nil {
glog.V(0).Infof("doFollowInMemoryMessage nil response")
return
}
if resp.Message != nil {
// process ctrl message or data message
switch m := resp.Message.(type) {
case *mq_pb.FollowInMemoryMessagesResponse_Data:
// process data message
print("d")
case *mq_pb.FollowInMemoryMessagesResponse_Ctrl:
// process ctrl message
if m.Ctrl.FlushedSequence > 0 {
flushTime := time.Unix(0, m.Ctrl.FlushedSequence)
glog.V(0).Infof("doFollowInMemoryMessage flushTime: %v", flushTime)
}
if m.Ctrl.FollowerChangedToId != 0 {
// follower changed
glog.V(0).Infof("doFollowInMemoryMessage follower changed to %d", m.Ctrl.FollowerChangedToId)
return
}
}
}
}
}

274
weed/mq/broker/broker_grpc_sub.go

@ -7,53 +7,81 @@ import (
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"sync/atomic"
"time"
)
func (b *MessageQueueBroker) Subscribe(req *mq_pb.SubscribeRequest, stream mq_pb.SeaweedMessaging_SubscribeServer) error {
func (b *MessageQueueBroker) SubscribeMessage(req *mq_pb.SubscribeMessageRequest, stream mq_pb.SeaweedMessaging_SubscribeMessageServer) (err error) {
ctx := stream.Context()
clientName := fmt.Sprintf("%s/%s-%s", req.GetInit().ConsumerGroup, req.GetInit().ConsumerId, req.GetInit().ClientId)
t := topic.FromPbTopic(req.GetInit().Topic)
partition := topic.FromPbPartition(req.GetInit().Partition)
localTopicPartition := b.localTopicManager.GetTopicPartition(t, partition)
if localTopicPartition == nil {
stream.Send(&mq_pb.SubscribeResponse{
Message: &mq_pb.SubscribeResponse_Ctrl{
Ctrl: &mq_pb.SubscribeResponse_CtrlMessage{
Error: "not initialized",
},
},
})
return nil
partition := topic.FromPbPartition(req.GetInit().GetPartitionOffset().GetPartition())
glog.V(0).Infof("Subscriber %s on %v %v connected", req.GetInit().ConsumerId, t, partition)
waitIntervalCount := 0
var localTopicPartition *topic.LocalPartition
for localTopicPartition == nil {
localTopicPartition, _, err = b.GetOrGenLocalPartition(t, partition)
if err != nil {
glog.V(1).Infof("topic %v partition %v not setup", t, partition)
}
if localTopicPartition != nil {
break
}
waitIntervalCount++
if waitIntervalCount > 10 {
waitIntervalCount = 10
}
time.Sleep(time.Duration(waitIntervalCount) * 337 * time.Millisecond)
// Check if the client has disconnected by monitoring the context
select {
case <-ctx.Done():
err := ctx.Err()
if err == context.Canceled {
// Client disconnected
return nil
}
glog.V(0).Infof("Subscriber %s disconnected: %v", clientName, err)
return nil
default:
// Continue processing the request
}
}
clientName := fmt.Sprintf("%s/%s-%s", req.GetInit().ConsumerGroup, req.GetInit().ConsumerId, req.GetInit().ClientId)
localTopicPartition.Subscribers.AddSubscriber(clientName, topic.NewLocalSubscriber())
glog.V(0).Infof("Subscriber %s connected on %v %v", clientName, t, partition)
isConnected := true
sleepIntervalCount := 0
var counter int64
defer func() {
isConnected = false
localTopicPartition.Subscribers.RemoveSubscriber(clientName)
glog.V(0).Infof("Subscriber %s on %v %v disconnected", clientName, t, partition)
glog.V(0).Infof("Subscriber %s on %v %v disconnected, sent %d", clientName, t, partition, counter)
if localTopicPartition.MaybeShutdownLocalPartition() {
b.localTopicManager.RemoveTopicPartition(t, partition)
}
}()
ctx := stream.Context()
var startTime time.Time
if startTs := req.GetInit().GetStartTimestampNs(); startTs > 0 {
startTime = time.Unix(0, startTs)
} else {
startTime = time.Now()
var startPosition log_buffer.MessagePosition
if req.GetInit() != nil && req.GetInit().GetPartitionOffset() != nil {
startPosition = getRequestPosition(req.GetInit().GetPartitionOffset())
}
localTopicPartition.Subscribe(clientName, startTime, func() bool {
return localTopicPartition.Subscribe(clientName, startPosition, func() bool {
if !isConnected {
return false
}
sleepIntervalCount++
if sleepIntervalCount > 10 {
sleepIntervalCount = 10
if sleepIntervalCount > 32 {
sleepIntervalCount = 32
}
time.Sleep(time.Duration(sleepIntervalCount) * 2339 * time.Millisecond)
time.Sleep(time.Duration(sleepIntervalCount) * 137 * time.Millisecond)
// Check if the client has disconnected by monitoring the context
select {
@ -70,23 +98,205 @@ func (b *MessageQueueBroker) Subscribe(req *mq_pb.SubscribeRequest, stream mq_pb
}
return true
}, func(logEntry *filer_pb.LogEntry) error {
}, func(logEntry *filer_pb.LogEntry) (bool, error) {
// reset the sleep interval count
sleepIntervalCount = 0
value := logEntry.GetData()
if err := stream.Send(&mq_pb.SubscribeResponse{Message: &mq_pb.SubscribeResponse_Data{
if err := stream.Send(&mq_pb.SubscribeMessageResponse{Message: &mq_pb.SubscribeMessageResponse_Data{
Data: &mq_pb.DataMessage{
Key: []byte(fmt.Sprintf("key-%d", logEntry.PartitionKeyHash)),
Value: value,
Key: logEntry.Key,
Value: logEntry.Data,
TsNs: logEntry.TsNs,
},
}}); err != nil {
glog.Errorf("Error sending data: %v", err)
return false, err
}
counter++
return false, nil
})
}
func getRequestPosition(offset *mq_pb.PartitionOffset) (startPosition log_buffer.MessagePosition) {
if offset.StartTsNs != 0 {
startPosition = log_buffer.NewMessagePosition(offset.StartTsNs, -2)
}
if offset.StartType == mq_pb.PartitionOffsetStartType_EARLIEST {
startPosition = log_buffer.NewMessagePosition(1, -3)
} else if offset.StartType == mq_pb.PartitionOffsetStartType_LATEST {
startPosition = log_buffer.NewMessagePosition(time.Now().UnixNano(), -4)
}
return
}
func (b *MessageQueueBroker) FollowInMemoryMessages(req *mq_pb.FollowInMemoryMessagesRequest, stream mq_pb.SeaweedMessaging_FollowInMemoryMessagesServer) (err error) {
ctx := stream.Context()
clientName := req.GetInit().ConsumerId
t := topic.FromPbTopic(req.GetInit().Topic)
partition := topic.FromPbPartition(req.GetInit().GetPartitionOffset().GetPartition())
glog.V(0).Infof("FollowInMemoryMessages %s on %v %v connected", clientName, t, partition)
waitIntervalCount := 0
var localTopicPartition *topic.LocalPartition
for localTopicPartition == nil {
localTopicPartition, _, err = b.GetOrGenLocalPartition(t, partition)
if err != nil {
glog.V(1).Infof("topic %v partition %v not setup", t, partition)
}
if localTopicPartition != nil {
break
}
waitIntervalCount++
if waitIntervalCount > 32 {
waitIntervalCount = 32
}
time.Sleep(time.Duration(waitIntervalCount) * 137 * time.Millisecond)
// Check if the client has disconnected by monitoring the context
select {
case <-ctx.Done():
err := ctx.Err()
if err == context.Canceled {
// Client disconnected
return nil
}
glog.V(0).Infof("FollowInMemoryMessages %s disconnected: %v", clientName, err)
return nil
default:
// Continue processing the request
}
}
// set the current follower id
followerId := req.GetInit().FollowerId
atomic.StoreInt32(&localTopicPartition.FollowerId, followerId)
glog.V(0).Infof("FollowInMemoryMessages %s connected on %v %v", clientName, t, partition)
isConnected := true
sleepIntervalCount := 0
var counter int64
defer func() {
isConnected = false
glog.V(0).Infof("FollowInMemoryMessages %s on %v %v disconnected, sent %d", clientName, t, partition, counter)
}()
// send first hello message
// to indicate the follower is connected
stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{
Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{},
},
})
var startPosition log_buffer.MessagePosition
if req.GetInit() != nil && req.GetInit().GetPartitionOffset() != nil {
startPosition = getRequestPosition(req.GetInit().GetPartitionOffset())
}
var prevFlushTsNs int64
_, _, err = localTopicPartition.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, func() bool {
if !isConnected {
return false
}
sleepIntervalCount++
if sleepIntervalCount > 32 {
sleepIntervalCount = 32
}
time.Sleep(time.Duration(sleepIntervalCount) * 137 * time.Millisecond)
if localTopicPartition.LogBuffer.IsStopping() {
newFollowerId := atomic.LoadInt32(&localTopicPartition.FollowerId)
glog.V(0).Infof("FollowInMemoryMessages1 %s on %v %v follower id changed to %d", clientName, t, partition, newFollowerId)
stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{
Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{
FollowerChangedToId: newFollowerId,
},
},
})
return false
}
// Check if the client has disconnected by monitoring the context
select {
case <-ctx.Done():
err := ctx.Err()
if err == context.Canceled {
// Client disconnected
return false
}
glog.V(0).Infof("Subscriber %s disconnected: %v", clientName, err)
return false
default:
// Continue processing the request
}
// send the last flushed sequence
flushTsNs := atomic.LoadInt64(&localTopicPartition.LogBuffer.LastFlushTsNs)
if flushTsNs != prevFlushTsNs {
prevFlushTsNs = flushTsNs
stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{
Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{
FlushedSequence: flushTsNs,
},
},
})
}
return true
}, func(logEntry *filer_pb.LogEntry) (bool, error) {
// reset the sleep interval count
sleepIntervalCount = 0
// check the follower id
newFollowerId := atomic.LoadInt32(&localTopicPartition.FollowerId)
if newFollowerId != followerId {
glog.V(0).Infof("FollowInMemoryMessages2 %s on %v %v follower id %d changed to %d", clientName, t, partition, followerId, newFollowerId)
stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{
Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{
FollowerChangedToId: newFollowerId,
},
},
})
return true, nil
}
// send the last flushed sequence
flushTsNs := atomic.LoadInt64(&localTopicPartition.LogBuffer.LastFlushTsNs)
if flushTsNs != prevFlushTsNs {
prevFlushTsNs = flushTsNs
stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
Message: &mq_pb.FollowInMemoryMessagesResponse_Ctrl{
Ctrl: &mq_pb.FollowInMemoryMessagesResponse_CtrlMessage{
FlushedSequence: flushTsNs,
},
},
})
}
// send the log entry
if err := stream.Send(&mq_pb.FollowInMemoryMessagesResponse{
Message: &mq_pb.FollowInMemoryMessagesResponse_Data{
Data: &mq_pb.DataMessage{
Key: logEntry.Key,
Value: logEntry.Data,
TsNs: logEntry.TsNs,
},
}}); err != nil {
glog.Errorf("Error sending setup response: %v", err)
return err
return false, err
}
return nil
counter++
return false, nil
})
return nil
return err
}

39
weed/mq/broker/broker_grpc_sub_coordinator.go

@ -4,6 +4,7 @@ import (
"context"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/sub_coordinator"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
@ -11,7 +12,7 @@ import (
// SubscriberToSubCoordinator coordinates the subscribers
func (b *MessageQueueBroker) SubscriberToSubCoordinator(stream mq_pb.SeaweedMessaging_SubscriberToSubCoordinatorServer) error {
if !b.lockAsBalancer.IsLocked() {
if !b.isLockOwner() {
return status.Errorf(codes.Unavailable, "not current broker balancer")
}
req, err := stream.Recv()
@ -23,24 +24,43 @@ func (b *MessageQueueBroker) SubscriberToSubCoordinator(stream mq_pb.SeaweedMess
// process init message
initMessage := req.GetInit()
if initMessage != nil {
cgi = b.Coordinator.AddSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic)
glog.V(0).Infof("subscriber %s/%s/%s connected", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic)
cgi = b.Coordinator.AddSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic)
glog.V(0).Infof("subscriber %s/%s/%s connected", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic)
} else {
return status.Errorf(codes.InvalidArgument, "subscriber init message is empty")
}
defer func() {
b.Coordinator.RemoveSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic)
glog.V(0).Infof("subscriber %s/%s/%s disconnected: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err)
b.Coordinator.RemoveSubscriber(initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic)
glog.V(0).Infof("subscriber %s/%s/%s disconnected: %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, err)
}()
ctx := stream.Context()
// process ack messages
go func() {
// try to load the partition assignment from filer
if conf, err := b.readTopicConfFromFiler(topic.FromPbTopic(initMessage.Topic)); err == nil {
assignedPartitions := make([]*mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition, len(conf.BrokerPartitionAssignments))
for i, assignment := range conf.BrokerPartitionAssignments {
assignedPartitions[i] = &mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition{
Partition: assignment.Partition,
Broker: assignment.LeaderBroker,
}
}
// send partition assignment to subscriber
cgi.ResponseChan <- &mq_pb.SubscriberToSubCoordinatorResponse{
Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{
Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{
AssignedPartitions: assignedPartitions,
},
},
}
}
// process ack messages
for {
_, err := stream.Recv()
if err != nil {
glog.V(0).Infof("subscriber %s/%s/%s receive: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err)
glog.V(0).Infof("subscriber %s/%s/%s receive: %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, err)
}
select {
@ -66,11 +86,12 @@ func (b *MessageQueueBroker) SubscriberToSubCoordinator(stream mq_pb.SeaweedMess
// Client disconnected
return err
}
glog.V(0).Infof("subscriber %s/%s/%s disconnected: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err)
glog.V(0).Infof("subscriber %s/%s/%s disconnected: %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, err)
return err
case message := <-cgi.ResponseChan:
glog.V(0).Infof("subscriber %s/%s/%s send: %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, message)
if err := stream.Send(message); err != nil {
glog.V(0).Infof("subscriber %s/%s/%s send: %v", initMessage.ConsumerGroup, initMessage.ConsumerInstanceId, initMessage.Topic, err)
glog.V(0).Infof("subscriber %s/%s/%s send: %v", initMessage.ConsumerGroup, initMessage.ConsumerGroupInstanceId, initMessage.Topic, err)
}
}
}

33
weed/mq/broker/broker_server.go

@ -1,11 +1,11 @@
package broker
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/sub_coordinator"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"sync"
"time"
"github.com/seaweedfs/seaweedfs/weed/cluster"
@ -28,6 +28,11 @@ type MessageQueueBrokerOption struct {
Ip string
Port int
Cipher bool
VolumeServerAccess string // how to access volume servers
}
func (option *MessageQueueBrokerOption) BrokerAddress() pb.ServerAddress {
return pb.NewServerAddress(option.Ip, option.Port, 0)
}
type MessageQueueBroker struct {
@ -40,24 +45,28 @@ type MessageQueueBroker struct {
localTopicManager *topic.LocalTopicManager
Balancer *pub_balancer.Balancer
lockAsBalancer *cluster.LiveLock
currentBalancer pb.ServerAddress
Coordinator *sub_coordinator.Coordinator
accessLock sync.Mutex
}
func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.DialOption) (mqBroker *MessageQueueBroker, err error) {
pub_broker_balancer := pub_balancer.NewBalancer()
coordinator := sub_coordinator.NewCoordinator(pub_broker_balancer)
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, option.BrokerAddress(), option.DataCenter, option.Rack, *pb.NewServiceDiscoveryFromMap(option.Masters)),
filers: make(map[pb.ServerAddress]struct{}),
localTopicManager: topic.NewLocalTopicManager(),
Balancer: pub_broker_balancer,
Coordinator: sub_coordinator.NewCoordinator(pub_broker_balancer),
Coordinator: coordinator,
}
mqBroker.MasterClient.SetOnPeerUpdateFn(mqBroker.OnBrokerUpdate)
pub_broker_balancer.OnPartitionChange = mqBroker.Coordinator.OnPartitionChange
pub_broker_balancer.OnAddBroker = mqBroker.Coordinator.OnSubAddBroker
pub_broker_balancer.OnRemoveBroker = mqBroker.Coordinator.OnSubRemoveBroker
go mqBroker.MasterClient.KeepConnectedToMaster()
@ -71,18 +80,16 @@ func NewMessageBroker(option *MessageQueueBrokerOption, grpcDialOption grpc.Dial
for mqBroker.currentFiler == "" {
time.Sleep(time.Millisecond * 237)
}
self := fmt.Sprintf("%s:%d", option.Ip, option.Port)
self := option.BrokerAddress()
glog.V(0).Infof("broker %s found filer %s", self, mqBroker.currentFiler)
newBrokerBalancerCh := make(chan string, 1)
lockClient := cluster.NewLockClient(grpcDialOption, mqBroker.currentFiler)
mqBroker.lockAsBalancer = lockClient.StartLock(pub_balancer.LockBrokerBalancer, self)
for {
err := mqBroker.BrokerConnectToBalancer(self)
if err != nil {
fmt.Printf("BrokerConnectToBalancer: %v\n", err)
}
time.Sleep(time.Second)
}
mqBroker.lockAsBalancer = lockClient.StartLongLivedLock(pub_balancer.LockBrokerBalancer, string(self), func(newLockOwner string) {
glog.V(0).Infof("broker %s found balanacer %s", self, newLockOwner)
newBrokerBalancerCh <- newLockOwner
})
mqBroker.KeepConnectedToBrokerBalancer(newBrokerBalancerCh)
}()
return mqBroker, nil

101
weed/mq/broker/broker_topic_conf_read_write.go

@ -0,0 +1,101 @@
package broker
import (
"bytes"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
jsonpb "google.golang.org/protobuf/encoding/protojson"
)
func (b *MessageQueueBroker) saveTopicConfToFiler(t *mq_pb.Topic, conf *mq_pb.ConfigureTopicResponse) error {
glog.V(0).Infof("save conf for topic %v to filer", t)
// save the topic configuration on filer
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
if err := b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
var buf bytes.Buffer
filer.ProtoToText(&buf, conf)
return filer.SaveInsideFiler(client, topicDir, "topic.conf", buf.Bytes())
}); err != nil {
return fmt.Errorf("save topic to %s: %v", topicDir, err)
}
return nil
}
// readTopicConfFromFiler reads the topic configuration from filer
// this should only be run in broker leader, to ensure correct active broker list.
func (b *MessageQueueBroker) readTopicConfFromFiler(t topic.Topic) (conf *mq_pb.ConfigureTopicResponse, err error) {
glog.V(0).Infof("load conf for topic %v from filer", t)
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
if err = b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
data, err := filer.ReadInsideFiler(client, topicDir, "topic.conf")
if err == filer_pb.ErrNotFound {
return err
}
if err != nil {
return fmt.Errorf("read topic.conf of %v: %v", t, err)
}
// parse into filer conf object
conf = &mq_pb.ConfigureTopicResponse{}
if err = jsonpb.Unmarshal(data, conf); err != nil {
return fmt.Errorf("unmarshal topic %v conf: %v", t, err)
}
return nil
}); err != nil {
return nil, err
}
return conf, nil
}
func (b *MessageQueueBroker) GetOrGenLocalPartition(t topic.Topic, partition topic.Partition) (localPartition *topic.LocalPartition, isGenerated bool, err error) {
b.accessLock.Lock()
defer b.accessLock.Unlock()
if localPartition = b.localTopicManager.GetTopicPartition(t, partition); localPartition == nil {
localPartition, isGenerated, err = b.genLocalPartitionFromFiler(t, partition)
if err != nil {
return nil, false, err
}
}
return localPartition, isGenerated, nil
}
func (b *MessageQueueBroker) genLocalPartitionFromFiler(t topic.Topic, partition topic.Partition) (localPartition *topic.LocalPartition, isGenerated bool, err error) {
self := b.option.BrokerAddress()
conf, err := b.readTopicConfFromFiler(t)
if err != nil {
return nil, isGenerated, err
}
for _, assignment := range conf.BrokerPartitionAssignments {
if assignment.LeaderBroker == string(self) && partition.Equals(topic.FromPbPartition(assignment.Partition)) {
localPartition = topic.FromPbBrokerPartitionAssignment(b.option.BrokerAddress(), partition, assignment, b.genLogFlushFunc(t, assignment.Partition), b.genLogOnDiskReadFunc(t, assignment.Partition))
b.localTopicManager.AddTopicPartition(t, localPartition)
isGenerated = true
break
}
}
return localPartition, isGenerated, nil
}
func (b *MessageQueueBroker) ensureTopicActiveAssignments(t topic.Topic, conf *mq_pb.ConfigureTopicResponse) (err error) {
// also fix assignee broker if invalid
hasChanges := pub_balancer.EnsureAssignmentsToActiveBrokers(b.Balancer.Brokers, 1, conf.BrokerPartitionAssignments)
if hasChanges {
glog.V(0).Infof("topic %v partition updated assignments: %v", t, conf.BrokerPartitionAssignments)
if err = b.saveTopicConfToFiler(t.ToPbTopic(), conf); err != nil {
return err
}
}
return err
}

168
weed/mq/broker/broker_topic_partition_read_write.go

@ -0,0 +1,168 @@
package broker
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/filer"
"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"
"github.com/seaweedfs/seaweedfs/weed/util"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"google.golang.org/protobuf/proto"
"math"
"sync/atomic"
"time"
)
func (b *MessageQueueBroker) genLogFlushFunc(t topic.Topic, partition *mq_pb.Partition) log_buffer.LogFlushFuncType {
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
partitionGeneration := time.Unix(0, partition.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, partition.RangeStart, partition.RangeStop)
return func(logBuffer *log_buffer.LogBuffer, startTime, stopTime time.Time, buf []byte) {
if len(buf) == 0 {
return
}
startTime, stopTime = startTime.UTC(), stopTime.UTC()
targetFile := fmt.Sprintf("%s/%s", partitionDir, startTime.Format(topic.TIME_FORMAT))
// TODO append block with more metadata
for {
if err := b.appendToFile(targetFile, buf); err != nil {
glog.V(0).Infof("metadata log write failed %s: %v", targetFile, err)
time.Sleep(737 * time.Millisecond)
} else {
break
}
}
atomic.StoreInt64(&logBuffer.LastFlushTsNs, stopTime.UnixNano())
}
}
func (b *MessageQueueBroker) genLogOnDiskReadFunc(t topic.Topic, partition *mq_pb.Partition) log_buffer.LogReadFromDiskFuncType {
topicDir := fmt.Sprintf("%s/%s/%s", filer.TopicsDir, t.Namespace, t.Name)
partitionGeneration := time.Unix(0, partition.UnixTimeNs).UTC().Format(topic.TIME_FORMAT)
partitionDir := fmt.Sprintf("%s/%s/%04d-%04d", topicDir, partitionGeneration, partition.RangeStart, partition.RangeStop)
lookupFileIdFn := func(fileId string) (targetUrls []string, err error) {
return b.MasterClient.LookupFileId(fileId)
}
eachChunkFn := func(buf []byte, eachLogEntryFn log_buffer.EachLogEntryFuncType, starTsNs, stopTsNs int64) (processedTsNs int64, err error) {
for pos := 0; pos+4 < len(buf); {
size := util.BytesToUint32(buf[pos : pos+4])
if pos+4+int(size) > len(buf) {
err = fmt.Errorf("LogOnDiskReadFunc: read [%d,%d) from [0,%d)", pos, pos+int(size)+4, len(buf))
return
}
entryData := buf[pos+4 : pos+4+int(size)]
logEntry := &filer_pb.LogEntry{}
if err = proto.Unmarshal(entryData, logEntry); err != nil {
pos += 4 + int(size)
err = fmt.Errorf("unexpected unmarshal mq_pb.Message: %v", err)
return
}
if logEntry.TsNs < starTsNs {
pos += 4 + int(size)
continue
}
if stopTsNs != 0 && logEntry.TsNs > stopTsNs {
println("stopTsNs", stopTsNs, "logEntry.TsNs", logEntry.TsNs)
return
}
if _, err = eachLogEntryFn(logEntry); err != nil {
err = fmt.Errorf("process log entry %v: %v", logEntry, err)
return
}
processedTsNs = logEntry.TsNs
pos += 4 + int(size)
}
return
}
eachFileFn := func(entry *filer_pb.Entry, eachLogEntryFn log_buffer.EachLogEntryFuncType, starTsNs, stopTsNs int64) (processedTsNs int64, err error) {
if len(entry.Content) > 0 {
glog.Warningf("this should not happen. unexpected content in %s/%s", partitionDir, entry.Name)
return
}
var urlStrings []string
for _, chunk := range entry.Chunks {
if chunk.Size == 0 {
continue
}
if chunk.IsChunkManifest {
glog.Warningf("this should not happen. unexpected chunk manifest in %s/%s", partitionDir, entry.Name)
return
}
urlStrings, err = lookupFileIdFn(chunk.FileId)
if err != nil {
err = fmt.Errorf("lookup %s: %v", chunk.FileId, err)
return
}
if len(urlStrings) == 0 {
err = fmt.Errorf("no url found for %s", chunk.FileId)
return
}
// try one of the urlString until util.Get(urlString) succeeds
var processed bool
for _, urlString := range urlStrings {
// TODO optimization opportunity: reuse the buffer
var data []byte
if data, _, err = util.Get(urlString); err == nil {
processed = true
if processedTsNs, err = eachChunkFn(data, eachLogEntryFn, starTsNs, stopTsNs); err != nil {
return
}
break
}
}
if !processed {
err = fmt.Errorf("no data processed for %s %s", entry.Name, chunk.FileId)
return
}
}
return
}
return func(startPosition log_buffer.MessagePosition, stopTsNs int64, eachLogEntryFn log_buffer.EachLogEntryFuncType) (lastReadPosition log_buffer.MessagePosition, isDone bool, err error) {
startFileName := startPosition.UTC().Format(topic.TIME_FORMAT)
startTsNs := startPosition.Time.UnixNano()
stopTime := time.Unix(0, stopTsNs)
var processedTsNs int64
err = b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
return filer_pb.SeaweedList(client, partitionDir, "", func(entry *filer_pb.Entry, isLast bool) error {
if entry.IsDirectory {
return nil
}
if stopTsNs != 0 && entry.Name > stopTime.UTC().Format(topic.TIME_FORMAT) {
isDone = true
return nil
}
if entry.Name < startPosition.UTC().Format(topic.TIME_FORMAT) {
return nil
}
if processedTsNs, err = eachFileFn(entry, eachLogEntryFn, startTsNs, stopTsNs); err != nil {
return err
}
return nil
}, startFileName, true, math.MaxInt32)
})
lastReadPosition = log_buffer.NewMessagePosition(processedTsNs, -2)
return
}
}

82
weed/mq/broker/broker_write.go

@ -0,0 +1,82 @@
package broker
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/filer"
"github.com/seaweedfs/seaweedfs/weed/operation"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
"github.com/seaweedfs/seaweedfs/weed/util"
"os"
"time"
)
func (b *MessageQueueBroker) appendToFile(targetFile string, data []byte) error {
fileId, uploadResult, err2 := b.assignAndUpload(targetFile, data)
if err2 != nil {
return err2
}
// find out existing entry
fullpath := util.FullPath(targetFile)
dir, name := fullpath.DirAndName()
entry, err := filer_pb.GetEntry(b, fullpath)
var offset int64 = 0
if err == filer_pb.ErrNotFound {
entry = &filer_pb.Entry{
Name: name,
IsDirectory: false,
Attributes: &filer_pb.FuseAttributes{
Crtime: time.Now().Unix(),
Mtime: time.Now().Unix(),
FileMode: uint32(os.FileMode(0644)),
Uid: uint32(os.Getuid()),
Gid: uint32(os.Getgid()),
},
}
} else if err != nil {
return fmt.Errorf("find %s: %v", fullpath, err)
} else {
offset = int64(filer.TotalSize(entry.GetChunks()))
}
// append to existing chunks
entry.Chunks = append(entry.GetChunks(), uploadResult.ToPbFileChunk(fileId, offset, time.Now().UnixNano()))
// update the entry
return b.WithFilerClient(false, func(client filer_pb.SeaweedFilerClient) error {
return filer_pb.CreateEntry(client, &filer_pb.CreateEntryRequest{
Directory: dir,
Entry: entry,
})
})
}
func (b *MessageQueueBroker) assignAndUpload(targetFile string, data []byte) (fileId string, uploadResult *operation.UploadResult, err error) {
reader := util.NewBytesReader(data)
fileId, uploadResult, err, _ = operation.UploadWithRetry(
b,
&filer_pb.AssignVolumeRequest{
Count: 1,
Replication: b.option.DefaultReplication,
Collection: "topics",
// TtlSec: wfs.option.TtlSec,
// DiskType: string(wfs.option.DiskType),
DataCenter: b.option.DataCenter,
Path: targetFile,
},
&operation.UploadOption{
Cipher: b.option.Cipher,
},
func(host, fileId string) string {
fileUrl := fmt.Sprintf("http://%s/%s", host, fileId)
if b.option.VolumeServerAccess == "filerProxy" {
fileUrl = fmt.Sprintf("http://%s/?proxyChunkId=%s", b.currentFiler, fileId)
}
return fileUrl
},
reader,
)
return
}

27
weed/mq/client/cmd/weed_pub/publisher.go

@ -4,6 +4,7 @@ import (
"flag"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/mq/client/pub_client"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"log"
"strings"
"sync"
@ -11,11 +12,12 @@ import (
)
var (
messageCount = flag.Int("n", 1000, "message count")
concurrency = flag.Int("c", 4, "concurrency count")
messageCount = flag.Int("n", 1000, "message count")
concurrency = flag.Int("c", 4, "concurrent publishers")
partitionCount = flag.Int("p", 6, "partition count")
namespace = flag.String("ns", "test", "namespace")
topic = flag.String("topic", "test", "topic")
t = flag.String("t", "test", "t")
seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
)
@ -25,7 +27,10 @@ func doPublish(publisher *pub_client.TopicPublisher, id int) {
// 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
if err := publisher.Publish(key, value); err != nil {
fmt.Println(err)
break
}
// println("Published", string(key), string(value))
}
elapsed := time.Since(startTime)
@ -35,19 +40,17 @@ func doPublish(publisher *pub_client.TopicPublisher, id int) {
func main() {
flag.Parse()
config := &pub_client.PublisherConfiguration{
CreateTopic: true,
}
publisher := pub_client.NewTopicPublisher(*namespace, *topic, config)
brokers := strings.Split(*seedBrokers, ",")
if err := publisher.Connect(brokers); err != nil {
fmt.Println(err)
return
Topic: topic.NewTopic(*namespace, *t),
CreateTopic: true,
CreateTopicPartitionCount: int32(*partitionCount),
Brokers: strings.Split(*seedBrokers, ","),
}
publisher := pub_client.NewTopicPublisher(config)
startTime := time.Now()
// Start multiple publishers
var wg sync.WaitGroup
// Start multiple publishers
for i := 0; i < *concurrency; i++ {
wg.Add(1)
go func(id int) {

36
weed/mq/client/cmd/weed_sub/subscriber.go

@ -3,7 +3,10 @@ package main
import (
"flag"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/client/sub_client"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/util"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"strings"
@ -12,37 +15,44 @@ import (
var (
namespace = flag.String("ns", "test", "namespace")
topic = flag.String("topic", "test", "topic")
t = flag.String("topic", "test", "topic")
seedBrokers = flag.String("brokers", "localhost:17777", "seed brokers")
clientId = flag.Uint("client_id", uint(util.RandomInt32()), "client id")
)
func main() {
flag.Parse()
subscriberConfig := &sub_client.SubscriberConfiguration{
ClientId: "testSubscriber",
GroupId: "test",
GroupInstanceId: "test",
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
ClientId: fmt.Sprintf("client-%d", *clientId),
ConsumerGroup: "test",
ConsumerGroupInstanceId: fmt.Sprintf("client-%d", *clientId),
GrpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
}
contentConfig := &sub_client.ContentConfiguration{
Namespace: *namespace,
Topic: *topic,
Topic: topic.NewTopic(*namespace, *t),
Filter: "",
StartTime: time.Now(),
StartTime: time.Unix(1, 1),
}
processorConfig := sub_client.ProcessorConfiguration{
ConcurrentPartitionLimit: 3,
}
brokers := strings.Split(*seedBrokers, ",")
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig)
subscriber := sub_client.NewTopicSubscriber(brokers, subscriberConfig, contentConfig, processorConfig)
subscriber.SetEachMessageFunc(func(key, value []byte) bool {
println(string(key), "=>", string(value))
return true
counter := 0
subscriber.SetEachMessageFunc(func(key, value []byte) (bool, error) {
counter++
println(string(key), "=>", string(value), counter)
return true, nil
})
subscriber.SetCompletionFunc(func() {
println("done subscribing")
glog.V(0).Infof("done recived %d messages", counter)
})
if err := subscriber.Subscribe(); err != nil {

73
weed/mq/client/pub_client/connect.go

@ -1,73 +0,0 @@
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"
"log"
)
// broker => publish client
// send init message
// save the publishing client
func (p *TopicPublisher) doConnect(partition *mq_pb.Partition, brokerAddress string) (publishClient *PublishClient, err error) {
log.Printf("connecting to %v for topic partition %+v", brokerAddress, partition)
grpcConnection, err := pb.GrpcDial(context.Background(), brokerAddress, true, p.grpcDialOption)
if err != nil {
return publishClient, fmt.Errorf("dial broker %s: %v", brokerAddress, err)
}
brokerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection)
stream, err := brokerClient.Publish(context.Background())
if err != nil {
return publishClient, 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, fmt.Errorf("send init message: %v", err)
}
resp, err := stream.Recv()
if err != nil {
return publishClient, fmt.Errorf("recv init response: %v", err)
}
if resp.Error != "" {
return publishClient, fmt.Errorf("init response error: %v", resp.Error)
}
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, nil
}

84
weed/mq/client/pub_client/lookup.go

@ -1,84 +0,0 @@
package pub_client
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
)
func (p *TopicPublisher) doLookupAndConnect(brokerAddress string) error {
if p.config.CreateTopic {
err := pb.WithBrokerGrpcClient(true,
brokerAddress,
p.grpcDialOption,
func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
PartitionCount: p.config.CreateTopicPartitionCount,
})
return err
})
if err != nil {
return fmt.Errorf("configure topic %s/%s: %v", p.namespace, p.topic, err)
}
}
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 p.config.CreateTopic && err != nil {
_, err = client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
Topic: &mq_pb.Topic{
Namespace: p.namespace,
Name: p.topic,
},
PartitionCount: p.config.CreateTopicPartitionCount,
})
if err != nil {
return err
}
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, err := p.doConnect(brokerPartitionAssignment.Partition, brokerPartitionAssignment.LeaderBroker)
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
}

31
weed/mq/client/pub_client/publish.go

@ -12,30 +12,13 @@ func (p *TopicPublisher) Publish(key, value []byte) error {
if hashKey < 0 {
hashKey = -hashKey
}
publishClient, found := p.partition2Broker.Floor(hashKey, hashKey)
inputBuffer, found := p.partition2Buffer.Floor(hashKey+1, hashKey+1)
if !found {
return fmt.Errorf("no broker found for key %d", hashKey)
return fmt.Errorf("no input buffer 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
return inputBuffer.Enqueue(&mq_pb.DataMessage{
Key: key,
Value: value,
})
}

56
weed/mq/client/pub_client/publisher.go

@ -1,69 +1,71 @@
package pub_client
import (
"fmt"
"github.com/rdleal/intervalst/interval"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/buffered_queue"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials/insecure"
"log"
"sync"
"time"
)
type PublisherConfiguration struct {
Topic topic.Topic
CreateTopic bool
CreateTopicPartitionCount int32
Brokers []string
}
type PublishClient struct {
mq_pb.SeaweedMessaging_PublishClient
mq_pb.SeaweedMessaging_PublishMessageClient
Broker string
Err error
}
type TopicPublisher struct {
namespace string
topic string
partition2Broker *interval.SearchTree[*PublishClient, int32]
partition2Buffer *interval.SearchTree[*buffered_queue.BufferedQueue[*mq_pb.DataMessage], int32]
grpcDialOption grpc.DialOption
sync.Mutex // protects grpc
config *PublisherConfiguration
jobs []*EachPartitionPublishJob
}
func NewTopicPublisher(namespace, topic string, config *PublisherConfiguration) *TopicPublisher {
return &TopicPublisher{
namespace: namespace,
topic: topic,
partition2Broker: interval.NewSearchTree[*PublishClient](func(a, b int32) int {
func NewTopicPublisher(config *PublisherConfiguration) *TopicPublisher {
tp := &TopicPublisher{
partition2Buffer: interval.NewSearchTree[*buffered_queue.BufferedQueue[*mq_pb.DataMessage]](func(a, b int32) int {
return int(a - b)
}),
grpcDialOption: grpc.WithTransportCredentials(insecure.NewCredentials()),
config: config,
}
}
func (p *TopicPublisher) Connect(bootstrapBrokers []string) (err error) {
if len(bootstrapBrokers) == 0 {
return nil
}
for _, b := range bootstrapBrokers {
err = p.doLookupAndConnect(b)
if err == nil {
return nil
wg := sync.WaitGroup{}
wg.Add(1)
go func() {
if err := tp.startSchedulerThread(&wg); err != nil {
log.Println(err)
return
}
fmt.Printf("failed to connect to %s: %v\n\n", b, err)
}
return err
}()
wg.Wait()
return tp
}
func (p *TopicPublisher) Shutdown() error {
if clients, found := p.partition2Broker.AllIntersections(0, pub_balancer.MaxPartitionCount); found {
for _, client := range clients {
client.CloseSend()
if inputBuffers, found := p.partition2Buffer.AllIntersections(0, pub_balancer.MaxPartitionCount); found {
for _, inputBuffer := range inputBuffers {
inputBuffer.CloseInput()
}
}
time.Sleep(1100 * time.Millisecond)
for _, job := range p.jobs {
job.wg.Wait()
}
return nil
}

273
weed/mq/client/pub_client/scheduler.go

@ -0,0 +1,273 @@
package pub_client
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/seaweedfs/seaweedfs/weed/util/buffered_queue"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"log"
"sort"
"sync"
"time"
)
type EachPartitionError struct {
*mq_pb.BrokerPartitionAssignment
Err error
generation int
}
type EachPartitionPublishJob struct {
*mq_pb.BrokerPartitionAssignment
stopChan chan bool
wg sync.WaitGroup
generation int
inputQueue *buffered_queue.BufferedQueue[*mq_pb.DataMessage]
}
func (p *TopicPublisher) startSchedulerThread(wg *sync.WaitGroup) error {
if err := p.doConfigureTopic(); err != nil {
return fmt.Errorf("configure topic %s: %v", p.config.Topic, err)
}
log.Printf("start scheduler thread for topic %s", p.config.Topic)
generation := 0
var errChan chan EachPartitionError
for {
glog.V(0).Infof("lookup partitions gen %d topic %s", generation+1, p.config.Topic)
if assignments, err := p.doLookupTopicPartitions(); err == nil {
generation++
glog.V(0).Infof("start generation %d with %d assignments", generation, len(assignments))
if errChan == nil {
errChan = make(chan EachPartitionError, len(assignments))
}
p.onEachAssignments(generation, assignments, errChan)
} else {
glog.Errorf("lookup topic %s: %v", p.config.Topic, err)
time.Sleep(5 * time.Second)
continue
}
if generation == 1 {
wg.Done()
}
// wait for any error to happen. If so, consume all remaining errors, and retry
for {
select {
case eachErr := <-errChan:
glog.Errorf("gen %d publish to topic %s partition %v: %v", eachErr.generation, p.config.Topic, eachErr.Partition, eachErr.Err)
if eachErr.generation < generation {
continue
}
break
}
}
}
}
func (p *TopicPublisher) onEachAssignments(generation int, assignments []*mq_pb.BrokerPartitionAssignment, errChan chan EachPartitionError) {
// TODO assuming this is not re-configured so the partitions are fixed.
sort.Slice(assignments, func(i, j int) bool {
return assignments[i].Partition.RangeStart < assignments[j].Partition.RangeStart
})
var jobs []*EachPartitionPublishJob
hasExistingJob := len(p.jobs) == len(assignments)
for i, assignment := range assignments {
if assignment.LeaderBroker == "" {
continue
}
if hasExistingJob {
var existingJob *EachPartitionPublishJob
existingJob = p.jobs[i]
if existingJob.BrokerPartitionAssignment.LeaderBroker == assignment.LeaderBroker {
existingJob.generation = generation
jobs = append(jobs, existingJob)
continue
} else {
if existingJob.LeaderBroker != "" {
close(existingJob.stopChan)
existingJob.LeaderBroker = ""
existingJob.wg.Wait()
}
}
}
// start a go routine to publish to this partition
job := &EachPartitionPublishJob{
BrokerPartitionAssignment: assignment,
stopChan: make(chan bool, 1),
generation: generation,
inputQueue: buffered_queue.NewBufferedQueue[*mq_pb.DataMessage](1024),
}
job.wg.Add(1)
go func(job *EachPartitionPublishJob) {
defer job.wg.Done()
if err := p.doPublishToPartition(job); err != nil {
errChan <- EachPartitionError{assignment, err, generation}
}
}(job)
jobs = append(jobs, job)
// TODO assuming this is not re-configured so the partitions are fixed.
// better just re-use the existing job
p.partition2Buffer.Insert(assignment.Partition.RangeStart, assignment.Partition.RangeStop, job.inputQueue)
}
p.jobs = jobs
}
func (p *TopicPublisher) doPublishToPartition(job *EachPartitionPublishJob) error {
log.Printf("connecting to %v for topic partition %+v", job.LeaderBroker, job.Partition)
grpcConnection, err := pb.GrpcDial(context.Background(), job.LeaderBroker, true, p.grpcDialOption)
if err != nil {
return fmt.Errorf("dial broker %s: %v", job.LeaderBroker, err)
}
brokerClient := mq_pb.NewSeaweedMessagingClient(grpcConnection)
stream, err := brokerClient.PublishMessage(context.Background())
if err != nil {
return fmt.Errorf("create publish client: %v", err)
}
publishClient := &PublishClient{
SeaweedMessaging_PublishMessageClient: stream,
Broker: job.LeaderBroker,
}
if err = publishClient.Send(&mq_pb.PublishMessageRequest{
Message: &mq_pb.PublishMessageRequest_Init{
Init: &mq_pb.PublishMessageRequest_InitMessage{
Topic: p.config.Topic.ToPbTopic(),
Partition: job.Partition,
AckInterval: 128,
FollowerBrokers: job.FollowerBrokers,
},
},
}); err != nil {
return fmt.Errorf("send init message: %v", err)
}
resp, err := stream.Recv()
if err != nil {
return fmt.Errorf("recv init response: %v", err)
}
if resp.Error != "" {
return fmt.Errorf("init response error: %v", resp.Error)
}
go func() {
for {
ackResp, err := publishClient.Recv()
if err != nil {
e, _ := status.FromError(err)
if e.Code() == codes.Unknown && e.Message() == "EOF" {
return
}
publishClient.Err = err
fmt.Printf("publish1 to %s error: %v\n", publishClient.Broker, err)
return
}
if ackResp.Error != "" {
publishClient.Err = fmt.Errorf("ack error: %v", ackResp.Error)
fmt.Printf("publish2 to %s error: %v\n", publishClient.Broker, ackResp.Error)
return
}
if ackResp.AckSequence > 0 {
log.Printf("ack %d", ackResp.AckSequence)
}
}
}()
publishCounter := 0
for data, hasData := job.inputQueue.Dequeue(); hasData; data, hasData = job.inputQueue.Dequeue() {
if err := publishClient.Send(&mq_pb.PublishMessageRequest{
Message: &mq_pb.PublishMessageRequest_Data{
Data: data,
},
}); err != nil {
return fmt.Errorf("send publish data: %v", err)
}
publishCounter++
}
if err := publishClient.CloseSend(); err != nil {
return fmt.Errorf("close send: %v", err)
}
time.Sleep(3 * time.Second)
log.Printf("published %d messages to %v for topic partition %+v", publishCounter, job.LeaderBroker, job.Partition)
return nil
}
func (p *TopicPublisher) doConfigureTopic() (err error) {
if len(p.config.Brokers) == 0 {
return fmt.Errorf("no bootstrap brokers")
}
var lastErr error
for _, brokerAddress := range p.config.Brokers {
err = pb.WithBrokerGrpcClient(false,
brokerAddress,
p.grpcDialOption,
func(client mq_pb.SeaweedMessagingClient) error {
_, err := client.ConfigureTopic(context.Background(), &mq_pb.ConfigureTopicRequest{
Topic: p.config.Topic.ToPbTopic(),
PartitionCount: p.config.CreateTopicPartitionCount,
})
return err
})
if err == nil {
return nil
} else {
lastErr = fmt.Errorf("%s: %v", brokerAddress, err)
}
}
if lastErr != nil {
return fmt.Errorf("doConfigureTopic %s: %v", p.config.Topic, err)
}
return nil
}
func (p *TopicPublisher) doLookupTopicPartitions() (assignments []*mq_pb.BrokerPartitionAssignment, err error) {
if len(p.config.Brokers) == 0 {
return nil, fmt.Errorf("no bootstrap brokers")
}
var lastErr error
for _, brokerAddress := range p.config.Brokers {
err := pb.WithBrokerGrpcClient(false,
brokerAddress,
p.grpcDialOption,
func(client mq_pb.SeaweedMessagingClient) error {
lookupResp, err := client.LookupTopicBrokers(context.Background(),
&mq_pb.LookupTopicBrokersRequest{
Topic: p.config.Topic.ToPbTopic(),
})
glog.V(0).Infof("lookup topic %s: %v", p.config.Topic, lookupResp)
if err != nil {
return err
}
if len(lookupResp.BrokerPartitionAssignments) == 0 {
return fmt.Errorf("no broker partition assignments")
}
assignments = lookupResp.BrokerPartitionAssignments
return nil
})
if err == nil {
return assignments, nil
} else {
lastErr = err
}
}
return nil, fmt.Errorf("lookup topic %s: %v", p.config.Topic, lastErr)
}

180
weed/mq/client/sub_client/connect_to_sub_coordinator.go

@ -0,0 +1,180 @@
package sub_client
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"io"
"sync"
"time"
)
func (sub *TopicSubscriber) doKeepConnectedToSubCoordinator() {
waitTime := 1 * time.Second
for {
for _, broker := range sub.bootstrapBrokers {
// lookup topic brokers
var brokerLeader string
err := pb.WithBrokerGrpcClient(false, broker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
resp, err := client.FindBrokerLeader(context.Background(), &mq_pb.FindBrokerLeaderRequest{})
if err != nil {
return err
}
brokerLeader = resp.Broker
return nil
})
if err != nil {
glog.V(0).Infof("broker coordinator on %s: %v", broker, err)
continue
}
glog.V(0).Infof("found broker coordinator: %v", brokerLeader)
// connect to the balancer
pb.WithBrokerGrpcClient(true, brokerLeader, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
stream, err := client.SubscriberToSubCoordinator(ctx)
if err != nil {
glog.V(0).Infof("subscriber %s: %v", sub.ContentConfig.Topic, err)
return err
}
waitTime = 1 * time.Second
// Maybe later: subscribe to multiple topics instead of just one
if err := stream.Send(&mq_pb.SubscriberToSubCoordinatorRequest{
Message: &mq_pb.SubscriberToSubCoordinatorRequest_Init{
Init: &mq_pb.SubscriberToSubCoordinatorRequest_InitMessage{
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
ConsumerGroupInstanceId: sub.SubscriberConfig.ConsumerGroupInstanceId,
Topic: sub.ContentConfig.Topic.ToPbTopic(),
},
},
}); err != nil {
glog.V(0).Infof("subscriber %s send init: %v", sub.ContentConfig.Topic, err)
return err
}
// keep receiving messages from the sub coordinator
for {
resp, err := stream.Recv()
if err != nil {
glog.V(0).Infof("subscriber %s receive: %v", sub.ContentConfig.Topic, err)
return err
}
assignment := resp.GetAssignment()
if assignment != nil {
glog.V(0).Infof("subscriber %s receive assignment: %v", sub.ContentConfig.Topic, assignment)
}
sub.onEachAssignment(assignment)
}
return nil
})
}
glog.V(0).Infof("subscriber %s/%s waiting for more assignments", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
if waitTime < 10*time.Second {
waitTime += 1 * time.Second
}
time.Sleep(waitTime)
}
}
func (sub *TopicSubscriber) onEachAssignment(assignment *mq_pb.SubscriberToSubCoordinatorResponse_Assignment) {
if assignment == nil {
return
}
// process each partition, with a concurrency limit
var wg sync.WaitGroup
semaphore := make(chan struct{}, sub.ProcessorConfig.ConcurrentPartitionLimit)
for _, assigned := range assignment.AssignedPartitions {
wg.Add(1)
semaphore <- struct{}{}
go func(partition *mq_pb.Partition, broker string) {
defer wg.Done()
defer func() { <-semaphore }()
glog.V(0).Infof("subscriber %s/%s assigned partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker)
err := sub.onEachPartition(partition, broker)
if err != nil {
glog.V(0).Infof("subscriber %s/%s partition %+v at %v: %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker, err)
}
}(assigned.Partition, assigned.Broker)
}
wg.Wait()
}
func (sub *TopicSubscriber) onEachPartition(partition *mq_pb.Partition, broker string) error {
// connect to the partition broker
return pb.WithBrokerGrpcClient(true, broker, sub.SubscriberConfig.GrpcDialOption, func(client mq_pb.SeaweedMessagingClient) error {
subscribeClient, err := client.SubscribeMessage(context.Background(), &mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Init{
Init: &mq_pb.SubscribeMessageRequest_InitMessage{
ConsumerGroup: sub.SubscriberConfig.ConsumerGroup,
ConsumerId: sub.SubscriberConfig.ConsumerGroupInstanceId,
Topic: sub.ContentConfig.Topic.ToPbTopic(),
PartitionOffset: &mq_pb.PartitionOffset{
Partition: partition,
StartTsNs: sub.alreadyProcessedTsNs,
StartType: mq_pb.PartitionOffsetStartType_EARLIEST_IN_MEMORY,
},
Filter: sub.ContentConfig.Filter,
},
},
})
if err != nil {
return fmt.Errorf("create subscribe client: %v", err)
}
glog.V(0).Infof("subscriber %s/%s connected to partition %+v at %v", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, partition, broker)
if sub.OnCompletionFunc != nil {
defer sub.OnCompletionFunc()
}
defer func() {
subscribeClient.SendMsg(&mq_pb.SubscribeMessageRequest{
Message: &mq_pb.SubscribeMessageRequest_Ack{
Ack: &mq_pb.SubscribeMessageRequest_AckMessage{
Sequence: 0,
},
},
})
subscribeClient.CloseSend()
}()
for {
// glog.V(0).Infof("subscriber %s/%s/%s waiting for message", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
resp, err := subscribeClient.Recv()
if err != nil {
return fmt.Errorf("subscribe recv: %v", err)
}
if resp.Message == nil {
glog.V(0).Infof("subscriber %s/%s received nil message", sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup)
continue
}
switch m := resp.Message.(type) {
case *mq_pb.SubscribeMessageResponse_Data:
shouldContinue, processErr := sub.OnEachMessageFunc(m.Data.Key, m.Data.Value)
if processErr != nil {
return fmt.Errorf("process error: %v", processErr)
}
sub.alreadyProcessedTsNs = m.Data.TsNs
if !shouldContinue {
return nil
}
case *mq_pb.SubscribeMessageResponse_Ctrl:
// glog.V(0).Infof("subscriber %s/%s/%s received control %+v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, sub.SubscriberConfig.ConsumerGroup, m.Ctrl)
if m.Ctrl.IsEndOfStream || m.Ctrl.IsEndOfTopic {
return io.EOF
}
}
}
return nil
})
}

34
weed/mq/client/sub_client/lookup.go

@ -1,34 +0,0 @@
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
}

85
weed/mq/client/sub_client/process.go

@ -1,85 +0,0 @@
package sub_client
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"sync"
)
func (sub *TopicSubscriber) doProcess() 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{
Message: &mq_pb.SubscribeRequest_Init{
Init: &mq_pb.SubscribeRequest_InitMessage{
ConsumerGroup: sub.SubscriberConfig.GroupId,
ConsumerId: sub.SubscriberConfig.GroupInstanceId,
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,
Offset: &mq_pb.SubscribeRequest_InitMessage_StartTimestampNs{
StartTimestampNs: sub.alreadyProcessedTsNs,
},
},
},
})
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()
}
defer func() {
subscribeClient.SendMsg(&mq_pb.SubscribeRequest{
Message: &mq_pb.SubscribeRequest_Ack{
Ack: &mq_pb.SubscribeRequest_AckMessage{
Sequence: 0,
},
},
})
subscribeClient.CloseSend()
}()
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
}
sub.alreadyProcessedTsNs = m.Data.TsNs
case *mq_pb.SubscribeResponse_Ctrl:
if m.Ctrl.IsEndOfStream || m.Ctrl.IsEndOfTopic {
return
}
}
}
}()
}
wg.Wait()
return nil
}

41
weed/mq/client/sub_client/subscribe.go

@ -1,48 +1,11 @@
package sub_client
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/util"
"io"
"log"
"time"
)
// 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 {
index := -1
util.RetryUntil("subscribe", func() error {
index++
index = index % len(sub.bootstrapBrokers)
// ask balancer for brokers of the topic
if err := sub.doLookup(sub.bootstrapBrokers[index]); err != nil {
return fmt.Errorf("lookup topic %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
}
if len(sub.brokerPartitionAssignments) == 0 {
if sub.waitForMoreMessage {
time.Sleep(1 * time.Second)
return fmt.Errorf("no broker partition assignments")
} else {
return nil
}
}
// treat the first broker as the topic leader
// connect to the leader broker
// loop forever
sub.doKeepConnectedToSubCoordinator()
// subscribe to the topic
if err := sub.doProcess(); err != nil {
return fmt.Errorf("subscribe topic %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
}
return nil
}, func(err error) bool {
if err == io.EOF {
log.Printf("subscriber %s/%s: %v", sub.ContentConfig.Namespace, sub.ContentConfig.Topic, err)
sub.waitForMoreMessage = false
return false
}
return true
})
return nil
}

28
weed/mq/client/sub_client/subscriber.go

@ -1,34 +1,39 @@
package sub_client
import (
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"google.golang.org/grpc"
"time"
)
type SubscriberConfiguration struct {
ClientId string
GroupId string
GroupInstanceId string
GroupMinimumPeers int32
GroupMaximumPeers int32
BootstrapServers []string
GrpcDialOption grpc.DialOption
ClientId string
ConsumerGroup string
ConsumerGroupInstanceId string
GroupMinimumPeers int32
GroupMaximumPeers int32
BootstrapServers []string
GrpcDialOption grpc.DialOption
}
type ContentConfiguration struct {
Namespace string
Topic string
Topic topic.Topic
Filter string
StartTime time.Time
}
type OnEachMessageFunc func(key, value []byte) (shouldContinue bool)
type ProcessorConfiguration struct {
ConcurrentPartitionLimit int // how many partitions to process concurrently
}
type OnEachMessageFunc func(key, value []byte) (shouldContinue bool, err error)
type OnCompletionFunc func()
type TopicSubscriber struct {
SubscriberConfig *SubscriberConfiguration
ContentConfig *ContentConfiguration
ProcessorConfig *ProcessorConfiguration
brokerPartitionAssignments []*mq_pb.BrokerPartitionAssignment
OnEachMessageFunc OnEachMessageFunc
OnCompletionFunc OnCompletionFunc
@ -37,10 +42,11 @@ type TopicSubscriber struct {
alreadyProcessedTsNs int64
}
func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration) *TopicSubscriber {
func NewTopicSubscriber(bootstrapBrokers []string, subscriber *SubscriberConfiguration, content *ContentConfiguration, processor ProcessorConfiguration) *TopicSubscriber {
return &TopicSubscriber{
SubscriberConfig: subscriber,
ContentConfig: content,
ProcessorConfig: &processor,
bootstrapBrokers: bootstrapBrokers,
waitForMoreMessage: true,
alreadyProcessedTsNs: content.StartTime.UnixNano(),

132
weed/mq/pub_balancer/allocate.go

@ -5,10 +5,12 @@ import (
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"math/rand"
"time"
)
func allocateTopicPartitions(brokers cmap.ConcurrentMap[string, *BrokerStats], partitionCount int32) (assignments []*mq_pb.BrokerPartitionAssignment) {
func AllocateTopicPartitions(brokers cmap.ConcurrentMap[string, *BrokerStats], partitionCount int32) (assignments []*mq_pb.BrokerPartitionAssignment) {
// divide the ring into partitions
now := time.Now().UnixNano()
rangeSize := MaxPartitionCount / partitionCount
for i := int32(0); i < partitionCount; i++ {
assignment := &mq_pb.BrokerPartitionAssignment{
@ -16,6 +18,7 @@ func allocateTopicPartitions(brokers cmap.ConcurrentMap[string, *BrokerStats], p
RingSize: MaxPartitionCount,
RangeStart: int32(i * rangeSize),
RangeStop: int32((i + 1) * rangeSize),
UnixTimeNs: now,
},
}
if i == partitionCount-1 {
@ -24,18 +27,13 @@ func allocateTopicPartitions(brokers cmap.ConcurrentMap[string, *BrokerStats], p
assignments = append(assignments, assignment)
}
// pick the brokers
pickedBrokers := pickBrokers(brokers, partitionCount)
EnsureAssignmentsToActiveBrokers(brokers, 1, assignments)
// assign the partitions to brokers
for i, assignment := range assignments {
assignment.LeaderBroker = pickedBrokers[i]
}
glog.V(0).Infof("allocate topic partitions %d: %v", len(assignments), assignments)
return
}
// for now: randomly pick brokers
// randomly pick n brokers, which may contain duplicates
// TODO pick brokers based on the broker stats
func pickBrokers(brokers cmap.ConcurrentMap[string, *BrokerStats], count int32) []string {
candidates := make([]string, 0, brokers.Count())
@ -44,11 +42,121 @@ func pickBrokers(brokers cmap.ConcurrentMap[string, *BrokerStats], count int32)
}
pickedBrokers := make([]string, 0, count)
for i := int32(0); i < count; i++ {
p := rand.Int() % len(candidates)
if p < 0 {
p = -p
}
p := rand.Intn(len(candidates))
pickedBrokers = append(pickedBrokers, candidates[p])
}
return pickedBrokers
}
// reservoir sampling select N brokers from the active brokers, with exclusion of the excluded brokers
func pickBrokersExcluded(brokers []string, count int, excludedLeadBroker string, excludedBrokers []string) []string {
// convert the excluded brokers to a map
excludedBrokerMap := make(map[string]bool)
for _, broker := range excludedBrokers {
excludedBrokerMap[broker] = true
}
if excludedLeadBroker != "" {
excludedBrokerMap[excludedLeadBroker] = true
}
pickedBrokers := make([]string, 0, count)
for i, broker := range brokers {
if _, found := excludedBrokerMap[broker]; found {
continue
}
if len(pickedBrokers) < count {
pickedBrokers = append(pickedBrokers, broker)
} else {
j := rand.Intn(i + 1)
if j < count {
pickedBrokers[j] = broker
}
}
}
// shuffle the picked brokers
count = len(pickedBrokers)
for i := 0; i < count; i++ {
j := rand.Intn(count)
pickedBrokers[i], pickedBrokers[j] = pickedBrokers[j], pickedBrokers[i]
}
return pickedBrokers
}
// EnsureAssignmentsToActiveBrokers ensures the assignments are assigned to active brokers
func EnsureAssignmentsToActiveBrokers(activeBrokers cmap.ConcurrentMap[string, *BrokerStats], followerCount int, assignments []*mq_pb.BrokerPartitionAssignment) (hasChanges bool) {
glog.V(0).Infof("EnsureAssignmentsToActiveBrokers: activeBrokers: %v, followerCount: %d, assignments: %v", activeBrokers.Count(), followerCount, assignments)
candidates := make([]string, 0, activeBrokers.Count())
for brokerStatsItem := range activeBrokers.IterBuffered() {
candidates = append(candidates, brokerStatsItem.Key)
}
for _, assignment := range assignments {
// count how many brokers are needed
count := 0
if assignment.LeaderBroker == "" {
count++
} else if _, found := activeBrokers.Get(assignment.LeaderBroker); !found {
assignment.LeaderBroker = ""
count++
}
for i := 0; i < followerCount; i++ {
if i >= len(assignment.FollowerBrokers) {
count++
continue
}
if assignment.FollowerBrokers[i] == "" {
count++
} else if _, found := activeBrokers.Get(assignment.FollowerBrokers[i]); !found {
assignment.FollowerBrokers[i] = ""
count++
}
}
if count > 0 {
pickedBrokers := pickBrokersExcluded(candidates, count, assignment.LeaderBroker, assignment.FollowerBrokers)
i := 0
if assignment.LeaderBroker == "" {
if i < len(pickedBrokers) {
assignment.LeaderBroker = pickedBrokers[i]
i++
hasChanges = true
}
}
hasEmptyFollowers := false
j := 0
for ; j < len(assignment.FollowerBrokers); j++ {
if assignment.FollowerBrokers[j] == "" {
hasChanges = true
if i < len(pickedBrokers) {
assignment.FollowerBrokers[j] = pickedBrokers[i]
i++
} else {
hasEmptyFollowers = true
}
}
}
if hasEmptyFollowers {
var followerBrokers []string
for _, follower := range assignment.FollowerBrokers {
if follower != "" {
followerBrokers = append(followerBrokers, follower)
}
}
assignment.FollowerBrokers = followerBrokers
}
if i < len(pickedBrokers) {
assignment.FollowerBrokers = append(assignment.FollowerBrokers, pickedBrokers[i:]...)
hasChanges = true
}
}
}
glog.V(0).Infof("EnsureAssignmentsToActiveBrokers: activeBrokers: %v, followerCount: %d, assignments: %v hasChanges: %v", activeBrokers.Count(), followerCount, assignments, hasChanges)
return
}

187
weed/mq/pub_balancer/allocate_test.go

@ -1,6 +1,7 @@
package pub_balancer
import (
"fmt"
cmap "github.com/orcaman/concurrent-map/v2"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"github.com/stretchr/testify/assert"
@ -53,7 +54,7 @@ func testThem(t *testing.T, tests []struct {
}) {
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
gotAssignments := allocateTopicPartitions(tt.args.brokers, tt.args.partitionCount)
gotAssignments := AllocateTopicPartitions(tt.args.brokers, tt.args.partitionCount)
assert.Equal(t, len(tt.wantAssignments), len(gotAssignments))
for i, gotAssignment := range gotAssignments {
assert.Equal(t, tt.wantAssignments[i].LeaderBroker, gotAssignment.LeaderBroker)
@ -65,3 +66,187 @@ func testThem(t *testing.T, tests []struct {
})
}
}
func TestEnsureAssignmentsToActiveBrokersX(t *testing.T) {
type args struct {
activeBrokers cmap.ConcurrentMap[string, *BrokerStats]
followerCount int
assignments []*mq_pb.BrokerPartitionAssignment
}
activeBrokers := cmap.New[*BrokerStats]()
activeBrokers.SetIfAbsent("localhost:1", &BrokerStats{})
activeBrokers.SetIfAbsent("localhost:2", &BrokerStats{})
activeBrokers.SetIfAbsent("localhost:3", &BrokerStats{})
activeBrokers.SetIfAbsent("localhost:4", &BrokerStats{})
activeBrokers.SetIfAbsent("localhost:5", &BrokerStats{})
activeBrokers.SetIfAbsent("localhost:6", &BrokerStats{})
lowActiveBrokers := cmap.New[*BrokerStats]()
lowActiveBrokers.SetIfAbsent("localhost:1", &BrokerStats{})
lowActiveBrokers.SetIfAbsent("localhost:2", &BrokerStats{})
singleActiveBroker := cmap.New[*BrokerStats]()
singleActiveBroker.SetIfAbsent("localhost:1", &BrokerStats{})
tests := []struct {
name string
args args
hasChanges bool
}{
{
name: "test empty leader",
args: args{
activeBrokers: activeBrokers,
followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "",
Partition: &mq_pb.Partition{},
FollowerBrokers: []string{
"localhost:2",
},
},
},
},
hasChanges: true,
},
{
name: "test empty follower",
args: args{
activeBrokers: activeBrokers,
followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
FollowerBrokers: []string{
"",
},
},
},
},
hasChanges: true,
},
{
name: "test dead follower",
args: args{
activeBrokers: activeBrokers,
followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
FollowerBrokers: []string{
"localhost:200",
},
},
},
},
hasChanges: true,
},
{
name: "test dead leader and follower",
args: args{
activeBrokers: activeBrokers,
followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:100",
Partition: &mq_pb.Partition{},
FollowerBrokers: []string{
"localhost:200",
},
},
},
},
hasChanges: true,
},
{
name: "test missing two followers",
args: args{
activeBrokers: activeBrokers,
followerCount: 3,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
FollowerBrokers: []string{
"localhost:2",
},
},
},
},
hasChanges: true,
},
{
name: "test missing some followers",
args: args{
activeBrokers: activeBrokers,
followerCount: 10,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
FollowerBrokers: []string{
"localhost:2",
},
},
},
},
hasChanges: true,
},
{
name: "test low active brokers",
args: args{
activeBrokers: lowActiveBrokers,
followerCount: 3,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
FollowerBrokers: []string{
"localhost:2",
},
},
},
},
hasChanges: false,
},
{
name: "test low active brokers with one follower",
args: args{
activeBrokers: lowActiveBrokers,
followerCount: 1,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
},
},
},
hasChanges: true,
},
{
name: "test single active broker",
args: args{
activeBrokers: singleActiveBroker,
followerCount: 3,
assignments: []*mq_pb.BrokerPartitionAssignment{
{
LeaderBroker: "localhost:1",
Partition: &mq_pb.Partition{},
FollowerBrokers: []string{
"localhost:2",
},
},
},
},
hasChanges: true,
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
fmt.Printf("%v before %v\n", tt.name, tt.args.assignments)
hasChanges := EnsureAssignmentsToActiveBrokers(tt.args.activeBrokers, tt.args.followerCount, tt.args.assignments)
assert.Equalf(t, tt.hasChanges, hasChanges, "EnsureAssignmentsToActiveBrokers(%v, %v, %v)", tt.args.activeBrokers, tt.args.followerCount, tt.args.assignments)
fmt.Printf("%v after %v\n", tt.name, tt.args.assignments)
})
}
}

37
weed/mq/pub_balancer/balancer.go

@ -2,6 +2,7 @@ package pub_balancer
import (
cmap "github.com/orcaman/concurrent-map/v2"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
)
@ -30,7 +31,10 @@ const (
type Balancer struct {
Brokers cmap.ConcurrentMap[string, *BrokerStats] // key: broker address
// Collected from all brokers when they connect to the broker leader
TopicToBrokers cmap.ConcurrentMap[string, *PartitionSlotToBrokerList] // key: topic name
TopicToBrokers cmap.ConcurrentMap[string, *PartitionSlotToBrokerList] // key: topic name
OnPartitionChange func(topic *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment)
OnAddBroker func(broker string, brokerStats *BrokerStats)
OnRemoveBroker func(broker string, brokerStats *BrokerStats)
}
func NewBalancer() *Balancer {
@ -40,7 +44,7 @@ func NewBalancer() *Balancer {
}
}
func (balancer *Balancer) OnBrokerConnected(broker string) (brokerStats *BrokerStats) {
func (balancer *Balancer) AddBroker(broker string) (brokerStats *BrokerStats) {
var found bool
brokerStats, found = balancer.Brokers.Get(broker)
if !found {
@ -49,10 +53,12 @@ func (balancer *Balancer) OnBrokerConnected(broker string) (brokerStats *BrokerS
brokerStats, _ = balancer.Brokers.Get(broker)
}
}
balancer.onPubAddBroker(broker, brokerStats)
balancer.OnAddBroker(broker, brokerStats)
return brokerStats
}
func (balancer *Balancer) OnBrokerDisconnected(broker string, stats *BrokerStats) {
func (balancer *Balancer) RemoveBroker(broker string, stats *BrokerStats) {
balancer.Brokers.Remove(broker)
// update TopicToBrokers
@ -61,8 +67,15 @@ func (balancer *Balancer) OnBrokerDisconnected(broker string, stats *BrokerStats
if !found {
continue
}
partitionSlotToBrokerList.RemoveBroker(broker)
pickedBroker := pickBrokers(balancer.Brokers, 1)
if len(pickedBroker) == 0 {
partitionSlotToBrokerList.RemoveBroker(broker)
} else {
partitionSlotToBrokerList.ReplaceBroker(broker, pickedBroker[0])
}
}
balancer.onPubRemoveBroker(broker, stats)
balancer.OnRemoveBroker(broker, stats)
}
func (balancer *Balancer) OnBrokerStatsUpdated(broker string, brokerStats *BrokerStats, receivedStats *mq_pb.BrokerStats) {
@ -70,15 +83,23 @@ func (balancer *Balancer) OnBrokerStatsUpdated(broker string, brokerStats *Broke
// update TopicToBrokers
for _, topicPartitionStats := range receivedStats.Stats {
topic := topicPartitionStats.Topic
topicKey := topic.FromPbTopic(topicPartitionStats.Topic).String()
partition := topicPartitionStats.Partition
partitionSlotToBrokerList, found := balancer.TopicToBrokers.Get(topic.String())
partitionSlotToBrokerList, found := balancer.TopicToBrokers.Get(topicKey)
if !found {
partitionSlotToBrokerList = NewPartitionSlotToBrokerList(MaxPartitionCount)
if !balancer.TopicToBrokers.SetIfAbsent(topic.String(), partitionSlotToBrokerList) {
partitionSlotToBrokerList, _ = balancer.TopicToBrokers.Get(topic.String())
if !balancer.TopicToBrokers.SetIfAbsent(topicKey, partitionSlotToBrokerList) {
partitionSlotToBrokerList, _ = balancer.TopicToBrokers.Get(topicKey)
}
}
partitionSlotToBrokerList.AddBroker(partition, broker)
}
}
// OnPubAddBroker is called when a broker is added for a publisher coordinator
func (balancer *Balancer) onPubAddBroker(broker string, brokerStats *BrokerStats) {
}
// OnPubRemoveBroker is called when a broker is removed for a publisher coordinator
func (balancer *Balancer) onPubRemoveBroker(broker string, brokerStats *BrokerStats) {
}

26
weed/mq/pub_balancer/broker_stats.go

@ -39,8 +39,13 @@ func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) {
for _, topicPartitionStats := range stats.Stats {
tps := &TopicPartitionStats{
TopicPartition: topic.TopicPartition{
Topic: topic.Topic{Namespace: topicPartitionStats.Topic.Namespace, Name: topicPartitionStats.Topic.Name},
Partition: topic.Partition{RangeStart: topicPartitionStats.Partition.RangeStart, RangeStop: topicPartitionStats.Partition.RangeStop, RingSize: topicPartitionStats.Partition.RingSize},
Topic: topic.Topic{Namespace: topicPartitionStats.Topic.Namespace, Name: topicPartitionStats.Topic.Name},
Partition: topic.Partition{
RangeStart: topicPartitionStats.Partition.RangeStart,
RangeStop: topicPartitionStats.Partition.RangeStop,
RingSize: topicPartitionStats.Partition.RingSize,
UnixTimeNs: topicPartitionStats.Partition.UnixTimeNs,
},
},
ConsumerCount: topicPartitionStats.ConsumerCount,
IsLeader: topicPartitionStats.IsLeader,
@ -58,15 +63,24 @@ func (bs *BrokerStats) UpdateStats(stats *mq_pb.BrokerStats) {
}
func (bs *BrokerStats) RegisterAssignment(t *mq_pb.Topic, partition *mq_pb.Partition) {
func (bs *BrokerStats) RegisterAssignment(t *mq_pb.Topic, partition *mq_pb.Partition, isAdd bool) {
tps := &TopicPartitionStats{
TopicPartition: topic.TopicPartition{
Topic: topic.Topic{Namespace: t.Namespace, Name: t.Name},
Partition: topic.Partition{RangeStart: partition.RangeStart, RangeStop: partition.RangeStop},
Topic: topic.Topic{Namespace: t.Namespace, Name: t.Name},
Partition: topic.Partition{
RangeStart: partition.RangeStart,
RangeStop: partition.RangeStop,
RingSize: partition.RingSize,
UnixTimeNs: partition.UnixTimeNs,
},
},
ConsumerCount: 0,
IsLeader: true,
}
key := tps.TopicPartition.String()
bs.TopicPartitionStats.Set(key, tps)
if isAdd {
bs.TopicPartitionStats.SetIfAbsent(key, tps)
} else {
bs.TopicPartitionStats.Remove(key)
}
}

24
weed/mq/pub_balancer/lookup.go

@ -2,7 +2,6 @@ package pub_balancer
import (
"errors"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
)
@ -10,10 +9,7 @@ var (
ErrNoBroker = errors.New("no broker")
)
func (balancer *Balancer) LookupOrAllocateTopicPartitions(topic *mq_pb.Topic, publish bool, partitionCount int32) (assignments []*mq_pb.BrokerPartitionAssignment, err error) {
if partitionCount == 0 {
partitionCount = 6
}
func (balancer *Balancer) LookupTopicPartitions(topic *mq_pb.Topic) (assignments []*mq_pb.BrokerPartitionAssignment) {
// find existing topic partition assignments
for brokerStatsItem := range balancer.Brokers.IterBuffered() {
broker, brokerStats := brokerStatsItem.Key, brokerStatsItem.Val
@ -26,6 +22,7 @@ func (balancer *Balancer) LookupOrAllocateTopicPartitions(topic *mq_pb.Topic, pu
RingSize: MaxPartitionCount,
RangeStart: topicPartitionStat.RangeStart,
RangeStop: topicPartitionStat.RangeStop,
UnixTimeNs: topicPartitionStat.UnixTimeNs,
},
}
// TODO fix follower setting
@ -34,20 +31,5 @@ func (balancer *Balancer) LookupOrAllocateTopicPartitions(topic *mq_pb.Topic, pu
}
}
}
if len(assignments) > 0 && len(assignments) == int(partitionCount) || !publish {
glog.V(0).Infof("existing topic partitions %d: %v", len(assignments), assignments)
return assignments, nil
}
// 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)
if balancer.Brokers.IsEmpty() {
return nil, ErrNoBroker
}
return allocateTopicPartitions(balancer.Brokers, partitionCount), nil
return
}

10
weed/mq/pub_balancer/partition_list_broker.go

@ -8,6 +8,7 @@ import (
type PartitionSlotToBroker struct {
RangeStart int32
RangeStop int32
UnixTimeNs int64
AssignedBroker string
}
@ -38,13 +39,18 @@ func (ps *PartitionSlotToBrokerList) AddBroker(partition *mq_pb.Partition, broke
ps.PartitionSlots = append(ps.PartitionSlots, &PartitionSlotToBroker{
RangeStart: partition.RangeStart,
RangeStop: partition.RangeStop,
UnixTimeNs: partition.UnixTimeNs,
AssignedBroker: broker,
})
}
func (ps *PartitionSlotToBrokerList) RemoveBroker(broker string) {
ps.ReplaceBroker(broker, "")
}
func (ps *PartitionSlotToBrokerList) ReplaceBroker(oldBroker string, newBroker string) {
for _, partitionSlot := range ps.PartitionSlots {
if partitionSlot.AssignedBroker == broker {
partitionSlot.AssignedBroker = ""
if partitionSlot.AssignedBroker == oldBroker {
partitionSlot.AssignedBroker = newBroker
}
}
}

91
weed/mq/sub_coordinator/consumer_group.go

@ -2,9 +2,11 @@ package sub_coordinator
import (
cmap "github.com/orcaman/concurrent-map/v2"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"
"time"
)
type ConsumerGroupInstance struct {
@ -14,15 +16,20 @@ type ConsumerGroupInstance struct {
ResponseChan chan *mq_pb.SubscriberToSubCoordinatorResponse
}
type ConsumerGroup struct {
topic topic.Topic
// map a consumer group instance id to a consumer group instance
ConsumerGroupInstances cmap.ConcurrentMap[string, *ConsumerGroupInstance]
mapping *PartitionConsumerMapping
reBalanceTimer *time.Timer
pubBalancer *pub_balancer.Balancer
}
func NewConsumerGroup() *ConsumerGroup {
func NewConsumerGroup(t *mq_pb.Topic, pubBalancer *pub_balancer.Balancer) *ConsumerGroup {
return &ConsumerGroup{
topic: topic.FromPbTopic(t),
ConsumerGroupInstances: cmap.New[*ConsumerGroupInstance](),
mapping: NewPartitionConsumerMapping(pub_balancer.MaxPartitionCount),
pubBalancer: pubBalancer,
}
}
@ -33,10 +40,90 @@ func NewConsumerGroupInstance(instanceId string) *ConsumerGroupInstance {
}
}
func (cg *ConsumerGroup) OnAddConsumerGroupInstance(consumerGroupInstance string, topic *mq_pb.Topic) {
cg.onConsumerGroupInstanceChange("add consumer instance " + consumerGroupInstance)
}
func (cg *ConsumerGroup) OnRemoveConsumerGroupInstance(consumerGroupInstance string, topic *mq_pb.Topic) {
cg.onConsumerGroupInstanceChange("remove consumer instance " + consumerGroupInstance)
}
func (cg *ConsumerGroup) onConsumerGroupInstanceChange(reason string) {
if cg.reBalanceTimer != nil {
cg.reBalanceTimer.Stop()
cg.reBalanceTimer = nil
}
cg.reBalanceTimer = time.AfterFunc(5*time.Second, func() {
cg.RebalanceConsumberGroupInstances(nil, reason)
cg.reBalanceTimer = nil
})
}
func (cg *ConsumerGroup) OnPartitionListChange(assignments []*mq_pb.BrokerPartitionAssignment) {
if cg.reBalanceTimer != nil {
cg.reBalanceTimer.Stop()
cg.reBalanceTimer = nil
}
partitionSlotToBrokerList := pub_balancer.NewPartitionSlotToBrokerList(pub_balancer.MaxPartitionCount)
for _, assignment := range assignments {
partitionSlotToBrokerList.AddBroker(assignment.Partition, assignment.LeaderBroker)
}
cg.RebalanceConsumberGroupInstances(partitionSlotToBrokerList, "partition list change")
}
func (cg *ConsumerGroup) OnPartitionListChange() {
func (cg *ConsumerGroup) RebalanceConsumberGroupInstances(knownPartitionSlotToBrokerList *pub_balancer.PartitionSlotToBrokerList, reason string) {
glog.V(0).Infof("rebalance consumer group %s due to %s", cg.topic.String(), reason)
// collect current topic partitions
partitionSlotToBrokerList := knownPartitionSlotToBrokerList
if partitionSlotToBrokerList == nil {
var found bool
partitionSlotToBrokerList, found = cg.pubBalancer.TopicToBrokers.Get(cg.topic.String())
if !found {
glog.V(0).Infof("topic %s not found in balancer", cg.topic.String())
return
}
}
// collect current consumer group instance ids
var consumerInstanceIds []string
for _, consumerGroupInstance := range cg.ConsumerGroupInstances.Items() {
consumerInstanceIds = append(consumerInstanceIds, consumerGroupInstance.InstanceId)
}
cg.mapping.BalanceToConsumerInstanceIds(partitionSlotToBrokerList, consumerInstanceIds)
// convert cg.mapping currentMapping to map of consumer group instance id to partition slots
consumerInstanceToPartitionSlots := make(map[string][]*PartitionSlotToConsumerInstance)
for _, partitionSlot := range cg.mapping.currentMapping.PartitionSlots {
consumerInstanceToPartitionSlots[partitionSlot.AssignedInstanceId] = append(consumerInstanceToPartitionSlots[partitionSlot.AssignedInstanceId], partitionSlot)
}
// notify consumer group instances
for _, consumerGroupInstance := range cg.ConsumerGroupInstances.Items() {
partitionSlots, found := consumerInstanceToPartitionSlots[consumerGroupInstance.InstanceId]
if !found {
partitionSlots = make([]*PartitionSlotToConsumerInstance, 0)
}
consumerGroupInstance.Partitions = ToPartitions(partitionSlotToBrokerList.RingSize, partitionSlots)
assignedPartitions := make([]*mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition, len(partitionSlots))
for i, partitionSlot := range partitionSlots {
assignedPartitions[i] = &mq_pb.SubscriberToSubCoordinatorResponse_AssignedPartition{
Partition: &mq_pb.Partition{
RangeStop: partitionSlot.RangeStop,
RangeStart: partitionSlot.RangeStart,
RingSize: partitionSlotToBrokerList.RingSize,
UnixTimeNs: partitionSlot.UnixTimeNs,
},
Broker: partitionSlot.Broker,
}
}
response := &mq_pb.SubscriberToSubCoordinatorResponse{
Message: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment_{
Assignment: &mq_pb.SubscriberToSubCoordinatorResponse_Assignment{
AssignedPartitions: assignedPartitions,
},
},
}
println("sending response to", consumerGroupInstance.InstanceId, "...")
consumerGroupInstance.ResponseChan <- response
}
}

42
weed/mq/sub_coordinator/coordinator.go

@ -28,14 +28,16 @@ func NewCoordinator(balancer *pub_balancer.Balancer) *Coordinator {
}
}
func (c *Coordinator) GetTopicConsumerGroups(topic *mq_pb.Topic) *TopicConsumerGroups {
func (c *Coordinator) GetTopicConsumerGroups(topic *mq_pb.Topic, createIfMissing bool) *TopicConsumerGroups {
topicName := toTopicName(topic)
tcg, _ := c.TopicSubscribers.Get(topicName)
if tcg == nil {
if tcg == nil && createIfMissing {
tcg = &TopicConsumerGroups{
ConsumerGroups: cmap.New[*ConsumerGroup](),
}
c.TopicSubscribers.Set(topicName, tcg)
if !c.TopicSubscribers.SetIfAbsent(topicName, tcg) {
tcg, _ = c.TopicSubscribers.Get(topicName)
}
}
return tcg
}
@ -50,23 +52,27 @@ func toTopicName(topic *mq_pb.Topic) string {
}
func (c *Coordinator) AddSubscriber(consumerGroup, consumerGroupInstance string, topic *mq_pb.Topic) *ConsumerGroupInstance {
tcg := c.GetTopicConsumerGroups(topic)
tcg := c.GetTopicConsumerGroups(topic, true)
cg, _ := tcg.ConsumerGroups.Get(consumerGroup)
if cg == nil {
cg = NewConsumerGroup()
tcg.ConsumerGroups.Set(consumerGroup, cg)
cg = NewConsumerGroup(topic, c.balancer)
if !tcg.ConsumerGroups.SetIfAbsent(consumerGroup, cg) {
cg, _ = tcg.ConsumerGroups.Get(consumerGroup)
}
}
cgi, _ := cg.ConsumerGroupInstances.Get(consumerGroupInstance)
if cgi == nil {
cgi = NewConsumerGroupInstance(consumerGroupInstance)
cg.ConsumerGroupInstances.Set(consumerGroupInstance, cgi)
if !cg.ConsumerGroupInstances.SetIfAbsent(consumerGroupInstance, cgi) {
cgi, _ = cg.ConsumerGroupInstances.Get(consumerGroupInstance)
}
}
cg.OnAddConsumerGroupInstance(consumerGroupInstance, topic)
return cgi
}
func (c *Coordinator) RemoveSubscriber(consumerGroup, consumerGroupInstance string, topic *mq_pb.Topic) {
tcg, _ := c.TopicSubscribers.Get(toTopicName(topic))
tcg := c.GetTopicConsumerGroups(topic, false)
if tcg == nil {
return
}
@ -83,3 +89,23 @@ func (c *Coordinator) RemoveSubscriber(consumerGroup, consumerGroupInstance stri
c.RemoveTopic(topic)
}
}
func (c *Coordinator) OnPartitionChange(topic *mq_pb.Topic, assignments []*mq_pb.BrokerPartitionAssignment) {
tcg, _ := c.TopicSubscribers.Get(toTopicName(topic))
if tcg == nil {
return
}
for _, cg := range tcg.ConsumerGroups.Items() {
cg.OnPartitionListChange(assignments)
}
}
// OnSubAddBroker is called when a broker is added to the balancer
func (c *Coordinator) OnSubAddBroker(broker string, brokerStats *pub_balancer.BrokerStats) {
}
// OnSubRemoveBroker is called when a broker is removed from the balancer
func (c *Coordinator) OnSubRemoveBroker(broker string, brokerStats *pub_balancer.BrokerStats) {
}

31
weed/mq/sub_coordinator/partition_consumer_mapping.go

@ -2,7 +2,7 @@ package sub_coordinator
import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"time"
)
@ -23,20 +23,29 @@ func NewPartitionConsumerMapping(ringSize int32) *PartitionConsumerMapping {
// 2. allow one consumer instance to be down unexpectedly
// without affecting the processing power utilization
func (pcm *PartitionConsumerMapping) BalanceToConsumerInstanceIds(partitions []*topic.Partition, consumerInstanceIds []string) {
if len(partitions) == 0 || len(consumerInstanceIds) == 0 {
func (pcm *PartitionConsumerMapping) BalanceToConsumerInstanceIds(partitionSlotToBrokerList *pub_balancer.PartitionSlotToBrokerList, consumerInstanceIds []string) {
if len(partitionSlotToBrokerList.PartitionSlots) == 0 || len(consumerInstanceIds) == 0 {
return
}
newVersion := time.Now().UnixNano()
newMapping := NewPartitionSlotToConsumerInstanceList(partitions[0].RingSize, newVersion)
newMapping.PartitionSlots = doBalanceSticky(partitions, consumerInstanceIds, pcm.prevMappings[0])
newMapping := NewPartitionSlotToConsumerInstanceList(partitionSlotToBrokerList.RingSize, newVersion)
var prevMapping *PartitionSlotToConsumerInstanceList
if len(pcm.prevMappings) > 0 {
prevMapping = pcm.prevMappings[len(pcm.prevMappings)-1]
} else {
prevMapping = nil
}
newMapping.PartitionSlots = doBalanceSticky(partitionSlotToBrokerList.PartitionSlots, consumerInstanceIds, prevMapping)
if pcm.currentMapping != nil {
pcm.prevMappings = append(pcm.prevMappings, pcm.currentMapping)
if len(pcm.prevMappings) > 10 {
pcm.prevMappings = pcm.prevMappings[1:]
}
}
pcm.currentMapping = newMapping
}
func doBalanceSticky(partitions []*topic.Partition, consumerInstanceIds []string, prevMapping *PartitionSlotToConsumerInstanceList) (partitionSlots []*PartitionSlotToConsumerInstance) {
func doBalanceSticky(partitions []*pub_balancer.PartitionSlotToBroker, consumerInstanceIds []string, prevMapping *PartitionSlotToConsumerInstanceList) (partitionSlots []*PartitionSlotToConsumerInstance) {
// collect previous consumer instance ids
prevConsumerInstanceIds := make(map[string]struct{})
if prevMapping != nil {
@ -70,7 +79,15 @@ func doBalanceSticky(partitions []*topic.Partition, consumerInstanceIds []string
}
// make a copy of old mapping, skipping the deleted consumer instances
newPartitionSlots := ToPartitionSlots(partitions)
newPartitionSlots := make([]*PartitionSlotToConsumerInstance, 0, len(partitions))
for _, partition := range partitions {
newPartitionSlots = append(newPartitionSlots, &PartitionSlotToConsumerInstance{
RangeStart: partition.RangeStart,
RangeStop: partition.RangeStop,
UnixTimeNs: partition.UnixTimeNs,
Broker: partition.AssignedBroker,
})
}
for _, newPartitionSlot := range newPartitionSlots {
key := fmt.Sprintf("%d-%d", newPartitionSlot.RangeStart, newPartitionSlot.RangeStop)
if prevPartitionSlot, ok := prevPartitionSlotMap[key]; ok {

20
weed/mq/sub_coordinator/partition_consumer_mapping_test.go

@ -1,14 +1,14 @@
package sub_coordinator
import (
"github.com/seaweedfs/seaweedfs/weed/mq/topic"
"github.com/seaweedfs/seaweedfs/weed/mq/pub_balancer"
"reflect"
"testing"
)
func Test_doBalanceSticky(t *testing.T) {
type args struct {
partitions []*topic.Partition
partitions []*pub_balancer.PartitionSlotToBroker
consumerInstanceIds []string
prevMapping *PartitionSlotToConsumerInstanceList
}
@ -20,7 +20,7 @@ func Test_doBalanceSticky(t *testing.T) {
{
name: "1 consumer instance, 1 partition",
args: args{
partitions: []*topic.Partition{
partitions: []*pub_balancer.PartitionSlotToBroker{
{
RangeStart: 0,
RangeStop: 100,
@ -40,7 +40,7 @@ func Test_doBalanceSticky(t *testing.T) {
{
name: "2 consumer instances, 1 partition",
args: args{
partitions: []*topic.Partition{
partitions: []*pub_balancer.PartitionSlotToBroker{
{
RangeStart: 0,
RangeStop: 100,
@ -60,7 +60,7 @@ func Test_doBalanceSticky(t *testing.T) {
{
name: "1 consumer instance, 2 partitions",
args: args{
partitions: []*topic.Partition{
partitions: []*pub_balancer.PartitionSlotToBroker{
{
RangeStart: 0,
RangeStop: 50,
@ -89,7 +89,7 @@ func Test_doBalanceSticky(t *testing.T) {
{
name: "2 consumer instances, 2 partitions",
args: args{
partitions: []*topic.Partition{
partitions: []*pub_balancer.PartitionSlotToBroker{
{
RangeStart: 0,
RangeStop: 50,
@ -118,7 +118,7 @@ func Test_doBalanceSticky(t *testing.T) {
{
name: "2 consumer instances, 2 partitions, 1 deleted consumer instance",
args: args{
partitions: []*topic.Partition{
partitions: []*pub_balancer.PartitionSlotToBroker{
{
RangeStart: 0,
RangeStop: 50,
@ -160,7 +160,7 @@ func Test_doBalanceSticky(t *testing.T) {
{
name: "2 consumer instances, 2 partitions, 1 new consumer instance",
args: args{
partitions: []*topic.Partition{
partitions: []*pub_balancer.PartitionSlotToBroker{
{
RangeStart: 0,
RangeStop: 50,
@ -202,7 +202,7 @@ func Test_doBalanceSticky(t *testing.T) {
{
name: "2 consumer instances, 2 partitions, 1 new partition",
args: args{
partitions: []*topic.Partition{
partitions: []*pub_balancer.PartitionSlotToBroker{
{
RangeStart: 0,
RangeStop: 50,
@ -253,7 +253,7 @@ func Test_doBalanceSticky(t *testing.T) {
{
name: "2 consumer instances, 2 partitions, 1 new partition, 1 new consumer instance",
args: args{
partitions: []*topic.Partition{
partitions: []*pub_balancer.PartitionSlotToBroker{
{
RangeStart: 0,
RangeStop: 50,

14
weed/mq/sub_coordinator/partition_list.go

@ -5,6 +5,8 @@ import "github.com/seaweedfs/seaweedfs/weed/mq/topic"
type PartitionSlotToConsumerInstance struct {
RangeStart int32
RangeStop int32
UnixTimeNs int64
Broker string
AssignedInstanceId string
}
@ -21,12 +23,10 @@ func NewPartitionSlotToConsumerInstanceList(ringSize int32, version int64) *Part
}
}
func ToPartitionSlots(partitions []*topic.Partition) (partitionSlots []*PartitionSlotToConsumerInstance) {
for _, partition := range partitions {
partitionSlots = append(partitionSlots, &PartitionSlotToConsumerInstance{
RangeStart: partition.RangeStart,
RangeStop: partition.RangeStop,
})
func ToPartitions(ringSize int32, slots []*PartitionSlotToConsumerInstance) []*topic.Partition {
partitions := make([]*topic.Partition, 0, len(slots))
for _, slot := range slots {
partitions = append(partitions, topic.NewPartition(slot.RangeStart, slot.RangeStop, ringSize, slot.UnixTimeNs))
}
return
return partitions
}

16
weed/mq/topic/local_manager.go

@ -34,7 +34,7 @@ func (manager *LocalTopicManager) AddTopicPartition(topic Topic, localPartition
localTopic.Partitions = append(localTopic.Partitions, localPartition)
}
// GetTopic gets a topic from the local topic manager
// GetTopicPartition gets a topic from the local topic manager
func (manager *LocalTopicManager) GetTopicPartition(topic Topic, partition Partition) *LocalPartition {
localTopic, ok := manager.topics.Get(topic.String())
if !ok {
@ -88,23 +88,15 @@ func (manager *LocalTopicManager) CollectStats(duration time.Duration) *mq_pb.Br
manager.topics.IterCb(func(topic string, localTopic *LocalTopic) {
for _, localPartition := range localTopic.Partitions {
topicPartition := &TopicPartition{
Topic: Topic{Namespace: localTopic.Namespace, Name: localTopic.Name},
Partition: Partition{
RingSize: localPartition.RingSize,
RangeStart: localPartition.RangeStart,
RangeStop: localPartition.RangeStop,
},
Topic: Topic{Namespace: localTopic.Namespace, Name: localTopic.Name},
Partition: localPartition.Partition,
}
stats.Stats[topicPartition.String()] = &mq_pb.TopicPartitionStats{
Topic: &mq_pb.Topic{
Namespace: string(localTopic.Namespace),
Name: localTopic.Name,
},
Partition: &mq_pb.Partition{
RingSize: localPartition.RingSize,
RangeStart: localPartition.RangeStart,
RangeStop: localPartition.RangeStop,
},
Partition: localPartition.Partition.ToPbPartition(),
ConsumerCount: localPartition.ConsumerCount,
}
// fmt.Printf("collect topic %+v partition %+v\n", topicPartition, localPartition.Partition)

104
weed/mq/topic/local_partition.go

@ -2,67 +2,100 @@ package topic
import (
"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"
"github.com/seaweedfs/seaweedfs/weed/util/log_buffer"
"sync/atomic"
"time"
)
type LocalPartition struct {
Partition
isLeader bool
FollowerBrokers []pb.ServerAddress
logBuffer *log_buffer.LogBuffer
ConsumerCount int32
StopPublishersCh chan struct{}
Publishers *LocalPartitionPublishers
StopSubscribersCh chan struct{}
Subscribers *LocalPartitionSubscribers
isLeader bool
FollowerBrokers []pb.ServerAddress
LogBuffer *log_buffer.LogBuffer
ConsumerCount int32
Publishers *LocalPartitionPublishers
Subscribers *LocalPartitionSubscribers
FollowerId int32
}
func NewLocalPartition(partition Partition, isLeader bool, followerBrokers []pb.ServerAddress) *LocalPartition {
var TIME_FORMAT = "2006-01-02-15-04-05"
func NewLocalPartition(partition Partition, isLeader bool, followerBrokers []pb.ServerAddress, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
return &LocalPartition{
Partition: partition,
isLeader: isLeader,
FollowerBrokers: followerBrokers,
logBuffer: log_buffer.NewLogBuffer(
fmt.Sprintf("%d/%4d-%4d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
2*time.Minute,
func(startTime, stopTime time.Time, buf []byte) {
},
func() {
},
),
LogBuffer: log_buffer.NewLogBuffer(fmt.Sprintf("%d/%04d-%04d", partition.UnixTimeNs, partition.RangeStart, partition.RangeStop),
2*time.Minute, logFlushFn, readFromDiskFn, func() {}),
Publishers: NewLocalPartitionPublishers(),
Subscribers: NewLocalPartitionSubscribers(),
}
}
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())
p.LogBuffer.AddToBuffer(message.Key, message.Value, time.Now().UnixNano())
}
func (p *LocalPartition) Subscribe(clientName string, startReadTime time.Time, onNoMessageFn func() bool, eachMessageFn OnEachMessageFn) {
p.logBuffer.LoopProcessLogData(clientName, startReadTime, 0, onNoMessageFn, eachMessageFn)
func (p *LocalPartition) Subscribe(clientName string, startPosition log_buffer.MessagePosition,
onNoMessageFn func() bool, eachMessageFn log_buffer.EachLogEntryFuncType) error {
var processedPosition log_buffer.MessagePosition
var readPersistedLogErr error
var readInMemoryLogErr error
var isDone bool
for {
processedPosition, isDone, readPersistedLogErr = p.LogBuffer.ReadFromDiskFn(startPosition, 0, eachMessageFn)
if readPersistedLogErr != nil {
glog.V(0).Infof("%s read %v persisted log: %v", clientName, p.Partition, readPersistedLogErr)
return readPersistedLogErr
}
if isDone {
return nil
}
startPosition = processedPosition
processedPosition, isDone, readInMemoryLogErr = p.LogBuffer.LoopProcessLogData(clientName, startPosition, 0, onNoMessageFn, eachMessageFn)
if isDone {
return nil
}
startPosition = processedPosition
if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
continue
}
if readInMemoryLogErr != nil {
glog.V(0).Infof("%s read %v in memory log: %v", clientName, p.Partition, readInMemoryLogErr)
return readInMemoryLogErr
}
}
}
func (p *LocalPartition) GetEarliestMessageTimeInMemory() time.Time {
return p.LogBuffer.GetEarliestTime()
}
func (p *LocalPartition) HasData() bool {
return !p.LogBuffer.GetEarliestTime().IsZero()
}
func (p *LocalPartition) GetEarliestInMemoryMessagePosition() log_buffer.MessagePosition {
return p.LogBuffer.GetEarliestPosition()
}
func FromPbBrokerPartitionAssignment(self pb.ServerAddress, assignment *mq_pb.BrokerPartitionAssignment) *LocalPartition {
func FromPbBrokerPartitionAssignment(self pb.ServerAddress, partition Partition, assignment *mq_pb.BrokerPartitionAssignment, logFlushFn log_buffer.LogFlushFuncType, readFromDiskFn log_buffer.LogReadFromDiskFuncType) *LocalPartition {
isLeader := assignment.LeaderBroker == string(self)
followers := make([]pb.ServerAddress, len(assignment.FollowerBrokers))
for i, followerBroker := range assignment.FollowerBrokers {
followers[i] = pb.ServerAddress(followerBroker)
}
return NewLocalPartition(FromPbPartition(assignment.Partition), isLeader, followers)
return NewLocalPartition(partition, isLeader, followers, logFlushFn, readFromDiskFn)
}
func (p *LocalPartition) closePublishers() {
p.Publishers.SignalShutdown()
close(p.StopPublishersCh)
}
func (p *LocalPartition) closeSubscribers() {
p.Subscribers.SignalShutdown()
@ -76,3 +109,18 @@ func (p *LocalPartition) WaitUntilNoPublishers() {
time.Sleep(113 * time.Millisecond)
}
}
func (p *LocalPartition) MaybeShutdownLocalPartition() (hasShutdown bool) {
if p.Publishers.IsEmpty() && p.Subscribers.IsEmpty() {
p.LogBuffer.ShutdownLogBuffer()
hasShutdown = true
}
return
}
func (p *LocalPartition) Shutdown() {
p.closePublishers()
p.closeSubscribers()
p.LogBuffer.ShutdownLogBuffer()
atomic.StoreInt32(&p.FollowerId, 0)
}

7
weed/mq/topic/local_partition_publishers.go

@ -50,3 +50,10 @@ func (p *LocalPartitionPublishers) IsEmpty() bool {
return len(p.publishers) == 0
}
func (p *LocalPartitionPublishers) Size() int {
p.publishersLock.RLock()
defer p.publishersLock.RUnlock()
return len(p.publishers)
}

14
weed/mq/topic/local_partition_subscribers.go

@ -47,3 +47,17 @@ func (p *LocalPartitionSubscribers) SignalShutdown() {
Subscriber.SignalShutdown()
}
}
func (p *LocalPartitionSubscribers) IsEmpty() bool {
p.SubscribersLock.RLock()
defer p.SubscribersLock.RUnlock()
return len(p.Subscribers) == 0
}
func (p *LocalPartitionSubscribers) Size() int {
p.SubscribersLock.RLock()
defer p.SubscribersLock.RUnlock()
return len(p.Subscribers)
}

1
weed/mq/topic/local_topic.go

@ -27,6 +27,7 @@ func (localTopic *LocalTopic) removePartition(partition Partition) bool {
for i, localPartition := range localTopic.Partitions {
if localPartition.Partition.Equals(partition) {
foundPartitionIndex = i
localPartition.Shutdown()
break
}
}

9
weed/mq/topic/partition.go

@ -11,6 +11,15 @@ type Partition struct {
UnixTimeNs int64 // in nanoseconds
}
func NewPartition(rangeStart, rangeStop, ringSize int32, unixTimeNs int64) *Partition {
return &Partition{
RangeStart: rangeStart,
RangeStop: rangeStop,
RingSize: ringSize,
UnixTimeNs: unixTimeNs,
}
}
func (partition Partition) Equals(other Partition) bool {
if partition.RangeStart != other.RangeStart {
return false

6
weed/pb/filer.proto

@ -364,6 +364,7 @@ message LogEntry {
int64 ts_ns = 1;
int32 partition_key_hash = 2;
bytes data = 3;
bytes key = 4;
}
message KeepConnectedRequest {
@ -452,8 +453,9 @@ message LockRequest {
}
message LockResponse {
string renew_token = 1;
string moved_to = 2;
string error = 3;
string lock_owner = 2;
string lock_host_moved_to = 3;
string error = 4;
}
message UnlockRequest {
string name = 1;

548
weed/pb/filer_pb/filer.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.31.0
// protoc v4.24.4
// protoc-gen-go v1.32.0
// protoc v4.25.3
// source: filer.proto
package filer_pb
@ -2981,6 +2981,7 @@ type LogEntry struct {
TsNs int64 `protobuf:"varint,1,opt,name=ts_ns,json=tsNs,proto3" json:"ts_ns,omitempty"`
PartitionKeyHash int32 `protobuf:"varint,2,opt,name=partition_key_hash,json=partitionKeyHash,proto3" json:"partition_key_hash,omitempty"`
Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"`
}
func (x *LogEntry) Reset() {
@ -3036,6 +3037,13 @@ func (x *LogEntry) GetData() []byte {
return nil
}
func (x *LogEntry) GetKey() []byte {
if x != nil {
return x.Key
}
return nil
}
type KeepConnectedRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -3696,9 +3704,10 @@ type LockResponse struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
RenewToken string `protobuf:"bytes,1,opt,name=renew_token,json=renewToken,proto3" json:"renew_token,omitempty"`
MovedTo string `protobuf:"bytes,2,opt,name=moved_to,json=movedTo,proto3" json:"moved_to,omitempty"`
Error string `protobuf:"bytes,3,opt,name=error,proto3" json:"error,omitempty"`
RenewToken string `protobuf:"bytes,1,opt,name=renew_token,json=renewToken,proto3" json:"renew_token,omitempty"`
LockOwner string `protobuf:"bytes,2,opt,name=lock_owner,json=lockOwner,proto3" json:"lock_owner,omitempty"`
LockHostMovedTo string `protobuf:"bytes,3,opt,name=lock_host_moved_to,json=lockHostMovedTo,proto3" json:"lock_host_moved_to,omitempty"`
Error string `protobuf:"bytes,4,opt,name=error,proto3" json:"error,omitempty"`
}
func (x *LockResponse) Reset() {
@ -3740,9 +3749,16 @@ func (x *LockResponse) GetRenewToken() string {
return ""
}
func (x *LockResponse) GetMovedTo() string {
func (x *LockResponse) GetLockOwner() string {
if x != nil {
return x.MovedTo
return x.LockOwner
}
return ""
}
func (x *LockResponse) GetLockHostMovedTo() string {
if x != nil {
return x.LockHostMovedTo
}
return ""
}
@ -4732,273 +4748,277 @@ var file_filer_proto_rawDesc = []byte{
0x62, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74,
0x69, 0x6f, 0x6e, 0x52, 0x11, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x69,
0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18,
0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x22, 0x61, 0x0a, 0x08, 0x4c,
0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x22, 0x73, 0x0a, 0x08, 0x4c,
0x6f, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73,
0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x12, 0x2c, 0x0a, 0x12,
0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x68, 0x61,
0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,
0x69, 0x6f, 0x6e, 0x4b, 0x65, 0x79, 0x48, 0x61, 0x73, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61,
0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x65,
0x0a, 0x14, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 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, 0x1b, 0x0a, 0x09, 0x67, 0x72,
0x70, 0x63, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x08, 0x67,
0x72, 0x70, 0x63, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75,
0x72, 0x63, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65, 0x73, 0x6f,
0x75, 0x72, 0x63, 0x65, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e,
0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x31,
0x0a, 0x13, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63,
0x65, 0x22, 0xcd, 0x01, 0x0a, 0x14, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b,
0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f,
0x75, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x75, 0x6e, 0x64,
0x12, 0x45, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x02, 0x20,
0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x4c,
0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x09, 0x72, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x1a, 0x58, 0x0a, 0x08, 0x52, 0x65, 0x73, 0x6f, 0x75,
0x72, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x67, 0x72, 0x70, 0x63, 0x5f, 0x61, 0x64, 0x64, 0x72,
0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x67, 0x72, 0x70,
0x63, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01,
0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e,
0x74, 0x22, 0x20, 0x0a, 0x0c, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03,
0x6b, 0x65, 0x79, 0x22, 0x3b, 0x0a, 0x0d, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70,
0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20,
0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72,
0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72,
0x22, 0x36, 0x0a, 0x0c, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b,
0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,
0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x25, 0x0a, 0x0d, 0x4b, 0x76, 0x50, 0x75,
0x74, 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, 0x22,
0xee, 0x03, 0x0a, 0x09, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x12, 0x18, 0x0a,
0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07,
0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3a, 0x0a, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x74,
0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x66, 0x69, 0x6c,
0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x2e,
0x50, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69,
0x6f, 0x6e, 0x73, 0x1a, 0x8a, 0x03, 0x0a, 0x08, 0x50, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x66,
0x12, 0x27, 0x0a, 0x0f, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70, 0x72, 0x65,
0x66, 0x69, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x6c, 0x6f, 0x63, 0x61, 0x74,
0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c,
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63,
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x72, 0x65, 0x70,
0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b,
0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x74,
0x74, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x74, 0x74, 0x6c, 0x12, 0x1b, 0x0a,
0x09, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09,
0x52, 0x08, 0x64, 0x69, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x73,
0x79, 0x6e, 0x63, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x73, 0x79, 0x6e, 0x63,
0x12, 0x2e, 0x0a, 0x13, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x77, 0x74,
0x68, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x11, 0x76,
0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x47, 0x72, 0x6f, 0x77, 0x74, 0x68, 0x43, 0x6f, 0x75, 0x6e, 0x74,
0x12, 0x1b, 0x0a, 0x09, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18, 0x08, 0x20,
0x01, 0x28, 0x08, 0x52, 0x08, 0x72, 0x65, 0x61, 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x12, 0x1f, 0x0a,
0x0b, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x09, 0x20, 0x01,
0x28, 0x09, 0x52, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x43, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x12,
0x0a, 0x04, 0x72, 0x61, 0x63, 0x6b, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x72, 0x61,
0x63, 0x6b, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18,
0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x12,
0x2f, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65,
0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x11, 0x6d,
0x61, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68,
0x22, 0x5a, 0x0a, 0x26, 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, 0x12, 0x1c, 0x0a, 0x09, 0x64, 0x69,
0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x64,
0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65,
0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x50, 0x0a, 0x27,
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, 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, 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, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x73, 0x65, 0x63,
0x6f, 0x6e, 0x64, 0x73, 0x54, 0x6f, 0x4c, 0x6f, 0x63, 0x6b, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65,
0x6e, 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, 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, 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,
0x74, 0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x10,
0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79,
0x22, 0x65, 0x0a, 0x14, 0x4b, 0x65, 0x65, 0x70, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65,
0x64, 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, 0x1b, 0x0a, 0x09,
0x67, 0x72, 0x70, 0x63, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52,
0x08, 0x67, 0x72, 0x70, 0x63, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x72, 0x65,
0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x4b, 0x65, 0x65, 0x70, 0x43,
0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x22, 0x31, 0x0a, 0x13, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x6f, 0x75,
0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x72, 0x65, 0x73, 0x6f, 0x75,
0x72, 0x63, 0x65, 0x22, 0xcd, 0x01, 0x0a, 0x14, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72,
0x6f, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05,
0x66, 0x6f, 0x75, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x75,
0x6e, 0x64, 0x12, 0x45, 0x0a, 0x09, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18,
0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x66, 0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62,
0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73,
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x09,
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x1a, 0x58, 0x0a, 0x08, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x67, 0x72, 0x70, 0x63, 0x5f, 0x61, 0x64,
0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x67,
0x72, 0x70, 0x63, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e,
0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02,
0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x43, 0x6f,
0x75, 0x6e, 0x74, 0x22, 0x20, 0x0a, 0x0c, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75,
0x65, 0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c,
0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0x3b, 0x0a, 0x0d, 0x4b, 0x76, 0x47, 0x65, 0x74, 0x52, 0x65,
0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18,
0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a, 0x05,
0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72,
0x6f, 0x72, 0x22, 0x36, 0x0a, 0x0c, 0x4b, 0x76, 0x50, 0x75, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65,
0x73, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52,
0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20,
0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x25, 0x0a, 0x0d, 0x4b, 0x76,
0x50, 0x75, 0x74, 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, 0x22, 0xee, 0x03, 0x0a, 0x09, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x12,
0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05,
0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x3a, 0x0a, 0x09, 0x6c, 0x6f, 0x63,
0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x66,
0x69, 0x6c, 0x65, 0x72, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x72, 0x43, 0x6f, 0x6e,
0x66, 0x2e, 0x50, 0x61, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x61,
0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x8a, 0x03, 0x0a, 0x08, 0x50, 0x61, 0x74, 0x68, 0x43, 0x6f,
0x6e, 0x66, 0x12, 0x27, 0x0a, 0x0f, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x70,
0x72, 0x65, 0x66, 0x69, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x6c, 0x6f, 0x63,
0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x1e, 0x0a, 0x0a, 0x63,
0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x72,
0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09,
0x52, 0x0b, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x10, 0x0a,
0x03, 0x74, 0x74, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x74, 0x74, 0x6c, 0x12,
0x1b, 0x0a, 0x09, 0x64, 0x69, 0x73, 0x6b, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01,
0x28, 0x09, 0x52, 0x08, 0x64, 0x69, 0x73, 0x6b, 0x54, 0x79, 0x70, 0x65, 0x12, 0x14, 0x0a, 0x05,
0x66, 0x73, 0x79, 0x6e, 0x63, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x73, 0x79,
0x6e, 0x63, 0x12, 0x2e, 0x0a, 0x13, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x67, 0x72, 0x6f,
0x77, 0x74, 0x68, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0d, 0x52,
0x11, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x47, 0x72, 0x6f, 0x77, 0x74, 0x68, 0x43, 0x6f, 0x75,
0x6e, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x6f, 0x6e, 0x6c, 0x79, 0x18,
0x08, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x72, 0x65, 0x61, 0x64, 0x4f, 0x6e, 0x6c, 0x79, 0x12,
0x1f, 0x0a, 0x0b, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x09,
0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x43, 0x65, 0x6e, 0x74, 0x65, 0x72,
0x12, 0x12, 0x0a, 0x04, 0x72, 0x61, 0x63, 0x6b, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04,
0x72, 0x61, 0x63, 0x6b, 0x12, 0x1b, 0x0a, 0x09, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x6e, 0x6f, 0x64,
0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x64, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64,
0x65, 0x12, 0x2f, 0x0a, 0x14, 0x6d, 0x61, 0x78, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61,
0x6d, 0x65, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0d, 0x52,
0x11, 0x6d, 0x61, 0x78, 0x46, 0x69, 0x6c, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x4c, 0x65, 0x6e, 0x67,
0x74, 0x68, 0x22, 0x5a, 0x0a, 0x26, 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, 0x12, 0x1c, 0x0a, 0x09,
0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
0x09, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6f, 0x72, 0x79, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61,
0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x50,
0x0a, 0x27, 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, 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, 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, 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, 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,
0x6e, 0x61, 0x6d, 0x65, 0x12, 0x26, 0x0a, 0x0f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x5f,
0x74, 0x6f, 0x5f, 0x6c, 0x6f, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x73,
0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x54, 0x6f, 0x4c, 0x6f, 0x63, 0x6b, 0x12, 0x1f, 0x0a, 0x0b,
0x72, 0x65, 0x6e, 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, 0x12, 0x14, 0x0a, 0x05, 0x6f, 0x77, 0x6e, 0x65,
0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x22, 0x91,
0x01, 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, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x6f, 0x77, 0x6e, 0x65, 0x72, 0x18, 0x02,
0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, 0x6f, 0x63, 0x6b, 0x4f, 0x77, 0x6e, 0x65, 0x72, 0x12,
0x2b, 0x0a, 0x12, 0x6c, 0x6f, 0x63, 0x6b, 0x5f, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x6d, 0x6f, 0x76,
0x65, 0x64, 0x5f, 0x74, 0x6f, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x6c, 0x6f, 0x63,
0x6b, 0x48, 0x6f, 0x73, 0x74, 0x4d, 0x6f, 0x76, 0x65, 0x64, 0x54, 0x6f, 0x12, 0x14, 0x0a, 0x05,
0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 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, 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, 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, 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,
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, 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,
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, 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, 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,
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, 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, 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,
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 (

2
weed/pb/filer_pb/filer_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v4.24.4
// - protoc v4.25.3
// source: filer.proto
package filer_pb

4
weed/pb/iam_pb/iam.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.31.0
// protoc v4.24.4
// protoc-gen-go v1.32.0
// protoc v4.25.3
// source: iam.proto
package iam_pb

2
weed/pb/iam_pb/iam_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v4.24.4
// - protoc v4.25.3
// source: iam.proto
package iam_pb

4
weed/pb/master_pb/master.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.31.0
// protoc v4.24.4
// protoc-gen-go v1.32.0
// protoc v4.25.3
// source: master.proto
package master_pb

2
weed/pb/master_pb/master_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v4.24.4
// - protoc v4.25.3
// source: master.proto
package master_pb

4
weed/pb/mount_pb/mount.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.31.0
// protoc v4.24.4
// protoc-gen-go v1.32.0
// protoc v4.25.3
// source: mount.proto
package mount_pb

2
weed/pb/mount_pb/mount_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v4.24.4
// - protoc v4.25.3
// source: mount.proto
package mount_pb

82
weed/pb/mq.proto

@ -41,9 +41,14 @@ service SeaweedMessaging {
}
// data plane for each topic partition
rpc Publish (stream PublishRequest) returns (stream PublishResponse) {
rpc PublishMessage (stream PublishMessageRequest) returns (stream PublishMessageResponse) {
}
rpc Subscribe (SubscribeRequest) returns (stream SubscribeResponse) {
rpc SubscribeMessage (SubscribeMessageRequest) returns (stream SubscribeMessageResponse) {
}
// The lead broker asks a follower broker to follow itself
rpc PublishFollowMe (PublishFollowMeRequest) returns (PublishFollowMeResponse) {
}
rpc FollowInMemoryMessages (FollowInMemoryMessagesRequest) returns (stream FollowInMemoryMessagesResponse) {
}
}
@ -68,6 +73,24 @@ message Partition {
int64 unix_time_ns = 4;
}
message Offset {
Topic topic = 1;
repeated PartitionOffset partition_offsets = 2;
}
enum PartitionOffsetStartType {
EARLIEST = 0;
EARLIEST_IN_MEMORY = 1;
LATEST = 2;
}
message PartitionOffset {
Partition partition = 1;
int64 start_ts_ns = 2;
int64 stop_ts_ns = 3;
PartitionOffsetStartType start_type = 4;
}
//////////////////////////////////////////////////
message BrokerStats {
int32 cpu_usage_percent = 1;
@ -113,7 +136,6 @@ message ListTopicsResponse {
}
message LookupTopicBrokersRequest {
Topic topic = 1;
bool is_for_publish = 2;
}
message LookupTopicBrokersResponse {
Topic topic = 1;
@ -137,7 +159,7 @@ message AssignTopicPartitionsResponse {
message SubscriberToSubCoordinatorRequest {
message InitMessage {
string consumer_group = 1;
string consumer_instance_id = 2;
string consumer_group_instance_id = 2;
Topic topic = 3;
}
message AckMessage {
@ -153,6 +175,7 @@ message SubscriberToSubCoordinatorResponse {
message AssignedPartition {
Partition partition = 1;
int64 ts_ns = 2;
string broker = 3;
}
message Assignment {
int64 generation = 1;
@ -169,11 +192,12 @@ message DataMessage {
bytes value = 2;
int64 ts_ns = 3;
}
message PublishRequest {
message PublishMessageRequest {
message InitMessage {
Topic topic = 1;
Partition partition = 2;
int32 ack_interval = 3;
repeated string follower_brokers = 4;
}
oneof message {
InitMessage init = 1;
@ -181,23 +205,27 @@ message PublishRequest {
}
int64 sequence = 3;
}
message PublishResponse {
message PublishMessageResponse {
int64 ack_sequence = 1;
string error = 2;
bool should_close = 3;
}
message SubscribeRequest {
message PublishFollowMeRequest {
Topic topic = 1;
Partition partition = 2;
string broker_self = 3;
}
message PublishFollowMeResponse {
string error = 1;
}
message SubscribeMessageRequest {
message InitMessage {
string consumer_group = 1;
string consumer_id = 2;
string client_id = 3;
Topic topic = 4;
Partition partition = 5;
oneof offset {
int64 start_offset = 6;
int64 start_timestamp_ns = 7;
}
string filter = 8;
PartitionOffset partition_offset = 5;
string filter = 6;
}
message AckMessage {
int64 sequence = 1;
@ -207,7 +235,7 @@ message SubscribeRequest {
AckMessage ack = 2;
}
}
message SubscribeResponse {
message SubscribeMessageResponse {
message CtrlMessage {
string error = 1;
bool is_end_of_stream = 2;
@ -218,6 +246,32 @@ message SubscribeResponse {
DataMessage data = 2;
}
}
message FollowInMemoryMessagesRequest {
message InitMessage {
string consumer_group = 1;
string consumer_id = 2;
int32 follower_id = 3;
Topic topic = 4;
PartitionOffset partition_offset = 5;
}
message AckMessage {
int64 sequence = 1;
}
oneof message {
InitMessage init = 1;
AckMessage ack = 2;
}
}
message FollowInMemoryMessagesResponse {
message CtrlMessage {
int64 flushed_sequence = 1;
int32 follower_changed_to_id = 2;
}
oneof message {
CtrlMessage ctrl = 1;
DataMessage data = 2;
}
}
message ClosePublishersRequest {
Topic topic = 1;
int64 unix_time_ns = 2;

2372
weed/pb/mq_pb/mq.pb.go
File diff suppressed because it is too large
View File

201
weed/pb/mq_pb/mq_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v4.24.4
// - protoc v4.25.3
// source: mq.proto
package mq_pb
@ -29,8 +29,10 @@ const (
SeaweedMessaging_ClosePublishers_FullMethodName = "/messaging_pb.SeaweedMessaging/ClosePublishers"
SeaweedMessaging_CloseSubscribers_FullMethodName = "/messaging_pb.SeaweedMessaging/CloseSubscribers"
SeaweedMessaging_SubscriberToSubCoordinator_FullMethodName = "/messaging_pb.SeaweedMessaging/SubscriberToSubCoordinator"
SeaweedMessaging_Publish_FullMethodName = "/messaging_pb.SeaweedMessaging/Publish"
SeaweedMessaging_Subscribe_FullMethodName = "/messaging_pb.SeaweedMessaging/Subscribe"
SeaweedMessaging_PublishMessage_FullMethodName = "/messaging_pb.SeaweedMessaging/PublishMessage"
SeaweedMessaging_SubscribeMessage_FullMethodName = "/messaging_pb.SeaweedMessaging/SubscribeMessage"
SeaweedMessaging_PublishFollowMe_FullMethodName = "/messaging_pb.SeaweedMessaging/PublishFollowMe"
SeaweedMessaging_FollowInMemoryMessages_FullMethodName = "/messaging_pb.SeaweedMessaging/FollowInMemoryMessages"
)
// SeaweedMessagingClient is the client API for SeaweedMessaging service.
@ -53,8 +55,11 @@ type SeaweedMessagingClient interface {
// subscriber connects to broker balancer, which coordinates with the subscribers
SubscriberToSubCoordinator(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_SubscriberToSubCoordinatorClient, error)
// data plane for each topic partition
Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error)
Subscribe(ctx context.Context, in *SubscribeRequest, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeClient, error)
PublishMessage(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishMessageClient, error)
SubscribeMessage(ctx context.Context, in *SubscribeMessageRequest, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeMessageClient, error)
// The lead broker asks a follower broker to follow itself
PublishFollowMe(ctx context.Context, in *PublishFollowMeRequest, opts ...grpc.CallOption) (*PublishFollowMeResponse, error)
FollowInMemoryMessages(ctx context.Context, in *FollowInMemoryMessagesRequest, opts ...grpc.CallOption) (SeaweedMessaging_FollowInMemoryMessagesClient, error)
}
type seaweedMessagingClient struct {
@ -199,43 +204,43 @@ func (x *seaweedMessagingSubscriberToSubCoordinatorClient) Recv() (*SubscriberTo
return m, nil
}
func (c *seaweedMessagingClient) Publish(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishClient, error) {
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[2], SeaweedMessaging_Publish_FullMethodName, opts...)
func (c *seaweedMessagingClient) PublishMessage(ctx context.Context, opts ...grpc.CallOption) (SeaweedMessaging_PublishMessageClient, error) {
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[2], SeaweedMessaging_PublishMessage_FullMethodName, opts...)
if err != nil {
return nil, err
}
x := &seaweedMessagingPublishClient{stream}
x := &seaweedMessagingPublishMessageClient{stream}
return x, nil
}
type SeaweedMessaging_PublishClient interface {
Send(*PublishRequest) error
Recv() (*PublishResponse, error)
type SeaweedMessaging_PublishMessageClient interface {
Send(*PublishMessageRequest) error
Recv() (*PublishMessageResponse, error)
grpc.ClientStream
}
type seaweedMessagingPublishClient struct {
type seaweedMessagingPublishMessageClient struct {
grpc.ClientStream
}
func (x *seaweedMessagingPublishClient) Send(m *PublishRequest) error {
func (x *seaweedMessagingPublishMessageClient) Send(m *PublishMessageRequest) error {
return x.ClientStream.SendMsg(m)
}
func (x *seaweedMessagingPublishClient) Recv() (*PublishResponse, error) {
m := new(PublishResponse)
func (x *seaweedMessagingPublishMessageClient) Recv() (*PublishMessageResponse, error) {
m := new(PublishMessageResponse)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
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[3], SeaweedMessaging_Subscribe_FullMethodName, opts...)
func (c *seaweedMessagingClient) SubscribeMessage(ctx context.Context, in *SubscribeMessageRequest, opts ...grpc.CallOption) (SeaweedMessaging_SubscribeMessageClient, error) {
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[3], SeaweedMessaging_SubscribeMessage_FullMethodName, opts...)
if err != nil {
return nil, err
}
x := &seaweedMessagingSubscribeClient{stream}
x := &seaweedMessagingSubscribeMessageClient{stream}
if err := x.ClientStream.SendMsg(in); err != nil {
return nil, err
}
@ -245,17 +250,58 @@ func (c *seaweedMessagingClient) Subscribe(ctx context.Context, in *SubscribeReq
return x, nil
}
type SeaweedMessaging_SubscribeClient interface {
Recv() (*SubscribeResponse, error)
type SeaweedMessaging_SubscribeMessageClient interface {
Recv() (*SubscribeMessageResponse, error)
grpc.ClientStream
}
type seaweedMessagingSubscribeClient struct {
type seaweedMessagingSubscribeMessageClient struct {
grpc.ClientStream
}
func (x *seaweedMessagingSubscribeClient) Recv() (*SubscribeResponse, error) {
m := new(SubscribeResponse)
func (x *seaweedMessagingSubscribeMessageClient) Recv() (*SubscribeMessageResponse, error) {
m := new(SubscribeMessageResponse)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func (c *seaweedMessagingClient) PublishFollowMe(ctx context.Context, in *PublishFollowMeRequest, opts ...grpc.CallOption) (*PublishFollowMeResponse, error) {
out := new(PublishFollowMeResponse)
err := c.cc.Invoke(ctx, SeaweedMessaging_PublishFollowMe_FullMethodName, in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
func (c *seaweedMessagingClient) FollowInMemoryMessages(ctx context.Context, in *FollowInMemoryMessagesRequest, opts ...grpc.CallOption) (SeaweedMessaging_FollowInMemoryMessagesClient, error) {
stream, err := c.cc.NewStream(ctx, &SeaweedMessaging_ServiceDesc.Streams[4], SeaweedMessaging_FollowInMemoryMessages_FullMethodName, opts...)
if err != nil {
return nil, err
}
x := &seaweedMessagingFollowInMemoryMessagesClient{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_FollowInMemoryMessagesClient interface {
Recv() (*FollowInMemoryMessagesResponse, error)
grpc.ClientStream
}
type seaweedMessagingFollowInMemoryMessagesClient struct {
grpc.ClientStream
}
func (x *seaweedMessagingFollowInMemoryMessagesClient) Recv() (*FollowInMemoryMessagesResponse, error) {
m := new(FollowInMemoryMessagesResponse)
if err := x.ClientStream.RecvMsg(m); err != nil {
return nil, err
}
@ -282,8 +328,11 @@ type SeaweedMessagingServer interface {
// subscriber connects to broker balancer, which coordinates with the subscribers
SubscriberToSubCoordinator(SeaweedMessaging_SubscriberToSubCoordinatorServer) error
// data plane for each topic partition
Publish(SeaweedMessaging_PublishServer) error
Subscribe(*SubscribeRequest, SeaweedMessaging_SubscribeServer) error
PublishMessage(SeaweedMessaging_PublishMessageServer) error
SubscribeMessage(*SubscribeMessageRequest, SeaweedMessaging_SubscribeMessageServer) error
// The lead broker asks a follower broker to follow itself
PublishFollowMe(context.Context, *PublishFollowMeRequest) (*PublishFollowMeResponse, error)
FollowInMemoryMessages(*FollowInMemoryMessagesRequest, SeaweedMessaging_FollowInMemoryMessagesServer) error
mustEmbedUnimplementedSeaweedMessagingServer()
}
@ -321,11 +370,17 @@ func (UnimplementedSeaweedMessagingServer) CloseSubscribers(context.Context, *Cl
func (UnimplementedSeaweedMessagingServer) SubscriberToSubCoordinator(SeaweedMessaging_SubscriberToSubCoordinatorServer) error {
return status.Errorf(codes.Unimplemented, "method SubscriberToSubCoordinator not implemented")
}
func (UnimplementedSeaweedMessagingServer) Publish(SeaweedMessaging_PublishServer) error {
return status.Errorf(codes.Unimplemented, "method Publish not implemented")
func (UnimplementedSeaweedMessagingServer) PublishMessage(SeaweedMessaging_PublishMessageServer) error {
return status.Errorf(codes.Unimplemented, "method PublishMessage not implemented")
}
func (UnimplementedSeaweedMessagingServer) SubscribeMessage(*SubscribeMessageRequest, SeaweedMessaging_SubscribeMessageServer) error {
return status.Errorf(codes.Unimplemented, "method SubscribeMessage not implemented")
}
func (UnimplementedSeaweedMessagingServer) PublishFollowMe(context.Context, *PublishFollowMeRequest) (*PublishFollowMeResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method PublishFollowMe not implemented")
}
func (UnimplementedSeaweedMessagingServer) Subscribe(*SubscribeRequest, SeaweedMessaging_SubscribeServer) error {
return status.Errorf(codes.Unimplemented, "method Subscribe not implemented")
func (UnimplementedSeaweedMessagingServer) FollowInMemoryMessages(*FollowInMemoryMessagesRequest, SeaweedMessaging_FollowInMemoryMessagesServer) error {
return status.Errorf(codes.Unimplemented, "method FollowInMemoryMessages not implemented")
}
func (UnimplementedSeaweedMessagingServer) mustEmbedUnimplementedSeaweedMessagingServer() {}
@ -536,50 +591,89 @@ func (x *seaweedMessagingSubscriberToSubCoordinatorServer) Recv() (*SubscriberTo
return m, nil
}
func _SeaweedMessaging_Publish_Handler(srv interface{}, stream grpc.ServerStream) error {
return srv.(SeaweedMessagingServer).Publish(&seaweedMessagingPublishServer{stream})
func _SeaweedMessaging_PublishMessage_Handler(srv interface{}, stream grpc.ServerStream) error {
return srv.(SeaweedMessagingServer).PublishMessage(&seaweedMessagingPublishMessageServer{stream})
}
type SeaweedMessaging_PublishServer interface {
Send(*PublishResponse) error
Recv() (*PublishRequest, error)
type SeaweedMessaging_PublishMessageServer interface {
Send(*PublishMessageResponse) error
Recv() (*PublishMessageRequest, error)
grpc.ServerStream
}
type seaweedMessagingPublishServer struct {
type seaweedMessagingPublishMessageServer struct {
grpc.ServerStream
}
func (x *seaweedMessagingPublishServer) Send(m *PublishResponse) error {
func (x *seaweedMessagingPublishMessageServer) Send(m *PublishMessageResponse) error {
return x.ServerStream.SendMsg(m)
}
func (x *seaweedMessagingPublishServer) Recv() (*PublishRequest, error) {
m := new(PublishRequest)
func (x *seaweedMessagingPublishMessageServer) Recv() (*PublishMessageRequest, error) {
m := new(PublishMessageRequest)
if err := x.ServerStream.RecvMsg(m); err != nil {
return nil, err
}
return m, nil
}
func _SeaweedMessaging_Subscribe_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(SubscribeRequest)
func _SeaweedMessaging_SubscribeMessage_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(SubscribeMessageRequest)
if err := stream.RecvMsg(m); err != nil {
return err
}
return srv.(SeaweedMessagingServer).Subscribe(m, &seaweedMessagingSubscribeServer{stream})
return srv.(SeaweedMessagingServer).SubscribeMessage(m, &seaweedMessagingSubscribeMessageServer{stream})
}
type SeaweedMessaging_SubscribeServer interface {
Send(*SubscribeResponse) error
type SeaweedMessaging_SubscribeMessageServer interface {
Send(*SubscribeMessageResponse) error
grpc.ServerStream
}
type seaweedMessagingSubscribeServer struct {
type seaweedMessagingSubscribeMessageServer struct {
grpc.ServerStream
}
func (x *seaweedMessagingSubscribeServer) Send(m *SubscribeResponse) error {
func (x *seaweedMessagingSubscribeMessageServer) Send(m *SubscribeMessageResponse) error {
return x.ServerStream.SendMsg(m)
}
func _SeaweedMessaging_PublishFollowMe_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(PublishFollowMeRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(SeaweedMessagingServer).PublishFollowMe(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: SeaweedMessaging_PublishFollowMe_FullMethodName,
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(SeaweedMessagingServer).PublishFollowMe(ctx, req.(*PublishFollowMeRequest))
}
return interceptor(ctx, in, info, handler)
}
func _SeaweedMessaging_FollowInMemoryMessages_Handler(srv interface{}, stream grpc.ServerStream) error {
m := new(FollowInMemoryMessagesRequest)
if err := stream.RecvMsg(m); err != nil {
return err
}
return srv.(SeaweedMessagingServer).FollowInMemoryMessages(m, &seaweedMessagingFollowInMemoryMessagesServer{stream})
}
type SeaweedMessaging_FollowInMemoryMessagesServer interface {
Send(*FollowInMemoryMessagesResponse) error
grpc.ServerStream
}
type seaweedMessagingFollowInMemoryMessagesServer struct {
grpc.ServerStream
}
func (x *seaweedMessagingFollowInMemoryMessagesServer) Send(m *FollowInMemoryMessagesResponse) error {
return x.ServerStream.SendMsg(m)
}
@ -622,6 +716,10 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
MethodName: "CloseSubscribers",
Handler: _SeaweedMessaging_CloseSubscribers_Handler,
},
{
MethodName: "PublishFollowMe",
Handler: _SeaweedMessaging_PublishFollowMe_Handler,
},
},
Streams: []grpc.StreamDesc{
{
@ -637,14 +735,19 @@ var SeaweedMessaging_ServiceDesc = grpc.ServiceDesc{
ClientStreams: true,
},
{
StreamName: "Publish",
Handler: _SeaweedMessaging_Publish_Handler,
StreamName: "PublishMessage",
Handler: _SeaweedMessaging_PublishMessage_Handler,
ServerStreams: true,
ClientStreams: true,
},
{
StreamName: "Subscribe",
Handler: _SeaweedMessaging_Subscribe_Handler,
StreamName: "SubscribeMessage",
Handler: _SeaweedMessaging_SubscribeMessage_Handler,
ServerStreams: true,
},
{
StreamName: "FollowInMemoryMessages",
Handler: _SeaweedMessaging_FollowInMemoryMessages_Handler,
ServerStreams: true,
},
},

4
weed/pb/remote_pb/remote.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.31.0
// protoc v4.24.4
// protoc-gen-go v1.32.0
// protoc v4.25.3
// source: remote.proto
package remote_pb

4
weed/pb/s3_pb/s3.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.31.0
// protoc v4.24.4
// protoc-gen-go v1.32.0
// protoc v4.25.3
// source: s3.proto
package s3_pb

2
weed/pb/s3_pb/s3_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v4.24.4
// - protoc v4.25.3
// source: s3.proto
package s3_pb

4
weed/pb/volume_server_pb/volume_server.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// versions:
// protoc-gen-go v1.31.0
// protoc v4.24.4
// protoc-gen-go v1.32.0
// protoc v4.25.3
// source: volume_server.proto
package volume_server_pb

2
weed/pb/volume_server_pb/volume_server_grpc.pb.go

@ -1,7 +1,7 @@
// Code generated by protoc-gen-go-grpc. DO NOT EDIT.
// versions:
// - protoc-gen-go-grpc v1.3.0
// - protoc v4.24.4
// - protoc v4.25.3
// source: volume_server.proto
package volume_server_pb

4
weed/server/filer_grpc_server.go

@ -245,8 +245,8 @@ func (fs *FilerServer) AppendToEntry(ctx context.Context, req *filer_pb.AppendTo
fullpath := util.NewFullPath(req.Directory, req.EntryName)
lockClient := cluster.NewLockClient(fs.grpcDialOption, fs.option.Host)
lock := lockClient.NewLock(string(fullpath), string(fs.option.Host))
defer lock.StopLock()
lock := lockClient.NewShortLivedLock(string(fullpath), string(fs.option.Host))
defer lock.StopShortLivedLock()
var offset int64 = 0
entry, err := fs.filer.FindEntry(ctx, fullpath)

21
weed/server/filer_grpc_server_dlm.go

@ -3,6 +3,7 @@ package weed_server
import (
"context"
"fmt"
"github.com/seaweedfs/seaweedfs/weed/cluster/lock_manager"
"github.com/seaweedfs/seaweedfs/weed/glog"
"github.com/seaweedfs/seaweedfs/weed/pb"
"github.com/seaweedfs/seaweedfs/weed/pb/filer_pb"
@ -18,7 +19,7 @@ func (fs *FilerServer) DistributedLock(ctx context.Context, req *filer_pb.LockRe
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, req.Owner)
resp.LockOwner, resp.RenewToken, movedTo, err = fs.filer.Dlm.LockWithTimeout(req.Name, expiredAtNs, req.RenewToken, req.Owner)
glog.V(3).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 {
@ -31,7 +32,7 @@ func (fs *FilerServer) DistributedLock(ctx context.Context, req *filer_pb.LockRe
})
if err == nil {
resp.RenewToken = secondResp.RenewToken
} else {
resp.LockOwner = secondResp.LockOwner
resp.Error = secondResp.Error
}
return err
@ -42,7 +43,7 @@ func (fs *FilerServer) DistributedLock(ctx context.Context, req *filer_pb.LockRe
resp.Error = fmt.Sprintf("%v", err)
}
if movedTo != "" {
resp.MovedTo = string(movedTo)
resp.LockHostMovedTo = string(movedTo)
}
return resp, nil
@ -81,10 +82,7 @@ func (fs *FilerServer) DistributedUnlock(ctx context.Context, req *filer_pb.Unlo
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 != "" {
if !req.IsMoved && movedTo != "" || err == lock_manager.LockNotFound {
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,
@ -100,6 +98,15 @@ func (fs *FilerServer) FindLockOwner(ctx context.Context, req *filer_pb.FindLock
return nil, err
}
}
if owner == "" {
glog.V(0).Infof("find lock %s moved to %v: %v", req.Name, movedTo, err)
return nil, status.Error(codes.NotFound, fmt.Sprintf("lock %s not found", req.Name))
}
if err != nil {
return nil, status.Error(codes.Internal, err.Error())
}
return &filer_pb.FindLockOwnerResponse{
Owner: owner,
}, nil

21
weed/server/filer_grpc_server_sub_meta.go

@ -4,6 +4,7 @@ import (
"fmt"
"github.com/seaweedfs/seaweedfs/weed/stats"
"strings"
"sync/atomic"
"time"
"google.golang.org/protobuf/proto"
@ -32,7 +33,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
}
defer fs.deleteClient("", clientName, req.ClientId, req.ClientEpoch)
lastReadTime := time.Unix(0, req.SinceNs)
lastReadTime := log_buffer.NewMessagePosition(req.SinceNs, -2)
glog.V(0).Infof(" %v starts to subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName)
@ -57,7 +58,7 @@ func (fs *FilerServer) SubscribeMetadata(req *filer_pb.SubscribeMetadataRequest,
}
if processedTsNs != 0 {
lastReadTime = time.Unix(0, processedTsNs)
lastReadTime = log_buffer.NewMessagePosition(processedTsNs, -2)
}
glog.V(4).Infof("read in memory %v aggregated subscribe %s from %+v", clientName, req.PathPrefix, lastReadTime)
@ -113,7 +114,7 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
fs.deleteClient("local", clientName, req.ClientId, req.ClientEpoch)
}()
lastReadTime := time.Unix(0, req.SinceNs)
lastReadTime := log_buffer.NewMessagePosition(req.SinceNs, -2)
glog.V(0).Infof(" + %v local subscribe %s from %+v clientId:%d", clientName, req.PathPrefix, lastReadTime, req.ClientId)
eachEventNotificationFn := fs.eachEventNotificationFn(req, stream, clientName)
@ -138,7 +139,7 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
}
if processedTsNs != 0 {
lastReadTime = time.Unix(0, processedTsNs)
lastReadTime = log_buffer.NewMessagePosition(processedTsNs, -2)
} else {
if readInMemoryLogErr == log_buffer.ResumeFromDiskError {
time.Sleep(1127 * time.Millisecond)
@ -150,7 +151,9 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
lastReadTime, isDone, readInMemoryLogErr = fs.filer.LocalMetaLogBuffer.LoopProcessLogData("localMeta:"+clientName, lastReadTime, req.UntilNs, func() bool {
fs.listenersLock.Lock()
atomic.AddInt64(&fs.listenersWaits, 1)
fs.listenersCond.Wait()
atomic.AddInt64(&fs.listenersWaits, -1)
fs.listenersLock.Unlock()
if !fs.hasClient(req.ClientId, req.ClientEpoch) {
return false
@ -178,19 +181,19 @@ func (fs *FilerServer) SubscribeLocalMetadata(req *filer_pb.SubscribeMetadataReq
}
func eachLogEntryFn(eachEventNotificationFn func(dirPath string, eventNotification *filer_pb.EventNotification, tsNs int64) error) func(logEntry *filer_pb.LogEntry) error {
return func(logEntry *filer_pb.LogEntry) error {
func eachLogEntryFn(eachEventNotificationFn func(dirPath string, eventNotification *filer_pb.EventNotification, tsNs int64) error) log_buffer.EachLogEntryFuncType {
return func(logEntry *filer_pb.LogEntry) (bool, error) {
event := &filer_pb.SubscribeMetadataResponse{}
if err := proto.Unmarshal(logEntry.Data, event); err != nil {
glog.Errorf("unexpected unmarshal filer_pb.SubscribeMetadataResponse: %v", err)
return fmt.Errorf("unexpected unmarshal filer_pb.SubscribeMetadataResponse: %v", err)
return false, fmt.Errorf("unexpected unmarshal filer_pb.SubscribeMetadataResponse: %v", err)
}
if err := eachEventNotificationFn(event.Directory, event.EventNotification, event.TsNs); err != nil {
return err
return false, err
}
return nil
return false, nil
}
}

17
weed/server/filer_server.go

@ -7,6 +7,7 @@ import (
"os"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/seaweedfs/seaweedfs/weed/stats"
@ -76,7 +77,13 @@ type FilerOption struct {
}
type FilerServer struct {
inFlightDataSize int64
inFlightDataSize int64
listenersWaits int64
// notifying clients
listenersLock sync.Mutex
listenersCond *sync.Cond
inFlightDataLimitCond *sync.Cond
filer_pb.UnimplementedSeaweedFilerServer
@ -90,10 +97,6 @@ type FilerServer struct {
metricsAddress string
metricsIntervalSec int
// notifying clients
listenersLock sync.Mutex
listenersCond *sync.Cond
// track known metadata listeners
knownListenersLock sync.Mutex
knownListeners map[int32]int32
@ -135,7 +138,9 @@ func NewFilerServer(defaultMux, readonlyMux *http.ServeMux, option *FilerOption)
v.SetDefault("filer.options.max_file_name_length", 255)
maxFilenameLength := v.GetUint32("filer.options.max_file_name_length")
fs.filer = filer.NewFiler(*option.Masters, fs.grpcDialOption, option.Host, option.FilerGroup, option.Collection, option.DefaultReplication, option.DataCenter, maxFilenameLength, func() {
fs.listenersCond.Broadcast()
if atomic.LoadInt64(&fs.listenersWaits) > 0 {
fs.listenersCond.Broadcast()
}
})
fs.filer.Cipher = option.Cipher
// we do not support IP whitelist right now

1
weed/shell/command_mq_topic_desc.go

@ -46,7 +46,6 @@ func (c *commandMqTopicDescribe) Do(args []string, commandEnv *CommandEnv, write
Namespace: *namespace,
Name: *topicName,
},
IsForPublish: false,
})
if err != nil {
return err

2
weed/shell/command_mq_topic_list.go

@ -55,7 +55,7 @@ func findBrokerBalancer(commandEnv *CommandEnv) (brokerBalancer string, err erro
Name: pub_balancer.LockBrokerBalancer,
})
if err != nil {
return err
return fmt.Errorf("FindLockOwner: %v", err)
}
brokerBalancer = resp.Owner
return nil

137
weed/util/buffered_queue/buffered_queue.go

@ -0,0 +1,137 @@
package buffered_queue
import (
"fmt"
"sync"
)
// ItemChunkNode represents a node in the linked list of job chunks
type ItemChunkNode[T any] struct {
items []T
headIndex int
tailIndex int
next *ItemChunkNode[T]
nodeId int
}
// BufferedQueue implements a buffered queue using a linked list of job chunks
type BufferedQueue[T any] struct {
chunkSize int // Maximum number of items per chunk
head *ItemChunkNode[T]
tail *ItemChunkNode[T]
last *ItemChunkNode[T] // Pointer to the last chunk, for reclaiming memory
count int // Total number of items in the queue
mutex sync.Mutex
nodeCounter int
waitCond *sync.Cond
isClosed bool
}
// NewBufferedQueue creates a new buffered queue with the specified chunk size
func NewBufferedQueue[T any](chunkSize int) *BufferedQueue[T] {
// Create an empty chunk to initialize head and tail
chunk := &ItemChunkNode[T]{items: make([]T, chunkSize), nodeId: 0}
bq := &BufferedQueue[T]{
chunkSize: chunkSize,
head: chunk,
tail: chunk,
last: chunk,
count: 0,
mutex: sync.Mutex{},
}
bq.waitCond = sync.NewCond(&bq.mutex)
return bq
}
// Enqueue adds a job to the queue
func (q *BufferedQueue[T]) Enqueue(job T) error {
if q.isClosed {
return fmt.Errorf("queue is closed")
}
q.mutex.Lock()
defer q.mutex.Unlock()
// If the tail chunk is full, create a new chunk (reusing empty chunks if available)
if q.tail.tailIndex == q.chunkSize {
if q.tail == q.last {
// Create a new chunk
q.nodeCounter++
newChunk := &ItemChunkNode[T]{items: make([]T, q.chunkSize), nodeId: q.nodeCounter}
q.tail.next = newChunk
q.tail = newChunk
q.last = newChunk
} else {
// Reuse an empty chunk
q.tail = q.tail.next
q.tail.headIndex = 0
q.tail.tailIndex = 0
// println("tail moved to chunk", q.tail.nodeId)
}
}
// Add the job to the tail chunk
q.tail.items[q.tail.tailIndex] = job
q.tail.tailIndex++
q.count++
if q.count == 1 {
q.waitCond.Signal()
}
return nil
}
// Dequeue removes and returns a job from the queue
func (q *BufferedQueue[T]) Dequeue() (T, bool) {
q.mutex.Lock()
defer q.mutex.Unlock()
for q.count <= 0 && !q.isClosed {
q.waitCond.Wait()
}
if q.count <= 0 && q.isClosed {
var a T
return a, false
}
job := q.head.items[q.head.headIndex]
q.head.headIndex++
q.count--
if q.head.headIndex == q.chunkSize {
q.last.next = q.head
q.head = q.head.next
q.last = q.last.next
q.last.next = nil
//println("reusing chunk", q.last.nodeId)
//fmt.Printf("head: %+v\n", q.head)
//fmt.Printf("tail: %+v\n", q.tail)
//fmt.Printf("last: %+v\n", q.last)
//fmt.Printf("count: %d\n", q.count)
//for p := q.head; p != nil ; p = p.next {
// fmt.Printf("Node: %+v\n", p)
//}
}
return job, true
}
// Size returns the number of items in the queue
func (q *BufferedQueue[T]) Size() int {
q.mutex.Lock()
defer q.mutex.Unlock()
return q.count
}
// IsEmpty returns true if the queue is empty
func (q *BufferedQueue[T]) IsEmpty() bool {
return q.Size() == 0
}
func (q *BufferedQueue[T]) CloseInput() {
q.mutex.Lock()
defer q.mutex.Unlock()
q.isClosed = true
q.waitCond.Broadcast()
}

128
weed/util/buffered_queue/buffered_queue_test.go

@ -0,0 +1,128 @@
package buffered_queue
import (
"sync"
"testing"
)
func TestJobQueue(t *testing.T) {
type Job[T any] struct {
ID int
Action string
Data T
}
queue := NewBufferedQueue[Job[string]](2) // Chunk size of 5
queue.Enqueue(Job[string]{ID: 1, Action: "task1", Data: "hello"})
queue.Enqueue(Job[string]{ID: 2, Action: "task2", Data: "world"})
if queue.Size() != 2 {
t.Errorf("Expected queue size of 2, got %d", queue.Size())
}
queue.Enqueue(Job[string]{ID: 3, Action: "task3", Data: "3!"})
queue.Enqueue(Job[string]{ID: 4, Action: "task4", Data: "4!"})
queue.Enqueue(Job[string]{ID: 5, Action: "task5", Data: "5!"})
if queue.Size() != 5 {
t.Errorf("Expected queue size of 5, got %d", queue.Size())
}
println("enqueued 5 items")
println("dequeue", 1)
job, ok := queue.Dequeue()
if !ok {
t.Errorf("Expected dequeue to return true")
}
if job.ID != 1 {
t.Errorf("Expected job ID of 1, got %d", job.ID)
}
println("dequeue", 2)
job, ok = queue.Dequeue()
if !ok {
t.Errorf("Expected dequeue to return true")
}
println("enqueue", 6)
queue.Enqueue(Job[string]{ID: 6, Action: "task6", Data: "6!"})
println("enqueue", 7)
queue.Enqueue(Job[string]{ID: 7, Action: "task7", Data: "7!"})
for i := 0; i < 5; i++ {
println("dequeue ...")
job, ok = queue.Dequeue()
if !ok {
t.Errorf("Expected dequeue to return true")
}
println("dequeued", job.ID)
}
if queue.Size() != 0 {
t.Errorf("Expected queue size of 0, got %d", queue.Size())
}
for i := 0; i < 5; i++ {
println("enqueue", i+8)
queue.Enqueue(Job[string]{ID: i + 8, Action: "task", Data: "data"})
}
for i := 0; i < 5; i++ {
job, ok = queue.Dequeue()
if !ok {
t.Errorf("Expected dequeue to return true")
}
if job.ID != i+8 {
t.Errorf("Expected job ID of %d, got %d", i, job.ID)
}
println("dequeued", job.ID)
}
}
func TestJobQueueClose(t *testing.T) {
type Job[T any] struct {
ID int
Action string
Data T
}
queue := NewBufferedQueue[Job[string]](2)
queue.Enqueue(Job[string]{ID: 1, Action: "task1", Data: "hello"})
queue.Enqueue(Job[string]{ID: 2, Action: "task2", Data: "world"})
wg := sync.WaitGroup{}
wg.Add(1)
go func() {
defer wg.Done()
for data, ok := queue.Dequeue(); ok; data, ok = queue.Dequeue() {
println("dequeued", data.ID)
}
}()
for i := 0; i < 5; i++ {
queue.Enqueue(Job[string]{ID: i + 3, Action: "task", Data: "data"})
}
queue.CloseInput()
wg.Wait()
}
func BenchmarkBufferedQueue(b *testing.B) {
type Job[T any] struct {
ID int
Action string
Data T
}
queue := NewBufferedQueue[Job[string]](1024)
for i := 0; i < b.N; i++ {
queue.Enqueue(Job[string]{ID: i, Action: "task", Data: "data"})
}
for i := 0; i < b.N; i++ {
_, _ = queue.Dequeue()
}
}

38
weed/util/http_util.go

@ -10,6 +10,7 @@ import (
"net/http"
"net/url"
"strings"
"time"
"github.com/seaweedfs/seaweedfs/weed/glog"
)
@ -450,3 +451,40 @@ func (r *CountingReader) Read(p []byte) (n int, err error) {
r.BytesRead += n
return n, err
}
func RetriedFetchChunkData(buffer []byte, urlStrings []string, cipherKey []byte, isGzipped bool, isFullChunk bool, offset int64) (n int, err error) {
var shouldRetry bool
for waitTime := time.Second; waitTime < RetryWaitTime; waitTime += waitTime / 2 {
for _, urlString := range urlStrings {
n = 0
if strings.Contains(urlString, "%") {
urlString = url.PathEscape(urlString)
}
shouldRetry, err = ReadUrlAsStream(urlString+"?readDeleted=true", cipherKey, isGzipped, isFullChunk, offset, len(buffer), func(data []byte) {
if n < len(buffer) {
x := copy(buffer[n:], data)
n += x
}
})
if !shouldRetry {
break
}
if err != nil {
glog.V(0).Infof("read %s failed, err: %v", urlString, err)
} else {
break
}
}
if err != nil && shouldRetry {
glog.V(0).Infof("retry reading in %v", waitTime)
time.Sleep(waitTime)
} else {
break
}
}
return n, err
}

257
weed/util/log_buffer/log_buffer.go

@ -22,53 +22,62 @@ type dataToFlush struct {
data *bytes.Buffer
}
type EachLogEntryFuncType func(logEntry *filer_pb.LogEntry) (isDone bool, err error)
type LogFlushFuncType func(logBuffer *LogBuffer, startTime, stopTime time.Time, buf []byte)
type LogReadFromDiskFuncType func(startPosition MessagePosition, stopTsNs int64, eachLogEntryFn EachLogEntryFuncType) (lastReadPosition MessagePosition, isDone bool, err error)
type LogBuffer struct {
name string
prevBuffers *SealedBuffers
buf []byte
idx []int
pos int
startTime time.Time
stopTime time.Time
lastFlushTime time.Time
sizeBuf []byte
flushInterval time.Duration
flushFn func(startTime, stopTime time.Time, buf []byte)
notifyFn func()
isStopping *atomic.Bool
flushChan chan *dataToFlush
lastTsNs int64
LastFlushTsNs int64
name string
prevBuffers *SealedBuffers
buf []byte
batchIndex int64
idx []int
pos int
startTime time.Time
stopTime time.Time
lastFlushDataTime time.Time
sizeBuf []byte
flushInterval time.Duration
flushFn LogFlushFuncType
ReadFromDiskFn LogReadFromDiskFuncType
notifyFn func()
isStopping *atomic.Bool
flushChan chan *dataToFlush
LastTsNs int64
sync.RWMutex
}
func NewLogBuffer(name string, flushInterval time.Duration, flushFn func(startTime, stopTime time.Time, buf []byte), notifyFn func()) *LogBuffer {
func NewLogBuffer(name string, flushInterval time.Duration, flushFn LogFlushFuncType,
readFromDiskFn LogReadFromDiskFuncType, notifyFn func()) *LogBuffer {
lb := &LogBuffer{
name: name,
prevBuffers: newSealedBuffers(PreviousBufferCount),
buf: make([]byte, BufferSize),
sizeBuf: make([]byte, 4),
flushInterval: flushInterval,
flushFn: flushFn,
notifyFn: notifyFn,
flushChan: make(chan *dataToFlush, 256),
isStopping: new(atomic.Bool),
name: name,
prevBuffers: newSealedBuffers(PreviousBufferCount),
buf: make([]byte, BufferSize),
sizeBuf: make([]byte, 4),
flushInterval: flushInterval,
flushFn: flushFn,
ReadFromDiskFn: readFromDiskFn,
notifyFn: notifyFn,
flushChan: make(chan *dataToFlush, 256),
isStopping: new(atomic.Bool),
}
go lb.loopFlush()
go lb.loopInterval()
return lb
}
func (m *LogBuffer) AddToBuffer(partitionKey, data []byte, processingTsNs int64) {
func (logBuffer *LogBuffer) AddToBuffer(partitionKey, data []byte, processingTsNs int64) {
var toFlush *dataToFlush
m.Lock()
logBuffer.Lock()
defer func() {
m.Unlock()
logBuffer.Unlock()
if toFlush != nil {
m.flushChan <- toFlush
logBuffer.flushChan <- toFlush
}
if m.notifyFn != nil {
m.notifyFn()
if logBuffer.notifyFn != nil {
logBuffer.notifyFn()
}
}()
@ -80,121 +89,136 @@ func (m *LogBuffer) AddToBuffer(partitionKey, data []byte, processingTsNs int64)
} else {
ts = time.Unix(0, processingTsNs)
}
if m.lastTsNs >= processingTsNs {
if logBuffer.LastTsNs >= processingTsNs {
// this is unlikely to happen, but just in case
processingTsNs = m.lastTsNs + 1
processingTsNs = logBuffer.LastTsNs + 1
ts = time.Unix(0, processingTsNs)
}
m.lastTsNs = processingTsNs
logBuffer.LastTsNs = processingTsNs
logEntry := &filer_pb.LogEntry{
TsNs: processingTsNs,
PartitionKeyHash: util.HashToInt32(partitionKey),
Data: data,
Key: partitionKey,
}
logEntryData, _ := proto.Marshal(logEntry)
size := len(logEntryData)
if m.pos == 0 {
m.startTime = ts
if logBuffer.pos == 0 {
logBuffer.startTime = ts
}
if m.startTime.Add(m.flushInterval).Before(ts) || len(m.buf)-m.pos < size+4 {
// glog.V(4).Infof("%s copyToFlush1 start time %v, ts %v, remaining %d bytes", m.name, m.startTime, ts, len(m.buf)-m.pos)
toFlush = m.copyToFlush()
m.startTime = ts
if len(m.buf) < size+4 {
m.buf = make([]byte, 2*size+4)
if logBuffer.startTime.Add(logBuffer.flushInterval).Before(ts) || len(logBuffer.buf)-logBuffer.pos < size+4 {
// glog.V(0).Infof("%s copyToFlush1 batch:%d count:%d start time %v, ts %v, remaining %d bytes", logBuffer.name, logBuffer.batchIndex, len(logBuffer.idx), logBuffer.startTime, ts, len(logBuffer.buf)-logBuffer.pos)
toFlush = logBuffer.copyToFlush()
logBuffer.startTime = ts
if len(logBuffer.buf) < size+4 {
logBuffer.buf = make([]byte, 2*size+4)
}
}
m.stopTime = ts
logBuffer.stopTime = ts
m.idx = append(m.idx, m.pos)
util.Uint32toBytes(m.sizeBuf, uint32(size))
copy(m.buf[m.pos:m.pos+4], m.sizeBuf)
copy(m.buf[m.pos+4:m.pos+4+size], logEntryData)
m.pos += size + 4
logBuffer.idx = append(logBuffer.idx, logBuffer.pos)
util.Uint32toBytes(logBuffer.sizeBuf, uint32(size))
copy(logBuffer.buf[logBuffer.pos:logBuffer.pos+4], logBuffer.sizeBuf)
copy(logBuffer.buf[logBuffer.pos+4:logBuffer.pos+4+size], logEntryData)
logBuffer.pos += size + 4
// fmt.Printf("entry size %d total %d count %d, buffer:%p\n", size, m.pos, len(m.idx), m)
// fmt.Printf("partitionKey %v entry size %d total %d count %d\n", string(partitionKey), size, m.pos, len(m.idx))
}
func (m *LogBuffer) IsStopping() bool {
return m.isStopping.Load()
func (logBuffer *LogBuffer) IsStopping() bool {
return logBuffer.isStopping.Load()
}
func (m *LogBuffer) Shutdown() {
isAlreadyStopped := m.isStopping.Swap(true)
func (logBuffer *LogBuffer) ShutdownLogBuffer() {
isAlreadyStopped := logBuffer.isStopping.Swap(true)
if isAlreadyStopped {
return
}
toFlush := m.copyToFlush()
m.flushChan <- toFlush
close(m.flushChan)
toFlush := logBuffer.copyToFlush()
logBuffer.flushChan <- toFlush
close(logBuffer.flushChan)
}
func (m *LogBuffer) loopFlush() {
for d := range m.flushChan {
func (logBuffer *LogBuffer) loopFlush() {
for d := range logBuffer.flushChan {
if d != nil {
// glog.V(4).Infof("%s flush [%v, %v] size %d", m.name, d.startTime, d.stopTime, len(d.data.Bytes()))
m.flushFn(d.startTime, d.stopTime, d.data.Bytes())
logBuffer.flushFn(logBuffer, d.startTime, d.stopTime, d.data.Bytes())
d.releaseMemory()
// local logbuffer is different from aggregate logbuffer here
m.lastFlushTime = d.stopTime
logBuffer.lastFlushDataTime = d.stopTime
}
}
}
func (m *LogBuffer) loopInterval() {
for !m.IsStopping() {
time.Sleep(m.flushInterval)
if m.IsStopping() {
func (logBuffer *LogBuffer) loopInterval() {
for !logBuffer.IsStopping() {
time.Sleep(logBuffer.flushInterval)
if logBuffer.IsStopping() {
return
}
m.Lock()
toFlush := m.copyToFlush()
m.Unlock()
logBuffer.Lock()
toFlush := logBuffer.copyToFlush()
logBuffer.Unlock()
if toFlush != nil {
m.flushChan <- toFlush
glog.V(0).Infof("%s flush [%v, %v] size %d", logBuffer.name, toFlush.startTime, toFlush.stopTime, len(toFlush.data.Bytes()))
logBuffer.flushChan <- toFlush
} else {
// glog.V(0).Infof("%s no flush", m.name)
}
}
}
func (m *LogBuffer) copyToFlush() *dataToFlush {
func (logBuffer *LogBuffer) copyToFlush() *dataToFlush {
if m.pos > 0 {
if logBuffer.pos > 0 {
// fmt.Printf("flush buffer %d pos %d empty space %d\n", len(m.buf), m.pos, len(m.buf)-m.pos)
var d *dataToFlush
if m.flushFn != nil {
if logBuffer.flushFn != nil {
d = &dataToFlush{
startTime: m.startTime,
stopTime: m.stopTime,
data: copiedBytes(m.buf[:m.pos]),
startTime: logBuffer.startTime,
stopTime: logBuffer.stopTime,
data: copiedBytes(logBuffer.buf[:logBuffer.pos]),
}
// glog.V(4).Infof("%s flushing [0,%d) with %d entries [%v, %v]", m.name, m.pos, len(m.idx), m.startTime, m.stopTime)
} else {
// glog.V(4).Infof("%s removed from memory [0,%d) with %d entries [%v, %v]", m.name, m.pos, len(m.idx), m.startTime, m.stopTime)
m.lastFlushTime = m.stopTime
logBuffer.lastFlushDataTime = logBuffer.stopTime
}
m.buf = m.prevBuffers.SealBuffer(m.startTime, m.stopTime, m.buf, m.pos)
m.startTime = time.Unix(0, 0)
m.stopTime = time.Unix(0, 0)
m.pos = 0
m.idx = m.idx[:0]
logBuffer.buf = logBuffer.prevBuffers.SealBuffer(logBuffer.startTime, logBuffer.stopTime, logBuffer.buf, logBuffer.pos, logBuffer.batchIndex)
logBuffer.startTime = time.Unix(0, 0)
logBuffer.stopTime = time.Unix(0, 0)
logBuffer.pos = 0
logBuffer.idx = logBuffer.idx[:0]
logBuffer.batchIndex++
return d
}
return nil
}
func (logBuffer *LogBuffer) GetEarliestTime() time.Time {
return logBuffer.startTime
}
func (logBuffer *LogBuffer) GetEarliestPosition() MessagePosition {
return MessagePosition{
Time: logBuffer.startTime,
BatchIndex: logBuffer.batchIndex,
}
}
func (d *dataToFlush) releaseMemory() {
d.data.Reset()
bufferPool.Put(d.data)
}
func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Buffer, err error) {
m.RLock()
defer m.RUnlock()
func (logBuffer *LogBuffer) ReadFromBuffer(lastReadPosition MessagePosition) (bufferCopy *bytes.Buffer, batchIndex int64, err error) {
logBuffer.RLock()
defer logBuffer.RUnlock()
// Read from disk and memory
// 1. read from disk, last time is = td
@ -206,52 +230,56 @@ func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Bu
// if td < tm, case 2.3
// read from disk again
var tsMemory time.Time
if !m.startTime.IsZero() {
tsMemory = m.startTime
var tsBatchIndex int64
if !logBuffer.startTime.IsZero() {
tsMemory = logBuffer.startTime
tsBatchIndex = logBuffer.batchIndex
}
for _, prevBuf := range m.prevBuffers.buffers {
for _, prevBuf := range logBuffer.prevBuffers.buffers {
if !prevBuf.startTime.IsZero() && prevBuf.startTime.Before(tsMemory) {
tsMemory = prevBuf.startTime
tsBatchIndex = prevBuf.batchIndex
}
}
if tsMemory.IsZero() { // case 2.2
return nil, nil
} else if lastReadTime.Before(tsMemory) { // case 2.3
if !m.lastFlushTime.IsZero() {
glog.V(0).Infof("resume with last flush time: %v", m.lastFlushTime)
return nil, ResumeFromDiskError
println("2.2 no data")
return nil, -2, nil
} else if lastReadPosition.Before(tsMemory) && lastReadPosition.BatchIndex+1 < tsBatchIndex { // case 2.3
if !logBuffer.lastFlushDataTime.IsZero() {
glog.V(0).Infof("resume with last flush time: %v", logBuffer.lastFlushDataTime)
return nil, -2, ResumeFromDiskError
}
}
// the following is case 2.1
if lastReadTime.Equal(m.stopTime) {
return nil, nil
if lastReadPosition.Equal(logBuffer.stopTime) {
return nil, logBuffer.batchIndex, nil
}
if lastReadTime.After(m.stopTime) {
// glog.Fatalf("unexpected last read time %v, older than latest %v", lastReadTime, m.stopTime)
return nil, nil
if lastReadPosition.After(logBuffer.stopTime) {
// glog.Fatalf("unexpected last read time %v, older than latest %v", lastReadPosition, m.stopTime)
return nil, logBuffer.batchIndex, nil
}
if lastReadTime.Before(m.startTime) {
// println("checking ", lastReadTime.UnixNano())
for _, buf := range m.prevBuffers.buffers {
if buf.startTime.After(lastReadTime) {
if lastReadPosition.Before(logBuffer.startTime) {
// println("checking ", lastReadPosition.UnixNano())
for _, buf := range logBuffer.prevBuffers.buffers {
if buf.startTime.After(lastReadPosition.Time) {
// glog.V(4).Infof("%s return the %d sealed buffer %v", m.name, i, buf.startTime)
// println("return the", i, "th in memory", buf.startTime.UnixNano())
return copiedBytes(buf.buf[:buf.size]), nil
return copiedBytes(buf.buf[:buf.size]), buf.batchIndex, nil
}
if !buf.startTime.After(lastReadTime) && buf.stopTime.After(lastReadTime) {
pos := buf.locateByTs(lastReadTime)
if !buf.startTime.After(lastReadPosition.Time) && buf.stopTime.After(lastReadPosition.Time) {
pos := buf.locateByTs(lastReadPosition.Time)
// fmt.Printf("locate buffer[%d] pos %d\n", i, pos)
return copiedBytes(buf.buf[pos:buf.size]), nil
return copiedBytes(buf.buf[pos:buf.size]), buf.batchIndex, nil
}
}
// glog.V(4).Infof("%s return the current buf %v", m.name, lastReadTime)
return copiedBytes(m.buf[:m.pos]), nil
// glog.V(4).Infof("%s return the current buf %v", m.name, lastReadPosition)
return copiedBytes(logBuffer.buf[:logBuffer.pos]), logBuffer.batchIndex, nil
}
lastTs := lastReadTime.UnixNano()
l, h := 0, len(m.idx)-1
lastTs := lastReadPosition.UnixNano()
l, h := 0, len(logBuffer.idx)-1
/*
for i, pos := range m.idx {
@ -269,18 +297,18 @@ func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Bu
for l <= h {
mid := (l + h) / 2
pos := m.idx[mid]
_, t := readTs(m.buf, pos)
pos := logBuffer.idx[mid]
_, t := readTs(logBuffer.buf, pos)
if t <= lastTs {
l = mid + 1
} else if lastTs < t {
var prevT int64
if mid > 0 {
_, prevT = readTs(m.buf, m.idx[mid-1])
_, prevT = readTs(logBuffer.buf, logBuffer.idx[mid-1])
}
if prevT <= lastTs {
// fmt.Printf("found l=%d, m-1=%d(ts=%d), m=%d(ts=%d), h=%d [%d, %d) \n", l, mid-1, prevT, mid, t, h, pos, m.pos)
return copiedBytes(m.buf[pos:m.pos]), nil
return copiedBytes(logBuffer.buf[pos:logBuffer.pos]), logBuffer.batchIndex, nil
}
h = mid
}
@ -288,10 +316,11 @@ func (m *LogBuffer) ReadFromBuffer(lastReadTime time.Time) (bufferCopy *bytes.Bu
}
// FIXME: this could be that the buffer has been flushed already
return nil, nil
println("Not sure why no data", lastReadPosition.BatchIndex, tsBatchIndex)
return nil, -2, nil
}
func (m *LogBuffer) ReleaseMemory(b *bytes.Buffer) {
func (logBuffer *LogBuffer) ReleaseMemory(b *bytes.Buffer) {
bufferPool.Put(b)
}

2
weed/util/log_buffer/log_buffer_test.go

@ -15,7 +15,7 @@ func TestNewLogBufferFirstBuffer(t *testing.T) {
flushInterval := time.Second
lb := NewLogBuffer("test", flushInterval, func(startTime, stopTime time.Time, buf []byte) {
fmt.Printf("flush from %v to %v %d bytes\n", startTime, stopTime, len(buf))
}, func() {
}, nil, func() {
})
startTime := time.Now()

66
weed/util/log_buffer/log_read.go

@ -17,15 +17,30 @@ var (
ResumeFromDiskError = fmt.Errorf("resumeFromDisk")
)
func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startReadTime time.Time, stopTsNs int64,
waitForDataFn func() bool, eachLogDataFn func(logEntry *filer_pb.LogEntry) error) (lastReadTime time.Time, isDone bool, err error) {
type MessagePosition struct {
time.Time // this is the timestamp of the message
BatchIndex int64 // this is only used when the timestamp is not enough to identify the next message, when the timestamp is in the previous batch.
}
func NewMessagePosition(tsNs int64, batchIndex int64) MessagePosition {
return MessagePosition{
Time: time.Unix(0, tsNs).UTC(),
BatchIndex: batchIndex,
}
}
func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startPosition MessagePosition, stopTsNs int64,
waitForDataFn func() bool, eachLogDataFn EachLogEntryFuncType) (lastReadPosition MessagePosition, isDone bool, err error) {
// loop through all messages
var bytesBuf *bytes.Buffer
lastReadTime = startReadTime
var batchIndex int64
lastReadPosition = startPosition
var entryCounter int64
defer func() {
if bytesBuf != nil {
logBuffer.ReleaseMemory(bytesBuf)
}
println("LoopProcessLogData", readerName, "sent messages total", entryCounter)
}()
for {
@ -33,26 +48,42 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startReadTime
if bytesBuf != nil {
logBuffer.ReleaseMemory(bytesBuf)
}
bytesBuf, err = logBuffer.ReadFromBuffer(lastReadTime)
bytesBuf, batchIndex, err = logBuffer.ReadFromBuffer(lastReadPosition)
if err == ResumeFromDiskError {
time.Sleep(1127 * time.Millisecond)
return lastReadTime, isDone, ResumeFromDiskError
return lastReadPosition, isDone, ResumeFromDiskError
}
readSize := 0
if bytesBuf != nil {
readSize = bytesBuf.Len()
}
// glog.V(4).Infof("%s ReadFromBuffer by %v", readerName, lastReadTime)
glog.V(0).Infof("%s ReadFromBuffer at %v batch %d. Read bytes %v batch %d", readerName, lastReadPosition, lastReadPosition.BatchIndex, readSize, batchIndex)
if bytesBuf == nil {
if batchIndex >= 0 {
lastReadPosition = NewMessagePosition(lastReadPosition.UnixNano(), batchIndex)
}
if stopTsNs != 0 {
isDone = true
return
}
if waitForDataFn() {
continue
} else {
lastTsNs := logBuffer.LastTsNs
for lastTsNs == logBuffer.LastTsNs {
if waitForDataFn() {
continue
} else {
isDone = true
return
}
}
if logBuffer.IsStopping() {
isDone = true
return
}
continue
}
buf := bytesBuf.Bytes()
// fmt.Printf("ReadFromBuffer %s by %v size %d\n", readerName, lastReadTime, len(buf))
// fmt.Printf("ReadFromBuffer %s by %v size %d\n", readerName, lastReadPosition, len(buf))
batchSize := 0
@ -61,7 +92,7 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startReadTime
size := util.BytesToUint32(buf[pos : pos+4])
if pos+4+int(size) > len(buf) {
err = ResumeError
glog.Errorf("LoopProcessLogData: %s read buffer %v read %d [%d,%d) from [0,%d)", readerName, lastReadTime, batchSize, pos, pos+int(size)+4, len(buf))
glog.Errorf("LoopProcessLogData: %s read buffer %v read %d entries [%d,%d) from [0,%d)", readerName, lastReadPosition, batchSize, pos, pos+int(size)+4, len(buf))
return
}
entryData := buf[pos+4 : pos+4+int(size)]
@ -74,20 +105,27 @@ func (logBuffer *LogBuffer) LoopProcessLogData(readerName string, startReadTime
}
if stopTsNs != 0 && logEntry.TsNs > stopTsNs {
isDone = true
println("stopTsNs", stopTsNs, "logEntry.TsNs", logEntry.TsNs)
return
}
lastReadTime = time.Unix(0, logEntry.TsNs)
lastReadPosition = NewMessagePosition(logEntry.TsNs, batchIndex)
if err = eachLogDataFn(logEntry); err != nil {
if isDone, err = eachLogDataFn(logEntry); err != nil {
glog.Errorf("LoopProcessLogData: %s process log entry %d %v: %v", readerName, batchSize+1, logEntry, err)
return
}
if isDone {
glog.V(0).Infof("LoopProcessLogData2: %s process log entry %d", readerName, batchSize+1)
return
}
pos += 4 + int(size)
batchSize++
entryCounter++
}
// glog.V(4).Infof("%s sent messages ts[%+v,%+v] size %d\n", readerName, startReadTime, lastReadTime, batchSize)
glog.V(0).Infof("%s sent messages ts[%+v,%+v] size %d\n", readerName, startPosition, lastReadPosition, batchSize)
}
}

13
weed/util/log_buffer/sealed_buffer.go

@ -6,10 +6,11 @@ import (
)
type MemBuffer struct {
buf []byte
size int
startTime time.Time
stopTime time.Time
buf []byte
size int
startTime time.Time
stopTime time.Time
batchIndex int64
}
type SealedBuffers struct {
@ -29,7 +30,7 @@ func newSealedBuffers(size int) *SealedBuffers {
return sbs
}
func (sbs *SealedBuffers) SealBuffer(startTime, stopTime time.Time, buf []byte, pos int) (newBuf []byte) {
func (sbs *SealedBuffers) SealBuffer(startTime, stopTime time.Time, buf []byte, pos int, batchIndex int64) (newBuf []byte) {
oldMemBuffer := sbs.buffers[0]
size := len(sbs.buffers)
for i := 0; i < size-1; i++ {
@ -37,11 +38,13 @@ func (sbs *SealedBuffers) SealBuffer(startTime, stopTime time.Time, buf []byte,
sbs.buffers[i].size = sbs.buffers[i+1].size
sbs.buffers[i].startTime = sbs.buffers[i+1].startTime
sbs.buffers[i].stopTime = sbs.buffers[i+1].stopTime
sbs.buffers[i].batchIndex = sbs.buffers[i+1].batchIndex
}
sbs.buffers[size-1].buf = buf
sbs.buffers[size-1].size = pos
sbs.buffers[size-1].startTime = startTime
sbs.buffers[size-1].stopTime = stopTime
sbs.buffers[size-1].batchIndex = batchIndex
return oldMemBuffer.buf
}

Loading…
Cancel
Save