From 41af6b1b467a98df1d24a1bb65eb3be5a57332bc Mon Sep 17 00:00:00 2001 From: wenfeng Date: Thu, 15 Dec 2022 17:07:56 +0800 Subject: [PATCH 1/4] feat: abstract cluster operation Signed-off-by: wenfeng --- client/client.go | 2 +- .../internal/vanus/eventbus/name_service.go | 5 +- .../internal/vanus/eventlog/name_service.go | 4 +- client/mock_client.go | 4 +- cmd/controller/main.go | 2 +- go.mod | 2 +- go.sum | 4 +- internal/controller/eventbus/controller.go | 55 +- .../controller/eventbus/controller_test.go | 10 +- .../controller/eventbus/server/manager.go | 24 + .../eventbus/server/mock_manager.go | 14 + internal/controller/trigger/controller.go | 36 +- internal/gateway/proxy/proxy.go | 12 +- internal/primitive/constants.go | 7 +- internal/primitive/vanus/id.go | 5 +- internal/store/segment/server.go | 11 +- internal/timer/timingwheel/bucket.go | 60 +- internal/timer/timingwheel/bucket_test.go | 77 +- internal/timer/timingwheel/timingwheel.go | 57 +- .../timer/timingwheel/timingwheel_test.go | 73 +- internal/trigger/worker.go | 9 +- observability/metrics/controller.go | 2 +- pkg/cluster/controller.go | 171 ++++ pkg/cluster/eventbus.go | 58 ++ pkg/cluster/eventlog.go | 18 + pkg/cluster/id.go | 18 + pkg/cluster/mock_controller.go | 404 ++++++++ .../raw_client}/conn.go | 20 +- .../raw_client}/conn_test.go | 2 +- .../raw_client}/eventbus.go | 18 +- .../raw_client}/eventlog.go | 9 +- .../raw_client}/heartbeat.go | 2 +- pkg/cluster/raw_client/ping.go | 45 + .../raw_client}/segment.go | 9 +- .../raw_client}/segment_test.go | 2 +- .../raw_client}/snowflake.go | 9 +- .../raw_client}/trigger.go | 9 +- pkg/cluster/segment.go | 24 + pkg/cluster/trigger.go | 24 + proto/pkg/controller/controller.pb.go | 945 +++++++++--------- proto/pkg/controller/mock_controller.go | 188 ++++ proto/pkg/proxy/proxy.pb.go | 4 +- proto/proto/controller.proto | 3 + vsctl/command/eventbus.go | 14 +- 44 files changed, 1766 insertions(+), 705 deletions(-) create mode 100644 pkg/cluster/controller.go create mode 100644 pkg/cluster/eventbus.go create mode 100644 pkg/cluster/eventlog.go create mode 100644 pkg/cluster/id.go create mode 100644 pkg/cluster/mock_controller.go rename pkg/{controller => cluster/raw_client}/conn.go (91%) rename pkg/{controller => cluster/raw_client}/conn_test.go (91%) rename pkg/{controller => cluster/raw_client}/eventbus.go (85%) rename pkg/{controller => cluster/raw_client}/eventlog.go (87%) rename pkg/{controller => cluster/raw_client}/heartbeat.go (98%) create mode 100644 pkg/cluster/raw_client/ping.go rename pkg/{controller => cluster/raw_client}/segment.go (95%) rename pkg/{controller => cluster/raw_client}/segment_test.go (98%) rename pkg/{controller => cluster/raw_client}/snowflake.go (89%) rename pkg/{controller => cluster/raw_client}/trigger.go (96%) create mode 100644 pkg/cluster/segment.go create mode 100644 pkg/cluster/trigger.go diff --git a/client/client.go b/client/client.go index b4ac97f72..2f7fbb0a6 100644 --- a/client/client.go +++ b/client/client.go @@ -40,7 +40,7 @@ type client struct { } func (c *client) Eventbus(ctx context.Context, ebName string) api.Eventbus { - _, span := c.tracer.Start(ctx, "Eventbus") + _, span := c.tracer.Start(ctx, "EventbusService") defer span.End() bus := func() api.Eventbus { diff --git a/client/internal/vanus/eventbus/name_service.go b/client/internal/vanus/eventbus/name_service.go index 2a2d1ec19..c46fdf5ca 100644 --- a/client/internal/vanus/eventbus/name_service.go +++ b/client/internal/vanus/eventbus/name_service.go @@ -17,12 +17,11 @@ package eventbus import ( // standard libraries "context" + "github.com/linkall-labs/vanus/pkg/cluster" "github.com/linkall-labs/vanus/observability/tracing" "go.opentelemetry.io/otel/trace" - // third-party libraries - "github.com/linkall-labs/vanus/pkg/controller" "google.golang.org/grpc/credentials/insecure" // first-party libraries @@ -33,7 +32,7 @@ import ( func NewNameService(endpoints []string) *NameService { return &NameService{ - client: controller.NewEventbusClient(endpoints, insecure.NewCredentials()), + client: cluster.NewClusterController(endpoints, insecure.NewCredentials()).EventbusService().RawClient(), tracer: tracing.NewTracer("internal.discovery.eventbus", trace.SpanKindClient), } } diff --git a/client/internal/vanus/eventlog/name_service.go b/client/internal/vanus/eventlog/name_service.go index c63bf6362..1fe325ba2 100644 --- a/client/internal/vanus/eventlog/name_service.go +++ b/client/internal/vanus/eventlog/name_service.go @@ -17,6 +17,7 @@ package eventlog import ( // standard libraries. "context" + "github.com/linkall-labs/vanus/pkg/cluster" "math" "time" @@ -27,7 +28,6 @@ import ( // first-party libraries. "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/observability/tracing" - "github.com/linkall-labs/vanus/pkg/controller" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" metapb "github.com/linkall-labs/vanus/proto/pkg/meta" @@ -38,7 +38,7 @@ import ( func NewNameService(endpoints []string) *NameService { return &NameService{ - client: controller.NewEventlogClient(endpoints, insecure.NewCredentials()), + client: cluster.NewClusterController(endpoints, insecure.NewCredentials()).EventlogService().RawClient(), tracer: tracing.NewTracer("internal.discovery.eventlog", trace.SpanKindClient), } } diff --git a/client/mock_client.go b/client/mock_client.go index 13fc50129..9d9de28a5 100644 --- a/client/mock_client.go +++ b/client/mock_client.go @@ -50,7 +50,7 @@ func (mr *MockClientMockRecorder) Disconnect(ctx interface{}) *gomock.Call { // Eventbus mocks base method. func (m *MockClient) Eventbus(ctx context.Context, ebName string) api.Eventbus { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Eventbus", ctx, ebName) + ret := m.ctrl.Call(m, "EventbusService", ctx, ebName) ret0, _ := ret[0].(api.Eventbus) return ret0 } @@ -58,5 +58,5 @@ func (m *MockClient) Eventbus(ctx context.Context, ebName string) api.Eventbus { // Eventbus indicates an expected call of Eventbus. func (mr *MockClientMockRecorder) Eventbus(ctx, ebName interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Eventbus", reflect.TypeOf((*MockClient)(nil).Eventbus), ctx, ebName) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EventbusService", reflect.TypeOf((*MockClient)(nil).Eventbus), ctx, ebName) } diff --git a/cmd/controller/main.go b/cmd/controller/main.go index e4a2ff8d6..154bcc407 100644 --- a/cmd/controller/main.go +++ b/cmd/controller/main.go @@ -87,7 +87,7 @@ func main() { segmentCtrl := eventbus.NewController(cfg.GetEventbusCtrlConfig(), etcd) if err = segmentCtrl.Start(ctx); err != nil { - log.Error(ctx, "start Eventbus Controller failed", map[string]interface{}{ + log.Error(ctx, "start EventbusService Controller failed", map[string]interface{}{ log.KeyError: err, }) os.Exit(-1) diff --git a/go.mod b/go.mod index 9e8494a25..fefb22192 100644 --- a/go.mod +++ b/go.mod @@ -23,7 +23,7 @@ require ( github.com/iceber/iouring-go v0.0.0-20220609112130-b1dc8dd9fbfd github.com/jedib0t/go-pretty/v6 v6.3.1 github.com/json-iterator/go v1.1.12 - github.com/linkall-labs/embed-etcd v0.1.1 + github.com/linkall-labs/embed-etcd v0.1.2 github.com/linkall-labs/vanus/client v0.5.1 github.com/linkall-labs/vanus/observability v0.5.1 github.com/linkall-labs/vanus/pkg v0.5.1 diff --git a/go.sum b/go.sum index c75465d22..a1018795f 100644 --- a/go.sum +++ b/go.sum @@ -292,8 +292,8 @@ github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= -github.com/linkall-labs/embed-etcd v0.1.1 h1:WxV9wbnRtNf7DMW8SJauVYqhFLXzRfY5wpplFypXK9k= -github.com/linkall-labs/embed-etcd v0.1.1/go.mod h1:dmleSy0Myllw6W5awwjyDMipgICVDHTHuTcRT4cqaIc= +github.com/linkall-labs/embed-etcd v0.1.2 h1:1mTdXLwVvn9gi3XWh/PGhaEAfG8Zmxvjqwnfontb+fA= +github.com/linkall-labs/embed-etcd v0.1.2/go.mod h1:QnecHaKt3WQBO9YGBckCDUTBd44VBR2VO8220BtWZ5U= github.com/mattn/go-colorable v0.1.9/go.mod h1:u6P/XSegPjTcexA+o6vUJrdnUu04hMope9wVRipJSqc= github.com/mattn/go-colorable v0.1.11 h1:nQ+aFkoE2TMGc0b68U2OKSexC+eq46+XwZzWXHRmPYs= github.com/mattn/go-colorable v0.1.11/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= diff --git a/internal/controller/eventbus/controller.go b/internal/controller/eventbus/controller.go index 7c7ef8466..04a83232b 100644 --- a/internal/controller/eventbus/controller.go +++ b/internal/controller/eventbus/controller.go @@ -32,6 +32,7 @@ import ( "github.com/linkall-labs/vanus/internal/controller/eventbus/volume" "github.com/linkall-labs/vanus/internal/kv" "github.com/linkall-labs/vanus/internal/kv/etcd" + "github.com/linkall-labs/vanus/internal/primitive" "github.com/linkall-labs/vanus/internal/primitive/vanus" "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/observability/metrics" @@ -94,7 +95,6 @@ func (ctrl *controller) Start(_ context.Context) error { return err } ctrl.kvStore = store - ctrl.cancelCtx, ctrl.cancelFunc = context.WithCancel(context.Background()) go ctrl.member.RegisterMembershipChangedProcessor(ctrl.membershipChangedProcessor) return nil @@ -113,9 +113,47 @@ func (ctrl *controller) StopNotify() <-chan error { } func (ctrl *controller) CreateEventBus(ctx context.Context, + req *ctrlpb.CreateEventBusRequest) (*metapb.EventBus, error) { + if err := isValidEventbusName(req.Name); err != nil { + return nil, err + } + return ctrl.createEventBus(ctx, req) +} + +func isValidEventbusName(name string) error { + for _, v := range name { + if v == '.' || v == '_' || v == '-' { + continue + } + c := v - 'a' + if c >= 0 || c <= 26 { + continue + } else { + c = v - '0' + if c >= 0 || c <= 9 { + continue + } + return errors.ErrInvalidRequest.WithMessage("eventbus name must be insist of 0-9a-zA-Z.-_") + } + } + return nil +} + +func (ctrl *controller) CreateSystemEventBus(ctx context.Context, + req *ctrlpb.CreateEventBusRequest) (*metapb.EventBus, error) { + if !strings.HasPrefix(req.Name, primitive.SystemEventbusNamePrefix) { + return nil, errors.ErrInvalidRequest.WithMessage("system eventbus must start with __") + } + return ctrl.createEventBus(ctx, req) +} + +func (ctrl *controller) createEventBus(ctx context.Context, req *ctrlpb.CreateEventBusRequest) (*metapb.EventBus, error) { ctrl.mutex.Lock() defer ctrl.mutex.Unlock() + if !ctrl.isReady(ctx) { + return nil, errors.ErrResourceCanNotOp.WithMessage("the cluster isn't ready to create eventbus") + } logNum := req.LogNumber if logNum == 0 { logNum = 1 @@ -451,12 +489,23 @@ func (ctrl *controller) ReportSegmentBlockIsFull(ctx context.Context, return &emptypb.Empty{}, nil } -func (ctrl *controller) Ping(_ context.Context, _ *emptypb.Empty) (*ctrlpb.PingResponse, error) { +func (ctrl *controller) Ping(ctx context.Context, _ *emptypb.Empty) (*ctrlpb.PingResponse, error) { return &ctrlpb.PingResponse{ - LeaderAddr: ctrl.member.GetLeaderAddr(), + LeaderAddr: ctrl.member.GetLeaderAddr(), + IsEventbusReady: ctrl.isReady(ctx), }, nil } +func (ctrl *controller) isReady(ctx context.Context) bool { + if ctrl.member == nil { + return false + } + if !ctrl.member.IsLeader() && !ctrl.member.IsReady() || ctrl.member.GetLeaderAddr() == "" { + return false + } + return ctrl.ssMgr.CanCreateEventbus(ctx, int(ctrl.cfg.Replicas)) +} + func (ctrl *controller) ReportSegmentLeader(ctx context.Context, req *ctrlpb.ReportSegmentLeaderRequest) (*emptypb.Empty, error) { err := ctrl.eventLogMgr.UpdateSegmentReplicas(ctx, vanus.NewIDFromUint64(req.LeaderId), req.Term) diff --git a/internal/controller/eventbus/controller_test.go b/internal/controller/eventbus/controller_test.go index 959464fd5..7de6c2cbb 100644 --- a/internal/controller/eventbus/controller_test.go +++ b/internal/controller/eventbus/controller_test.go @@ -20,6 +20,8 @@ import ( "sort" "testing" + "github.com/golang/mock/gomock" + embedetcd "github.com/linkall-labs/embed-etcd" "github.com/linkall-labs/vanus/internal/controller/eventbus/eventlog" "github.com/linkall-labs/vanus/internal/controller/eventbus/metadata" "github.com/linkall-labs/vanus/internal/kv" @@ -27,8 +29,6 @@ import ( "github.com/linkall-labs/vanus/pkg/errors" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" metapb "github.com/linkall-labs/vanus/proto/pkg/meta" - - "github.com/golang/mock/gomock" . "github.com/smartystreets/goconvey/convey" ) @@ -44,6 +44,12 @@ func TestController_CreateEventBus(t *testing.T) { ctrl.eventLogMgr = elMgr ctx := stdCtx.Background() + mockMember := embedetcd.NewMockMember(mockCtrl) + ctrl.member = mockMember + mockMember.EXPECT().IsLeader().AnyTimes().Return(true) + mockMember.EXPECT().IsReady().AnyTimes().Return(true) + mockMember.EXPECT().GetLeaderAddr().AnyTimes().Return("test") + Convey("test create a eventbus two times", func() { kvCli.EXPECT().Exists(ctx, metadata.GetEventbusMetadataKey("test-1")).Times(1).Return(false, nil) kvCli.EXPECT().Set(ctx, metadata.GetEventbusMetadataKey("test-1"), gomock.Any()). diff --git a/internal/controller/eventbus/server/manager.go b/internal/controller/eventbus/server/manager.go index 7f4873a75..f7a3f3f9a 100644 --- a/internal/controller/eventbus/server/manager.go +++ b/internal/controller/eventbus/server/manager.go @@ -12,11 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. +//go:generate mockgen -source=manager.go -destination=mock_manager.go -package=server package server import ( "context" "sync" + "sync/atomic" "time" "github.com/golang/protobuf/ptypes/empty" @@ -36,6 +38,7 @@ type Manager interface { GetServerByServerID(id vanus.ID) Server Run(ctx context.Context) error Stop(ctx context.Context) + CanCreateEventbus(ctx context.Context, replicaNum int) bool } const ( @@ -59,6 +62,7 @@ type segmentServerManager struct { cancelCtx context.Context cancel func() ticker *time.Ticker + onlineServerNumber int64 } func (mgr *segmentServerManager) AddServer(ctx context.Context, srv Server) error { @@ -78,9 +82,11 @@ func (mgr *segmentServerManager) AddServer(ctx context.Context, srv Server) erro } mgr.segmentServerMapByIP.Store(srv.Address(), srv) mgr.segmentServerMapByID.Store(srv.ID().Key(), srv) + atomic.AddInt64(&mgr.onlineServerNumber, 1) log.Info(ctx, "the segment server added", map[string]interface{}{ "server_id": srv.ID(), "addr": srv.Address(), + "online": atomic.LoadInt64(&mgr.onlineServerNumber), }) return nil } @@ -93,6 +99,12 @@ func (mgr *segmentServerManager) RemoveServer(ctx context.Context, srv Server) e defer mgr.mutex.Unlock() mgr.segmentServerMapByIP.Delete(srv.Address()) mgr.segmentServerMapByID.Delete(srv.ID().Key()) + atomic.AddInt64(&mgr.onlineServerNumber, -1) + log.Info(ctx, "the segment server was removed", map[string]interface{}{ + "server_id": srv.ID(), + "addr": srv.Address(), + "online": atomic.LoadInt64(&mgr.onlineServerNumber), + }) return nil } @@ -171,6 +183,18 @@ func (mgr *segmentServerManager) Stop(ctx context.Context) { }) } +func (mgr *segmentServerManager) CanCreateEventbus(ctx context.Context, replicaNum int) bool { + activeNum := 0 + mgr.segmentServerMapByID.Range(func(_, value any) bool { + s, _ := value.(Server) + if s.IsActive(ctx) { + activeNum++ + } + return true + }) + return activeNum >= replicaNum +} + type Server interface { RemoteStart(ctx context.Context) error RemoteStop(ctx context.Context) diff --git a/internal/controller/eventbus/server/mock_manager.go b/internal/controller/eventbus/server/mock_manager.go index 16610a989..dc2e7c332 100644 --- a/internal/controller/eventbus/server/mock_manager.go +++ b/internal/controller/eventbus/server/mock_manager.go @@ -51,6 +51,20 @@ func (mr *MockManagerMockRecorder) AddServer(ctx, srv interface{}) *gomock.Call return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddServer", reflect.TypeOf((*MockManager)(nil).AddServer), ctx, srv) } +// CanCreateEventbus mocks base method. +func (m *MockManager) CanCreateEventbus(ctx context.Context, replicaNum int) bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CanCreateEventbus", ctx, replicaNum) + ret0, _ := ret[0].(bool) + return ret0 +} + +// CanCreateEventbus indicates an expected call of CanCreateEventbus. +func (mr *MockManagerMockRecorder) CanCreateEventbus(ctx, replicaNum interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CanCreateEventbus", reflect.TypeOf((*MockManager)(nil).CanCreateEventbus), ctx, replicaNum) +} + // GetServerByAddress mocks base method. func (m *MockManager) GetServerByAddress(addr string) Server { m.ctrl.T.Helper() diff --git a/internal/controller/trigger/controller.go b/internal/controller/trigger/controller.go index 66f4d79cf..f1b134240 100644 --- a/internal/controller/trigger/controller.go +++ b/internal/controller/trigger/controller.go @@ -18,6 +18,7 @@ import ( "context" stdErr "errors" "io" + "os" "sync" "time" @@ -33,6 +34,7 @@ import ( "github.com/linkall-labs/vanus/internal/primitive/vanus" "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/observability/metrics" + "github.com/linkall-labs/vanus/pkg/cluster" "github.com/linkall-labs/vanus/pkg/errors" "github.com/linkall-labs/vanus/pkg/util" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" @@ -46,6 +48,7 @@ var ( const ( defaultGcSubscriptionInterval = time.Second * 10 + defaultSystemEventbusEventlog = 1 ) func NewController(config Config, member embedetcd.Member) *controller { @@ -74,6 +77,7 @@ type controller struct { ctx context.Context stopFunc context.CancelFunc state primitive.ServerState + cl cluster.Cluster } func (ctrl *controller) CommitOffset(ctx context.Context, @@ -401,6 +405,7 @@ func (ctrl *controller) requeueSubscription(ctx context.Context, id vanus.ID, ad } func (ctrl *controller) init(ctx context.Context) error { + ctrl.initTriggerSystemEventbus() err := ctrl.subscriptionManager.Init(ctx) if err != nil { return err @@ -464,7 +469,7 @@ func (ctrl *controller) membershipChangedProcessor(ctx context.Context, return nil } -func (ctrl *controller) stop(ctx context.Context) error { +func (ctrl *controller) stop(_ context.Context) error { ctrl.member.ResignIfLeader() ctrl.state = primitive.ServerStateStopping ctrl.stopFunc() @@ -503,3 +508,32 @@ func (ctrl *controller) Stop(ctx context.Context) { }) } } + +func (ctrl *controller) initTriggerSystemEventbus() { + // avoid blocking starting + go func() { + ctx := context.Background() + log.Info(ctx, "trigger controller is ready to check system eventbus", nil) + if err := ctrl.cl.WaitForControllerReady(true); err != nil { + log.Error(context.Background(), "trigger controller try to create system eventbus, "+ + "but Vanus cluster hasn't ready, exit", nil) + os.Exit(-1) + } + + if err := ctrl.cl.EventbusService().CreateSystemEventbusIfNotExist(ctx, primitive.RetryEventbusName, + defaultSystemEventbusEventlog, "System Eventbus For Trigger Service"); err != nil { + log.Error(context.Background(), "failed to create RetryEventbus, exit", map[string]interface{}{ + log.KeyError: err, + }) + os.Exit(-1) + } + + if err := ctrl.cl.EventbusService().CreateSystemEventbusIfNotExist(ctx, primitive.DeadLetterEventbusName, + defaultSystemEventbusEventlog, "System Eventbus For Trigger Service"); err != nil { + log.Error(context.Background(), "failed to create DeadLetterEventbus, exit", map[string]interface{}{ + log.KeyError: err, + }) + } + log.Info(ctx, "trigger controller has finished for checking system eventbus", nil) + }() +} diff --git a/internal/gateway/proxy/proxy.go b/internal/gateway/proxy/proxy.go index ac02cb312..fffdf18c5 100644 --- a/internal/gateway/proxy/proxy.go +++ b/internal/gateway/proxy/proxy.go @@ -33,7 +33,7 @@ import ( "github.com/linkall-labs/vanus/internal/primitive/vanus" "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/observability/tracing" - "github.com/linkall-labs/vanus/pkg/controller" + "github.com/linkall-labs/vanus/pkg/cluster" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" proxypb "github.com/linkall-labs/vanus/proto/pkg/proxy" "go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc" @@ -41,6 +41,7 @@ import ( "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials" + "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/reflection" "google.golang.org/grpc/status" "google.golang.org/protobuf/types/known/emptypb" @@ -71,16 +72,19 @@ type ControllerProxy struct { eventlogCtrl ctrlpb.EventLogControllerClient triggerCtrl ctrlpb.TriggerControllerClient grpcSrv *grpc.Server + ctrl cluster.Cluster } func NewControllerProxy(cfg Config) *ControllerProxy { + ctrl := cluster.NewClusterController(cfg.Endpoints, insecure.NewCredentials()) return &ControllerProxy{ cfg: cfg, + ctrl: ctrl, client: eb.Connect(cfg.Endpoints), tracer: tracing.NewTracer("controller-proxy", trace.SpanKindServer), - eventbusCtrl: controller.NewEventbusClient(cfg.Endpoints, cfg.Credentials), - eventlogCtrl: controller.NewEventlogClient(cfg.Endpoints, cfg.Credentials), - triggerCtrl: controller.NewTriggerClient(cfg.Endpoints, cfg.Credentials), + eventbusCtrl: ctrl.EventbusService().RawClient(), + eventlogCtrl: ctrl.EventlogService().RawClient(), + triggerCtrl: ctrl.TriggerService().RawClient(), } } diff --git a/internal/primitive/constants.go b/internal/primitive/constants.go index b75133ffc..dfcbe1a38 100644 --- a/internal/primitive/constants.go +++ b/internal/primitive/constants.go @@ -15,9 +15,10 @@ package primitive const ( - RetryEventbusName = "__retry_eb" - DeadLetterEventbusName = "__dl_eb" - TimerEventbusName = "__Timer_RS" + SystemEventbusNamePrefix = "__" + RetryEventbusName = "__retry_eb" + DeadLetterEventbusName = "__dl_eb" + TimerEventbusName = "__Timer_RS" XVanus = "xvanus" XVanusEventbus = XVanus + "eventbus" diff --git a/internal/primitive/vanus/id.go b/internal/primitive/vanus/id.go index 2e96e2e49..78a679006 100644 --- a/internal/primitive/vanus/id.go +++ b/internal/primitive/vanus/id.go @@ -24,7 +24,7 @@ import ( "github.com/golang/protobuf/ptypes/empty" "github.com/linkall-labs/vanus/observability/log" - "github.com/linkall-labs/vanus/pkg/controller" + "github.com/linkall-labs/vanus/pkg/cluster" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" "github.com/sony/sonyflake" "go.uber.org/atomic" @@ -141,8 +141,9 @@ func InitSnowflake(ctx context.Context, ctrlAddr []string, n *node) error { var err error once.Do(func() { + ctrl := cluster.NewClusterController(ctrlAddr, insecure.NewCredentials()) snow := &snowflake{ - client: controller.NewSnowflakeController(ctrlAddr, insecure.NewCredentials()), + client: ctrl.IDService().RawClient(), ctrlAddr: ctrlAddr, n: n, } diff --git a/internal/store/segment/server.go b/internal/store/segment/server.go index 3a5c89a65..1fd69910e 100644 --- a/internal/store/segment/server.go +++ b/internal/store/segment/server.go @@ -44,7 +44,7 @@ import ( "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/observability/metrics" "github.com/linkall-labs/vanus/observability/tracing" - "github.com/linkall-labs/vanus/pkg/controller" + "github.com/linkall-labs/vanus/pkg/cluster" "github.com/linkall-labs/vanus/pkg/util" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" metapb "github.com/linkall-labs/vanus/proto/pkg/meta" @@ -124,7 +124,8 @@ func NewServer(cfg store.Config) Server { tracer: tracing.NewTracer("store.segment.server", trace.SpanKindServer), } - srv.cc = controller.NewSegmentClient(cfg.ControllerAddresses, srv.credentials) + srv.ctrl = cluster.NewClusterController(cfg.ControllerAddresses, srv.credentials) + srv.cc = srv.ctrl.SegmentService().RawClient() return srv } @@ -155,6 +156,7 @@ type server struct { ctrlAddress []string credentials credentials.TransportCredentials + ctrl cluster.Cluster cc ctrlpb.SegmentControllerClient leaderC chan leaderInfo @@ -258,6 +260,9 @@ func (s *server) reconcileBlocks(ctx context.Context) error { func (s *server) registerSelf(ctx context.Context) error { // TODO(james.yin): pass information of blocks. + if err := s.ctrl.WaitForControllerReady(false); err != nil { + return err + } res, err := s.cc.RegisterSegmentServer(ctx, &ctrlpb.RegisterSegmentServerRequest{ Address: s.localAddress, VolumeId: s.volumeID, @@ -403,7 +408,7 @@ func (s *server) runHeartbeat(_ context.Context) error { } } - return controller.RegisterHeartbeat(ctx, time.Second, s.cc, f) + return s.ctrl.SegmentService().RegisterHeartbeat(ctx, time.Second, f) } func (s *server) leaderChanged(blockID, leaderID vanus.ID, term uint64) { diff --git a/internal/timer/timingwheel/bucket.go b/internal/timer/timingwheel/bucket.go index f66d2201f..35611fe29 100644 --- a/internal/timer/timingwheel/bucket.go +++ b/internal/timer/timingwheel/bucket.go @@ -23,20 +23,17 @@ import ( "sync" "time" - "github.com/linkall-labs/vanus/internal/kv" - "github.com/linkall-labs/vanus/internal/timer/metadata" - "github.com/linkall-labs/vanus/observability/log" - "github.com/linkall-labs/vanus/proto/pkg/meta" - "k8s.io/apimachinery/pkg/util/wait" - ce "github.com/cloudevents/sdk-go/v2" "github.com/cloudevents/sdk-go/v2/types" "github.com/linkall-labs/vanus/client" "github.com/linkall-labs/vanus/client/pkg/api" "github.com/linkall-labs/vanus/client/pkg/option" "github.com/linkall-labs/vanus/client/pkg/policy" + "github.com/linkall-labs/vanus/internal/kv" + "github.com/linkall-labs/vanus/internal/timer/metadata" + "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/pkg/errors" - ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" + "k8s.io/apimachinery/pkg/util/wait" ) const ( @@ -103,7 +100,6 @@ type bucket struct { wg sync.WaitGroup exitC chan struct{} kvStore kv.Client - ctrlCli ctrlpb.EventBusControllerClient client client.Client eventbusWriter api.BusWriter eventbusReader api.BusReader @@ -126,7 +122,6 @@ func newBucket(tw *timingWheel, element *list.Element, tick time.Duration, ebNam eventbus: ebName, exitC: make(chan struct{}), kvStore: tw.kvStore, - ctrlCli: tw.ctrlCli, client: tw.client, timingwheel: tw, element: element, @@ -327,31 +322,14 @@ func (b *bucket) push(ctx context.Context, tm *timingMsg) bool { return b.putEvent(ctx, tm) == nil } -func (b *bucket) isExistEventbus(ctx context.Context) bool { - _, err := b.ctrlCli.GetEventBus(ctx, &meta.EventBus{Name: b.eventbus}) - return err == nil -} - func (b *bucket) createEventbus(ctx context.Context) error { b.mu.Lock() defer b.mu.Unlock() - if !b.isLeader() || b.isExistEventbus(ctx) { + if !b.isLeader() { return nil } - _, err := b.ctrlCli.CreateEventBus(ctx, &ctrlpb.CreateEventBusRequest{ - Name: b.eventbus, - }) - if err != nil { - log.Error(ctx, "create eventbus failed", map[string]interface{}{ - log.KeyError: err, - "eventbus": b.eventbus, - }) - return err - } - log.Info(ctx, "create eventbus success.", map[string]interface{}{ - "eventbus": b.eventbus, - }) - return nil + return b.timingwheel.ctrl.EventbusService().CreateSystemEventbusIfNotExist(ctx, b.eventbus, + 1, "System Eventbus For Timing Service") } func (b *bucket) connectEventbus(ctx context.Context) { @@ -359,28 +337,6 @@ func (b *bucket) connectEventbus(ctx context.Context) { b.eventbusReader = b.client.Eventbus(ctx, b.eventbus).Reader() } -func (b *bucket) deleteEventbus(ctx context.Context) error { - b.mu.Lock() - defer b.mu.Unlock() - if !b.isLeader() || !b.isExistEventbus(ctx) { - return nil - } - _, err := b.ctrlCli.DeleteEventBus(ctx, &meta.EventBus{ - Name: b.eventbus, - }) - if err != nil { - log.Error(ctx, "delete eventbus failed", map[string]interface{}{ - log.KeyError: err, - "eventbus": b.eventbus, - }) - return err - } - log.Info(ctx, "delete eventbus success.", map[string]interface{}{ - "eventbus": b.eventbus, - }) - return nil -} - func (b *bucket) putEvent(ctx context.Context, tm *timingMsg) (err error) { defer func() { if errOfPanic := recover(); errOfPanic != nil { @@ -524,7 +480,7 @@ func (b *bucket) hasOnEnd(ctx context.Context) bool { } func (b *bucket) recycle(ctx context.Context) { - _ = b.deleteEventbus(ctx) + _ = b.timingwheel.ctrl.EventbusService().Delete(ctx, b.eventbus) _ = b.deleteOffsetMeta(ctx) } diff --git a/internal/timer/timingwheel/bucket_test.go b/internal/timer/timingwheel/bucket_test.go index 5602c13cb..ffbadf8c4 100644 --- a/internal/timer/timingwheel/bucket_test.go +++ b/internal/timer/timingwheel/bucket_test.go @@ -26,9 +26,8 @@ import ( "github.com/linkall-labs/vanus/client/pkg/api" "github.com/linkall-labs/vanus/client/pkg/record" "github.com/linkall-labs/vanus/internal/kv" + "github.com/linkall-labs/vanus/pkg/cluster" "github.com/linkall-labs/vanus/pkg/errors" - ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" - . "github.com/smartystreets/goconvey/convey" ) @@ -86,21 +85,12 @@ func TestBucket_start(t *testing.T) { mockClient.EXPECT().Eventbus(Any(), Any()).AnyTimes().Return(mockEventbus) mockEventbus.EXPECT().Writer().AnyTimes().Return(mockBusWriter) mockEventbus.EXPECT().Reader().AnyTimes().Return(mockBusReader) - mockEventbusCtrlCli := ctrlpb.NewMockEventBusControllerClient(mockCtrl) bucket.client = mockClient - bucket.ctrlCli = mockEventbusCtrlCli ls := make([]*record.Eventlog, 1) ls[0] = &record.Eventlog{ ID: 1, } - Convey("test bucket start with create eventbus failed", func() { - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) - mockEventbusCtrlCli.EXPECT().CreateEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) - err := bucket.start(ctx) - So(err, ShouldNotBeNil) - }) - // Convey("test bucket start with connect eventbus failed", func() { // mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, errors.New("test")) // mockEventbusCtrlCli.EXPECT().CreateEventBus(Any(), Any()).Times(1).Return(nil, nil) @@ -257,64 +247,19 @@ func TestBucket_createEventBus(t *testing.T) { ctx := context.Background() tw := newtimingwheel(cfg()) tw.SetLeader(true) - bucket := newBucket(tw, nil, time.Second, "", 1, 0) - bucket.timingwheel = tw - mockCtrl := NewController(t) - mockEventbusCtrlCli := ctrlpb.NewMockEventBusControllerClient(mockCtrl) - bucket.ctrlCli = mockEventbusCtrlCli - - Convey("eventbus has exist", func() { - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, nil) - err := bucket.createEventbus(ctx) - So(err, ShouldBeNil) - }) - - Convey("create failed", func() { - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) - mockEventbusCtrlCli.EXPECT().CreateEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) - err := bucket.createEventbus(ctx) - So(err, ShouldNotBeNil) - }) - - Convey("create success", func() { - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) - mockEventbusCtrlCli.EXPECT().CreateEventBus(Any(), Any()).Times(1).Return(nil, nil) - err := bucket.createEventbus(ctx) - So(err, ShouldBeNil) - }) - }) -} -func TestBucket_deleteEventBus(t *testing.T) { - Convey("test bucket delete eventbus", t, func() { - ctx := context.Background() - tw := newtimingwheel(cfg()) - tw.SetLeader(true) bucket := newBucket(tw, nil, time.Second, "", 1, 0) bucket.timingwheel = tw mockCtrl := NewController(t) - mockEventbusCtrlCli := ctrlpb.NewMockEventBusControllerClient(mockCtrl) - bucket.ctrlCli = mockEventbusCtrlCli - Convey("eventbus has not exist", func() { - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) - err := bucket.deleteEventbus(ctx) - So(err, ShouldBeNil) - }) - - Convey("delete failed", func() { - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, nil) - mockEventbusCtrlCli.EXPECT().DeleteEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) - err := bucket.deleteEventbus(ctx) - So(err, ShouldNotBeNil) - }) + mockCl := cluster.NewMockCluster(mockCtrl) + tw.ctrl = mockCl + mockSvc := cluster.NewMockEventbusService(mockCtrl) + mockCl.EXPECT().EventbusService().Times(1).Return(mockSvc) + mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any(), Any()).Times(1).Return(nil) - Convey("delete success", func() { - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, nil) - mockEventbusCtrlCli.EXPECT().DeleteEventBus(Any(), Any()).Times(1).Return(nil, nil) - err := bucket.deleteEventbus(ctx) - So(err, ShouldBeNil) - }) + err := bucket.createEventbus(ctx) + So(err, ShouldBeNil) }) } @@ -571,6 +516,12 @@ func TestBucket_recycle(t *testing.T) { bucket.eventbusWriter = mockEventbusWriter bucket.eventbusReader = mockEventbusReader + mockCl := cluster.NewMockCluster(mockCtrl) + tw.ctrl = mockCl + mockSvc := cluster.NewMockEventbusService(mockCtrl) + mockCl.EXPECT().EventbusService().Times(1).Return(mockSvc) + mockSvc.EXPECT().Delete(Any(), Any()).Times(1).Return(nil) + Convey("test bucket wait success", func() { bucket.recycle(ctx) }) diff --git a/internal/timer/timingwheel/timingwheel.go b/internal/timer/timingwheel/timingwheel.go index d6eb5df06..4e61ea262 100644 --- a/internal/timer/timingwheel/timingwheel.go +++ b/internal/timer/timingwheel/timingwheel.go @@ -20,9 +20,6 @@ import ( "encoding/json" stderr "errors" "fmt" - "io" - "sync" - "time" ce "github.com/cloudevents/sdk-go/v2" "github.com/linkall-labs/vanus/client" @@ -31,11 +28,14 @@ import ( "github.com/linkall-labs/vanus/internal/timer/metadata" "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/observability/metrics" - "github.com/linkall-labs/vanus/pkg/controller" + "github.com/linkall-labs/vanus/pkg/cluster" "github.com/linkall-labs/vanus/pkg/errors" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" "google.golang.org/grpc/credentials/insecure" + "io" "k8s.io/apimachinery/pkg/util/wait" + "sync" + "time" ) const ( @@ -81,6 +81,8 @@ type timingWheel struct { client client.Client twList *list.List // element: *timingWheelElement + ctrl cluster.Cluster + receivingStation *bucket distributionStation *bucket @@ -90,16 +92,6 @@ type timingWheel struct { } func NewTimingWheel(c *Config) Manager { - store, err := newEtcdClientV3(c.EtcdEndpoints, c.KeyPrefix) - if err != nil { - log.Error(context.Background(), "new etcd client v3 failed", map[string]interface{}{ - log.KeyError: err, - "endpoints": c.EtcdEndpoints, - "key_prefix": c.KeyPrefix, - }) - panic("new etcd client failed") - } - log.Info(context.Background(), "new timingwheel manager", map[string]interface{}{ "tick": c.Tick, "layers": c.Layers, @@ -112,20 +104,36 @@ func NewTimingWheel(c *Config) Manager { metrics.TimingWheelSizeGauge.Set(float64(c.WheelSize)) metrics.TimingWheelLayersGauge.Set(float64(c.Layers)) return &timingWheel{ - config: c, - kvStore: store, - ctrlCli: controller.NewEventbusClient(c.CtrlEndpoints, insecure.NewCredentials()), - client: client.Connect(c.CtrlEndpoints), - twList: list.New(), - leader: false, - exitC: make(chan struct{}), + config: c, + client: client.Connect(c.CtrlEndpoints), + twList: list.New(), + leader: false, + exitC: make(chan struct{}), } } -// Init init the current timing wheel. +// Init the current timing wheel. func (tw *timingWheel) Init(ctx context.Context) error { log.Info(ctx, "init timingwheel", nil) // Init Hierarchical Timing Wheels. + ctrl := cluster.NewClusterController(tw.config.CtrlEndpoints, insecure.NewCredentials()) + if err := ctrl.WaitForControllerReady(true); err != nil { + panic("wait for controller ready timeout") + } + tw.ctrl = ctrl + tw.ctrlCli = ctrl.EventbusService().RawClient() + + store, err := newEtcdClientV3(tw.config.EtcdEndpoints, tw.config.KeyPrefix) + if err != nil { + log.Error(context.Background(), "new etcd client v3 failed", map[string]interface{}{ + log.KeyError: err, + "endpoints": tw.config.EtcdEndpoints, + "key_prefix": tw.config.KeyPrefix, + }) + panic("new etcd client failed") + } + tw.kvStore = store + for layer := int64(1); layer <= tw.config.Layers+1; layer++ { tick := exponent(tw.config.Tick, tw.config.WheelSize, layer-1) twe := newTimingWheelElement(tw, tick, layer) @@ -182,7 +190,7 @@ func (tw *timingWheel) Start(ctx context.Context) error { } } - // start receving station for scheduled events receiving + // start receiving station for scheduled events receiving if err = tw.startReceivingStation(ctx); err != nil { return err } @@ -225,7 +233,8 @@ func (tw *timingWheel) IsLeader() bool { } func (tw *timingWheel) IsDeployed(ctx context.Context) bool { - return tw.receivingStation.isExistEventbus(ctx) && tw.distributionStation.isExistEventbus(ctx) + return tw.ctrl.EventbusService().IsExist(ctx, tw.receivingStation.eventbus) && + tw.ctrl.EventbusService().IsExist(ctx, tw.distributionStation.eventbus) } func (tw *timingWheel) Recover(ctx context.Context) error { diff --git a/internal/timer/timingwheel/timingwheel_test.go b/internal/timer/timingwheel/timingwheel_test.go index 714dc5a5d..f4c66a517 100644 --- a/internal/timer/timingwheel/timingwheel_test.go +++ b/internal/timer/timingwheel/timingwheel_test.go @@ -30,6 +30,7 @@ import ( "github.com/linkall-labs/vanus/client/pkg/record" "github.com/linkall-labs/vanus/internal/kv" "github.com/linkall-labs/vanus/internal/timer/metadata" + "github.com/linkall-labs/vanus/pkg/cluster" "github.com/linkall-labs/vanus/pkg/errors" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" . "github.com/prashantv/gostub" @@ -61,12 +62,10 @@ func TestTimingWheel_Start(t *testing.T) { mockEventbus.EXPECT().Writer().AnyTimes().Return(mockBusWriter) mockEventbus.EXPECT().Reader().AnyTimes().Return(mockBusReader) mockEventbus.EXPECT().ListLog(Any()).AnyTimes().Return([]api.Eventlog{mockEventlog}, nil) - - mockEventbusCtrlCli := ctrlpb.NewMockEventBusControllerClient(mockCtrl) + mockBusReader.EXPECT().Read(Any(), Any(), Any()).AnyTimes().Return([]*ce.Event{}, int64(0), uint64(0), nil) for e := tw.twList.Front(); e != nil; { for _, bucket := range e.Value.(*timingWheelElement).buckets { bucket.eventbusReader = mockBusReader - bucket.ctrlCli = mockEventbusCtrlCli bucket.client = mockClient bucket.timingwheel = tw } @@ -74,9 +73,7 @@ func TestTimingWheel_Start(t *testing.T) { e = next } tw.client = mockClient - tw.receivingStation.ctrlCli = mockEventbusCtrlCli tw.receivingStation.client = mockClient - tw.distributionStation.ctrlCli = mockEventbusCtrlCli tw.distributionStation.client = mockClient tw.receivingStation.timingwheel = tw tw.distributionStation.timingwheel = tw @@ -85,17 +82,15 @@ func TestTimingWheel_Start(t *testing.T) { ID: 1, } - Convey("test timingwheel start bucket with start failure", func() { - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).AnyTimes().Return(nil, stderr.New("test")) - mockEventbusCtrlCli.EXPECT().CreateEventBus(Any(), Any()).AnyTimes().Return(nil, stderr.New("test")) - err := tw.Start(ctx) - So(err, ShouldNotBeNil) - }) + mockCl := cluster.NewMockCluster(mockCtrl) + tw.ctrl = mockCl + mockSvc := cluster.NewMockEventbusService(mockCtrl) + mockCl.EXPECT().EventbusService().AnyTimes().Return(mockSvc) + mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any(), Any()).AnyTimes().Return(nil) + mockSvc.EXPECT().IsExist(Any(), Any()).AnyTimes().Return(true) Convey("test timingwheel start bucket start success", func() { tw.SetLeader(false) - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).AnyTimes().Return(nil, nil) - mockEventbusCtrlCli.EXPECT().CreateEventBus(Any(), Any()).AnyTimes().Return(nil, stderr.New("test")) err := tw.Start(ctx) cancel() So(err, ShouldBeNil) @@ -151,16 +146,17 @@ func TestTimingWheel_IsLeader(t *testing.T) { } func TestTimingWheel_IsDeployed(t *testing.T) { - Convey("test timingwheel isdeployed", t, func() { + Convey("test timingwheel is deployed", t, func() { ctx := context.Background() tw := newtimingwheel(cfg()) mockCtrl := NewController(t) - mockEventbusCtrlCli := ctrlpb.NewMockEventBusControllerClient(mockCtrl) - tw.receivingStation.ctrlCli = mockEventbusCtrlCli - tw.distributionStation.ctrlCli = mockEventbusCtrlCli + mockCl := cluster.NewMockCluster(mockCtrl) + tw.ctrl = mockCl + mockSvc := cluster.NewMockEventbusService(mockCtrl) + mockCl.EXPECT().EventbusService().AnyTimes().Return(mockSvc) + mockSvc.EXPECT().IsExist(Any(), Any()).AnyTimes().Return(true) - Convey("test timingwheel is not deployed", func() { - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).AnyTimes().Return(nil, nil) + Convey("test timingwheel is deployed", func() { ret := tw.IsDeployed(ctx) So(ret, ShouldBeTrue) }) @@ -263,18 +259,13 @@ func TestTimingWheel_startReceivingStation(t *testing.T) { tw := newtimingwheel(cfg()) tw.SetLeader(true) mockCtrl := NewController(t) - mockClient := client.NewMockClient(mockCtrl) - // mockEventbus := eventbus.NewMockEventbus(mockCtrl) - // mockBusWriter := eventbus.NewMockBusWriter(mockCtrl) - // mockBusReader := eventbus.NewMockBusReader(mockCtrl) - mockEventbusCtrlCli := ctrlpb.NewMockEventBusControllerClient(mockCtrl) - tw.client = mockClient - tw.receivingStation.ctrlCli = mockEventbusCtrlCli - tw.receivingStation.client = mockClient + mockCl := cluster.NewMockCluster(mockCtrl) + tw.ctrl = mockCl + mockSvc := cluster.NewMockEventbusService(mockCtrl) + mockCl.EXPECT().EventbusService().AnyTimes().Return(mockSvc) Convey("test timingwheel start receiving station with create eventbus failed", func() { - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) - mockEventbusCtrlCli.EXPECT().CreateEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) + mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any(), Any()).Times(1).Return(errors.New("test")) err := tw.startReceivingStation(ctx) So(err, ShouldNotBeNil) }) @@ -282,7 +273,7 @@ func TestTimingWheel_startReceivingStation(t *testing.T) { // Convey("test timingwheel start receiving station success", func() { // mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, errors.New("test")) // mockEventbusCtrlCli.EXPECT().CreateEventBus(Any(), Any()).Times(1).Return(nil, nil) - // mockClient.EXPECT().Eventbus(Any(), Any()).AnyTimes().Return(mockEventbus) + // mockClient.EXPECT().EventbusService(Any(), Any()).AnyTimes().Return(mockEventbus) // mockEventbus.EXPECT().Writer().AnyTimes().Return(mockBusWriter) // mockEventbus.EXPECT().Reader().AnyTimes().Return(mockBusReader) // err := tw.startReceivingStation(ctx) @@ -372,14 +363,15 @@ func TestTimingWheel_startDistributionStation(t *testing.T) { mockEventbusCtrlCli := ctrlpb.NewMockEventBusControllerClient(mockCtrl) tw.client = mockClient tw.ctrlCli = mockEventbusCtrlCli - tw.receivingStation.ctrlCli = mockEventbusCtrlCli tw.receivingStation.client = mockClient - tw.distributionStation.ctrlCli = mockEventbusCtrlCli tw.distributionStation.client = mockClient + mockCl := cluster.NewMockCluster(mockCtrl) + tw.ctrl = mockCl + mockSvc := cluster.NewMockEventbusService(mockCtrl) + mockCl.EXPECT().EventbusService().AnyTimes().Return(mockSvc) Convey("test timingwheel start distribution station with create eventbus failed", func() { - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) - mockEventbusCtrlCli.EXPECT().CreateEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) + mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any(), Any()).Times(1).Return(errors.New("test")) err := tw.startDistributionStation(ctx) So(err, ShouldNotBeNil) }) @@ -388,7 +380,7 @@ func TestTimingWheel_startDistributionStation(t *testing.T) { // mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, errors.New("test")) // mockEventbusCtrlCli.EXPECT().CreateEventBus(Any(), Any()).Times(1).Return(nil, nil) // mockEventbus.EXPECT().ListLog(Any()).AnyTimes().Return([]eventlog.Eventlog{mockEventlog}, nil) - // mockClient.EXPECT().Eventbus(Any(), Any()).AnyTimes().Return(mockEventbus) + // mockClient.EXPECT().EventbusService(Any(), Any()).AnyTimes().Return(mockEventbus) // mockEventbus.EXPECT().Writer().AnyTimes().Return(mockBusWriter) // err := tw.startDistributionStation(ctx) // So(err, ShouldBeNil) @@ -544,7 +536,7 @@ func TestTimingWheelElement_push(t *testing.T) { } func TestTimingWheelElement_pushBack(t *testing.T) { - Convey("test timingwheelelement push back", t, func() { + Convey("test timingwheel element push back", t, func() { ctx := context.Background() tw := newtimingwheel(cfg()) mockCtrl := NewController(t) @@ -557,14 +549,16 @@ func TestTimingWheelElement_pushBack(t *testing.T) { mockClient.EXPECT().Eventbus(Any(), Any()).AnyTimes().Return(mockEventbus) mockEventbus.EXPECT().Writer().AnyTimes().Return(mockBusWriter) mockEventbus.EXPECT().Reader().AnyTimes().Return(mockBusReader) - mockEventbusCtrlCli := ctrlpb.NewMockEventBusControllerClient(mockCtrl) - tw.ctrlCli = mockEventbusCtrlCli tw.client = mockClient ls := make([]*record.Eventlog, 1) ls[0] = &record.Eventlog{ ID: 0, } + mockCl := cluster.NewMockCluster(mockCtrl) + tw.ctrl = mockCl + mockSvc := cluster.NewMockEventbusService(mockCtrl) + mockCl.EXPECT().EventbusService().AnyTimes().Return(mockSvc) for e := tw.twList.Front(); e != nil; e = e.Next() { for _, bucket := range e.Value.(*timingWheelElement).buckets { @@ -597,8 +591,7 @@ func TestTimingWheelElement_pushBack(t *testing.T) { Convey("push timing message failure causes start failed", func() { tw.SetLeader(true) - mockEventbusCtrlCli.EXPECT().GetEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) - mockEventbusCtrlCli.EXPECT().CreateEventBus(Any(), Any()).Times(1).Return(nil, stderr.New("test")) + mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any(), Any()).Times(1).Return(errors.New("test")) tm := newTimingMsg(ctx, event(1000)) twe := tw.twList.Back().Value.(*timingWheelElement) result := twe.pushBack(ctx, tm) diff --git a/internal/trigger/worker.go b/internal/trigger/worker.go index e39ab1223..d2029fb8b 100644 --- a/internal/trigger/worker.go +++ b/internal/trigger/worker.go @@ -28,7 +28,7 @@ import ( "github.com/linkall-labs/vanus/internal/trigger/trigger" "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/observability/metrics" - "github.com/linkall-labs/vanus/pkg/controller" + "github.com/linkall-labs/vanus/pkg/cluster" "github.com/linkall-labs/vanus/pkg/errors" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" metapb "github.com/linkall-labs/vanus/proto/pkg/meta" @@ -64,6 +64,7 @@ type worker struct { lock sync.RWMutex tgLock sync.RWMutex client ctrlpb.TriggerControllerClient + ctrl cluster.Cluster } func NewWorker(config Config) Worker { @@ -73,10 +74,12 @@ func NewWorker(config Config) Worker { m := &worker{ config: config, - client: controller.NewTriggerClient(config.ControllerAddr, insecure.NewCredentials()), + ctrl: cluster.NewClusterController(config.ControllerAddr, insecure.NewCredentials()), triggerMap: make(map[vanus.ID]trigger.Trigger), newTrigger: trigger.NewTrigger, } + m.client = m.ctrl.TriggerService().RawClient() + m.ctx, m.stop = context.WithCancel(context.Background()) return m } @@ -230,7 +233,7 @@ func (w *worker) startHeartbeat(ctx context.Context) error { SubscriptionInfo: w.getAllSubscriptionInfo(ctx), } } - return controller.RegisterHeartbeat(ctx, w.config.HeartbeatInterval, w.client, f) + return w.ctrl.TriggerService().RegisterHeartbeat(ctx, w.config.HeartbeatInterval, f) } func (w *worker) stopSubscription(ctx context.Context, id vanus.ID) error { diff --git a/observability/metrics/controller.go b/observability/metrics/controller.go index 5701d86be..d1efe0c0e 100644 --- a/observability/metrics/controller.go +++ b/observability/metrics/controller.go @@ -23,7 +23,7 @@ var ( Namespace: namespace, Subsystem: moduleOfController, Name: "eventbus_number_total", - Help: "The number of Eventbus.", + Help: "The number of EventbusService.", }) EventlogGaugeVec = prometheus.NewGauge(prometheus.GaugeOpts{ diff --git a/pkg/cluster/controller.go b/pkg/cluster/controller.go new file mode 100644 index 000000000..4442ca9b7 --- /dev/null +++ b/pkg/cluster/controller.go @@ -0,0 +1,171 @@ +// Copyright 2022 Linkall Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +//go:generate mockgen -source=controller.go -destination=mock_controller.go -package=cluster +package cluster + +import ( + "context" + "errors" + "google.golang.org/protobuf/types/known/emptypb" + "sync" + "time" + + "github.com/linkall-labs/vanus/observability/log" + "github.com/linkall-labs/vanus/pkg/cluster/raw_client" + ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" + "google.golang.org/grpc/credentials" +) + +var ( + defaultClusterStartTimeout = 3 * time.Minute +) + +type Topology struct { + ControllerLeader string + ControllerURLs []string + Uptime time.Time +} + +type Cluster interface { + WaitForControllerReady(createEventbus bool) error + Status() Topology + IsReady(createEventbus bool) bool + EventbusService() EventbusService + SegmentService() SegmentService + EventlogService() EventlogService + TriggerService() TriggerService + IDService() IDService +} + +type EventbusService interface { + IsExist(ctx context.Context, name string) bool + CreateSystemEventbusIfNotExist(ctx context.Context, name string, logNum int, desc string) error + Delete(ctx context.Context, name string) error + RawClient() ctrlpb.EventBusControllerClient +} + +type EventlogService interface { + RawClient() ctrlpb.EventLogControllerClient +} + +type TriggerService interface { + RawClient() ctrlpb.TriggerControllerClient + RegisterHeartbeat(ctx context.Context, interval time.Duration, reqFunc func() interface{}) error +} + +type IDService interface { + RawClient() ctrlpb.SnowflakeControllerClient +} + +type SegmentService interface { + RegisterHeartbeat(ctx context.Context, interval time.Duration, reqFunc func() interface{}) error + RawClient() ctrlpb.SegmentControllerClient +} + +var ( + mutex sync.Mutex + cl Cluster +) + +func NewClusterController(endpoints []string, credentials credentials.TransportCredentials) Cluster { + mutex.Lock() + defer mutex.Unlock() + + // single instance + if cl == nil { + cc := raw_client.NewConnection(endpoints, credentials) + cl = &cluster{ + cc: cc, + ebSvc: newEventbusService(cc), + segmentSvc: newSegmentService(cc), + elSvc: newEventlogService(cc), + triggerSvc: newTriggerService(cc), + idSvc: newIDService(cc), + ping: raw_client.NewPingClient(cc), + controllerAddress: endpoints, + } + } + return cl +} + +type cluster struct { + controllerAddress []string + cc *raw_client.Conn + ebSvc EventbusService + elSvc EventlogService + triggerSvc TriggerService + idSvc IDService + segmentSvc SegmentService + ping ctrlpb.PingServerClient +} + +func (c *cluster) WaitForControllerReady(createEventbus bool) error { + start := time.Now() + log.Info(context.Background(), "wait for controller is ready", nil) + t := time.NewTicker(defaultClusterStartTimeout) + defer t.Stop() + for !c.IsReady(createEventbus) { + select { + case <-t.C: + return errors.New("cluster isn't ready") + default: + time.Sleep(time.Second) + } + } + + log.Info(context.Background(), "controller is ready", map[string]interface{}{ + "waiting_time": time.Now().Sub(start), + }) + return nil +} + +func (c *cluster) IsReady(createEventbus bool) bool { + res, err := c.ping.Ping(context.Background(), &emptypb.Empty{}) + if err != nil { + log.Warning(context.Background(), "failed to ping controller", map[string]interface{}{ + log.KeyError: err, + }) + return false + } + if res.LeaderAddr == "" { + return false + } + return createEventbus && res.GetIsEventbusReady() +} + +func (c *cluster) Status() Topology { + // TODO(wenfeng) + return Topology{} +} + +func (c *cluster) EventbusService() EventbusService { + return c.ebSvc +} + +func (c *cluster) SegmentService() SegmentService { + return c.segmentSvc +} + +func (c *cluster) EventlogService() EventlogService { + return c.elSvc +} + +func (c *cluster) TriggerService() TriggerService { + return c.triggerSvc +} + +func (c *cluster) IDService() IDService { + return c.idSvc +} diff --git a/pkg/cluster/eventbus.go b/pkg/cluster/eventbus.go new file mode 100644 index 000000000..3502c2aed --- /dev/null +++ b/pkg/cluster/eventbus.go @@ -0,0 +1,58 @@ +package cluster + +import ( + "context" + "fmt" + "github.com/linkall-labs/vanus/pkg/cluster/raw_client" + ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" + metapb "github.com/linkall-labs/vanus/proto/pkg/meta" + "strings" +) + +var ( + systemEventbusPrefix = "__" +) + +type eventbusService struct { + client ctrlpb.EventBusControllerClient +} + +func newEventbusService(cc *raw_client.Conn) EventbusService { + return &eventbusService{client: raw_client.NewEventbusClient(cc)} +} + +func (es *eventbusService) IsExist(ctx context.Context, name string) bool { + _, err := es.client.GetEventBus(ctx, &metapb.EventBus{ + Name: name, + }) + return err == nil +} + +func (es *eventbusService) CreateSystemEventbusIfNotExist(ctx context.Context, name string, logNum int, desc string) error { + if es.IsExist(ctx, name) { + return nil + } + + // TODO 创建前需要等到Store就绪,而store的就绪在controller之后,创建又在controller就绪过程中 + _, err := es.client.CreateSystemEventBus(ctx, &ctrlpb.CreateEventBusRequest{ + Name: name, + LogNumber: int32(logNum), + Description: desc, + }) + return err +} + +func (es *eventbusService) Delete(ctx context.Context, name string) error { + if !strings.HasPrefix(name, systemEventbusPrefix) { + return fmt.Errorf("the system eventbus must start with %s", systemEventbusPrefix) + } + + _, err := es.client.DeleteEventBus(ctx, &metapb.EventBus{ + Name: name, + }) + return err +} + +func (es *eventbusService) RawClient() ctrlpb.EventBusControllerClient { + return es.client +} diff --git a/pkg/cluster/eventlog.go b/pkg/cluster/eventlog.go new file mode 100644 index 000000000..eec0d8647 --- /dev/null +++ b/pkg/cluster/eventlog.go @@ -0,0 +1,18 @@ +package cluster + +import ( + "github.com/linkall-labs/vanus/pkg/cluster/raw_client" + ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" +) + +type eventlogService struct { + client ctrlpb.EventLogControllerClient +} + +func newEventlogService(cc *raw_client.Conn) EventlogService { + return &eventlogService{client: raw_client.NewEventlogClient(cc)} +} + +func (es *eventlogService) RawClient() ctrlpb.EventLogControllerClient { + return es.client +} diff --git a/pkg/cluster/id.go b/pkg/cluster/id.go new file mode 100644 index 000000000..c47bde88d --- /dev/null +++ b/pkg/cluster/id.go @@ -0,0 +1,18 @@ +package cluster + +import ( + "github.com/linkall-labs/vanus/pkg/cluster/raw_client" + ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" +) + +type idService struct { + client ctrlpb.SnowflakeControllerClient +} + +func newIDService(cc *raw_client.Conn) IDService { + return &idService{client: raw_client.NewSnowflakeController(cc)} +} + +func (es *idService) RawClient() ctrlpb.SnowflakeControllerClient { + return es.client +} diff --git a/pkg/cluster/mock_controller.go b/pkg/cluster/mock_controller.go new file mode 100644 index 000000000..131a81de9 --- /dev/null +++ b/pkg/cluster/mock_controller.go @@ -0,0 +1,404 @@ +// Code generated by MockGen. DO NOT EDIT. +// Source: controller.go + +// Package cluster is a generated GoMock package. +package cluster + +import ( + context "context" + reflect "reflect" + time "time" + + gomock "github.com/golang/mock/gomock" + controller "github.com/linkall-labs/vanus/proto/pkg/controller" +) + +// MockCluster is a mock of Cluster interface. +type MockCluster struct { + ctrl *gomock.Controller + recorder *MockClusterMockRecorder +} + +// MockClusterMockRecorder is the mock recorder for MockCluster. +type MockClusterMockRecorder struct { + mock *MockCluster +} + +// NewMockCluster creates a new mock instance. +func NewMockCluster(ctrl *gomock.Controller) *MockCluster { + mock := &MockCluster{ctrl: ctrl} + mock.recorder = &MockClusterMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockCluster) EXPECT() *MockClusterMockRecorder { + return m.recorder +} + +// EventbusService mocks base method. +func (m *MockCluster) EventbusService() EventbusService { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "EventbusService") + ret0, _ := ret[0].(EventbusService) + return ret0 +} + +// EventbusService indicates an expected call of EventbusService. +func (mr *MockClusterMockRecorder) EventbusService() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EventbusService", reflect.TypeOf((*MockCluster)(nil).EventbusService)) +} + +// EventlogService mocks base method. +func (m *MockCluster) EventlogService() EventlogService { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "EventlogService") + ret0, _ := ret[0].(EventlogService) + return ret0 +} + +// EventlogService indicates an expected call of EventlogService. +func (mr *MockClusterMockRecorder) EventlogService() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "EventlogService", reflect.TypeOf((*MockCluster)(nil).EventlogService)) +} + +// IDService mocks base method. +func (m *MockCluster) IDService() IDService { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "IDService") + ret0, _ := ret[0].(IDService) + return ret0 +} + +// IDService indicates an expected call of IDService. +func (mr *MockClusterMockRecorder) IDService() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IDService", reflect.TypeOf((*MockCluster)(nil).IDService)) +} + +// IsReady mocks base method. +func (m *MockCluster) IsReady(createEventbus bool) bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "IsReady", createEventbus) + ret0, _ := ret[0].(bool) + return ret0 +} + +// IsReady indicates an expected call of IsReady. +func (mr *MockClusterMockRecorder) IsReady(createEventbus interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsReady", reflect.TypeOf((*MockCluster)(nil).IsReady), createEventbus) +} + +// SegmentService mocks base method. +func (m *MockCluster) SegmentService() SegmentService { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "SegmentService") + ret0, _ := ret[0].(SegmentService) + return ret0 +} + +// SegmentService indicates an expected call of SegmentService. +func (mr *MockClusterMockRecorder) SegmentService() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SegmentService", reflect.TypeOf((*MockCluster)(nil).SegmentService)) +} + +// Status mocks base method. +func (m *MockCluster) Status() Topology { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Status") + ret0, _ := ret[0].(Topology) + return ret0 +} + +// Status indicates an expected call of Status. +func (mr *MockClusterMockRecorder) Status() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Status", reflect.TypeOf((*MockCluster)(nil).Status)) +} + +// TriggerService mocks base method. +func (m *MockCluster) TriggerService() TriggerService { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "TriggerService") + ret0, _ := ret[0].(TriggerService) + return ret0 +} + +// TriggerService indicates an expected call of TriggerService. +func (mr *MockClusterMockRecorder) TriggerService() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TriggerService", reflect.TypeOf((*MockCluster)(nil).TriggerService)) +} + +// WaitForControllerReady mocks base method. +func (m *MockCluster) WaitForControllerReady(createEventbus bool) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "WaitForControllerReady", createEventbus) + ret0, _ := ret[0].(error) + return ret0 +} + +// WaitForControllerReady indicates an expected call of WaitForControllerReady. +func (mr *MockClusterMockRecorder) WaitForControllerReady(createEventbus interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "WaitForControllerReady", reflect.TypeOf((*MockCluster)(nil).WaitForControllerReady), createEventbus) +} + +// MockEventbusService is a mock of EventbusService interface. +type MockEventbusService struct { + ctrl *gomock.Controller + recorder *MockEventbusServiceMockRecorder +} + +// MockEventbusServiceMockRecorder is the mock recorder for MockEventbusService. +type MockEventbusServiceMockRecorder struct { + mock *MockEventbusService +} + +// NewMockEventbusService creates a new mock instance. +func NewMockEventbusService(ctrl *gomock.Controller) *MockEventbusService { + mock := &MockEventbusService{ctrl: ctrl} + mock.recorder = &MockEventbusServiceMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockEventbusService) EXPECT() *MockEventbusServiceMockRecorder { + return m.recorder +} + +// CreateSystemEventbusIfNotExist mocks base method. +func (m *MockEventbusService) CreateSystemEventbusIfNotExist(ctx context.Context, name string, logNum int, desc string) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CreateSystemEventbusIfNotExist", ctx, name, logNum, desc) + ret0, _ := ret[0].(error) + return ret0 +} + +// CreateSystemEventbusIfNotExist indicates an expected call of CreateSystemEventbusIfNotExist. +func (mr *MockEventbusServiceMockRecorder) CreateSystemEventbusIfNotExist(ctx, name, logNum, desc interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateSystemEventbusIfNotExist", reflect.TypeOf((*MockEventbusService)(nil).CreateSystemEventbusIfNotExist), ctx, name, logNum, desc) +} + +// Delete mocks base method. +func (m *MockEventbusService) Delete(ctx context.Context, name string) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Delete", ctx, name) + ret0, _ := ret[0].(error) + return ret0 +} + +// Delete indicates an expected call of Delete. +func (mr *MockEventbusServiceMockRecorder) Delete(ctx, name interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Delete", reflect.TypeOf((*MockEventbusService)(nil).Delete), ctx, name) +} + +// IsExist mocks base method. +func (m *MockEventbusService) IsExist(ctx context.Context, name string) bool { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "IsExist", ctx, name) + ret0, _ := ret[0].(bool) + return ret0 +} + +// IsExist indicates an expected call of IsExist. +func (mr *MockEventbusServiceMockRecorder) IsExist(ctx, name interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "IsExist", reflect.TypeOf((*MockEventbusService)(nil).IsExist), ctx, name) +} + +// RawClient mocks base method. +func (m *MockEventbusService) RawClient() controller.EventBusControllerClient { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RawClient") + ret0, _ := ret[0].(controller.EventBusControllerClient) + return ret0 +} + +// RawClient indicates an expected call of RawClient. +func (mr *MockEventbusServiceMockRecorder) RawClient() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RawClient", reflect.TypeOf((*MockEventbusService)(nil).RawClient)) +} + +// MockEventlogService is a mock of EventlogService interface. +type MockEventlogService struct { + ctrl *gomock.Controller + recorder *MockEventlogServiceMockRecorder +} + +// MockEventlogServiceMockRecorder is the mock recorder for MockEventlogService. +type MockEventlogServiceMockRecorder struct { + mock *MockEventlogService +} + +// NewMockEventlogService creates a new mock instance. +func NewMockEventlogService(ctrl *gomock.Controller) *MockEventlogService { + mock := &MockEventlogService{ctrl: ctrl} + mock.recorder = &MockEventlogServiceMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockEventlogService) EXPECT() *MockEventlogServiceMockRecorder { + return m.recorder +} + +// RawClient mocks base method. +func (m *MockEventlogService) RawClient() controller.EventLogControllerClient { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RawClient") + ret0, _ := ret[0].(controller.EventLogControllerClient) + return ret0 +} + +// RawClient indicates an expected call of RawClient. +func (mr *MockEventlogServiceMockRecorder) RawClient() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RawClient", reflect.TypeOf((*MockEventlogService)(nil).RawClient)) +} + +// MockTriggerService is a mock of TriggerService interface. +type MockTriggerService struct { + ctrl *gomock.Controller + recorder *MockTriggerServiceMockRecorder +} + +// MockTriggerServiceMockRecorder is the mock recorder for MockTriggerService. +type MockTriggerServiceMockRecorder struct { + mock *MockTriggerService +} + +// NewMockTriggerService creates a new mock instance. +func NewMockTriggerService(ctrl *gomock.Controller) *MockTriggerService { + mock := &MockTriggerService{ctrl: ctrl} + mock.recorder = &MockTriggerServiceMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockTriggerService) EXPECT() *MockTriggerServiceMockRecorder { + return m.recorder +} + +// RawClient mocks base method. +func (m *MockTriggerService) RawClient() controller.TriggerControllerClient { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RawClient") + ret0, _ := ret[0].(controller.TriggerControllerClient) + return ret0 +} + +// RawClient indicates an expected call of RawClient. +func (mr *MockTriggerServiceMockRecorder) RawClient() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RawClient", reflect.TypeOf((*MockTriggerService)(nil).RawClient)) +} + +// RegisterHeartbeat mocks base method. +func (m *MockTriggerService) RegisterHeartbeat(ctx context.Context, interval time.Duration, reqFunc func() interface{}) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RegisterHeartbeat", ctx, interval, reqFunc) + ret0, _ := ret[0].(error) + return ret0 +} + +// RegisterHeartbeat indicates an expected call of RegisterHeartbeat. +func (mr *MockTriggerServiceMockRecorder) RegisterHeartbeat(ctx, interval, reqFunc interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterHeartbeat", reflect.TypeOf((*MockTriggerService)(nil).RegisterHeartbeat), ctx, interval, reqFunc) +} + +// MockIDService is a mock of IDService interface. +type MockIDService struct { + ctrl *gomock.Controller + recorder *MockIDServiceMockRecorder +} + +// MockIDServiceMockRecorder is the mock recorder for MockIDService. +type MockIDServiceMockRecorder struct { + mock *MockIDService +} + +// NewMockIDService creates a new mock instance. +func NewMockIDService(ctrl *gomock.Controller) *MockIDService { + mock := &MockIDService{ctrl: ctrl} + mock.recorder = &MockIDServiceMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockIDService) EXPECT() *MockIDServiceMockRecorder { + return m.recorder +} + +// RawClient mocks base method. +func (m *MockIDService) RawClient() controller.SnowflakeControllerClient { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RawClient") + ret0, _ := ret[0].(controller.SnowflakeControllerClient) + return ret0 +} + +// RawClient indicates an expected call of RawClient. +func (mr *MockIDServiceMockRecorder) RawClient() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RawClient", reflect.TypeOf((*MockIDService)(nil).RawClient)) +} + +// MockSegmentService is a mock of SegmentService interface. +type MockSegmentService struct { + ctrl *gomock.Controller + recorder *MockSegmentServiceMockRecorder +} + +// MockSegmentServiceMockRecorder is the mock recorder for MockSegmentService. +type MockSegmentServiceMockRecorder struct { + mock *MockSegmentService +} + +// NewMockSegmentService creates a new mock instance. +func NewMockSegmentService(ctrl *gomock.Controller) *MockSegmentService { + mock := &MockSegmentService{ctrl: ctrl} + mock.recorder = &MockSegmentServiceMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockSegmentService) EXPECT() *MockSegmentServiceMockRecorder { + return m.recorder +} + +// RawClient mocks base method. +func (m *MockSegmentService) RawClient() controller.SegmentControllerClient { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RawClient") + ret0, _ := ret[0].(controller.SegmentControllerClient) + return ret0 +} + +// RawClient indicates an expected call of RawClient. +func (mr *MockSegmentServiceMockRecorder) RawClient() *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RawClient", reflect.TypeOf((*MockSegmentService)(nil).RawClient)) +} + +// RegisterHeartbeat mocks base method. +func (m *MockSegmentService) RegisterHeartbeat(ctx context.Context, interval time.Duration, reqFunc func() interface{}) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RegisterHeartbeat", ctx, interval, reqFunc) + ret0, _ := ret[0].(error) + return ret0 +} + +// RegisterHeartbeat indicates an expected call of RegisterHeartbeat. +func (mr *MockSegmentServiceMockRecorder) RegisterHeartbeat(ctx, interval, reqFunc interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterHeartbeat", reflect.TypeOf((*MockSegmentService)(nil).RegisterHeartbeat), ctx, interval, reqFunc) +} diff --git a/pkg/controller/conn.go b/pkg/cluster/raw_client/conn.go similarity index 91% rename from pkg/controller/conn.go rename to pkg/cluster/raw_client/conn.go index d3aaeee99..79e50e5f0 100644 --- a/pkg/controller/conn.go +++ b/pkg/cluster/raw_client/conn.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package controller +package raw_client import ( "context" @@ -39,7 +39,7 @@ const ( vanusConnBypass = "VANUS_CONN_BYPASS" ) -type conn struct { +type Conn struct { mutex sync.Mutex leader string leaderClient *grpc.ClientConn @@ -49,13 +49,13 @@ type conn struct { bypass bool } -func newConn(endpoints []string, credentials credentials.TransportCredentials) *conn { +func NewConnection(endpoints []string, credentials credentials.TransportCredentials) *Conn { // TODO temporary implement v, _ := strconv.ParseBool(os.Getenv(vanusConnBypass)) - log.Info(context.Background(), "init conn", map[string]interface{}{ + log.Info(context.Background(), "init Conn", map[string]interface{}{ "endpoints": endpoints, }) - return &conn{ + return &Conn{ endpoints: endpoints, grpcConn: map[string]*grpc.ClientConn{}, credentials: credentials, @@ -63,7 +63,7 @@ func newConn(endpoints []string, credentials credentials.TransportCredentials) * } } -func (c *conn) invoke(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { +func (c *Conn) invoke(ctx context.Context, method string, args, reply interface{}, opts ...grpc.CallOption) error { log.Debug(ctx, "grpc invoke", map[string]interface{}{ "method": method, "args": fmt.Sprintf("%v", args), @@ -95,7 +95,7 @@ func (c *conn) invoke(ctx context.Context, method string, args, reply interface{ return err } -func (c *conn) close() error { +func (c *Conn) close() error { var err error for ip, conn := range c.grpcConn { if _err := conn.Close(); _err != nil { @@ -109,7 +109,7 @@ func (c *conn) close() error { return err } -func (c *conn) makeSureClient(ctx context.Context, renew bool) *grpc.ClientConn { +func (c *Conn) makeSureClient(ctx context.Context, renew bool) *grpc.ClientConn { c.mutex.Lock() defer c.mutex.Unlock() @@ -146,7 +146,7 @@ func (c *conn) makeSureClient(ctx context.Context, renew bool) *grpc.ClientConn conn := c.getGRPCConn(ctx, c.leader) if conn == nil { - log.Info(ctx, "failed to get conn", map[string]interface{}{}) + log.Info(ctx, "failed to get Conn", map[string]interface{}{}) return nil } log.Info(ctx, "success to get connection", map[string]interface{}{ @@ -157,7 +157,7 @@ func (c *conn) makeSureClient(ctx context.Context, renew bool) *grpc.ClientConn return c.leaderClient } -func (c *conn) getGRPCConn(ctx context.Context, addr string) *grpc.ClientConn { +func (c *Conn) getGRPCConn(ctx context.Context, addr string) *grpc.ClientConn { if addr == "" { return nil } diff --git a/pkg/controller/conn_test.go b/pkg/cluster/raw_client/conn_test.go similarity index 91% rename from pkg/controller/conn_test.go rename to pkg/cluster/raw_client/conn_test.go index 2084923cb..cf260227c 100644 --- a/pkg/controller/conn_test.go +++ b/pkg/cluster/raw_client/conn_test.go @@ -1,4 +1,4 @@ -package controller +package raw_client import ( "errors" diff --git a/pkg/controller/eventbus.go b/pkg/cluster/raw_client/eventbus.go similarity index 85% rename from pkg/controller/eventbus.go rename to pkg/cluster/raw_client/eventbus.go index 2d11c600e..ff53d6023 100644 --- a/pkg/controller/eventbus.go +++ b/pkg/cluster/raw_client/eventbus.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package controller +package raw_client import ( "context" @@ -21,7 +21,6 @@ import ( ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" metapb "github.com/linkall-labs/vanus/proto/pkg/meta" "google.golang.org/grpc" - "google.golang.org/grpc/credentials" "google.golang.org/protobuf/types/known/emptypb" ) @@ -30,12 +29,12 @@ var ( ) type eventbusClient struct { - cc *conn + cc *Conn } -func NewEventbusClient(endpoints []string, credentials credentials.TransportCredentials) ctrlpb.EventBusControllerClient { +func NewEventbusClient(cc *Conn) ctrlpb.EventBusControllerClient { return &eventbusClient{ - cc: newConn(endpoints, credentials), + cc: cc, } } @@ -52,6 +51,15 @@ func (ec *eventbusClient) CreateEventBus(ctx context.Context, in *ctrlpb.CreateE return out, nil } +func (ec *eventbusClient) CreateSystemEventBus(ctx context.Context, in *ctrlpb.CreateEventBusRequest, opts ...grpc.CallOption) (*metapb.EventBus, error) { + out := new(metapb.EventBus) + err := ec.cc.invoke(ctx, "/linkall.vanus.controller.EventBusController/CreateSystemEventBus", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (ec *eventbusClient) DeleteEventBus(ctx context.Context, in *metapb.EventBus, opts ...grpc.CallOption) (*emptypb.Empty, error) { out := new(emptypb.Empty) err := ec.cc.invoke(ctx, "/linkall.vanus.controller.EventBusController/DeleteEventBus", in, out, opts...) diff --git a/pkg/controller/eventlog.go b/pkg/cluster/raw_client/eventlog.go similarity index 87% rename from pkg/controller/eventlog.go rename to pkg/cluster/raw_client/eventlog.go index 9f02d3001..392436b2b 100644 --- a/pkg/controller/eventlog.go +++ b/pkg/cluster/raw_client/eventlog.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package controller +package raw_client import ( "context" @@ -20,21 +20,20 @@ import ( ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" "google.golang.org/grpc" - "google.golang.org/grpc/credentials" ) var ( _ io.Closer = (*eventlogClient)(nil) ) -func NewEventlogClient(endpoints []string, credentials credentials.TransportCredentials) ctrlpb.EventLogControllerClient { +func NewEventlogClient(cc *Conn) ctrlpb.EventLogControllerClient { return &eventlogClient{ - cc: newConn(endpoints, credentials), + cc: cc, } } type eventlogClient struct { - cc *conn + cc *Conn } func (elc *eventlogClient) Close() error { diff --git a/pkg/controller/heartbeat.go b/pkg/cluster/raw_client/heartbeat.go similarity index 98% rename from pkg/controller/heartbeat.go rename to pkg/cluster/raw_client/heartbeat.go index 80aa4c0f6..b7d9e3ff2 100644 --- a/pkg/controller/heartbeat.go +++ b/pkg/cluster/raw_client/heartbeat.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package controller +package raw_client import ( "context" diff --git a/pkg/cluster/raw_client/ping.go b/pkg/cluster/raw_client/ping.go new file mode 100644 index 000000000..0a177e3ea --- /dev/null +++ b/pkg/cluster/raw_client/ping.go @@ -0,0 +1,45 @@ +// Copyright 2022 Linkall Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package raw_client + +import ( + "context" + ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" + "google.golang.org/grpc" + "google.golang.org/protobuf/types/known/emptypb" +) + +var ( + _ ctrlpb.PingServerClient = (*pingClient)(nil) +) + +func NewPingClient(cc *Conn) ctrlpb.PingServerClient { + return &pingClient{ + cc: cc, + } +} + +type pingClient struct { + cc *Conn +} + +func (p pingClient) Ping(ctx context.Context, in *emptypb.Empty, opts ...grpc.CallOption) (*ctrlpb.PingResponse, error) { + out := new(ctrlpb.PingResponse) + err := p.cc.invoke(ctx, "/linkall.vanus.controller.PingServer/Ping", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} diff --git a/pkg/controller/segment.go b/pkg/cluster/raw_client/segment.go similarity index 95% rename from pkg/controller/segment.go rename to pkg/cluster/raw_client/segment.go index a8ef34b7d..225feb8d7 100644 --- a/pkg/controller/segment.go +++ b/pkg/cluster/raw_client/segment.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package controller +package raw_client import ( // standard libraries. @@ -25,7 +25,6 @@ import ( // third-party libraries. "google.golang.org/grpc" - "google.golang.org/grpc/credentials" "google.golang.org/protobuf/types/known/emptypb" "github.com/linkall-labs/vanus/pkg/errors" @@ -38,13 +37,13 @@ var ( ) type segmentClient struct { - cc *conn + cc *Conn heartBeatClient ctrlpb.SegmentController_SegmentHeartbeatClient } -func NewSegmentClient(ctrlAddrs []string, credentials credentials.TransportCredentials) ctrlpb.SegmentControllerClient { +func NewSegmentClient(cc *Conn) ctrlpb.SegmentControllerClient { return &segmentClient{ - cc: newConn(ctrlAddrs, credentials), + cc: cc, } } diff --git a/pkg/controller/segment_test.go b/pkg/cluster/raw_client/segment_test.go similarity index 98% rename from pkg/controller/segment_test.go rename to pkg/cluster/raw_client/segment_test.go index 14b8612ac..64eba50e4 100644 --- a/pkg/controller/segment_test.go +++ b/pkg/cluster/raw_client/segment_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package controller +package raw_client import ( "fmt" diff --git a/pkg/controller/snowflake.go b/pkg/cluster/raw_client/snowflake.go similarity index 89% rename from pkg/controller/snowflake.go rename to pkg/cluster/raw_client/snowflake.go index 3c3d86691..08652949f 100644 --- a/pkg/controller/snowflake.go +++ b/pkg/cluster/raw_client/snowflake.go @@ -12,26 +12,25 @@ // See the License for the specific language governing permissions and // limitations under the License. -package controller +package raw_client import ( "context" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" "google.golang.org/grpc" - "google.golang.org/grpc/credentials" "google.golang.org/protobuf/types/known/emptypb" "google.golang.org/protobuf/types/known/timestamppb" "google.golang.org/protobuf/types/known/wrapperspb" ) -func NewSnowflakeController(endpoints []string, credentials credentials.TransportCredentials) ctrlpb.SnowflakeControllerClient { +func NewSnowflakeController(cc *Conn) ctrlpb.SnowflakeControllerClient { return &snowflakeClient{ - cc: newConn(endpoints, credentials), + cc: cc, } } type snowflakeClient struct { - cc *conn + cc *Conn } func (sfc *snowflakeClient) Close() error { diff --git a/pkg/controller/trigger.go b/pkg/cluster/raw_client/trigger.go similarity index 96% rename from pkg/controller/trigger.go rename to pkg/cluster/raw_client/trigger.go index 1263286ae..e57000494 100644 --- a/pkg/controller/trigger.go +++ b/pkg/cluster/raw_client/trigger.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package controller +package raw_client import ( "context" @@ -26,7 +26,6 @@ import ( ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" metapb "github.com/linkall-labs/vanus/proto/pkg/meta" "google.golang.org/grpc" - "google.golang.org/grpc/credentials" "google.golang.org/protobuf/types/known/emptypb" ) @@ -35,14 +34,14 @@ var ( _ Heartbeat = (*triggerClient)(nil) ) -func NewTriggerClient(ctrlAddrs []string, credentials credentials.TransportCredentials) ctrlpb.TriggerControllerClient { +func NewTriggerClient(cc *Conn) ctrlpb.TriggerControllerClient { return &triggerClient{ - cc: newConn(ctrlAddrs, credentials), + cc: cc, } } type triggerClient struct { - cc *conn + cc *Conn heartBeatClient ctrlpb.TriggerController_TriggerWorkerHeartbeatClient } diff --git a/pkg/cluster/segment.go b/pkg/cluster/segment.go new file mode 100644 index 000000000..5c85807eb --- /dev/null +++ b/pkg/cluster/segment.go @@ -0,0 +1,24 @@ +package cluster + +import ( + "context" + "github.com/linkall-labs/vanus/pkg/cluster/raw_client" + ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" + "time" +) + +type segmentService struct { + client ctrlpb.SegmentControllerClient +} + +func newSegmentService(cc *raw_client.Conn) SegmentService { + return &segmentService{client: raw_client.NewSegmentClient(cc)} +} + +func (es *segmentService) RawClient() ctrlpb.SegmentControllerClient { + return es.client +} + +func (es *segmentService) RegisterHeartbeat(ctx context.Context, interval time.Duration, reqFunc func() interface{}) error { + return raw_client.RegisterHeartbeat(ctx, interval, es.client, reqFunc) +} diff --git a/pkg/cluster/trigger.go b/pkg/cluster/trigger.go new file mode 100644 index 000000000..326d2a7c2 --- /dev/null +++ b/pkg/cluster/trigger.go @@ -0,0 +1,24 @@ +package cluster + +import ( + "context" + "github.com/linkall-labs/vanus/pkg/cluster/raw_client" + ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" + "time" +) + +type triggerService struct { + client ctrlpb.TriggerControllerClient +} + +func newTriggerService(cc *raw_client.Conn) TriggerService { + return &triggerService{client: raw_client.NewTriggerClient(cc)} +} + +func (es *triggerService) RawClient() ctrlpb.TriggerControllerClient { + return es.client +} + +func (es *triggerService) RegisterHeartbeat(ctx context.Context, interval time.Duration, reqFunc func() interface{}) error { + return raw_client.RegisterHeartbeat(ctx, interval, es.client, reqFunc) +} diff --git a/proto/pkg/controller/controller.pb.go b/proto/pkg/controller/controller.pb.go index a3d0bb67c..4c1789d5f 100644 --- a/proto/pkg/controller/controller.pb.go +++ b/proto/pkg/controller/controller.pb.go @@ -47,8 +47,9 @@ type PingResponse struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields - LeaderAddr string `protobuf:"bytes,1,opt,name=leader_addr,json=leaderAddr,proto3" json:"leader_addr,omitempty"` - GatewayAddr string `protobuf:"bytes,2,opt,name=gateway_addr,json=gatewayAddr,proto3" json:"gateway_addr,omitempty"` + LeaderAddr string `protobuf:"bytes,1,opt,name=leader_addr,json=leaderAddr,proto3" json:"leader_addr,omitempty"` + GatewayAddr string `protobuf:"bytes,2,opt,name=gateway_addr,json=gatewayAddr,proto3" json:"gateway_addr,omitempty"` + IsEventbusReady bool `protobuf:"varint,3,opt,name=is_eventbus_ready,json=isEventbusReady,proto3" json:"is_eventbus_ready,omitempty"` } func (x *PingResponse) Reset() { @@ -97,6 +98,13 @@ func (x *PingResponse) GetGatewayAddr() string { return "" } +func (x *PingResponse) GetIsEventbusReady() bool { + if x != nil { + return x.IsEventbusReady + } + return false +} + type CreateEventBusRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1797,417 +1805,426 @@ var file_controller_proto_rawDesc = []byte{ 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x77, 0x72, 0x61, 0x70, 0x70, 0x65, 0x72, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0a, 0x6d, 0x65, 0x74, 0x61, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x52, 0x0a, 0x0c, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x65, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x7e, 0x0a, 0x0c, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x67, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x67, - 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x41, 0x64, 0x64, 0x72, 0x22, 0x6c, 0x0a, 0x15, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 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, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x67, 0x5f, 0x6e, - 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x6c, 0x6f, 0x67, - 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, - 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x50, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x62, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x38, 0x0a, 0x08, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x62, 0x75, 0x73, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, - 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, - 0x52, 0x08, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x62, 0x75, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x22, 0x1e, 0x0a, 0x1c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x22, 0x1f, 0x0a, 0x1d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xdd, 0x01, 0x0a, 0x17, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b, 0x0a, - 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x08, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x64, 0x12, 0x46, 0x0a, 0x0b, 0x68, 0x65, - 0x61, 0x6c, 0x74, 0x68, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x25, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, - 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x6c, - 0x74, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0a, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x49, 0x6e, - 0x66, 0x6f, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, - 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x54, - 0x69, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x61, 0x64, - 0x64, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, - 0x41, 0x64, 0x64, 0x72, 0x22, 0x1a, 0x0a, 0x18, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x48, - 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x71, 0x0a, 0x1c, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, - 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x76, - 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, - 0x69, 0x74, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, - 0x69, 0x74, 0x79, 0x22, 0x96, 0x02, 0x0a, 0x1d, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, - 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, - 0x49, 0x64, 0x12, 0x61, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, - 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, - 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, - 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x73, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, - 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, - 0x49, 0x64, 0x1a, 0x58, 0x0a, 0x0d, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x31, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, - 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x74, 0x0a, 0x1e, - 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, - 0x0a, 0x09, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x04, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x61, - 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, - 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, - 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, - 0x49, 0x64, 0x22, 0x21, 0x0a, 0x1f, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x6c, 0x0a, 0x1a, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, - 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, - 0x12, 0x0a, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, - 0x65, 0x72, 0x6d, 0x22, 0xd6, 0x04, 0x0a, 0x13, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x05, 0x74, 0x79, 0x70, 0x65, 0x73, 0x12, 0x3e, 0x0a, 0x06, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, - 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, - 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x34, 0x0a, 0x07, 0x66, 0x69, 0x6c, - 0x74, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x6c, 0x69, 0x6e, - 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, - 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x12, - 0x12, 0x0a, 0x04, 0x73, 0x69, 0x6e, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x73, - 0x69, 0x6e, 0x6b, 0x12, 0x4b, 0x0a, 0x0f, 0x73, 0x69, 0x6e, 0x6b, 0x5f, 0x63, 0x72, 0x65, 0x64, - 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6c, - 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, - 0x61, 0x2e, 0x53, 0x69, 0x6e, 0x6b, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, - 0x52, 0x0e, 0x73, 0x69, 0x6e, 0x6b, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, - 0x12, 0x38, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x07, 0x20, 0x01, - 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, - 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, - 0x52, 0x08, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x50, 0x0a, 0x11, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x5f, 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x18, - 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, - 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x6f, 0x74, 0x6f, - 0x63, 0x6f, 0x6c, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x10, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x63, 0x6f, 0x6c, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x1b, 0x0a, 0x09, - 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x62, 0x75, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x41, 0x0a, 0x0b, 0x74, 0x72, 0x61, - 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x65, 0x72, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, + 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x41, 0x64, 0x64, 0x72, 0x12, 0x2a, 0x0a, 0x11, 0x69, 0x73, + 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x62, 0x75, 0x73, 0x5f, 0x72, 0x65, 0x61, 0x64, 0x79, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x69, 0x73, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x62, 0x75, + 0x73, 0x52, 0x65, 0x61, 0x64, 0x79, 0x22, 0x6c, 0x0a, 0x15, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 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, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x67, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, + 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x6c, 0x6f, 0x67, 0x4e, 0x75, 0x6d, 0x62, + 0x65, 0x72, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x50, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x62, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x38, 0x0a, 0x08, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x62, 0x75, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, - 0x65, 0x74, 0x61, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x65, 0x72, 0x52, - 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x0d, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x07, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x22, 0x6e, 0x0a, 0x19, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x0c, 0x73, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x2d, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, - 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0c, - 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x7e, 0x0a, 0x19, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x02, 0x69, 0x64, 0x12, 0x51, 0x0a, 0x0c, 0x73, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x2d, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, - 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0c, - 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x28, 0x0a, 0x16, - 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x02, 0x69, 0x64, 0x22, 0x2b, 0x0a, 0x19, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, - 0x02, 0x69, 0x64, 0x22, 0x60, 0x0a, 0x18, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x44, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, - 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x38, 0x0a, 0x1c, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, - 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x22, - 0x1f, 0x0a, 0x1d, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, - 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x3a, 0x0a, 0x1e, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, - 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x22, 0x21, 0x0a, 0x1f, - 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, - 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0xa6, 0x01, 0x0a, 0x1d, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, - 0x72, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x73, - 0x74, 0x61, 0x72, 0x74, 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x74, - 0x61, 0x72, 0x74, 0x65, 0x64, 0x12, 0x51, 0x0a, 0x11, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x24, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, - 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x10, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x20, 0x0a, 0x1e, 0x54, 0x72, 0x69, 0x67, - 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, - 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x66, 0x0a, 0x1d, 0x52, 0x65, - 0x73, 0x65, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x54, 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x73, - 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x04, 0x52, 0x0e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, - 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, - 0x6d, 0x70, 0x22, 0x8b, 0x01, 0x0a, 0x13, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4f, 0x66, 0x66, - 0x73, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x11, 0x73, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, + 0x65, 0x74, 0x61, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x52, 0x08, 0x65, 0x76, + 0x65, 0x6e, 0x74, 0x62, 0x75, 0x73, 0x22, 0x17, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, + 0x1e, 0x0a, 0x1c, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, + 0x6f, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, + 0x1f, 0x0a, 0x1d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, + 0x6f, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0xdd, 0x01, 0x0a, 0x17, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x72, + 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, + 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x08, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c, + 0x75, 0x6d, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x76, 0x6f, + 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x64, 0x12, 0x46, 0x0a, 0x0b, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, + 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6c, 0x69, + 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, + 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x0a, 0x68, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1f, + 0x0a, 0x0b, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0a, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, + 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, + 0x22, 0x1a, 0x0a, 0x18, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x72, 0x74, + 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x71, 0x0a, 0x1c, + 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, + 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, + 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, + 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, + 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x76, 0x6f, 0x6c, 0x75, 0x6d, + 0x65, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x22, + 0x96, 0x02, 0x0a, 0x1d, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x73, 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x12, 0x61, + 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x45, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, + 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, + 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x64, 0x1a, 0x58, + 0x0a, 0x0d, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x31, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1b, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, + 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x74, 0x0a, 0x1e, 0x55, 0x6e, 0x72, 0x65, + 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x73, 0x65, + 0x72, 0x76, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x73, + 0x65, 0x72, 0x76, 0x65, 0x72, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, + 0x73, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, + 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x76, 0x6f, 0x6c, 0x75, 0x6d, 0x65, 0x49, 0x64, 0x22, 0x21, + 0x0a, 0x1f, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x6c, 0x0a, 0x1a, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, 0x1b, + 0x0a, 0x09, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x08, 0x6c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, + 0x65, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x74, 0x65, 0x72, 0x6d, 0x22, + 0xd6, 0x04, 0x0a, 0x13, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x74, 0x79, 0x70, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, + 0x74, 0x79, 0x70, 0x65, 0x73, 0x12, 0x3e, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x10, 0x73, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x21, 0x0a, - 0x0c, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x0b, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, - 0x22, 0x48, 0x0a, 0x14, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x61, 0x69, 0x6c, - 0x5f, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x04, 0x52, 0x12, 0x66, 0x61, 0x69, 0x6c, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0xb4, 0x01, 0x0a, 0x12, 0x4c, - 0x69, 0x73, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x20, 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x62, 0x75, 0x73, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, - 0x73, 0x49, 0x64, 0x12, 0x20, 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x6f, 0x67, - 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, - 0x4c, 0x6f, 0x67, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6f, - 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x6e, 0x64, 0x5f, - 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x65, 0x6e, - 0x64, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x6c, 0x69, 0x6d, 0x69, 0x74, - 0x65, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, - 0x64, 0x22, 0x4e, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x37, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6c, 0x69, 0x6e, + 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x34, 0x0a, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, + 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, + 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x46, 0x69, 0x6c, 0x74, + 0x65, 0x72, 0x52, 0x07, 0x66, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x73, + 0x69, 0x6e, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x73, 0x69, 0x6e, 0x6b, 0x12, + 0x4b, 0x0a, 0x0f, 0x73, 0x69, 0x6e, 0x6b, 0x5f, 0x63, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, + 0x61, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, + 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x69, + 0x6e, 0x6b, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x52, 0x0e, 0x73, 0x69, + 0x6e, 0x6b, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x12, 0x38, 0x0a, 0x08, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, + 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, + 0x65, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x52, 0x08, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x50, 0x0a, 0x11, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, + 0x6f, 0x6c, 0x5f, 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x23, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, + 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x53, + 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x10, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, + 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x65, 0x76, 0x65, 0x6e, + 0x74, 0x5f, 0x62, 0x75, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x65, 0x76, 0x65, + 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x41, 0x0a, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, + 0x72, 0x6d, 0x65, 0x72, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x73, 0x22, 0x7b, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x61, 0x62, - 0x6c, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x20, 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x62, 0x75, 0x73, 0x5f, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, - 0x49, 0x64, 0x12, 0x20, 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, - 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x4c, - 0x6f, 0x67, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x64, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x64, 0x22, 0x57, - 0x0a, 0x1c, 0x47, 0x65, 0x74, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x37, - 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x1b, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, - 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x73, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x32, 0x54, 0x0a, 0x0a, 0x50, 0x69, 0x6e, 0x67, 0x53, - 0x65, 0x72, 0x76, 0x65, 0x72, 0x12, 0x46, 0x0a, 0x04, 0x50, 0x69, 0x6e, 0x67, 0x12, 0x16, 0x2e, - 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x26, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, - 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, - 0x2e, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, 0xc1, 0x03, - 0x0a, 0x12, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, - 0x6c, 0x6c, 0x65, 0x72, 0x12, 0x5f, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, - 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x2f, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, - 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, - 0x72, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, - 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x45, 0x76, 0x65, - 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x46, 0x0a, 0x0e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x45, - 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x1c, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, - 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x45, 0x76, 0x65, - 0x6e, 0x74, 0x42, 0x75, 0x73, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x49, 0x0a, - 0x0b, 0x47, 0x65, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x1c, 0x2e, 0x6c, + 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x65, 0x72, 0x52, 0x0b, 0x74, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0b, + 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x0c, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, + 0x0a, 0x07, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x07, 0x64, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x22, 0x6e, 0x0a, 0x19, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6c, 0x69, + 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, + 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x7e, 0x0a, 0x19, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x02, 0x69, 0x64, 0x12, 0x51, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6c, 0x69, + 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, + 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x28, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x02, + 0x69, 0x64, 0x22, 0x2b, 0x0a, 0x19, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x02, 0x69, 0x64, 0x22, + 0x60, 0x0a, 0x18, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x44, 0x0a, 0x0c, 0x73, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x20, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, + 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x22, 0x38, 0x0a, 0x1c, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, + 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x22, 0x1f, 0x0a, 0x1d, 0x52, + 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x3a, 0x0a, 0x1e, + 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, + 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, + 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x22, 0x21, 0x0a, 0x1f, 0x55, 0x6e, 0x72, 0x65, + 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa6, 0x01, 0x0a, 0x1d, + 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x48, 0x65, 0x61, + 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, + 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x73, 0x74, 0x61, 0x72, 0x74, + 0x65, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x73, 0x74, 0x61, 0x72, 0x74, 0x65, + 0x64, 0x12, 0x51, 0x0a, 0x11, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, - 0x61, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x1a, 0x1c, 0x2e, 0x6c, 0x69, 0x6e, - 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x56, 0x0a, 0x0c, 0x4c, 0x69, 0x73, 0x74, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, - 0x1a, 0x2e, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, - 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x4c, 0x69, 0x73, 0x74, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x62, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x5f, 0x0a, 0x0e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, + 0x61, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x10, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x20, 0x0a, 0x1e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x66, 0x0a, 0x1d, 0x52, 0x65, 0x73, 0x65, 0x74, 0x4f, + 0x66, 0x66, 0x73, 0x65, 0x74, 0x54, 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x73, 0x75, 0x62, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x0e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, + 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0x8b, + 0x01, 0x0a, 0x13, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x51, 0x0a, 0x11, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, + 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x10, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x21, 0x0a, 0x0c, 0x66, 0x6f, 0x72, + 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0b, 0x66, 0x6f, 0x72, 0x63, 0x65, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x22, 0x48, 0x0a, 0x14, + 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x30, 0x0a, 0x14, 0x66, 0x61, 0x69, 0x6c, 0x5f, 0x73, 0x75, 0x62, + 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x04, 0x52, 0x12, 0x66, 0x61, 0x69, 0x6c, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0xb4, 0x01, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, + 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x62, 0x75, 0x73, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x49, 0x64, 0x12, + 0x20, 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x69, 0x64, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x67, 0x49, + 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x6f, 0x66, 0x66, 0x73, 0x65, + 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x73, 0x74, 0x61, 0x72, 0x74, 0x4f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66, 0x66, 0x73, + 0x65, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x65, 0x6e, 0x64, 0x4f, 0x66, 0x66, + 0x73, 0x65, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x64, 0x18, 0x05, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x07, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x64, 0x22, 0x4e, 0x0a, + 0x13, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x37, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, + 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x7b, 0x0a, + 0x1b, 0x47, 0x65, 0x74, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x0c, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x62, 0x75, 0x73, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x49, 0x64, 0x12, 0x20, + 0x0a, 0x0c, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x5f, 0x6c, 0x6f, 0x67, 0x5f, 0x69, 0x64, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x67, 0x49, 0x64, + 0x12, 0x18, 0x0a, 0x07, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x07, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x65, 0x64, 0x22, 0x57, 0x0a, 0x1c, 0x47, 0x65, + 0x74, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x37, 0x0a, 0x08, 0x73, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6c, + 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, + 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x73, 0x32, 0x54, 0x0a, 0x0a, 0x50, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x12, 0x46, 0x0a, 0x04, 0x50, 0x69, 0x6e, 0x67, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, + 0x79, 0x1a, 0x26, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, + 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x50, 0x69, 0x6e, + 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, 0xa8, 0x04, 0x0a, 0x12, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, + 0x12, 0x5f, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x2f, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, - 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x43, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, - 0x73, 0x32, 0x88, 0x02, 0x0a, 0x12, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x67, 0x43, 0x6f, - 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x12, 0x6a, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x2c, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, - 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, - 0x65, 0x72, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, + 0x73, 0x12, 0x65, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x79, 0x73, 0x74, 0x65, + 0x6d, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x2f, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, + 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, + 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x42, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x6c, 0x69, 0x6e, + 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x46, 0x0a, 0x0e, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x1c, 0x2e, 0x6c, 0x69, 0x6e, + 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x12, 0x49, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, + 0x1c, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, + 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x1a, 0x1c, 0x2e, + 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, + 0x74, 0x61, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x56, 0x0a, 0x0c, 0x4c, + 0x69, 0x73, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x16, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, + 0x70, 0x74, 0x79, 0x1a, 0x2e, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, + 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x4c, + 0x69, 0x73, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x62, 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x5f, 0x0a, 0x0e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x42, 0x75, 0x73, 0x12, 0x2f, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, - 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x85, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x41, 0x70, 0x70, 0x65, - 0x6e, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x35, 0x2e, - 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, - 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x70, 0x70, 0x65, - 0x6e, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, - 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, - 0x47, 0x65, 0x74, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, 0x83, 0x06, 0x0a, - 0x11, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, - 0x65, 0x72, 0x12, 0x88, 0x01, 0x0a, 0x15, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x36, 0x2e, 0x6c, - 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, - 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, - 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, - 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x75, 0x74, - 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7b, 0x0a, - 0x10, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, - 0x74, 0x12, 0x31, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, - 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, + 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, + 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x42, 0x75, 0x73, 0x32, 0x88, 0x02, 0x0a, 0x12, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x4c, 0x6f, + 0x67, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x12, 0x6a, 0x0a, 0x0b, 0x4c, + 0x69, 0x73, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x2c, 0x2e, 0x6c, 0x69, 0x6e, + 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, + 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, + 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, + 0x6c, 0x65, 0x72, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x85, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x41, + 0x70, 0x70, 0x65, 0x6e, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x12, 0x35, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, + 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x47, 0x65, 0x74, 0x41, + 0x70, 0x70, 0x65, 0x6e, 0x64, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, + 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, + 0x65, 0x72, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, + 0x83, 0x06, 0x0a, 0x11, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, + 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x12, 0x88, 0x01, 0x0a, 0x15, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, + 0x36, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, + 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, + 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, + 0x65, 0x72, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, + 0x6f, 0x75, 0x74, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x7b, 0x0a, 0x10, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x72, 0x74, + 0x62, 0x65, 0x61, 0x74, 0x12, 0x31, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, 0x12, 0x88, 0x01, 0x0a, 0x15, 0x52, - 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, - 0x72, 0x76, 0x65, 0x72, 0x12, 0x36, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, - 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, - 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, - 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x6c, - 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, - 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x8e, 0x01, 0x0a, 0x17, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, - 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, - 0x72, 0x12, 0x38, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, - 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x55, 0x6e, 0x72, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, + 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, + 0x65, 0x72, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, + 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, 0x12, 0x88, 0x01, + 0x0a, 0x15, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x12, 0x36, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, + 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, + 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x37, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, + 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, + 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x8e, 0x01, 0x0a, 0x17, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, - 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x39, 0x2e, 0x6c, 0x69, - 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, - 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, - 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x65, 0x0a, 0x18, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x49, 0x73, 0x46, 0x75, - 0x6c, 0x6c, 0x12, 0x31, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, - 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x63, 0x0a, - 0x13, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, - 0x61, 0x64, 0x65, 0x72, 0x12, 0x34, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, + 0x72, 0x76, 0x65, 0x72, 0x12, 0x38, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, - 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, - 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, - 0x74, 0x79, 0x32, 0xa7, 0x09, 0x0a, 0x11, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x43, 0x6f, - 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x12, 0x6d, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x33, + 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x39, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, - 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, + 0x73, 0x74, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x65, 0x0a, 0x18, 0x52, 0x65, 0x70, + 0x6f, 0x72, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x42, 0x6c, 0x6f, 0x63, 0x6b, 0x49, + 0x73, 0x46, 0x75, 0x6c, 0x6c, 0x12, 0x31, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, + 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, + 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x12, 0x63, 0x0a, 0x13, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x34, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, + 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, + 0x65, 0x72, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, + 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, + 0x45, 0x6d, 0x70, 0x74, 0x79, 0x32, 0xa7, 0x09, 0x0a, 0x11, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, + 0x72, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x12, 0x6d, 0x0a, 0x12, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x33, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, + 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, + 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x75, 0x62, 0x73, + 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x12, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x33, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, + 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, + 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, + 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x12, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x33, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, + 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x00, 0x12, 0x67, + 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x30, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, + 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x12, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x33, 0x2e, - 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, - 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, - 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, - 0x75, 0x73, 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x12, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x33, 0x2e, 0x6c, - 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, - 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x22, 0x00, 0x12, 0x67, 0x0a, 0x0f, 0x47, - 0x65, 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x30, - 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, - 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x20, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, - 0x2e, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x10, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, - 0x1a, 0x32, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, - 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x4c, 0x69, 0x73, 0x74, - 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8d, 0x01, 0x0a, 0x16, 0x54, 0x72, 0x69, 0x67, 0x67, - 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, - 0x74, 0x12, 0x37, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, - 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x54, 0x72, 0x69, - 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, - 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x6c, 0x69, 0x6e, - 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, - 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, - 0x6b, 0x65, 0x72, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, 0x12, 0x88, 0x01, 0x0a, 0x15, 0x52, 0x65, 0x67, 0x69, 0x73, - 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, - 0x12, 0x36, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, - 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, - 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, - 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, - 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, - 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, - 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x12, 0x8e, 0x01, 0x0a, 0x17, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, - 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x12, 0x38, 0x2e, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x10, 0x4c, 0x69, 0x73, 0x74, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x2e, 0x67, 0x6f, + 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, + 0x70, 0x74, 0x79, 0x1a, 0x32, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, + 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x4c, + 0x69, 0x73, 0x74, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x8d, 0x01, 0x0a, 0x16, 0x54, 0x72, + 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x48, 0x65, 0x61, 0x72, 0x74, + 0x62, 0x65, 0x61, 0x74, 0x12, 0x37, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, + 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, + 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x48, 0x65, 0x61, + 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, - 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, + 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, + 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, 0x12, 0x88, 0x01, 0x0a, 0x15, 0x52, 0x65, + 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x12, 0x36, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, + 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x52, + 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, + 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x6c, 0x69, + 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, + 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, + 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x8e, 0x01, 0x0a, 0x17, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x39, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, - 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, - 0x65, 0x72, 0x2e, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, - 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x69, 0x0a, 0x16, 0x52, 0x65, 0x73, 0x65, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, - 0x74, 0x54, 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x37, 0x2e, 0x6c, - 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, - 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, 0x4f, 0x66, 0x66, - 0x73, 0x65, 0x74, 0x54, 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x6d, 0x0a, - 0x0c, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x2d, 0x2e, - 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, - 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4f, - 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6c, - 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, - 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4f, 0x66, - 0x66, 0x73, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, 0xee, 0x01, 0x0a, - 0x13, 0x53, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, - 0x6c, 0x6c, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x43, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x16, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, - 0x70, 0x74, 0x79, 0x1a, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, - 0x44, 0x0a, 0x0c, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x12, - 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x55, 0x49, 0x6e, 0x74, 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x1a, 0x16, 0x2e, - 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x46, 0x0a, 0x0e, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, - 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, 0x49, 0x6e, 0x74, 0x33, 0x32, - 0x56, 0x61, 0x6c, 0x75, 0x65, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x34, 0x5a, - 0x32, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6c, 0x69, 0x6e, 0x6b, - 0x61, 0x6c, 0x6c, 0x2d, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2f, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, - 0x6c, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x12, 0x38, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, + 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x55, 0x6e, 0x72, 0x65, + 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x39, 0x2e, 0x6c, 0x69, 0x6e, + 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, + 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x55, 0x6e, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, + 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, 0x16, 0x52, 0x65, 0x73, 0x65, 0x74, 0x4f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x54, 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, + 0x37, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, + 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x74, + 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x54, 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x12, 0x6d, 0x0a, 0x0c, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, + 0x12, 0x2d, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, + 0x2e, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, + 0x69, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x2e, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, + 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, + 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, + 0xee, 0x01, 0x0a, 0x13, 0x53, 0x6e, 0x6f, 0x77, 0x66, 0x6c, 0x61, 0x6b, 0x65, 0x43, 0x6f, 0x6e, + 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x12, 0x49, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x43, 0x6c, + 0x75, 0x73, 0x74, 0x65, 0x72, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x16, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, + 0x6d, 0x70, 0x12, 0x44, 0x0a, 0x0c, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x6f, + 0x64, 0x65, 0x12, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, 0x49, 0x6e, 0x74, 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, 0x65, + 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x46, 0x0a, 0x0e, 0x55, 0x6e, 0x72, 0x65, + 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x1c, 0x2e, 0x67, 0x6f, 0x6f, + 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x55, 0x49, 0x6e, + 0x74, 0x33, 0x32, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x42, 0x34, 0x5a, 0x32, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6c, + 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2d, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x76, 0x61, 0x6e, 0x75, + 0x73, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x63, 0x6f, 0x6e, 0x74, + 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2292,60 +2309,62 @@ var file_controller_proto_depIdxs = []int32{ 43, // 16: linkall.vanus.controller.RegisterSegmentServerResponse.SegmentsEntry.value:type_name -> linkall.vanus.meta.Segment 44, // 17: linkall.vanus.controller.PingServer.Ping:input_type -> google.protobuf.Empty 1, // 18: linkall.vanus.controller.EventBusController.CreateEventBus:input_type -> linkall.vanus.controller.CreateEventBusRequest - 33, // 19: linkall.vanus.controller.EventBusController.DeleteEventBus:input_type -> linkall.vanus.meta.EventBus - 33, // 20: linkall.vanus.controller.EventBusController.GetEventBus:input_type -> linkall.vanus.meta.EventBus - 44, // 21: linkall.vanus.controller.EventBusController.ListEventBus:input_type -> google.protobuf.Empty - 3, // 22: linkall.vanus.controller.EventBusController.UpdateEventBus:input_type -> linkall.vanus.controller.UpdateEventBusRequest - 28, // 23: linkall.vanus.controller.EventLogController.ListSegment:input_type -> linkall.vanus.controller.ListSegmentRequest - 30, // 24: linkall.vanus.controller.EventLogController.GetAppendableSegment:input_type -> linkall.vanus.controller.GetAppendableSegmentRequest - 4, // 25: linkall.vanus.controller.SegmentController.QuerySegmentRouteInfo:input_type -> linkall.vanus.controller.QuerySegmentRouteInfoRequest - 6, // 26: linkall.vanus.controller.SegmentController.SegmentHeartbeat:input_type -> linkall.vanus.controller.SegmentHeartbeatRequest - 8, // 27: linkall.vanus.controller.SegmentController.RegisterSegmentServer:input_type -> linkall.vanus.controller.RegisterSegmentServerRequest - 10, // 28: linkall.vanus.controller.SegmentController.UnregisterSegmentServer:input_type -> linkall.vanus.controller.UnregisterSegmentServerRequest - 6, // 29: linkall.vanus.controller.SegmentController.ReportSegmentBlockIsFull:input_type -> linkall.vanus.controller.SegmentHeartbeatRequest - 12, // 30: linkall.vanus.controller.SegmentController.ReportSegmentLeader:input_type -> linkall.vanus.controller.ReportSegmentLeaderRequest - 14, // 31: linkall.vanus.controller.TriggerController.CreateSubscription:input_type -> linkall.vanus.controller.CreateSubscriptionRequest - 15, // 32: linkall.vanus.controller.TriggerController.UpdateSubscription:input_type -> linkall.vanus.controller.UpdateSubscriptionRequest - 17, // 33: linkall.vanus.controller.TriggerController.DeleteSubscription:input_type -> linkall.vanus.controller.DeleteSubscriptionRequest - 16, // 34: linkall.vanus.controller.TriggerController.GetSubscription:input_type -> linkall.vanus.controller.GetSubscriptionRequest - 44, // 35: linkall.vanus.controller.TriggerController.ListSubscription:input_type -> google.protobuf.Empty - 23, // 36: linkall.vanus.controller.TriggerController.TriggerWorkerHeartbeat:input_type -> linkall.vanus.controller.TriggerWorkerHeartbeatRequest - 19, // 37: linkall.vanus.controller.TriggerController.RegisterTriggerWorker:input_type -> linkall.vanus.controller.RegisterTriggerWorkerRequest - 21, // 38: linkall.vanus.controller.TriggerController.UnregisterTriggerWorker:input_type -> linkall.vanus.controller.UnregisterTriggerWorkerRequest - 25, // 39: linkall.vanus.controller.TriggerController.ResetOffsetToTimestamp:input_type -> linkall.vanus.controller.ResetOffsetToTimestampRequest - 26, // 40: linkall.vanus.controller.TriggerController.CommitOffset:input_type -> linkall.vanus.controller.CommitOffsetRequest - 44, // 41: linkall.vanus.controller.SnowflakeController.GetClusterStartTime:input_type -> google.protobuf.Empty - 45, // 42: linkall.vanus.controller.SnowflakeController.RegisterNode:input_type -> google.protobuf.UInt32Value - 45, // 43: linkall.vanus.controller.SnowflakeController.UnregisterNode:input_type -> google.protobuf.UInt32Value - 0, // 44: linkall.vanus.controller.PingServer.Ping:output_type -> linkall.vanus.controller.PingResponse - 33, // 45: linkall.vanus.controller.EventBusController.CreateEventBus:output_type -> linkall.vanus.meta.EventBus - 44, // 46: linkall.vanus.controller.EventBusController.DeleteEventBus:output_type -> google.protobuf.Empty - 33, // 47: linkall.vanus.controller.EventBusController.GetEventBus:output_type -> linkall.vanus.meta.EventBus - 2, // 48: linkall.vanus.controller.EventBusController.ListEventBus:output_type -> linkall.vanus.controller.ListEventbusResponse - 33, // 49: linkall.vanus.controller.EventBusController.UpdateEventBus:output_type -> linkall.vanus.meta.EventBus - 29, // 50: linkall.vanus.controller.EventLogController.ListSegment:output_type -> linkall.vanus.controller.ListSegmentResponse - 31, // 51: linkall.vanus.controller.EventLogController.GetAppendableSegment:output_type -> linkall.vanus.controller.GetAppendableSegmentResponse - 5, // 52: linkall.vanus.controller.SegmentController.QuerySegmentRouteInfo:output_type -> linkall.vanus.controller.QuerySegmentRouteInfoResponse - 7, // 53: linkall.vanus.controller.SegmentController.SegmentHeartbeat:output_type -> linkall.vanus.controller.SegmentHeartbeatResponse - 9, // 54: linkall.vanus.controller.SegmentController.RegisterSegmentServer:output_type -> linkall.vanus.controller.RegisterSegmentServerResponse - 11, // 55: linkall.vanus.controller.SegmentController.UnregisterSegmentServer:output_type -> linkall.vanus.controller.UnregisterSegmentServerResponse - 44, // 56: linkall.vanus.controller.SegmentController.ReportSegmentBlockIsFull:output_type -> google.protobuf.Empty - 44, // 57: linkall.vanus.controller.SegmentController.ReportSegmentLeader:output_type -> google.protobuf.Empty - 41, // 58: linkall.vanus.controller.TriggerController.CreateSubscription:output_type -> linkall.vanus.meta.Subscription - 41, // 59: linkall.vanus.controller.TriggerController.UpdateSubscription:output_type -> linkall.vanus.meta.Subscription - 44, // 60: linkall.vanus.controller.TriggerController.DeleteSubscription:output_type -> google.protobuf.Empty - 41, // 61: linkall.vanus.controller.TriggerController.GetSubscription:output_type -> linkall.vanus.meta.Subscription - 18, // 62: linkall.vanus.controller.TriggerController.ListSubscription:output_type -> linkall.vanus.controller.ListSubscriptionResponse - 24, // 63: linkall.vanus.controller.TriggerController.TriggerWorkerHeartbeat:output_type -> linkall.vanus.controller.TriggerWorkerHeartbeatResponse - 20, // 64: linkall.vanus.controller.TriggerController.RegisterTriggerWorker:output_type -> linkall.vanus.controller.RegisterTriggerWorkerResponse - 22, // 65: linkall.vanus.controller.TriggerController.UnregisterTriggerWorker:output_type -> linkall.vanus.controller.UnregisterTriggerWorkerResponse - 44, // 66: linkall.vanus.controller.TriggerController.ResetOffsetToTimestamp:output_type -> google.protobuf.Empty - 27, // 67: linkall.vanus.controller.TriggerController.CommitOffset:output_type -> linkall.vanus.controller.CommitOffsetResponse - 46, // 68: linkall.vanus.controller.SnowflakeController.GetClusterStartTime:output_type -> google.protobuf.Timestamp - 44, // 69: linkall.vanus.controller.SnowflakeController.RegisterNode:output_type -> google.protobuf.Empty - 44, // 70: linkall.vanus.controller.SnowflakeController.UnregisterNode:output_type -> google.protobuf.Empty - 44, // [44:71] is the sub-list for method output_type - 17, // [17:44] is the sub-list for method input_type + 1, // 19: linkall.vanus.controller.EventBusController.CreateSystemEventBus:input_type -> linkall.vanus.controller.CreateEventBusRequest + 33, // 20: linkall.vanus.controller.EventBusController.DeleteEventBus:input_type -> linkall.vanus.meta.EventBus + 33, // 21: linkall.vanus.controller.EventBusController.GetEventBus:input_type -> linkall.vanus.meta.EventBus + 44, // 22: linkall.vanus.controller.EventBusController.ListEventBus:input_type -> google.protobuf.Empty + 3, // 23: linkall.vanus.controller.EventBusController.UpdateEventBus:input_type -> linkall.vanus.controller.UpdateEventBusRequest + 28, // 24: linkall.vanus.controller.EventLogController.ListSegment:input_type -> linkall.vanus.controller.ListSegmentRequest + 30, // 25: linkall.vanus.controller.EventLogController.GetAppendableSegment:input_type -> linkall.vanus.controller.GetAppendableSegmentRequest + 4, // 26: linkall.vanus.controller.SegmentController.QuerySegmentRouteInfo:input_type -> linkall.vanus.controller.QuerySegmentRouteInfoRequest + 6, // 27: linkall.vanus.controller.SegmentController.SegmentHeartbeat:input_type -> linkall.vanus.controller.SegmentHeartbeatRequest + 8, // 28: linkall.vanus.controller.SegmentController.RegisterSegmentServer:input_type -> linkall.vanus.controller.RegisterSegmentServerRequest + 10, // 29: linkall.vanus.controller.SegmentController.UnregisterSegmentServer:input_type -> linkall.vanus.controller.UnregisterSegmentServerRequest + 6, // 30: linkall.vanus.controller.SegmentController.ReportSegmentBlockIsFull:input_type -> linkall.vanus.controller.SegmentHeartbeatRequest + 12, // 31: linkall.vanus.controller.SegmentController.ReportSegmentLeader:input_type -> linkall.vanus.controller.ReportSegmentLeaderRequest + 14, // 32: linkall.vanus.controller.TriggerController.CreateSubscription:input_type -> linkall.vanus.controller.CreateSubscriptionRequest + 15, // 33: linkall.vanus.controller.TriggerController.UpdateSubscription:input_type -> linkall.vanus.controller.UpdateSubscriptionRequest + 17, // 34: linkall.vanus.controller.TriggerController.DeleteSubscription:input_type -> linkall.vanus.controller.DeleteSubscriptionRequest + 16, // 35: linkall.vanus.controller.TriggerController.GetSubscription:input_type -> linkall.vanus.controller.GetSubscriptionRequest + 44, // 36: linkall.vanus.controller.TriggerController.ListSubscription:input_type -> google.protobuf.Empty + 23, // 37: linkall.vanus.controller.TriggerController.TriggerWorkerHeartbeat:input_type -> linkall.vanus.controller.TriggerWorkerHeartbeatRequest + 19, // 38: linkall.vanus.controller.TriggerController.RegisterTriggerWorker:input_type -> linkall.vanus.controller.RegisterTriggerWorkerRequest + 21, // 39: linkall.vanus.controller.TriggerController.UnregisterTriggerWorker:input_type -> linkall.vanus.controller.UnregisterTriggerWorkerRequest + 25, // 40: linkall.vanus.controller.TriggerController.ResetOffsetToTimestamp:input_type -> linkall.vanus.controller.ResetOffsetToTimestampRequest + 26, // 41: linkall.vanus.controller.TriggerController.CommitOffset:input_type -> linkall.vanus.controller.CommitOffsetRequest + 44, // 42: linkall.vanus.controller.SnowflakeController.GetClusterStartTime:input_type -> google.protobuf.Empty + 45, // 43: linkall.vanus.controller.SnowflakeController.RegisterNode:input_type -> google.protobuf.UInt32Value + 45, // 44: linkall.vanus.controller.SnowflakeController.UnregisterNode:input_type -> google.protobuf.UInt32Value + 0, // 45: linkall.vanus.controller.PingServer.Ping:output_type -> linkall.vanus.controller.PingResponse + 33, // 46: linkall.vanus.controller.EventBusController.CreateEventBus:output_type -> linkall.vanus.meta.EventBus + 33, // 47: linkall.vanus.controller.EventBusController.CreateSystemEventBus:output_type -> linkall.vanus.meta.EventBus + 44, // 48: linkall.vanus.controller.EventBusController.DeleteEventBus:output_type -> google.protobuf.Empty + 33, // 49: linkall.vanus.controller.EventBusController.GetEventBus:output_type -> linkall.vanus.meta.EventBus + 2, // 50: linkall.vanus.controller.EventBusController.ListEventBus:output_type -> linkall.vanus.controller.ListEventbusResponse + 33, // 51: linkall.vanus.controller.EventBusController.UpdateEventBus:output_type -> linkall.vanus.meta.EventBus + 29, // 52: linkall.vanus.controller.EventLogController.ListSegment:output_type -> linkall.vanus.controller.ListSegmentResponse + 31, // 53: linkall.vanus.controller.EventLogController.GetAppendableSegment:output_type -> linkall.vanus.controller.GetAppendableSegmentResponse + 5, // 54: linkall.vanus.controller.SegmentController.QuerySegmentRouteInfo:output_type -> linkall.vanus.controller.QuerySegmentRouteInfoResponse + 7, // 55: linkall.vanus.controller.SegmentController.SegmentHeartbeat:output_type -> linkall.vanus.controller.SegmentHeartbeatResponse + 9, // 56: linkall.vanus.controller.SegmentController.RegisterSegmentServer:output_type -> linkall.vanus.controller.RegisterSegmentServerResponse + 11, // 57: linkall.vanus.controller.SegmentController.UnregisterSegmentServer:output_type -> linkall.vanus.controller.UnregisterSegmentServerResponse + 44, // 58: linkall.vanus.controller.SegmentController.ReportSegmentBlockIsFull:output_type -> google.protobuf.Empty + 44, // 59: linkall.vanus.controller.SegmentController.ReportSegmentLeader:output_type -> google.protobuf.Empty + 41, // 60: linkall.vanus.controller.TriggerController.CreateSubscription:output_type -> linkall.vanus.meta.Subscription + 41, // 61: linkall.vanus.controller.TriggerController.UpdateSubscription:output_type -> linkall.vanus.meta.Subscription + 44, // 62: linkall.vanus.controller.TriggerController.DeleteSubscription:output_type -> google.protobuf.Empty + 41, // 63: linkall.vanus.controller.TriggerController.GetSubscription:output_type -> linkall.vanus.meta.Subscription + 18, // 64: linkall.vanus.controller.TriggerController.ListSubscription:output_type -> linkall.vanus.controller.ListSubscriptionResponse + 24, // 65: linkall.vanus.controller.TriggerController.TriggerWorkerHeartbeat:output_type -> linkall.vanus.controller.TriggerWorkerHeartbeatResponse + 20, // 66: linkall.vanus.controller.TriggerController.RegisterTriggerWorker:output_type -> linkall.vanus.controller.RegisterTriggerWorkerResponse + 22, // 67: linkall.vanus.controller.TriggerController.UnregisterTriggerWorker:output_type -> linkall.vanus.controller.UnregisterTriggerWorkerResponse + 44, // 68: linkall.vanus.controller.TriggerController.ResetOffsetToTimestamp:output_type -> google.protobuf.Empty + 27, // 69: linkall.vanus.controller.TriggerController.CommitOffset:output_type -> linkall.vanus.controller.CommitOffsetResponse + 46, // 70: linkall.vanus.controller.SnowflakeController.GetClusterStartTime:output_type -> google.protobuf.Timestamp + 44, // 71: linkall.vanus.controller.SnowflakeController.RegisterNode:output_type -> google.protobuf.Empty + 44, // 72: linkall.vanus.controller.SnowflakeController.UnregisterNode:output_type -> google.protobuf.Empty + 45, // [45:73] is the sub-list for method output_type + 17, // [17:45] is the sub-list for method input_type 17, // [17:17] is the sub-list for extension type_name 17, // [17:17] is the sub-list for extension extendee 0, // [0:17] is the sub-list for field type_name @@ -2848,6 +2867,7 @@ var _PingServer_serviceDesc = grpc.ServiceDesc{ type EventBusControllerClient interface { // grpc -> HTTP CreateEventBus(ctx context.Context, in *CreateEventBusRequest, opts ...grpc.CallOption) (*meta.EventBus, error) + CreateSystemEventBus(ctx context.Context, in *CreateEventBusRequest, opts ...grpc.CallOption) (*meta.EventBus, error) DeleteEventBus(ctx context.Context, in *meta.EventBus, opts ...grpc.CallOption) (*emptypb.Empty, error) GetEventBus(ctx context.Context, in *meta.EventBus, opts ...grpc.CallOption) (*meta.EventBus, error) ListEventBus(ctx context.Context, in *emptypb.Empty, opts ...grpc.CallOption) (*ListEventbusResponse, error) @@ -2871,6 +2891,15 @@ func (c *eventBusControllerClient) CreateEventBus(ctx context.Context, in *Creat return out, nil } +func (c *eventBusControllerClient) CreateSystemEventBus(ctx context.Context, in *CreateEventBusRequest, opts ...grpc.CallOption) (*meta.EventBus, error) { + out := new(meta.EventBus) + err := c.cc.Invoke(ctx, "/linkall.vanus.controller.EventBusController/CreateSystemEventBus", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *eventBusControllerClient) DeleteEventBus(ctx context.Context, in *meta.EventBus, opts ...grpc.CallOption) (*emptypb.Empty, error) { out := new(emptypb.Empty) err := c.cc.Invoke(ctx, "/linkall.vanus.controller.EventBusController/DeleteEventBus", in, out, opts...) @@ -2911,6 +2940,7 @@ func (c *eventBusControllerClient) UpdateEventBus(ctx context.Context, in *Updat type EventBusControllerServer interface { // grpc -> HTTP CreateEventBus(context.Context, *CreateEventBusRequest) (*meta.EventBus, error) + CreateSystemEventBus(context.Context, *CreateEventBusRequest) (*meta.EventBus, error) DeleteEventBus(context.Context, *meta.EventBus) (*emptypb.Empty, error) GetEventBus(context.Context, *meta.EventBus) (*meta.EventBus, error) ListEventBus(context.Context, *emptypb.Empty) (*ListEventbusResponse, error) @@ -2924,6 +2954,9 @@ type UnimplementedEventBusControllerServer struct { func (*UnimplementedEventBusControllerServer) CreateEventBus(context.Context, *CreateEventBusRequest) (*meta.EventBus, error) { return nil, status.Errorf(codes.Unimplemented, "method CreateEventBus not implemented") } +func (*UnimplementedEventBusControllerServer) CreateSystemEventBus(context.Context, *CreateEventBusRequest) (*meta.EventBus, error) { + return nil, status.Errorf(codes.Unimplemented, "method CreateSystemEventBus not implemented") +} func (*UnimplementedEventBusControllerServer) DeleteEventBus(context.Context, *meta.EventBus) (*emptypb.Empty, error) { return nil, status.Errorf(codes.Unimplemented, "method DeleteEventBus not implemented") } @@ -2959,6 +2992,24 @@ func _EventBusController_CreateEventBus_Handler(srv interface{}, ctx context.Con return interceptor(ctx, in, info, handler) } +func _EventBusController_CreateSystemEventBus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(CreateEventBusRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(EventBusControllerServer).CreateSystemEventBus(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/linkall.vanus.controller.EventBusController/CreateSystemEventBus", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(EventBusControllerServer).CreateSystemEventBus(ctx, req.(*CreateEventBusRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _EventBusController_DeleteEventBus_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(meta.EventBus) if err := dec(in); err != nil { @@ -3039,6 +3090,10 @@ var _EventBusController_serviceDesc = grpc.ServiceDesc{ MethodName: "CreateEventBus", Handler: _EventBusController_CreateEventBus_Handler, }, + { + MethodName: "CreateSystemEventBus", + Handler: _EventBusController_CreateSystemEventBus_Handler, + }, { MethodName: "DeleteEventBus", Handler: _EventBusController_DeleteEventBus_Handler, diff --git a/proto/pkg/controller/mock_controller.go b/proto/pkg/controller/mock_controller.go index 6b50a2a8d..49838b4dc 100644 --- a/proto/pkg/controller/mock_controller.go +++ b/proto/pkg/controller/mock_controller.go @@ -13,6 +13,8 @@ import ( grpc "google.golang.org/grpc" metadata "google.golang.org/grpc/metadata" emptypb "google.golang.org/protobuf/types/known/emptypb" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" + wrapperspb "google.golang.org/protobuf/types/known/wrapperspb" ) // MockPingServerClient is a mock of PingServerClient interface. @@ -139,6 +141,26 @@ func (mr *MockEventBusControllerClientMockRecorder) CreateEventBus(ctx, in inter return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateEventBus", reflect.TypeOf((*MockEventBusControllerClient)(nil).CreateEventBus), varargs...) } +// CreateSystemEventBus mocks base method. +func (m *MockEventBusControllerClient) CreateSystemEventBus(ctx context.Context, in *CreateEventBusRequest, opts ...grpc.CallOption) (*meta.EventBus, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "CreateSystemEventBus", varargs...) + ret0, _ := ret[0].(*meta.EventBus) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CreateSystemEventBus indicates an expected call of CreateSystemEventBus. +func (mr *MockEventBusControllerClientMockRecorder) CreateSystemEventBus(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateSystemEventBus", reflect.TypeOf((*MockEventBusControllerClient)(nil).CreateSystemEventBus), varargs...) +} + // DeleteEventBus mocks base method. func (m *MockEventBusControllerClient) DeleteEventBus(ctx context.Context, in *meta.EventBus, opts ...grpc.CallOption) (*emptypb.Empty, error) { m.ctrl.T.Helper() @@ -257,6 +279,21 @@ func (mr *MockEventBusControllerServerMockRecorder) CreateEventBus(arg0, arg1 in return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateEventBus", reflect.TypeOf((*MockEventBusControllerServer)(nil).CreateEventBus), arg0, arg1) } +// CreateSystemEventBus mocks base method. +func (m *MockEventBusControllerServer) CreateSystemEventBus(arg0 context.Context, arg1 *CreateEventBusRequest) (*meta.EventBus, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "CreateSystemEventBus", arg0, arg1) + ret0, _ := ret[0].(*meta.EventBus) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// CreateSystemEventBus indicates an expected call of CreateSystemEventBus. +func (mr *MockEventBusControllerServerMockRecorder) CreateSystemEventBus(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateSystemEventBus", reflect.TypeOf((*MockEventBusControllerServer)(nil).CreateSystemEventBus), arg0, arg1) +} + // DeleteEventBus mocks base method. func (m *MockEventBusControllerServer) DeleteEventBus(arg0 context.Context, arg1 *meta.EventBus) (*emptypb.Empty, error) { m.ctrl.T.Helper() @@ -1624,3 +1661,154 @@ func (mr *MockTriggerController_TriggerWorkerHeartbeatServerMockRecorder) SetTra mr.mock.ctrl.T.Helper() return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SetTrailer", reflect.TypeOf((*MockTriggerController_TriggerWorkerHeartbeatServer)(nil).SetTrailer), arg0) } + +// MockSnowflakeControllerClient is a mock of SnowflakeControllerClient interface. +type MockSnowflakeControllerClient struct { + ctrl *gomock.Controller + recorder *MockSnowflakeControllerClientMockRecorder +} + +// MockSnowflakeControllerClientMockRecorder is the mock recorder for MockSnowflakeControllerClient. +type MockSnowflakeControllerClientMockRecorder struct { + mock *MockSnowflakeControllerClient +} + +// NewMockSnowflakeControllerClient creates a new mock instance. +func NewMockSnowflakeControllerClient(ctrl *gomock.Controller) *MockSnowflakeControllerClient { + mock := &MockSnowflakeControllerClient{ctrl: ctrl} + mock.recorder = &MockSnowflakeControllerClientMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockSnowflakeControllerClient) EXPECT() *MockSnowflakeControllerClientMockRecorder { + return m.recorder +} + +// GetClusterStartTime mocks base method. +func (m *MockSnowflakeControllerClient) GetClusterStartTime(ctx context.Context, in *emptypb.Empty, opts ...grpc.CallOption) (*timestamppb.Timestamp, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetClusterStartTime", varargs...) + ret0, _ := ret[0].(*timestamppb.Timestamp) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetClusterStartTime indicates an expected call of GetClusterStartTime. +func (mr *MockSnowflakeControllerClientMockRecorder) GetClusterStartTime(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetClusterStartTime", reflect.TypeOf((*MockSnowflakeControllerClient)(nil).GetClusterStartTime), varargs...) +} + +// RegisterNode mocks base method. +func (m *MockSnowflakeControllerClient) RegisterNode(ctx context.Context, in *wrapperspb.UInt32Value, opts ...grpc.CallOption) (*emptypb.Empty, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "RegisterNode", varargs...) + ret0, _ := ret[0].(*emptypb.Empty) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RegisterNode indicates an expected call of RegisterNode. +func (mr *MockSnowflakeControllerClientMockRecorder) RegisterNode(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterNode", reflect.TypeOf((*MockSnowflakeControllerClient)(nil).RegisterNode), varargs...) +} + +// UnregisterNode mocks base method. +func (m *MockSnowflakeControllerClient) UnregisterNode(ctx context.Context, in *wrapperspb.UInt32Value, opts ...grpc.CallOption) (*emptypb.Empty, error) { + m.ctrl.T.Helper() + varargs := []interface{}{ctx, in} + for _, a := range opts { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "UnregisterNode", varargs...) + ret0, _ := ret[0].(*emptypb.Empty) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UnregisterNode indicates an expected call of UnregisterNode. +func (mr *MockSnowflakeControllerClientMockRecorder) UnregisterNode(ctx, in interface{}, opts ...interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]interface{}{ctx, in}, opts...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UnregisterNode", reflect.TypeOf((*MockSnowflakeControllerClient)(nil).UnregisterNode), varargs...) +} + +// MockSnowflakeControllerServer is a mock of SnowflakeControllerServer interface. +type MockSnowflakeControllerServer struct { + ctrl *gomock.Controller + recorder *MockSnowflakeControllerServerMockRecorder +} + +// MockSnowflakeControllerServerMockRecorder is the mock recorder for MockSnowflakeControllerServer. +type MockSnowflakeControllerServerMockRecorder struct { + mock *MockSnowflakeControllerServer +} + +// NewMockSnowflakeControllerServer creates a new mock instance. +func NewMockSnowflakeControllerServer(ctrl *gomock.Controller) *MockSnowflakeControllerServer { + mock := &MockSnowflakeControllerServer{ctrl: ctrl} + mock.recorder = &MockSnowflakeControllerServerMockRecorder{mock} + return mock +} + +// EXPECT returns an object that allows the caller to indicate expected use. +func (m *MockSnowflakeControllerServer) EXPECT() *MockSnowflakeControllerServerMockRecorder { + return m.recorder +} + +// GetClusterStartTime mocks base method. +func (m *MockSnowflakeControllerServer) GetClusterStartTime(arg0 context.Context, arg1 *emptypb.Empty) (*timestamppb.Timestamp, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetClusterStartTime", arg0, arg1) + ret0, _ := ret[0].(*timestamppb.Timestamp) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetClusterStartTime indicates an expected call of GetClusterStartTime. +func (mr *MockSnowflakeControllerServerMockRecorder) GetClusterStartTime(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetClusterStartTime", reflect.TypeOf((*MockSnowflakeControllerServer)(nil).GetClusterStartTime), arg0, arg1) +} + +// RegisterNode mocks base method. +func (m *MockSnowflakeControllerServer) RegisterNode(arg0 context.Context, arg1 *wrapperspb.UInt32Value) (*emptypb.Empty, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "RegisterNode", arg0, arg1) + ret0, _ := ret[0].(*emptypb.Empty) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// RegisterNode indicates an expected call of RegisterNode. +func (mr *MockSnowflakeControllerServerMockRecorder) RegisterNode(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RegisterNode", reflect.TypeOf((*MockSnowflakeControllerServer)(nil).RegisterNode), arg0, arg1) +} + +// UnregisterNode mocks base method. +func (m *MockSnowflakeControllerServer) UnregisterNode(arg0 context.Context, arg1 *wrapperspb.UInt32Value) (*emptypb.Empty, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UnregisterNode", arg0, arg1) + ret0, _ := ret[0].(*emptypb.Empty) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UnregisterNode indicates an expected call of UnregisterNode. +func (mr *MockSnowflakeControllerServerMockRecorder) UnregisterNode(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UnregisterNode", reflect.TypeOf((*MockSnowflakeControllerServer)(nil).UnregisterNode), arg0, arg1) +} diff --git a/proto/pkg/proxy/proxy.pb.go b/proto/pkg/proxy/proxy.pb.go index 047d87003..020f05548 100644 --- a/proto/pkg/proxy/proxy.pb.go +++ b/proto/pkg/proxy/proxy.pb.go @@ -644,7 +644,7 @@ const _ = grpc.SupportPackageIsVersion6 // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type ControllerProxyClient interface { - // Eventbus + // EventbusService CreateEventBus(ctx context.Context, in *controller.CreateEventBusRequest, opts ...grpc.CallOption) (*meta.EventBus, error) DeleteEventBus(ctx context.Context, in *meta.EventBus, opts ...grpc.CallOption) (*emptypb.Empty, error) GetEventBus(ctx context.Context, in *meta.EventBus, opts ...grpc.CallOption) (*meta.EventBus, error) @@ -799,7 +799,7 @@ func (c *controllerProxyClient) GetEvent(ctx context.Context, in *GetEventReques // ControllerProxyServer is the server API for ControllerProxy service. type ControllerProxyServer interface { - // Eventbus + // EventbusService CreateEventBus(context.Context, *controller.CreateEventBusRequest) (*meta.EventBus, error) DeleteEventBus(context.Context, *meta.EventBus) (*emptypb.Empty, error) GetEventBus(context.Context, *meta.EventBus) (*meta.EventBus, error) diff --git a/proto/proto/controller.proto b/proto/proto/controller.proto index 6b08c8a77..fec9a5e80 100644 --- a/proto/proto/controller.proto +++ b/proto/proto/controller.proto @@ -31,6 +31,8 @@ service EventBusController { // grpc -> HTTP rpc CreateEventBus(CreateEventBusRequest) returns (linkall.vanus.meta.EventBus); + rpc CreateSystemEventBus(CreateEventBusRequest) + returns (linkall.vanus.meta.EventBus); rpc DeleteEventBus(meta.EventBus) returns (google.protobuf.Empty); rpc GetEventBus(meta.EventBus) returns (linkall.vanus.meta.EventBus); rpc ListEventBus(google.protobuf.Empty) returns (ListEventbusResponse); @@ -92,6 +94,7 @@ service SnowflakeController { message PingResponse { string leader_addr = 1; string gateway_addr = 2; + bool is_eventbus_ready=3; } message CreateEventBusRequest { diff --git a/vsctl/command/eventbus.go b/vsctl/command/eventbus.go index f8752c7e2..abda877c0 100644 --- a/vsctl/command/eventbus.go +++ b/vsctl/command/eventbus.go @@ -67,11 +67,11 @@ func createEventbusCommand() *cobra.Command { cmdFailedf(cmd, "create eventbus failed: %s", err) } if IsFormatJSON(cmd) { - data, _ := json.Marshal(map[string]interface{}{"Result": "Create Success", "Eventbus": eventbus}) + data, _ := json.Marshal(map[string]interface{}{"Result": "Create Success", "EventbusService": eventbus}) color.Green(string(data)) } else { t := table.NewWriter() - t.AppendHeader(table.Row{"Result", "Eventbus"}) + t.AppendHeader(table.Row{"Result", "EventbusService"}) t.AppendRow(table.Row{"Create Success", eventbus}) t.SetColumnConfigs([]table.ColumnConfig{ {Number: 1, VAlign: text.VAlignMiddle, Align: text.AlignCenter, AlignHeader: text.AlignCenter}, @@ -102,11 +102,11 @@ func deleteEventbusCommand() *cobra.Command { cmdFailedf(cmd, "delete eventbus failed: %s", err) } if IsFormatJSON(cmd) { - data, _ := json.Marshal(map[string]interface{}{"Result": "Delete Success", "Eventbus": eventbus}) + data, _ := json.Marshal(map[string]interface{}{"Result": "Delete Success", "EventbusService": eventbus}) color.Green(string(data)) } else { t := table.NewWriter() - t.AppendHeader(table.Row{"Result", "Eventbus"}) + t.AppendHeader(table.Row{"Result", "EventbusService"}) t.AppendRow(table.Row{"Delete Success", eventbus}) t.SetColumnConfigs([]table.ColumnConfig{ {Number: 1, VAlign: text.VAlignMiddle, Align: text.AlignCenter, AlignHeader: text.AlignCenter}, @@ -166,7 +166,7 @@ func getEventbusInfoCommand() *cobra.Command { color.Yellow("WARN: this command doesn't support --output-format\n") } if !showSegment && !showBlock { - t.AppendHeader(table.Row{"Eventbus", "Description", "Created_At", "Updated_At", + t.AppendHeader(table.Row{"EventbusService", "Description", "Created_At", "Updated_At", "Eventlog", "Segment Number"}) for _, res := range busMetas { for idx := 0; idx < len(res.Logs); idx++ { @@ -192,7 +192,7 @@ func getEventbusInfoCommand() *cobra.Command { } } else { if !showBlock { - t.AppendHeader(table.Row{"Eventbus", "Description", "Created_At", "Updated_At", + t.AppendHeader(table.Row{"EventbusService", "Description", "Created_At", "Updated_At", "Eventlog", "Segment", "Capacity", "Size", "Start", "End"}) for _, eb := range busMetas { for idx := 0; idx < len(eb.Logs); idx++ { @@ -219,7 +219,7 @@ func getEventbusInfoCommand() *cobra.Command { }...) t.SetColumnConfigs(cfgs) } else { - t.AppendHeader(table.Row{"Eventbus", "Description", "Created_At", "Updated_At", "Eventlog", + t.AppendHeader(table.Row{"EventbusService", "Description", "Created_At", "Updated_At", "Eventlog", "Segment", "Capacity", "Size", "Start", "End", "Block", "Leader", "Volume", "Endpoint"}) multiReplica := false for _, res := range busMetas { From 3c55d1b5dfe9fbdd94c706104e951353ba68c329 Mon Sep 17 00:00:00 2001 From: wenfeng Date: Thu, 15 Dec 2022 17:21:52 +0800 Subject: [PATCH 2/4] fix ut&lint Signed-off-by: wenfeng --- client/internal/vanus/eventlog/name_service.go | 2 +- internal/controller/eventbus/controller.go | 1 + internal/controller/trigger/controller.go | 5 ++--- internal/timer/timingwheel/bucket.go | 2 +- internal/timer/timingwheel/bucket_test.go | 2 +- internal/timer/timingwheel/timingwheel.go | 6 +++--- internal/timer/timingwheel/timingwheel_test.go | 8 ++++---- pkg/cluster/controller.go | 4 ++-- pkg/cluster/eventbus.go | 7 ++++--- pkg/cluster/mock_controller.go | 8 ++++---- 10 files changed, 23 insertions(+), 22 deletions(-) diff --git a/client/internal/vanus/eventlog/name_service.go b/client/internal/vanus/eventlog/name_service.go index 1fe325ba2..b30abceb3 100644 --- a/client/internal/vanus/eventlog/name_service.go +++ b/client/internal/vanus/eventlog/name_service.go @@ -17,7 +17,6 @@ package eventlog import ( // standard libraries. "context" - "github.com/linkall-labs/vanus/pkg/cluster" "math" "time" @@ -28,6 +27,7 @@ import ( // first-party libraries. "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/observability/tracing" + "github.com/linkall-labs/vanus/pkg/cluster" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" metapb "github.com/linkall-labs/vanus/proto/pkg/meta" diff --git a/internal/controller/eventbus/controller.go b/internal/controller/eventbus/controller.go index 04a83232b..0c894fa45 100644 --- a/internal/controller/eventbus/controller.go +++ b/internal/controller/eventbus/controller.go @@ -121,6 +121,7 @@ func (ctrl *controller) CreateEventBus(ctx context.Context, } func isValidEventbusName(name string) error { + name = strings.ToLower(name) for _, v := range name { if v == '.' || v == '_' || v == '-' { continue diff --git a/internal/controller/trigger/controller.go b/internal/controller/trigger/controller.go index f1b134240..3ba5243a2 100644 --- a/internal/controller/trigger/controller.go +++ b/internal/controller/trigger/controller.go @@ -48,7 +48,6 @@ var ( const ( defaultGcSubscriptionInterval = time.Second * 10 - defaultSystemEventbusEventlog = 1 ) func NewController(config Config, member embedetcd.Member) *controller { @@ -521,7 +520,7 @@ func (ctrl *controller) initTriggerSystemEventbus() { } if err := ctrl.cl.EventbusService().CreateSystemEventbusIfNotExist(ctx, primitive.RetryEventbusName, - defaultSystemEventbusEventlog, "System Eventbus For Trigger Service"); err != nil { + "System Eventbus For Trigger Service"); err != nil { log.Error(context.Background(), "failed to create RetryEventbus, exit", map[string]interface{}{ log.KeyError: err, }) @@ -529,7 +528,7 @@ func (ctrl *controller) initTriggerSystemEventbus() { } if err := ctrl.cl.EventbusService().CreateSystemEventbusIfNotExist(ctx, primitive.DeadLetterEventbusName, - defaultSystemEventbusEventlog, "System Eventbus For Trigger Service"); err != nil { + "System Eventbus For Trigger Service"); err != nil { log.Error(context.Background(), "failed to create DeadLetterEventbus, exit", map[string]interface{}{ log.KeyError: err, }) diff --git a/internal/timer/timingwheel/bucket.go b/internal/timer/timingwheel/bucket.go index 35611fe29..c4903ada1 100644 --- a/internal/timer/timingwheel/bucket.go +++ b/internal/timer/timingwheel/bucket.go @@ -329,7 +329,7 @@ func (b *bucket) createEventbus(ctx context.Context) error { return nil } return b.timingwheel.ctrl.EventbusService().CreateSystemEventbusIfNotExist(ctx, b.eventbus, - 1, "System Eventbus For Timing Service") + "System Eventbus For Timing Service") } func (b *bucket) connectEventbus(ctx context.Context) { diff --git a/internal/timer/timingwheel/bucket_test.go b/internal/timer/timingwheel/bucket_test.go index ffbadf8c4..8ef2cbf0a 100644 --- a/internal/timer/timingwheel/bucket_test.go +++ b/internal/timer/timingwheel/bucket_test.go @@ -256,7 +256,7 @@ func TestBucket_createEventBus(t *testing.T) { tw.ctrl = mockCl mockSvc := cluster.NewMockEventbusService(mockCtrl) mockCl.EXPECT().EventbusService().Times(1).Return(mockSvc) - mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any(), Any()).Times(1).Return(nil) + mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any()).Times(1).Return(nil) err := bucket.createEventbus(ctx) So(err, ShouldBeNil) diff --git a/internal/timer/timingwheel/timingwheel.go b/internal/timer/timingwheel/timingwheel.go index 4e61ea262..63d18444b 100644 --- a/internal/timer/timingwheel/timingwheel.go +++ b/internal/timer/timingwheel/timingwheel.go @@ -20,6 +20,9 @@ import ( "encoding/json" stderr "errors" "fmt" + "io" + "sync" + "time" ce "github.com/cloudevents/sdk-go/v2" "github.com/linkall-labs/vanus/client" @@ -32,10 +35,7 @@ import ( "github.com/linkall-labs/vanus/pkg/errors" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" "google.golang.org/grpc/credentials/insecure" - "io" "k8s.io/apimachinery/pkg/util/wait" - "sync" - "time" ) const ( diff --git a/internal/timer/timingwheel/timingwheel_test.go b/internal/timer/timingwheel/timingwheel_test.go index f4c66a517..e60a17d13 100644 --- a/internal/timer/timingwheel/timingwheel_test.go +++ b/internal/timer/timingwheel/timingwheel_test.go @@ -86,7 +86,7 @@ func TestTimingWheel_Start(t *testing.T) { tw.ctrl = mockCl mockSvc := cluster.NewMockEventbusService(mockCtrl) mockCl.EXPECT().EventbusService().AnyTimes().Return(mockSvc) - mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any(), Any()).AnyTimes().Return(nil) + mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any()).AnyTimes().Return(nil) mockSvc.EXPECT().IsExist(Any(), Any()).AnyTimes().Return(true) Convey("test timingwheel start bucket start success", func() { @@ -265,7 +265,7 @@ func TestTimingWheel_startReceivingStation(t *testing.T) { mockCl.EXPECT().EventbusService().AnyTimes().Return(mockSvc) Convey("test timingwheel start receiving station with create eventbus failed", func() { - mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any(), Any()).Times(1).Return(errors.New("test")) + mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any()).Times(1).Return(errors.New("test")) err := tw.startReceivingStation(ctx) So(err, ShouldNotBeNil) }) @@ -371,7 +371,7 @@ func TestTimingWheel_startDistributionStation(t *testing.T) { mockCl.EXPECT().EventbusService().AnyTimes().Return(mockSvc) Convey("test timingwheel start distribution station with create eventbus failed", func() { - mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any(), Any()).Times(1).Return(errors.New("test")) + mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any()).Times(1).Return(errors.New("test")) err := tw.startDistributionStation(ctx) So(err, ShouldNotBeNil) }) @@ -591,7 +591,7 @@ func TestTimingWheelElement_pushBack(t *testing.T) { Convey("push timing message failure causes start failed", func() { tw.SetLeader(true) - mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any(), Any()).Times(1).Return(errors.New("test")) + mockSvc.EXPECT().CreateSystemEventbusIfNotExist(Any(), Any(), Any()).Times(1).Return(errors.New("test")) tm := newTimingMsg(ctx, event(1000)) twe := tw.twList.Back().Value.(*timingWheelElement) result := twe.pushBack(ctx, tm) diff --git a/pkg/cluster/controller.go b/pkg/cluster/controller.go index 4442ca9b7..7393d7534 100644 --- a/pkg/cluster/controller.go +++ b/pkg/cluster/controller.go @@ -18,7 +18,6 @@ package cluster import ( "context" "errors" - "google.golang.org/protobuf/types/known/emptypb" "sync" "time" @@ -26,6 +25,7 @@ import ( "github.com/linkall-labs/vanus/pkg/cluster/raw_client" ctrlpb "github.com/linkall-labs/vanus/proto/pkg/controller" "google.golang.org/grpc/credentials" + "google.golang.org/protobuf/types/known/emptypb" ) var ( @@ -51,7 +51,7 @@ type Cluster interface { type EventbusService interface { IsExist(ctx context.Context, name string) bool - CreateSystemEventbusIfNotExist(ctx context.Context, name string, logNum int, desc string) error + CreateSystemEventbusIfNotExist(ctx context.Context, name string, desc string) error Delete(ctx context.Context, name string) error RawClient() ctrlpb.EventBusControllerClient } diff --git a/pkg/cluster/eventbus.go b/pkg/cluster/eventbus.go index 3502c2aed..c0b938ebb 100644 --- a/pkg/cluster/eventbus.go +++ b/pkg/cluster/eventbus.go @@ -10,7 +10,8 @@ import ( ) var ( - systemEventbusPrefix = "__" + systemEventbusPrefix = "__" + defaultSystemEventbusEventlog = 1 ) type eventbusService struct { @@ -28,7 +29,7 @@ func (es *eventbusService) IsExist(ctx context.Context, name string) bool { return err == nil } -func (es *eventbusService) CreateSystemEventbusIfNotExist(ctx context.Context, name string, logNum int, desc string) error { +func (es *eventbusService) CreateSystemEventbusIfNotExist(ctx context.Context, name string, desc string) error { if es.IsExist(ctx, name) { return nil } @@ -36,7 +37,7 @@ func (es *eventbusService) CreateSystemEventbusIfNotExist(ctx context.Context, n // TODO 创建前需要等到Store就绪,而store的就绪在controller之后,创建又在controller就绪过程中 _, err := es.client.CreateSystemEventBus(ctx, &ctrlpb.CreateEventBusRequest{ Name: name, - LogNumber: int32(logNum), + LogNumber: int32(defaultSystemEventbusEventlog), Description: desc, }) return err diff --git a/pkg/cluster/mock_controller.go b/pkg/cluster/mock_controller.go index 131a81de9..ecf03ac26 100644 --- a/pkg/cluster/mock_controller.go +++ b/pkg/cluster/mock_controller.go @@ -172,17 +172,17 @@ func (m *MockEventbusService) EXPECT() *MockEventbusServiceMockRecorder { } // CreateSystemEventbusIfNotExist mocks base method. -func (m *MockEventbusService) CreateSystemEventbusIfNotExist(ctx context.Context, name string, logNum int, desc string) error { +func (m *MockEventbusService) CreateSystemEventbusIfNotExist(ctx context.Context, name, desc string) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "CreateSystemEventbusIfNotExist", ctx, name, logNum, desc) + ret := m.ctrl.Call(m, "CreateSystemEventbusIfNotExist", ctx, name, desc) ret0, _ := ret[0].(error) return ret0 } // CreateSystemEventbusIfNotExist indicates an expected call of CreateSystemEventbusIfNotExist. -func (mr *MockEventbusServiceMockRecorder) CreateSystemEventbusIfNotExist(ctx, name, logNum, desc interface{}) *gomock.Call { +func (mr *MockEventbusServiceMockRecorder) CreateSystemEventbusIfNotExist(ctx, name, desc interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateSystemEventbusIfNotExist", reflect.TypeOf((*MockEventbusService)(nil).CreateSystemEventbusIfNotExist), ctx, name, logNum, desc) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateSystemEventbusIfNotExist", reflect.TypeOf((*MockEventbusService)(nil).CreateSystemEventbusIfNotExist), ctx, name, desc) } // Delete mocks base method. From 7a73f852a03dccf97c77c6646282f3df6af3e9bb Mon Sep 17 00:00:00 2001 From: wenfeng Date: Mon, 19 Dec 2022 12:17:51 +0800 Subject: [PATCH 3/4] update Signed-off-by: wenfeng --- internal/controller/trigger/controller.go | 7 ++++--- pkg/cluster/eventbus.go | 1 - 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/internal/controller/trigger/controller.go b/internal/controller/trigger/controller.go index 3ba5243a2..5ae157142 100644 --- a/internal/controller/trigger/controller.go +++ b/internal/controller/trigger/controller.go @@ -514,14 +514,14 @@ func (ctrl *controller) initTriggerSystemEventbus() { ctx := context.Background() log.Info(ctx, "trigger controller is ready to check system eventbus", nil) if err := ctrl.cl.WaitForControllerReady(true); err != nil { - log.Error(context.Background(), "trigger controller try to create system eventbus, "+ + log.Error(ctx, "trigger controller try to create system eventbus, "+ "but Vanus cluster hasn't ready, exit", nil) os.Exit(-1) } if err := ctrl.cl.EventbusService().CreateSystemEventbusIfNotExist(ctx, primitive.RetryEventbusName, "System Eventbus For Trigger Service"); err != nil { - log.Error(context.Background(), "failed to create RetryEventbus, exit", map[string]interface{}{ + log.Error(ctx, "failed to create RetryEventbus, exit", map[string]interface{}{ log.KeyError: err, }) os.Exit(-1) @@ -529,9 +529,10 @@ func (ctrl *controller) initTriggerSystemEventbus() { if err := ctrl.cl.EventbusService().CreateSystemEventbusIfNotExist(ctx, primitive.DeadLetterEventbusName, "System Eventbus For Trigger Service"); err != nil { - log.Error(context.Background(), "failed to create DeadLetterEventbus, exit", map[string]interface{}{ + log.Error(ctx, "failed to create DeadLetterEventbus, exit", map[string]interface{}{ log.KeyError: err, }) + os.Exit(-1) } log.Info(ctx, "trigger controller has finished for checking system eventbus", nil) }() diff --git a/pkg/cluster/eventbus.go b/pkg/cluster/eventbus.go index c0b938ebb..2f0cf5037 100644 --- a/pkg/cluster/eventbus.go +++ b/pkg/cluster/eventbus.go @@ -34,7 +34,6 @@ func (es *eventbusService) CreateSystemEventbusIfNotExist(ctx context.Context, n return nil } - // TODO 创建前需要等到Store就绪,而store的就绪在controller之后,创建又在controller就绪过程中 _, err := es.client.CreateSystemEventBus(ctx, &ctrlpb.CreateEventBusRequest{ Name: name, LogNumber: int32(defaultSystemEventbusEventlog), From b2c89d56eef6ab74c38be2c2e940e5c29b0a9b6c Mon Sep 17 00:00:00 2001 From: wenfeng Date: Mon, 19 Dec 2022 15:24:42 +0800 Subject: [PATCH 4/4] add waiting into trigger Signed-off-by: wenfeng --- internal/trigger/worker.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/internal/trigger/worker.go b/internal/trigger/worker.go index d2029fb8b..eced639be 100644 --- a/internal/trigger/worker.go +++ b/internal/trigger/worker.go @@ -78,6 +78,9 @@ func NewWorker(config Config) Worker { triggerMap: make(map[vanus.ID]trigger.Trigger), newTrigger: trigger.NewTrigger, } + if err := m.ctrl.WaitForControllerReady(false); err != nil { + panic("wait for controller ready timeout") + } m.client = m.ctrl.TriggerService().RawClient() m.ctx, m.stop = context.WithCancel(context.Background())