diff --git a/.gitignore b/.gitignore index 9a8e5d112..8fc65d330 100644 --- a/.gitignore +++ b/.gitignore @@ -91,4 +91,6 @@ weed/mq/client/cmd/weed_pub/weed_pub docker/weed # test generated files -weed/*/*.jpg \ No newline at end of file +weed/*/*.jpg +docker/weed_sub +docker/weed_pub diff --git a/docker/Dockerfile.local b/docker/Dockerfile.local index 53cfd9571..b87c0d8e9 100644 --- a/docker/Dockerfile.local +++ b/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 diff --git a/docker/Makefile b/docker/Makefile index 6d6c88190..f36ffed4a 100644 --- a/docker/Makefile +++ b/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 diff --git a/docker/compose/local-brokers-compose.yml b/docker/compose/local-brokers-compose.yml new file mode 100644 index 000000000..5f92d9bad --- /dev/null +++ b/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 diff --git a/other/java/client/src/main/proto/filer.proto b/other/java/client/src/main/proto/filer.proto index 0c3b8eddc..807430c5d 100644 --- a/other/java/client/src/main/proto/filer.proto +++ b/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; diff --git a/weed/cluster/lock_client.go b/weed/cluster/lock_client.go index c21f20874..6618f5d2f 100644 --- a/weed/cluster/lock_client.go +++ b/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 +} diff --git a/weed/cluster/lock_manager/distributed_lock_manager.go b/weed/cluster/lock_manager/distributed_lock_manager.go index 9479b38b0..472d60778 100644 --- a/weed/cluster/lock_manager/distributed_lock_manager.go +++ b/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) diff --git a/weed/cluster/lock_manager/lock_manager.go b/weed/cluster/lock_manager/lock_manager.go index 49b951dd9..ebc9dfeaa 100644 --- a/weed/cluster/lock_manager/lock_manager.go +++ b/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 } diff --git a/weed/filer/filechunk_manifest.go b/weed/filer/filechunk_manifest.go index d9d0331be..60a5c538b 100644 --- a/weed/filer/filechunk_manifest.go +++ b/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) { diff --git a/weed/filer/filer.go b/weed/filer/filer.go index 1c6b3c338..9be8d5259 100644 --- a/weed/filer/filer.go +++ b/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() } diff --git a/weed/filer/filer_notify.go b/weed/filer/filer_notify.go index f8a1dd603..db78b3d3d 100644 --- a/weed/filer/filer_notify.go +++ b/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 diff --git a/weed/filer/meta_aggregator.go b/weed/filer/meta_aggregator.go index e18e69216..663fdfe9f 100644 --- a/weed/filer/meta_aggregator.go +++ b/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) } diff --git a/weed/filer/reader_cache.go b/weed/filer/reader_cache.go index aabef6e1c..7be54b193 100644 --- a/weed/filer/reader_cache.go +++ b/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 diff --git a/weed/filer/stream.go b/weed/filer/stream.go index a402bc30c..2686fd833 100644 --- a/weed/filer/stream.go +++ b/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 } diff --git a/weed/mq/broker/broker_stats.go b/weed/mq/broker/broker_connect.go similarity index 52% rename from weed/mq/broker/broker_stats.go rename to weed/mq/broker/broker_connect.go index 3cd217519..3d58d89ea 100644 --- a/weed/mq/broker/broker_stats.go +++ b/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: + } + + } + }() + } + } + } } diff --git a/weed/mq/broker/broker_grpc_admin.go b/weed/mq/broker/broker_grpc_admin.go index 1313d09ec..3c9ef282c 100644 --- a/weed/mq/broker/broker_grpc_admin.go +++ b/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 diff --git a/weed/mq/broker/broker_grpc_assign.go b/weed/mq/broker/broker_grpc_assign.go new file mode 100644 index 000000000..264565b7b --- /dev/null +++ b/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 +} diff --git a/weed/mq/broker/broker_grpc_balance.go b/weed/mq/broker/broker_grpc_balance.go index c09161ff9..412407211 100644 --- a/weed/mq/broker/broker_grpc_balance.go +++ b/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 }) diff --git a/weed/mq/broker/broker_grpc_configure.go b/weed/mq/broker/broker_grpc_configure.go index 7f7c8f84b..6a6e92922 100644 --- a/weed/mq/broker/broker_grpc_configure.go +++ b/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 } diff --git a/weed/mq/broker/broker_grpc_lookup.go b/weed/mq/broker/broker_grpc_lookup.go index ac13a7581..14c1f37da 100644 --- a/weed/mq/broker/broker_grpc_lookup.go +++ b/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() +} diff --git a/weed/mq/broker/broker_grpc_pub.go b/weed/mq/broker/broker_grpc_pub.go index 43280e9be..3b68db1af 100644 --- a/weed/mq/broker/broker_grpc_pub.go +++ b/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 diff --git a/weed/mq/broker/broker_grpc_pub_balancer.go b/weed/mq/broker/broker_grpc_pub_balancer.go index e3d49f816..2cf90b4bc 100644 --- a/weed/mq/broker/broker_grpc_pub_balancer.go +++ b/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) } } diff --git a/weed/mq/broker/broker_grpc_pub_follow.go b/weed/mq/broker/broker_grpc_pub_follow.go new file mode 100644 index 000000000..8ef85110a --- /dev/null +++ b/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 + } + } + } + } +} diff --git a/weed/mq/broker/broker_grpc_sub.go b/weed/mq/broker/broker_grpc_sub.go index c98ce4684..1141ff47f 100644 --- a/weed/mq/broker/broker_grpc_sub.go +++ b/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 } diff --git a/weed/mq/broker/broker_grpc_sub_coordinator.go b/weed/mq/broker/broker_grpc_sub_coordinator.go index b8438b61f..89c221af5 100644 --- a/weed/mq/broker/broker_grpc_sub_coordinator.go +++ b/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) } } } diff --git a/weed/mq/broker/broker_server.go b/weed/mq/broker/broker_server.go index 52b34ddbc..9c321744b 100644 --- a/weed/mq/broker/broker_server.go +++ b/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 diff --git a/weed/mq/broker/broker_topic_conf_read_write.go b/weed/mq/broker/broker_topic_conf_read_write.go new file mode 100644 index 000000000..35d95c0e4 --- /dev/null +++ b/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 +} diff --git a/weed/mq/broker/broker_topic_partition_read_write.go b/weed/mq/broker/broker_topic_partition_read_write.go new file mode 100644 index 000000000..a058d8da5 --- /dev/null +++ b/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 + } +} diff --git a/weed/mq/broker/broker_write.go b/weed/mq/broker/broker_write.go new file mode 100644 index 000000000..896f0ee75 --- /dev/null +++ b/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 +} diff --git a/weed/mq/client/cmd/weed_pub/publisher.go b/weed/mq/client/cmd/weed_pub/publisher.go index ee00be9f8..2873ba21f 100644 --- a/weed/mq/client/cmd/weed_pub/publisher.go +++ b/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) { diff --git a/weed/mq/client/cmd/weed_sub/subscriber.go b/weed/mq/client/cmd/weed_sub/subscriber.go index d5bd8f12d..6d0e49be1 100644 --- a/weed/mq/client/cmd/weed_sub/subscriber.go +++ b/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 { diff --git a/weed/mq/client/pub_client/connect.go b/weed/mq/client/pub_client/connect.go deleted file mode 100644 index fc7ff4d77..000000000 --- a/weed/mq/client/pub_client/connect.go +++ /dev/null @@ -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 -} diff --git a/weed/mq/client/pub_client/lookup.go b/weed/mq/client/pub_client/lookup.go deleted file mode 100644 index e55bfd256..000000000 --- a/weed/mq/client/pub_client/lookup.go +++ /dev/null @@ -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 -} diff --git a/weed/mq/client/pub_client/publish.go b/weed/mq/client/pub_client/publish.go index 1e250ede3..1c5891049 100644 --- a/weed/mq/client/pub_client/publish.go +++ b/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, + }) } diff --git a/weed/mq/client/pub_client/publisher.go b/weed/mq/client/pub_client/publisher.go index a0c26db36..9262d6e0c 100644 --- a/weed/mq/client/pub_client/publisher.go +++ b/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 } diff --git a/weed/mq/client/pub_client/scheduler.go b/weed/mq/client/pub_client/scheduler.go new file mode 100644 index 000000000..e6caf896c --- /dev/null +++ b/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) + +} diff --git a/weed/mq/client/sub_client/connect_to_sub_coordinator.go b/weed/mq/client/sub_client/connect_to_sub_coordinator.go new file mode 100644 index 000000000..4cc3c8ff2 --- /dev/null +++ b/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 + }) +} diff --git a/weed/mq/client/sub_client/lookup.go b/weed/mq/client/sub_client/lookup.go deleted file mode 100644 index b6d2a8c53..000000000 --- a/weed/mq/client/sub_client/lookup.go +++ /dev/null @@ -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 -} diff --git a/weed/mq/client/sub_client/process.go b/weed/mq/client/sub_client/process.go deleted file mode 100644 index b6bdb14ee..000000000 --- a/weed/mq/client/sub_client/process.go +++ /dev/null @@ -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 -} diff --git a/weed/mq/client/sub_client/subscribe.go b/weed/mq/client/sub_client/subscribe.go index 370f5aa3c..df62ea674 100644 --- a/weed/mq/client/sub_client/subscribe.go +++ b/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 } diff --git a/weed/mq/client/sub_client/subscriber.go b/weed/mq/client/sub_client/subscriber.go index 9b96b14cb..982c3f13b 100644 --- a/weed/mq/client/sub_client/subscriber.go +++ b/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(), diff --git a/weed/mq/pub_balancer/allocate.go b/weed/mq/pub_balancer/allocate.go index 9b2113162..d7632f8d6 100644 --- a/weed/mq/pub_balancer/allocate.go +++ b/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 +} diff --git a/weed/mq/pub_balancer/allocate_test.go b/weed/mq/pub_balancer/allocate_test.go index a9056cdb9..89a6bb23c 100644 --- a/weed/mq/pub_balancer/allocate_test.go +++ b/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) + }) + } +} diff --git a/weed/mq/pub_balancer/balancer.go b/weed/mq/pub_balancer/balancer.go index 5e8c8275e..ad894b1d8 100644 --- a/weed/mq/pub_balancer/balancer.go +++ b/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) { +} diff --git a/weed/mq/pub_balancer/broker_stats.go b/weed/mq/pub_balancer/broker_stats.go index 461e93c61..b4bb28e42 100644 --- a/weed/mq/pub_balancer/broker_stats.go +++ b/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) + } } diff --git a/weed/mq/pub_balancer/lookup.go b/weed/mq/pub_balancer/lookup.go index 3e103a650..052932c04 100644 --- a/weed/mq/pub_balancer/lookup.go +++ b/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 } diff --git a/weed/mq/pub_balancer/partition_list_broker.go b/weed/mq/pub_balancer/partition_list_broker.go index 9dc6140b3..441b61898 100644 --- a/weed/mq/pub_balancer/partition_list_broker.go +++ b/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 } } } diff --git a/weed/mq/sub_coordinator/consumer_group.go b/weed/mq/sub_coordinator/consumer_group.go index 566a26ef7..f897fe2b3 100644 --- a/weed/mq/sub_coordinator/consumer_group.go +++ b/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 + } } diff --git a/weed/mq/sub_coordinator/coordinator.go b/weed/mq/sub_coordinator/coordinator.go index 7ca536c6b..bb50991ab 100644 --- a/weed/mq/sub_coordinator/coordinator.go +++ b/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) { + +} diff --git a/weed/mq/sub_coordinator/partition_consumer_mapping.go b/weed/mq/sub_coordinator/partition_consumer_mapping.go index 9c80b09c6..c7f104af1 100644 --- a/weed/mq/sub_coordinator/partition_consumer_mapping.go +++ b/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 { diff --git a/weed/mq/sub_coordinator/partition_consumer_mapping_test.go b/weed/mq/sub_coordinator/partition_consumer_mapping_test.go index 1d3050ef4..9a9abe011 100644 --- a/weed/mq/sub_coordinator/partition_consumer_mapping_test.go +++ b/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, diff --git a/weed/mq/sub_coordinator/partition_list.go b/weed/mq/sub_coordinator/partition_list.go index ca097f2b3..fa0e3761f 100644 --- a/weed/mq/sub_coordinator/partition_list.go +++ b/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 } diff --git a/weed/mq/topic/local_manager.go b/weed/mq/topic/local_manager.go index 173df090d..aa2eefcdc 100644 --- a/weed/mq/topic/local_manager.go +++ b/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) diff --git a/weed/mq/topic/local_partition.go b/weed/mq/topic/local_partition.go index aa1274ff5..798949736 100644 --- a/weed/mq/topic/local_partition.go +++ b/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) +} diff --git a/weed/mq/topic/local_partition_publishers.go b/weed/mq/topic/local_partition_publishers.go index 367ccce5f..c12f66336 100644 --- a/weed/mq/topic/local_partition_publishers.go +++ b/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) +} diff --git a/weed/mq/topic/local_partition_subscribers.go b/weed/mq/topic/local_partition_subscribers.go index e177ec7e8..d3b989d72 100644 --- a/weed/mq/topic/local_partition_subscribers.go +++ b/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) +} diff --git a/weed/mq/topic/local_topic.go b/weed/mq/topic/local_topic.go index 7825d2168..8ab2a0db5 100644 --- a/weed/mq/topic/local_topic.go +++ b/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 } } diff --git a/weed/mq/topic/partition.go b/weed/mq/topic/partition.go index ca34c2390..45b55c43b 100644 --- a/weed/mq/topic/partition.go +++ b/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 diff --git a/weed/pb/filer.proto b/weed/pb/filer.proto index 0c3b8eddc..807430c5d 100644 --- a/weed/pb/filer.proto +++ b/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; diff --git a/weed/pb/filer_pb/filer.pb.go b/weed/pb/filer_pb/filer.pb.go index bacdcac5b..8a1bad2d7 100644 --- a/weed/pb/filer_pb/filer.pb.go +++ b/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 ( diff --git a/weed/pb/filer_pb/filer_grpc.pb.go b/weed/pb/filer_pb/filer_grpc.pb.go index ebe714eab..ae1564f43 100644 --- a/weed/pb/filer_pb/filer_grpc.pb.go +++ b/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 diff --git a/weed/pb/iam_pb/iam.pb.go b/weed/pb/iam_pb/iam.pb.go index e0a55394e..8918daa39 100644 --- a/weed/pb/iam_pb/iam.pb.go +++ b/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 diff --git a/weed/pb/iam_pb/iam_grpc.pb.go b/weed/pb/iam_pb/iam_grpc.pb.go index 6c63a3174..3c2a10a90 100644 --- a/weed/pb/iam_pb/iam_grpc.pb.go +++ b/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 diff --git a/weed/pb/master_pb/master.pb.go b/weed/pb/master_pb/master.pb.go index 874b20be8..c0cee36b2 100644 --- a/weed/pb/master_pb/master.pb.go +++ b/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 diff --git a/weed/pb/master_pb/master_grpc.pb.go b/weed/pb/master_pb/master_grpc.pb.go index 5406444fe..4c41658cf 100644 --- a/weed/pb/master_pb/master_grpc.pb.go +++ b/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 diff --git a/weed/pb/mount_pb/mount.pb.go b/weed/pb/mount_pb/mount.pb.go index aaea1369c..b64592fb1 100644 --- a/weed/pb/mount_pb/mount.pb.go +++ b/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 diff --git a/weed/pb/mount_pb/mount_grpc.pb.go b/weed/pb/mount_pb/mount_grpc.pb.go index 0212220a5..3dd6d126b 100644 --- a/weed/pb/mount_pb/mount_grpc.pb.go +++ b/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 diff --git a/weed/pb/mq.proto b/weed/pb/mq.proto index 5c309aaf1..369f82fb3 100644 --- a/weed/pb/mq.proto +++ b/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; diff --git a/weed/pb/mq_pb/mq.pb.go b/weed/pb/mq_pb/mq.pb.go index e1879d98a..ed9f5c002 100644 --- a/weed/pb/mq_pb/mq.pb.go +++ b/weed/pb/mq_pb/mq.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: mq.proto package mq_pb @@ -20,6 +20,55 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) +type PartitionOffsetStartType int32 + +const ( + PartitionOffsetStartType_EARLIEST PartitionOffsetStartType = 0 + PartitionOffsetStartType_EARLIEST_IN_MEMORY PartitionOffsetStartType = 1 + PartitionOffsetStartType_LATEST PartitionOffsetStartType = 2 +) + +// Enum value maps for PartitionOffsetStartType. +var ( + PartitionOffsetStartType_name = map[int32]string{ + 0: "EARLIEST", + 1: "EARLIEST_IN_MEMORY", + 2: "LATEST", + } + PartitionOffsetStartType_value = map[string]int32{ + "EARLIEST": 0, + "EARLIEST_IN_MEMORY": 1, + "LATEST": 2, + } +) + +func (x PartitionOffsetStartType) Enum() *PartitionOffsetStartType { + p := new(PartitionOffsetStartType) + *p = x + return p +} + +func (x PartitionOffsetStartType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (PartitionOffsetStartType) Descriptor() protoreflect.EnumDescriptor { + return file_mq_proto_enumTypes[0].Descriptor() +} + +func (PartitionOffsetStartType) Type() protoreflect.EnumType { + return &file_mq_proto_enumTypes[0] +} + +func (x PartitionOffsetStartType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use PartitionOffsetStartType.Descriptor instead. +func (PartitionOffsetStartType) EnumDescriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{0} +} + type FindBrokerLeaderRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -240,6 +289,132 @@ func (x *Partition) GetUnixTimeNs() int64 { return 0 } +type Offset struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + PartitionOffsets []*PartitionOffset `protobuf:"bytes,2,rep,name=partition_offsets,json=partitionOffsets,proto3" json:"partition_offsets,omitempty"` +} + +func (x *Offset) Reset() { + *x = Offset{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Offset) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Offset) ProtoMessage() {} + +func (x *Offset) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[4] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Offset.ProtoReflect.Descriptor instead. +func (*Offset) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{4} +} + +func (x *Offset) GetTopic() *Topic { + if x != nil { + return x.Topic + } + return nil +} + +func (x *Offset) GetPartitionOffsets() []*PartitionOffset { + if x != nil { + return x.PartitionOffsets + } + return nil +} + +type PartitionOffset struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` + StartTsNs int64 `protobuf:"varint,2,opt,name=start_ts_ns,json=startTsNs,proto3" json:"start_ts_ns,omitempty"` + StopTsNs int64 `protobuf:"varint,3,opt,name=stop_ts_ns,json=stopTsNs,proto3" json:"stop_ts_ns,omitempty"` + StartType PartitionOffsetStartType `protobuf:"varint,4,opt,name=start_type,json=startType,proto3,enum=messaging_pb.PartitionOffsetStartType" json:"start_type,omitempty"` +} + +func (x *PartitionOffset) Reset() { + *x = PartitionOffset{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PartitionOffset) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PartitionOffset) ProtoMessage() {} + +func (x *PartitionOffset) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[5] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PartitionOffset.ProtoReflect.Descriptor instead. +func (*PartitionOffset) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{5} +} + +func (x *PartitionOffset) GetPartition() *Partition { + if x != nil { + return x.Partition + } + return nil +} + +func (x *PartitionOffset) GetStartTsNs() int64 { + if x != nil { + return x.StartTsNs + } + return 0 +} + +func (x *PartitionOffset) GetStopTsNs() int64 { + if x != nil { + return x.StopTsNs + } + return 0 +} + +func (x *PartitionOffset) GetStartType() PartitionOffsetStartType { + if x != nil { + return x.StartType + } + return PartitionOffsetStartType_EARLIEST +} + // //////////////////////////////////////////////// type BrokerStats struct { state protoimpl.MessageState @@ -253,7 +428,7 @@ type BrokerStats struct { func (x *BrokerStats) Reset() { *x = BrokerStats{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[4] + mi := &file_mq_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -266,7 +441,7 @@ func (x *BrokerStats) String() string { func (*BrokerStats) ProtoMessage() {} func (x *BrokerStats) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[4] + mi := &file_mq_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -279,7 +454,7 @@ func (x *BrokerStats) ProtoReflect() protoreflect.Message { // Deprecated: Use BrokerStats.ProtoReflect.Descriptor instead. func (*BrokerStats) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{4} + return file_mq_proto_rawDescGZIP(), []int{6} } func (x *BrokerStats) GetCpuUsagePercent() int32 { @@ -310,7 +485,7 @@ type TopicPartitionStats struct { func (x *TopicPartitionStats) Reset() { *x = TopicPartitionStats{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[5] + mi := &file_mq_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -323,7 +498,7 @@ func (x *TopicPartitionStats) String() string { func (*TopicPartitionStats) ProtoMessage() {} func (x *TopicPartitionStats) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[5] + mi := &file_mq_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -336,7 +511,7 @@ func (x *TopicPartitionStats) ProtoReflect() protoreflect.Message { // Deprecated: Use TopicPartitionStats.ProtoReflect.Descriptor instead. func (*TopicPartitionStats) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{5} + return file_mq_proto_rawDescGZIP(), []int{7} } func (x *TopicPartitionStats) GetTopic() *Topic { @@ -382,7 +557,7 @@ type PublisherToPubBalancerRequest struct { func (x *PublisherToPubBalancerRequest) Reset() { *x = PublisherToPubBalancerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[6] + mi := &file_mq_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -395,7 +570,7 @@ func (x *PublisherToPubBalancerRequest) String() string { func (*PublisherToPubBalancerRequest) ProtoMessage() {} func (x *PublisherToPubBalancerRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[6] + mi := &file_mq_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -408,7 +583,7 @@ func (x *PublisherToPubBalancerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PublisherToPubBalancerRequest.ProtoReflect.Descriptor instead. func (*PublisherToPubBalancerRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{6} + return file_mq_proto_rawDescGZIP(), []int{8} } func (m *PublisherToPubBalancerRequest) GetMessage() isPublisherToPubBalancerRequest_Message { @@ -457,7 +632,7 @@ type PublisherToPubBalancerResponse struct { func (x *PublisherToPubBalancerResponse) Reset() { *x = PublisherToPubBalancerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[7] + mi := &file_mq_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -470,7 +645,7 @@ func (x *PublisherToPubBalancerResponse) String() string { func (*PublisherToPubBalancerResponse) ProtoMessage() {} func (x *PublisherToPubBalancerResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[7] + mi := &file_mq_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -483,7 +658,7 @@ func (x *PublisherToPubBalancerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PublisherToPubBalancerResponse.ProtoReflect.Descriptor instead. func (*PublisherToPubBalancerResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{7} + return file_mq_proto_rawDescGZIP(), []int{9} } type BalanceTopicsRequest struct { @@ -495,7 +670,7 @@ type BalanceTopicsRequest struct { func (x *BalanceTopicsRequest) Reset() { *x = BalanceTopicsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[8] + mi := &file_mq_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -508,7 +683,7 @@ func (x *BalanceTopicsRequest) String() string { func (*BalanceTopicsRequest) ProtoMessage() {} func (x *BalanceTopicsRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[8] + mi := &file_mq_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -521,7 +696,7 @@ func (x *BalanceTopicsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use BalanceTopicsRequest.ProtoReflect.Descriptor instead. func (*BalanceTopicsRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{8} + return file_mq_proto_rawDescGZIP(), []int{10} } type BalanceTopicsResponse struct { @@ -533,7 +708,7 @@ type BalanceTopicsResponse struct { func (x *BalanceTopicsResponse) Reset() { *x = BalanceTopicsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[9] + mi := &file_mq_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -546,7 +721,7 @@ func (x *BalanceTopicsResponse) String() string { func (*BalanceTopicsResponse) ProtoMessage() {} func (x *BalanceTopicsResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[9] + mi := &file_mq_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -559,7 +734,7 @@ func (x *BalanceTopicsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use BalanceTopicsResponse.ProtoReflect.Descriptor instead. func (*BalanceTopicsResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{9} + return file_mq_proto_rawDescGZIP(), []int{11} } // //////////////////////////////////////////////// @@ -575,7 +750,7 @@ type ConfigureTopicRequest struct { func (x *ConfigureTopicRequest) Reset() { *x = ConfigureTopicRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[10] + mi := &file_mq_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -588,7 +763,7 @@ func (x *ConfigureTopicRequest) String() string { func (*ConfigureTopicRequest) ProtoMessage() {} func (x *ConfigureTopicRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[10] + mi := &file_mq_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -601,7 +776,7 @@ func (x *ConfigureTopicRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ConfigureTopicRequest.ProtoReflect.Descriptor instead. func (*ConfigureTopicRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{10} + return file_mq_proto_rawDescGZIP(), []int{12} } func (x *ConfigureTopicRequest) GetTopic() *Topic { @@ -629,7 +804,7 @@ type ConfigureTopicResponse struct { func (x *ConfigureTopicResponse) Reset() { *x = ConfigureTopicResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[11] + mi := &file_mq_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -642,7 +817,7 @@ func (x *ConfigureTopicResponse) String() string { func (*ConfigureTopicResponse) ProtoMessage() {} func (x *ConfigureTopicResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[11] + mi := &file_mq_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -655,7 +830,7 @@ func (x *ConfigureTopicResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ConfigureTopicResponse.ProtoReflect.Descriptor instead. func (*ConfigureTopicResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{11} + return file_mq_proto_rawDescGZIP(), []int{13} } func (x *ConfigureTopicResponse) GetBrokerPartitionAssignments() []*BrokerPartitionAssignment { @@ -674,7 +849,7 @@ type ListTopicsRequest struct { func (x *ListTopicsRequest) Reset() { *x = ListTopicsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[12] + mi := &file_mq_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -687,7 +862,7 @@ func (x *ListTopicsRequest) String() string { func (*ListTopicsRequest) ProtoMessage() {} func (x *ListTopicsRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[12] + mi := &file_mq_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -700,7 +875,7 @@ func (x *ListTopicsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListTopicsRequest.ProtoReflect.Descriptor instead. func (*ListTopicsRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{12} + return file_mq_proto_rawDescGZIP(), []int{14} } type ListTopicsResponse struct { @@ -714,7 +889,7 @@ type ListTopicsResponse struct { func (x *ListTopicsResponse) Reset() { *x = ListTopicsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[13] + mi := &file_mq_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -727,7 +902,7 @@ func (x *ListTopicsResponse) String() string { func (*ListTopicsResponse) ProtoMessage() {} func (x *ListTopicsResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[13] + mi := &file_mq_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -740,7 +915,7 @@ func (x *ListTopicsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListTopicsResponse.ProtoReflect.Descriptor instead. func (*ListTopicsResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{13} + return file_mq_proto_rawDescGZIP(), []int{15} } func (x *ListTopicsResponse) GetTopics() []*Topic { @@ -755,14 +930,13 @@ type LookupTopicBrokersRequest struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` - IsForPublish bool `protobuf:"varint,2,opt,name=is_for_publish,json=isForPublish,proto3" json:"is_for_publish,omitempty"` + Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` } func (x *LookupTopicBrokersRequest) Reset() { *x = LookupTopicBrokersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[14] + mi := &file_mq_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -775,7 +949,7 @@ func (x *LookupTopicBrokersRequest) String() string { func (*LookupTopicBrokersRequest) ProtoMessage() {} func (x *LookupTopicBrokersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[14] + mi := &file_mq_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -788,7 +962,7 @@ func (x *LookupTopicBrokersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupTopicBrokersRequest.ProtoReflect.Descriptor instead. func (*LookupTopicBrokersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{14} + return file_mq_proto_rawDescGZIP(), []int{16} } func (x *LookupTopicBrokersRequest) GetTopic() *Topic { @@ -798,13 +972,6 @@ func (x *LookupTopicBrokersRequest) GetTopic() *Topic { return nil } -func (x *LookupTopicBrokersRequest) GetIsForPublish() bool { - if x != nil { - return x.IsForPublish - } - return false -} - type LookupTopicBrokersResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -817,7 +984,7 @@ type LookupTopicBrokersResponse struct { func (x *LookupTopicBrokersResponse) Reset() { *x = LookupTopicBrokersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[15] + mi := &file_mq_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -830,7 +997,7 @@ func (x *LookupTopicBrokersResponse) String() string { func (*LookupTopicBrokersResponse) ProtoMessage() {} func (x *LookupTopicBrokersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[15] + mi := &file_mq_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -843,7 +1010,7 @@ func (x *LookupTopicBrokersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use LookupTopicBrokersResponse.ProtoReflect.Descriptor instead. func (*LookupTopicBrokersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{15} + return file_mq_proto_rawDescGZIP(), []int{17} } func (x *LookupTopicBrokersResponse) GetTopic() *Topic { @@ -873,7 +1040,7 @@ type BrokerPartitionAssignment struct { func (x *BrokerPartitionAssignment) Reset() { *x = BrokerPartitionAssignment{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[16] + mi := &file_mq_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -886,7 +1053,7 @@ func (x *BrokerPartitionAssignment) String() string { func (*BrokerPartitionAssignment) ProtoMessage() {} func (x *BrokerPartitionAssignment) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[16] + mi := &file_mq_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -899,7 +1066,7 @@ func (x *BrokerPartitionAssignment) ProtoReflect() protoreflect.Message { // Deprecated: Use BrokerPartitionAssignment.ProtoReflect.Descriptor instead. func (*BrokerPartitionAssignment) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{16} + return file_mq_proto_rawDescGZIP(), []int{18} } func (x *BrokerPartitionAssignment) GetPartition() *Partition { @@ -937,7 +1104,7 @@ type AssignTopicPartitionsRequest struct { func (x *AssignTopicPartitionsRequest) Reset() { *x = AssignTopicPartitionsRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[17] + mi := &file_mq_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -950,7 +1117,7 @@ func (x *AssignTopicPartitionsRequest) String() string { func (*AssignTopicPartitionsRequest) ProtoMessage() {} func (x *AssignTopicPartitionsRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[17] + mi := &file_mq_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -963,7 +1130,7 @@ func (x *AssignTopicPartitionsRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignTopicPartitionsRequest.ProtoReflect.Descriptor instead. func (*AssignTopicPartitionsRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{17} + return file_mq_proto_rawDescGZIP(), []int{19} } func (x *AssignTopicPartitionsRequest) GetTopic() *Topic { @@ -1003,7 +1170,7 @@ type AssignTopicPartitionsResponse struct { func (x *AssignTopicPartitionsResponse) Reset() { *x = AssignTopicPartitionsResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[18] + mi := &file_mq_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1016,7 +1183,7 @@ func (x *AssignTopicPartitionsResponse) String() string { func (*AssignTopicPartitionsResponse) ProtoMessage() {} func (x *AssignTopicPartitionsResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[18] + mi := &file_mq_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1029,7 +1196,7 @@ func (x *AssignTopicPartitionsResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use AssignTopicPartitionsResponse.ProtoReflect.Descriptor instead. func (*AssignTopicPartitionsResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{18} + return file_mq_proto_rawDescGZIP(), []int{20} } type SubscriberToSubCoordinatorRequest struct { @@ -1047,7 +1214,7 @@ type SubscriberToSubCoordinatorRequest struct { func (x *SubscriberToSubCoordinatorRequest) Reset() { *x = SubscriberToSubCoordinatorRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[19] + mi := &file_mq_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1060,7 +1227,7 @@ func (x *SubscriberToSubCoordinatorRequest) String() string { func (*SubscriberToSubCoordinatorRequest) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[19] + mi := &file_mq_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1073,7 +1240,7 @@ func (x *SubscriberToSubCoordinatorRequest) ProtoReflect() protoreflect.Message // Deprecated: Use SubscriberToSubCoordinatorRequest.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{19} + return file_mq_proto_rawDescGZIP(), []int{21} } func (m *SubscriberToSubCoordinatorRequest) GetMessage() isSubscriberToSubCoordinatorRequest_Message { @@ -1127,7 +1294,7 @@ type SubscriberToSubCoordinatorResponse struct { func (x *SubscriberToSubCoordinatorResponse) Reset() { *x = SubscriberToSubCoordinatorResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[20] + mi := &file_mq_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1140,7 +1307,7 @@ func (x *SubscriberToSubCoordinatorResponse) String() string { func (*SubscriberToSubCoordinatorResponse) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[20] + mi := &file_mq_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1153,7 +1320,7 @@ func (x *SubscriberToSubCoordinatorResponse) ProtoReflect() protoreflect.Message // Deprecated: Use SubscriberToSubCoordinatorResponse.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{20} + return file_mq_proto_rawDescGZIP(), []int{22} } func (m *SubscriberToSubCoordinatorResponse) GetMessage() isSubscriberToSubCoordinatorResponse_Message { @@ -1195,7 +1362,7 @@ type DataMessage struct { func (x *DataMessage) Reset() { *x = DataMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[21] + mi := &file_mq_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1208,7 +1375,7 @@ func (x *DataMessage) String() string { func (*DataMessage) ProtoMessage() {} func (x *DataMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[21] + mi := &file_mq_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1221,7 +1388,7 @@ func (x *DataMessage) ProtoReflect() protoreflect.Message { // Deprecated: Use DataMessage.ProtoReflect.Descriptor instead. func (*DataMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{21} + return file_mq_proto_rawDescGZIP(), []int{23} } func (x *DataMessage) GetKey() []byte { @@ -1245,36 +1412,36 @@ func (x *DataMessage) GetTsNs() int64 { return 0 } -type PublishRequest struct { +type PublishMessageRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields // Types that are assignable to Message: // - // *PublishRequest_Init - // *PublishRequest_Data - Message isPublishRequest_Message `protobuf_oneof:"message"` - Sequence int64 `protobuf:"varint,3,opt,name=sequence,proto3" json:"sequence,omitempty"` + // *PublishMessageRequest_Init + // *PublishMessageRequest_Data + Message isPublishMessageRequest_Message `protobuf_oneof:"message"` + Sequence int64 `protobuf:"varint,3,opt,name=sequence,proto3" json:"sequence,omitempty"` } -func (x *PublishRequest) Reset() { - *x = PublishRequest{} +func (x *PublishMessageRequest) Reset() { + *x = PublishMessageRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[22] + mi := &file_mq_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *PublishRequest) String() string { +func (x *PublishMessageRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*PublishRequest) ProtoMessage() {} +func (*PublishMessageRequest) ProtoMessage() {} -func (x *PublishRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[22] +func (x *PublishMessageRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1285,56 +1452,56 @@ func (x *PublishRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PublishRequest.ProtoReflect.Descriptor instead. -func (*PublishRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{22} +// Deprecated: Use PublishMessageRequest.ProtoReflect.Descriptor instead. +func (*PublishMessageRequest) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{24} } -func (m *PublishRequest) GetMessage() isPublishRequest_Message { +func (m *PublishMessageRequest) GetMessage() isPublishMessageRequest_Message { if m != nil { return m.Message } return nil } -func (x *PublishRequest) GetInit() *PublishRequest_InitMessage { - if x, ok := x.GetMessage().(*PublishRequest_Init); ok { +func (x *PublishMessageRequest) GetInit() *PublishMessageRequest_InitMessage { + if x, ok := x.GetMessage().(*PublishMessageRequest_Init); ok { return x.Init } return nil } -func (x *PublishRequest) GetData() *DataMessage { - if x, ok := x.GetMessage().(*PublishRequest_Data); ok { +func (x *PublishMessageRequest) GetData() *DataMessage { + if x, ok := x.GetMessage().(*PublishMessageRequest_Data); ok { return x.Data } return nil } -func (x *PublishRequest) GetSequence() int64 { +func (x *PublishMessageRequest) GetSequence() int64 { if x != nil { return x.Sequence } return 0 } -type isPublishRequest_Message interface { - isPublishRequest_Message() +type isPublishMessageRequest_Message interface { + isPublishMessageRequest_Message() } -type PublishRequest_Init struct { - Init *PublishRequest_InitMessage `protobuf:"bytes,1,opt,name=init,proto3,oneof"` +type PublishMessageRequest_Init struct { + Init *PublishMessageRequest_InitMessage `protobuf:"bytes,1,opt,name=init,proto3,oneof"` } -type PublishRequest_Data struct { +type PublishMessageRequest_Data struct { Data *DataMessage `protobuf:"bytes,2,opt,name=data,proto3,oneof"` } -func (*PublishRequest_Init) isPublishRequest_Message() {} +func (*PublishMessageRequest_Init) isPublishMessageRequest_Message() {} -func (*PublishRequest_Data) isPublishRequest_Message() {} +func (*PublishMessageRequest_Data) isPublishMessageRequest_Message() {} -type PublishResponse struct { +type PublishMessageResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -1344,23 +1511,23 @@ type PublishResponse struct { ShouldClose bool `protobuf:"varint,3,opt,name=should_close,json=shouldClose,proto3" json:"should_close,omitempty"` } -func (x *PublishResponse) Reset() { - *x = PublishResponse{} +func (x *PublishMessageResponse) Reset() { + *x = PublishMessageResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[23] + mi := &file_mq_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *PublishResponse) String() string { +func (x *PublishMessageResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*PublishResponse) ProtoMessage() {} +func (*PublishMessageResponse) ProtoMessage() {} -func (x *PublishResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[23] +func (x *PublishMessageResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1371,61 +1538,59 @@ func (x *PublishResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PublishResponse.ProtoReflect.Descriptor instead. -func (*PublishResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{23} +// Deprecated: Use PublishMessageResponse.ProtoReflect.Descriptor instead. +func (*PublishMessageResponse) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{25} } -func (x *PublishResponse) GetAckSequence() int64 { +func (x *PublishMessageResponse) GetAckSequence() int64 { if x != nil { return x.AckSequence } return 0 } -func (x *PublishResponse) GetError() string { +func (x *PublishMessageResponse) GetError() string { if x != nil { return x.Error } return "" } -func (x *PublishResponse) GetShouldClose() bool { +func (x *PublishMessageResponse) GetShouldClose() bool { if x != nil { return x.ShouldClose } return false } -type SubscribeRequest struct { +type PublishFollowMeRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - // Types that are assignable to Message: - // - // *SubscribeRequest_Init - // *SubscribeRequest_Ack - Message isSubscribeRequest_Message `protobuf_oneof:"message"` + Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Partition *Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"` + BrokerSelf string `protobuf:"bytes,3,opt,name=broker_self,json=brokerSelf,proto3" json:"broker_self,omitempty"` } -func (x *SubscribeRequest) Reset() { - *x = SubscribeRequest{} +func (x *PublishFollowMeRequest) Reset() { + *x = PublishFollowMeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[24] + mi := &file_mq_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *SubscribeRequest) String() string { +func (x *PublishFollowMeRequest) String() string { return protoimpl.X.MessageStringOf(x) } -func (*SubscribeRequest) ProtoMessage() {} +func (*PublishFollowMeRequest) ProtoMessage() {} -func (x *SubscribeRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[24] +func (x *PublishFollowMeRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1436,77 +1601,189 @@ func (x *SubscribeRequest) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SubscribeRequest.ProtoReflect.Descriptor instead. -func (*SubscribeRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{24} +// Deprecated: Use PublishFollowMeRequest.ProtoReflect.Descriptor instead. +func (*PublishFollowMeRequest) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{26} } -func (m *SubscribeRequest) GetMessage() isSubscribeRequest_Message { - if m != nil { - return m.Message +func (x *PublishFollowMeRequest) GetTopic() *Topic { + if x != nil { + return x.Topic } return nil } -func (x *SubscribeRequest) GetInit() *SubscribeRequest_InitMessage { - if x, ok := x.GetMessage().(*SubscribeRequest_Init); ok { - return x.Init +func (x *PublishFollowMeRequest) GetPartition() *Partition { + if x != nil { + return x.Partition } return nil } -func (x *SubscribeRequest) GetAck() *SubscribeRequest_AckMessage { - if x, ok := x.GetMessage().(*SubscribeRequest_Ack); ok { - return x.Ack +func (x *PublishFollowMeRequest) GetBrokerSelf() string { + if x != nil { + return x.BrokerSelf + } + return "" +} + +type PublishFollowMeResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` +} + +func (x *PublishFollowMeResponse) Reset() { + *x = PublishFollowMeResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PublishFollowMeResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PublishFollowMeResponse) ProtoMessage() {} + +func (x *PublishFollowMeResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[27] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PublishFollowMeResponse.ProtoReflect.Descriptor instead. +func (*PublishFollowMeResponse) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{27} +} + +func (x *PublishFollowMeResponse) GetError() string { + if x != nil { + return x.Error + } + return "" +} + +type SubscribeMessageRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Message: + // + // *SubscribeMessageRequest_Init + // *SubscribeMessageRequest_Ack + Message isSubscribeMessageRequest_Message `protobuf_oneof:"message"` +} + +func (x *SubscribeMessageRequest) Reset() { + *x = SubscribeMessageRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SubscribeMessageRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubscribeMessageRequest) ProtoMessage() {} + +func (x *SubscribeMessageRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[28] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SubscribeMessageRequest.ProtoReflect.Descriptor instead. +func (*SubscribeMessageRequest) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{28} +} + +func (m *SubscribeMessageRequest) GetMessage() isSubscribeMessageRequest_Message { + if m != nil { + return m.Message } return nil } -type isSubscribeRequest_Message interface { - isSubscribeRequest_Message() +func (x *SubscribeMessageRequest) GetInit() *SubscribeMessageRequest_InitMessage { + if x, ok := x.GetMessage().(*SubscribeMessageRequest_Init); ok { + return x.Init + } + return nil } -type SubscribeRequest_Init struct { - Init *SubscribeRequest_InitMessage `protobuf:"bytes,1,opt,name=init,proto3,oneof"` +func (x *SubscribeMessageRequest) GetAck() *SubscribeMessageRequest_AckMessage { + if x, ok := x.GetMessage().(*SubscribeMessageRequest_Ack); ok { + return x.Ack + } + return nil } -type SubscribeRequest_Ack struct { - Ack *SubscribeRequest_AckMessage `protobuf:"bytes,2,opt,name=ack,proto3,oneof"` +type isSubscribeMessageRequest_Message interface { + isSubscribeMessageRequest_Message() } -func (*SubscribeRequest_Init) isSubscribeRequest_Message() {} +type SubscribeMessageRequest_Init struct { + Init *SubscribeMessageRequest_InitMessage `protobuf:"bytes,1,opt,name=init,proto3,oneof"` +} + +type SubscribeMessageRequest_Ack struct { + Ack *SubscribeMessageRequest_AckMessage `protobuf:"bytes,2,opt,name=ack,proto3,oneof"` +} -func (*SubscribeRequest_Ack) isSubscribeRequest_Message() {} +func (*SubscribeMessageRequest_Init) isSubscribeMessageRequest_Message() {} -type SubscribeResponse struct { +func (*SubscribeMessageRequest_Ack) isSubscribeMessageRequest_Message() {} + +type SubscribeMessageResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields // Types that are assignable to Message: // - // *SubscribeResponse_Ctrl - // *SubscribeResponse_Data - Message isSubscribeResponse_Message `protobuf_oneof:"message"` + // *SubscribeMessageResponse_Ctrl + // *SubscribeMessageResponse_Data + Message isSubscribeMessageResponse_Message `protobuf_oneof:"message"` } -func (x *SubscribeResponse) Reset() { - *x = SubscribeResponse{} +func (x *SubscribeMessageResponse) Reset() { + *x = SubscribeMessageResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[25] + mi := &file_mq_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *SubscribeResponse) String() string { +func (x *SubscribeMessageResponse) String() string { return protoimpl.X.MessageStringOf(x) } -func (*SubscribeResponse) ProtoMessage() {} +func (*SubscribeMessageResponse) ProtoMessage() {} -func (x *SubscribeResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[25] +func (x *SubscribeMessageResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1517,47 +1794,209 @@ func (x *SubscribeResponse) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SubscribeResponse.ProtoReflect.Descriptor instead. -func (*SubscribeResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{25} +// Deprecated: Use SubscribeMessageResponse.ProtoReflect.Descriptor instead. +func (*SubscribeMessageResponse) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{29} } -func (m *SubscribeResponse) GetMessage() isSubscribeResponse_Message { +func (m *SubscribeMessageResponse) GetMessage() isSubscribeMessageResponse_Message { if m != nil { return m.Message } return nil } -func (x *SubscribeResponse) GetCtrl() *SubscribeResponse_CtrlMessage { - if x, ok := x.GetMessage().(*SubscribeResponse_Ctrl); ok { +func (x *SubscribeMessageResponse) GetCtrl() *SubscribeMessageResponse_CtrlMessage { + if x, ok := x.GetMessage().(*SubscribeMessageResponse_Ctrl); ok { return x.Ctrl } return nil } -func (x *SubscribeResponse) GetData() *DataMessage { - if x, ok := x.GetMessage().(*SubscribeResponse_Data); ok { +func (x *SubscribeMessageResponse) GetData() *DataMessage { + if x, ok := x.GetMessage().(*SubscribeMessageResponse_Data); ok { return x.Data } return nil } -type isSubscribeResponse_Message interface { - isSubscribeResponse_Message() +type isSubscribeMessageResponse_Message interface { + isSubscribeMessageResponse_Message() } -type SubscribeResponse_Ctrl struct { - Ctrl *SubscribeResponse_CtrlMessage `protobuf:"bytes,1,opt,name=ctrl,proto3,oneof"` +type SubscribeMessageResponse_Ctrl struct { + Ctrl *SubscribeMessageResponse_CtrlMessage `protobuf:"bytes,1,opt,name=ctrl,proto3,oneof"` } -type SubscribeResponse_Data struct { +type SubscribeMessageResponse_Data struct { Data *DataMessage `protobuf:"bytes,2,opt,name=data,proto3,oneof"` } -func (*SubscribeResponse_Ctrl) isSubscribeResponse_Message() {} +func (*SubscribeMessageResponse_Ctrl) isSubscribeMessageResponse_Message() {} + +func (*SubscribeMessageResponse_Data) isSubscribeMessageResponse_Message() {} + +type FollowInMemoryMessagesRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Message: + // + // *FollowInMemoryMessagesRequest_Init + // *FollowInMemoryMessagesRequest_Ack + Message isFollowInMemoryMessagesRequest_Message `protobuf_oneof:"message"` +} + +func (x *FollowInMemoryMessagesRequest) Reset() { + *x = FollowInMemoryMessagesRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[30] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FollowInMemoryMessagesRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FollowInMemoryMessagesRequest) ProtoMessage() {} + +func (x *FollowInMemoryMessagesRequest) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[30] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FollowInMemoryMessagesRequest.ProtoReflect.Descriptor instead. +func (*FollowInMemoryMessagesRequest) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{30} +} + +func (m *FollowInMemoryMessagesRequest) GetMessage() isFollowInMemoryMessagesRequest_Message { + if m != nil { + return m.Message + } + return nil +} + +func (x *FollowInMemoryMessagesRequest) GetInit() *FollowInMemoryMessagesRequest_InitMessage { + if x, ok := x.GetMessage().(*FollowInMemoryMessagesRequest_Init); ok { + return x.Init + } + return nil +} + +func (x *FollowInMemoryMessagesRequest) GetAck() *FollowInMemoryMessagesRequest_AckMessage { + if x, ok := x.GetMessage().(*FollowInMemoryMessagesRequest_Ack); ok { + return x.Ack + } + return nil +} + +type isFollowInMemoryMessagesRequest_Message interface { + isFollowInMemoryMessagesRequest_Message() +} + +type FollowInMemoryMessagesRequest_Init struct { + Init *FollowInMemoryMessagesRequest_InitMessage `protobuf:"bytes,1,opt,name=init,proto3,oneof"` +} + +type FollowInMemoryMessagesRequest_Ack struct { + Ack *FollowInMemoryMessagesRequest_AckMessage `protobuf:"bytes,2,opt,name=ack,proto3,oneof"` +} + +func (*FollowInMemoryMessagesRequest_Init) isFollowInMemoryMessagesRequest_Message() {} + +func (*FollowInMemoryMessagesRequest_Ack) isFollowInMemoryMessagesRequest_Message() {} + +type FollowInMemoryMessagesResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields -func (*SubscribeResponse_Data) isSubscribeResponse_Message() {} + // Types that are assignable to Message: + // + // *FollowInMemoryMessagesResponse_Ctrl + // *FollowInMemoryMessagesResponse_Data + Message isFollowInMemoryMessagesResponse_Message `protobuf_oneof:"message"` +} + +func (x *FollowInMemoryMessagesResponse) Reset() { + *x = FollowInMemoryMessagesResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[31] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *FollowInMemoryMessagesResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*FollowInMemoryMessagesResponse) ProtoMessage() {} + +func (x *FollowInMemoryMessagesResponse) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[31] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use FollowInMemoryMessagesResponse.ProtoReflect.Descriptor instead. +func (*FollowInMemoryMessagesResponse) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{31} +} + +func (m *FollowInMemoryMessagesResponse) GetMessage() isFollowInMemoryMessagesResponse_Message { + if m != nil { + return m.Message + } + return nil +} + +func (x *FollowInMemoryMessagesResponse) GetCtrl() *FollowInMemoryMessagesResponse_CtrlMessage { + if x, ok := x.GetMessage().(*FollowInMemoryMessagesResponse_Ctrl); ok { + return x.Ctrl + } + return nil +} + +func (x *FollowInMemoryMessagesResponse) GetData() *DataMessage { + if x, ok := x.GetMessage().(*FollowInMemoryMessagesResponse_Data); ok { + return x.Data + } + return nil +} + +type isFollowInMemoryMessagesResponse_Message interface { + isFollowInMemoryMessagesResponse_Message() +} + +type FollowInMemoryMessagesResponse_Ctrl struct { + Ctrl *FollowInMemoryMessagesResponse_CtrlMessage `protobuf:"bytes,1,opt,name=ctrl,proto3,oneof"` +} + +type FollowInMemoryMessagesResponse_Data struct { + Data *DataMessage `protobuf:"bytes,2,opt,name=data,proto3,oneof"` +} + +func (*FollowInMemoryMessagesResponse_Ctrl) isFollowInMemoryMessagesResponse_Message() {} + +func (*FollowInMemoryMessagesResponse_Data) isFollowInMemoryMessagesResponse_Message() {} type ClosePublishersRequest struct { state protoimpl.MessageState @@ -1571,7 +2010,7 @@ type ClosePublishersRequest struct { func (x *ClosePublishersRequest) Reset() { *x = ClosePublishersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[26] + mi := &file_mq_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1584,7 +2023,7 @@ func (x *ClosePublishersRequest) String() string { func (*ClosePublishersRequest) ProtoMessage() {} func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[26] + mi := &file_mq_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1597,7 +2036,7 @@ func (x *ClosePublishersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ClosePublishersRequest.ProtoReflect.Descriptor instead. func (*ClosePublishersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{26} + return file_mq_proto_rawDescGZIP(), []int{32} } func (x *ClosePublishersRequest) GetTopic() *Topic { @@ -1623,7 +2062,7 @@ type ClosePublishersResponse struct { func (x *ClosePublishersResponse) Reset() { *x = ClosePublishersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[27] + mi := &file_mq_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1636,7 +2075,7 @@ func (x *ClosePublishersResponse) String() string { func (*ClosePublishersResponse) ProtoMessage() {} func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[27] + mi := &file_mq_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1649,7 +2088,7 @@ func (x *ClosePublishersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ClosePublishersResponse.ProtoReflect.Descriptor instead. func (*ClosePublishersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{27} + return file_mq_proto_rawDescGZIP(), []int{33} } type CloseSubscribersRequest struct { @@ -1664,7 +2103,7 @@ type CloseSubscribersRequest struct { func (x *CloseSubscribersRequest) Reset() { *x = CloseSubscribersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[28] + mi := &file_mq_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1677,7 +2116,7 @@ func (x *CloseSubscribersRequest) String() string { func (*CloseSubscribersRequest) ProtoMessage() {} func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[28] + mi := &file_mq_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1690,7 +2129,7 @@ func (x *CloseSubscribersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseSubscribersRequest.ProtoReflect.Descriptor instead. func (*CloseSubscribersRequest) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{28} + return file_mq_proto_rawDescGZIP(), []int{34} } func (x *CloseSubscribersRequest) GetTopic() *Topic { @@ -1716,7 +2155,7 @@ type CloseSubscribersResponse struct { func (x *CloseSubscribersResponse) Reset() { *x = CloseSubscribersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[29] + mi := &file_mq_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1729,7 +2168,7 @@ func (x *CloseSubscribersResponse) String() string { func (*CloseSubscribersResponse) ProtoMessage() {} func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[29] + mi := &file_mq_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1742,7 +2181,7 @@ func (x *CloseSubscribersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CloseSubscribersResponse.ProtoReflect.Descriptor instead. func (*CloseSubscribersResponse) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{29} + return file_mq_proto_rawDescGZIP(), []int{35} } type PublisherToPubBalancerRequest_InitMessage struct { @@ -1756,7 +2195,7 @@ type PublisherToPubBalancerRequest_InitMessage struct { func (x *PublisherToPubBalancerRequest_InitMessage) Reset() { *x = PublisherToPubBalancerRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[31] + mi := &file_mq_proto_msgTypes[37] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1769,7 +2208,7 @@ func (x *PublisherToPubBalancerRequest_InitMessage) String() string { func (*PublisherToPubBalancerRequest_InitMessage) ProtoMessage() {} func (x *PublisherToPubBalancerRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[31] + mi := &file_mq_proto_msgTypes[37] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1782,7 +2221,7 @@ func (x *PublisherToPubBalancerRequest_InitMessage) ProtoReflect() protoreflect. // Deprecated: Use PublisherToPubBalancerRequest_InitMessage.ProtoReflect.Descriptor instead. func (*PublisherToPubBalancerRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{6, 0} + return file_mq_proto_rawDescGZIP(), []int{8, 0} } func (x *PublisherToPubBalancerRequest_InitMessage) GetBroker() string { @@ -1797,15 +2236,15 @@ type SubscriberToSubCoordinatorRequest_InitMessage struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - ConsumerGroup string `protobuf:"bytes,1,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"` - ConsumerInstanceId string `protobuf:"bytes,2,opt,name=consumer_instance_id,json=consumerInstanceId,proto3" json:"consumer_instance_id,omitempty"` - Topic *Topic `protobuf:"bytes,3,opt,name=topic,proto3" json:"topic,omitempty"` + ConsumerGroup string `protobuf:"bytes,1,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"` + ConsumerGroupInstanceId string `protobuf:"bytes,2,opt,name=consumer_group_instance_id,json=consumerGroupInstanceId,proto3" json:"consumer_group_instance_id,omitempty"` + Topic *Topic `protobuf:"bytes,3,opt,name=topic,proto3" json:"topic,omitempty"` } func (x *SubscriberToSubCoordinatorRequest_InitMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[32] + mi := &file_mq_proto_msgTypes[38] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1818,7 +2257,7 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) String() string { func (*SubscriberToSubCoordinatorRequest_InitMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[32] + mi := &file_mq_proto_msgTypes[38] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1831,7 +2270,7 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) ProtoReflect() protorefl // Deprecated: Use SubscriberToSubCoordinatorRequest_InitMessage.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{19, 0} + return file_mq_proto_rawDescGZIP(), []int{21, 0} } func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetConsumerGroup() string { @@ -1841,9 +2280,9 @@ func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetConsumerGroup() strin return "" } -func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetConsumerInstanceId() string { +func (x *SubscriberToSubCoordinatorRequest_InitMessage) GetConsumerGroupInstanceId() string { if x != nil { - return x.ConsumerInstanceId + return x.ConsumerGroupInstanceId } return "" } @@ -1867,7 +2306,7 @@ type SubscriberToSubCoordinatorRequest_AckMessage struct { func (x *SubscriberToSubCoordinatorRequest_AckMessage) Reset() { *x = SubscriberToSubCoordinatorRequest_AckMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[33] + mi := &file_mq_proto_msgTypes[39] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1880,7 +2319,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckMessage) String() string { func (*SubscriberToSubCoordinatorRequest_AckMessage) ProtoMessage() {} func (x *SubscriberToSubCoordinatorRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[33] + mi := &file_mq_proto_msgTypes[39] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1893,7 +2332,7 @@ func (x *SubscriberToSubCoordinatorRequest_AckMessage) ProtoReflect() protorefle // Deprecated: Use SubscriberToSubCoordinatorRequest_AckMessage.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorRequest_AckMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{19, 1} + return file_mq_proto_rawDescGZIP(), []int{21, 1} } func (x *SubscriberToSubCoordinatorRequest_AckMessage) GetPartition() *Partition { @@ -1917,12 +2356,13 @@ type SubscriberToSubCoordinatorResponse_AssignedPartition struct { Partition *Partition `protobuf:"bytes,1,opt,name=partition,proto3" json:"partition,omitempty"` TsNs int64 `protobuf:"varint,2,opt,name=ts_ns,json=tsNs,proto3" json:"ts_ns,omitempty"` + Broker string `protobuf:"bytes,3,opt,name=broker,proto3" json:"broker,omitempty"` } func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) Reset() { *x = SubscriberToSubCoordinatorResponse_AssignedPartition{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[34] + mi := &file_mq_proto_msgTypes[40] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1935,7 +2375,7 @@ func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) String() string { func (*SubscriberToSubCoordinatorResponse_AssignedPartition) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[34] + mi := &file_mq_proto_msgTypes[40] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1948,7 +2388,7 @@ func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) ProtoReflect() pr // Deprecated: Use SubscriberToSubCoordinatorResponse_AssignedPartition.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorResponse_AssignedPartition) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{20, 0} + return file_mq_proto_rawDescGZIP(), []int{22, 0} } func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) GetPartition() *Partition { @@ -1965,6 +2405,13 @@ func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) GetTsNs() int64 { return 0 } +func (x *SubscriberToSubCoordinatorResponse_AssignedPartition) GetBroker() string { + if x != nil { + return x.Broker + } + return "" +} + type SubscriberToSubCoordinatorResponse_Assignment struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1977,7 +2424,7 @@ type SubscriberToSubCoordinatorResponse_Assignment struct { func (x *SubscriberToSubCoordinatorResponse_Assignment) Reset() { *x = SubscriberToSubCoordinatorResponse_Assignment{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[35] + mi := &file_mq_proto_msgTypes[41] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1990,7 +2437,7 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) String() string { func (*SubscriberToSubCoordinatorResponse_Assignment) ProtoMessage() {} func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[35] + mi := &file_mq_proto_msgTypes[41] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2003,50 +2450,255 @@ func (x *SubscriberToSubCoordinatorResponse_Assignment) ProtoReflect() protorefl // Deprecated: Use SubscriberToSubCoordinatorResponse_Assignment.ProtoReflect.Descriptor instead. func (*SubscriberToSubCoordinatorResponse_Assignment) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{20, 1} + return file_mq_proto_rawDescGZIP(), []int{22, 1} +} + +func (x *SubscriberToSubCoordinatorResponse_Assignment) GetGeneration() int64 { + if x != nil { + return x.Generation + } + return 0 +} + +func (x *SubscriberToSubCoordinatorResponse_Assignment) GetAssignedPartitions() []*SubscriberToSubCoordinatorResponse_AssignedPartition { + if x != nil { + return x.AssignedPartitions + } + return nil +} + +type PublishMessageRequest_InitMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` + Partition *Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"` + AckInterval int32 `protobuf:"varint,3,opt,name=ack_interval,json=ackInterval,proto3" json:"ack_interval,omitempty"` + FollowerBrokers []string `protobuf:"bytes,4,rep,name=follower_brokers,json=followerBrokers,proto3" json:"follower_brokers,omitempty"` +} + +func (x *PublishMessageRequest_InitMessage) Reset() { + *x = PublishMessageRequest_InitMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PublishMessageRequest_InitMessage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PublishMessageRequest_InitMessage) ProtoMessage() {} + +func (x *PublishMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[42] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PublishMessageRequest_InitMessage.ProtoReflect.Descriptor instead. +func (*PublishMessageRequest_InitMessage) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{24, 0} +} + +func (x *PublishMessageRequest_InitMessage) GetTopic() *Topic { + if x != nil { + return x.Topic + } + return nil +} + +func (x *PublishMessageRequest_InitMessage) GetPartition() *Partition { + if x != nil { + return x.Partition + } + return nil +} + +func (x *PublishMessageRequest_InitMessage) GetAckInterval() int32 { + if x != nil { + return x.AckInterval + } + return 0 +} + +func (x *PublishMessageRequest_InitMessage) GetFollowerBrokers() []string { + if x != nil { + return x.FollowerBrokers + } + return nil +} + +type SubscribeMessageRequest_InitMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + ConsumerGroup string `protobuf:"bytes,1,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"` + ConsumerId string `protobuf:"bytes,2,opt,name=consumer_id,json=consumerId,proto3" json:"consumer_id,omitempty"` + ClientId string `protobuf:"bytes,3,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"` + Topic *Topic `protobuf:"bytes,4,opt,name=topic,proto3" json:"topic,omitempty"` + PartitionOffset *PartitionOffset `protobuf:"bytes,5,opt,name=partition_offset,json=partitionOffset,proto3" json:"partition_offset,omitempty"` + Filter string `protobuf:"bytes,6,opt,name=filter,proto3" json:"filter,omitempty"` +} + +func (x *SubscribeMessageRequest_InitMessage) Reset() { + *x = SubscribeMessageRequest_InitMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SubscribeMessageRequest_InitMessage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubscribeMessageRequest_InitMessage) ProtoMessage() {} + +func (x *SubscribeMessageRequest_InitMessage) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[43] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SubscribeMessageRequest_InitMessage.ProtoReflect.Descriptor instead. +func (*SubscribeMessageRequest_InitMessage) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{28, 0} +} + +func (x *SubscribeMessageRequest_InitMessage) GetConsumerGroup() string { + if x != nil { + return x.ConsumerGroup + } + return "" +} + +func (x *SubscribeMessageRequest_InitMessage) GetConsumerId() string { + if x != nil { + return x.ConsumerId + } + return "" +} + +func (x *SubscribeMessageRequest_InitMessage) GetClientId() string { + if x != nil { + return x.ClientId + } + return "" +} + +func (x *SubscribeMessageRequest_InitMessage) GetTopic() *Topic { + if x != nil { + return x.Topic + } + return nil +} + +func (x *SubscribeMessageRequest_InitMessage) GetPartitionOffset() *PartitionOffset { + if x != nil { + return x.PartitionOffset + } + return nil +} + +func (x *SubscribeMessageRequest_InitMessage) GetFilter() string { + if x != nil { + return x.Filter + } + return "" +} + +type SubscribeMessageRequest_AckMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Sequence int64 `protobuf:"varint,1,opt,name=sequence,proto3" json:"sequence,omitempty"` +} + +func (x *SubscribeMessageRequest_AckMessage) Reset() { + *x = SubscribeMessageRequest_AckMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_mq_proto_msgTypes[44] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SubscribeMessageRequest_AckMessage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SubscribeMessageRequest_AckMessage) ProtoMessage() {} + +func (x *SubscribeMessageRequest_AckMessage) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[44] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) } -func (x *SubscriberToSubCoordinatorResponse_Assignment) GetGeneration() int64 { - if x != nil { - return x.Generation - } - return 0 +// Deprecated: Use SubscribeMessageRequest_AckMessage.ProtoReflect.Descriptor instead. +func (*SubscribeMessageRequest_AckMessage) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{28, 1} } -func (x *SubscriberToSubCoordinatorResponse_Assignment) GetAssignedPartitions() []*SubscriberToSubCoordinatorResponse_AssignedPartition { +func (x *SubscribeMessageRequest_AckMessage) GetSequence() int64 { if x != nil { - return x.AssignedPartitions + return x.Sequence } - return nil + return 0 } -type PublishRequest_InitMessage struct { +type SubscribeMessageResponse_CtrlMessage struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Topic *Topic `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"` - Partition *Partition `protobuf:"bytes,2,opt,name=partition,proto3" json:"partition,omitempty"` - AckInterval int32 `protobuf:"varint,3,opt,name=ack_interval,json=ackInterval,proto3" json:"ack_interval,omitempty"` + Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` + IsEndOfStream bool `protobuf:"varint,2,opt,name=is_end_of_stream,json=isEndOfStream,proto3" json:"is_end_of_stream,omitempty"` + IsEndOfTopic bool `protobuf:"varint,3,opt,name=is_end_of_topic,json=isEndOfTopic,proto3" json:"is_end_of_topic,omitempty"` } -func (x *PublishRequest_InitMessage) Reset() { - *x = PublishRequest_InitMessage{} +func (x *SubscribeMessageResponse_CtrlMessage) Reset() { + *x = SubscribeMessageResponse_CtrlMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[36] + mi := &file_mq_proto_msgTypes[45] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *PublishRequest_InitMessage) String() string { +func (x *SubscribeMessageResponse_CtrlMessage) String() string { return protoimpl.X.MessageStringOf(x) } -func (*PublishRequest_InitMessage) ProtoMessage() {} +func (*SubscribeMessageResponse_CtrlMessage) ProtoMessage() {} -func (x *PublishRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[36] +func (x *SubscribeMessageResponse_CtrlMessage) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[45] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2057,67 +2709,61 @@ func (x *PublishRequest_InitMessage) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use PublishRequest_InitMessage.ProtoReflect.Descriptor instead. -func (*PublishRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{22, 0} +// Deprecated: Use SubscribeMessageResponse_CtrlMessage.ProtoReflect.Descriptor instead. +func (*SubscribeMessageResponse_CtrlMessage) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{29, 0} } -func (x *PublishRequest_InitMessage) GetTopic() *Topic { +func (x *SubscribeMessageResponse_CtrlMessage) GetError() string { if x != nil { - return x.Topic + return x.Error } - return nil + return "" } -func (x *PublishRequest_InitMessage) GetPartition() *Partition { +func (x *SubscribeMessageResponse_CtrlMessage) GetIsEndOfStream() bool { if x != nil { - return x.Partition + return x.IsEndOfStream } - return nil + return false } -func (x *PublishRequest_InitMessage) GetAckInterval() int32 { +func (x *SubscribeMessageResponse_CtrlMessage) GetIsEndOfTopic() bool { if x != nil { - return x.AckInterval + return x.IsEndOfTopic } - return 0 + return false } -type SubscribeRequest_InitMessage struct { +type FollowInMemoryMessagesRequest_InitMessage struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - ConsumerGroup string `protobuf:"bytes,1,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"` - ConsumerId string `protobuf:"bytes,2,opt,name=consumer_id,json=consumerId,proto3" json:"consumer_id,omitempty"` - ClientId string `protobuf:"bytes,3,opt,name=client_id,json=clientId,proto3" json:"client_id,omitempty"` - Topic *Topic `protobuf:"bytes,4,opt,name=topic,proto3" json:"topic,omitempty"` - Partition *Partition `protobuf:"bytes,5,opt,name=partition,proto3" json:"partition,omitempty"` - // Types that are assignable to Offset: - // - // *SubscribeRequest_InitMessage_StartOffset - // *SubscribeRequest_InitMessage_StartTimestampNs - Offset isSubscribeRequest_InitMessage_Offset `protobuf_oneof:"offset"` - Filter string `protobuf:"bytes,8,opt,name=filter,proto3" json:"filter,omitempty"` + ConsumerGroup string `protobuf:"bytes,1,opt,name=consumer_group,json=consumerGroup,proto3" json:"consumer_group,omitempty"` + ConsumerId string `protobuf:"bytes,2,opt,name=consumer_id,json=consumerId,proto3" json:"consumer_id,omitempty"` + FollowerId int32 `protobuf:"varint,3,opt,name=follower_id,json=followerId,proto3" json:"follower_id,omitempty"` + Topic *Topic `protobuf:"bytes,4,opt,name=topic,proto3" json:"topic,omitempty"` + PartitionOffset *PartitionOffset `protobuf:"bytes,5,opt,name=partition_offset,json=partitionOffset,proto3" json:"partition_offset,omitempty"` } -func (x *SubscribeRequest_InitMessage) Reset() { - *x = SubscribeRequest_InitMessage{} +func (x *FollowInMemoryMessagesRequest_InitMessage) Reset() { + *x = FollowInMemoryMessagesRequest_InitMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[37] + mi := &file_mq_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *SubscribeRequest_InitMessage) String() string { +func (x *FollowInMemoryMessagesRequest_InitMessage) String() string { return protoimpl.X.MessageStringOf(x) } -func (*SubscribeRequest_InitMessage) ProtoMessage() {} +func (*FollowInMemoryMessagesRequest_InitMessage) ProtoMessage() {} -func (x *SubscribeRequest_InitMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[37] +func (x *FollowInMemoryMessagesRequest_InitMessage) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[46] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2128,91 +2774,47 @@ func (x *SubscribeRequest_InitMessage) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SubscribeRequest_InitMessage.ProtoReflect.Descriptor instead. -func (*SubscribeRequest_InitMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{24, 0} +// Deprecated: Use FollowInMemoryMessagesRequest_InitMessage.ProtoReflect.Descriptor instead. +func (*FollowInMemoryMessagesRequest_InitMessage) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{30, 0} } -func (x *SubscribeRequest_InitMessage) GetConsumerGroup() string { +func (x *FollowInMemoryMessagesRequest_InitMessage) GetConsumerGroup() string { if x != nil { return x.ConsumerGroup } return "" } -func (x *SubscribeRequest_InitMessage) GetConsumerId() string { +func (x *FollowInMemoryMessagesRequest_InitMessage) GetConsumerId() string { if x != nil { return x.ConsumerId } return "" } -func (x *SubscribeRequest_InitMessage) GetClientId() string { +func (x *FollowInMemoryMessagesRequest_InitMessage) GetFollowerId() int32 { if x != nil { - return x.ClientId + return x.FollowerId } - return "" + return 0 } -func (x *SubscribeRequest_InitMessage) GetTopic() *Topic { +func (x *FollowInMemoryMessagesRequest_InitMessage) GetTopic() *Topic { if x != nil { return x.Topic } return nil } -func (x *SubscribeRequest_InitMessage) GetPartition() *Partition { +func (x *FollowInMemoryMessagesRequest_InitMessage) GetPartitionOffset() *PartitionOffset { if x != nil { - return x.Partition - } - return nil -} - -func (m *SubscribeRequest_InitMessage) GetOffset() isSubscribeRequest_InitMessage_Offset { - if m != nil { - return m.Offset + return x.PartitionOffset } return nil } -func (x *SubscribeRequest_InitMessage) GetStartOffset() int64 { - if x, ok := x.GetOffset().(*SubscribeRequest_InitMessage_StartOffset); ok { - return x.StartOffset - } - return 0 -} - -func (x *SubscribeRequest_InitMessage) GetStartTimestampNs() int64 { - if x, ok := x.GetOffset().(*SubscribeRequest_InitMessage_StartTimestampNs); ok { - return x.StartTimestampNs - } - return 0 -} - -func (x *SubscribeRequest_InitMessage) GetFilter() string { - if x != nil { - return x.Filter - } - return "" -} - -type isSubscribeRequest_InitMessage_Offset interface { - isSubscribeRequest_InitMessage_Offset() -} - -type SubscribeRequest_InitMessage_StartOffset struct { - StartOffset int64 `protobuf:"varint,6,opt,name=start_offset,json=startOffset,proto3,oneof"` -} - -type SubscribeRequest_InitMessage_StartTimestampNs struct { - StartTimestampNs int64 `protobuf:"varint,7,opt,name=start_timestamp_ns,json=startTimestampNs,proto3,oneof"` -} - -func (*SubscribeRequest_InitMessage_StartOffset) isSubscribeRequest_InitMessage_Offset() {} - -func (*SubscribeRequest_InitMessage_StartTimestampNs) isSubscribeRequest_InitMessage_Offset() {} - -type SubscribeRequest_AckMessage struct { +type FollowInMemoryMessagesRequest_AckMessage struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields @@ -2220,23 +2822,23 @@ type SubscribeRequest_AckMessage struct { Sequence int64 `protobuf:"varint,1,opt,name=sequence,proto3" json:"sequence,omitempty"` } -func (x *SubscribeRequest_AckMessage) Reset() { - *x = SubscribeRequest_AckMessage{} +func (x *FollowInMemoryMessagesRequest_AckMessage) Reset() { + *x = FollowInMemoryMessagesRequest_AckMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[38] + mi := &file_mq_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *SubscribeRequest_AckMessage) String() string { +func (x *FollowInMemoryMessagesRequest_AckMessage) String() string { return protoimpl.X.MessageStringOf(x) } -func (*SubscribeRequest_AckMessage) ProtoMessage() {} +func (*FollowInMemoryMessagesRequest_AckMessage) ProtoMessage() {} -func (x *SubscribeRequest_AckMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[38] +func (x *FollowInMemoryMessagesRequest_AckMessage) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[47] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2247,45 +2849,44 @@ func (x *SubscribeRequest_AckMessage) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SubscribeRequest_AckMessage.ProtoReflect.Descriptor instead. -func (*SubscribeRequest_AckMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{24, 1} +// Deprecated: Use FollowInMemoryMessagesRequest_AckMessage.ProtoReflect.Descriptor instead. +func (*FollowInMemoryMessagesRequest_AckMessage) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{30, 1} } -func (x *SubscribeRequest_AckMessage) GetSequence() int64 { +func (x *FollowInMemoryMessagesRequest_AckMessage) GetSequence() int64 { if x != nil { return x.Sequence } return 0 } -type SubscribeResponse_CtrlMessage struct { +type FollowInMemoryMessagesResponse_CtrlMessage struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"` - IsEndOfStream bool `protobuf:"varint,2,opt,name=is_end_of_stream,json=isEndOfStream,proto3" json:"is_end_of_stream,omitempty"` - IsEndOfTopic bool `protobuf:"varint,3,opt,name=is_end_of_topic,json=isEndOfTopic,proto3" json:"is_end_of_topic,omitempty"` + FlushedSequence int64 `protobuf:"varint,1,opt,name=flushed_sequence,json=flushedSequence,proto3" json:"flushed_sequence,omitempty"` + FollowerChangedToId int32 `protobuf:"varint,2,opt,name=follower_changed_to_id,json=followerChangedToId,proto3" json:"follower_changed_to_id,omitempty"` } -func (x *SubscribeResponse_CtrlMessage) Reset() { - *x = SubscribeResponse_CtrlMessage{} +func (x *FollowInMemoryMessagesResponse_CtrlMessage) Reset() { + *x = FollowInMemoryMessagesResponse_CtrlMessage{} if protoimpl.UnsafeEnabled { - mi := &file_mq_proto_msgTypes[39] + mi := &file_mq_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } } -func (x *SubscribeResponse_CtrlMessage) String() string { +func (x *FollowInMemoryMessagesResponse_CtrlMessage) String() string { return protoimpl.X.MessageStringOf(x) } -func (*SubscribeResponse_CtrlMessage) ProtoMessage() {} +func (*FollowInMemoryMessagesResponse_CtrlMessage) ProtoMessage() {} -func (x *SubscribeResponse_CtrlMessage) ProtoReflect() protoreflect.Message { - mi := &file_mq_proto_msgTypes[39] +func (x *FollowInMemoryMessagesResponse_CtrlMessage) ProtoReflect() protoreflect.Message { + mi := &file_mq_proto_msgTypes[48] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2296,30 +2897,23 @@ func (x *SubscribeResponse_CtrlMessage) ProtoReflect() protoreflect.Message { return mi.MessageOf(x) } -// Deprecated: Use SubscribeResponse_CtrlMessage.ProtoReflect.Descriptor instead. -func (*SubscribeResponse_CtrlMessage) Descriptor() ([]byte, []int) { - return file_mq_proto_rawDescGZIP(), []int{25, 0} -} - -func (x *SubscribeResponse_CtrlMessage) GetError() string { - if x != nil { - return x.Error - } - return "" +// Deprecated: Use FollowInMemoryMessagesResponse_CtrlMessage.ProtoReflect.Descriptor instead. +func (*FollowInMemoryMessagesResponse_CtrlMessage) Descriptor() ([]byte, []int) { + return file_mq_proto_rawDescGZIP(), []int{31, 0} } -func (x *SubscribeResponse_CtrlMessage) GetIsEndOfStream() bool { +func (x *FollowInMemoryMessagesResponse_CtrlMessage) GetFlushedSequence() int64 { if x != nil { - return x.IsEndOfStream + return x.FlushedSequence } - return false + return 0 } -func (x *SubscribeResponse_CtrlMessage) GetIsEndOfTopic() bool { +func (x *FollowInMemoryMessagesResponse_CtrlMessage) GetFollowerChangedToId() int32 { if x != nil { - return x.IsEndOfTopic + return x.FollowerChangedToId } - return false + return 0 } var File_mq_proto protoreflect.FileDescriptor @@ -2346,269 +2940,357 @@ var file_mq_proto_rawDesc = []byte{ 0x01, 0x28, 0x05, 0x52, 0x09, 0x72, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x74, 0x6f, 0x70, 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, 0x6d, 0x65, 0x4e, 0x73, - 0x22, 0xd2, 0x01, 0x0a, 0x0b, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, - 0x12, 0x2a, 0x0a, 0x11, 0x63, 0x70, 0x75, 0x5f, 0x75, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x70, 0x65, - 0x72, 0x63, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x63, 0x70, 0x75, - 0x55, 0x73, 0x61, 0x67, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x12, 0x3a, 0x0a, 0x05, - 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x65, + 0x22, 0x7f, 0x0a, 0x06, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, + 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x4a, 0x0a, 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, + 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, + 0x73, 0x22, 0xcd, 0x01, 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, + 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0b, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x73, 0x4e, 0x73, 0x12, 0x1c, 0x0a, 0x0a, + 0x73, 0x74, 0x6f, 0x70, 0x5f, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x08, 0x73, 0x74, 0x6f, 0x70, 0x54, 0x73, 0x4e, 0x73, 0x12, 0x45, 0x0a, 0x0a, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, + 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61, + 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x79, 0x70, + 0x65, 0x22, 0xd2, 0x01, 0x0a, 0x0b, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x12, 0x2a, 0x0a, 0x11, 0x63, 0x70, 0x75, 0x5f, 0x75, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x70, + 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0f, 0x63, 0x70, + 0x75, 0x55, 0x73, 0x61, 0x67, 0x65, 0x50, 0x65, 0x72, 0x63, 0x65, 0x6e, 0x74, 0x12, 0x3a, 0x0a, + 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, + 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x5b, 0x0a, 0x0a, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xbb, 0x01, 0x0a, 0x13, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x29, + 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, + 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x22, 0xd3, 0x01, 0x0a, 0x1d, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, + 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, + 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, + 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x31, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x48, + 0x00, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x25, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, + 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x42, + 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x20, 0x0a, 0x1e, 0x50, 0x75, + 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, + 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0x0a, 0x14, + 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x22, 0x17, 0x0a, 0x15, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x6b, 0x0a, + 0x15, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, + 0x63, 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, + 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x70, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x83, 0x01, 0x0a, 0x16, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, - 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x5b, 0x0a, 0x0a, 0x53, 0x74, 0x61, 0x74, - 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x37, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xbb, 0x01, 0x0a, 0x13, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x29, 0x0a, - 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, - 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x63, 0x6f, 0x75, 0x6e, - 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, - 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x22, 0xd3, 0x01, 0x0a, 0x1d, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, - 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, - 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, - 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, - 0x69, 0x6e, 0x69, 0x74, 0x12, 0x31, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, - 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x73, 0x48, 0x00, - 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x25, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x42, 0x09, - 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x20, 0x0a, 0x1e, 0x50, 0x75, 0x62, - 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x54, 0x6f, 0x50, 0x75, 0x62, 0x42, 0x61, 0x6c, 0x61, 0x6e, - 0x63, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x16, 0x0a, 0x14, 0x42, - 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x22, 0x17, 0x0a, 0x15, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x54, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x6b, 0x0a, 0x15, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, + 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x22, 0x13, 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x41, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, + 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x52, 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x22, 0x46, 0x0a, 0x19, 0x4c, 0x6f, 0x6f, 0x6b, + 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, - 0x12, 0x27, 0x0a, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f, - 0x75, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x83, 0x01, 0x0a, 0x16, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x65, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, - 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, + 0x22, 0xb2, 0x01, 0x0a, 0x1a, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, + 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, + 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, + 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, + 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xa2, 0x01, 0x0a, 0x19, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, - 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, - 0x13, 0x0a, 0x11, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x22, 0x41, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x54, 0x6f, 0x70, 0x69, - 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2b, 0x0a, 0x06, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, + 0x65, 0x6e, 0x74, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x6c, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0c, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x12, + 0x29, 0x0a, 0x10, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, + 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, + 0x77, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x22, 0xf2, 0x01, 0x0a, 0x1c, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, + 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, - 0x06, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x73, 0x22, 0x6c, 0x0a, 0x19, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, - 0x70, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, - 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, - 0x24, 0x0a, 0x0e, 0x69, 0x73, 0x5f, 0x66, 0x6f, 0x72, 0x5f, 0x70, 0x75, 0x62, 0x6c, 0x69, 0x73, - 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x46, 0x6f, 0x72, 0x50, 0x75, - 0x62, 0x6c, 0x69, 0x73, 0x68, 0x22, 0xb2, 0x01, 0x0a, 0x1a, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, - 0x54, 0x6f, 0x70, 0x69, 0x63, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, - 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, - 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, - 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, - 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xa2, 0x01, 0x0a, 0x19, 0x42, - 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, - 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, + 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x69, 0x0a, 0x1c, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, + 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x61, 0x73, 0x73, 0x69, 0x67, + 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x42, 0x72, 0x6f, 0x6b, + 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, + 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x1f, + 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x64, 0x72, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x44, 0x72, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x22, + 0x1f, 0x0a, 0x1d, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0xca, 0x03, 0x0a, 0x21, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, + 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, + 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x4e, 0x0a, 0x03, 0x61, 0x63, 0x6b, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, + 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x1a, 0x9c, 0x01, 0x0a, 0x0b, 0x49, 0x6e, + 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, + 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, + 0x12, 0x3b, 0x0a, 0x1a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x17, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, + 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x12, 0x29, 0x0a, + 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, + 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x1a, 0x58, 0x0a, 0x0a, 0x41, 0x63, 0x6b, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x13, 0x0a, + 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, + 0x4e, 0x73, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0xab, 0x03, + 0x0a, 0x22, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, + 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, 0x0a, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, + 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, + 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, + 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x00, 0x52, 0x0a, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, + 0x65, 0x6e, 0x74, 0x1a, 0x77, 0x0a, 0x11, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x23, 0x0a, 0x0d, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x42, 0x72, - 0x6f, 0x6b, 0x65, 0x72, 0x12, 0x29, 0x0a, 0x10, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, - 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, - 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x22, - 0xf2, 0x01, 0x0a, 0x1c, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, - 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x69, 0x0a, 0x1c, 0x62, - 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, - 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x1a, 0x62, 0x72, 0x6f, 0x6b, - 0x65, 0x72, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x73, 0x73, 0x69, 0x67, - 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x6c, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x4c, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x12, 0x1f, 0x0a, 0x0b, 0x69, 0x73, 0x5f, 0x64, 0x72, 0x61, 0x69, 0x6e, 0x69, - 0x6e, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x69, 0x73, 0x44, 0x72, 0x61, 0x69, - 0x6e, 0x69, 0x6e, 0x67, 0x22, 0x1f, 0x0a, 0x1d, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x54, 0x6f, - 0x70, 0x69, 0x63, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xbf, 0x03, 0x0a, 0x21, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, - 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x04, 0x69, - 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, - 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x4e, - 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, - 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x1a, 0x91, - 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, - 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x30, 0x0a, 0x14, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, - 0x72, 0x5f, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x12, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x6e, 0x73, - 0x74, 0x61, 0x6e, 0x63, 0x65, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, - 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x1a, 0x58, 0x0a, 0x0a, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, + 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, + 0x74, 0x73, 0x4e, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x1a, 0xa1, 0x01, 0x0a, + 0x0a, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1e, 0x0a, 0x0a, 0x67, + 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0a, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x73, 0x0a, 0x13, 0x61, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, + 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, + 0x6e, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, 0x61, 0x73, + 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x4a, 0x0a, 0x0b, 0x44, + 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 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, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x22, 0xf6, 0x02, 0x0a, 0x15, 0x50, 0x75, 0x62, 0x6c, + 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x45, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x2f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, + 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x71, + 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, 0x71, + 0x75, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0xbd, 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x42, 0x09, 0x0a, 0x07, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x93, 0x03, 0x0a, 0x22, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, - 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, - 0x0a, 0x0a, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, - 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x48, - 0x00, 0x52, 0x0a, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x1a, 0x5f, 0x0a, - 0x11, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, - 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x1a, 0xa1, - 0x01, 0x0a, 0x0a, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1e, 0x0a, - 0x0a, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x0a, 0x67, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x73, 0x0a, - 0x13, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, - 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x41, 0x73, 0x73, - 0x69, 0x67, 0x6e, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x12, - 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x4a, 0x0a, - 0x0b, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 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, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x73, 0x5f, 0x6e, 0x73, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x04, 0x74, 0x73, 0x4e, 0x73, 0x22, 0xbd, 0x02, 0x0a, 0x0e, 0x50, 0x75, - 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3e, 0x0a, 0x04, - 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, - 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x2f, 0x0a, 0x04, - 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x1a, 0x0a, - 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x1a, 0x92, 0x01, 0x0a, 0x0b, 0x49, 0x6e, - 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, - 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, - 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x61, - 0x63, 0x6b, 0x5f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x42, 0x09, - 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x6d, 0x0a, 0x0f, 0x50, 0x75, 0x62, - 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, - 0x61, 0x63, 0x6b, 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, - 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, - 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, - 0x63, 0x6c, 0x6f, 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x68, 0x6f, - 0x75, 0x6c, 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0x96, 0x04, 0x0a, 0x10, 0x53, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x40, 0x0a, - 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, - 0x3d, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, - 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x1a, 0xcb, - 0x02, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, - 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, - 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, - 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, - 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, 0x6e, 0x74, - 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, 0x65, 0x6e, - 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, - 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, - 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, - 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6f, - 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x0b, 0x73, - 0x74, 0x61, 0x72, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x2e, 0x0a, 0x12, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x6e, 0x73, - 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x48, 0x00, 0x52, 0x10, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x4e, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, - 0x6c, 0x74, 0x65, 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, - 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x1a, 0x28, 0x0a, 0x0a, - 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, - 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, - 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x65, 0x22, 0x87, 0x02, 0x0a, 0x11, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x41, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, - 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x43, 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, - 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x1a, 0x73, 0x0a, 0x0b, 0x43, - 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, - 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, - 0x12, 0x27, 0x0a, 0x10, 0x69, 0x73, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x73, 0x74, - 0x72, 0x65, 0x61, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, 0x69, 0x73, 0x45, 0x6e, - 0x64, 0x4f, 0x66, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x25, 0x0a, 0x0f, 0x69, 0x73, 0x5f, - 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, 0x6f, 0x70, 0x69, 0x63, - 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x65, 0x0a, 0x16, 0x43, - 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, 0x6b, 0x5f, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x61, + 0x63, 0x6b, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x29, 0x0a, 0x10, 0x66, 0x6f, + 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x0f, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x42, 0x72, + 0x6f, 0x6b, 0x65, 0x72, 0x73, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x22, 0x74, 0x0a, 0x16, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x61, 0x63, + 0x6b, 0x5f, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0b, 0x61, 0x63, 0x6b, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x14, 0x0a, + 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x68, 0x6f, 0x75, 0x6c, 0x64, 0x5f, 0x63, 0x6c, + 0x6f, 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x73, 0x68, 0x6f, 0x75, 0x6c, + 0x64, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x22, 0x9b, 0x01, 0x0a, 0x16, 0x50, 0x75, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x35, 0x0a, 0x09, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x17, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x5f, 0x73, 0x65, + 0x6c, 0x66, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x62, 0x72, 0x6f, 0x6b, 0x65, 0x72, + 0x53, 0x65, 0x6c, 0x66, 0x22, 0x2f, 0x0a, 0x17, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, + 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 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, 0xdf, 0x03, 0x0a, 0x17, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x47, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x31, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x12, 0x44, 0x0a, 0x03, 0x61, 0x63, + 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, + 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, + 0x1a, 0xff, 0x01, 0x0a, 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6d, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, + 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6c, 0x69, 0x65, + 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x63, 0x6c, 0x69, + 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x12, 0x48, 0x0a, 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x66, 0x69, + 0x6c, 0x74, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x66, 0x69, 0x6c, 0x74, + 0x65, 0x72, 0x1a, 0x28, 0x0a, 0x0a, 0x41, 0x63, 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x42, 0x09, 0x0a, 0x07, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x95, 0x02, 0x0a, 0x18, 0x53, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, + 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x43, 0x74, 0x72, 0x6c, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x12, 0x2f, + 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x44, 0x61, 0x74, 0x61, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x1a, + 0x73, 0x0a, 0x0b, 0x43, 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x14, + 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, + 0x72, 0x72, 0x6f, 0x72, 0x12, 0x27, 0x0a, 0x10, 0x69, 0x73, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x6f, + 0x66, 0x5f, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0d, + 0x69, 0x73, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x25, 0x0a, + 0x0f, 0x69, 0x73, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x5f, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x69, 0x73, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x54, + 0x6f, 0x70, 0x69, 0x63, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, + 0xdd, 0x03, 0x0a, 0x1d, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, + 0x72, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x4d, 0x0a, 0x04, 0x69, 0x6e, 0x69, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x37, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, + 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x49, 0x6e, 0x69, + 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x69, 0x6e, 0x69, 0x74, + 0x12, 0x4a, 0x0a, 0x03, 0x61, 0x63, 0x6b, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x6f, 0x6c, + 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x41, 0x63, 0x6b, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x03, 0x61, 0x63, 0x6b, 0x1a, 0xeb, 0x01, 0x0a, + 0x0b, 0x49, 0x6e, 0x69, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x0a, 0x0e, + 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x47, 0x72, + 0x6f, 0x75, 0x70, 0x12, 0x1f, 0x0a, 0x0b, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, 0x65, 0x72, 0x5f, + 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6d, + 0x65, 0x72, 0x49, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, + 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, + 0x77, 0x65, 0x72, 0x49, 0x64, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, - 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x6e, 0x73, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, 0x6d, 0x65, - 0x4e, 0x73, 0x22, 0x19, 0x0a, 0x17, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, - 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x66, 0x0a, - 0x17, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, - 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, - 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, - 0x70, 0x69, 0x63, 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, - 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, - 0x69, 0x6d, 0x65, 0x4e, 0x73, 0x22, 0x1a, 0x0a, 0x18, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x32, 0xce, 0x09, 0x0a, 0x10, 0x53, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x4d, 0x65, 0x73, + 0x12, 0x48, 0x0a, 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x0f, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x1a, 0x28, 0x0a, 0x0a, 0x41, 0x63, + 0x6b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x71, 0x75, + 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x73, 0x65, 0x71, 0x75, + 0x65, 0x6e, 0x63, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, + 0x9b, 0x02, 0x0a, 0x1e, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, + 0x72, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x4e, 0x0a, 0x04, 0x63, 0x74, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x38, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x43, + 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x63, 0x74, + 0x72, 0x6c, 0x12, 0x2f, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x19, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x44, 0x61, 0x74, 0x61, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00, 0x52, 0x04, 0x64, + 0x61, 0x74, 0x61, 0x1a, 0x6d, 0x0a, 0x0b, 0x43, 0x74, 0x72, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x12, 0x29, 0x0a, 0x10, 0x66, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x5f, 0x73, 0x65, + 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x66, 0x6c, + 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x33, 0x0a, + 0x16, 0x66, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, + 0x64, 0x5f, 0x74, 0x6f, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x13, 0x66, + 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x64, 0x54, 0x6f, + 0x49, 0x64, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x65, 0x0a, + 0x16, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, + 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, 0x74, 0x6f, 0x70, + 0x69, 0x63, 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, + 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, 0x78, 0x54, 0x69, + 0x6d, 0x65, 0x4e, 0x73, 0x22, 0x19, 0x0a, 0x17, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x50, 0x75, 0x62, + 0x6c, 0x69, 0x73, 0x68, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x66, 0x0a, 0x17, 0x43, 0x6c, 0x6f, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x29, 0x0a, 0x05, 0x74, 0x6f, + 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x13, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x52, 0x05, + 0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x20, 0x0a, 0x0c, 0x75, 0x6e, 0x69, 0x78, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x5f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x6e, 0x69, + 0x78, 0x54, 0x69, 0x6d, 0x65, 0x4e, 0x73, 0x22, 0x1a, 0x0a, 0x18, 0x43, 0x6c, 0x6f, 0x73, 0x65, + 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x2a, 0x4c, 0x0a, 0x18, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x0c, 0x0a, 0x08, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45, 0x53, 0x54, 0x10, 0x00, 0x12, 0x16, 0x0a, + 0x12, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x4d, 0x45, 0x4d, + 0x4f, 0x52, 0x59, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x10, + 0x02, 0x32, 0xd3, 0x0b, 0x0a, 0x10, 0x53, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x12, 0x63, 0x0a, 0x10, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, 0x6f, 0x6b, 0x65, 0x72, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x69, 0x6e, 0x64, 0x42, 0x72, @@ -2675,22 +3357,38 @@ var file_mq_proto_rawDesc = []byte{ 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x72, 0x54, 0x6f, 0x53, 0x75, 0x62, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x69, 0x6e, 0x61, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, - 0x30, 0x01, 0x12, 0x4c, 0x0a, 0x07, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x12, 0x1c, 0x2e, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, - 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x6d, 0x65, - 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, - 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, - 0x12, 0x50, 0x0a, 0x09, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x12, 0x1e, 0x2e, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, - 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x30, 0x01, 0x42, 0x4f, 0x0a, 0x0c, 0x73, 0x65, 0x61, 0x77, 0x65, 0x65, 0x64, 0x66, 0x73, 0x2e, - 0x6d, 0x71, 0x42, 0x11, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x51, 0x75, 0x65, 0x75, 0x65, - 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2c, 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, 0x6d, 0x71, - 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x30, 0x01, 0x12, 0x61, 0x0a, 0x0e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x12, 0x23, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, + 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x4d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x65, 0x0a, 0x10, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, + 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x26, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, + 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12, 0x60, 0x0a, 0x0f, + 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x12, + 0x24, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, + 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x50, 0x75, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x46, 0x6f, 0x6c, 0x6c, + 0x6f, 0x77, 0x4d, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x77, + 0x0a, 0x16, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, + 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x12, 0x2b, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, + 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x69, 0x6e, + 0x67, 0x5f, 0x70, 0x62, 0x2e, 0x46, 0x6f, 0x6c, 0x6c, 0x6f, 0x77, 0x49, 0x6e, 0x4d, 0x65, 0x6d, + 0x6f, 0x72, 0x79, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x42, 0x4f, 0x0a, 0x0c, 0x73, 0x65, 0x61, 0x77, 0x65, + 0x65, 0x64, 0x66, 0x73, 0x2e, 0x6d, 0x71, 0x42, 0x11, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x51, 0x75, 0x65, 0x75, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x5a, 0x2c, 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, 0x6d, 0x71, 0x5f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2705,113 +3403,140 @@ func file_mq_proto_rawDescGZIP() []byte { return file_mq_proto_rawDescData } -var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 40) +var file_mq_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_mq_proto_msgTypes = make([]protoimpl.MessageInfo, 49) var file_mq_proto_goTypes = []interface{}{ - (*FindBrokerLeaderRequest)(nil), // 0: messaging_pb.FindBrokerLeaderRequest - (*FindBrokerLeaderResponse)(nil), // 1: messaging_pb.FindBrokerLeaderResponse - (*Topic)(nil), // 2: messaging_pb.Topic - (*Partition)(nil), // 3: messaging_pb.Partition - (*BrokerStats)(nil), // 4: messaging_pb.BrokerStats - (*TopicPartitionStats)(nil), // 5: messaging_pb.TopicPartitionStats - (*PublisherToPubBalancerRequest)(nil), // 6: messaging_pb.PublisherToPubBalancerRequest - (*PublisherToPubBalancerResponse)(nil), // 7: messaging_pb.PublisherToPubBalancerResponse - (*BalanceTopicsRequest)(nil), // 8: messaging_pb.BalanceTopicsRequest - (*BalanceTopicsResponse)(nil), // 9: messaging_pb.BalanceTopicsResponse - (*ConfigureTopicRequest)(nil), // 10: messaging_pb.ConfigureTopicRequest - (*ConfigureTopicResponse)(nil), // 11: messaging_pb.ConfigureTopicResponse - (*ListTopicsRequest)(nil), // 12: messaging_pb.ListTopicsRequest - (*ListTopicsResponse)(nil), // 13: messaging_pb.ListTopicsResponse - (*LookupTopicBrokersRequest)(nil), // 14: messaging_pb.LookupTopicBrokersRequest - (*LookupTopicBrokersResponse)(nil), // 15: messaging_pb.LookupTopicBrokersResponse - (*BrokerPartitionAssignment)(nil), // 16: messaging_pb.BrokerPartitionAssignment - (*AssignTopicPartitionsRequest)(nil), // 17: messaging_pb.AssignTopicPartitionsRequest - (*AssignTopicPartitionsResponse)(nil), // 18: messaging_pb.AssignTopicPartitionsResponse - (*SubscriberToSubCoordinatorRequest)(nil), // 19: messaging_pb.SubscriberToSubCoordinatorRequest - (*SubscriberToSubCoordinatorResponse)(nil), // 20: messaging_pb.SubscriberToSubCoordinatorResponse - (*DataMessage)(nil), // 21: messaging_pb.DataMessage - (*PublishRequest)(nil), // 22: messaging_pb.PublishRequest - (*PublishResponse)(nil), // 23: messaging_pb.PublishResponse - (*SubscribeRequest)(nil), // 24: messaging_pb.SubscribeRequest - (*SubscribeResponse)(nil), // 25: messaging_pb.SubscribeResponse - (*ClosePublishersRequest)(nil), // 26: messaging_pb.ClosePublishersRequest - (*ClosePublishersResponse)(nil), // 27: messaging_pb.ClosePublishersResponse - (*CloseSubscribersRequest)(nil), // 28: messaging_pb.CloseSubscribersRequest - (*CloseSubscribersResponse)(nil), // 29: messaging_pb.CloseSubscribersResponse - nil, // 30: messaging_pb.BrokerStats.StatsEntry - (*PublisherToPubBalancerRequest_InitMessage)(nil), // 31: messaging_pb.PublisherToPubBalancerRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 32: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage - (*SubscriberToSubCoordinatorRequest_AckMessage)(nil), // 33: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage - (*SubscriberToSubCoordinatorResponse_AssignedPartition)(nil), // 34: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition - (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 35: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - (*PublishRequest_InitMessage)(nil), // 36: messaging_pb.PublishRequest.InitMessage - (*SubscribeRequest_InitMessage)(nil), // 37: messaging_pb.SubscribeRequest.InitMessage - (*SubscribeRequest_AckMessage)(nil), // 38: messaging_pb.SubscribeRequest.AckMessage - (*SubscribeResponse_CtrlMessage)(nil), // 39: messaging_pb.SubscribeResponse.CtrlMessage + (PartitionOffsetStartType)(0), // 0: messaging_pb.PartitionOffsetStartType + (*FindBrokerLeaderRequest)(nil), // 1: messaging_pb.FindBrokerLeaderRequest + (*FindBrokerLeaderResponse)(nil), // 2: messaging_pb.FindBrokerLeaderResponse + (*Topic)(nil), // 3: messaging_pb.Topic + (*Partition)(nil), // 4: messaging_pb.Partition + (*Offset)(nil), // 5: messaging_pb.Offset + (*PartitionOffset)(nil), // 6: messaging_pb.PartitionOffset + (*BrokerStats)(nil), // 7: messaging_pb.BrokerStats + (*TopicPartitionStats)(nil), // 8: messaging_pb.TopicPartitionStats + (*PublisherToPubBalancerRequest)(nil), // 9: messaging_pb.PublisherToPubBalancerRequest + (*PublisherToPubBalancerResponse)(nil), // 10: messaging_pb.PublisherToPubBalancerResponse + (*BalanceTopicsRequest)(nil), // 11: messaging_pb.BalanceTopicsRequest + (*BalanceTopicsResponse)(nil), // 12: messaging_pb.BalanceTopicsResponse + (*ConfigureTopicRequest)(nil), // 13: messaging_pb.ConfigureTopicRequest + (*ConfigureTopicResponse)(nil), // 14: messaging_pb.ConfigureTopicResponse + (*ListTopicsRequest)(nil), // 15: messaging_pb.ListTopicsRequest + (*ListTopicsResponse)(nil), // 16: messaging_pb.ListTopicsResponse + (*LookupTopicBrokersRequest)(nil), // 17: messaging_pb.LookupTopicBrokersRequest + (*LookupTopicBrokersResponse)(nil), // 18: messaging_pb.LookupTopicBrokersResponse + (*BrokerPartitionAssignment)(nil), // 19: messaging_pb.BrokerPartitionAssignment + (*AssignTopicPartitionsRequest)(nil), // 20: messaging_pb.AssignTopicPartitionsRequest + (*AssignTopicPartitionsResponse)(nil), // 21: messaging_pb.AssignTopicPartitionsResponse + (*SubscriberToSubCoordinatorRequest)(nil), // 22: messaging_pb.SubscriberToSubCoordinatorRequest + (*SubscriberToSubCoordinatorResponse)(nil), // 23: messaging_pb.SubscriberToSubCoordinatorResponse + (*DataMessage)(nil), // 24: messaging_pb.DataMessage + (*PublishMessageRequest)(nil), // 25: messaging_pb.PublishMessageRequest + (*PublishMessageResponse)(nil), // 26: messaging_pb.PublishMessageResponse + (*PublishFollowMeRequest)(nil), // 27: messaging_pb.PublishFollowMeRequest + (*PublishFollowMeResponse)(nil), // 28: messaging_pb.PublishFollowMeResponse + (*SubscribeMessageRequest)(nil), // 29: messaging_pb.SubscribeMessageRequest + (*SubscribeMessageResponse)(nil), // 30: messaging_pb.SubscribeMessageResponse + (*FollowInMemoryMessagesRequest)(nil), // 31: messaging_pb.FollowInMemoryMessagesRequest + (*FollowInMemoryMessagesResponse)(nil), // 32: messaging_pb.FollowInMemoryMessagesResponse + (*ClosePublishersRequest)(nil), // 33: messaging_pb.ClosePublishersRequest + (*ClosePublishersResponse)(nil), // 34: messaging_pb.ClosePublishersResponse + (*CloseSubscribersRequest)(nil), // 35: messaging_pb.CloseSubscribersRequest + (*CloseSubscribersResponse)(nil), // 36: messaging_pb.CloseSubscribersResponse + nil, // 37: messaging_pb.BrokerStats.StatsEntry + (*PublisherToPubBalancerRequest_InitMessage)(nil), // 38: messaging_pb.PublisherToPubBalancerRequest.InitMessage + (*SubscriberToSubCoordinatorRequest_InitMessage)(nil), // 39: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage + (*SubscriberToSubCoordinatorRequest_AckMessage)(nil), // 40: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage + (*SubscriberToSubCoordinatorResponse_AssignedPartition)(nil), // 41: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition + (*SubscriberToSubCoordinatorResponse_Assignment)(nil), // 42: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment + (*PublishMessageRequest_InitMessage)(nil), // 43: messaging_pb.PublishMessageRequest.InitMessage + (*SubscribeMessageRequest_InitMessage)(nil), // 44: messaging_pb.SubscribeMessageRequest.InitMessage + (*SubscribeMessageRequest_AckMessage)(nil), // 45: messaging_pb.SubscribeMessageRequest.AckMessage + (*SubscribeMessageResponse_CtrlMessage)(nil), // 46: messaging_pb.SubscribeMessageResponse.CtrlMessage + (*FollowInMemoryMessagesRequest_InitMessage)(nil), // 47: messaging_pb.FollowInMemoryMessagesRequest.InitMessage + (*FollowInMemoryMessagesRequest_AckMessage)(nil), // 48: messaging_pb.FollowInMemoryMessagesRequest.AckMessage + (*FollowInMemoryMessagesResponse_CtrlMessage)(nil), // 49: messaging_pb.FollowInMemoryMessagesResponse.CtrlMessage } var file_mq_proto_depIdxs = []int32{ - 30, // 0: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry - 2, // 1: messaging_pb.TopicPartitionStats.topic:type_name -> messaging_pb.Topic - 3, // 2: messaging_pb.TopicPartitionStats.partition:type_name -> messaging_pb.Partition - 31, // 3: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage - 4, // 4: messaging_pb.PublisherToPubBalancerRequest.stats:type_name -> messaging_pb.BrokerStats - 2, // 5: messaging_pb.ConfigureTopicRequest.topic:type_name -> messaging_pb.Topic - 16, // 6: messaging_pb.ConfigureTopicResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 2, // 7: messaging_pb.ListTopicsResponse.topics:type_name -> messaging_pb.Topic - 2, // 8: messaging_pb.LookupTopicBrokersRequest.topic:type_name -> messaging_pb.Topic - 2, // 9: messaging_pb.LookupTopicBrokersResponse.topic:type_name -> messaging_pb.Topic - 16, // 10: messaging_pb.LookupTopicBrokersResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 3, // 11: messaging_pb.BrokerPartitionAssignment.partition:type_name -> messaging_pb.Partition - 2, // 12: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> messaging_pb.Topic - 16, // 13: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment - 32, // 14: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage - 33, // 15: messaging_pb.SubscriberToSubCoordinatorRequest.ack:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage - 35, // 16: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment - 36, // 17: messaging_pb.PublishRequest.init:type_name -> messaging_pb.PublishRequest.InitMessage - 21, // 18: messaging_pb.PublishRequest.data:type_name -> messaging_pb.DataMessage - 37, // 19: messaging_pb.SubscribeRequest.init:type_name -> messaging_pb.SubscribeRequest.InitMessage - 38, // 20: messaging_pb.SubscribeRequest.ack:type_name -> messaging_pb.SubscribeRequest.AckMessage - 39, // 21: messaging_pb.SubscribeResponse.ctrl:type_name -> messaging_pb.SubscribeResponse.CtrlMessage - 21, // 22: messaging_pb.SubscribeResponse.data:type_name -> messaging_pb.DataMessage - 2, // 23: messaging_pb.ClosePublishersRequest.topic:type_name -> messaging_pb.Topic - 2, // 24: messaging_pb.CloseSubscribersRequest.topic:type_name -> messaging_pb.Topic - 5, // 25: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats - 2, // 26: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 3, // 27: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage.partition:type_name -> messaging_pb.Partition - 3, // 28: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition.partition:type_name -> messaging_pb.Partition - 34, // 29: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.assigned_partitions:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition - 2, // 30: messaging_pb.PublishRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 3, // 31: messaging_pb.PublishRequest.InitMessage.partition:type_name -> messaging_pb.Partition - 2, // 32: messaging_pb.SubscribeRequest.InitMessage.topic:type_name -> messaging_pb.Topic - 3, // 33: messaging_pb.SubscribeRequest.InitMessage.partition:type_name -> messaging_pb.Partition - 0, // 34: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest - 6, // 35: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest - 8, // 36: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest - 12, // 37: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest - 10, // 38: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest - 14, // 39: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest - 17, // 40: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest - 26, // 41: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest - 28, // 42: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest - 19, // 43: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest - 22, // 44: messaging_pb.SeaweedMessaging.Publish:input_type -> messaging_pb.PublishRequest - 24, // 45: messaging_pb.SeaweedMessaging.Subscribe:input_type -> messaging_pb.SubscribeRequest - 1, // 46: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse - 7, // 47: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse - 9, // 48: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse - 13, // 49: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse - 11, // 50: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse - 15, // 51: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse - 18, // 52: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse - 27, // 53: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse - 29, // 54: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse - 20, // 55: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse - 23, // 56: messaging_pb.SeaweedMessaging.Publish:output_type -> messaging_pb.PublishResponse - 25, // 57: messaging_pb.SeaweedMessaging.Subscribe:output_type -> messaging_pb.SubscribeResponse - 46, // [46:58] is the sub-list for method output_type - 34, // [34:46] is the sub-list for method input_type - 34, // [34:34] is the sub-list for extension type_name - 34, // [34:34] is the sub-list for extension extendee - 0, // [0:34] is the sub-list for field type_name + 3, // 0: messaging_pb.Offset.topic:type_name -> messaging_pb.Topic + 6, // 1: messaging_pb.Offset.partition_offsets:type_name -> messaging_pb.PartitionOffset + 4, // 2: messaging_pb.PartitionOffset.partition:type_name -> messaging_pb.Partition + 0, // 3: messaging_pb.PartitionOffset.start_type:type_name -> messaging_pb.PartitionOffsetStartType + 37, // 4: messaging_pb.BrokerStats.stats:type_name -> messaging_pb.BrokerStats.StatsEntry + 3, // 5: messaging_pb.TopicPartitionStats.topic:type_name -> messaging_pb.Topic + 4, // 6: messaging_pb.TopicPartitionStats.partition:type_name -> messaging_pb.Partition + 38, // 7: messaging_pb.PublisherToPubBalancerRequest.init:type_name -> messaging_pb.PublisherToPubBalancerRequest.InitMessage + 7, // 8: messaging_pb.PublisherToPubBalancerRequest.stats:type_name -> messaging_pb.BrokerStats + 3, // 9: messaging_pb.ConfigureTopicRequest.topic:type_name -> messaging_pb.Topic + 19, // 10: messaging_pb.ConfigureTopicResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment + 3, // 11: messaging_pb.ListTopicsResponse.topics:type_name -> messaging_pb.Topic + 3, // 12: messaging_pb.LookupTopicBrokersRequest.topic:type_name -> messaging_pb.Topic + 3, // 13: messaging_pb.LookupTopicBrokersResponse.topic:type_name -> messaging_pb.Topic + 19, // 14: messaging_pb.LookupTopicBrokersResponse.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment + 4, // 15: messaging_pb.BrokerPartitionAssignment.partition:type_name -> messaging_pb.Partition + 3, // 16: messaging_pb.AssignTopicPartitionsRequest.topic:type_name -> messaging_pb.Topic + 19, // 17: messaging_pb.AssignTopicPartitionsRequest.broker_partition_assignments:type_name -> messaging_pb.BrokerPartitionAssignment + 39, // 18: messaging_pb.SubscriberToSubCoordinatorRequest.init:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage + 40, // 19: messaging_pb.SubscriberToSubCoordinatorRequest.ack:type_name -> messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage + 42, // 20: messaging_pb.SubscriberToSubCoordinatorResponse.assignment:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.Assignment + 43, // 21: messaging_pb.PublishMessageRequest.init:type_name -> messaging_pb.PublishMessageRequest.InitMessage + 24, // 22: messaging_pb.PublishMessageRequest.data:type_name -> messaging_pb.DataMessage + 3, // 23: messaging_pb.PublishFollowMeRequest.topic:type_name -> messaging_pb.Topic + 4, // 24: messaging_pb.PublishFollowMeRequest.partition:type_name -> messaging_pb.Partition + 44, // 25: messaging_pb.SubscribeMessageRequest.init:type_name -> messaging_pb.SubscribeMessageRequest.InitMessage + 45, // 26: messaging_pb.SubscribeMessageRequest.ack:type_name -> messaging_pb.SubscribeMessageRequest.AckMessage + 46, // 27: messaging_pb.SubscribeMessageResponse.ctrl:type_name -> messaging_pb.SubscribeMessageResponse.CtrlMessage + 24, // 28: messaging_pb.SubscribeMessageResponse.data:type_name -> messaging_pb.DataMessage + 47, // 29: messaging_pb.FollowInMemoryMessagesRequest.init:type_name -> messaging_pb.FollowInMemoryMessagesRequest.InitMessage + 48, // 30: messaging_pb.FollowInMemoryMessagesRequest.ack:type_name -> messaging_pb.FollowInMemoryMessagesRequest.AckMessage + 49, // 31: messaging_pb.FollowInMemoryMessagesResponse.ctrl:type_name -> messaging_pb.FollowInMemoryMessagesResponse.CtrlMessage + 24, // 32: messaging_pb.FollowInMemoryMessagesResponse.data:type_name -> messaging_pb.DataMessage + 3, // 33: messaging_pb.ClosePublishersRequest.topic:type_name -> messaging_pb.Topic + 3, // 34: messaging_pb.CloseSubscribersRequest.topic:type_name -> messaging_pb.Topic + 8, // 35: messaging_pb.BrokerStats.StatsEntry.value:type_name -> messaging_pb.TopicPartitionStats + 3, // 36: messaging_pb.SubscriberToSubCoordinatorRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 4, // 37: messaging_pb.SubscriberToSubCoordinatorRequest.AckMessage.partition:type_name -> messaging_pb.Partition + 4, // 38: messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition.partition:type_name -> messaging_pb.Partition + 41, // 39: messaging_pb.SubscriberToSubCoordinatorResponse.Assignment.assigned_partitions:type_name -> messaging_pb.SubscriberToSubCoordinatorResponse.AssignedPartition + 3, // 40: messaging_pb.PublishMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 4, // 41: messaging_pb.PublishMessageRequest.InitMessage.partition:type_name -> messaging_pb.Partition + 3, // 42: messaging_pb.SubscribeMessageRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 6, // 43: messaging_pb.SubscribeMessageRequest.InitMessage.partition_offset:type_name -> messaging_pb.PartitionOffset + 3, // 44: messaging_pb.FollowInMemoryMessagesRequest.InitMessage.topic:type_name -> messaging_pb.Topic + 6, // 45: messaging_pb.FollowInMemoryMessagesRequest.InitMessage.partition_offset:type_name -> messaging_pb.PartitionOffset + 1, // 46: messaging_pb.SeaweedMessaging.FindBrokerLeader:input_type -> messaging_pb.FindBrokerLeaderRequest + 9, // 47: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:input_type -> messaging_pb.PublisherToPubBalancerRequest + 11, // 48: messaging_pb.SeaweedMessaging.BalanceTopics:input_type -> messaging_pb.BalanceTopicsRequest + 15, // 49: messaging_pb.SeaweedMessaging.ListTopics:input_type -> messaging_pb.ListTopicsRequest + 13, // 50: messaging_pb.SeaweedMessaging.ConfigureTopic:input_type -> messaging_pb.ConfigureTopicRequest + 17, // 51: messaging_pb.SeaweedMessaging.LookupTopicBrokers:input_type -> messaging_pb.LookupTopicBrokersRequest + 20, // 52: messaging_pb.SeaweedMessaging.AssignTopicPartitions:input_type -> messaging_pb.AssignTopicPartitionsRequest + 33, // 53: messaging_pb.SeaweedMessaging.ClosePublishers:input_type -> messaging_pb.ClosePublishersRequest + 35, // 54: messaging_pb.SeaweedMessaging.CloseSubscribers:input_type -> messaging_pb.CloseSubscribersRequest + 22, // 55: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:input_type -> messaging_pb.SubscriberToSubCoordinatorRequest + 25, // 56: messaging_pb.SeaweedMessaging.PublishMessage:input_type -> messaging_pb.PublishMessageRequest + 29, // 57: messaging_pb.SeaweedMessaging.SubscribeMessage:input_type -> messaging_pb.SubscribeMessageRequest + 27, // 58: messaging_pb.SeaweedMessaging.PublishFollowMe:input_type -> messaging_pb.PublishFollowMeRequest + 31, // 59: messaging_pb.SeaweedMessaging.FollowInMemoryMessages:input_type -> messaging_pb.FollowInMemoryMessagesRequest + 2, // 60: messaging_pb.SeaweedMessaging.FindBrokerLeader:output_type -> messaging_pb.FindBrokerLeaderResponse + 10, // 61: messaging_pb.SeaweedMessaging.PublisherToPubBalancer:output_type -> messaging_pb.PublisherToPubBalancerResponse + 12, // 62: messaging_pb.SeaweedMessaging.BalanceTopics:output_type -> messaging_pb.BalanceTopicsResponse + 16, // 63: messaging_pb.SeaweedMessaging.ListTopics:output_type -> messaging_pb.ListTopicsResponse + 14, // 64: messaging_pb.SeaweedMessaging.ConfigureTopic:output_type -> messaging_pb.ConfigureTopicResponse + 18, // 65: messaging_pb.SeaweedMessaging.LookupTopicBrokers:output_type -> messaging_pb.LookupTopicBrokersResponse + 21, // 66: messaging_pb.SeaweedMessaging.AssignTopicPartitions:output_type -> messaging_pb.AssignTopicPartitionsResponse + 34, // 67: messaging_pb.SeaweedMessaging.ClosePublishers:output_type -> messaging_pb.ClosePublishersResponse + 36, // 68: messaging_pb.SeaweedMessaging.CloseSubscribers:output_type -> messaging_pb.CloseSubscribersResponse + 23, // 69: messaging_pb.SeaweedMessaging.SubscriberToSubCoordinator:output_type -> messaging_pb.SubscriberToSubCoordinatorResponse + 26, // 70: messaging_pb.SeaweedMessaging.PublishMessage:output_type -> messaging_pb.PublishMessageResponse + 30, // 71: messaging_pb.SeaweedMessaging.SubscribeMessage:output_type -> messaging_pb.SubscribeMessageResponse + 28, // 72: messaging_pb.SeaweedMessaging.PublishFollowMe:output_type -> messaging_pb.PublishFollowMeResponse + 32, // 73: messaging_pb.SeaweedMessaging.FollowInMemoryMessages:output_type -> messaging_pb.FollowInMemoryMessagesResponse + 60, // [60:74] is the sub-list for method output_type + 46, // [46:60] is the sub-list for method input_type + 46, // [46:46] is the sub-list for extension type_name + 46, // [46:46] is the sub-list for extension extendee + 0, // [0:46] is the sub-list for field type_name } func init() { file_mq_proto_init() } @@ -2869,7 +3594,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BrokerStats); i { + switch v := v.(*Offset); i { case 0: return &v.state case 1: @@ -2881,7 +3606,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TopicPartitionStats); i { + switch v := v.(*PartitionOffset); i { case 0: return &v.state case 1: @@ -2893,7 +3618,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PublisherToPubBalancerRequest); i { + switch v := v.(*BrokerStats); i { case 0: return &v.state case 1: @@ -2905,7 +3630,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PublisherToPubBalancerResponse); i { + switch v := v.(*TopicPartitionStats); i { case 0: return &v.state case 1: @@ -2917,7 +3642,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BalanceTopicsRequest); i { + switch v := v.(*PublisherToPubBalancerRequest); i { case 0: return &v.state case 1: @@ -2929,7 +3654,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BalanceTopicsResponse); i { + switch v := v.(*PublisherToPubBalancerResponse); i { case 0: return &v.state case 1: @@ -2941,7 +3666,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ConfigureTopicRequest); i { + switch v := v.(*BalanceTopicsRequest); i { case 0: return &v.state case 1: @@ -2953,7 +3678,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ConfigureTopicResponse); i { + switch v := v.(*BalanceTopicsResponse); i { case 0: return &v.state case 1: @@ -2965,7 +3690,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListTopicsRequest); i { + switch v := v.(*ConfigureTopicRequest); i { case 0: return &v.state case 1: @@ -2977,7 +3702,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListTopicsResponse); i { + switch v := v.(*ConfigureTopicResponse); i { case 0: return &v.state case 1: @@ -2989,7 +3714,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LookupTopicBrokersRequest); i { + switch v := v.(*ListTopicsRequest); i { case 0: return &v.state case 1: @@ -3001,7 +3726,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LookupTopicBrokersResponse); i { + switch v := v.(*ListTopicsResponse); i { case 0: return &v.state case 1: @@ -3013,7 +3738,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BrokerPartitionAssignment); i { + switch v := v.(*LookupTopicBrokersRequest); i { case 0: return &v.state case 1: @@ -3025,7 +3750,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AssignTopicPartitionsRequest); i { + switch v := v.(*LookupTopicBrokersResponse); i { case 0: return &v.state case 1: @@ -3037,7 +3762,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*AssignTopicPartitionsResponse); i { + switch v := v.(*BrokerPartitionAssignment); i { case 0: return &v.state case 1: @@ -3049,7 +3774,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscriberToSubCoordinatorRequest); i { + switch v := v.(*AssignTopicPartitionsRequest); i { case 0: return &v.state case 1: @@ -3061,7 +3786,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscriberToSubCoordinatorResponse); i { + switch v := v.(*AssignTopicPartitionsResponse); i { case 0: return &v.state case 1: @@ -3073,7 +3798,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DataMessage); i { + switch v := v.(*SubscriberToSubCoordinatorRequest); i { case 0: return &v.state case 1: @@ -3085,7 +3810,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PublishRequest); i { + switch v := v.(*SubscriberToSubCoordinatorResponse); i { case 0: return &v.state case 1: @@ -3097,7 +3822,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PublishResponse); i { + switch v := v.(*DataMessage); i { case 0: return &v.state case 1: @@ -3109,7 +3834,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscribeRequest); i { + switch v := v.(*PublishMessageRequest); i { case 0: return &v.state case 1: @@ -3121,7 +3846,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscribeResponse); i { + switch v := v.(*PublishMessageResponse); i { case 0: return &v.state case 1: @@ -3133,7 +3858,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ClosePublishersRequest); i { + switch v := v.(*PublishFollowMeRequest); i { case 0: return &v.state case 1: @@ -3145,7 +3870,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ClosePublishersResponse); i { + switch v := v.(*PublishFollowMeResponse); i { case 0: return &v.state case 1: @@ -3157,7 +3882,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseSubscribersRequest); i { + switch v := v.(*SubscribeMessageRequest); i { case 0: return &v.state case 1: @@ -3169,7 +3894,19 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CloseSubscribersResponse); i { + switch v := v.(*SubscribeMessageResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FollowInMemoryMessagesRequest); i { case 0: return &v.state case 1: @@ -3181,7 +3918,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PublisherToPubBalancerRequest_InitMessage); i { + switch v := v.(*FollowInMemoryMessagesResponse); i { case 0: return &v.state case 1: @@ -3193,7 +3930,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscriberToSubCoordinatorRequest_InitMessage); i { + switch v := v.(*ClosePublishersRequest); i { case 0: return &v.state case 1: @@ -3205,7 +3942,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscriberToSubCoordinatorRequest_AckMessage); i { + switch v := v.(*ClosePublishersResponse); i { case 0: return &v.state case 1: @@ -3217,7 +3954,7 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscriberToSubCoordinatorResponse_AssignedPartition); i { + switch v := v.(*CloseSubscribersRequest); i { case 0: return &v.state case 1: @@ -3229,6 +3966,66 @@ func file_mq_proto_init() { } } file_mq_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*CloseSubscribersResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PublisherToPubBalancerRequest_InitMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscriberToSubCoordinatorRequest_InitMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscriberToSubCoordinatorRequest_AckMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscriberToSubCoordinatorResponse_AssignedPartition); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SubscriberToSubCoordinatorResponse_Assignment); i { case 0: return &v.state @@ -3240,8 +4037,8 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PublishRequest_InitMessage); i { + file_mq_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PublishMessageRequest_InitMessage); i { case 0: return &v.state case 1: @@ -3252,8 +4049,8 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscribeRequest_InitMessage); i { + file_mq_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscribeMessageRequest_InitMessage); i { case 0: return &v.state case 1: @@ -3264,8 +4061,8 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscribeRequest_AckMessage); i { + file_mq_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscribeMessageRequest_AckMessage); i { case 0: return &v.state case 1: @@ -3276,8 +4073,44 @@ func file_mq_proto_init() { return nil } } - file_mq_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SubscribeResponse_CtrlMessage); i { + file_mq_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SubscribeMessageResponse_CtrlMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FollowInMemoryMessagesRequest_InitMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FollowInMemoryMessagesRequest_AckMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_mq_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*FollowInMemoryMessagesResponse_CtrlMessage); i { case 0: return &v.state case 1: @@ -3289,45 +4122,50 @@ func file_mq_proto_init() { } } } - file_mq_proto_msgTypes[6].OneofWrappers = []interface{}{ + file_mq_proto_msgTypes[8].OneofWrappers = []interface{}{ (*PublisherToPubBalancerRequest_Init)(nil), (*PublisherToPubBalancerRequest_Stats)(nil), } - file_mq_proto_msgTypes[19].OneofWrappers = []interface{}{ + file_mq_proto_msgTypes[21].OneofWrappers = []interface{}{ (*SubscriberToSubCoordinatorRequest_Init)(nil), (*SubscriberToSubCoordinatorRequest_Ack)(nil), } - file_mq_proto_msgTypes[20].OneofWrappers = []interface{}{ - (*SubscriberToSubCoordinatorResponse_Assignment_)(nil), - } file_mq_proto_msgTypes[22].OneofWrappers = []interface{}{ - (*PublishRequest_Init)(nil), - (*PublishRequest_Data)(nil), + (*SubscriberToSubCoordinatorResponse_Assignment_)(nil), } file_mq_proto_msgTypes[24].OneofWrappers = []interface{}{ - (*SubscribeRequest_Init)(nil), - (*SubscribeRequest_Ack)(nil), + (*PublishMessageRequest_Init)(nil), + (*PublishMessageRequest_Data)(nil), + } + file_mq_proto_msgTypes[28].OneofWrappers = []interface{}{ + (*SubscribeMessageRequest_Init)(nil), + (*SubscribeMessageRequest_Ack)(nil), + } + file_mq_proto_msgTypes[29].OneofWrappers = []interface{}{ + (*SubscribeMessageResponse_Ctrl)(nil), + (*SubscribeMessageResponse_Data)(nil), } - file_mq_proto_msgTypes[25].OneofWrappers = []interface{}{ - (*SubscribeResponse_Ctrl)(nil), - (*SubscribeResponse_Data)(nil), + file_mq_proto_msgTypes[30].OneofWrappers = []interface{}{ + (*FollowInMemoryMessagesRequest_Init)(nil), + (*FollowInMemoryMessagesRequest_Ack)(nil), } - file_mq_proto_msgTypes[37].OneofWrappers = []interface{}{ - (*SubscribeRequest_InitMessage_StartOffset)(nil), - (*SubscribeRequest_InitMessage_StartTimestampNs)(nil), + file_mq_proto_msgTypes[31].OneofWrappers = []interface{}{ + (*FollowInMemoryMessagesResponse_Ctrl)(nil), + (*FollowInMemoryMessagesResponse_Data)(nil), } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_mq_proto_rawDesc, - NumEnums: 0, - NumMessages: 40, + NumEnums: 1, + NumMessages: 49, NumExtensions: 0, NumServices: 1, }, GoTypes: file_mq_proto_goTypes, DependencyIndexes: file_mq_proto_depIdxs, + EnumInfos: file_mq_proto_enumTypes, MessageInfos: file_mq_proto_msgTypes, }.Build() File_mq_proto = out.File diff --git a/weed/pb/mq_pb/mq_grpc.pb.go b/weed/pb/mq_pb/mq_grpc.pb.go index b8e1387ba..64518d605 100644 --- a/weed/pb/mq_pb/mq_grpc.pb.go +++ b/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, }, }, diff --git a/weed/pb/remote_pb/remote.pb.go b/weed/pb/remote_pb/remote.pb.go index d0d34d474..870db5ad2 100644 --- a/weed/pb/remote_pb/remote.pb.go +++ b/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 diff --git a/weed/pb/s3_pb/s3.pb.go b/weed/pb/s3_pb/s3.pb.go index 55ca52875..c9fca3d2b 100644 --- a/weed/pb/s3_pb/s3.pb.go +++ b/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 diff --git a/weed/pb/s3_pb/s3_grpc.pb.go b/weed/pb/s3_pb/s3_grpc.pb.go index 9acc87da4..2fedf571b 100644 --- a/weed/pb/s3_pb/s3_grpc.pb.go +++ b/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 diff --git a/weed/pb/volume_server_pb/volume_server.pb.go b/weed/pb/volume_server_pb/volume_server.pb.go index bda2974ff..025e40610 100644 --- a/weed/pb/volume_server_pb/volume_server.pb.go +++ b/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 diff --git a/weed/pb/volume_server_pb/volume_server_grpc.pb.go b/weed/pb/volume_server_pb/volume_server_grpc.pb.go index f1dc45fa4..940adf339 100644 --- a/weed/pb/volume_server_pb/volume_server_grpc.pb.go +++ b/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 diff --git a/weed/server/filer_grpc_server.go b/weed/server/filer_grpc_server.go index f32273f26..eeb031cd1 100644 --- a/weed/server/filer_grpc_server.go +++ b/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) diff --git a/weed/server/filer_grpc_server_dlm.go b/weed/server/filer_grpc_server_dlm.go index 1f5644680..5ec147835 100644 --- a/weed/server/filer_grpc_server_dlm.go +++ b/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 diff --git a/weed/server/filer_grpc_server_sub_meta.go b/weed/server/filer_grpc_server_sub_meta.go index eb69a6aeb..436c4158f 100644 --- a/weed/server/filer_grpc_server_sub_meta.go +++ b/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 } } diff --git a/weed/server/filer_server.go b/weed/server/filer_server.go index 9428c2edf..356761f30 100644 --- a/weed/server/filer_server.go +++ b/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 diff --git a/weed/shell/command_mq_topic_desc.go b/weed/shell/command_mq_topic_desc.go index a4bf805f9..cedad6ed4 100644 --- a/weed/shell/command_mq_topic_desc.go +++ b/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 diff --git a/weed/shell/command_mq_topic_list.go b/weed/shell/command_mq_topic_list.go index cf6b97e96..0a193cb4e 100644 --- a/weed/shell/command_mq_topic_list.go +++ b/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 diff --git a/weed/util/buffered_queue/buffered_queue.go b/weed/util/buffered_queue/buffered_queue.go new file mode 100644 index 000000000..edaa0a7ce --- /dev/null +++ b/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() +} diff --git a/weed/util/buffered_queue/buffered_queue_test.go b/weed/util/buffered_queue/buffered_queue_test.go new file mode 100644 index 000000000..97c9f25a7 --- /dev/null +++ b/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() + } + +} diff --git a/weed/util/http_util.go b/weed/util/http_util.go index ef4b29158..d1505f673 100644 --- a/weed/util/http_util.go +++ b/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 + +} diff --git a/weed/util/log_buffer/log_buffer.go b/weed/util/log_buffer/log_buffer.go index bd124908e..4d393d88b 100644 --- a/weed/util/log_buffer/log_buffer.go +++ b/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) } diff --git a/weed/util/log_buffer/log_buffer_test.go b/weed/util/log_buffer/log_buffer_test.go index 9ecb90762..ac46a096c 100644 --- a/weed/util/log_buffer/log_buffer_test.go +++ b/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() diff --git a/weed/util/log_buffer/log_read.go b/weed/util/log_buffer/log_read.go index 059f74286..0354f0e7f 100644 --- a/weed/util/log_buffer/log_read.go +++ b/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) } } diff --git a/weed/util/log_buffer/sealed_buffer.go b/weed/util/log_buffer/sealed_buffer.go index d133cf8d3..c41b30fcc 100644 --- a/weed/util/log_buffer/sealed_buffer.go +++ b/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 }