From 9900716b9e4af6e6e68cd17f89201c0df931ffa3 Mon Sep 17 00:00:00 2001 From: delu Date: Wed, 4 Jan 2023 22:26:33 +0800 Subject: [PATCH 1/9] feat: subscription support disable and resume (#378) * feat: support subscription pause Signed-off-by: xdlbdy * feat: support subscription pause Signed-off-by: xdlbdy * feat: support subscription pause Signed-off-by: xdlbdy * feat: support subscription pause Signed-off-by: xdlbdy * feat: vsctl add subscription pause Signed-off-by: xdlbdy * feat: vsctl add reset offset command Signed-off-by: xdlbdy * feat: vsctl add reset offset command Signed-off-by: xdlbdy * feat: vsctl add reset offset command Signed-off-by: xdlbdy Signed-off-by: xdlbdy --- cmd/controller/main.go | 2 +- internal/controller/config.go | 1 + internal/controller/trigger/config.go | 2 + internal/controller/trigger/controller.go | 102 +- .../controller/trigger/controller_test.go | 30 +- internal/controller/trigger/metadata/info.go | 16 +- .../trigger/subscription/mock_subscription.go | 30 + .../controller/trigger/subscription/offset.go | 139 ++ .../trigger/subscription/offset/offset.go | 2 +- .../trigger/subscription/offset_test.go | 184 +++ .../trigger/subscription/subscription.go | 34 +- .../trigger/subscription/subscription_test.go | 49 +- .../controller/trigger/worker/mock_worker.go | 20 +- .../controller/trigger/worker/scheduler.go | 13 +- internal/controller/trigger/worker/worker.go | 36 +- .../controller/trigger/worker/worker_test.go | 19 +- internal/convert/convert.go | 5 +- internal/gateway/proxy/direct.go | 15 + internal/trigger/mock_worker.go | 14 - internal/trigger/reader/mock_reader.go | 17 - internal/trigger/reader/reader.go | 73 +- internal/trigger/reader/reader_test.go | 116 +- internal/trigger/server.go | 16 - internal/trigger/server_test.go | 10 - internal/trigger/trigger/mock_trigger.go | 15 - internal/trigger/trigger/trigger.go | 30 +- internal/trigger/trigger/trigger_test.go | 20 - internal/trigger/worker.go | 45 - internal/trigger/worker_test.go | 33 - pkg/cluster/raw_client/trigger.go | 22 +- proto/pkg/controller/controller.pb.go | 1144 ++++++++++------- proto/pkg/controller/mock_controller.go | 78 +- proto/pkg/proxy/proxy.pb.go | 287 +++-- proto/pkg/trigger/mock_trigger.go | 36 - proto/pkg/trigger/trigger.pb.go | 414 +++--- proto/proto/controller.proto | 21 +- proto/proto/proxy.proto | 6 + proto/proto/trigger.proto | 8 - test/e2e/sink/main.go | 11 +- vsctl/command/flag.go | 1 + vsctl/command/subscription.go | 119 ++ 41 files changed, 1863 insertions(+), 1372 deletions(-) create mode 100644 internal/controller/trigger/subscription/offset.go create mode 100644 internal/controller/trigger/subscription/offset_test.go diff --git a/cmd/controller/main.go b/cmd/controller/main.go index e12ccd013..154bcc407 100644 --- a/cmd/controller/main.go +++ b/cmd/controller/main.go @@ -94,7 +94,7 @@ func main() { } //trigger controller - triggerCtrlStv := trigger.NewController(cfg.GetTriggerConfig(), cfg.GetControllerAddrs(), etcd) + triggerCtrlStv := trigger.NewController(cfg.GetTriggerConfig(), etcd) if err = triggerCtrlStv.Start(); err != nil { log.Error(ctx, "start trigger controller fail", map[string]interface{}{ log.KeyError: err, diff --git a/internal/controller/config.go b/internal/controller/config.go index 3fd4fb90d..6dc759b03 100644 --- a/internal/controller/config.go +++ b/internal/controller/config.go @@ -86,6 +86,7 @@ func (c *Config) GetTriggerConfig() trigger.Config { ServerList: c.EtcdEndpoints, }, SecretEncryptionSalt: c.SecretEncryptionSalt, + ControllerAddr: c.GetControllerAddrs(), } } diff --git a/internal/controller/trigger/config.go b/internal/controller/trigger/config.go index 8471d1c07..810ce39ff 100644 --- a/internal/controller/trigger/config.go +++ b/internal/controller/trigger/config.go @@ -23,4 +23,6 @@ type Config struct { Storage primitive.KvStorageConfig SecretEncryptionSalt string + + ControllerAddr []string } diff --git a/internal/controller/trigger/controller.go b/internal/controller/trigger/controller.go index 150d4e427..863907028 100644 --- a/internal/controller/trigger/controller.go +++ b/internal/controller/trigger/controller.go @@ -17,12 +17,14 @@ package trigger import ( "context" stdErr "errors" + "fmt" "io" "os" "sync" "time" embedetcd "github.com/linkall-labs/embed-etcd" + eb "github.com/linkall-labs/vanus/client" "github.com/linkall-labs/vanus/internal/controller/trigger/metadata" "github.com/linkall-labs/vanus/internal/controller/trigger/secret" "github.com/linkall-labs/vanus/internal/controller/trigger/storage" @@ -51,13 +53,14 @@ const ( defaultGcSubscriptionInterval = time.Second * 10 ) -func NewController(config Config, controllerAddr []string, member embedetcd.Member) *controller { +func NewController(config Config, member embedetcd.Member) *controller { ctrl := &controller{ config: config, member: member, needCleanSubscription: map[vanus.ID]string{}, state: primitive.ServerStateCreated, - cl: cluster.NewClusterController(controllerAddr, insecure.NewCredentials()), + cl: cluster.NewClusterController(config.ControllerAddr, insecure.NewCredentials()), + ebClient: eb.Connect(config.ControllerAddr), } ctrl.ctx, ctrl.stopFunc = context.WithCancel(context.Background()) return ctrl @@ -79,6 +82,7 @@ type controller struct { stopFunc context.CancelFunc state primitive.ServerState cl cluster.Cluster + ebClient eb.Client } func (ctrl *controller) CommitOffset(ctx context.Context, @@ -106,7 +110,7 @@ func (ctrl *controller) CommitOffset(ctx context.Context, } func (ctrl *controller) ResetOffsetToTimestamp(ctx context.Context, - request *ctrlpb.ResetOffsetToTimestampRequest) (*emptypb.Empty, error) { + request *ctrlpb.ResetOffsetToTimestampRequest) (*ctrlpb.ResetOffsetToTimestampResponse, error) { if ctrl.state != primitive.ServerStateRunning { return nil, errors.ErrServerNotStart } @@ -118,18 +122,16 @@ func (ctrl *controller) ResetOffsetToTimestamp(ctx context.Context, if sub == nil { return nil, errors.ErrResourceNotFound.WithMessage("subscription not exist") } - if sub.Phase != metadata.SubscriptionPhaseRunning { - return nil, errors.ErrResourceCanNotOp.WithMessage("subscription is not running") + if sub.Phase != metadata.SubscriptionPhaseStopped { + return nil, errors.ErrResourceCanNotOp.WithMessage("subscription must be disable can reset offset") } - tWorker := ctrl.workerManager.GetTriggerWorker(sub.TriggerWorker) - if tWorker == nil { - return nil, errors.ErrInternal.WithMessage("trigger worker is not running") - } - err := tWorker.ResetOffsetToTimestamp(subID, request.Timestamp) + offsets, err := ctrl.subscriptionManager.ResetOffsetByTimestamp(ctx, subID, request.Timestamp) if err != nil { - return nil, err + return nil, errors.ErrInternal.WithMessage("reset offset by timestamp error").Wrap(err) } - return &emptypb.Empty{}, nil + return &ctrlpb.ResetOffsetToTimestampResponse{ + Offsets: convert.ToPbOffsetInfos(offsets), + }, nil } func (ctrl *controller) CreateSubscription(ctx context.Context, @@ -151,12 +153,18 @@ func (ctrl *controller) CreateSubscription(ctx context.Context, if err != nil { return nil, err } - sub.Phase = metadata.SubscriptionPhaseCreated + if request.Subscription.Disable { + sub.Phase = metadata.SubscriptionPhaseStopped + } else { + sub.Phase = metadata.SubscriptionPhaseCreated + } err = ctrl.subscriptionManager.AddSubscription(ctx, sub) if err != nil { return nil, err } - ctrl.scheduler.EnqueueNormalSubscription(sub.ID) + if !request.Subscription.Disable { + ctrl.scheduler.EnqueueNormalSubscription(sub.ID) + } resp := convert.ToPbSubscription(sub, nil) return resp, nil } @@ -171,6 +179,9 @@ func (ctrl *controller) UpdateSubscription(ctx context.Context, if sub == nil { return nil, errors.ErrResourceNotFound.WithMessage("subscription not exist") } + if sub.Phase != metadata.SubscriptionPhaseStopped { + return nil, errors.ErrResourceCanNotOp.WithMessage("subscription must be disabled can update") + } if err := validation.ValidateSubscriptionRequest(ctx, request.Subscription); err != nil { return nil, err } @@ -194,14 +205,12 @@ func (ctrl *controller) UpdateSubscription(ctx context.Context, return nil, errors.ErrInvalidRequest.WithMessage("no change") } sub.UpdatedAt = time.Now() - sub.Phase = metadata.SubscriptionPhasePending if err := ctrl.subscriptionManager.UpdateSubscription(ctx, sub); err != nil { return nil, err } if transChange != 0 { metrics.SubscriptionTransformerGauge.WithLabelValues(sub.EventBus).Add(float64(transChange)) } - ctrl.scheduler.EnqueueNormalSubscription(sub.ID) resp := convert.ToPbSubscription(sub, nil) return resp, nil } @@ -231,6 +240,53 @@ func (ctrl *controller) DeleteSubscription(ctx context.Context, return &emptypb.Empty{}, nil } +func (ctrl *controller) DisableSubscription(ctx context.Context, + request *ctrlpb.DisableSubscriptionRequest) (*emptypb.Empty, error) { + if ctrl.state != primitive.ServerStateRunning { + return nil, errors.ErrServerNotStart + } + subID := vanus.ID(request.Id) + sub := ctrl.subscriptionManager.GetSubscription(ctx, subID) + if sub == nil { + return nil, errors.ErrResourceNotFound.WithMessage(fmt.Sprintf("subscrption %d not exist", subID)) + } + if sub.Phase == metadata.SubscriptionPhaseStopped { + return nil, errors.ErrResourceCanNotOp.WithMessage("subscription is disable") + } + if sub.Phase == metadata.SubscriptionPhaseStopping { + return nil, errors.ErrResourceCanNotOp.WithMessage("subscription is disabling") + } + sub.Phase = metadata.SubscriptionPhaseStopping + err := ctrl.subscriptionManager.UpdateSubscription(ctx, sub) + if err != nil { + return nil, err + } + ctrl.scheduler.EnqueueSubscription(sub.ID) + return &emptypb.Empty{}, nil +} + +func (ctrl *controller) ResumeSubscription(ctx context.Context, + request *ctrlpb.ResumeSubscriptionRequest) (*emptypb.Empty, error) { + if ctrl.state != primitive.ServerStateRunning { + return nil, errors.ErrServerNotStart + } + subID := vanus.ID(request.Id) + sub := ctrl.subscriptionManager.GetSubscription(ctx, subID) + if sub == nil { + return nil, errors.ErrResourceNotFound.WithMessage(fmt.Sprintf("subscrption %d not exist", subID)) + } + if sub.Phase != metadata.SubscriptionPhaseStopped { + return nil, errors.ErrResourceCanNotOp.WithMessage("subscription is not disable") + } + sub.Phase = metadata.SubscriptionPhasePending + err := ctrl.subscriptionManager.UpdateSubscription(ctx, sub) + if err != nil { + return nil, err + } + ctrl.scheduler.EnqueueSubscription(sub.ID) + return &emptypb.Empty{}, nil +} + func (ctrl *controller) GetSubscription(ctx context.Context, request *ctrlpb.GetSubscriptionRequest) (*meta.Subscription, error) { if ctrl.state != primitive.ServerStateRunning { @@ -360,7 +416,10 @@ func (ctrl *controller) ListSubscription(ctx context.Context, func (ctrl *controller) gcSubscription(ctx context.Context, id vanus.ID, addr string) error { tWorker := ctrl.workerManager.GetTriggerWorker(addr) if tWorker != nil { - tWorker.UnAssignSubscription(id) + err := tWorker.UnAssignSubscription(id) + if err != nil { + return err + } } err := ctrl.subscriptionManager.DeleteSubscription(ctx, id) if err != nil { @@ -420,7 +479,7 @@ func (ctrl *controller) init(ctx context.Context) error { switch sub.Phase { case metadata.SubscriptionPhaseCreated: ctrl.scheduler.EnqueueNormalSubscription(sub.ID) - case metadata.SubscriptionPhasePending: + case metadata.SubscriptionPhasePending, metadata.SubscriptionPhaseStopping: ctrl.scheduler.EnqueueSubscription(sub.ID) case metadata.SubscriptionPhaseToDelete: ctrl.needCleanSubscription[sub.ID] = sub.TriggerWorker @@ -438,7 +497,7 @@ func (ctrl *controller) membershipChangedProcessor(ctx context.Context, if ctrl.isLeader { return nil } - log.Info(context.TODO(), "become leader", nil) + log.Info(context.TODO(), "trigger become leader", nil) err := ctrl.init(ctx) if err != nil { _err := ctrl.stop(ctx) @@ -447,6 +506,9 @@ func (ctrl *controller) membershipChangedProcessor(ctx context.Context, log.KeyError: _err, }) } + log.Error(ctx, "controller init has error", map[string]interface{}{ + log.KeyError: err, + }) return err } ctrl.workerManager.Start() @@ -493,7 +555,7 @@ func (ctrl *controller) Start() error { return err } ctrl.secretStorage = secretStorage - ctrl.subscriptionManager = subscription.NewSubscriptionManager(ctrl.storage, ctrl.secretStorage) + ctrl.subscriptionManager = subscription.NewSubscriptionManager(ctrl.storage, ctrl.secretStorage, ctrl.ebClient) ctrl.workerManager = worker.NewTriggerWorkerManager(worker.Config{}, ctrl.storage, ctrl.subscriptionManager, ctrl.requeueSubscription) ctrl.scheduler = worker.NewSubscriptionScheduler(ctrl.workerManager, ctrl.subscriptionManager) diff --git a/internal/controller/trigger/controller_test.go b/internal/controller/trigger/controller_test.go index bbab6edbf..1b865242e 100644 --- a/internal/controller/trigger/controller_test.go +++ b/internal/controller/trigger/controller_test.go @@ -38,7 +38,7 @@ func TestController_CommitOffset(t *testing.T) { Convey("test reset offset", t, func() { mockCtrl := gomock.NewController(t) defer mockCtrl.Finish() - ctrl := NewController(Config{}, nil, nil) + ctrl := NewController(Config{}, nil) ctx := context.Background() workerManager := worker.NewMockManager(mockCtrl) ctrl.workerManager = workerManager @@ -77,14 +77,13 @@ func TestController_ResetOffsetToTimestamp(t *testing.T) { Convey("test reset offset", t, func() { mockCtrl := gomock.NewController(t) defer mockCtrl.Finish() - ctrl := NewController(Config{}, nil, nil) + ctrl := NewController(Config{}, nil) ctx := context.Background() workerManager := worker.NewMockManager(mockCtrl) ctrl.workerManager = workerManager subManager := subscription.NewMockManager(mockCtrl) ctrl.subscriptionManager = subManager - addr := "test" subID := vanus.NewTestID() ctrl.state = primitive.ServerStateRunning Convey("reset offset subscription not exist", func() { @@ -97,14 +96,11 @@ func TestController_ResetOffsetToTimestamp(t *testing.T) { }) Convey("reset offset subscription exist", func() { sub := &metadata.Subscription{ - ID: subID, - Phase: metadata.SubscriptionPhaseRunning, - TriggerWorker: addr, + ID: subID, + Phase: metadata.SubscriptionPhaseStopped, } subManager.EXPECT().GetSubscription(gomock.Any(), gomock.Eq(subID)).AnyTimes().Return(sub) - tWorker := worker.NewMockTriggerWorker(mockCtrl) - tWorker.EXPECT().ResetOffsetToTimestamp(gomock.Eq(subID), gomock.Any()).Return(nil) - workerManager.EXPECT().GetTriggerWorker(addr).Return(tWorker) + subManager.EXPECT().ResetOffsetByTimestamp(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, nil) _, err := ctrl.ResetOffsetToTimestamp(ctx, &ctrlpb.ResetOffsetToTimestampRequest{ SubscriptionId: subID.Uint64(), Timestamp: uint64(time.Now().Unix()), @@ -118,7 +114,7 @@ func TestController_CreateSubscription(t *testing.T) { Convey("test create subscription", t, func() { mockCtrl := gomock.NewController(t) defer mockCtrl.Finish() - ctrl := NewController(Config{}, nil, nil) + ctrl := NewController(Config{}, nil) ctx := context.Background() workerManager := worker.NewMockManager(mockCtrl) ctrl.workerManager = workerManager @@ -154,7 +150,7 @@ func TestController_UpdateSubscription(t *testing.T) { Convey("test update subscription", t, func() { mockCtrl := gomock.NewController(t) defer mockCtrl.Finish() - ctrl := NewController(Config{}, nil, nil) + ctrl := NewController(Config{}, nil) ctx := context.Background() workerManager := worker.NewMockManager(mockCtrl) ctrl.workerManager = workerManager @@ -177,7 +173,7 @@ func TestController_UpdateSubscription(t *testing.T) { }) sub := metadata.Subscription{ ID: subID, - Phase: metadata.SubscriptionPhaseRunning, + Phase: metadata.SubscriptionPhaseStopped, TriggerWorker: "test-addr", EventBus: "test-eb", Sink: "test-sink", @@ -355,7 +351,7 @@ func TestController_DeleteSubscription(t *testing.T) { Convey("test delete subscription", t, func() { mockCtrl := gomock.NewController(t) defer mockCtrl.Finish() - ctrl := NewController(Config{}, nil, nil) + ctrl := NewController(Config{}, nil) ctx := context.Background() workerManager := worker.NewMockManager(mockCtrl) ctrl.workerManager = workerManager @@ -383,7 +379,7 @@ func TestController_DeleteSubscription(t *testing.T) { subManager.EXPECT().GetSubscription(gomock.Any(), gomock.Eq(subID)).Return(sub) subManager.EXPECT().UpdateSubscription(gomock.Any(), gomock.Any()).Return(nil) workerManager.EXPECT().GetTriggerWorker(addr).Return(tWorker) - tWorker.EXPECT().UnAssignSubscription(gomock.Eq(subID)).Return() + tWorker.EXPECT().UnAssignSubscription(gomock.Eq(subID)).Return(nil) Convey("delete subscription success", func() { subManager.EXPECT().DeleteSubscription(gomock.Any(), gomock.Eq(subID)).Return(nil) _, err := ctrl.DeleteSubscription(ctx, request) @@ -414,7 +410,7 @@ func TestController_GetSubscription(t *testing.T) { Convey("test get subscription", t, func() { mockCtrl := gomock.NewController(t) defer mockCtrl.Finish() - ctrl := NewController(Config{}, nil, nil) + ctrl := NewController(Config{}, nil) ctx := context.Background() workerManager := worker.NewMockManager(mockCtrl) ctrl.workerManager = workerManager @@ -451,7 +447,7 @@ func TestController_ListSubscription(t *testing.T) { Convey("test list subscription", t, func() { mockCtrl := gomock.NewController(t) defer mockCtrl.Finish() - ctrl := NewController(Config{}, nil, nil) + ctrl := NewController(Config{}, nil) ctx := context.Background() workerManager := worker.NewMockManager(mockCtrl) ctrl.workerManager = workerManager @@ -476,7 +472,7 @@ func TestController_TriggerWorkerHeartbeat(t *testing.T) { Convey("test trigger worker heartbeat", t, func() { mockCtrl := gomock.NewController(t) defer mockCtrl.Finish() - ctrl := NewController(Config{}, nil, nil) + ctrl := NewController(Config{}, nil) ctx := context.Background() workerManager := worker.NewMockManager(mockCtrl) ctrl.workerManager = workerManager diff --git a/internal/controller/trigger/metadata/info.go b/internal/controller/trigger/metadata/info.go index 6c5ce1d95..cf8fed938 100644 --- a/internal/controller/trigger/metadata/info.go +++ b/internal/controller/trigger/metadata/info.go @@ -55,11 +55,12 @@ func (tw *TriggerWorkerInfo) String() string { type SubscriptionPhase string const ( - SubscriptionPhaseCreated = "created" - SubscriptionPhasePending = "pending" - SubscriptionPhaseScheduled = "scheduled" - SubscriptionPhaseRunning = "running" - SubscriptionPhaseToDelete = "toDelete" + SubscriptionPhaseCreated = "created" + SubscriptionPhasePending = "pending" + SubscriptionPhaseRunning = "running" + SubscriptionPhaseStopping = "stopping" + SubscriptionPhaseStopped = "stopped" + SubscriptionPhaseToDelete = "toDelete" ) type Subscription struct { @@ -76,7 +77,6 @@ type Subscription struct { EventBus string `json:"eventbus"` Transformer *primitive.Transformer `json:"transformer,omitempty"` Name string `json:"name"` - Disable bool `json:"disable"` Description string `json:"description"` CreatedAt time.Time `json:"created_at"` UpdatedAt time.Time `json:"updated_at"` @@ -98,10 +98,6 @@ func (s *Subscription) Update(update *Subscription) bool { change = true s.Description = update.Description } - if s.Disable != update.Disable { - change = true - s.Disable = update.Disable - } if !reflect.DeepEqual(s.Types, update.Types) { change = true s.Types = update.Types diff --git a/internal/controller/trigger/subscription/mock_subscription.go b/internal/controller/trigger/subscription/mock_subscription.go index 6b9c47e81..bfee26f78 100644 --- a/internal/controller/trigger/subscription/mock_subscription.go +++ b/internal/controller/trigger/subscription/mock_subscription.go @@ -81,6 +81,21 @@ func (mr *MockManagerMockRecorder) GetOffset(ctx, id interface{}) *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetOffset", reflect.TypeOf((*MockManager)(nil).GetOffset), ctx, id) } +// GetOrSaveOffset mocks base method. +func (m *MockManager) GetOrSaveOffset(ctx context.Context, id vanus.ID) (info.ListOffsetInfo, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetOrSaveOffset", ctx, id) + ret0, _ := ret[0].(info.ListOffsetInfo) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetOrSaveOffset indicates an expected call of GetOrSaveOffset. +func (mr *MockManagerMockRecorder) GetOrSaveOffset(ctx, id interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetOrSaveOffset", reflect.TypeOf((*MockManager)(nil).GetOrSaveOffset), ctx, id) +} + // GetSubscription mocks base method. func (m *MockManager) GetSubscription(ctx context.Context, id vanus.ID) *metadata.Subscription { m.ctrl.T.Helper() @@ -137,6 +152,21 @@ func (mr *MockManagerMockRecorder) ListSubscription(ctx interface{}) *gomock.Cal return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ListSubscription", reflect.TypeOf((*MockManager)(nil).ListSubscription), ctx) } +// ResetOffsetByTimestamp mocks base method. +func (m *MockManager) ResetOffsetByTimestamp(ctx context.Context, id vanus.ID, timestamp uint64) (info.ListOffsetInfo, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ResetOffsetByTimestamp", ctx, id, timestamp) + ret0, _ := ret[0].(info.ListOffsetInfo) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ResetOffsetByTimestamp indicates an expected call of ResetOffsetByTimestamp. +func (mr *MockManagerMockRecorder) ResetOffsetByTimestamp(ctx, id, timestamp interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetOffsetByTimestamp", reflect.TypeOf((*MockManager)(nil).ResetOffsetByTimestamp), ctx, id, timestamp) +} + // SaveOffset mocks base method. func (m *MockManager) SaveOffset(ctx context.Context, id vanus.ID, offsets info.ListOffsetInfo, commit bool) error { m.ctrl.T.Helper() diff --git a/internal/controller/trigger/subscription/offset.go b/internal/controller/trigger/subscription/offset.go new file mode 100644 index 000000000..ecd924619 --- /dev/null +++ b/internal/controller/trigger/subscription/offset.go @@ -0,0 +1,139 @@ +// 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 subscription + +import ( + "context" + + "github.com/linkall-labs/vanus/internal/primitive" + "github.com/linkall-labs/vanus/internal/primitive/info" + "github.com/linkall-labs/vanus/internal/primitive/vanus" + "github.com/linkall-labs/vanus/observability/log" +) + +func (m *manager) SaveOffset(ctx context.Context, id vanus.ID, offsets info.ListOffsetInfo, commit bool) error { + subscription := m.GetSubscription(ctx, id) + if subscription == nil { + return nil + } + return m.offsetManager.Offset(ctx, id, offsets, commit) +} + +func (m *manager) ResetOffsetByTimestamp(ctx context.Context, id vanus.ID, + timestamp uint64) (info.ListOffsetInfo, error) { + subscription := m.GetSubscription(ctx, id) + if subscription == nil { + return nil, ErrSubscriptionNotExist + } + offsets, err := m.getOffsetFromCli(ctx, subscription.EventBus, primitive.SubscriptionConfig{ + OffsetTimestamp: ×tamp, + OffsetType: primitive.Timestamp, + }, false) + if err != nil { + return nil, err + } + err = m.offsetManager.Offset(ctx, id, offsets, true) + if err != nil { + return nil, err + } + log.Info(ctx, "reset offset by timestamp", map[string]interface{}{ + "offsets": offsets, + }) + return offsets, err +} + +func (m *manager) GetOffset(ctx context.Context, id vanus.ID) (info.ListOffsetInfo, error) { + subscription := m.GetSubscription(ctx, id) + if subscription == nil { + return info.ListOffsetInfo{}, ErrSubscriptionNotExist + } + offsets, err := m.offsetManager.GetOffset(ctx, id) + if err != nil { + return nil, err + } + // todo filter retry eventlog + return offsets, nil +} + +func (m *manager) GetOrSaveOffset(ctx context.Context, id vanus.ID) (info.ListOffsetInfo, error) { + subscription := m.GetSubscription(ctx, id) + if subscription == nil { + return info.ListOffsetInfo{}, ErrSubscriptionNotExist + } + offsets, err := m.offsetManager.GetOffset(ctx, id) + if err != nil { + return nil, err + } + if len(offsets) > 0 { + return offsets, nil + } + + offsets, err = m.getOffsetFromCli(ctx, subscription.EventBus, subscription.Config, false) + if err != nil { + return nil, err + } + // get retry eb offset. + retryOffset, err := m.getOffsetFromCli(ctx, primitive.RetryEventbusName, primitive.SubscriptionConfig{ + OffsetType: primitive.LatestOffset, + }, true) + if err != nil { + return nil, err + } + offsets = append(offsets, retryOffset...) + err = m.offsetManager.Offset(ctx, id, offsets, true) + if err != nil { + return nil, err + } + log.Info(ctx, "save offset from cli", map[string]interface{}{ + "offsets": offsets, + }) + return offsets, nil +} + +func (m *manager) getOffsetFromCli(ctx context.Context, eventbus string, + config primitive.SubscriptionConfig, retryEventBus bool) (info.ListOffsetInfo, error) { + logs, err := m.ebCli.Eventbus(ctx, eventbus).ListLog(ctx) + if err != nil { + return nil, err + } + offsets := make(info.ListOffsetInfo, len(logs)) + for i, l := range logs { + var v int64 + switch config.OffsetType { + case primitive.EarliestOffset: + if v, err = l.EarliestOffset(ctx); err != nil { + return nil, err + } + case primitive.Timestamp: + t := config.OffsetTimestamp + if v, err = l.QueryOffsetByTime(ctx, int64(*t)); err != nil { + return nil, err + } + default: + if v, err = l.LatestOffset(ctx); err != nil { + return nil, err + } + } + // fix offset is negative which convert to uint64 is big. + if v < 0 { + v = 0 + } + offsets[i] = info.OffsetInfo{ + EventLogID: vanus.NewIDFromUint64(l.ID()), + Offset: uint64(v), + } + } + return offsets, nil +} diff --git a/internal/controller/trigger/subscription/offset/offset.go b/internal/controller/trigger/subscription/offset/offset.go index 2f8e2eca0..7a2b910c7 100644 --- a/internal/controller/trigger/subscription/offset/offset.go +++ b/internal/controller/trigger/subscription/offset/offset.go @@ -255,7 +255,7 @@ func (o *eventLogOffset) commitOffset(ctx context.Context, storage storage.Offse if err != nil { return err } - log.Debug(ctx, "create offset", map[string]interface{}{ + log.Info(ctx, "create offset", map[string]interface{}{ log.KeySubscriptionID: o.subscriptionID, log.KeyEventlogID: o.eventLogID, "offset": offset, diff --git a/internal/controller/trigger/subscription/offset_test.go b/internal/controller/trigger/subscription/offset_test.go new file mode 100644 index 000000000..fd1a51964 --- /dev/null +++ b/internal/controller/trigger/subscription/offset_test.go @@ -0,0 +1,184 @@ +// 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 subscription + +import ( + "context" + "testing" + "time" + + "github.com/golang/mock/gomock" + "github.com/linkall-labs/vanus/client" + "github.com/linkall-labs/vanus/client/pkg/api" + "github.com/linkall-labs/vanus/internal/controller/trigger/metadata" + "github.com/linkall-labs/vanus/internal/controller/trigger/secret" + "github.com/linkall-labs/vanus/internal/controller/trigger/storage" + "github.com/linkall-labs/vanus/internal/controller/trigger/subscription/offset" + "github.com/linkall-labs/vanus/internal/primitive" + "github.com/linkall-labs/vanus/internal/primitive/info" + "github.com/linkall-labs/vanus/internal/primitive/vanus" + . "github.com/smartystreets/goconvey/convey" +) + +func TestSaveOffset(t *testing.T) { + Convey("test save offset", t, func() { + ctx := context.Background() + ctrl := gomock.NewController(t) + defer ctrl.Finish() + storage := storage.NewMockStorage(ctrl) + secret := secret.NewMockStorage(ctrl) + + m := NewSubscriptionManager(storage, secret, nil).(*manager) + offsetManager := offset.NewMockManager(ctrl) + m.offsetManager = offsetManager + storage.MockSubscriptionStorage.EXPECT().CreateSubscription(ctx, gomock.Any()).AnyTimes().Return(nil) + noExistID := vanus.NewTestID() + id := vanus.NewTestID() + sub := &metadata.Subscription{ + ID: id, + } + m.AddSubscription(ctx, sub) + logID := vanus.NewTestID() + offsetV := uint64(10) + listOffsetInfo := info.ListOffsetInfo{ + {EventLogID: logID, Offset: offsetV}, + } + Convey("save offset subscription no exist", func() { + err := m.SaveOffset(ctx, noExistID, listOffsetInfo, false) + So(err, ShouldBeNil) + }) + Convey("save offset valid", func() { + offsetManager.EXPECT().Offset(ctx, id, listOffsetInfo, false).Return(nil) + err := m.SaveOffset(ctx, id, listOffsetInfo, false) + So(err, ShouldBeNil) + }) + }) +} + +func TestGetOffset(t *testing.T) { + Convey("test get offset", t, func() { + ctx := context.Background() + ctrl := gomock.NewController(t) + defer ctrl.Finish() + storage := storage.NewMockStorage(ctrl) + secret := secret.NewMockStorage(ctrl) + mockClient := client.NewMockClient(ctrl) + + m := NewSubscriptionManager(storage, secret, mockClient).(*manager) + offsetManager := offset.NewMockManager(ctrl) + m.offsetManager = offsetManager + storage.MockSubscriptionStorage.EXPECT().CreateSubscription(ctx, gomock.Any()).AnyTimes().Return(nil) + storage.MockSubscriptionStorage.EXPECT().UpdateSubscription(ctx, gomock.Any()).AnyTimes().Return(nil) + noExistID := vanus.NewTestID() + id := vanus.NewTestID() + sub := &metadata.Subscription{ + ID: id, + } + m.AddSubscription(ctx, sub) + logID := vanus.NewTestID() + offsetV := uint64(10) + listOffsetInfo := info.ListOffsetInfo{ + {EventLogID: logID, Offset: offsetV}, + } + Convey("get offset subscription no exist", func() { + offsets, err := m.GetOffset(ctx, noExistID) + So(err, ShouldNotBeNil) + So(len(offsets), ShouldEqual, 0) + }) + Convey("get offset from storage offset", func() { + offsetManager.EXPECT().GetOffset(ctx, id).Return(listOffsetInfo, nil) + offsets, err := m.GetOffset(ctx, id) + So(err, ShouldBeNil) + So(len(offsets), ShouldEqual, len(listOffsetInfo)) + So(offsets[0].EventLogID, ShouldEqual, logID) + So(offsets[0].Offset, ShouldEqual, offsetV) + }) + Convey("get offset from client", func() { + offsetManager.EXPECT().GetOffset(ctx, id).AnyTimes().Return(nil, nil) + offsets, err := m.GetOffset(ctx, id) + So(err, ShouldBeNil) + So(len(offsets), ShouldEqual, 0) + offsetManager.EXPECT().Offset(gomock.Any(), gomock.Any(), gomock.Any(), true).AnyTimes().Return(nil) + mockEventbus := api.NewMockEventbus(ctrl) + mockEventLog := api.NewMockEventlog(ctrl) + mockClient.EXPECT().Eventbus(gomock.Any(), gomock.Any()).AnyTimes().Return(mockEventbus) + mockEventbus.EXPECT().ListLog(gomock.Any()).AnyTimes().Return([]api.Eventlog{mockEventLog}, nil) + mockEventLog.EXPECT().ID().AnyTimes().Return(logID.Uint64()) + mockEventLog.EXPECT().LatestOffset(gomock.Any()).AnyTimes().Return(int64(offsetV), nil) + Convey("test get offset from latest", func() { + offsets, err = m.GetOrSaveOffset(ctx, id) + So(err, ShouldBeNil) + So(len(offsets), ShouldEqual, 2*len(listOffsetInfo)) + So(offsets[0].EventLogID, ShouldEqual, logID) + So(offsets[0].Offset, ShouldEqual, offsetV) + }) + Convey("test get offset from earliest", func() { + sub.Config.OffsetType = primitive.EarliestOffset + m.UpdateSubscription(ctx, sub) + mockEventLog.EXPECT().EarliestOffset(gomock.Any()).Return(int64(offsetV), nil) + offsets, err = m.GetOrSaveOffset(ctx, id) + So(err, ShouldBeNil) + So(len(offsets), ShouldEqual, 2*len(listOffsetInfo)) + So(offsets[0].EventLogID, ShouldEqual, logID) + So(offsets[0].Offset, ShouldEqual, offsetV) + }) + Convey("test get offset from timestamp", func() { + sub.Config.OffsetType = primitive.Timestamp + time := uint64(time.Now().Unix()) + sub.Config.OffsetTimestamp = &time + m.UpdateSubscription(ctx, sub) + mockEventLog.EXPECT().QueryOffsetByTime(gomock.Any(), int64(time)).Return(int64(offsetV), nil) + offsets, err = m.GetOrSaveOffset(ctx, id) + So(err, ShouldBeNil) + So(len(offsets), ShouldEqual, 2*len(listOffsetInfo)) + So(offsets[0].EventLogID, ShouldEqual, logID) + So(offsets[0].Offset, ShouldEqual, offsetV) + }) + }) + }) +} + +func TestResetOffsetByTimestamp(t *testing.T) { + Convey("test reset offset by timestamp", t, func() { + ctx := context.Background() + ctrl := gomock.NewController(t) + defer ctrl.Finish() + storage := storage.NewMockStorage(ctrl) + secret := secret.NewMockStorage(ctrl) + mockClient := client.NewMockClient(ctrl) + + m := NewSubscriptionManager(storage, secret, mockClient).(*manager) + offsetManager := offset.NewMockManager(ctrl) + m.offsetManager = offsetManager + storage.MockSubscriptionStorage.EXPECT().CreateSubscription(ctx, gomock.Any()).AnyTimes().Return(nil) + id := vanus.NewTestID() + sub := &metadata.Subscription{ + ID: id, + } + m.AddSubscription(ctx, sub) + logID := vanus.NewTestID() + + offsetManager.EXPECT().Offset(ctx, id, gomock.Any(), true).Return(nil) + mockEventbus := api.NewMockEventbus(ctrl) + mockEventLog := api.NewMockEventlog(ctrl) + mockClient.EXPECT().Eventbus(gomock.Any(), gomock.Any()).AnyTimes().Return(mockEventbus) + mockEventbus.EXPECT().ListLog(gomock.Any()).AnyTimes().Return([]api.Eventlog{mockEventLog}, nil) + mockEventLog.EXPECT().ID().AnyTimes().Return(logID.Uint64()) + time := uint64(time.Now().Unix()) + mockEventLog.EXPECT().QueryOffsetByTime(gomock.Any(), int64(time)).Return(int64(100), nil) + _, err := m.ResetOffsetByTimestamp(ctx, id, time) + So(err, ShouldBeNil) + }) +} diff --git a/internal/controller/trigger/subscription/subscription.go b/internal/controller/trigger/subscription/subscription.go index 7cbcbb8a3..52fbe5238 100644 --- a/internal/controller/trigger/subscription/subscription.go +++ b/internal/controller/trigger/subscription/subscription.go @@ -22,19 +22,25 @@ import ( "sync" "time" + eb "github.com/linkall-labs/vanus/client" "github.com/linkall-labs/vanus/internal/controller/trigger/metadata" "github.com/linkall-labs/vanus/internal/controller/trigger/secret" "github.com/linkall-labs/vanus/internal/controller/trigger/storage" "github.com/linkall-labs/vanus/internal/controller/trigger/subscription/offset" - iInfo "github.com/linkall-labs/vanus/internal/primitive/info" + "github.com/linkall-labs/vanus/internal/primitive/info" "github.com/linkall-labs/vanus/internal/primitive/vanus" "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/observability/metrics" ) type Manager interface { - SaveOffset(ctx context.Context, id vanus.ID, offsets iInfo.ListOffsetInfo, commit bool) error - GetOffset(ctx context.Context, id vanus.ID) (iInfo.ListOffsetInfo, error) + SaveOffset(ctx context.Context, id vanus.ID, offsets info.ListOffsetInfo, commit bool) error + // GetOrSaveOffset get offset only from etcd, if it isn't exist will get from cli and save to etcd, + // and it contains retry eb offset + GetOrSaveOffset(ctx context.Context, id vanus.ID) (info.ListOffsetInfo, error) + // GetOffset get offset only from etcd, it doesn't contain retry eb offset + GetOffset(ctx context.Context, id vanus.ID) (info.ListOffsetInfo, error) + ResetOffsetByTimestamp(ctx context.Context, id vanus.ID, timestamp uint64) (info.ListOffsetInfo, error) ListSubscription(ctx context.Context) []*metadata.Subscription GetSubscription(ctx context.Context, id vanus.ID) *metadata.Subscription AddSubscription(ctx context.Context, subscription *metadata.Subscription) error @@ -55,6 +61,7 @@ const ( ) type manager struct { + ebCli eb.Client secretStorage secret.Storage storage storage.Storage offsetManager offset.Manager @@ -62,8 +69,9 @@ type manager struct { subscriptionMap map[vanus.ID]*metadata.Subscription } -func NewSubscriptionManager(storage storage.Storage, secretStorage secret.Storage) Manager { +func NewSubscriptionManager(storage storage.Storage, secretStorage secret.Storage, ebCli eb.Client) Manager { m := &manager{ + ebCli: ebCli, storage: storage, secretStorage: secretStorage, subscriptionMap: map[vanus.ID]*metadata.Subscription{}, @@ -72,23 +80,7 @@ func NewSubscriptionManager(storage storage.Storage, secretStorage secret.Storag return m } -func (m *manager) SaveOffset(ctx context.Context, id vanus.ID, offsets iInfo.ListOffsetInfo, commit bool) error { - subscription := m.GetSubscription(ctx, id) - if subscription == nil { - return nil - } - return m.offsetManager.Offset(ctx, id, offsets, commit) -} - -func (m *manager) GetOffset(ctx context.Context, id vanus.ID) (iInfo.ListOffsetInfo, error) { - subscription := m.GetSubscription(ctx, id) - if subscription == nil { - return iInfo.ListOffsetInfo{}, ErrSubscriptionNotExist - } - return m.offsetManager.GetOffset(ctx, id) -} - -func (m *manager) ListSubscription(ctx context.Context) []*metadata.Subscription { +func (m *manager) ListSubscription(_ context.Context) []*metadata.Subscription { m.lock.RLock() defer m.lock.RUnlock() list := make([]*metadata.Subscription, 0, len(m.subscriptionMap)) diff --git a/internal/controller/trigger/subscription/subscription_test.go b/internal/controller/trigger/subscription/subscription_test.go index 8dea9d143..2d1b5bbfc 100644 --- a/internal/controller/trigger/subscription/subscription_test.go +++ b/internal/controller/trigger/subscription/subscription_test.go @@ -25,7 +25,6 @@ import ( "github.com/linkall-labs/vanus/internal/controller/trigger/storage" "github.com/linkall-labs/vanus/internal/controller/trigger/subscription/offset" "github.com/linkall-labs/vanus/internal/primitive" - "github.com/linkall-labs/vanus/internal/primitive/info" "github.com/linkall-labs/vanus/internal/primitive/vanus" . "github.com/smartystreets/goconvey/convey" ) @@ -36,7 +35,7 @@ func TestSubscriptionInit(t *testing.T) { defer ctrl.Finish() storage := storage.NewMockStorage(ctrl) secret := secret.NewMockStorage(ctrl) - m := NewSubscriptionManager(storage, secret) + m := NewSubscriptionManager(storage, secret, nil) Convey("init ", t, func() { subID := vanus.NewTestID() @@ -63,7 +62,7 @@ func TestGetListSubscription(t *testing.T) { defer ctrl.Finish() storage := storage.NewMockStorage(ctrl) secret := secret.NewMockStorage(ctrl) - m := NewSubscriptionManager(storage, secret) + m := NewSubscriptionManager(storage, secret, nil) id := vanus.NewTestID() Convey("list subscription size 0", func() { subscriptions := m.ListSubscription(ctx) @@ -104,7 +103,7 @@ func TestSubscription(t *testing.T) { defer ctrl.Finish() storage := storage.NewMockStorage(ctrl) secret := secret.NewMockStorage(ctrl) - m := NewSubscriptionManager(storage, secret) + m := NewSubscriptionManager(storage, secret, nil) subID := vanus.NewTestID() Convey("test add subscription", func() { storage.MockSubscriptionStorage.EXPECT().CreateSubscription(ctx, gomock.Any()).Return(nil) @@ -168,45 +167,3 @@ func TestSubscription(t *testing.T) { }) }) } - -func TestOffset(t *testing.T) { - Convey("test offset", t, func() { - ctx := context.Background() - ctrl := gomock.NewController(t) - defer ctrl.Finish() - storage := storage.NewMockStorage(ctrl) - secret := secret.NewMockStorage(ctrl) - m := NewSubscriptionManager(storage, secret).(*manager) - offsetManager := offset.NewMockManager(ctrl) - m.offsetManager = offsetManager - storage.MockSubscriptionStorage.EXPECT().CreateSubscription(ctx, gomock.Any()).Return(nil) - m.AddSubscription(ctx, &metadata.Subscription{}) - subscriptions := m.ListSubscription(ctx) - id := subscriptions[0].ID - listOffsetInfo := info.ListOffsetInfo{ - {EventLogID: 1, Offset: 10}, - } - Convey("set offset subscription no exist", func() { - err := m.SaveOffset(ctx, 1, listOffsetInfo, false) - So(err, ShouldBeNil) - }) - Convey("set offset ", func() { - offsetManager.EXPECT().Offset(ctx, id, listOffsetInfo, false).Return(nil) - err := m.SaveOffset(ctx, id, listOffsetInfo, false) - So(err, ShouldBeNil) - }) - Convey("get offset subscription no exist", func() { - offsets, err := m.GetOffset(ctx, 1) - So(err, ShouldNotBeNil) - So(len(offsets), ShouldEqual, 0) - }) - Convey("get offset", func() { - offsetManager.EXPECT().GetOffset(ctx, id).Return(listOffsetInfo, nil) - offsets, err := m.GetOffset(ctx, id) - So(err, ShouldBeNil) - So(len(offsets), ShouldEqual, 1) - So(offsets[0].EventLogID, ShouldEqual, 1) - So(offsets[0].Offset, ShouldEqual, 10) - }) - }) -} diff --git a/internal/controller/trigger/worker/mock_worker.go b/internal/controller/trigger/worker/mock_worker.go index 5788c707c..c6d6247a2 100644 --- a/internal/controller/trigger/worker/mock_worker.go +++ b/internal/controller/trigger/worker/mock_worker.go @@ -213,20 +213,6 @@ func (mr *MockTriggerWorkerMockRecorder) Reset() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Reset", reflect.TypeOf((*MockTriggerWorker)(nil).Reset)) } -// ResetOffsetToTimestamp mocks base method. -func (m *MockTriggerWorker) ResetOffsetToTimestamp(id vanus.ID, timestamp uint64) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ResetOffsetToTimestamp", id, timestamp) - ret0, _ := ret[0].(error) - return ret0 -} - -// ResetOffsetToTimestamp indicates an expected call of ResetOffsetToTimestamp. -func (mr *MockTriggerWorkerMockRecorder) ResetOffsetToTimestamp(id, timestamp interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetOffsetToTimestamp", reflect.TypeOf((*MockTriggerWorker)(nil).ResetOffsetToTimestamp), id, timestamp) -} - // SetPhase mocks base method. func (m *MockTriggerWorker) SetPhase(arg0 metadata.TriggerWorkerPhase) { m.ctrl.T.Helper() @@ -254,9 +240,11 @@ func (mr *MockTriggerWorkerMockRecorder) Start(ctx interface{}) *gomock.Call { } // UnAssignSubscription mocks base method. -func (m *MockTriggerWorker) UnAssignSubscription(id vanus.ID) { +func (m *MockTriggerWorker) UnAssignSubscription(id vanus.ID) error { m.ctrl.T.Helper() - m.ctrl.Call(m, "UnAssignSubscription", id) + ret := m.ctrl.Call(m, "UnAssignSubscription", id) + ret0, _ := ret[0].(error) + return ret0 } // UnAssignSubscription indicates an expected call of UnAssignSubscription. diff --git a/internal/controller/trigger/worker/scheduler.go b/internal/controller/trigger/worker/scheduler.go index bc3029890..80243b4be 100644 --- a/internal/controller/trigger/worker/scheduler.go +++ b/internal/controller/trigger/worker/scheduler.go @@ -18,7 +18,6 @@ import ( "context" "time" - "github.com/linkall-labs/vanus/internal/controller/trigger/metadata" "github.com/linkall-labs/vanus/internal/controller/trigger/subscription" "github.com/linkall-labs/vanus/internal/primitive/queue" "github.com/linkall-labs/vanus/internal/primitive/vanus" @@ -119,15 +118,13 @@ func (s *SubscriptionScheduler) handler(ctx context.Context, subscriptionID vanu return ErrTriggerWorkerNotFound } if subscription.TriggerWorker == "" { + subscription.TriggerWorker = twAddr + err := s.subscriptionManager.UpdateSubscription(ctx, subscription) + if err != nil { + return err + } metrics.CtrlTriggerGauge.WithLabelValues(twAddr).Inc() } - subscription.TriggerWorker = twAddr - subscription.Phase = metadata.SubscriptionPhaseScheduled - subscription.HeartbeatTime = time.Now() - err := s.subscriptionManager.UpdateSubscription(ctx, subscription) - if err != nil { - return err - } tWorker.AssignSubscription(subscriptionID) return nil } diff --git a/internal/controller/trigger/worker/worker.go b/internal/controller/trigger/worker/worker.go index 392a12c57..93908f7c0 100644 --- a/internal/controller/trigger/worker/worker.go +++ b/internal/controller/trigger/worker/worker.go @@ -49,9 +49,8 @@ type TriggerWorker interface { GetHeartbeatTime() time.Time Polish() AssignSubscription(id vanus.ID) - UnAssignSubscription(id vanus.ID) + UnAssignSubscription(id vanus.ID) error GetAssignedSubscriptions() []vanus.ID - ResetOffsetToTimestamp(id vanus.ID, timestamp uint64) error } // triggerWorker send subscription to trigger worker server. @@ -133,7 +132,24 @@ func (tw *triggerWorker) handler(ctx context.Context, subscriptionID vanus.ID) e if sub == nil { return nil } - offsets, err := tw.subscriptionManager.GetOffset(ctx, subscriptionID) + switch sub.Phase { + case metadata.SubscriptionPhaseStopping, metadata.SubscriptionPhaseStopped: + err := tw.removeSubscription(ctx, subscriptionID) + if err != nil { + return err + } + if sub.Phase != metadata.SubscriptionPhaseStopped { + // modify phase to stopped. + sub.Phase = metadata.SubscriptionPhaseStopped + sub.TriggerWorker = "" + err = tw.subscriptionManager.UpdateSubscription(ctx, sub) + if err != nil { + return err + } + } + return nil + } + offsets, err := tw.subscriptionManager.GetOrSaveOffset(ctx, subscriptionID) if err != nil { return err } @@ -246,7 +262,7 @@ func (tw *triggerWorker) AssignSubscription(id vanus.ID) { tw.subscriptionQueue.Add(id) } -func (tw *triggerWorker) UnAssignSubscription(id vanus.ID) { +func (tw *triggerWorker) UnAssignSubscription(id vanus.ID) error { log.Info(context.Background(), "trigger worker remove a subscription", map[string]interface{}{ log.KeyTriggerWorkerAddr: tw.info.Addr, log.KeySubscriptionID: id, @@ -260,9 +276,10 @@ func (tw *triggerWorker) UnAssignSubscription(id vanus.ID) { log.KeyTriggerWorkerAddr: tw.info.Addr, log.KeySubscriptionID: id, }) - tw.subscriptionQueue.Add(id) + return err } } + return nil } func (tw *triggerWorker) GetAssignedSubscriptions() []vanus.ID { @@ -329,15 +346,6 @@ func (tw *triggerWorker) RemoteStart(ctx context.Context) error { return nil } -func (tw *triggerWorker) ResetOffsetToTimestamp(id vanus.ID, timestamp uint64) error { - request := &trigger.ResetOffsetToTimestampRequest{SubscriptionId: id.Uint64(), Timestamp: timestamp} - _, err := tw.client.ResetOffsetToTimestamp(tw.ctx, request) - if err != nil { - return errors.ErrTriggerWorker.WithMessage("reset offset to timestamp").Wrap(err) - } - return nil -} - func (tw *triggerWorker) addSubscription(ctx context.Context, sub *primitive.Subscription) error { request := convert.ToPbAddSubscription(sub) _, err := tw.client.AddSubscription(ctx, request) diff --git a/internal/controller/trigger/worker/worker_test.go b/internal/controller/trigger/worker/worker_test.go index a430b6082..4acc4678f 100644 --- a/internal/controller/trigger/worker/worker_test.go +++ b/internal/controller/trigger/worker/worker_test.go @@ -31,23 +31,6 @@ import ( . "github.com/smartystreets/goconvey/convey" ) -func TestTriggerWorker_ResetOffsetToTimestamp(t *testing.T) { - Convey("test reset offset to timestamp", t, func() { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - subscriptionManager := subscription.NewMockManager(ctrl) - client := pbtrigger.NewMockTriggerWorkerClient(ctrl) - addr := "test" - tWorker := NewTriggerWorkerByAddr(addr, subscriptionManager).(*triggerWorker) - tWorker.client = client - id := vanus.NewTestID() - client.EXPECT().ResetOffsetToTimestamp(gomock.Any(), gomock.Any()).Return(nil, nil) - err := tWorker.ResetOffsetToTimestamp(id, uint64(time.Now().Unix())) - So(err, ShouldBeNil) - _ = tWorker.Close() - }) -} - func TestTriggerWorker_AssignSubscription(t *testing.T) { Convey("test assign subscription", t, func() { ctrl := gomock.NewController(t) @@ -153,7 +136,7 @@ func TestTriggerWorker_Handler(t *testing.T) { }, } subscriptionManager.EXPECT().GetSubscription(gomock.Any(), gomock.Any()).AnyTimes().Return(sub) - subscriptionManager.EXPECT().GetOffset(gomock.Any(), gomock.Any()).AnyTimes().Return(info.ListOffsetInfo{}, nil) + subscriptionManager.EXPECT().GetOrSaveOffset(gomock.Any(), gomock.Any()).AnyTimes().Return(info.ListOffsetInfo{}, nil) client.EXPECT().AddSubscription(gomock.Any(), gomock.Any()).Return(nil, nil) subscriptionManager.EXPECT().UpdateSubscription(gomock.Any(), gomock.Any()).AnyTimes().Return(nil) err := tWorker.handler(ctx, id) diff --git a/internal/convert/convert.go b/internal/convert/convert.go index 801cd6c4e..c0bd67c0c 100644 --- a/internal/convert/convert.go +++ b/internal/convert/convert.go @@ -39,7 +39,6 @@ func FromPbSubscriptionRequest(sub *ctrl.SubscriptionRequest) *metadata.Subscrip Transformer: fromPbTransformer(sub.Transformer), EventBus: sub.EventBus, Name: sub.Name, - Disable: sub.Disable, Description: sub.Description, } return to @@ -288,11 +287,13 @@ func ToPbSubscription(sub *metadata.Subscription, offsets info.ListOffsetInfo) * Transformer: ToPbTransformer(sub.Transformer), Offsets: ToPbOffsetInfos(offsets), Name: sub.Name, - Disable: sub.Disable, Description: sub.Description, CreatedAt: sub.CreatedAt.UnixMilli(), UpdatedAt: sub.UpdatedAt.UnixMilli(), } + if sub.Phase == metadata.SubscriptionPhaseStopped { + to.Disable = true + } return to } diff --git a/internal/gateway/proxy/direct.go b/internal/gateway/proxy/direct.go index 24b6c5a13..e638769fa 100644 --- a/internal/gateway/proxy/direct.go +++ b/internal/gateway/proxy/direct.go @@ -81,3 +81,18 @@ func (cp *ControllerProxy) ListSubscription(ctx context.Context, req *emptypb.Empty) (*ctrlpb.ListSubscriptionResponse, error) { return cp.triggerCtrl.ListSubscription(ctx, req) } + +func (cp *ControllerProxy) DisableSubscription(ctx context.Context, + req *ctrlpb.DisableSubscriptionRequest) (*emptypb.Empty, error) { + return cp.triggerCtrl.DisableSubscription(ctx, req) +} + +func (cp *ControllerProxy) ResumeSubscription(ctx context.Context, + req *ctrlpb.ResumeSubscriptionRequest) (*emptypb.Empty, error) { + return cp.triggerCtrl.ResumeSubscription(ctx, req) +} + +func (cp *ControllerProxy) ResetOffsetToTimestamp(ctx context.Context, + req *ctrlpb.ResetOffsetToTimestampRequest) (*ctrlpb.ResetOffsetToTimestampResponse, error) { + return cp.triggerCtrl.ResetOffsetToTimestamp(ctx, req) +} diff --git a/internal/trigger/mock_worker.go b/internal/trigger/mock_worker.go index 0213b3e33..94e63fb2c 100644 --- a/internal/trigger/mock_worker.go +++ b/internal/trigger/mock_worker.go @@ -106,20 +106,6 @@ func (mr *MockWorkerMockRecorder) RemoveSubscription(ctx, id interface{}) *gomoc return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RemoveSubscription", reflect.TypeOf((*MockWorker)(nil).RemoveSubscription), ctx, id) } -// ResetOffsetToTimestamp mocks base method. -func (m *MockWorker) ResetOffsetToTimestamp(ctx context.Context, id vanus.ID, timestamp int64) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ResetOffsetToTimestamp", ctx, id, timestamp) - ret0, _ := ret[0].(error) - return ret0 -} - -// ResetOffsetToTimestamp indicates an expected call of ResetOffsetToTimestamp. -func (mr *MockWorkerMockRecorder) ResetOffsetToTimestamp(ctx, id, timestamp interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetOffsetToTimestamp", reflect.TypeOf((*MockWorker)(nil).ResetOffsetToTimestamp), ctx, id, timestamp) -} - // Start mocks base method. func (m *MockWorker) Start(ctx context.Context) error { m.ctrl.T.Helper() diff --git a/internal/trigger/reader/mock_reader.go b/internal/trigger/reader/mock_reader.go index 13a7139e1..dcc137588 100644 --- a/internal/trigger/reader/mock_reader.go +++ b/internal/trigger/reader/mock_reader.go @@ -5,11 +5,9 @@ package reader import ( - context "context" reflect "reflect" gomock "github.com/golang/mock/gomock" - info "github.com/linkall-labs/vanus/internal/primitive/info" ) // MockReader is a mock of Reader interface. @@ -47,21 +45,6 @@ func (mr *MockReaderMockRecorder) Close() *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockReader)(nil).Close)) } -// GetOffsetByTimestamp mocks base method. -func (m *MockReader) GetOffsetByTimestamp(ctx context.Context, timestamp int64) (info.ListOffsetInfo, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "GetOffsetByTimestamp", ctx, timestamp) - ret0, _ := ret[0].(info.ListOffsetInfo) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// GetOffsetByTimestamp indicates an expected call of GetOffsetByTimestamp. -func (mr *MockReaderMockRecorder) GetOffsetByTimestamp(ctx, timestamp interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetOffsetByTimestamp", reflect.TypeOf((*MockReader)(nil).GetOffsetByTimestamp), ctx, timestamp) -} - // Start mocks base method. func (m *MockReader) Start() error { m.ctrl.T.Helper() diff --git a/internal/trigger/reader/reader.go b/internal/trigger/reader/reader.go index 733f55f1d..7aa9f3623 100644 --- a/internal/trigger/reader/reader.go +++ b/internal/trigger/reader/reader.go @@ -28,7 +28,6 @@ import ( "github.com/linkall-labs/vanus/client/pkg/eventlog" "github.com/linkall-labs/vanus/client/pkg/option" "github.com/linkall-labs/vanus/client/pkg/policy" - "github.com/linkall-labs/vanus/internal/primitive" pInfo "github.com/linkall-labs/vanus/internal/primitive/info" "github.com/linkall-labs/vanus/internal/primitive/vanus" "github.com/linkall-labs/vanus/internal/trigger/info" @@ -41,7 +40,6 @@ import ( const ( checkEventLogInterval = 30 * time.Second lookupReadableLogsTimeout = 5 * time.Second - readerSeekTimeout = 5 * time.Second readEventTimeout = 5 * time.Second initErrSleepTime = 5 * time.Second readErrSleepTime = 2 * time.Second @@ -50,13 +48,10 @@ const ( type Config struct { EventBusName string - Controllers []string Client eb.Client SubscriptionID vanus.ID SubscriptionIDStr string Offset EventLogOffset - OffsetType primitive.OffsetType - OffsetTimestamp int64 CheckEventLogInterval time.Duration } @@ -64,7 +59,6 @@ type EventLogOffset map[vanus.ID]uint64 type Reader interface { Start() error - GetOffsetByTimestamp(ctx context.Context, timestamp int64) (pInfo.ListOffsetInfo, error) Close() } @@ -92,26 +86,6 @@ func NewReader(config Config, events chan<- info.EventRecord) Reader { return r } -func (r *reader) GetOffsetByTimestamp(ctx context.Context, timestamp int64) (pInfo.ListOffsetInfo, error) { - offsets := make(pInfo.ListOffsetInfo, 0, len(r.elReader)) - bus := r.config.Client.Eventbus(ctx, r.config.EventBusName) - for id := range r.elReader { - log, err := bus.GetLog(ctx, id.Uint64()) - if err != nil { - return offsets, err - } - offset, err := log.QueryOffsetByTime(ctx, timestamp) - if err != nil { - return offsets, err - } - offsets = append(offsets, pInfo.OffsetInfo{ - EventLogID: id, - Offset: uint64(offset), - }) - } - return offsets, nil -} - func (r *reader) Close() { r.stop() r.wg.Wait() @@ -165,36 +139,17 @@ func (r *reader) checkEventLogChange() { } } -func (r *reader) getOffset(ctx context.Context, eventLogID vanus.ID) (uint64, error) { - l, err := r.config.Client.Eventbus(ctx, r.config.EventBusName).GetLog(ctx, eventLogID.Uint64()) - if err != nil { - return 0, err +func (r *reader) getOffset(eventLogID vanus.ID) uint64 { + v, exist := r.config.Offset[eventLogID] + if exist { + return v } - offset, exist := r.config.Offset[eventLogID] - if !exist { - var err error - var v int64 - switch r.config.OffsetType { - case primitive.LatestOffset: - if v, err = l.LatestOffset(ctx); err != nil { - return 0, err - } - case primitive.EarliestOffset: - if v, err = l.EarliestOffset(ctx); err != nil { - return 0, err - } - case primitive.Timestamp: - if v, err = l.QueryOffsetByTime(ctx, r.config.OffsetTimestamp); err != nil { - return 0, err - } - } - // fix offset is negative which convert to uint64 is big. - if v < 0 { - v = 0 - } - offset = uint64(v) - } - return offset, nil + log.Warning(r.stctx, "offset no exist, will use 0", map[string]interface{}{ + log.KeyEventbusName: r.config.EventBusName, + log.KeySubscriptionID: r.config.SubscriptionID, + log.KeyEventlogID: eventLogID, + }) + return 0 } func (r *reader) start(els []uint64) { @@ -205,13 +160,7 @@ func (r *reader) start(els []uint64) { if _, exist := r.elReader[eventLogID]; exist { continue } - offset, err := r.getOffset(r.stctx, eventLogID) - if err != nil { - log.Error(r.stctx, "event log get offset error", map[string]interface{}{ - log.KeyError: err, - }) - continue - } + offset := r.getOffset(eventLogID) l, err := r.config.Client.Eventbus(r.stctx, r.config.EventBusName).GetLog(r.stctx, eventLogID.Uint64()) if err != nil { log.Error(r.stctx, "get eventlog error", map[string]interface{}{ diff --git a/internal/trigger/reader/reader_test.go b/internal/trigger/reader/reader_test.go index 71ac9f47a..be888b4f8 100644 --- a/internal/trigger/reader/reader_test.go +++ b/internal/trigger/reader/reader_test.go @@ -17,132 +17,20 @@ package reader import ( "context" "encoding/binary" - "math/rand" "sync" "testing" "time" + ce "github.com/cloudevents/sdk-go/v2" + . "github.com/golang/mock/gomock" "github.com/google/uuid" "github.com/linkall-labs/vanus/client" "github.com/linkall-labs/vanus/client/pkg/api" "github.com/linkall-labs/vanus/client/pkg/eventlog" - "github.com/linkall-labs/vanus/internal/primitive" - "github.com/linkall-labs/vanus/internal/primitive/vanus" "github.com/linkall-labs/vanus/internal/trigger/info" - - ce "github.com/cloudevents/sdk-go/v2" - . "github.com/golang/mock/gomock" . "github.com/smartystreets/goconvey/convey" ) -func TestGetOffsetByTimestamp(t *testing.T) { - Convey("test get offset by timestamp", t, func() { - events := make(chan info.EventRecord, 10) - r := NewReader(Config{}, events).(*reader) - eventLogID := vanus.NewTestID() - r.elReader[eventLogID] = struct{}{} - rand.Seed(time.Now().Unix()) - offset := rand.Uint64() - mockCtrl := NewController(t) - mockClient := client.NewMockClient(mockCtrl) - mockEventbus := api.NewMockEventbus(mockCtrl) - mockEventlog := api.NewMockEventlog(mockCtrl) - mockBusWriter := api.NewMockBusWriter(mockCtrl) - mockBusReader := api.NewMockBusReader(mockCtrl) - mockClient.EXPECT().Eventbus(Any(), Any()).AnyTimes().Return(mockEventbus) - mockEventbus.EXPECT().Writer().AnyTimes().Return(mockBusWriter) - mockEventbus.EXPECT().Reader().AnyTimes().Return(mockBusReader) - mockEventbus.EXPECT().GetLog(Any(), Any()).AnyTimes().Return(mockEventlog, nil) - mockEventlog.EXPECT().QueryOffsetByTime(Any(), Any()).AnyTimes().Return(int64(offset), nil) - r.config.Client = mockClient - offsets, err := r.GetOffsetByTimestamp(context.Background(), time.Now().Unix()) - So(err, ShouldBeNil) - So(len(offsets), ShouldEqual, 1) - So(offsets[0].EventLogID, ShouldEqual, eventLogID) - So(offsets[0].Offset, ShouldEqual, offset) - }) -} - -func TestGetOffset(t *testing.T) { - Convey("test get offset", t, func() { - events := make(chan info.EventRecord, 10) - r := NewReader(Config{}, events).(*reader) - eventLogID := vanus.NewTestID() - r.elReader = map[vanus.ID]struct{}{} - mockCtrl := NewController(t) - mockClient := client.NewMockClient(mockCtrl) - mockEventbus := api.NewMockEventbus(mockCtrl) - mockEventlog := api.NewMockEventlog(mockCtrl) - mockBusWriter := api.NewMockBusWriter(mockCtrl) - mockBusReader := api.NewMockBusReader(mockCtrl) - mockClient.EXPECT().Eventbus(Any(), Any()).AnyTimes().Return(mockEventbus) - mockEventbus.EXPECT().Writer().AnyTimes().Return(mockBusWriter) - mockEventbus.EXPECT().Reader().AnyTimes().Return(mockBusReader) - mockEventbus.EXPECT().GetLog(Any(), Any()).AnyTimes().Return(mockEventlog, nil) - r.config.Client = mockClient - Convey("test latest", func() { - r.config.OffsetType = primitive.LatestOffset - rand.Seed(time.Now().Unix()) - offset := rand.Uint32() - Convey("negative number", func() { - mockEventlog.EXPECT().LatestOffset(Any()).AnyTimes().Return(int64(offset)*-1, nil) - v, err := r.getOffset(context.Background(), eventLogID) - So(err, ShouldBeNil) - So(v, ShouldEqual, 0) - }) - Convey("non negative number", func() { - mockEventlog.EXPECT().LatestOffset(Any()).AnyTimes().Return(int64(offset), nil) - v, err := r.getOffset(context.Background(), eventLogID) - So(err, ShouldBeNil) - So(v, ShouldEqual, offset) - }) - }) - Convey("test earliest", func() { - r.config.OffsetType = primitive.EarliestOffset - rand.Seed(time.Now().Unix()) - offset := rand.Uint32() - Convey("negative number", func() { - mockEventlog.EXPECT().EarliestOffset(Any()).AnyTimes().Return(int64(offset)*-1, nil) - v, err := r.getOffset(context.Background(), eventLogID) - So(err, ShouldBeNil) - So(v, ShouldEqual, 0) - }) - Convey("non negative number", func() { - mockEventlog.EXPECT().EarliestOffset(Any()).AnyTimes().Return(int64(offset), nil) - v, err := r.getOffset(context.Background(), eventLogID) - So(err, ShouldBeNil) - So(v, ShouldEqual, offset) - }) - }) - Convey("test timestamp", func() { - r.config.OffsetType = primitive.Timestamp - r.config.OffsetTimestamp = time.Now().Unix() - rand.Seed(time.Now().Unix()) - offset := rand.Uint32() - Convey("negative number", func() { - mockEventlog.EXPECT().QueryOffsetByTime(Any(), Any()).AnyTimes().Return(int64(offset)*-1, nil) - v, err := r.getOffset(context.Background(), eventLogID) - So(err, ShouldBeNil) - So(v, ShouldEqual, 0) - }) - Convey("non negative number", func() { - mockEventlog.EXPECT().QueryOffsetByTime(Any(), Any()).AnyTimes().Return(int64(offset), nil) - v, err := r.getOffset(context.Background(), eventLogID) - So(err, ShouldBeNil) - So(v, ShouldEqual, offset) - }) - }) - Convey("test exist", func() { - rand.Seed(time.Now().Unix()) - offset := rand.Uint64() - r.config.Offset = map[vanus.ID]uint64{eventLogID: offset} - v, err := r.getOffset(context.Background(), eventLogID) - So(err, ShouldBeNil) - So(v, ShouldEqual, offset) - }) - }) -} - func TestReaderStart(t *testing.T) { mockCtrl := NewController(t) defer mockCtrl.Finish() diff --git a/internal/trigger/server.go b/internal/trigger/server.go index a65dc9ade..7881fd75c 100644 --- a/internal/trigger/server.go +++ b/internal/trigger/server.go @@ -25,7 +25,6 @@ import ( "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/pkg/errors" pbtrigger "github.com/linkall-labs/vanus/proto/pkg/trigger" - "google.golang.org/protobuf/types/known/emptypb" ) var ( @@ -140,21 +139,6 @@ func (s *server) ResumeSubscription(ctx context.Context, return &pbtrigger.ResumeSubscriptionResponse{}, nil } -func (s *server) ResetOffsetToTimestamp(ctx context.Context, - request *pbtrigger.ResetOffsetToTimestampRequest) (*emptypb.Empty, error) { - log.Info(ctx, "subscription reset offset ", map[string]interface{}{"request": request}) - id := vanus.NewIDFromUint64(request.SubscriptionId) - err := s.worker.ResetOffsetToTimestamp(ctx, id, int64(request.Timestamp)) - if err != nil { - log.Error(ctx, "reset offset error", map[string]interface{}{ - log.KeySubscriptionID: id, - log.KeyError: err, - }) - return nil, err - } - return &emptypb.Empty{}, nil -} - func (s *server) Initialize(ctx context.Context) error { err := s.worker.Init(ctx) if err != nil { diff --git a/internal/trigger/server_test.go b/internal/trigger/server_test.go index a44d5f6bc..827dfda0f 100644 --- a/internal/trigger/server_test.go +++ b/internal/trigger/server_test.go @@ -80,16 +80,6 @@ func TestServerApi(t *testing.T) { _, err := s.ResumeSubscription(ctx, &pbtrigger.ResumeSubscriptionRequest{}) So(err, ShouldNotBeNil) }) - Convey("test reset offset to timestamp", func() { - w.EXPECT().ResetOffsetToTimestamp(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) - _, err := s.ResetOffsetToTimestamp(ctx, &pbtrigger.ResetOffsetToTimestampRequest{}) - So(err, ShouldBeNil) - }) - Convey("test reset offset to timestamp has error", func() { - w.EXPECT().ResetOffsetToTimestamp(gomock.Any(), gomock.Any(), gomock.Any()).Return(fmt.Errorf("test error")) - _, err := s.ResetOffsetToTimestamp(ctx, &pbtrigger.ResetOffsetToTimestampRequest{}) - So(err, ShouldNotBeNil) - }) }) } diff --git a/internal/trigger/trigger/mock_trigger.go b/internal/trigger/trigger/mock_trigger.go index 26f0ed06e..f06aae501 100644 --- a/internal/trigger/trigger/mock_trigger.go +++ b/internal/trigger/trigger/mock_trigger.go @@ -78,21 +78,6 @@ func (mr *MockTriggerMockRecorder) Init(ctx interface{}) *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Init", reflect.TypeOf((*MockTrigger)(nil).Init), ctx) } -// ResetOffsetToTimestamp mocks base method. -func (m *MockTrigger) ResetOffsetToTimestamp(ctx context.Context, timestamp int64) (info.ListOffsetInfo, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ResetOffsetToTimestamp", ctx, timestamp) - ret0, _ := ret[0].(info.ListOffsetInfo) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ResetOffsetToTimestamp indicates an expected call of ResetOffsetToTimestamp. -func (mr *MockTriggerMockRecorder) ResetOffsetToTimestamp(ctx, timestamp interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetOffsetToTimestamp", reflect.TypeOf((*MockTrigger)(nil).ResetOffsetToTimestamp), ctx, timestamp) -} - // Start mocks base method. func (m *MockTrigger) Start(ctx context.Context) error { m.ctrl.T.Helper() diff --git a/internal/trigger/trigger/trigger.go b/internal/trigger/trigger/trigger.go index bf1238f76..87a0f4f02 100644 --- a/internal/trigger/trigger/trigger.go +++ b/internal/trigger/trigger/trigger.go @@ -57,7 +57,6 @@ type Trigger interface { Stop(ctx context.Context) error Change(ctx context.Context, subscription *primitive.Subscription) error GetOffsets(ctx context.Context) pInfo.ListOffsetInfo - ResetOffsetToTimestamp(ctx context.Context, timestamp int64) (pInfo.ListOffsetInfo, error) } type trigger struct { @@ -416,33 +415,22 @@ func (t *trigger) writeEventToDeadLetter(ctx context.Context, e *ce.Event, reaso } func (t *trigger) getReaderConfig() reader.Config { - controllers := t.config.Controllers sub := t.subscription - var offsetTimestamp int64 - if sub.Config.OffsetTimestamp != nil { - offsetTimestamp = int64(*sub.Config.OffsetTimestamp) - } return reader.Config{ - EventBusName: sub.EventBus, - Controllers: controllers, - Client: t.client, - SubscriptionID: sub.ID, - OffsetType: sub.Config.OffsetType, - OffsetTimestamp: offsetTimestamp, - Offset: getOffset(t.offsetManager, sub), + EventBusName: sub.EventBus, + Client: t.client, + SubscriptionID: sub.ID, + Offset: getOffset(t.offsetManager, sub), } } func (t *trigger) getRetryEventReaderConfig() reader.Config { - controllers := t.config.Controllers sub := t.subscription ebName := primitive.RetryEventbusName return reader.Config{ EventBusName: ebName, - Controllers: controllers, Client: t.client, SubscriptionID: sub.ID, - OffsetType: primitive.LatestOffset, Offset: getOffset(t.offsetManager, sub), } } @@ -541,16 +529,6 @@ func (t *trigger) Change(ctx context.Context, subscription *primitive.Subscripti return nil } -func (t *trigger) ResetOffsetToTimestamp(ctx context.Context, timestamp int64) (pInfo.ListOffsetInfo, error) { - offsets, err := t.reader.GetOffsetByTimestamp(ctx, timestamp) - if err != nil { - return nil, err - } - t.subscription.Offsets = offsets - t.offsetManager.Clear() - return offsets, nil -} - // GetOffsets contains retry eventlog. func (t *trigger) GetOffsets(ctx context.Context) pInfo.ListOffsetInfo { return t.offsetManager.GetCommit() diff --git a/internal/trigger/trigger/trigger_test.go b/internal/trigger/trigger/trigger_test.go index 483af9a8b..a259dabeb 100644 --- a/internal/trigger/trigger/trigger_test.go +++ b/internal/trigger/trigger/trigger_test.go @@ -30,7 +30,6 @@ import ( eb "github.com/linkall-labs/vanus/client" "github.com/linkall-labs/vanus/client/pkg/api" "github.com/linkall-labs/vanus/internal/primitive" - pInfo "github.com/linkall-labs/vanus/internal/primitive/info" "github.com/linkall-labs/vanus/internal/primitive/vanus" "github.com/linkall-labs/vanus/internal/trigger/client" "github.com/linkall-labs/vanus/internal/trigger/info" @@ -308,22 +307,3 @@ func TestChangeSubscription(t *testing.T) { }) }) } - -func TestResetOffset(t *testing.T) { - Convey("test reset offset", t, func() { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - ctx := context.Background() - id := vanus.NewTestID() - tg := NewTrigger(makeSubscription(id), WithControllers([]string{"test"})).(*trigger) - r := reader.NewMockReader(ctrl) - tg.reader = r - Convey("reset offset to timestamp", func() { - offsets := pInfo.ListOffsetInfo{{EventLogID: vanus.NewTestID(), Offset: uint64(100)}} - r.EXPECT().GetOffsetByTimestamp(gomock.Any(), gomock.Any()).Return(offsets, nil) - v, err := tg.ResetOffsetToTimestamp(ctx, time.Now().Unix()) - So(err, ShouldBeNil) - So(len(v), ShouldEqual, len(offsets)) - }) - }) -} diff --git a/internal/trigger/worker.go b/internal/trigger/worker.go index 512256add..10357e739 100644 --- a/internal/trigger/worker.go +++ b/internal/trigger/worker.go @@ -23,7 +23,6 @@ import ( "github.com/linkall-labs/vanus/internal/convert" "github.com/linkall-labs/vanus/internal/primitive" - "github.com/linkall-labs/vanus/internal/primitive/info" "github.com/linkall-labs/vanus/internal/primitive/vanus" "github.com/linkall-labs/vanus/internal/trigger/trigger" "github.com/linkall-labs/vanus/observability/log" @@ -45,7 +44,6 @@ type Worker interface { RemoveSubscription(ctx context.Context, id vanus.ID) error PauseSubscription(ctx context.Context, id vanus.ID) error StartSubscription(ctx context.Context, id vanus.ID) error - ResetOffsetToTimestamp(ctx context.Context, id vanus.ID, timestamp int64) error } const ( @@ -197,38 +195,6 @@ func (w *worker) StartSubscription(ctx context.Context, id vanus.ID) error { return w.startSubscription(ctx, id) } -func (w *worker) ResetOffsetToTimestamp(ctx context.Context, - id vanus.ID, - timestamp int64) error { - t, exist := w.getTrigger(id) - if !exist { - return errors.ErrResourceNotFound.WithMessage("subscription not exist") - } - // pause subscription - _ = w.stopSubscription(ctx, id) - // reset offset - offsets, err := t.ResetOffsetToTimestamp(ctx, timestamp) - if err != nil { - return err - } - // commit offset - log.Info(ctx, "reset offset to timestamp offsets info", map[string]interface{}{ - log.KeySubscriptionID: id, - "offsets": offsets, - }) - err = w.commitOffset(ctx, id, offsets) - if err != nil { - return err - } - // start subscription - err = w.startSubscription(ctx, id) - if err != nil { - // todo process start fail - return err - } - return nil -} - func (w *worker) startHeartbeat(ctx context.Context) error { w.wg.Add(1) defer w.wg.Done() @@ -261,17 +227,6 @@ func (w *worker) startSubscription(ctx context.Context, id vanus.ID) error { return t.Start(ctx) } -func (w *worker) commitOffset(ctx context.Context, id vanus.ID, offsets info.ListOffsetInfo) error { - _, err := w.client.CommitOffset(ctx, &ctrlpb.CommitOffsetRequest{ - ForceCommit: true, - SubscriptionInfo: []*metapb.SubscriptionInfo{convert.ToPbSubscriptionInfo(info.SubscriptionInfo{ - SubscriptionID: id, - Offsets: offsets, - })}, - }) - return err -} - func (w *worker) commitOffsets(ctx context.Context) error { _, err := w.client.CommitOffset(ctx, &ctrlpb.CommitOffsetRequest{ ForceCommit: true, diff --git a/internal/trigger/worker_test.go b/internal/trigger/worker_test.go index 0baba6325..20d2890e0 100644 --- a/internal/trigger/worker_test.go +++ b/internal/trigger/worker_test.go @@ -18,7 +18,6 @@ import ( "context" "fmt" "testing" - "time" "github.com/golang/mock/gomock" "github.com/linkall-labs/vanus/internal/primitive" @@ -159,38 +158,6 @@ func TestPauseStartSubscription(t *testing.T) { }) } -func TestResetOffsetToTimestamp(t *testing.T) { - ctx := context.Background() - Convey("test reset offset to timestamp", t, func() { - id := vanus.NewTestID() - ctrl := gomock.NewController(t) - defer ctrl.Finish() - tg := trigger.NewMockTrigger(ctrl) - m := NewWorker(Config{}).(*worker) - m.newTrigger = testNewTrigger(tg) - Convey("reset offset no exist subscription", func() { - err := m.ResetOffsetToTimestamp(ctx, id, time.Now().Unix()) - So(err, ShouldNotBeNil) - }) - Convey("reset offset exist subscription", func() { - tg.EXPECT().Init(gomock.Any()).AnyTimes().Return(nil) - tg.EXPECT().Start(gomock.Any()).AnyTimes().Return(nil) - err := m.AddSubscription(ctx, &primitive.Subscription{ - ID: id, - }) - So(err, ShouldBeNil) - tg.EXPECT().Stop(gomock.Any()).Return(nil) - offsets := info.ListOffsetInfo{{EventLogID: vanus.NewTestID(), Offset: uint64(100)}} - tg.EXPECT().ResetOffsetToTimestamp(gomock.Any(), gomock.Any()).Return(offsets, nil) - triggerClient := controller.NewMockTriggerControllerClient(ctrl) - m.client = triggerClient - triggerClient.EXPECT().CommitOffset(gomock.Any(), gomock.Any()).Return(nil, nil) - err = m.ResetOffsetToTimestamp(ctx, id, time.Now().Unix()) - So(err, ShouldBeNil) - }) - }) -} - func TestWorker_Stop(t *testing.T) { ctx := context.Background() Convey("start stop", t, func() { diff --git a/pkg/cluster/raw_client/trigger.go b/pkg/cluster/raw_client/trigger.go index e57000494..361039c15 100644 --- a/pkg/cluster/raw_client/trigger.go +++ b/pkg/cluster/raw_client/trigger.go @@ -173,8 +173,8 @@ func (tc *triggerClient) UnregisterTriggerWorker(ctx context.Context, in *ctrlpb } func (tc *triggerClient) ResetOffsetToTimestamp(ctx context.Context, in *ctrlpb.ResetOffsetToTimestampRequest, - opts ...grpc.CallOption) (*emptypb.Empty, error) { - out := new(emptypb.Empty) + opts ...grpc.CallOption) (*ctrlpb.ResetOffsetToTimestampResponse, error) { + out := new(ctrlpb.ResetOffsetToTimestampResponse) err := tc.cc.invoke(ctx, "/linkall.vanus.controller.TriggerController/ResetOffsetToTimestamp", in, out, opts...) if err != nil { return nil, err @@ -192,6 +192,24 @@ func (tc *triggerClient) CommitOffset(ctx context.Context, in *ctrlpb.CommitOffs return out, nil } +func (tc *triggerClient) DisableSubscription(ctx context.Context, in *ctrlpb.DisableSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) { + out := new(emptypb.Empty) + err := tc.cc.invoke(ctx, "/linkall.vanus.controller.TriggerController/DisableSubscription", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (tc *triggerClient) ResumeSubscription(ctx context.Context, in *ctrlpb.ResumeSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) { + out := new(emptypb.Empty) + err := tc.cc.invoke(ctx, "/linkall.vanus.controller.TriggerController/ResumeSubscription", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (tc *triggerClient) TriggerWorkerHeartbeat(_ context.Context, _ ...grpc.CallOption) (ctrlpb.TriggerController_TriggerWorkerHeartbeatClient, error) { panic("unsupported method, please use controller.RegisterHeartbeat") diff --git a/proto/pkg/controller/controller.pb.go b/proto/pkg/controller/controller.pb.go index 4c1789d5f..b3f3a05f7 100644 --- a/proto/pkg/controller/controller.pb.go +++ b/proto/pkg/controller/controller.pb.go @@ -15,13 +15,17 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.26.0 -// protoc v3.19.4 +// protoc v3.19.1 // source: controller.proto +//go:generate mockgen -source=controller.pb.go -destination=mock_controller.go -package=controller package controller import ( context "context" + reflect "reflect" + sync "sync" + meta "github.com/linkall-labs/vanus/proto/pkg/meta" grpc "google.golang.org/grpc" codes "google.golang.org/grpc/codes" @@ -31,8 +35,6 @@ import ( emptypb "google.golang.org/protobuf/types/known/emptypb" timestamppb "google.golang.org/protobuf/types/known/timestamppb" wrapperspb "google.golang.org/protobuf/types/known/wrapperspb" - reflect "reflect" - sync "sync" ) const ( @@ -1075,6 +1077,100 @@ func (x *DeleteSubscriptionRequest) GetId() uint64 { return 0 } +type DisableSubscriptionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id uint64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` +} + +func (x *DisableSubscriptionRequest) Reset() { + *x = DisableSubscriptionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_controller_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DisableSubscriptionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DisableSubscriptionRequest) ProtoMessage() {} + +func (x *DisableSubscriptionRequest) ProtoReflect() protoreflect.Message { + mi := &file_controller_proto_msgTypes[18] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DisableSubscriptionRequest.ProtoReflect.Descriptor instead. +func (*DisableSubscriptionRequest) Descriptor() ([]byte, []int) { + return file_controller_proto_rawDescGZIP(), []int{18} +} + +func (x *DisableSubscriptionRequest) GetId() uint64 { + if x != nil { + return x.Id + } + return 0 +} + +type ResumeSubscriptionRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id uint64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` +} + +func (x *ResumeSubscriptionRequest) Reset() { + *x = ResumeSubscriptionRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_controller_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResumeSubscriptionRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResumeSubscriptionRequest) ProtoMessage() {} + +func (x *ResumeSubscriptionRequest) ProtoReflect() protoreflect.Message { + mi := &file_controller_proto_msgTypes[19] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResumeSubscriptionRequest.ProtoReflect.Descriptor instead. +func (*ResumeSubscriptionRequest) Descriptor() ([]byte, []int) { + return file_controller_proto_rawDescGZIP(), []int{19} +} + +func (x *ResumeSubscriptionRequest) GetId() uint64 { + if x != nil { + return x.Id + } + return 0 +} + type ListSubscriptionResponse struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1086,7 +1182,7 @@ type ListSubscriptionResponse struct { func (x *ListSubscriptionResponse) Reset() { *x = ListSubscriptionResponse{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[18] + mi := &file_controller_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1099,7 +1195,7 @@ func (x *ListSubscriptionResponse) String() string { func (*ListSubscriptionResponse) ProtoMessage() {} func (x *ListSubscriptionResponse) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[18] + mi := &file_controller_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1112,7 +1208,7 @@ func (x *ListSubscriptionResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListSubscriptionResponse.ProtoReflect.Descriptor instead. func (*ListSubscriptionResponse) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{18} + return file_controller_proto_rawDescGZIP(), []int{20} } func (x *ListSubscriptionResponse) GetSubscription() []*meta.Subscription { @@ -1133,7 +1229,7 @@ type RegisterTriggerWorkerRequest struct { func (x *RegisterTriggerWorkerRequest) Reset() { *x = RegisterTriggerWorkerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[19] + mi := &file_controller_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1146,7 +1242,7 @@ func (x *RegisterTriggerWorkerRequest) String() string { func (*RegisterTriggerWorkerRequest) ProtoMessage() {} func (x *RegisterTriggerWorkerRequest) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[19] + mi := &file_controller_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1159,7 +1255,7 @@ func (x *RegisterTriggerWorkerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use RegisterTriggerWorkerRequest.ProtoReflect.Descriptor instead. func (*RegisterTriggerWorkerRequest) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{19} + return file_controller_proto_rawDescGZIP(), []int{21} } func (x *RegisterTriggerWorkerRequest) GetAddress() string { @@ -1178,7 +1274,7 @@ type RegisterTriggerWorkerResponse struct { func (x *RegisterTriggerWorkerResponse) Reset() { *x = RegisterTriggerWorkerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[20] + mi := &file_controller_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1191,7 +1287,7 @@ func (x *RegisterTriggerWorkerResponse) String() string { func (*RegisterTriggerWorkerResponse) ProtoMessage() {} func (x *RegisterTriggerWorkerResponse) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[20] + mi := &file_controller_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1204,7 +1300,7 @@ func (x *RegisterTriggerWorkerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use RegisterTriggerWorkerResponse.ProtoReflect.Descriptor instead. func (*RegisterTriggerWorkerResponse) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{20} + return file_controller_proto_rawDescGZIP(), []int{22} } type UnregisterTriggerWorkerRequest struct { @@ -1218,7 +1314,7 @@ type UnregisterTriggerWorkerRequest struct { func (x *UnregisterTriggerWorkerRequest) Reset() { *x = UnregisterTriggerWorkerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[21] + mi := &file_controller_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1231,7 +1327,7 @@ func (x *UnregisterTriggerWorkerRequest) String() string { func (*UnregisterTriggerWorkerRequest) ProtoMessage() {} func (x *UnregisterTriggerWorkerRequest) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[21] + mi := &file_controller_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1244,7 +1340,7 @@ func (x *UnregisterTriggerWorkerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UnregisterTriggerWorkerRequest.ProtoReflect.Descriptor instead. func (*UnregisterTriggerWorkerRequest) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{21} + return file_controller_proto_rawDescGZIP(), []int{23} } func (x *UnregisterTriggerWorkerRequest) GetAddress() string { @@ -1263,7 +1359,7 @@ type UnregisterTriggerWorkerResponse struct { func (x *UnregisterTriggerWorkerResponse) Reset() { *x = UnregisterTriggerWorkerResponse{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[22] + mi := &file_controller_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1276,7 +1372,7 @@ func (x *UnregisterTriggerWorkerResponse) String() string { func (*UnregisterTriggerWorkerResponse) ProtoMessage() {} func (x *UnregisterTriggerWorkerResponse) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[22] + mi := &file_controller_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1289,7 +1385,7 @@ func (x *UnregisterTriggerWorkerResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use UnregisterTriggerWorkerResponse.ProtoReflect.Descriptor instead. func (*UnregisterTriggerWorkerResponse) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{22} + return file_controller_proto_rawDescGZIP(), []int{24} } type TriggerWorkerHeartbeatRequest struct { @@ -1305,7 +1401,7 @@ type TriggerWorkerHeartbeatRequest struct { func (x *TriggerWorkerHeartbeatRequest) Reset() { *x = TriggerWorkerHeartbeatRequest{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[23] + mi := &file_controller_proto_msgTypes[25] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1318,7 +1414,7 @@ func (x *TriggerWorkerHeartbeatRequest) String() string { func (*TriggerWorkerHeartbeatRequest) ProtoMessage() {} func (x *TriggerWorkerHeartbeatRequest) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[23] + mi := &file_controller_proto_msgTypes[25] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1331,7 +1427,7 @@ func (x *TriggerWorkerHeartbeatRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use TriggerWorkerHeartbeatRequest.ProtoReflect.Descriptor instead. func (*TriggerWorkerHeartbeatRequest) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{23} + return file_controller_proto_rawDescGZIP(), []int{25} } func (x *TriggerWorkerHeartbeatRequest) GetAddress() string { @@ -1364,7 +1460,7 @@ type TriggerWorkerHeartbeatResponse struct { func (x *TriggerWorkerHeartbeatResponse) Reset() { *x = TriggerWorkerHeartbeatResponse{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[24] + mi := &file_controller_proto_msgTypes[26] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1377,7 +1473,7 @@ func (x *TriggerWorkerHeartbeatResponse) String() string { func (*TriggerWorkerHeartbeatResponse) ProtoMessage() {} func (x *TriggerWorkerHeartbeatResponse) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[24] + mi := &file_controller_proto_msgTypes[26] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1390,7 +1486,7 @@ func (x *TriggerWorkerHeartbeatResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use TriggerWorkerHeartbeatResponse.ProtoReflect.Descriptor instead. func (*TriggerWorkerHeartbeatResponse) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{24} + return file_controller_proto_rawDescGZIP(), []int{26} } type ResetOffsetToTimestampRequest struct { @@ -1406,7 +1502,7 @@ type ResetOffsetToTimestampRequest struct { func (x *ResetOffsetToTimestampRequest) Reset() { *x = ResetOffsetToTimestampRequest{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[25] + mi := &file_controller_proto_msgTypes[27] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1419,7 +1515,7 @@ func (x *ResetOffsetToTimestampRequest) String() string { func (*ResetOffsetToTimestampRequest) ProtoMessage() {} func (x *ResetOffsetToTimestampRequest) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[25] + mi := &file_controller_proto_msgTypes[27] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1432,7 +1528,7 @@ func (x *ResetOffsetToTimestampRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ResetOffsetToTimestampRequest.ProtoReflect.Descriptor instead. func (*ResetOffsetToTimestampRequest) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{25} + return file_controller_proto_rawDescGZIP(), []int{27} } func (x *ResetOffsetToTimestampRequest) GetSubscriptionId() uint64 { @@ -1449,6 +1545,53 @@ func (x *ResetOffsetToTimestampRequest) GetTimestamp() uint64 { return 0 } +type ResetOffsetToTimestampResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Offsets []*meta.OffsetInfo `protobuf:"bytes,1,rep,name=offsets,proto3" json:"offsets,omitempty"` +} + +func (x *ResetOffsetToTimestampResponse) Reset() { + *x = ResetOffsetToTimestampResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_controller_proto_msgTypes[28] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ResetOffsetToTimestampResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ResetOffsetToTimestampResponse) ProtoMessage() {} + +func (x *ResetOffsetToTimestampResponse) ProtoReflect() protoreflect.Message { + mi := &file_controller_proto_msgTypes[28] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ResetOffsetToTimestampResponse.ProtoReflect.Descriptor instead. +func (*ResetOffsetToTimestampResponse) Descriptor() ([]byte, []int) { + return file_controller_proto_rawDescGZIP(), []int{28} +} + +func (x *ResetOffsetToTimestampResponse) GetOffsets() []*meta.OffsetInfo { + if x != nil { + return x.Offsets + } + return nil +} + type CommitOffsetRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1461,7 +1604,7 @@ type CommitOffsetRequest struct { func (x *CommitOffsetRequest) Reset() { *x = CommitOffsetRequest{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[26] + mi := &file_controller_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1474,7 +1617,7 @@ func (x *CommitOffsetRequest) String() string { func (*CommitOffsetRequest) ProtoMessage() {} func (x *CommitOffsetRequest) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[26] + mi := &file_controller_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1487,7 +1630,7 @@ func (x *CommitOffsetRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CommitOffsetRequest.ProtoReflect.Descriptor instead. func (*CommitOffsetRequest) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{26} + return file_controller_proto_rawDescGZIP(), []int{29} } func (x *CommitOffsetRequest) GetSubscriptionInfo() []*meta.SubscriptionInfo { @@ -1515,7 +1658,7 @@ type CommitOffsetResponse struct { func (x *CommitOffsetResponse) Reset() { *x = CommitOffsetResponse{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[27] + mi := &file_controller_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1528,7 +1671,7 @@ func (x *CommitOffsetResponse) String() string { func (*CommitOffsetResponse) ProtoMessage() {} func (x *CommitOffsetResponse) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[27] + mi := &file_controller_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1541,7 +1684,7 @@ func (x *CommitOffsetResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CommitOffsetResponse.ProtoReflect.Descriptor instead. func (*CommitOffsetResponse) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{27} + return file_controller_proto_rawDescGZIP(), []int{30} } func (x *CommitOffsetResponse) GetFailSubscriptionId() []uint64 { @@ -1571,7 +1714,7 @@ type ListSegmentRequest struct { func (x *ListSegmentRequest) Reset() { *x = ListSegmentRequest{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[28] + mi := &file_controller_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1584,7 +1727,7 @@ func (x *ListSegmentRequest) String() string { func (*ListSegmentRequest) ProtoMessage() {} func (x *ListSegmentRequest) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[28] + mi := &file_controller_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1597,7 +1740,7 @@ func (x *ListSegmentRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListSegmentRequest.ProtoReflect.Descriptor instead. func (*ListSegmentRequest) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{28} + return file_controller_proto_rawDescGZIP(), []int{31} } func (x *ListSegmentRequest) GetEventBusId() uint64 { @@ -1646,7 +1789,7 @@ type ListSegmentResponse struct { func (x *ListSegmentResponse) Reset() { *x = ListSegmentResponse{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[29] + mi := &file_controller_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1659,7 +1802,7 @@ func (x *ListSegmentResponse) String() string { func (*ListSegmentResponse) ProtoMessage() {} func (x *ListSegmentResponse) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[29] + mi := &file_controller_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1672,7 +1815,7 @@ func (x *ListSegmentResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListSegmentResponse.ProtoReflect.Descriptor instead. func (*ListSegmentResponse) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{29} + return file_controller_proto_rawDescGZIP(), []int{32} } func (x *ListSegmentResponse) GetSegments() []*meta.Segment { @@ -1696,7 +1839,7 @@ type GetAppendableSegmentRequest struct { func (x *GetAppendableSegmentRequest) Reset() { *x = GetAppendableSegmentRequest{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[30] + mi := &file_controller_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1709,7 +1852,7 @@ func (x *GetAppendableSegmentRequest) String() string { func (*GetAppendableSegmentRequest) ProtoMessage() {} func (x *GetAppendableSegmentRequest) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[30] + mi := &file_controller_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1722,7 +1865,7 @@ func (x *GetAppendableSegmentRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetAppendableSegmentRequest.ProtoReflect.Descriptor instead. func (*GetAppendableSegmentRequest) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{30} + return file_controller_proto_rawDescGZIP(), []int{33} } func (x *GetAppendableSegmentRequest) GetEventBusId() uint64 { @@ -1757,7 +1900,7 @@ type GetAppendableSegmentResponse struct { func (x *GetAppendableSegmentResponse) Reset() { *x = GetAppendableSegmentResponse{} if protoimpl.UnsafeEnabled { - mi := &file_controller_proto_msgTypes[31] + mi := &file_controller_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1770,7 +1913,7 @@ func (x *GetAppendableSegmentResponse) String() string { func (*GetAppendableSegmentResponse) ProtoMessage() {} func (x *GetAppendableSegmentResponse) ProtoReflect() protoreflect.Message { - mi := &file_controller_proto_msgTypes[31] + mi := &file_controller_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1783,7 +1926,7 @@ func (x *GetAppendableSegmentResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetAppendableSegmentResponse.ProtoReflect.Descriptor instead. func (*GetAppendableSegmentResponse) Descriptor() ([]byte, []int) { - return file_controller_proto_rawDescGZIP(), []int{31} + return file_controller_proto_rawDescGZIP(), []int{34} } func (x *GetAppendableSegmentResponse) GetSegments() []*meta.Segment { @@ -1946,285 +2089,311 @@ var file_controller_proto_rawDesc = []byte{ 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, + 0x2c, 0x0a, 0x1a, 0x44, 0x69, 0x73, 0x61, 0x62, 0x6c, 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, 0x2b, 0x0a, + 0x19, 0x52, 0x65, 0x73, 0x75, 0x6d, 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, 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, + 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, 0x5a, 0x0a, 0x1e, 0x52, 0x65, 0x73, + 0x65, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x54, 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x38, 0x0a, 0x07, 0x6f, + 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 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, 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, 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, 0x2e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x6f, 0x66, + 0x66, 0x73, 0x65, 0x74, 0x73, 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, 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, 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, + 0x45, 0x76, 0x65, 0x6e, 0x74, 0x42, 0x75, 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, 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, + 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, 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, + 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, 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, 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, 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, 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, + 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, 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, + 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, 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, 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, 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, + 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, 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, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x32, 0x92, 0x0b, 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, 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, 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, + 0x70, 0x74, 0x79, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x13, 0x44, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, + 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 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, 0x44, 0x69, 0x73, 0x61, 0x62, 0x6c, 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, 0x12, 0x61, 0x0a, 0x12, 0x52, 0x65, + 0x73, 0x75, 0x6d, 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, 0x52, 0x65, 0x73, 0x75, + 0x6d, 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, 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, 0x39, 0x2e, 0x6c, 0x69, 0x6e, + 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, 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, + 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, 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, 0x8b, 0x01, 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, + 0x70, 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, 0x52, 0x65, 0x73, 0x65, 0x74, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x54, + 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 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 ( @@ -2239,7 +2408,7 @@ func file_controller_proto_rawDescGZIP() []byte { return file_controller_proto_rawDescData } -var file_controller_proto_msgTypes = make([]protoimpl.MessageInfo, 33) +var file_controller_proto_msgTypes = make([]protoimpl.MessageInfo, 36) var file_controller_proto_goTypes = []interface{}{ (*PingResponse)(nil), // 0: linkall.vanus.controller.PingResponse (*CreateEventBusRequest)(nil), // 1: linkall.vanus.controller.CreateEventBusRequest @@ -2259,115 +2428,124 @@ var file_controller_proto_goTypes = []interface{}{ (*UpdateSubscriptionRequest)(nil), // 15: linkall.vanus.controller.UpdateSubscriptionRequest (*GetSubscriptionRequest)(nil), // 16: linkall.vanus.controller.GetSubscriptionRequest (*DeleteSubscriptionRequest)(nil), // 17: linkall.vanus.controller.DeleteSubscriptionRequest - (*ListSubscriptionResponse)(nil), // 18: linkall.vanus.controller.ListSubscriptionResponse - (*RegisterTriggerWorkerRequest)(nil), // 19: linkall.vanus.controller.RegisterTriggerWorkerRequest - (*RegisterTriggerWorkerResponse)(nil), // 20: linkall.vanus.controller.RegisterTriggerWorkerResponse - (*UnregisterTriggerWorkerRequest)(nil), // 21: linkall.vanus.controller.UnregisterTriggerWorkerRequest - (*UnregisterTriggerWorkerResponse)(nil), // 22: linkall.vanus.controller.UnregisterTriggerWorkerResponse - (*TriggerWorkerHeartbeatRequest)(nil), // 23: linkall.vanus.controller.TriggerWorkerHeartbeatRequest - (*TriggerWorkerHeartbeatResponse)(nil), // 24: linkall.vanus.controller.TriggerWorkerHeartbeatResponse - (*ResetOffsetToTimestampRequest)(nil), // 25: linkall.vanus.controller.ResetOffsetToTimestampRequest - (*CommitOffsetRequest)(nil), // 26: linkall.vanus.controller.CommitOffsetRequest - (*CommitOffsetResponse)(nil), // 27: linkall.vanus.controller.CommitOffsetResponse - (*ListSegmentRequest)(nil), // 28: linkall.vanus.controller.ListSegmentRequest - (*ListSegmentResponse)(nil), // 29: linkall.vanus.controller.ListSegmentResponse - (*GetAppendableSegmentRequest)(nil), // 30: linkall.vanus.controller.GetAppendableSegmentRequest - (*GetAppendableSegmentResponse)(nil), // 31: linkall.vanus.controller.GetAppendableSegmentResponse - nil, // 32: linkall.vanus.controller.RegisterSegmentServerResponse.SegmentsEntry - (*meta.EventBus)(nil), // 33: linkall.vanus.meta.EventBus - (*meta.SegmentHealthInfo)(nil), // 34: linkall.vanus.meta.SegmentHealthInfo - (*meta.SubscriptionConfig)(nil), // 35: linkall.vanus.meta.SubscriptionConfig - (*meta.Filter)(nil), // 36: linkall.vanus.meta.Filter - (*meta.SinkCredential)(nil), // 37: linkall.vanus.meta.SinkCredential - (meta.Protocol)(0), // 38: linkall.vanus.meta.Protocol - (*meta.ProtocolSetting)(nil), // 39: linkall.vanus.meta.ProtocolSetting - (*meta.Transformer)(nil), // 40: linkall.vanus.meta.Transformer - (*meta.Subscription)(nil), // 41: linkall.vanus.meta.Subscription - (*meta.SubscriptionInfo)(nil), // 42: linkall.vanus.meta.SubscriptionInfo - (*meta.Segment)(nil), // 43: linkall.vanus.meta.Segment - (*emptypb.Empty)(nil), // 44: google.protobuf.Empty - (*wrapperspb.UInt32Value)(nil), // 45: google.protobuf.UInt32Value - (*timestamppb.Timestamp)(nil), // 46: google.protobuf.Timestamp + (*DisableSubscriptionRequest)(nil), // 18: linkall.vanus.controller.DisableSubscriptionRequest + (*ResumeSubscriptionRequest)(nil), // 19: linkall.vanus.controller.ResumeSubscriptionRequest + (*ListSubscriptionResponse)(nil), // 20: linkall.vanus.controller.ListSubscriptionResponse + (*RegisterTriggerWorkerRequest)(nil), // 21: linkall.vanus.controller.RegisterTriggerWorkerRequest + (*RegisterTriggerWorkerResponse)(nil), // 22: linkall.vanus.controller.RegisterTriggerWorkerResponse + (*UnregisterTriggerWorkerRequest)(nil), // 23: linkall.vanus.controller.UnregisterTriggerWorkerRequest + (*UnregisterTriggerWorkerResponse)(nil), // 24: linkall.vanus.controller.UnregisterTriggerWorkerResponse + (*TriggerWorkerHeartbeatRequest)(nil), // 25: linkall.vanus.controller.TriggerWorkerHeartbeatRequest + (*TriggerWorkerHeartbeatResponse)(nil), // 26: linkall.vanus.controller.TriggerWorkerHeartbeatResponse + (*ResetOffsetToTimestampRequest)(nil), // 27: linkall.vanus.controller.ResetOffsetToTimestampRequest + (*ResetOffsetToTimestampResponse)(nil), // 28: linkall.vanus.controller.ResetOffsetToTimestampResponse + (*CommitOffsetRequest)(nil), // 29: linkall.vanus.controller.CommitOffsetRequest + (*CommitOffsetResponse)(nil), // 30: linkall.vanus.controller.CommitOffsetResponse + (*ListSegmentRequest)(nil), // 31: linkall.vanus.controller.ListSegmentRequest + (*ListSegmentResponse)(nil), // 32: linkall.vanus.controller.ListSegmentResponse + (*GetAppendableSegmentRequest)(nil), // 33: linkall.vanus.controller.GetAppendableSegmentRequest + (*GetAppendableSegmentResponse)(nil), // 34: linkall.vanus.controller.GetAppendableSegmentResponse + nil, // 35: linkall.vanus.controller.RegisterSegmentServerResponse.SegmentsEntry + (*meta.EventBus)(nil), // 36: linkall.vanus.meta.EventBus + (*meta.SegmentHealthInfo)(nil), // 37: linkall.vanus.meta.SegmentHealthInfo + (*meta.SubscriptionConfig)(nil), // 38: linkall.vanus.meta.SubscriptionConfig + (*meta.Filter)(nil), // 39: linkall.vanus.meta.Filter + (*meta.SinkCredential)(nil), // 40: linkall.vanus.meta.SinkCredential + (meta.Protocol)(0), // 41: linkall.vanus.meta.Protocol + (*meta.ProtocolSetting)(nil), // 42: linkall.vanus.meta.ProtocolSetting + (*meta.Transformer)(nil), // 43: linkall.vanus.meta.Transformer + (*meta.Subscription)(nil), // 44: linkall.vanus.meta.Subscription + (*meta.SubscriptionInfo)(nil), // 45: linkall.vanus.meta.SubscriptionInfo + (*meta.OffsetInfo)(nil), // 46: linkall.vanus.meta.OffsetInfo + (*meta.Segment)(nil), // 47: linkall.vanus.meta.Segment + (*emptypb.Empty)(nil), // 48: google.protobuf.Empty + (*wrapperspb.UInt32Value)(nil), // 49: google.protobuf.UInt32Value + (*timestamppb.Timestamp)(nil), // 50: google.protobuf.Timestamp } var file_controller_proto_depIdxs = []int32{ - 33, // 0: linkall.vanus.controller.ListEventbusResponse.eventbus:type_name -> linkall.vanus.meta.EventBus - 34, // 1: linkall.vanus.controller.SegmentHeartbeatRequest.health_info:type_name -> linkall.vanus.meta.SegmentHealthInfo - 32, // 2: linkall.vanus.controller.RegisterSegmentServerResponse.segments:type_name -> linkall.vanus.controller.RegisterSegmentServerResponse.SegmentsEntry - 35, // 3: linkall.vanus.controller.SubscriptionRequest.config:type_name -> linkall.vanus.meta.SubscriptionConfig - 36, // 4: linkall.vanus.controller.SubscriptionRequest.filters:type_name -> linkall.vanus.meta.Filter - 37, // 5: linkall.vanus.controller.SubscriptionRequest.sink_credential:type_name -> linkall.vanus.meta.SinkCredential - 38, // 6: linkall.vanus.controller.SubscriptionRequest.protocol:type_name -> linkall.vanus.meta.Protocol - 39, // 7: linkall.vanus.controller.SubscriptionRequest.protocol_settings:type_name -> linkall.vanus.meta.ProtocolSetting - 40, // 8: linkall.vanus.controller.SubscriptionRequest.transformer:type_name -> linkall.vanus.meta.Transformer + 36, // 0: linkall.vanus.controller.ListEventbusResponse.eventbus:type_name -> linkall.vanus.meta.EventBus + 37, // 1: linkall.vanus.controller.SegmentHeartbeatRequest.health_info:type_name -> linkall.vanus.meta.SegmentHealthInfo + 35, // 2: linkall.vanus.controller.RegisterSegmentServerResponse.segments:type_name -> linkall.vanus.controller.RegisterSegmentServerResponse.SegmentsEntry + 38, // 3: linkall.vanus.controller.SubscriptionRequest.config:type_name -> linkall.vanus.meta.SubscriptionConfig + 39, // 4: linkall.vanus.controller.SubscriptionRequest.filters:type_name -> linkall.vanus.meta.Filter + 40, // 5: linkall.vanus.controller.SubscriptionRequest.sink_credential:type_name -> linkall.vanus.meta.SinkCredential + 41, // 6: linkall.vanus.controller.SubscriptionRequest.protocol:type_name -> linkall.vanus.meta.Protocol + 42, // 7: linkall.vanus.controller.SubscriptionRequest.protocol_settings:type_name -> linkall.vanus.meta.ProtocolSetting + 43, // 8: linkall.vanus.controller.SubscriptionRequest.transformer:type_name -> linkall.vanus.meta.Transformer 13, // 9: linkall.vanus.controller.CreateSubscriptionRequest.subscription:type_name -> linkall.vanus.controller.SubscriptionRequest 13, // 10: linkall.vanus.controller.UpdateSubscriptionRequest.subscription:type_name -> linkall.vanus.controller.SubscriptionRequest - 41, // 11: linkall.vanus.controller.ListSubscriptionResponse.subscription:type_name -> linkall.vanus.meta.Subscription - 42, // 12: linkall.vanus.controller.TriggerWorkerHeartbeatRequest.subscription_info:type_name -> linkall.vanus.meta.SubscriptionInfo - 42, // 13: linkall.vanus.controller.CommitOffsetRequest.subscription_info:type_name -> linkall.vanus.meta.SubscriptionInfo - 43, // 14: linkall.vanus.controller.ListSegmentResponse.segments:type_name -> linkall.vanus.meta.Segment - 43, // 15: linkall.vanus.controller.GetAppendableSegmentResponse.segments:type_name -> linkall.vanus.meta.Segment - 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 - 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 + 44, // 11: linkall.vanus.controller.ListSubscriptionResponse.subscription:type_name -> linkall.vanus.meta.Subscription + 45, // 12: linkall.vanus.controller.TriggerWorkerHeartbeatRequest.subscription_info:type_name -> linkall.vanus.meta.SubscriptionInfo + 46, // 13: linkall.vanus.controller.ResetOffsetToTimestampResponse.offsets:type_name -> linkall.vanus.meta.OffsetInfo + 45, // 14: linkall.vanus.controller.CommitOffsetRequest.subscription_info:type_name -> linkall.vanus.meta.SubscriptionInfo + 47, // 15: linkall.vanus.controller.ListSegmentResponse.segments:type_name -> linkall.vanus.meta.Segment + 47, // 16: linkall.vanus.controller.GetAppendableSegmentResponse.segments:type_name -> linkall.vanus.meta.Segment + 47, // 17: linkall.vanus.controller.RegisterSegmentServerResponse.SegmentsEntry.value:type_name -> linkall.vanus.meta.Segment + 48, // 18: linkall.vanus.controller.PingServer.Ping:input_type -> google.protobuf.Empty + 1, // 19: linkall.vanus.controller.EventBusController.CreateEventBus:input_type -> linkall.vanus.controller.CreateEventBusRequest + 1, // 20: linkall.vanus.controller.EventBusController.CreateSystemEventBus:input_type -> linkall.vanus.controller.CreateEventBusRequest + 36, // 21: linkall.vanus.controller.EventBusController.DeleteEventBus:input_type -> linkall.vanus.meta.EventBus + 36, // 22: linkall.vanus.controller.EventBusController.GetEventBus:input_type -> linkall.vanus.meta.EventBus + 48, // 23: linkall.vanus.controller.EventBusController.ListEventBus:input_type -> google.protobuf.Empty + 3, // 24: linkall.vanus.controller.EventBusController.UpdateEventBus:input_type -> linkall.vanus.controller.UpdateEventBusRequest + 31, // 25: linkall.vanus.controller.EventLogController.ListSegment:input_type -> linkall.vanus.controller.ListSegmentRequest + 33, // 26: linkall.vanus.controller.EventLogController.GetAppendableSegment:input_type -> linkall.vanus.controller.GetAppendableSegmentRequest + 4, // 27: linkall.vanus.controller.SegmentController.QuerySegmentRouteInfo:input_type -> linkall.vanus.controller.QuerySegmentRouteInfoRequest + 6, // 28: linkall.vanus.controller.SegmentController.SegmentHeartbeat:input_type -> linkall.vanus.controller.SegmentHeartbeatRequest + 8, // 29: linkall.vanus.controller.SegmentController.RegisterSegmentServer:input_type -> linkall.vanus.controller.RegisterSegmentServerRequest + 10, // 30: linkall.vanus.controller.SegmentController.UnregisterSegmentServer:input_type -> linkall.vanus.controller.UnregisterSegmentServerRequest + 6, // 31: linkall.vanus.controller.SegmentController.ReportSegmentBlockIsFull:input_type -> linkall.vanus.controller.SegmentHeartbeatRequest + 12, // 32: linkall.vanus.controller.SegmentController.ReportSegmentLeader:input_type -> linkall.vanus.controller.ReportSegmentLeaderRequest + 14, // 33: linkall.vanus.controller.TriggerController.CreateSubscription:input_type -> linkall.vanus.controller.CreateSubscriptionRequest + 15, // 34: linkall.vanus.controller.TriggerController.UpdateSubscription:input_type -> linkall.vanus.controller.UpdateSubscriptionRequest + 17, // 35: linkall.vanus.controller.TriggerController.DeleteSubscription:input_type -> linkall.vanus.controller.DeleteSubscriptionRequest + 18, // 36: linkall.vanus.controller.TriggerController.DisableSubscription:input_type -> linkall.vanus.controller.DisableSubscriptionRequest + 19, // 37: linkall.vanus.controller.TriggerController.ResumeSubscription:input_type -> linkall.vanus.controller.ResumeSubscriptionRequest + 16, // 38: linkall.vanus.controller.TriggerController.GetSubscription:input_type -> linkall.vanus.controller.GetSubscriptionRequest + 48, // 39: linkall.vanus.controller.TriggerController.ListSubscription:input_type -> google.protobuf.Empty + 25, // 40: linkall.vanus.controller.TriggerController.TriggerWorkerHeartbeat:input_type -> linkall.vanus.controller.TriggerWorkerHeartbeatRequest + 21, // 41: linkall.vanus.controller.TriggerController.RegisterTriggerWorker:input_type -> linkall.vanus.controller.RegisterTriggerWorkerRequest + 23, // 42: linkall.vanus.controller.TriggerController.UnregisterTriggerWorker:input_type -> linkall.vanus.controller.UnregisterTriggerWorkerRequest + 27, // 43: linkall.vanus.controller.TriggerController.ResetOffsetToTimestamp:input_type -> linkall.vanus.controller.ResetOffsetToTimestampRequest + 29, // 44: linkall.vanus.controller.TriggerController.CommitOffset:input_type -> linkall.vanus.controller.CommitOffsetRequest + 48, // 45: linkall.vanus.controller.SnowflakeController.GetClusterStartTime:input_type -> google.protobuf.Empty + 49, // 46: linkall.vanus.controller.SnowflakeController.RegisterNode:input_type -> google.protobuf.UInt32Value + 49, // 47: linkall.vanus.controller.SnowflakeController.UnregisterNode:input_type -> google.protobuf.UInt32Value + 0, // 48: linkall.vanus.controller.PingServer.Ping:output_type -> linkall.vanus.controller.PingResponse + 36, // 49: linkall.vanus.controller.EventBusController.CreateEventBus:output_type -> linkall.vanus.meta.EventBus + 36, // 50: linkall.vanus.controller.EventBusController.CreateSystemEventBus:output_type -> linkall.vanus.meta.EventBus + 48, // 51: linkall.vanus.controller.EventBusController.DeleteEventBus:output_type -> google.protobuf.Empty + 36, // 52: linkall.vanus.controller.EventBusController.GetEventBus:output_type -> linkall.vanus.meta.EventBus + 2, // 53: linkall.vanus.controller.EventBusController.ListEventBus:output_type -> linkall.vanus.controller.ListEventbusResponse + 36, // 54: linkall.vanus.controller.EventBusController.UpdateEventBus:output_type -> linkall.vanus.meta.EventBus + 32, // 55: linkall.vanus.controller.EventLogController.ListSegment:output_type -> linkall.vanus.controller.ListSegmentResponse + 34, // 56: linkall.vanus.controller.EventLogController.GetAppendableSegment:output_type -> linkall.vanus.controller.GetAppendableSegmentResponse + 5, // 57: linkall.vanus.controller.SegmentController.QuerySegmentRouteInfo:output_type -> linkall.vanus.controller.QuerySegmentRouteInfoResponse + 7, // 58: linkall.vanus.controller.SegmentController.SegmentHeartbeat:output_type -> linkall.vanus.controller.SegmentHeartbeatResponse + 9, // 59: linkall.vanus.controller.SegmentController.RegisterSegmentServer:output_type -> linkall.vanus.controller.RegisterSegmentServerResponse + 11, // 60: linkall.vanus.controller.SegmentController.UnregisterSegmentServer:output_type -> linkall.vanus.controller.UnregisterSegmentServerResponse + 48, // 61: linkall.vanus.controller.SegmentController.ReportSegmentBlockIsFull:output_type -> google.protobuf.Empty + 48, // 62: linkall.vanus.controller.SegmentController.ReportSegmentLeader:output_type -> google.protobuf.Empty + 44, // 63: linkall.vanus.controller.TriggerController.CreateSubscription:output_type -> linkall.vanus.meta.Subscription + 44, // 64: linkall.vanus.controller.TriggerController.UpdateSubscription:output_type -> linkall.vanus.meta.Subscription + 48, // 65: linkall.vanus.controller.TriggerController.DeleteSubscription:output_type -> google.protobuf.Empty + 48, // 66: linkall.vanus.controller.TriggerController.DisableSubscription:output_type -> google.protobuf.Empty + 48, // 67: linkall.vanus.controller.TriggerController.ResumeSubscription:output_type -> google.protobuf.Empty + 44, // 68: linkall.vanus.controller.TriggerController.GetSubscription:output_type -> linkall.vanus.meta.Subscription + 20, // 69: linkall.vanus.controller.TriggerController.ListSubscription:output_type -> linkall.vanus.controller.ListSubscriptionResponse + 26, // 70: linkall.vanus.controller.TriggerController.TriggerWorkerHeartbeat:output_type -> linkall.vanus.controller.TriggerWorkerHeartbeatResponse + 22, // 71: linkall.vanus.controller.TriggerController.RegisterTriggerWorker:output_type -> linkall.vanus.controller.RegisterTriggerWorkerResponse + 24, // 72: linkall.vanus.controller.TriggerController.UnregisterTriggerWorker:output_type -> linkall.vanus.controller.UnregisterTriggerWorkerResponse + 28, // 73: linkall.vanus.controller.TriggerController.ResetOffsetToTimestamp:output_type -> linkall.vanus.controller.ResetOffsetToTimestampResponse + 30, // 74: linkall.vanus.controller.TriggerController.CommitOffset:output_type -> linkall.vanus.controller.CommitOffsetResponse + 50, // 75: linkall.vanus.controller.SnowflakeController.GetClusterStartTime:output_type -> google.protobuf.Timestamp + 48, // 76: linkall.vanus.controller.SnowflakeController.RegisterNode:output_type -> google.protobuf.Empty + 48, // 77: linkall.vanus.controller.SnowflakeController.UnregisterNode:output_type -> google.protobuf.Empty + 48, // [48:78] is the sub-list for method output_type + 18, // [18:48] is the sub-list for method input_type + 18, // [18:18] is the sub-list for extension type_name + 18, // [18:18] is the sub-list for extension extendee + 0, // [0:18] is the sub-list for field type_name } func init() { file_controller_proto_init() } @@ -2593,7 +2771,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListSubscriptionResponse); i { + switch v := v.(*DisableSubscriptionRequest); i { case 0: return &v.state case 1: @@ -2605,7 +2783,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RegisterTriggerWorkerRequest); i { + switch v := v.(*ResumeSubscriptionRequest); i { case 0: return &v.state case 1: @@ -2617,7 +2795,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RegisterTriggerWorkerResponse); i { + switch v := v.(*ListSubscriptionResponse); i { case 0: return &v.state case 1: @@ -2629,7 +2807,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UnregisterTriggerWorkerRequest); i { + switch v := v.(*RegisterTriggerWorkerRequest); i { case 0: return &v.state case 1: @@ -2641,7 +2819,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UnregisterTriggerWorkerResponse); i { + switch v := v.(*RegisterTriggerWorkerResponse); i { case 0: return &v.state case 1: @@ -2653,7 +2831,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TriggerWorkerHeartbeatRequest); i { + switch v := v.(*UnregisterTriggerWorkerRequest); i { case 0: return &v.state case 1: @@ -2665,7 +2843,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TriggerWorkerHeartbeatResponse); i { + switch v := v.(*UnregisterTriggerWorkerResponse); i { case 0: return &v.state case 1: @@ -2677,7 +2855,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResetOffsetToTimestampRequest); i { + switch v := v.(*TriggerWorkerHeartbeatRequest); i { case 0: return &v.state case 1: @@ -2689,7 +2867,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CommitOffsetRequest); i { + switch v := v.(*TriggerWorkerHeartbeatResponse); i { case 0: return &v.state case 1: @@ -2701,7 +2879,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CommitOffsetResponse); i { + switch v := v.(*ResetOffsetToTimestampRequest); i { case 0: return &v.state case 1: @@ -2713,7 +2891,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListSegmentRequest); i { + switch v := v.(*ResetOffsetToTimestampResponse); i { case 0: return &v.state case 1: @@ -2725,7 +2903,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListSegmentResponse); i { + switch v := v.(*CommitOffsetRequest); i { case 0: return &v.state case 1: @@ -2737,7 +2915,7 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetAppendableSegmentRequest); i { + switch v := v.(*CommitOffsetResponse); i { case 0: return &v.state case 1: @@ -2749,6 +2927,42 @@ func file_controller_proto_init() { } } file_controller_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListSegmentRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_controller_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ListSegmentResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_controller_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*GetAppendableSegmentRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_controller_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*GetAppendableSegmentResponse); i { case 0: return &v.state @@ -2767,7 +2981,7 @@ func file_controller_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_controller_proto_rawDesc, NumEnums: 0, - NumMessages: 33, + NumMessages: 36, NumExtensions: 0, NumServices: 6, }, @@ -3517,12 +3731,14 @@ type TriggerControllerClient interface { CreateSubscription(ctx context.Context, in *CreateSubscriptionRequest, opts ...grpc.CallOption) (*meta.Subscription, error) UpdateSubscription(ctx context.Context, in *UpdateSubscriptionRequest, opts ...grpc.CallOption) (*meta.Subscription, error) DeleteSubscription(ctx context.Context, in *DeleteSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) + DisableSubscription(ctx context.Context, in *DisableSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) + ResumeSubscription(ctx context.Context, in *ResumeSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) GetSubscription(ctx context.Context, in *GetSubscriptionRequest, opts ...grpc.CallOption) (*meta.Subscription, error) ListSubscription(ctx context.Context, in *emptypb.Empty, opts ...grpc.CallOption) (*ListSubscriptionResponse, error) TriggerWorkerHeartbeat(ctx context.Context, opts ...grpc.CallOption) (TriggerController_TriggerWorkerHeartbeatClient, error) RegisterTriggerWorker(ctx context.Context, in *RegisterTriggerWorkerRequest, opts ...grpc.CallOption) (*RegisterTriggerWorkerResponse, error) UnregisterTriggerWorker(ctx context.Context, in *UnregisterTriggerWorkerRequest, opts ...grpc.CallOption) (*UnregisterTriggerWorkerResponse, error) - ResetOffsetToTimestamp(ctx context.Context, in *ResetOffsetToTimestampRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) + ResetOffsetToTimestamp(ctx context.Context, in *ResetOffsetToTimestampRequest, opts ...grpc.CallOption) (*ResetOffsetToTimestampResponse, error) CommitOffset(ctx context.Context, in *CommitOffsetRequest, opts ...grpc.CallOption) (*CommitOffsetResponse, error) } @@ -3561,6 +3777,24 @@ func (c *triggerControllerClient) DeleteSubscription(ctx context.Context, in *De return out, nil } +func (c *triggerControllerClient) DisableSubscription(ctx context.Context, in *DisableSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) { + out := new(emptypb.Empty) + err := c.cc.Invoke(ctx, "/linkall.vanus.controller.TriggerController/DisableSubscription", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *triggerControllerClient) ResumeSubscription(ctx context.Context, in *ResumeSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) { + out := new(emptypb.Empty) + err := c.cc.Invoke(ctx, "/linkall.vanus.controller.TriggerController/ResumeSubscription", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *triggerControllerClient) GetSubscription(ctx context.Context, in *GetSubscriptionRequest, opts ...grpc.CallOption) (*meta.Subscription, error) { out := new(meta.Subscription) err := c.cc.Invoke(ctx, "/linkall.vanus.controller.TriggerController/GetSubscription", in, out, opts...) @@ -3631,8 +3865,8 @@ func (c *triggerControllerClient) UnregisterTriggerWorker(ctx context.Context, i return out, nil } -func (c *triggerControllerClient) ResetOffsetToTimestamp(ctx context.Context, in *ResetOffsetToTimestampRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) { - out := new(emptypb.Empty) +func (c *triggerControllerClient) ResetOffsetToTimestamp(ctx context.Context, in *ResetOffsetToTimestampRequest, opts ...grpc.CallOption) (*ResetOffsetToTimestampResponse, error) { + out := new(ResetOffsetToTimestampResponse) err := c.cc.Invoke(ctx, "/linkall.vanus.controller.TriggerController/ResetOffsetToTimestamp", in, out, opts...) if err != nil { return nil, err @@ -3654,12 +3888,14 @@ type TriggerControllerServer interface { CreateSubscription(context.Context, *CreateSubscriptionRequest) (*meta.Subscription, error) UpdateSubscription(context.Context, *UpdateSubscriptionRequest) (*meta.Subscription, error) DeleteSubscription(context.Context, *DeleteSubscriptionRequest) (*emptypb.Empty, error) + DisableSubscription(context.Context, *DisableSubscriptionRequest) (*emptypb.Empty, error) + ResumeSubscription(context.Context, *ResumeSubscriptionRequest) (*emptypb.Empty, error) GetSubscription(context.Context, *GetSubscriptionRequest) (*meta.Subscription, error) ListSubscription(context.Context, *emptypb.Empty) (*ListSubscriptionResponse, error) TriggerWorkerHeartbeat(TriggerController_TriggerWorkerHeartbeatServer) error RegisterTriggerWorker(context.Context, *RegisterTriggerWorkerRequest) (*RegisterTriggerWorkerResponse, error) UnregisterTriggerWorker(context.Context, *UnregisterTriggerWorkerRequest) (*UnregisterTriggerWorkerResponse, error) - ResetOffsetToTimestamp(context.Context, *ResetOffsetToTimestampRequest) (*emptypb.Empty, error) + ResetOffsetToTimestamp(context.Context, *ResetOffsetToTimestampRequest) (*ResetOffsetToTimestampResponse, error) CommitOffset(context.Context, *CommitOffsetRequest) (*CommitOffsetResponse, error) } @@ -3676,6 +3912,12 @@ func (*UnimplementedTriggerControllerServer) UpdateSubscription(context.Context, func (*UnimplementedTriggerControllerServer) DeleteSubscription(context.Context, *DeleteSubscriptionRequest) (*emptypb.Empty, error) { return nil, status.Errorf(codes.Unimplemented, "method DeleteSubscription not implemented") } +func (*UnimplementedTriggerControllerServer) DisableSubscription(context.Context, *DisableSubscriptionRequest) (*emptypb.Empty, error) { + return nil, status.Errorf(codes.Unimplemented, "method DisableSubscription not implemented") +} +func (*UnimplementedTriggerControllerServer) ResumeSubscription(context.Context, *ResumeSubscriptionRequest) (*emptypb.Empty, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResumeSubscription not implemented") +} func (*UnimplementedTriggerControllerServer) GetSubscription(context.Context, *GetSubscriptionRequest) (*meta.Subscription, error) { return nil, status.Errorf(codes.Unimplemented, "method GetSubscription not implemented") } @@ -3691,7 +3933,7 @@ func (*UnimplementedTriggerControllerServer) RegisterTriggerWorker(context.Conte func (*UnimplementedTriggerControllerServer) UnregisterTriggerWorker(context.Context, *UnregisterTriggerWorkerRequest) (*UnregisterTriggerWorkerResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method UnregisterTriggerWorker not implemented") } -func (*UnimplementedTriggerControllerServer) ResetOffsetToTimestamp(context.Context, *ResetOffsetToTimestampRequest) (*emptypb.Empty, error) { +func (*UnimplementedTriggerControllerServer) ResetOffsetToTimestamp(context.Context, *ResetOffsetToTimestampRequest) (*ResetOffsetToTimestampResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResetOffsetToTimestamp not implemented") } func (*UnimplementedTriggerControllerServer) CommitOffset(context.Context, *CommitOffsetRequest) (*CommitOffsetResponse, error) { @@ -3756,6 +3998,42 @@ func _TriggerController_DeleteSubscription_Handler(srv interface{}, ctx context. return interceptor(ctx, in, info, handler) } +func _TriggerController_DisableSubscription_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(DisableSubscriptionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TriggerControllerServer).DisableSubscription(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/linkall.vanus.controller.TriggerController/DisableSubscription", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TriggerControllerServer).DisableSubscription(ctx, req.(*DisableSubscriptionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _TriggerController_ResumeSubscription_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ResumeSubscriptionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(TriggerControllerServer).ResumeSubscription(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/linkall.vanus.controller.TriggerController/ResumeSubscription", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(TriggerControllerServer).ResumeSubscription(ctx, req.(*ResumeSubscriptionRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _TriggerController_GetSubscription_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(GetSubscriptionRequest) if err := dec(in); err != nil { @@ -3906,6 +4184,14 @@ var _TriggerController_serviceDesc = grpc.ServiceDesc{ MethodName: "DeleteSubscription", Handler: _TriggerController_DeleteSubscription_Handler, }, + { + MethodName: "DisableSubscription", + Handler: _TriggerController_DisableSubscription_Handler, + }, + { + MethodName: "ResumeSubscription", + Handler: _TriggerController_ResumeSubscription_Handler, + }, { MethodName: "GetSubscription", Handler: _TriggerController_GetSubscription_Handler, diff --git a/proto/pkg/controller/mock_controller.go b/proto/pkg/controller/mock_controller.go index 49838b4dc..febca4c90 100644 --- a/proto/pkg/controller/mock_controller.go +++ b/proto/pkg/controller/mock_controller.go @@ -1079,6 +1079,26 @@ func (mr *MockTriggerControllerClientMockRecorder) DeleteSubscription(ctx, in in return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteSubscription", reflect.TypeOf((*MockTriggerControllerClient)(nil).DeleteSubscription), varargs...) } +// DisableSubscription mocks base method. +func (m *MockTriggerControllerClient) DisableSubscription(ctx context.Context, in *DisableSubscriptionRequest, 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, "DisableSubscription", varargs...) + ret0, _ := ret[0].(*emptypb.Empty) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DisableSubscription indicates an expected call of DisableSubscription. +func (mr *MockTriggerControllerClientMockRecorder) DisableSubscription(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, "DisableSubscription", reflect.TypeOf((*MockTriggerControllerClient)(nil).DisableSubscription), varargs...) +} + // GetSubscription mocks base method. func (m *MockTriggerControllerClient) GetSubscription(ctx context.Context, in *GetSubscriptionRequest, opts ...grpc.CallOption) (*meta.Subscription, error) { m.ctrl.T.Helper() @@ -1140,14 +1160,14 @@ func (mr *MockTriggerControllerClientMockRecorder) RegisterTriggerWorker(ctx, in } // ResetOffsetToTimestamp mocks base method. -func (m *MockTriggerControllerClient) ResetOffsetToTimestamp(ctx context.Context, in *ResetOffsetToTimestampRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) { +func (m *MockTriggerControllerClient) ResetOffsetToTimestamp(ctx context.Context, in *ResetOffsetToTimestampRequest, opts ...grpc.CallOption) (*ResetOffsetToTimestampResponse, error) { m.ctrl.T.Helper() varargs := []interface{}{ctx, in} for _, a := range opts { varargs = append(varargs, a) } ret := m.ctrl.Call(m, "ResetOffsetToTimestamp", varargs...) - ret0, _ := ret[0].(*emptypb.Empty) + ret0, _ := ret[0].(*ResetOffsetToTimestampResponse) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -1159,6 +1179,26 @@ func (mr *MockTriggerControllerClientMockRecorder) ResetOffsetToTimestamp(ctx, i return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetOffsetToTimestamp", reflect.TypeOf((*MockTriggerControllerClient)(nil).ResetOffsetToTimestamp), varargs...) } +// ResumeSubscription mocks base method. +func (m *MockTriggerControllerClient) ResumeSubscription(ctx context.Context, in *ResumeSubscriptionRequest, 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, "ResumeSubscription", varargs...) + ret0, _ := ret[0].(*emptypb.Empty) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ResumeSubscription indicates an expected call of ResumeSubscription. +func (mr *MockTriggerControllerClientMockRecorder) ResumeSubscription(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, "ResumeSubscription", reflect.TypeOf((*MockTriggerControllerClient)(nil).ResumeSubscription), varargs...) +} + // TriggerWorkerHeartbeat mocks base method. func (m *MockTriggerControllerClient) TriggerWorkerHeartbeat(ctx context.Context, opts ...grpc.CallOption) (TriggerController_TriggerWorkerHeartbeatClient, error) { m.ctrl.T.Helper() @@ -1424,6 +1464,21 @@ func (mr *MockTriggerControllerServerMockRecorder) DeleteSubscription(arg0, arg1 return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteSubscription", reflect.TypeOf((*MockTriggerControllerServer)(nil).DeleteSubscription), arg0, arg1) } +// DisableSubscription mocks base method. +func (m *MockTriggerControllerServer) DisableSubscription(arg0 context.Context, arg1 *DisableSubscriptionRequest) (*emptypb.Empty, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DisableSubscription", arg0, arg1) + ret0, _ := ret[0].(*emptypb.Empty) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// DisableSubscription indicates an expected call of DisableSubscription. +func (mr *MockTriggerControllerServerMockRecorder) DisableSubscription(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DisableSubscription", reflect.TypeOf((*MockTriggerControllerServer)(nil).DisableSubscription), arg0, arg1) +} + // GetSubscription mocks base method. func (m *MockTriggerControllerServer) GetSubscription(arg0 context.Context, arg1 *GetSubscriptionRequest) (*meta.Subscription, error) { m.ctrl.T.Helper() @@ -1470,10 +1525,10 @@ func (mr *MockTriggerControllerServerMockRecorder) RegisterTriggerWorker(arg0, a } // ResetOffsetToTimestamp mocks base method. -func (m *MockTriggerControllerServer) ResetOffsetToTimestamp(arg0 context.Context, arg1 *ResetOffsetToTimestampRequest) (*emptypb.Empty, error) { +func (m *MockTriggerControllerServer) ResetOffsetToTimestamp(arg0 context.Context, arg1 *ResetOffsetToTimestampRequest) (*ResetOffsetToTimestampResponse, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "ResetOffsetToTimestamp", arg0, arg1) - ret0, _ := ret[0].(*emptypb.Empty) + ret0, _ := ret[0].(*ResetOffsetToTimestampResponse) ret1, _ := ret[1].(error) return ret0, ret1 } @@ -1484,6 +1539,21 @@ func (mr *MockTriggerControllerServerMockRecorder) ResetOffsetToTimestamp(arg0, return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetOffsetToTimestamp", reflect.TypeOf((*MockTriggerControllerServer)(nil).ResetOffsetToTimestamp), arg0, arg1) } +// ResumeSubscription mocks base method. +func (m *MockTriggerControllerServer) ResumeSubscription(arg0 context.Context, arg1 *ResumeSubscriptionRequest) (*emptypb.Empty, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "ResumeSubscription", arg0, arg1) + ret0, _ := ret[0].(*emptypb.Empty) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// ResumeSubscription indicates an expected call of ResumeSubscription. +func (mr *MockTriggerControllerServerMockRecorder) ResumeSubscription(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResumeSubscription", reflect.TypeOf((*MockTriggerControllerServer)(nil).ResumeSubscription), arg0, arg1) +} + // TriggerWorkerHeartbeat mocks base method. func (m *MockTriggerControllerServer) TriggerWorkerHeartbeat(arg0 TriggerController_TriggerWorkerHeartbeatServer) error { m.ctrl.T.Helper() diff --git a/proto/pkg/proxy/proxy.pb.go b/proto/pkg/proxy/proxy.pb.go index 051d394b4..917308164 100644 --- a/proto/pkg/proxy/proxy.pb.go +++ b/proto/pkg/proxy/proxy.pb.go @@ -15,7 +15,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.26.0 -// protoc v3.19.4 +// protoc v3.19.1 // source: proxy.proto package proxy @@ -547,7 +547,7 @@ var file_proxy_proto_rawDesc = []byte{ 0x75, 0x6c, 0x74, 0x12, 0x2d, 0x0a, 0x12, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x65, 0x72, 0x5f, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x11, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x65, 0x72, 0x52, 0x65, 0x73, 0x75, - 0x6c, 0x74, 0x32, 0xba, 0x0b, 0x0a, 0x0f, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, + 0x6c, 0x74, 0x32, 0x90, 0x0e, 0x0a, 0x0f, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x6c, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x78, 0x79, 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, @@ -614,35 +614,56 @@ var file_proxy_proto_rawDesc = []byte{ 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, 0x12, 0x4f, 0x0a, 0x0b, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e, - 0x66, 0x6f, 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, 0x28, 0x2e, 0x6c, 0x69, 0x6e, + 0x6e, 0x73, 0x65, 0x12, 0x63, 0x0a, 0x13, 0x44, 0x69, 0x73, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 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, 0x44, 0x69, 0x73, 0x61, 0x62, 0x6c, 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, 0x12, 0x61, 0x0a, 0x12, 0x52, 0x65, 0x73, 0x75, + 0x6d, 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, 0x52, 0x65, 0x73, 0x75, 0x6d, 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, 0x12, 0x8b, 0x01, 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, + 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, 0x73, 0x65, 0x74, + 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x54, 0x6f, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4f, 0x0a, 0x0b, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 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, 0x28, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x63, 0x0a, 0x0c, 0x4c, 0x6f, + 0x6f, 0x6b, 0x75, 0x70, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x28, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, - 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x63, 0x0a, 0x0c, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x4f, 0x66, - 0x66, 0x73, 0x65, 0x74, 0x12, 0x28, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, + 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, - 0x70, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, - 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x4f, 0x66, 0x66, 0x73, 0x65, - 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x57, 0x0a, 0x08, 0x47, 0x65, 0x74, - 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x24, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, - 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x45, - 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x25, 0x2e, 0x6c, 0x69, + 0x70, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, + 0x57, 0x0a, 0x08, 0x47, 0x65, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x24, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x78, - 0x79, 0x2e, 0x47, 0x65, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x7b, 0x0a, 0x14, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 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, 0x70, 0x72, 0x6f, 0x78, 0x79, - 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6c, - 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x78, 0x79, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, - 0x2f, 0x5a, 0x2d, 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, 0x70, 0x72, 0x6f, 0x78, 0x79, - 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x79, 0x2e, 0x47, 0x65, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x25, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7b, 0x0a, 0x14, 0x56, 0x61, 0x6c, 0x69, + 0x64, 0x61, 0x74, 0x65, 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, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x2f, 0x5a, 0x2d, 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, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -659,29 +680,33 @@ func file_proxy_proto_rawDescGZIP() []byte { var file_proxy_proto_msgTypes = make([]protoimpl.MessageInfo, 8) var file_proxy_proto_goTypes = []interface{}{ - (*LookupOffsetRequest)(nil), // 0: linkall.vanus.proxy.LookupOffsetRequest - (*LookupOffsetResponse)(nil), // 1: linkall.vanus.proxy.LookupOffsetResponse - (*GetEventRequest)(nil), // 2: linkall.vanus.proxy.GetEventRequest - (*GetEventResponse)(nil), // 3: linkall.vanus.proxy.GetEventResponse - (*ClusterInfoResponse)(nil), // 4: linkall.vanus.proxy.ClusterInfoResponse - (*ValidateSubscriptionRequest)(nil), // 5: linkall.vanus.proxy.ValidateSubscriptionRequest - (*ValidateSubscriptionResponse)(nil), // 6: linkall.vanus.proxy.ValidateSubscriptionResponse - nil, // 7: linkall.vanus.proxy.LookupOffsetResponse.OffsetsEntry - (*wrapperspb.BytesValue)(nil), // 8: google.protobuf.BytesValue - (*controller.SubscriptionRequest)(nil), // 9: linkall.vanus.controller.SubscriptionRequest - (*controller.CreateEventBusRequest)(nil), // 10: linkall.vanus.controller.CreateEventBusRequest - (*meta.EventBus)(nil), // 11: linkall.vanus.meta.EventBus - (*emptypb.Empty)(nil), // 12: google.protobuf.Empty - (*controller.UpdateEventBusRequest)(nil), // 13: linkall.vanus.controller.UpdateEventBusRequest - (*controller.ListSegmentRequest)(nil), // 14: linkall.vanus.controller.ListSegmentRequest - (*controller.CreateSubscriptionRequest)(nil), // 15: linkall.vanus.controller.CreateSubscriptionRequest - (*controller.UpdateSubscriptionRequest)(nil), // 16: linkall.vanus.controller.UpdateSubscriptionRequest - (*controller.DeleteSubscriptionRequest)(nil), // 17: linkall.vanus.controller.DeleteSubscriptionRequest - (*controller.GetSubscriptionRequest)(nil), // 18: linkall.vanus.controller.GetSubscriptionRequest - (*controller.ListEventbusResponse)(nil), // 19: linkall.vanus.controller.ListEventbusResponse - (*controller.ListSegmentResponse)(nil), // 20: linkall.vanus.controller.ListSegmentResponse - (*meta.Subscription)(nil), // 21: linkall.vanus.meta.Subscription - (*controller.ListSubscriptionResponse)(nil), // 22: linkall.vanus.controller.ListSubscriptionResponse + (*LookupOffsetRequest)(nil), // 0: linkall.vanus.proxy.LookupOffsetRequest + (*LookupOffsetResponse)(nil), // 1: linkall.vanus.proxy.LookupOffsetResponse + (*GetEventRequest)(nil), // 2: linkall.vanus.proxy.GetEventRequest + (*GetEventResponse)(nil), // 3: linkall.vanus.proxy.GetEventResponse + (*ClusterInfoResponse)(nil), // 4: linkall.vanus.proxy.ClusterInfoResponse + (*ValidateSubscriptionRequest)(nil), // 5: linkall.vanus.proxy.ValidateSubscriptionRequest + (*ValidateSubscriptionResponse)(nil), // 6: linkall.vanus.proxy.ValidateSubscriptionResponse + nil, // 7: linkall.vanus.proxy.LookupOffsetResponse.OffsetsEntry + (*wrapperspb.BytesValue)(nil), // 8: google.protobuf.BytesValue + (*controller.SubscriptionRequest)(nil), // 9: linkall.vanus.controller.SubscriptionRequest + (*controller.CreateEventBusRequest)(nil), // 10: linkall.vanus.controller.CreateEventBusRequest + (*meta.EventBus)(nil), // 11: linkall.vanus.meta.EventBus + (*emptypb.Empty)(nil), // 12: google.protobuf.Empty + (*controller.UpdateEventBusRequest)(nil), // 13: linkall.vanus.controller.UpdateEventBusRequest + (*controller.ListSegmentRequest)(nil), // 14: linkall.vanus.controller.ListSegmentRequest + (*controller.CreateSubscriptionRequest)(nil), // 15: linkall.vanus.controller.CreateSubscriptionRequest + (*controller.UpdateSubscriptionRequest)(nil), // 16: linkall.vanus.controller.UpdateSubscriptionRequest + (*controller.DeleteSubscriptionRequest)(nil), // 17: linkall.vanus.controller.DeleteSubscriptionRequest + (*controller.GetSubscriptionRequest)(nil), // 18: linkall.vanus.controller.GetSubscriptionRequest + (*controller.DisableSubscriptionRequest)(nil), // 19: linkall.vanus.controller.DisableSubscriptionRequest + (*controller.ResumeSubscriptionRequest)(nil), // 20: linkall.vanus.controller.ResumeSubscriptionRequest + (*controller.ResetOffsetToTimestampRequest)(nil), // 21: linkall.vanus.controller.ResetOffsetToTimestampRequest + (*controller.ListEventbusResponse)(nil), // 22: linkall.vanus.controller.ListEventbusResponse + (*controller.ListSegmentResponse)(nil), // 23: linkall.vanus.controller.ListSegmentResponse + (*meta.Subscription)(nil), // 24: linkall.vanus.meta.Subscription + (*controller.ListSubscriptionResponse)(nil), // 25: linkall.vanus.controller.ListSubscriptionResponse + (*controller.ResetOffsetToTimestampResponse)(nil), // 26: linkall.vanus.controller.ResetOffsetToTimestampResponse } var file_proxy_proto_depIdxs = []int32{ 7, // 0: linkall.vanus.proxy.LookupOffsetResponse.offsets:type_name -> linkall.vanus.proxy.LookupOffsetResponse.OffsetsEntry @@ -698,27 +723,33 @@ var file_proxy_proto_depIdxs = []int32{ 17, // 11: linkall.vanus.proxy.ControllerProxy.DeleteSubscription:input_type -> linkall.vanus.controller.DeleteSubscriptionRequest 18, // 12: linkall.vanus.proxy.ControllerProxy.GetSubscription:input_type -> linkall.vanus.controller.GetSubscriptionRequest 12, // 13: linkall.vanus.proxy.ControllerProxy.ListSubscription:input_type -> google.protobuf.Empty - 12, // 14: linkall.vanus.proxy.ControllerProxy.ClusterInfo:input_type -> google.protobuf.Empty - 0, // 15: linkall.vanus.proxy.ControllerProxy.LookupOffset:input_type -> linkall.vanus.proxy.LookupOffsetRequest - 2, // 16: linkall.vanus.proxy.ControllerProxy.GetEvent:input_type -> linkall.vanus.proxy.GetEventRequest - 5, // 17: linkall.vanus.proxy.ControllerProxy.ValidateSubscription:input_type -> linkall.vanus.proxy.ValidateSubscriptionRequest - 11, // 18: linkall.vanus.proxy.ControllerProxy.CreateEventBus:output_type -> linkall.vanus.meta.EventBus - 12, // 19: linkall.vanus.proxy.ControllerProxy.DeleteEventBus:output_type -> google.protobuf.Empty - 11, // 20: linkall.vanus.proxy.ControllerProxy.GetEventBus:output_type -> linkall.vanus.meta.EventBus - 19, // 21: linkall.vanus.proxy.ControllerProxy.ListEventBus:output_type -> linkall.vanus.controller.ListEventbusResponse - 11, // 22: linkall.vanus.proxy.ControllerProxy.UpdateEventBus:output_type -> linkall.vanus.meta.EventBus - 20, // 23: linkall.vanus.proxy.ControllerProxy.ListSegment:output_type -> linkall.vanus.controller.ListSegmentResponse - 21, // 24: linkall.vanus.proxy.ControllerProxy.CreateSubscription:output_type -> linkall.vanus.meta.Subscription - 21, // 25: linkall.vanus.proxy.ControllerProxy.UpdateSubscription:output_type -> linkall.vanus.meta.Subscription - 12, // 26: linkall.vanus.proxy.ControllerProxy.DeleteSubscription:output_type -> google.protobuf.Empty - 21, // 27: linkall.vanus.proxy.ControllerProxy.GetSubscription:output_type -> linkall.vanus.meta.Subscription - 22, // 28: linkall.vanus.proxy.ControllerProxy.ListSubscription:output_type -> linkall.vanus.controller.ListSubscriptionResponse - 4, // 29: linkall.vanus.proxy.ControllerProxy.ClusterInfo:output_type -> linkall.vanus.proxy.ClusterInfoResponse - 1, // 30: linkall.vanus.proxy.ControllerProxy.LookupOffset:output_type -> linkall.vanus.proxy.LookupOffsetResponse - 3, // 31: linkall.vanus.proxy.ControllerProxy.GetEvent:output_type -> linkall.vanus.proxy.GetEventResponse - 6, // 32: linkall.vanus.proxy.ControllerProxy.ValidateSubscription:output_type -> linkall.vanus.proxy.ValidateSubscriptionResponse - 18, // [18:33] is the sub-list for method output_type - 3, // [3:18] is the sub-list for method input_type + 19, // 14: linkall.vanus.proxy.ControllerProxy.DisableSubscription:input_type -> linkall.vanus.controller.DisableSubscriptionRequest + 20, // 15: linkall.vanus.proxy.ControllerProxy.ResumeSubscription:input_type -> linkall.vanus.controller.ResumeSubscriptionRequest + 21, // 16: linkall.vanus.proxy.ControllerProxy.ResetOffsetToTimestamp:input_type -> linkall.vanus.controller.ResetOffsetToTimestampRequest + 12, // 17: linkall.vanus.proxy.ControllerProxy.ClusterInfo:input_type -> google.protobuf.Empty + 0, // 18: linkall.vanus.proxy.ControllerProxy.LookupOffset:input_type -> linkall.vanus.proxy.LookupOffsetRequest + 2, // 19: linkall.vanus.proxy.ControllerProxy.GetEvent:input_type -> linkall.vanus.proxy.GetEventRequest + 5, // 20: linkall.vanus.proxy.ControllerProxy.ValidateSubscription:input_type -> linkall.vanus.proxy.ValidateSubscriptionRequest + 11, // 21: linkall.vanus.proxy.ControllerProxy.CreateEventBus:output_type -> linkall.vanus.meta.EventBus + 12, // 22: linkall.vanus.proxy.ControllerProxy.DeleteEventBus:output_type -> google.protobuf.Empty + 11, // 23: linkall.vanus.proxy.ControllerProxy.GetEventBus:output_type -> linkall.vanus.meta.EventBus + 22, // 24: linkall.vanus.proxy.ControllerProxy.ListEventBus:output_type -> linkall.vanus.controller.ListEventbusResponse + 11, // 25: linkall.vanus.proxy.ControllerProxy.UpdateEventBus:output_type -> linkall.vanus.meta.EventBus + 23, // 26: linkall.vanus.proxy.ControllerProxy.ListSegment:output_type -> linkall.vanus.controller.ListSegmentResponse + 24, // 27: linkall.vanus.proxy.ControllerProxy.CreateSubscription:output_type -> linkall.vanus.meta.Subscription + 24, // 28: linkall.vanus.proxy.ControllerProxy.UpdateSubscription:output_type -> linkall.vanus.meta.Subscription + 12, // 29: linkall.vanus.proxy.ControllerProxy.DeleteSubscription:output_type -> google.protobuf.Empty + 24, // 30: linkall.vanus.proxy.ControllerProxy.GetSubscription:output_type -> linkall.vanus.meta.Subscription + 25, // 31: linkall.vanus.proxy.ControllerProxy.ListSubscription:output_type -> linkall.vanus.controller.ListSubscriptionResponse + 12, // 32: linkall.vanus.proxy.ControllerProxy.DisableSubscription:output_type -> google.protobuf.Empty + 12, // 33: linkall.vanus.proxy.ControllerProxy.ResumeSubscription:output_type -> google.protobuf.Empty + 26, // 34: linkall.vanus.proxy.ControllerProxy.ResetOffsetToTimestamp:output_type -> linkall.vanus.controller.ResetOffsetToTimestampResponse + 4, // 35: linkall.vanus.proxy.ControllerProxy.ClusterInfo:output_type -> linkall.vanus.proxy.ClusterInfoResponse + 1, // 36: linkall.vanus.proxy.ControllerProxy.LookupOffset:output_type -> linkall.vanus.proxy.LookupOffsetResponse + 3, // 37: linkall.vanus.proxy.ControllerProxy.GetEvent:output_type -> linkall.vanus.proxy.GetEventResponse + 6, // 38: linkall.vanus.proxy.ControllerProxy.ValidateSubscription:output_type -> linkall.vanus.proxy.ValidateSubscriptionResponse + 21, // [21:39] is the sub-list for method output_type + 3, // [3:21] is the sub-list for method input_type 3, // [3:3] is the sub-list for extension type_name 3, // [3:3] is the sub-list for extension extendee 0, // [0:3] is the sub-list for field type_name @@ -847,7 +878,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 { - // EventbusService + // Eventbus 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) @@ -860,6 +891,9 @@ type ControllerProxyClient interface { DeleteSubscription(ctx context.Context, in *controller.DeleteSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) GetSubscription(ctx context.Context, in *controller.GetSubscriptionRequest, opts ...grpc.CallOption) (*meta.Subscription, error) ListSubscription(ctx context.Context, in *emptypb.Empty, opts ...grpc.CallOption) (*controller.ListSubscriptionResponse, error) + DisableSubscription(ctx context.Context, in *controller.DisableSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) + ResumeSubscription(ctx context.Context, in *controller.ResumeSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) + ResetOffsetToTimestamp(ctx context.Context, in *controller.ResetOffsetToTimestampRequest, opts ...grpc.CallOption) (*controller.ResetOffsetToTimestampResponse, error) // custom ClusterInfo(ctx context.Context, in *emptypb.Empty, opts ...grpc.CallOption) (*ClusterInfoResponse, error) LookupOffset(ctx context.Context, in *LookupOffsetRequest, opts ...grpc.CallOption) (*LookupOffsetResponse, error) @@ -974,6 +1008,33 @@ func (c *controllerProxyClient) ListSubscription(ctx context.Context, in *emptyp return out, nil } +func (c *controllerProxyClient) DisableSubscription(ctx context.Context, in *controller.DisableSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) { + out := new(emptypb.Empty) + err := c.cc.Invoke(ctx, "/linkall.vanus.proxy.ControllerProxy/DisableSubscription", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *controllerProxyClient) ResumeSubscription(ctx context.Context, in *controller.ResumeSubscriptionRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) { + out := new(emptypb.Empty) + err := c.cc.Invoke(ctx, "/linkall.vanus.proxy.ControllerProxy/ResumeSubscription", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + +func (c *controllerProxyClient) ResetOffsetToTimestamp(ctx context.Context, in *controller.ResetOffsetToTimestampRequest, opts ...grpc.CallOption) (*controller.ResetOffsetToTimestampResponse, error) { + out := new(controller.ResetOffsetToTimestampResponse) + err := c.cc.Invoke(ctx, "/linkall.vanus.proxy.ControllerProxy/ResetOffsetToTimestamp", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *controllerProxyClient) ClusterInfo(ctx context.Context, in *emptypb.Empty, opts ...grpc.CallOption) (*ClusterInfoResponse, error) { out := new(ClusterInfoResponse) err := c.cc.Invoke(ctx, "/linkall.vanus.proxy.ControllerProxy/ClusterInfo", in, out, opts...) @@ -1012,7 +1073,7 @@ func (c *controllerProxyClient) ValidateSubscription(ctx context.Context, in *Va // ControllerProxyServer is the server API for ControllerProxy service. type ControllerProxyServer interface { - // EventbusService + // Eventbus CreateEventBus(context.Context, *controller.CreateEventBusRequest) (*meta.EventBus, error) DeleteEventBus(context.Context, *meta.EventBus) (*emptypb.Empty, error) GetEventBus(context.Context, *meta.EventBus) (*meta.EventBus, error) @@ -1025,6 +1086,9 @@ type ControllerProxyServer interface { DeleteSubscription(context.Context, *controller.DeleteSubscriptionRequest) (*emptypb.Empty, error) GetSubscription(context.Context, *controller.GetSubscriptionRequest) (*meta.Subscription, error) ListSubscription(context.Context, *emptypb.Empty) (*controller.ListSubscriptionResponse, error) + DisableSubscription(context.Context, *controller.DisableSubscriptionRequest) (*emptypb.Empty, error) + ResumeSubscription(context.Context, *controller.ResumeSubscriptionRequest) (*emptypb.Empty, error) + ResetOffsetToTimestamp(context.Context, *controller.ResetOffsetToTimestampRequest) (*controller.ResetOffsetToTimestampResponse, error) // custom ClusterInfo(context.Context, *emptypb.Empty) (*ClusterInfoResponse, error) LookupOffset(context.Context, *LookupOffsetRequest) (*LookupOffsetResponse, error) @@ -1069,6 +1133,15 @@ func (*UnimplementedControllerProxyServer) GetSubscription(context.Context, *con func (*UnimplementedControllerProxyServer) ListSubscription(context.Context, *emptypb.Empty) (*controller.ListSubscriptionResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ListSubscription not implemented") } +func (*UnimplementedControllerProxyServer) DisableSubscription(context.Context, *controller.DisableSubscriptionRequest) (*emptypb.Empty, error) { + return nil, status.Errorf(codes.Unimplemented, "method DisableSubscription not implemented") +} +func (*UnimplementedControllerProxyServer) ResumeSubscription(context.Context, *controller.ResumeSubscriptionRequest) (*emptypb.Empty, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResumeSubscription not implemented") +} +func (*UnimplementedControllerProxyServer) ResetOffsetToTimestamp(context.Context, *controller.ResetOffsetToTimestampRequest) (*controller.ResetOffsetToTimestampResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ResetOffsetToTimestamp not implemented") +} func (*UnimplementedControllerProxyServer) ClusterInfo(context.Context, *emptypb.Empty) (*ClusterInfoResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ClusterInfo not implemented") } @@ -1284,6 +1357,60 @@ func _ControllerProxy_ListSubscription_Handler(srv interface{}, ctx context.Cont return interceptor(ctx, in, info, handler) } +func _ControllerProxy_DisableSubscription_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(controller.DisableSubscriptionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ControllerProxyServer).DisableSubscription(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/linkall.vanus.proxy.ControllerProxy/DisableSubscription", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ControllerProxyServer).DisableSubscription(ctx, req.(*controller.DisableSubscriptionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ControllerProxy_ResumeSubscription_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(controller.ResumeSubscriptionRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ControllerProxyServer).ResumeSubscription(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/linkall.vanus.proxy.ControllerProxy/ResumeSubscription", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ControllerProxyServer).ResumeSubscription(ctx, req.(*controller.ResumeSubscriptionRequest)) + } + return interceptor(ctx, in, info, handler) +} + +func _ControllerProxy_ResetOffsetToTimestamp_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(controller.ResetOffsetToTimestampRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(ControllerProxyServer).ResetOffsetToTimestamp(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/linkall.vanus.proxy.ControllerProxy/ResetOffsetToTimestamp", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(ControllerProxyServer).ResetOffsetToTimestamp(ctx, req.(*controller.ResetOffsetToTimestampRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _ControllerProxy_ClusterInfo_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(emptypb.Empty) if err := dec(in); err != nil { @@ -1404,6 +1531,18 @@ var _ControllerProxy_serviceDesc = grpc.ServiceDesc{ MethodName: "ListSubscription", Handler: _ControllerProxy_ListSubscription_Handler, }, + { + MethodName: "DisableSubscription", + Handler: _ControllerProxy_DisableSubscription_Handler, + }, + { + MethodName: "ResumeSubscription", + Handler: _ControllerProxy_ResumeSubscription_Handler, + }, + { + MethodName: "ResetOffsetToTimestamp", + Handler: _ControllerProxy_ResetOffsetToTimestamp_Handler, + }, { MethodName: "ClusterInfo", Handler: _ControllerProxy_ClusterInfo_Handler, diff --git a/proto/pkg/trigger/mock_trigger.go b/proto/pkg/trigger/mock_trigger.go index f5c7543f9..fefa2c5ea 100644 --- a/proto/pkg/trigger/mock_trigger.go +++ b/proto/pkg/trigger/mock_trigger.go @@ -10,7 +10,6 @@ import ( gomock "github.com/golang/mock/gomock" grpc "google.golang.org/grpc" - emptypb "google.golang.org/protobuf/types/known/emptypb" ) // MockTriggerWorkerClient is a mock of TriggerWorkerClient interface. @@ -96,26 +95,6 @@ func (mr *MockTriggerWorkerClientMockRecorder) RemoveSubscription(ctx, in interf return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RemoveSubscription", reflect.TypeOf((*MockTriggerWorkerClient)(nil).RemoveSubscription), varargs...) } -// ResetOffsetToTimestamp mocks base method. -func (m *MockTriggerWorkerClient) ResetOffsetToTimestamp(ctx context.Context, in *ResetOffsetToTimestampRequest, 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, "ResetOffsetToTimestamp", varargs...) - ret0, _ := ret[0].(*emptypb.Empty) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ResetOffsetToTimestamp indicates an expected call of ResetOffsetToTimestamp. -func (mr *MockTriggerWorkerClientMockRecorder) ResetOffsetToTimestamp(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, "ResetOffsetToTimestamp", reflect.TypeOf((*MockTriggerWorkerClient)(nil).ResetOffsetToTimestamp), varargs...) -} - // ResumeSubscription mocks base method. func (m *MockTriggerWorkerClient) ResumeSubscription(ctx context.Context, in *ResumeSubscriptionRequest, opts ...grpc.CallOption) (*ResumeSubscriptionResponse, error) { m.ctrl.T.Helper() @@ -244,21 +223,6 @@ func (mr *MockTriggerWorkerServerMockRecorder) RemoveSubscription(arg0, arg1 int return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "RemoveSubscription", reflect.TypeOf((*MockTriggerWorkerServer)(nil).RemoveSubscription), arg0, arg1) } -// ResetOffsetToTimestamp mocks base method. -func (m *MockTriggerWorkerServer) ResetOffsetToTimestamp(arg0 context.Context, arg1 *ResetOffsetToTimestampRequest) (*emptypb.Empty, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "ResetOffsetToTimestamp", arg0, arg1) - ret0, _ := ret[0].(*emptypb.Empty) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// ResetOffsetToTimestamp indicates an expected call of ResetOffsetToTimestamp. -func (mr *MockTriggerWorkerServerMockRecorder) ResetOffsetToTimestamp(arg0, arg1 interface{}) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ResetOffsetToTimestamp", reflect.TypeOf((*MockTriggerWorkerServer)(nil).ResetOffsetToTimestamp), arg0, arg1) -} - // ResumeSubscription mocks base method. func (m *MockTriggerWorkerServer) ResumeSubscription(arg0 context.Context, arg1 *ResumeSubscriptionRequest) (*ResumeSubscriptionResponse, error) { m.ctrl.T.Helper() diff --git a/proto/pkg/trigger/trigger.pb.go b/proto/pkg/trigger/trigger.pb.go index 905444b5c..497f30050 100644 --- a/proto/pkg/trigger/trigger.pb.go +++ b/proto/pkg/trigger/trigger.pb.go @@ -18,10 +18,14 @@ // protoc v3.19.1 // source: trigger.proto +//go:generate mockgen -source=trigger.pb.go -destination=mock_trigger.go -package=trigger package trigger import ( context "context" + reflect "reflect" + sync "sync" + config "github.com/linkall-labs/vanus/proto/pkg/config" meta "github.com/linkall-labs/vanus/proto/pkg/meta" grpc "google.golang.org/grpc" @@ -29,9 +33,6 @@ import ( status "google.golang.org/grpc/status" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" - emptypb "google.golang.org/protobuf/types/known/emptypb" - reflect "reflect" - sync "sync" ) const ( @@ -614,197 +615,128 @@ func (*ResumeSubscriptionResponse) Descriptor() ([]byte, []int) { return file_trigger_proto_rawDescGZIP(), []int{11} } -type ResetOffsetToTimestampRequest struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - SubscriptionId uint64 `protobuf:"varint,1,opt,name=subscription_id,json=subscriptionId,proto3" json:"subscription_id,omitempty"` - Timestamp uint64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` -} - -func (x *ResetOffsetToTimestampRequest) Reset() { - *x = ResetOffsetToTimestampRequest{} - if protoimpl.UnsafeEnabled { - mi := &file_trigger_proto_msgTypes[12] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *ResetOffsetToTimestampRequest) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*ResetOffsetToTimestampRequest) ProtoMessage() {} - -func (x *ResetOffsetToTimestampRequest) ProtoReflect() protoreflect.Message { - mi := &file_trigger_proto_msgTypes[12] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use ResetOffsetToTimestampRequest.ProtoReflect.Descriptor instead. -func (*ResetOffsetToTimestampRequest) Descriptor() ([]byte, []int) { - return file_trigger_proto_rawDescGZIP(), []int{12} -} - -func (x *ResetOffsetToTimestampRequest) GetSubscriptionId() uint64 { - if x != nil { - return x.SubscriptionId - } - return 0 -} - -func (x *ResetOffsetToTimestampRequest) GetTimestamp() uint64 { - if x != nil { - return x.Timestamp - } - return 0 -} - var File_trigger_proto protoreflect.FileDescriptor var file_trigger_proto_rawDesc = []byte{ 0x0a, 0x0d, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x15, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, - 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x1a, 0x1b, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x65, 0x6d, 0x70, 0x74, 0x79, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x1a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x57, 0x0a, - 0x19, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, - 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, 0x06, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6c, 0x69, 0x6e, - 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x1c, 0x0a, 0x1a, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, - 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1a, 0x0a, 0x18, 0x53, 0x74, 0x6f, 0x70, 0x54, 0x72, 0x69, 0x67, - 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x22, 0x1b, 0x0a, 0x19, 0x53, 0x74, 0x6f, 0x70, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, - 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa5, 0x04, - 0x0a, 0x16, 0x41, 0x64, 0x64, 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, 0x3e, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x18, 0x02, 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, 0x03, 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, 0x04, 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, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6c, 0x69, + 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x1a, 0x0c, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x22, 0x57, 0x0a, 0x19, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, + 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3a, 0x0a, + 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, + 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0x1c, 0x0a, 0x1a, 0x53, 0x74, 0x61, + 0x72, 0x74, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x1a, 0x0a, 0x18, 0x53, 0x74, 0x6f, 0x70, 0x54, + 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x22, 0x1b, 0x0a, 0x19, 0x53, 0x74, 0x6f, 0x70, 0x54, 0x72, 0x69, 0x67, 0x67, + 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0xa5, 0x04, 0x0a, 0x16, 0x41, 0x64, 0x64, 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, 0x3e, 0x0a, 0x06, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 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, 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, 0x06, 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, 0x07, - 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, 0x08, 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, 0x09, 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, 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, 0x38, 0x0a, 0x07, 0x6f, - 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6c, + 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, 0x03, 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, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x6f, 0x66, - 0x66, 0x73, 0x65, 0x74, 0x73, 0x22, 0x19, 0x0a, 0x17, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x44, 0x0a, 0x19, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, - 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 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, 0x22, 0x1c, 0x0a, 0x1a, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, - 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x43, 0x0a, 0x18, 0x50, 0x61, 0x75, 0x73, 0x65, 0x53, 0x75, 0x62, + 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, 0x04, 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, 0x05, 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, 0x06, + 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, 0x07, 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, 0x08, 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, 0x09, 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, 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, 0x38, + 0x0a, 0x07, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1e, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, + 0x6d, 0x65, 0x74, 0x61, 0x2e, 0x4f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x07, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x73, 0x22, 0x19, 0x0a, 0x17, 0x41, 0x64, 0x64, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x44, 0x0a, 0x19, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 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, 0x22, 0x1b, 0x0a, 0x19, 0x50, 0x61, 0x75, - 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x44, 0x0a, 0x19, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, + 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x1c, 0x0a, 0x1a, 0x52, 0x65, 0x6d, + 0x6f, 0x76, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x43, 0x0a, 0x18, 0x50, 0x61, 0x75, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 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, 0x22, 0x1c, 0x0a, 0x1a, - 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 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, 0x32, 0xb0, 0x06, 0x0a, 0x0d, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, - 0x72, 0x6b, 0x65, 0x72, 0x12, 0x6c, 0x0a, 0x05, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x30, 0x2e, - 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, - 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x72, 0x69, 0x67, 0x67, - 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x31, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, - 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x72, 0x69, - 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x69, 0x0a, 0x04, 0x53, 0x74, 0x6f, 0x70, 0x12, 0x2f, 0x2e, 0x6c, 0x69, 0x6e, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x1b, 0x0a, 0x19, + 0x50, 0x61, 0x75, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x44, 0x0a, 0x19, 0x52, 0x65, 0x73, + 0x75, 0x6d, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 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, 0x22, + 0x1c, 0x0a, 0x1a, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, 0xc8, 0x05, + 0x0a, 0x0d, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x12, + 0x6c, 0x0a, 0x05, 0x53, 0x74, 0x61, 0x72, 0x74, 0x12, 0x30, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, + 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, + 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, + 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, - 0x65, 0x72, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, - 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6c, 0x69, + 0x65, 0x72, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, + 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x69, 0x0a, + 0x04, 0x53, 0x74, 0x6f, 0x70, 0x12, 0x2f, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, + 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x74, + 0x6f, 0x70, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, + 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x53, + 0x74, 0x6f, 0x70, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x70, 0x0a, 0x0f, 0x41, 0x64, 0x64, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2d, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, - 0x67, 0x65, 0x72, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x57, - 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x70, 0x0a, - 0x0f, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x2d, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, - 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, - 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x2e, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, - 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x79, 0x0a, 0x12, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 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, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x52, 0x65, + 0x67, 0x65, 0x72, 0x2e, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6c, 0x69, 0x6e, + 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, + 0x65, 0x72, 0x2e, 0x41, 0x64, 0x64, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x79, 0x0a, 0x12, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, - 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, - 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x76, 0x0a, 0x11, 0x50, 0x61, - 0x75, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x2f, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, - 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x50, 0x61, 0x75, 0x73, 0x65, 0x53, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x30, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, - 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x50, 0x61, 0x75, 0x73, 0x65, 0x53, 0x75, - 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x79, 0x0a, 0x12, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 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, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, - 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6c, 0x69, 0x6e, + 0x12, 0x30, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, + 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x53, + 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, + 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, + 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x76, 0x0a, 0x11, 0x50, 0x61, 0x75, 0x73, 0x65, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2f, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, - 0x65, 0x72, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x66, 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, 0x34, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, - 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, 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, 0x42, 0x31, 0x5a, 0x2f, 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, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x65, 0x72, 0x2e, 0x50, 0x61, 0x75, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6c, 0x69, + 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, + 0x67, 0x65, 0x72, 0x2e, 0x50, 0x61, 0x75, 0x73, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, + 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x79, 0x0a, + 0x12, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 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, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x73, 0x75, + 0x6d, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6c, 0x69, 0x6e, 0x6b, 0x61, 0x6c, 0x6c, 0x2e, + 0x76, 0x61, 0x6e, 0x75, 0x73, 0x2e, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x52, 0x65, + 0x73, 0x75, 0x6d, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x31, 0x5a, 0x2f, 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, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, } var ( @@ -819,56 +751,52 @@ func file_trigger_proto_rawDescGZIP() []byte { return file_trigger_proto_rawDescData } -var file_trigger_proto_msgTypes = make([]protoimpl.MessageInfo, 13) +var file_trigger_proto_msgTypes = make([]protoimpl.MessageInfo, 12) var file_trigger_proto_goTypes = []interface{}{ - (*StartTriggerWorkerRequest)(nil), // 0: linkall.vanus.trigger.StartTriggerWorkerRequest - (*StartTriggerWorkerResponse)(nil), // 1: linkall.vanus.trigger.StartTriggerWorkerResponse - (*StopTriggerWorkerRequest)(nil), // 2: linkall.vanus.trigger.StopTriggerWorkerRequest - (*StopTriggerWorkerResponse)(nil), // 3: linkall.vanus.trigger.StopTriggerWorkerResponse - (*AddSubscriptionRequest)(nil), // 4: linkall.vanus.trigger.AddSubscriptionRequest - (*AddSubscriptionResponse)(nil), // 5: linkall.vanus.trigger.AddSubscriptionResponse - (*RemoveSubscriptionRequest)(nil), // 6: linkall.vanus.trigger.RemoveSubscriptionRequest - (*RemoveSubscriptionResponse)(nil), // 7: linkall.vanus.trigger.RemoveSubscriptionResponse - (*PauseSubscriptionRequest)(nil), // 8: linkall.vanus.trigger.PauseSubscriptionRequest - (*PauseSubscriptionResponse)(nil), // 9: linkall.vanus.trigger.PauseSubscriptionResponse - (*ResumeSubscriptionRequest)(nil), // 10: linkall.vanus.trigger.ResumeSubscriptionRequest - (*ResumeSubscriptionResponse)(nil), // 11: linkall.vanus.trigger.ResumeSubscriptionResponse - (*ResetOffsetToTimestampRequest)(nil), // 12: linkall.vanus.trigger.ResetOffsetToTimestampRequest - (*config.ServerConfig)(nil), // 13: linkall.vanus.config.ServerConfig - (*meta.SubscriptionConfig)(nil), // 14: linkall.vanus.meta.SubscriptionConfig - (*meta.Filter)(nil), // 15: linkall.vanus.meta.Filter - (*meta.SinkCredential)(nil), // 16: linkall.vanus.meta.SinkCredential - (meta.Protocol)(0), // 17: linkall.vanus.meta.Protocol - (*meta.ProtocolSetting)(nil), // 18: linkall.vanus.meta.ProtocolSetting - (*meta.Transformer)(nil), // 19: linkall.vanus.meta.Transformer - (*meta.OffsetInfo)(nil), // 20: linkall.vanus.meta.OffsetInfo - (*emptypb.Empty)(nil), // 21: google.protobuf.Empty + (*StartTriggerWorkerRequest)(nil), // 0: linkall.vanus.trigger.StartTriggerWorkerRequest + (*StartTriggerWorkerResponse)(nil), // 1: linkall.vanus.trigger.StartTriggerWorkerResponse + (*StopTriggerWorkerRequest)(nil), // 2: linkall.vanus.trigger.StopTriggerWorkerRequest + (*StopTriggerWorkerResponse)(nil), // 3: linkall.vanus.trigger.StopTriggerWorkerResponse + (*AddSubscriptionRequest)(nil), // 4: linkall.vanus.trigger.AddSubscriptionRequest + (*AddSubscriptionResponse)(nil), // 5: linkall.vanus.trigger.AddSubscriptionResponse + (*RemoveSubscriptionRequest)(nil), // 6: linkall.vanus.trigger.RemoveSubscriptionRequest + (*RemoveSubscriptionResponse)(nil), // 7: linkall.vanus.trigger.RemoveSubscriptionResponse + (*PauseSubscriptionRequest)(nil), // 8: linkall.vanus.trigger.PauseSubscriptionRequest + (*PauseSubscriptionResponse)(nil), // 9: linkall.vanus.trigger.PauseSubscriptionResponse + (*ResumeSubscriptionRequest)(nil), // 10: linkall.vanus.trigger.ResumeSubscriptionRequest + (*ResumeSubscriptionResponse)(nil), // 11: linkall.vanus.trigger.ResumeSubscriptionResponse + (*config.ServerConfig)(nil), // 12: linkall.vanus.config.ServerConfig + (*meta.SubscriptionConfig)(nil), // 13: linkall.vanus.meta.SubscriptionConfig + (*meta.Filter)(nil), // 14: linkall.vanus.meta.Filter + (*meta.SinkCredential)(nil), // 15: linkall.vanus.meta.SinkCredential + (meta.Protocol)(0), // 16: linkall.vanus.meta.Protocol + (*meta.ProtocolSetting)(nil), // 17: linkall.vanus.meta.ProtocolSetting + (*meta.Transformer)(nil), // 18: linkall.vanus.meta.Transformer + (*meta.OffsetInfo)(nil), // 19: linkall.vanus.meta.OffsetInfo } var file_trigger_proto_depIdxs = []int32{ - 13, // 0: linkall.vanus.trigger.StartTriggerWorkerRequest.config:type_name -> linkall.vanus.config.ServerConfig - 14, // 1: linkall.vanus.trigger.AddSubscriptionRequest.config:type_name -> linkall.vanus.meta.SubscriptionConfig - 15, // 2: linkall.vanus.trigger.AddSubscriptionRequest.filters:type_name -> linkall.vanus.meta.Filter - 16, // 3: linkall.vanus.trigger.AddSubscriptionRequest.sink_credential:type_name -> linkall.vanus.meta.SinkCredential - 17, // 4: linkall.vanus.trigger.AddSubscriptionRequest.protocol:type_name -> linkall.vanus.meta.Protocol - 18, // 5: linkall.vanus.trigger.AddSubscriptionRequest.protocol_settings:type_name -> linkall.vanus.meta.ProtocolSetting - 19, // 6: linkall.vanus.trigger.AddSubscriptionRequest.transformer:type_name -> linkall.vanus.meta.Transformer - 20, // 7: linkall.vanus.trigger.AddSubscriptionRequest.offsets:type_name -> linkall.vanus.meta.OffsetInfo + 12, // 0: linkall.vanus.trigger.StartTriggerWorkerRequest.config:type_name -> linkall.vanus.config.ServerConfig + 13, // 1: linkall.vanus.trigger.AddSubscriptionRequest.config:type_name -> linkall.vanus.meta.SubscriptionConfig + 14, // 2: linkall.vanus.trigger.AddSubscriptionRequest.filters:type_name -> linkall.vanus.meta.Filter + 15, // 3: linkall.vanus.trigger.AddSubscriptionRequest.sink_credential:type_name -> linkall.vanus.meta.SinkCredential + 16, // 4: linkall.vanus.trigger.AddSubscriptionRequest.protocol:type_name -> linkall.vanus.meta.Protocol + 17, // 5: linkall.vanus.trigger.AddSubscriptionRequest.protocol_settings:type_name -> linkall.vanus.meta.ProtocolSetting + 18, // 6: linkall.vanus.trigger.AddSubscriptionRequest.transformer:type_name -> linkall.vanus.meta.Transformer + 19, // 7: linkall.vanus.trigger.AddSubscriptionRequest.offsets:type_name -> linkall.vanus.meta.OffsetInfo 0, // 8: linkall.vanus.trigger.TriggerWorker.Start:input_type -> linkall.vanus.trigger.StartTriggerWorkerRequest 2, // 9: linkall.vanus.trigger.TriggerWorker.Stop:input_type -> linkall.vanus.trigger.StopTriggerWorkerRequest 4, // 10: linkall.vanus.trigger.TriggerWorker.AddSubscription:input_type -> linkall.vanus.trigger.AddSubscriptionRequest 6, // 11: linkall.vanus.trigger.TriggerWorker.RemoveSubscription:input_type -> linkall.vanus.trigger.RemoveSubscriptionRequest 8, // 12: linkall.vanus.trigger.TriggerWorker.PauseSubscription:input_type -> linkall.vanus.trigger.PauseSubscriptionRequest 10, // 13: linkall.vanus.trigger.TriggerWorker.ResumeSubscription:input_type -> linkall.vanus.trigger.ResumeSubscriptionRequest - 12, // 14: linkall.vanus.trigger.TriggerWorker.ResetOffsetToTimestamp:input_type -> linkall.vanus.trigger.ResetOffsetToTimestampRequest - 1, // 15: linkall.vanus.trigger.TriggerWorker.Start:output_type -> linkall.vanus.trigger.StartTriggerWorkerResponse - 3, // 16: linkall.vanus.trigger.TriggerWorker.Stop:output_type -> linkall.vanus.trigger.StopTriggerWorkerResponse - 5, // 17: linkall.vanus.trigger.TriggerWorker.AddSubscription:output_type -> linkall.vanus.trigger.AddSubscriptionResponse - 7, // 18: linkall.vanus.trigger.TriggerWorker.RemoveSubscription:output_type -> linkall.vanus.trigger.RemoveSubscriptionResponse - 9, // 19: linkall.vanus.trigger.TriggerWorker.PauseSubscription:output_type -> linkall.vanus.trigger.PauseSubscriptionResponse - 11, // 20: linkall.vanus.trigger.TriggerWorker.ResumeSubscription:output_type -> linkall.vanus.trigger.ResumeSubscriptionResponse - 21, // 21: linkall.vanus.trigger.TriggerWorker.ResetOffsetToTimestamp:output_type -> google.protobuf.Empty - 15, // [15:22] is the sub-list for method output_type - 8, // [8:15] is the sub-list for method input_type + 1, // 14: linkall.vanus.trigger.TriggerWorker.Start:output_type -> linkall.vanus.trigger.StartTriggerWorkerResponse + 3, // 15: linkall.vanus.trigger.TriggerWorker.Stop:output_type -> linkall.vanus.trigger.StopTriggerWorkerResponse + 5, // 16: linkall.vanus.trigger.TriggerWorker.AddSubscription:output_type -> linkall.vanus.trigger.AddSubscriptionResponse + 7, // 17: linkall.vanus.trigger.TriggerWorker.RemoveSubscription:output_type -> linkall.vanus.trigger.RemoveSubscriptionResponse + 9, // 18: linkall.vanus.trigger.TriggerWorker.PauseSubscription:output_type -> linkall.vanus.trigger.PauseSubscriptionResponse + 11, // 19: linkall.vanus.trigger.TriggerWorker.ResumeSubscription:output_type -> linkall.vanus.trigger.ResumeSubscriptionResponse + 14, // [14:20] is the sub-list for method output_type + 8, // [8:14] is the sub-list for method input_type 8, // [8:8] is the sub-list for extension type_name 8, // [8:8] is the sub-list for extension extendee 0, // [0:8] is the sub-list for field type_name @@ -1024,18 +952,6 @@ func file_trigger_proto_init() { return nil } } - file_trigger_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResetOffsetToTimestampRequest); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } } type x struct{} out := protoimpl.TypeBuilder{ @@ -1043,7 +959,7 @@ func file_trigger_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_trigger_proto_rawDesc, NumEnums: 0, - NumMessages: 13, + NumMessages: 12, NumExtensions: 0, NumServices: 1, }, @@ -1075,7 +991,6 @@ type TriggerWorkerClient interface { RemoveSubscription(ctx context.Context, in *RemoveSubscriptionRequest, opts ...grpc.CallOption) (*RemoveSubscriptionResponse, error) PauseSubscription(ctx context.Context, in *PauseSubscriptionRequest, opts ...grpc.CallOption) (*PauseSubscriptionResponse, error) ResumeSubscription(ctx context.Context, in *ResumeSubscriptionRequest, opts ...grpc.CallOption) (*ResumeSubscriptionResponse, error) - ResetOffsetToTimestamp(ctx context.Context, in *ResetOffsetToTimestampRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) } type triggerWorkerClient struct { @@ -1140,15 +1055,6 @@ func (c *triggerWorkerClient) ResumeSubscription(ctx context.Context, in *Resume return out, nil } -func (c *triggerWorkerClient) ResetOffsetToTimestamp(ctx context.Context, in *ResetOffsetToTimestampRequest, opts ...grpc.CallOption) (*emptypb.Empty, error) { - out := new(emptypb.Empty) - err := c.cc.Invoke(ctx, "/linkall.vanus.trigger.TriggerWorker/ResetOffsetToTimestamp", in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - // TriggerWorkerServer is the server API for TriggerWorker service. type TriggerWorkerServer interface { Start(context.Context, *StartTriggerWorkerRequest) (*StartTriggerWorkerResponse, error) @@ -1157,7 +1063,6 @@ type TriggerWorkerServer interface { RemoveSubscription(context.Context, *RemoveSubscriptionRequest) (*RemoveSubscriptionResponse, error) PauseSubscription(context.Context, *PauseSubscriptionRequest) (*PauseSubscriptionResponse, error) ResumeSubscription(context.Context, *ResumeSubscriptionRequest) (*ResumeSubscriptionResponse, error) - ResetOffsetToTimestamp(context.Context, *ResetOffsetToTimestampRequest) (*emptypb.Empty, error) } // UnimplementedTriggerWorkerServer can be embedded to have forward compatible implementations. @@ -1182,9 +1087,6 @@ func (*UnimplementedTriggerWorkerServer) PauseSubscription(context.Context, *Pau func (*UnimplementedTriggerWorkerServer) ResumeSubscription(context.Context, *ResumeSubscriptionRequest) (*ResumeSubscriptionResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ResumeSubscription not implemented") } -func (*UnimplementedTriggerWorkerServer) ResetOffsetToTimestamp(context.Context, *ResetOffsetToTimestampRequest) (*emptypb.Empty, error) { - return nil, status.Errorf(codes.Unimplemented, "method ResetOffsetToTimestamp not implemented") -} func RegisterTriggerWorkerServer(s *grpc.Server, srv TriggerWorkerServer) { s.RegisterService(&_TriggerWorker_serviceDesc, srv) @@ -1298,24 +1200,6 @@ func _TriggerWorker_ResumeSubscription_Handler(srv interface{}, ctx context.Cont return interceptor(ctx, in, info, handler) } -func _TriggerWorker_ResetOffsetToTimestamp_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(ResetOffsetToTimestampRequest) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(TriggerWorkerServer).ResetOffsetToTimestamp(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: "/linkall.vanus.trigger.TriggerWorker/ResetOffsetToTimestamp", - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(TriggerWorkerServer).ResetOffsetToTimestamp(ctx, req.(*ResetOffsetToTimestampRequest)) - } - return interceptor(ctx, in, info, handler) -} - var _TriggerWorker_serviceDesc = grpc.ServiceDesc{ ServiceName: "linkall.vanus.trigger.TriggerWorker", HandlerType: (*TriggerWorkerServer)(nil), @@ -1344,10 +1228,6 @@ var _TriggerWorker_serviceDesc = grpc.ServiceDesc{ MethodName: "ResumeSubscription", Handler: _TriggerWorker_ResumeSubscription_Handler, }, - { - MethodName: "ResetOffsetToTimestamp", - Handler: _TriggerWorker_ResetOffsetToTimestamp_Handler, - }, }, Streams: []grpc.StreamDesc{}, Metadata: "trigger.proto", diff --git a/proto/proto/controller.proto b/proto/proto/controller.proto index fec9a5e80..a9f811343 100644 --- a/proto/proto/controller.proto +++ b/proto/proto/controller.proto @@ -69,6 +69,10 @@ service TriggerController { returns (linkall.vanus.meta.Subscription) {} rpc DeleteSubscription(DeleteSubscriptionRequest) returns (google.protobuf.Empty) {} + rpc DisableSubscription(DisableSubscriptionRequest) + returns (google.protobuf.Empty); + rpc ResumeSubscription(ResumeSubscriptionRequest) + returns (google.protobuf.Empty); rpc GetSubscription(GetSubscriptionRequest) returns (linkall.vanus.meta.Subscription) {} rpc ListSubscription(google.protobuf.Empty) @@ -80,7 +84,7 @@ service TriggerController { rpc UnregisterTriggerWorker(UnregisterTriggerWorkerRequest) returns (UnregisterTriggerWorkerResponse); rpc ResetOffsetToTimestamp(ResetOffsetToTimestampRequest) - returns (google.protobuf.Empty); + returns (ResetOffsetToTimestampResponse); rpc CommitOffset(CommitOffsetRequest) returns (CommitOffsetResponse); } @@ -94,7 +98,7 @@ service SnowflakeController { message PingResponse { string leader_addr = 1; string gateway_addr = 2; - bool is_eventbus_ready=3; + bool is_eventbus_ready = 3; } message CreateEventBusRequest { @@ -182,6 +186,14 @@ message DeleteSubscriptionRequest { uint64 id = 1; } +message DisableSubscriptionRequest { + uint64 id = 1; +} + +message ResumeSubscriptionRequest { + uint64 id = 1; +} + message ListSubscriptionResponse { repeated linkall.vanus.meta.Subscription subscription = 1; } @@ -212,6 +224,11 @@ message ResetOffsetToTimestampRequest { uint64 timestamp = 2; } +message ResetOffsetToTimestampResponse { + repeated meta.OffsetInfo offsets = 1; +} + + message CommitOffsetRequest { repeated meta.SubscriptionInfo subscription_info = 1; bool force_commit = 2; diff --git a/proto/proto/proxy.proto b/proto/proto/proxy.proto index 623091f2e..165514b2e 100644 --- a/proto/proto/proxy.proto +++ b/proto/proto/proxy.proto @@ -45,6 +45,12 @@ service ControllerProxy { returns (meta.Subscription); rpc ListSubscription(google.protobuf.Empty) returns (controller.ListSubscriptionResponse); + rpc DisableSubscription(controller.DisableSubscriptionRequest) + returns (google.protobuf.Empty); + rpc ResumeSubscription(controller.ResumeSubscriptionRequest) + returns (google.protobuf.Empty); + rpc ResetOffsetToTimestamp(controller.ResetOffsetToTimestampRequest) + returns (controller.ResetOffsetToTimestampResponse); // custom rpc ClusterInfo(google.protobuf.Empty) returns (ClusterInfoResponse); diff --git a/proto/proto/trigger.proto b/proto/proto/trigger.proto index 78ae061e2..76aa5167b 100644 --- a/proto/proto/trigger.proto +++ b/proto/proto/trigger.proto @@ -16,7 +16,6 @@ syntax = "proto3"; package linkall.vanus.trigger; -import "google/protobuf/empty.proto"; import "config.proto"; import "meta.proto"; @@ -32,8 +31,6 @@ service TriggerWorker { returns (PauseSubscriptionResponse); rpc ResumeSubscription(ResumeSubscriptionRequest) returns (ResumeSubscriptionResponse); - rpc ResetOffsetToTimestamp(ResetOffsetToTimestampRequest) - returns (google.protobuf.Empty); } message StartTriggerWorkerRequest { @@ -78,8 +75,3 @@ message ResumeSubscriptionRequest { } message ResumeSubscriptionResponse {} - -message ResetOffsetToTimestampRequest { - uint64 subscription_id = 1; - uint64 timestamp = 2; -} diff --git a/test/e2e/sink/main.go b/test/e2e/sink/main.go index 3c4baa405..1459ea6da 100644 --- a/test/e2e/sink/main.go +++ b/test/e2e/sink/main.go @@ -16,14 +16,18 @@ package main import ( "context" + "fmt" + "net" + "sync/atomic" + ce "github.com/cloudevents/sdk-go/v2" "github.com/cloudevents/sdk-go/v2/client" cehttp "github.com/cloudevents/sdk-go/v2/protocol/http" "github.com/linkall-labs/vanus/observability/log" - "net" ) func main() { + var total int64 ls, err := net.Listen("tcp4", ":18080") if err != nil { panic(err) @@ -34,8 +38,7 @@ func main() { } log.Info(context.Background(), "start success", nil) c.StartReceiver(context.Background(), func(e ce.Event) { - log.Info(context.Background(), "receive event", map[string]interface{}{ - "event": e, - }) + fmt.Println(fmt.Sprintf("---total: %d", atomic.AddInt64(&total, 1))) + fmt.Println(e) }) } diff --git a/vsctl/command/flag.go b/vsctl/command/flag.go index c0acee2cd..eaef4c524 100644 --- a/vsctl/command/flag.go +++ b/vsctl/command/flag.go @@ -52,6 +52,7 @@ var ( sinkCredential string deliveryTimeout uint32 maxRetryAttempts int32 + offsetTimestamp uint64 showSegment bool showBlock bool diff --git a/vsctl/command/subscription.go b/vsctl/command/subscription.go index f964a2d97..9ec8a876b 100644 --- a/vsctl/command/subscription.go +++ b/vsctl/command/subscription.go @@ -50,8 +50,11 @@ func NewSubscriptionCommand() *cobra.Command { } cmd.AddCommand(createSubscriptionCommand()) cmd.AddCommand(deleteSubscriptionCommand()) + cmd.AddCommand(disableSubscriptionCommand()) + cmd.AddCommand(resumeSubscriptionCommand()) cmd.AddCommand(getSubscriptionCommand()) cmd.AddCommand(listSubscriptionCommand()) + cmd.AddCommand(resetOffsetCommand()) return cmd } @@ -261,6 +264,122 @@ func deleteSubscriptionCommand() *cobra.Command { return cmd } +func resumeSubscriptionCommand() *cobra.Command { + cmd := &cobra.Command{ + Use: "resume", + Short: "resume a subscription", + Run: func(cmd *cobra.Command, args []string) { + id, err := vanus.NewIDFromString(subscriptionIDStr) + if err != nil { + cmdFailedWithHelpNotice(cmd, fmt.Sprintf("invalid subscription id: %s\n", err.Error())) + } + + _, err = client.ResumeSubscription(context.Background(), &ctrlpb.ResumeSubscriptionRequest{ + Id: id.Uint64(), + }) + if err != nil { + cmdFailedf(cmd, "resume subscription failed: %s", err) + } + + if IsFormatJSON(cmd) { + data, _ := json.Marshal(map[string]interface{}{"subscription_id": subscriptionIDStr}) + color.Green(string(data)) + } else { + t := table.NewWriter() + t.AppendHeader(table.Row{"subscription_id"}) + t.AppendRow(table.Row{subscriptionIDStr}) + t.SetColumnConfigs([]table.ColumnConfig{ + {Number: 1, Align: text.AlignCenter, AlignHeader: text.AlignCenter}, + }) + t.SetOutputMirror(os.Stdout) + t.Render() + } + color.Green("resume subscription: %d success\n", subscriptionIDStr) + }, + } + cmd.Flags().StringVar(&subscriptionIDStr, "id", "", "subscription id to resume") + return cmd +} + +func disableSubscriptionCommand() *cobra.Command { + cmd := &cobra.Command{ + Use: "disable", + Short: "disable a subscription", + Run: func(cmd *cobra.Command, args []string) { + id, err := vanus.NewIDFromString(subscriptionIDStr) + if err != nil { + cmdFailedWithHelpNotice(cmd, fmt.Sprintf("invalid subscription id: %s\n", err.Error())) + } + + _, err = client.DisableSubscription(context.Background(), &ctrlpb.DisableSubscriptionRequest{ + Id: id.Uint64(), + }) + if err != nil { + cmdFailedf(cmd, "disable subscription failed: %s", err) + } + + if IsFormatJSON(cmd) { + data, _ := json.Marshal(map[string]interface{}{"subscription_id": subscriptionIDStr}) + color.Green(string(data)) + } else { + t := table.NewWriter() + t.AppendHeader(table.Row{"subscription_id"}) + t.AppendRow(table.Row{subscriptionIDStr}) + t.SetColumnConfigs([]table.ColumnConfig{ + {Number: 1, Align: text.AlignCenter, AlignHeader: text.AlignCenter}, + }) + t.SetOutputMirror(os.Stdout) + t.Render() + } + color.Green("disable subscription: %d success\n", subscriptionIDStr) + }, + } + cmd.Flags().StringVar(&subscriptionIDStr, "id", "", "subscription id to disable") + return cmd +} + +func resetOffsetCommand() *cobra.Command { + cmd := &cobra.Command{ + Use: "reset-offset", + Short: "reset offset a subscription", + Run: func(cmd *cobra.Command, args []string) { + id, err := vanus.NewIDFromString(subscriptionIDStr) + if err != nil { + cmdFailedWithHelpNotice(cmd, fmt.Sprintf("invalid subscription id: %s\n", err.Error())) + } + if offsetTimestamp == 0 { + cmdFailedf(cmd, "reset offset timestamp must gt 0") + } + res, err := client.ResetOffsetToTimestamp(context.Background(), &ctrlpb.ResetOffsetToTimestampRequest{ + SubscriptionId: id.Uint64(), + Timestamp: offsetTimestamp, + }) + if err != nil { + cmdFailedf(cmd, "reset offset subscription failed: %s", err) + } + data, _ := json.MarshalIndent(res.Offsets, "", " ") + if IsFormatJSON(cmd) { + color.Green(string(data)) + } else { + t := table.NewWriter() + t.AppendHeader(table.Row{"subscription_id", "filters"}) + t.AppendSeparator() + t.AppendRow(table.Row{subscriptionIDStr, string(data)}) + t.SetColumnConfigs([]table.ColumnConfig{ + {Number: 1, VAlign: text.VAlignMiddle, AlignHeader: text.AlignCenter}, + {Number: 2, VAlign: text.VAlignMiddle, AlignHeader: text.AlignCenter}, + }) + t.SetOutputMirror(os.Stdout) + t.Render() + } + color.Green("reset offset by subscription: %s success\n", subscriptionIDStr) + }, + } + cmd.Flags().StringVar(&subscriptionIDStr, "id", "", "subscription id to disable") + cmd.Flags().Uint64Var(&offsetTimestamp, "timestamp", 0, "reset offset to UTC second") + return cmd +} + func getSubscriptionCommand() *cobra.Command { cmd := &cobra.Command{ Use: "info", From 1ca00750c8f69cce7032c0f76959f72b3de1f9b9 Mon Sep 17 00:00:00 2001 From: wenfeng Date: Thu, 5 Jan 2023 11:09:44 +0800 Subject: [PATCH 2/9] fix: roundrobin policy (#383) * fix: roundrobin policy Signed-off-by: wenfeng * fix Signed-off-by: wenfeng Signed-off-by: wenfeng --- client/pkg/policy/policy.go | 41 ++++++++++++++++++++++++++++++++----- 1 file changed, 36 insertions(+), 5 deletions(-) diff --git a/client/pkg/policy/policy.go b/client/pkg/policy/policy.go index 0bf763ff1..0f2b5b2db 100644 --- a/client/pkg/policy/policy.go +++ b/client/pkg/policy/policy.go @@ -16,6 +16,8 @@ package policy import ( "context" + "sort" + "sync" "sync/atomic" "github.com/linkall-labs/vanus/client/pkg/api" @@ -30,8 +32,10 @@ func NewRoundRobinWritePolicy(eb api.Eventbus) api.WritePolicy { } type roundRobinWritePolicy struct { - bus api.Eventbus - idx uint64 + bus api.Eventbus + idx uint64 + cached []api.Eventlog + mutex sync.Mutex } func (w *roundRobinWritePolicy) Type() api.PolicyType { @@ -47,6 +51,18 @@ func (w *roundRobinWritePolicy) NextLog(ctx context.Context) (api.Eventlog, erro if len(logs) == 0 { continue } + + if len(logs) == len(w.cached) { + logs = w.cached + } else { + w.mutex.Lock() + sort.Slice(logs, func(i, j int) bool { + return logs[i].ID() > logs[j].ID() + }) + w.cached = logs + w.mutex.Unlock() + } + l := len(logs) i := atomic.AddUint64(&w.idx, 1) % uint64(l) return logs[i], nil @@ -82,28 +98,43 @@ type roundRobinReadPolicy struct { bus api.Eventbus idx uint64 offset int64 + cached []api.Eventlog + mutex sync.Mutex } -func (r roundRobinReadPolicy) Type() api.PolicyType { +func (r *roundRobinReadPolicy) Type() api.PolicyType { return api.RoundRobin } -func (r roundRobinReadPolicy) NextLog(ctx context.Context) (api.Eventlog, error) { +func (r *roundRobinReadPolicy) NextLog(ctx context.Context) (api.Eventlog, error) { for { logs, err := r.bus.ListLog(ctx) if err != nil { return nil, err } + if len(logs) == 0 { continue } + + if len(logs) == len(r.cached) { + logs = r.cached + } else { + r.mutex.Lock() + sort.Slice(logs, func(i, j int) bool { + return logs[i].ID() > logs[j].ID() + }) + r.cached = logs + r.mutex.Unlock() + } + l := len(logs) i := atomic.AddUint64(&r.idx, 1) % uint64(l) return logs[i], nil } } -func (r roundRobinReadPolicy) Offset() int64 { +func (r *roundRobinReadPolicy) Offset() int64 { return atomic.LoadInt64(&r.offset) } From 6c6ddc4d3086f44e7d3c3991f4c00887aaf28f5e Mon Sep 17 00:00:00 2001 From: delu Date: Thu, 5 Jan 2023 13:50:18 +0800 Subject: [PATCH 3/9] feat: support batch when use grpc (#384) * feat: support trigger batch Signed-off-by: xdlbdy * feat: support batch Signed-off-by: xdlbdy * feat: support batch Signed-off-by: xdlbdy * feat: support batch Signed-off-by: xdlbdy * feat: support batch Signed-off-by: xdlbdy Signed-off-by: xdlbdy --- client/internal/vanus/store/block_store.go | 4 +- .../{internal/vanus => pkg}/codec/protobuf.go | 0 go.mod | 1 + go.sum | 2 + .../trigger/validation/subscripton.go | 3 + internal/convert/convert.go | 4 + internal/primitive/subscription.go | 1 + internal/trigger/client/gcloud_functions.go | 3 +- internal/trigger/client/grpc.go | 84 ++++++++ internal/trigger/client/http.go | 5 +- internal/trigger/client/interface.go | 4 +- internal/trigger/client/lambda.go | 3 +- internal/trigger/client/mock_interface.go | 26 ++- internal/trigger/trigger/config.go | 7 + internal/trigger/trigger/trigger.go | 200 ++++++++++++------ internal/trigger/trigger/trigger_test.go | 4 +- internal/trigger/trigger/util.go | 13 +- proto/pkg/meta/meta.pb.go | 16 +- proto/proto/meta.proto | 1 + vsctl/command/subscription.go | 6 +- 20 files changed, 299 insertions(+), 88 deletions(-) rename client/{internal/vanus => pkg}/codec/protobuf.go (100%) create mode 100644 internal/trigger/client/grpc.go diff --git a/client/internal/vanus/store/block_store.go b/client/internal/vanus/store/block_store.go index 62e8bc011..bb36fbeeb 100644 --- a/client/internal/vanus/store/block_store.go +++ b/client/internal/vanus/store/block_store.go @@ -19,6 +19,8 @@ import ( "context" "time" + "github.com/linkall-labs/vanus/client/pkg/codec" + "github.com/linkall-labs/vanus/observability/tracing" "go.opentelemetry.io/otel/trace" @@ -30,8 +32,6 @@ import ( cepb "github.com/linkall-labs/vanus/proto/pkg/cloudevents" segpb "github.com/linkall-labs/vanus/proto/pkg/segment" - // this project - "github.com/linkall-labs/vanus/client/internal/vanus/codec" "github.com/linkall-labs/vanus/client/internal/vanus/net/rpc" "github.com/linkall-labs/vanus/client/internal/vanus/net/rpc/bare" "github.com/linkall-labs/vanus/client/pkg/primitive" diff --git a/client/internal/vanus/codec/protobuf.go b/client/pkg/codec/protobuf.go similarity index 100% rename from client/internal/vanus/codec/protobuf.go rename to client/pkg/codec/protobuf.go diff --git a/go.mod b/go.mod index 010109cf0..dfbe6104f 100644 --- a/go.mod +++ b/go.mod @@ -30,6 +30,7 @@ require ( github.com/linkall-labs/vanus/raft v0.5.1 github.com/ncw/directio v1.0.5 github.com/ohler55/ojg v1.14.5 + github.com/panjf2000/ants/v2 v2.7.1 github.com/pkg/errors v0.9.1 github.com/prashantv/gostub v1.1.0 github.com/prometheus/client_golang v1.14.0 diff --git a/go.sum b/go.sum index 4810c5efe..e4be51190 100644 --- a/go.sum +++ b/go.sum @@ -322,6 +322,8 @@ github.com/ohler55/ojg v1.14.5/go.mod h1:7Ghirupn8NC8hSSDpI0gcjorPxj+vSVIONDWfli github.com/onsi/ginkgo v1.16.5 h1:8xi0RTUf59SOSfEtZMvwTvXYMzG4gV23XVHOZiXNtnE= github.com/onsi/gomega v1.18.1 h1:M1GfJqGRrBrrGGsbxzV5dqM2U2ApXefZCQpkukxYRLE= github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= +github.com/panjf2000/ants/v2 v2.7.1 h1:qBy5lfSdbxvrR0yUnZfaEDjf0FlCw4ufsbcsxmE7r+M= +github.com/panjf2000/ants/v2 v2.7.1/go.mod h1:KIBmYG9QQX5U2qzFP/yQJaq/nSb6rahS9iEHkrCMgM8= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= diff --git a/internal/controller/trigger/validation/subscripton.go b/internal/controller/trigger/validation/subscripton.go index 574bac348..7755bf709 100644 --- a/internal/controller/trigger/validation/subscripton.go +++ b/internal/controller/trigger/validation/subscripton.go @@ -63,6 +63,8 @@ func validateProtocol(ctx context.Context, protocol metapb.Protocol) error { case metapb.Protocol_HTTP: case metapb.Protocol_AWS_LAMBDA: case metapb.Protocol_GCLOUD_FUNCTIONS: + case metapb.Protocol_GRPC: + default: return errors.ErrInvalidRequest.WithMessage("protocol is invalid") } @@ -96,6 +98,7 @@ func ValidateSinkAndProtocol(ctx context.Context, return errors.ErrInvalidRequest. WithMessage("protocol is http, sink is url,url parse error").Wrap(err) } + case metapb.Protocol_GRPC: } return nil } diff --git a/internal/convert/convert.go b/internal/convert/convert.go index c0bd67c0c..7ff66da16 100644 --- a/internal/convert/convert.go +++ b/internal/convert/convert.go @@ -53,6 +53,8 @@ func fromPbProtocol(from pb.Protocol) primitive.Protocol { to = primitive.AwsLambdaProtocol case pb.Protocol_GCLOUD_FUNCTIONS: to = primitive.GCloudFunctions + case pb.Protocol_GRPC: + to = primitive.GRPC } return to } @@ -66,6 +68,8 @@ func toPbProtocol(from primitive.Protocol) pb.Protocol { to = pb.Protocol_AWS_LAMBDA case primitive.GCloudFunctions: to = pb.Protocol_GCLOUD_FUNCTIONS + case primitive.GRPC: + to = pb.Protocol_GRPC } return to } diff --git a/internal/primitive/subscription.go b/internal/primitive/subscription.go index 1e0b644b9..1e239a8ab 100644 --- a/internal/primitive/subscription.go +++ b/internal/primitive/subscription.go @@ -49,6 +49,7 @@ const ( HTTPProtocol Protocol = "http" AwsLambdaProtocol Protocol = "aws-lambda" GCloudFunctions Protocol = "gcloud-functions" + GRPC Protocol = "grpc" ) type ProtocolSetting struct { diff --git a/internal/trigger/client/gcloud_functions.go b/internal/trigger/client/gcloud_functions.go index d873d10a5..797d20760 100644 --- a/internal/trigger/client/gcloud_functions.go +++ b/internal/trigger/client/gcloud_functions.go @@ -54,7 +54,8 @@ func (c *gcloudFunctions) init(ctx context.Context) error { return nil } -func (c *gcloudFunctions) Send(ctx context.Context, event ce.Event) Result { +func (c *gcloudFunctions) Send(ctx context.Context, events ...*ce.Event) Result { + event := events[0] if c.client == nil { err := c.init(ctx) if err != nil { diff --git a/internal/trigger/client/grpc.go b/internal/trigger/client/grpc.go new file mode 100644 index 000000000..a6027f96e --- /dev/null +++ b/internal/trigger/client/grpc.go @@ -0,0 +1,84 @@ +// 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 client + +import ( + "context" + "sync" + "time" + + ce "github.com/cloudevents/sdk-go/v2" + "github.com/linkall-labs/vanus/client/pkg/codec" + "github.com/linkall-labs/vanus/proto/pkg/cloudevents" + "github.com/pkg/errors" + stdGrpc "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" +) + +type grpc struct { + client cloudevents.CloudEventsClient + url string + lock sync.Mutex +} + +func NewGRPCClient(url string) EventClient { + return &grpc{ + url: url, + } +} + +func (c *grpc) init() error { + c.lock.Lock() + defer c.lock.Unlock() + if c.client != nil { + return nil + } + opts := []stdGrpc.DialOption{ + stdGrpc.WithBlock(), + stdGrpc.WithTransportCredentials(insecure.NewCredentials()), + } + //nolint:gomnd //wrong check + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + defer cancel() + conn, err := stdGrpc.DialContext(ctx, c.url, opts...) + if err != nil { + return err + } + c.client = cloudevents.NewCloudEventsClient(conn) + return nil +} + +func (c *grpc) Send(ctx context.Context, events ...*ce.Event) Result { + if c.client == nil { + err := c.init() + if err != nil { + return newUndefinedErr(err) + } + } + es := make([]*cloudevents.CloudEvent, len(events)) + for idx := range events { + es[idx], _ = codec.ToProto(events[idx]) + } + _, err := c.client.Send(ctx, &cloudevents.BatchEvent{ + Events: &cloudevents.CloudEventBatch{Events: es}, + }) + if err != nil { + if errors.Is(err, context.DeadlineExceeded) { + return DeliveryTimeout + } + return newUndefinedErr(err) + } + return Success +} diff --git a/internal/trigger/client/http.go b/internal/trigger/client/http.go index 5b0f782f1..22230a947 100644 --- a/internal/trigger/client/http.go +++ b/internal/trigger/client/http.go @@ -33,8 +33,9 @@ func NewHTTPClient(url string) EventClient { } } -func (c *http) Send(ctx context.Context, event ce.Event) Result { - res := c.client.Send(ctx, event) +func (c *http) Send(ctx context.Context, events ...*ce.Event) Result { + event := events[0] + res := c.client.Send(ctx, *event) if ce.IsACK(res) { return Success } diff --git a/internal/trigger/client/interface.go b/internal/trigger/client/interface.go index c733340fb..476ea04cc 100644 --- a/internal/trigger/client/interface.go +++ b/internal/trigger/client/interface.go @@ -25,7 +25,7 @@ import ( ) type Sender interface { - Send(ctx context.Context, event ce.Event) Result + Send(ctx context.Context, events ...*ce.Event) Result } type EventClient interface { @@ -83,5 +83,5 @@ const ( errStatusCode = nethttp.StatusBadRequest ErrDeliveryTimeout = 601 - ErrUndefined = 700 + ErrUndefined = 602 ) diff --git a/internal/trigger/client/lambda.go b/internal/trigger/client/lambda.go index d47569869..520733a97 100644 --- a/internal/trigger/client/lambda.go +++ b/internal/trigger/client/lambda.go @@ -43,7 +43,8 @@ func NewAwsLambdaClient(accessKeyID, secretKeyID, arnStr string) EventClient { } } -func (l *awsLambda) Send(ctx context.Context, event ce.Event) Result { +func (l *awsLambda) Send(ctx context.Context, events ...*ce.Event) Result { + event := events[0] payload, err := event.MarshalJSON() if err != nil { return newInternalErr(err) diff --git a/internal/trigger/client/mock_interface.go b/internal/trigger/client/mock_interface.go index 8530b87d8..a70bb1c47 100644 --- a/internal/trigger/client/mock_interface.go +++ b/internal/trigger/client/mock_interface.go @@ -36,17 +36,22 @@ func (m *MockSender) EXPECT() *MockSenderMockRecorder { } // Send mocks base method. -func (m *MockSender) Send(ctx context.Context, event v2.Event) Result { +func (m *MockSender) Send(ctx context.Context, events ...*v2.Event) Result { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Send", ctx, event) + varargs := []interface{}{ctx} + for _, a := range events { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "Send", varargs...) ret0, _ := ret[0].(Result) return ret0 } // Send indicates an expected call of Send. -func (mr *MockSenderMockRecorder) Send(ctx, event interface{}) *gomock.Call { +func (mr *MockSenderMockRecorder) Send(ctx interface{}, events ...interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Send", reflect.TypeOf((*MockSender)(nil).Send), ctx, event) + varargs := append([]interface{}{ctx}, events...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Send", reflect.TypeOf((*MockSender)(nil).Send), varargs...) } // MockEventClient is a mock of EventClient interface. @@ -73,15 +78,20 @@ func (m *MockEventClient) EXPECT() *MockEventClientMockRecorder { } // Send mocks base method. -func (m *MockEventClient) Send(ctx context.Context, event v2.Event) Result { +func (m *MockEventClient) Send(ctx context.Context, events ...*v2.Event) Result { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Send", ctx, event) + varargs := []interface{}{ctx} + for _, a := range events { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "Send", varargs...) ret0, _ := ret[0].(Result) return ret0 } // Send indicates an expected call of Send. -func (mr *MockEventClientMockRecorder) Send(ctx, event interface{}) *gomock.Call { +func (mr *MockEventClientMockRecorder) Send(ctx interface{}, events ...interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Send", reflect.TypeOf((*MockEventClient)(nil).Send), ctx, event) + varargs := append([]interface{}{ctx}, events...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Send", reflect.TypeOf((*MockEventClient)(nil).Send), varargs...) } diff --git a/internal/trigger/trigger/config.go b/internal/trigger/trigger/config.go index 6e6470688..0d1377263 100644 --- a/internal/trigger/trigger/config.go +++ b/internal/trigger/trigger/config.go @@ -27,6 +27,8 @@ const ( defaultFilterProcessSize = 2 defaultDeliveryTimeout = 5 * time.Second defaultMaxWriteAttempt = 3 + defaultGoroutineSize = 10000 + defaultBatchSize = 32 ) type Config struct { @@ -39,6 +41,9 @@ type Config struct { DeadLetterEventbus string MaxWriteAttempt int Ordered bool + + GoroutineSize int + BatchSize int } func defaultConfig() Config { @@ -49,6 +54,8 @@ func defaultConfig() Config { DeliveryTimeout: defaultDeliveryTimeout, DeadLetterEventbus: primitive.DeadLetterEventbusName, MaxWriteAttempt: defaultMaxWriteAttempt, + GoroutineSize: defaultGoroutineSize, + BatchSize: defaultBatchSize, } return c } diff --git a/internal/trigger/trigger/trigger.go b/internal/trigger/trigger/trigger.go index 87a0f4f02..12b71e10e 100644 --- a/internal/trigger/trigger/trigger.go +++ b/internal/trigger/trigger/trigger.go @@ -36,6 +36,7 @@ import ( "github.com/linkall-labs/vanus/observability/log" "github.com/linkall-labs/vanus/observability/metrics" "github.com/linkall-labs/vanus/pkg/util" + "github.com/panjf2000/ants/v2" "go.uber.org/ratelimit" ) @@ -66,13 +67,15 @@ type trigger struct { offsetManager *offset.SubscriptionOffset reader reader.Reader eventCh chan info.EventRecord - sendCh chan info.EventRecord + sendCh chan *toSendEvent + batchSendCh chan []*toSendEvent eventCli client.EventClient client eb.Client filter filter.Filter transformer *transform.Transformer rateLimiter ratelimit.Limiter config Config + batch bool retryEventCh chan info.EventRecord retryEventReader reader.Reader @@ -83,6 +86,13 @@ type trigger struct { stop context.CancelFunc lock sync.RWMutex wg util.Group + + pool *ants.Pool +} + +type toSendEvent struct { + record info.EventRecord + transform *ce.Event } func NewTrigger(subscription *primitive.Subscription, opts ...Option) Trigger { @@ -96,10 +106,14 @@ func NewTrigger(subscription *primitive.Subscription, opts ...Option) Trigger { subscriptionIDStr: subscription.ID.String(), transformer: transform.NewTransformer(subscription.Transformer), } + if subscription.Protocol == primitive.GRPC { + t.batch = true + } t.applyOptions(opts...) if t.rateLimiter == nil { t.rateLimiter = ratelimit.NewUnlimited() } + t.pool, _ = ants.NewPool(t.config.GoroutineSize) return t } @@ -187,83 +201,138 @@ func (t *trigger) eventArrived(ctx context.Context, event info.EventRecord) erro } } -func (t *trigger) sendEvent(ctx context.Context, e *ce.Event) (int, error) { - var err error +func (t *trigger) transformEvent(record info.EventRecord) (*toSendEvent, error) { transformer := t.getTransformer() - sendEvent := *e + event := record.Event if transformer != nil { // transform will chang event which lost origin event - sendEvent = e.Clone() + clone := record.Event.Clone() + event = &clone startTime := time.Now() - err = transformer.Execute(&sendEvent) + err := transformer.Execute(event) metrics.TriggerTransformCostSecond.WithLabelValues(t.subscriptionIDStr).Observe(time.Since(startTime).Seconds()) if err != nil { - return -1, err + return nil, err } } + return &toSendEvent{record: record, transform: event}, nil +} + +func (t *trigger) sendEvent(ctx context.Context, events ...*ce.Event) (int, error) { timeoutCtx, cancel := context.WithTimeout(ctx, t.getConfig().DeliveryTimeout) defer cancel() t.rateLimiter.Take() startTime := time.Now() - r := t.getClient().Send(timeoutCtx, sendEvent) + r := t.getClient().Send(timeoutCtx, events...) if r == client.Success { metrics.TriggerPushEventTime.WithLabelValues(t.subscriptionIDStr).Observe(time.Since(startTime).Seconds()) } return r.StatusCode, r.Err } -func (t *trigger) runRetryEventFilter(ctx context.Context) { +func (t *trigger) runRetryEventFilterTransform(ctx context.Context) { for { select { case <-ctx.Done(): return - case event, ok := <-t.retryEventCh: + case record, ok := <-t.retryEventCh: if !ok { return } - t.offsetManager.EventReceive(event.OffsetInfo) - ec, _ := event.Event.Context.(*ce.EventContextV1) - if len(ec.Extensions) == 0 { - t.offsetManager.EventCommit(event.OffsetInfo) - continue - } - v, exist := ec.Extensions[primitive.XVanusSubscriptionID] - if !exist || t.subscriptionIDStr != v.(string) { - t.offsetManager.EventCommit(event.OffsetInfo) - continue - } - startTime := time.Now() - res := filter.Run(t.getFilter(), *event.Event) - metrics.TriggerFilterCostSecond.WithLabelValues(t.subscriptionIDStr).Observe(time.Since(startTime).Seconds()) - if res == filter.FailFilter { - t.offsetManager.EventCommit(event.OffsetInfo) - continue + _ = t.pool.Submit(func() { + t.offsetManager.EventReceive(record.OffsetInfo) + ec, _ := record.Event.Context.(*ce.EventContextV1) + if len(ec.Extensions) == 0 { + t.offsetManager.EventCommit(record.OffsetInfo) + return + } + v, exist := ec.Extensions[primitive.XVanusSubscriptionID] + if !exist || t.subscriptionIDStr != v.(string) { + t.offsetManager.EventCommit(record.OffsetInfo) + return + } + startTime := time.Now() + res := filter.Run(t.getFilter(), *record.Event) + metrics.TriggerFilterCostSecond.WithLabelValues(t.subscriptionIDStr).Observe(time.Since(startTime).Seconds()) + if res == filter.FailFilter { + t.offsetManager.EventCommit(record.OffsetInfo) + return + } + metrics.TriggerFilterMatchRetryEventCounter.WithLabelValues(t.subscriptionIDStr).Inc() + event, err := t.transformEvent(record) + if err != nil { + t.writeFailEvent(ctx, record.Event, ErrTransformCode, err) + t.offsetManager.EventCommit(record.OffsetInfo) + return + } + t.sendCh <- event + }) + } + } +} + +func (t *trigger) runEventFilterTransform(ctx context.Context) { + for { + select { + case <-ctx.Done(): + return + case record, ok := <-t.eventCh: + if !ok { + return } - t.sendCh <- event - metrics.TriggerFilterMatchRetryEventCounter.WithLabelValues(t.subscriptionIDStr).Inc() + _ = t.pool.Submit(func() { + t.offsetManager.EventReceive(record.OffsetInfo) + startTime := time.Now() + res := filter.Run(t.getFilter(), *record.Event) + metrics.TriggerFilterCostSecond.WithLabelValues(t.subscriptionIDStr).Observe(time.Since(startTime).Seconds()) + if res == filter.FailFilter { + t.offsetManager.EventCommit(record.OffsetInfo) + return + } + metrics.TriggerFilterMatchEventCounter.WithLabelValues(t.subscriptionIDStr).Inc() + event, err := t.transformEvent(record) + if err != nil { + t.writeFailEvent(ctx, record.Event, ErrTransformCode, err) + t.offsetManager.EventCommit(record.OffsetInfo) + return + } + t.sendCh <- event + }) } } } -func (t *trigger) runEventFilter(ctx context.Context) { +func (t *trigger) runEventToBatch(ctx context.Context) { + var events []*toSendEvent + ticker := time.NewTicker(500 * time.Millisecond) ////nolint:gomnd + defer ticker.Stop() + var lock sync.Mutex for { select { case <-ctx.Done(): return - case event, ok := <-t.eventCh: + case <-ticker.C: + lock.Lock() + if len(events) != 0 { + e := make([]*toSendEvent, len(events)) + copy(e, events) + t.batchSendCh <- e + events = nil + } + lock.Unlock() + case event, ok := <-t.sendCh: if !ok { return } - t.offsetManager.EventReceive(event.OffsetInfo) - startTime := time.Now() - res := filter.Run(t.getFilter(), *event.Event) - metrics.TriggerFilterCostSecond.WithLabelValues(t.subscriptionIDStr).Observe(time.Since(startTime).Seconds()) - if res == filter.FailFilter { - t.offsetManager.EventCommit(event.OffsetInfo) - continue + lock.Lock() + events = append(events, event) + if !t.batch || len(events) >= t.config.BatchSize { + e := make([]*toSendEvent, len(events)) + copy(e, events) + t.batchSendCh <- e + events = nil } - t.sendCh <- event - metrics.TriggerFilterMatchEventCounter.WithLabelValues(t.subscriptionIDStr).Inc() + lock.Unlock() } } } @@ -273,43 +342,52 @@ func (t *trigger) runEventSend(ctx context.Context) { select { case <-ctx.Done(): return - case event, ok := <-t.sendCh: + case events, ok := <-t.batchSendCh: if !ok { return } if t.config.Ordered { - t.processEvent(ctx, event) + t.processEvent(ctx, events...) } else { - go func(event info.EventRecord) { - t.processEvent(ctx, event) - }(event) + _ = t.pool.Submit(func() { + t.processEvent(ctx, events...) + }) } } } } -func (t *trigger) processEvent(ctx context.Context, event info.EventRecord) { - code, err := t.sendEvent(ctx, event.Event) +func (t *trigger) processEvent(ctx context.Context, events ...*toSendEvent) { + es := make([]*ce.Event, len(events)) + for i := range events { + es[i] = events[i].transform + } + code, err := t.sendEvent(ctx, es...) if err != nil { metrics.TriggerPushEventCounter.WithLabelValues(t.subscriptionIDStr, metrics.LabelValuePushEventFail).Inc() log.Info(ctx, "send event fail", map[string]interface{}{ log.KeyError: err, - "event": event.Event, + "count": len(es), }) if t.config.Ordered { // ordered event no need retry direct into dead letter - code = NoNeedRetryCode + code = OrderEventCode + } + for _, event := range events { + t.writeFailEvent(ctx, event.record.Event, code, err) } - t.writeFailEvent(ctx, event.Event, code, err) } else { metrics.TriggerPushEventCounter.WithLabelValues(t.subscriptionIDStr, metrics.LabelValuePushEventSuccess).Inc() log.Debug(ctx, "send event success", map[string]interface{}{ - "event": event.Event, + "count": len(es), }) } - t.offsetManager.EventCommit(event.OffsetInfo) + for _, event := range events { + t.offsetManager.EventCommit(event.record.OffsetInfo) + } } -func (t *trigger) writeFailEvent(ctx context.Context, e *ce.Event, code int, sendErr error) { + +func (t *trigger) writeFailEvent(ctx context.Context, e *ce.Event, code int, err error) { needRetry, reason := isShouldRetry(code) ec, _ := e.Context.(*ce.EventContextV1) if ec.Extensions == nil { @@ -334,7 +412,7 @@ func (t *trigger) writeFailEvent(ctx context.Context, e *ce.Event, code int, sen } if !needRetry { // dead letter - t.writeEventToDeadLetter(ctx, e, reason, sendErr.Error()) + t.writeEventToDeadLetter(ctx, e, reason, err.Error()) metrics.TriggerDeadLetterEventCounter.WithLabelValues(t.subscriptionIDStr).Inc() return } @@ -457,7 +535,8 @@ func (t *trigger) Init(ctx context.Context) error { t.timerEventWriter = t.client.Eventbus(ctx, primitive.TimerEventbusName).Writer() t.dlEventWriter = t.client.Eventbus(ctx, t.config.DeadLetterEventbus).Writer() t.eventCh = make(chan info.EventRecord, t.config.BufferSize) - t.sendCh = make(chan info.EventRecord, t.config.BufferSize) + t.sendCh = make(chan *toSendEvent, t.config.BufferSize) + t.batchSendCh = make(chan []*toSendEvent, t.config.BufferSize) t.reader = reader.NewReader(t.getReaderConfig(), t.eventCh) t.retryEventCh = make(chan info.EventRecord, t.config.BufferSize) t.retryEventReader = reader.NewReader(t.getRetryEventReaderConfig(), t.retryEventCh) @@ -473,13 +552,12 @@ func (t *trigger) Start(ctx context.Context) error { t.stop = cancel // eb event _ = t.reader.Start() - for i := 0; i < t.config.FilterProcessSize; i++ { - t.wg.StartWithContext(ctx, t.runEventFilter) - } + t.wg.StartWithContext(ctx, t.runEventFilterTransform) + t.wg.StartWithContext(ctx, t.runEventToBatch) t.wg.StartWithContext(ctx, t.runEventSend) // retry event _ = t.retryEventReader.Start() - t.wg.StartWithContext(ctx, t.runRetryEventFilter) + t.wg.StartWithContext(ctx, t.runRetryEventFilterTransform) t.state = TriggerRunning log.Info(ctx, "trigger started", map[string]interface{}{ log.KeySubscriptionID: t.subscription.ID, @@ -499,8 +577,10 @@ func (t *trigger) Stop(ctx context.Context) error { t.retryEventReader.Close() t.wg.Wait() close(t.eventCh) - close(t.sendCh) close(t.retryEventCh) + close(t.sendCh) + close(t.batchSendCh) + t.pool.Release() t.state = TriggerStopped log.Info(ctx, "trigger stopped", map[string]interface{}{ log.KeySubscriptionID: t.subscription.ID, diff --git a/internal/trigger/trigger/trigger_test.go b/internal/trigger/trigger/trigger_test.go index a259dabeb..6178f25f3 100644 --- a/internal/trigger/trigger/trigger_test.go +++ b/internal/trigger/trigger/trigger_test.go @@ -190,7 +190,7 @@ func TestTriggerRunEventSend(t *testing.T) { wg.Add(1) go func() { defer wg.Done() - tg.runEventFilter(ctx) + tg.runEventFilterTransform(ctx) }() time.Sleep(100 * time.Millisecond) So(len(tg.sendCh), ShouldEqual, size) @@ -202,7 +202,7 @@ func TestTriggerRunEventSend(t *testing.T) { wg.Add(1) go func() { defer wg.Done() - tg.runEventSend(ctx) + tg.runEventToBatch(ctx) }() time.Sleep(100 * time.Millisecond) close(tg.sendCh) diff --git a/internal/trigger/trigger/util.go b/internal/trigger/trigger/util.go index c6151f72d..6b8afe9bd 100644 --- a/internal/trigger/trigger/util.go +++ b/internal/trigger/trigger/util.go @@ -34,17 +34,24 @@ func newEventClient(sink primitive.URI, case primitive.GCloudFunctions: _credential, _ := credential.(*primitive.GCloudSinkCredential) return client.NewGCloudFunctionClient(string(sink), _credential.CredentialJSON) + case primitive.GRPC: + return client.NewGRPCClient(string(sink)) default: return client.NewHTTPClient(string(sink)) } } -const NoNeedRetryCode = -1 +const ( + OrderEventCode = -1 + ErrTransformCode = 1 +) func isShouldRetry(statusCode int) (bool, string) { switch statusCode { - case NoNeedRetryCode: - return false, "NoNeedRetry" + case ErrTransformCode: + return false, "TransformError" + case OrderEventCode: + return false, "OrderEvent" case 400: return false, "BadRequest" case 403: diff --git a/proto/pkg/meta/meta.pb.go b/proto/pkg/meta/meta.pb.go index ba358e273..9bbd9b308 100644 --- a/proto/pkg/meta/meta.pb.go +++ b/proto/pkg/meta/meta.pb.go @@ -139,6 +139,7 @@ const ( Protocol_HTTP Protocol = 0 Protocol_AWS_LAMBDA Protocol = 1 Protocol_GCLOUD_FUNCTIONS Protocol = 2 + Protocol_GRPC Protocol = 3 ) // Enum value maps for Protocol. @@ -147,11 +148,13 @@ var ( 0: "HTTP", 1: "AWS_LAMBDA", 2: "GCLOUD_FUNCTIONS", + 3: "GRPC", } Protocol_value = map[string]int32{ "HTTP": 0, "AWS_LAMBDA": 1, "GCLOUD_FUNCTIONS": 2, + "GRPC": 3, } ) @@ -2050,14 +2053,15 @@ var file_meta_proto_rawDesc = []byte{ 0x44, 0x10, 0x02, 0x12, 0x06, 0x0a, 0x02, 0x53, 0x33, 0x10, 0x03, 0x2a, 0x26, 0x0a, 0x11, 0x43, 0x6f, 0x6d, 0x70, 0x72, 0x65, 0x73, 0x73, 0x41, 0x6c, 0x67, 0x6f, 0x72, 0x69, 0x74, 0x68, 0x6d, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x4f, 0x4e, 0x45, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x4c, 0x5a, - 0x34, 0x10, 0x01, 0x2a, 0x3a, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, + 0x34, 0x10, 0x01, 0x2a, 0x44, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x08, 0x0a, 0x04, 0x48, 0x54, 0x54, 0x50, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x41, 0x57, 0x53, 0x5f, 0x4c, 0x41, 0x4d, 0x42, 0x44, 0x41, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x47, 0x43, 0x4c, - 0x4f, 0x55, 0x44, 0x5f, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x02, 0x42, - 0x2e, 0x5a, 0x2c, 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, 0x6d, 0x65, 0x74, 0x61, 0x62, - 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x4f, 0x55, 0x44, 0x5f, 0x46, 0x55, 0x4e, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x02, 0x12, + 0x08, 0x0a, 0x04, 0x47, 0x52, 0x50, 0x43, 0x10, 0x03, 0x42, 0x2e, 0x5a, 0x2c, 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, 0x6d, 0x65, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, } var ( diff --git a/proto/proto/meta.proto b/proto/proto/meta.proto index 588395d27..da725e2c4 100644 --- a/proto/proto/meta.proto +++ b/proto/proto/meta.proto @@ -118,6 +118,7 @@ enum Protocol{ HTTP = 0; AWS_LAMBDA = 1; GCLOUD_FUNCTIONS = 2; + GRPC = 3; } message SinkCredential { diff --git a/vsctl/command/subscription.go b/vsctl/command/subscription.go index 9ec8a876b..d4442f029 100644 --- a/vsctl/command/subscription.go +++ b/vsctl/command/subscription.go @@ -87,6 +87,8 @@ func createSubscriptionCommand() *cobra.Command { if sinkCredentialType != GCloudCredentialType { cmdFailedf(cmd, "protocol is aws-lambda, credential-type must be %s\n", GCloudCredentialType) } + case "grpc": + p = meta.Protocol_GRPC default: cmdFailedf(cmd, "protocol is invalid\n") } @@ -213,7 +215,7 @@ func createSubscriptionCommand() *cobra.Command { cmd.Flags().StringVar(&transformer, "transformer", "", "transformer, JSON format required") cmd.Flags().Uint32Var(&rateLimit, "rate-limit", 0, "max event number pushing to sink per second, default is 0, means unlimited") cmd.Flags().StringVar(&from, "from", "", "consume events from, latest,earliest or RFC3339 format time") - cmd.Flags().StringVar(&subProtocol, "protocol", "http", "protocol,http or aws-lambda or gcloud-functions") + cmd.Flags().StringVar(&subProtocol, "protocol", "http", "protocol,http or aws-lambda or gcloud-functions or grpc") cmd.Flags().StringVar(&sinkCredentialType, "credential-type", "", "sink credential type: aws or gcloud") cmd.Flags().StringVar(&sinkCredential, "credential", "", "sink credential info, JSON format or @file") cmd.Flags().Uint32Var(&deliveryTimeout, "delivery-timeout", 0, "event delivery to sink timeout by millisecond, default is 0, means using server-side default value: 5s") @@ -486,6 +488,8 @@ func getSubscriptionRow(sub *meta.Subscription) []interface{} { protocol = "aws-lambda" case meta.Protocol_GCLOUD_FUNCTIONS: protocol = "gcloud-functions" + case meta.Protocol_GRPC: + protocol = "grpc" } result = append(result, protocol) From b7c5d8775e89baea3caa208d97ae0c22d8acd56b Mon Sep 17 00:00:00 2001 From: wenfeng Date: Thu, 5 Jan 2023 16:09:14 +0800 Subject: [PATCH 4/9] feat: enrich vanus test infrastructure (#382) * feat: vanus-bench support batch sending Signed-off-by: wenfeng * update infra Signed-off-by: wenfeng * update Signed-off-by: wenfeng * update Signed-off-by: wenfeng Signed-off-by: wenfeng --- .github/workflows/codecov.yml | 3 +- build/images/controller/Dockerfile | 2 +- build/images/gateway/Dockerfile | 2 +- build/images/store/Dockerfile | 2 +- build/images/timer/Dockerfile | 2 +- build/images/trigger/Dockerfile | 2 +- client/go.mod | 6 +- client/pkg/eventbus/eventbus.go | 4 +- go.mod | 10 +- internal/gateway/proxy/proxy.go | 13 +- internal/raft/transport/host.go | 5 + internal/raft/transport/peer.go | 14 +- internal/store/block/raft/appender.go | 5 +- internal/store/block/raft/transport.go | 27 +- internal/store/io/stream/stream.go | 9 +- internal/store/vsb/block_append.go | 12 + internal/store/vsb/block_read.go | 15 +- internal/store/vsb/block_snapshot.go | 13 +- internal/store/wal/record/record.go | 8 +- pkg/go.mod | 4 +- proto/go.mod | 2 +- raft/go.mod | 2 +- test/benchmark/command/common.go | 4 - test/benchmark/command/component.go | 53 +++- test/benchmark/command/performance.go | 369 +++++++++++++++++-------- test/benchmark/main.go | 4 - test/infra/Dockerfile | 9 +- test/infra/benchmark.yml | 6 +- test/infra/case1/job-a.yml | 6 +- test/infra/case1/job-b.yml | 6 +- test/infra/case1/job-c.yml | 9 +- test/infra/case2/play.sh | 4 +- test/infra/consumer/consumer.yml | 51 ++++ test/infra/consumer/play.sh | 7 + test/infra/producer/play.sh | 14 + test/infra/producer/producer.yml | 47 ++++ test/infra/run.sh | 4 - test/infra/sc.yml | 259 +++++++++++++++++ test/infra/secret.yml.example | 2 +- test/infra/vanus.yml | 83 ++++-- 40 files changed, 874 insertions(+), 225 deletions(-) create mode 100644 test/infra/consumer/consumer.yml create mode 100644 test/infra/consumer/play.sh create mode 100644 test/infra/producer/play.sh create mode 100644 test/infra/producer/producer.yml delete mode 100644 test/infra/run.sh create mode 100644 test/infra/sc.yml diff --git a/.github/workflows/codecov.yml b/.github/workflows/codecov.yml index 16b5db9f5..8c9e77971 100644 --- a/.github/workflows/codecov.yml +++ b/.github/workflows/codecov.yml @@ -21,7 +21,8 @@ jobs: startsWith(github.event.pull_request.title, 'fix') || startsWith(github.event.pull_request.title, 'feat') || startsWith(github.event.pull_request.title, 'refactor') || - startsWith(github.event.pull_request.title, 'perf') + startsWith(github.event.pull_request.title, 'perf') || + startsWith(github.event.pull_request.title, 'test') runs-on: ${{ matrix.os }} env: CODECOV_TOKEN: ${{ secrets.CODECOV_TOKEN }} diff --git a/build/images/controller/Dockerfile b/build/images/controller/Dockerfile index 56fb86ee1..c6be0225c 100644 --- a/build/images/controller/Dockerfile +++ b/build/images/controller/Dockerfile @@ -8,7 +8,7 @@ ARG TARGETOS ARG TARGETARCH RUN GOOS=$TARGETOS GOARCH=$TARGETARCH make build-controller -FROM alpine:3.15.4 +FROM ubuntu:22.10 WORKDIR /vanus COPY --from=builder /workspace/bin/controller bin/controller ENTRYPOINT ["bin/controller"] diff --git a/build/images/gateway/Dockerfile b/build/images/gateway/Dockerfile index 396dc45a9..c335f7831 100644 --- a/build/images/gateway/Dockerfile +++ b/build/images/gateway/Dockerfile @@ -8,7 +8,7 @@ ARG TARGETOS ARG TARGETARCH RUN GOOS=$TARGETOS GOARCH=$TARGETARCH make build-gateway -FROM alpine:3.15.4 +FROM ubuntu:22.10 WORKDIR /vanus COPY --from=builder /workspace/bin/gateway bin/gateway ENTRYPOINT ["bin/gateway"] diff --git a/build/images/store/Dockerfile b/build/images/store/Dockerfile index ebb6d1952..cc33f6d13 100644 --- a/build/images/store/Dockerfile +++ b/build/images/store/Dockerfile @@ -8,7 +8,7 @@ ARG TARGETOS ARG TARGETARCH RUN GOOS=$TARGETOS GOARCH=$TARGETARCH make build-store -FROM alpine:3.15.4 +FROM ubuntu:22.10 WORKDIR /vanus COPY --from=builder /workspace/bin/store bin/store ENTRYPOINT ["bin/store"] diff --git a/build/images/timer/Dockerfile b/build/images/timer/Dockerfile index edaa3a5ec..e67ede69f 100644 --- a/build/images/timer/Dockerfile +++ b/build/images/timer/Dockerfile @@ -8,7 +8,7 @@ ARG TARGETOS ARG TARGETARCH RUN GOOS=$TARGETOS GOARCH=$TARGETARCH make build-timer -FROM alpine:3.15.4 +FROM ubuntu:22.10 WORKDIR /vanus COPY --from=builder /workspace/bin/timer bin/timer ENTRYPOINT ["bin/timer"] diff --git a/build/images/trigger/Dockerfile b/build/images/trigger/Dockerfile index 630f878e1..9b45b92cc 100644 --- a/build/images/trigger/Dockerfile +++ b/build/images/trigger/Dockerfile @@ -8,7 +8,7 @@ ARG TARGETOS ARG TARGETARCH RUN GOOS=$TARGETOS GOARCH=$TARGETARCH make build-trigger -FROM alpine:3.15.4 +FROM ubuntu:22.10 WORKDIR /vanus COPY --from=builder /workspace/bin/trigger bin/trigger ENTRYPOINT ["bin/trigger"] diff --git a/client/go.mod b/client/go.mod index 1aa246039..a7e037dd2 100644 --- a/client/go.mod +++ b/client/go.mod @@ -5,9 +5,9 @@ go 1.18 require ( github.com/cloudevents/sdk-go/v2 v2.12.0 github.com/golang/mock v1.6.0 - github.com/linkall-labs/vanus/observability v0.5.1 - github.com/linkall-labs/vanus/pkg v0.5.1 - github.com/linkall-labs/vanus/proto v0.5.1 + github.com/linkall-labs/vanus/observability v0.5.6 + github.com/linkall-labs/vanus/pkg v0.5.6 + github.com/linkall-labs/vanus/proto v0.5.6 github.com/scylladb/go-set v1.0.2 go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.36.4 go.opentelemetry.io/otel/trace v1.11.2 diff --git a/client/pkg/eventbus/eventbus.go b/client/pkg/eventbus/eventbus.go index 813a8a806..b42d28745 100644 --- a/client/pkg/eventbus/eventbus.go +++ b/client/pkg/eventbus/eventbus.go @@ -450,7 +450,7 @@ func (w *busWriter) AppendBatch(ctx context.Context, events *cloudevents.CloudEv _ctx, span := w.tracer.Start(ctx, "CloudEventBatch") defer span.End() - var writeOpts *api.WriteOptions = w.opts + var writeOpts = w.opts if len(opts) > 0 { writeOpts = w.opts.Copy() for _, opt := range opts { @@ -475,7 +475,7 @@ func (w *busWriter) AppendOne(ctx context.Context, event *ce.Event, opts ...api. _ctx, span := w.tracer.Start(ctx, "AppendOne") defer span.End() - var writeOpts *api.WriteOptions = w.opts + var writeOpts = w.opts if len(opts) > 0 { writeOpts = w.opts.Copy() for _, opt := range opts { diff --git a/go.mod b/go.mod index dfbe6104f..cf8c85a46 100644 --- a/go.mod +++ b/go.mod @@ -23,11 +23,11 @@ require ( 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.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 - github.com/linkall-labs/vanus/proto v0.5.1 - github.com/linkall-labs/vanus/raft v0.5.1 + github.com/linkall-labs/vanus/client v0.5.6 + github.com/linkall-labs/vanus/observability v0.5.6 + github.com/linkall-labs/vanus/pkg v0.5.6 + github.com/linkall-labs/vanus/proto v0.5.6 + github.com/linkall-labs/vanus/raft v0.5.6 github.com/ncw/directio v1.0.5 github.com/ohler55/ojg v1.14.5 github.com/panjf2000/ants/v2 v2.7.1 diff --git a/internal/gateway/proxy/proxy.go b/internal/gateway/proxy/proxy.go index 7d2b7728c..9f1dffe00 100644 --- a/internal/gateway/proxy/proxy.go +++ b/internal/gateway/proxy/proxy.go @@ -86,6 +86,7 @@ type ControllerProxy struct { triggerCtrl ctrlpb.TriggerControllerClient grpcSrv *grpc.Server ctrl cluster.Cluster + writerMap sync.Map } func (cp *ControllerProxy) Send(ctx context.Context, batch *cloudevents.BatchEvent) (*emptypb.Empty, error) { @@ -102,6 +103,9 @@ func (cp *ControllerProxy) Send(ctx context.Context, batch *cloudevents.BatchEve if err != nil { return nil, v2.NewHTTPResult(http.StatusBadRequest, err.Error()) } + if e.Attributes == nil { + e.Attributes = make(map[string]*cloudevents.CloudEvent_CloudEventAttributeValue, 0) + } e.Attributes[primitive.XVanusEventbus] = &cloudevents.CloudEvent_CloudEventAttributeValue{ Attr: &cloudevents.CloudEvent_CloudEventAttributeValue_CeString{CeString: batch.EventbusName}, } @@ -119,7 +123,14 @@ func (cp *ControllerProxy) Send(ctx context.Context, batch *cloudevents.BatchEve } } - err := cp.client.Eventbus(ctx, batch.GetEventbusName()).Writer().AppendBatch(_ctx, batch.GetEvents()) + val, exist := cp.writerMap.Load(batch.GetEventbusName()) + if !exist { + val, _ = cp.writerMap.LoadOrStore(batch.GetEventbusName(), + cp.client.Eventbus(ctx, batch.GetEventbusName()).Writer()) + } + + w, _ := val.(api.BusWriter) + err := w.AppendBatch(_ctx, batch.GetEvents()) if err != nil { log.Warning(_ctx, "append to failed", map[string]interface{}{ log.KeyError: err, diff --git a/internal/raft/transport/host.go b/internal/raft/transport/host.go index 85135d78d..ec2451632 100644 --- a/internal/raft/transport/host.go +++ b/internal/raft/transport/host.go @@ -17,6 +17,7 @@ package transport import ( // standard libraries. "context" + "github.com/linkall-labs/vanus/observability/log" "sync" // third-party libraries. @@ -80,6 +81,10 @@ func (h *host) Send(ctx context.Context, msg *raftpb.Message, to uint64, endpoin mux := h.resolveMultiplexer(ctx, to, endpoint) if mux == nil { + log.Info(ctx, "found not mux", map[string]interface{}{ + "to": to, + "endpoint": endpoint, + }) cb(ErrNotReachable) return } diff --git a/internal/raft/transport/peer.go b/internal/raft/transport/peer.go index 36d722b12..4e89fd369 100644 --- a/internal/raft/transport/peer.go +++ b/internal/raft/transport/peer.go @@ -31,8 +31,9 @@ import ( ) const ( + minConnectTimeout = 100 * time.Millisecond defaultConnectTimeout = 300 * time.Millisecond - defaultMessageChainSize = 32 + defaultMessageChainSize = 2048 ) type task struct { @@ -148,8 +149,15 @@ func (p *peer) Send(ctx context.Context, msg *raftpb.Message, cb SendCallback) { } func (p *peer) connect(ctx context.Context, opts ...grpc.DialOption) (vsraftpb.RaftServer_SendMessageClient, error) { - ctx, cancel := context.WithTimeout(ctx, defaultConnectTimeout) - defer cancel() + if dl, ok := ctx.Deadline(); !ok { + cancelCtx, cancel := context.WithTimeout(ctx, defaultConnectTimeout) + defer cancel() + ctx = cancelCtx + } else if dl.Sub(time.Now()) < minConnectTimeout { + cancelCtx, cancel := context.WithTimeout(context.Background(), minConnectTimeout) + defer cancel() + ctx = cancelCtx + } conn, err := grpc.DialContext(ctx, p.addr, opts...) if err != nil { diff --git a/internal/store/block/raft/appender.go b/internal/store/block/raft/appender.go index 7d4bcba93..2f84a226e 100644 --- a/internal/store/block/raft/appender.go +++ b/internal/store/block/raft/appender.go @@ -46,6 +46,7 @@ const ( defaultHeartbeatTick = 3 defaultMaxSizePerMsg = 4096 defaultMaxInflightMsgs = 256 + defaultSendTimeout = 80 * time.Millisecond ) type Peer struct { @@ -215,7 +216,9 @@ func (a *appender) run(ctx context.Context) { } // NOTE: Messages to be sent AFTER HardState and Entries are committed to stable storage. - a.send(rCtx, rd.Messages) + sCtx, cancel := context.WithTimeout(rCtx, defaultSendTimeout) + a.send(sCtx, rd.Messages) + cancel() // TODO(james.yin): snapshot if !raft.IsEmptySnap(rd.Snapshot) { diff --git a/internal/store/block/raft/transport.go b/internal/store/block/raft/transport.go index 2adca62f5..d24be2d1e 100644 --- a/internal/store/block/raft/transport.go +++ b/internal/store/block/raft/transport.go @@ -18,11 +18,14 @@ import ( // standard libraries. "context" - // first-party libraries. - "github.com/linkall-labs/vanus/raft/raftpb" + // third-party libraries. "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" + // first-party libraries. + "github.com/linkall-labs/vanus/observability/log" + "github.com/linkall-labs/vanus/raft/raftpb" + // this project. "github.com/linkall-labs/vanus/internal/raft/transport" ) @@ -41,9 +44,15 @@ func (a *appender) send(ctx context.Context, msgs []raftpb.Message) { if len(msgs) == 1 { msg := &msgs[0] - endpoint := a.peerHint(ctx, msg.To) - a.host.Send(ctx, msg, msg.To, endpoint, func(err error) { + to := msg.To + endpoint := a.peerHint(ctx, to) + a.host.Send(ctx, msg, to, endpoint, func(err error) { if err != nil { + log.Warning(ctx, "send message failed", map[string]interface{}{ + log.KeyError: err, + "to": to, + "endpoint": endpoint, + }) a.node.ReportUnreachable(msg.To) } }) @@ -64,6 +73,11 @@ func (a *appender) send(ctx context.Context, msgs []raftpb.Message) { for i := 0; i < len(msgs); i++ { a.host.Send(ctx, &msgs[i], to, endpoint, func(err error) { if err != nil { + log.Warning(ctx, "send message failed", map[string]interface{}{ + log.KeyError: err, + "to": to, + "endpoint": endpoint, + }) a.node.ReportUnreachable(to) } }) @@ -81,6 +95,11 @@ func (a *appender) send(ctx context.Context, msgs []raftpb.Message) { for _, m := range msgs { a.host.Send(ctx, m, to, endpoint, func(err error) { if err != nil { + log.Warning(ctx, "send message failed", map[string]interface{}{ + log.KeyError: err, + "to": to, + "endpoint": endpoint, + }) a.node.ReportUnreachable(to) } }) diff --git a/internal/store/io/stream/stream.go b/internal/store/io/stream/stream.go index e3a59df75..ab73842f5 100644 --- a/internal/store/io/stream/stream.go +++ b/internal/store/io/stream/stream.go @@ -118,6 +118,10 @@ func (s *stream) Append(r stdio.Reader, cb io.WriteCallback) { if empty { s.waiting = append(s.waiting, cb) + if last == nil && !s.dirty { + s.dirty = true + s.startFlushTimer() + } } if last != nil { @@ -241,7 +245,10 @@ func (s *stream) onFlushed(base int64, off int, cbs []io.WriteCallback) { return } - v, _ = s.pending.LoadAndDelete(base) + v, loaded = s.pending.LoadAndDelete(base) + if !loaded { + continue + } ft, _ := v.(*flushTask) // FIXME(james.yin): pass n diff --git a/internal/store/vsb/block_append.go b/internal/store/vsb/block_append.go index 1a04d3136..9534889dd 100644 --- a/internal/store/vsb/block_append.go +++ b/internal/store/vsb/block_append.go @@ -169,8 +169,14 @@ func (b *vsBlock) CommitAppend(ctx context.Context, frag block.Fragment, cb bloc if !archived { b.s.Append(bytes.NewReader(frag.Payload()), func(n int, err error) { + log.Info(context.Background(), "acquiring index write lock", map[string]interface{}{ + "block_id": b.id, + }) b.mu.Lock() b.indexes = append(b.indexes, indexes...) + log.Info(context.Background(), "release index write lock", map[string]interface{}{ + "block_id": b.id, + }) b.mu.Unlock() cb() @@ -183,8 +189,14 @@ func (b *vsBlock) CommitAppend(ctx context.Context, frag block.Fragment, cb bloc b.wg.Add(1) b.s.Append(bytes.NewReader(frag.Payload()), func(n int, err error) { if len(indexes) != 0 { + log.Info(context.Background(), "acquiring index write lock", map[string]interface{}{ + "block_id": b.id, + }) b.mu.Lock() b.indexes = append(b.indexes, indexes...) + log.Info(context.Background(), "release index write lock", map[string]interface{}{ + "block_id": b.id, + }) b.mu.Unlock() } diff --git a/internal/store/vsb/block_read.go b/internal/store/vsb/block_read.go index 4fad2401b..115ea0b44 100644 --- a/internal/store/vsb/block_read.go +++ b/internal/store/vsb/block_read.go @@ -21,6 +21,9 @@ import ( // third-party libraries. "go.opentelemetry.io/otel/trace" + // first-party libraries. + "github.com/linkall-labs/vanus/observability/log" + // this project. "github.com/linkall-labs/vanus/internal/store/block" ) @@ -57,8 +60,18 @@ func (b *vsBlock) Read(ctx context.Context, seq int64, num int) ([]block.Entry, func (b *vsBlock) entryRange(start, num int) (int64, int64, int, error) { // TODO(james.yin): optimize lock. + log.Info(context.Background(), "acquiring index read lock", map[string]interface{}{ + "block_id": b.id, + "start": start, + "num": num, + }) b.mu.RLock() - defer b.mu.RUnlock() + defer func() { + log.Info(context.Background(), "release index read lock", map[string]interface{}{ + "block_id": b.id, + }) + b.mu.RUnlock() + }() sz := len(b.indexes) diff --git a/internal/store/vsb/block_snapshot.go b/internal/store/vsb/block_snapshot.go index 674ddeff7..d25a17132 100644 --- a/internal/store/vsb/block_snapshot.go +++ b/internal/store/vsb/block_snapshot.go @@ -20,6 +20,9 @@ import ( "encoding/binary" "sync/atomic" + // first-party libraries. + "github.com/linkall-labs/vanus/observability/log" + // this project. "github.com/linkall-labs/vanus/internal/store/block" ceschema "github.com/linkall-labs/vanus/internal/store/schema/ce" @@ -30,8 +33,16 @@ import ( var _ block.Snapshoter = (*vsBlock)(nil) func (b *vsBlock) makeSnapshot() (meta, []index.Index) { + log.Info(context.Background(), "acquiring index read lock", map[string]interface{}{ + "block_id": b.id, + }) b.mu.RLock() - defer b.mu.RUnlock() + defer func() { + log.Info(context.Background(), "release index read lock", map[string]interface{}{ + "block_id": b.id, + }) + b.mu.RUnlock() + }() return makeSnapshot(b.actx, b.indexes) } diff --git a/internal/store/wal/record/record.go b/internal/store/wal/record/record.go index 22653228f..6c675f10f 100644 --- a/internal/store/wal/record/record.go +++ b/internal/store/wal/record/record.go @@ -23,12 +23,12 @@ import ( const ( crcFieldSO = 0 - crcFieldEO = crcFieldSO + 4 + crcFieldEO = crcFieldSO + 4 // [0,4) lengthFieldSO = crcFieldEO - lengthFieldEO = lengthFieldSO + 2 + lengthFieldEO = lengthFieldSO + 2 // [4,6) typeFieldSO = lengthFieldEO - typeFieldEO = typeFieldSO + 1 - dataFieldSO = typeFieldEO + typeFieldEO = typeFieldSO + 1 // [6,7) + dataFieldSO = typeFieldEO // [7,n) ) const HeaderSize = dataFieldSO diff --git a/pkg/go.mod b/pkg/go.mod index b9a10d0c6..82e3f8cc5 100644 --- a/pkg/go.mod +++ b/pkg/go.mod @@ -4,8 +4,8 @@ go 1.18 require ( github.com/golang/mock v1.6.0 - github.com/linkall-labs/vanus/observability v0.5.1 - github.com/linkall-labs/vanus/proto v0.5.1 + github.com/linkall-labs/vanus/observability v0.5.6 + github.com/linkall-labs/vanus/proto v0.5.6 github.com/pkg/errors v0.9.1 github.com/smartystreets/goconvey v1.7.2 google.golang.org/grpc v1.51.0 diff --git a/proto/go.mod b/proto/go.mod index 392e168a0..a5ee0c9da 100644 --- a/proto/go.mod +++ b/proto/go.mod @@ -4,7 +4,7 @@ go 1.18 require ( github.com/golang/mock v1.6.0 - github.com/linkall-labs/vanus/raft v0.5.1 + github.com/linkall-labs/vanus/raft v0.5.6 google.golang.org/grpc v1.51.0 google.golang.org/protobuf v1.28.1 ) diff --git a/raft/go.mod b/raft/go.mod index 86ec49132..7a439ad2f 100644 --- a/raft/go.mod +++ b/raft/go.mod @@ -6,7 +6,7 @@ require ( github.com/cockroachdb/datadriven v0.0.0-20200714090401-bf6692d28da5 github.com/gogo/protobuf v1.3.2 github.com/golang/protobuf v1.5.2 - github.com/linkall-labs/vanus/observability v0.5.1 + github.com/linkall-labs/vanus/observability v0.5.6 go.etcd.io/etcd/client/pkg/v3 v3.6.0-alpha.0 go.opentelemetry.io/otel/trace v1.11.2 ) diff --git a/test/benchmark/command/common.go b/test/benchmark/command/common.go index 94f232bce..c5fc1d426 100644 --- a/test/benchmark/command/common.go +++ b/test/benchmark/command/common.go @@ -33,10 +33,6 @@ type BlockRecord struct { Replicas map[uint64]string } -func SetCaseName(_name string) { - name = _name -} - type ResultType string const ( diff --git a/test/benchmark/command/component.go b/test/benchmark/command/component.go index 0717d3131..f0f1da42c 100644 --- a/test/benchmark/command/component.go +++ b/test/benchmark/command/component.go @@ -18,6 +18,9 @@ import ( "context" "encoding/json" "fmt" + "github.com/google/uuid" + "github.com/linkall-labs/vanus/internal/primitive/vanus" + "google.golang.org/protobuf/types/known/timestamppb" "math/rand" "net" "net/http" @@ -37,7 +40,7 @@ import ( v1 "github.com/linkall-labs/vanus/proto/pkg/cloudevents" segpb "github.com/linkall-labs/vanus/proto/pkg/segment" - "github.com/linkall-labs/vanus/internal/primitive/vanus" + //"github.com/linkall-labs/vanus/internal/primitive/vanus" "github.com/linkall-labs/vanus/internal/store" "github.com/linkall-labs/vanus/internal/store/config" "github.com/linkall-labs/vanus/internal/store/segment" @@ -55,6 +58,7 @@ var ( noCleanCache bool replicaNum int blockSize int64 + batchSize int ) func ComponentCommand() *cobra.Command { @@ -266,9 +270,10 @@ func sendCommand() *cobra.Command { for idx := 0; idx < parallelism; idx++ { go func(br BlockRecord, c segpb.SegmentServerClient) { for atomic.LoadInt64(&count)+atomic.LoadInt64(&failed) < totalSent { + events := generateEvents() _, err := c.AppendToBlock(context.Background(), &segpb.AppendToBlockRequest{ BlockId: br.LeaderID, - Events: &v1.CloudEventBatch{Events: generateEvents()}, + Events: &v1.CloudEventBatch{Events: events}, }) if err != nil { atomic.AddInt64(&failed, 1) @@ -276,7 +281,7 @@ func sendCommand() *cobra.Command { fmt.Printf("failed to append events to %s, block [%s], error: [%s]\n", br.LeaderAddr, vanus.ID(br.LeaderID).String(), err.Error()) } else { - atomic.AddInt64(&count, 1) + atomic.AddInt64(&count, int64(len(events))) } } }(abr, cli) @@ -298,6 +303,7 @@ func sendCommand() *cobra.Command { cmd.Flags().Int64Var(&totalSent, "total-number", 100000, "") cmd.Flags().IntVar(¶llelism, "parallelism", 4, "") cmd.Flags().IntVar(&payloadSize, "payload-size", 1024, "") + cmd.Flags().IntVar(&batchSize, "batch-size", 1, "") return cmd } @@ -337,21 +343,40 @@ func runStore(cfg store.Config) { } var ( - gOnce sync.Once - rd = rand.New(rand.NewSource(time.Now().UnixNano())) - e []*v1.CloudEvent + rd = rand.New(rand.NewSource(time.Now().UnixNano())) + payload string + mutex sync.RWMutex ) func generateEvents() []*v1.CloudEvent { - gOnce.Do(func() { - e = []*v1.CloudEvent{{ - Id: "example-event", - Source: "example/uri", + mutex.RLock() + defer mutex.RUnlock() + if payload == "" { + mutex.RUnlock() + mutex.Lock() + if payload == "" { + payload = genStr(rd, payloadSize) + } + mutex.Unlock() + mutex.RLock() + } + var e []*v1.CloudEvent + for idx := 0; idx < batchSize; idx++ { + e = append(e, &v1.CloudEvent{ + Id: uuid.NewString(), + Source: "performance.benchmark.vanus", SpecVersion: "1.0", - Type: "example.type", - Data: &v1.CloudEvent_TextData{TextData: genStr(rd, payloadSize)}, - }} - }) + Type: "performance.benchmark.vanus", + Data: &v1.CloudEvent_TextData{TextData: payload}, + Attributes: map[string]*v1.CloudEvent_CloudEventAttributeValue{ + "time": { + Attr: &v1.CloudEvent_CloudEventAttributeValue_CeTimestamp{ + CeTimestamp: timestamppb.New(time.Now()), + }, + }, + }, + }) + } return e } diff --git a/test/benchmark/command/performance.go b/test/benchmark/command/performance.go index 23e899d42..02c93b632 100644 --- a/test/benchmark/command/performance.go +++ b/test/benchmark/command/performance.go @@ -19,6 +19,10 @@ import ( "encoding/json" "errors" "fmt" + "github.com/linkall-labs/vanus/proto/pkg/cloudevents" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/protobuf/types/known/emptypb" "math/rand" "net" "net/http" @@ -31,8 +35,6 @@ import ( "github.com/HdrHistogram/hdrhistogram-go" ce "github.com/cloudevents/sdk-go/v2" - "github.com/cloudevents/sdk-go/v2/client" - "github.com/cloudevents/sdk-go/v2/protocol" cehttp "github.com/cloudevents/sdk-go/v2/protocol/http" "github.com/fatih/color" "github.com/go-redis/redis/v8" @@ -52,18 +54,16 @@ const ( ) var ( - name string eventbusList []string number int64 parallelism int payloadSize int - port int - benchType string + port int + benchType string + clientProtocol string ) -var ebCh = make(chan string, 1024) - func E2ECommand() *cobra.Command { cmd := &cobra.Command{ Use: "e2e SUB-COMMAND", @@ -80,8 +80,6 @@ func runCommand() *cobra.Command { Use: "run SUB-COMMAND", Short: "vanus performance benchmark program", Run: func(cmd *cobra.Command, args []string) { - endpoint := mustGetGatewayEndpoint(cmd) - if len(eventbusList) == 0 { panic("eventbus list is empty") } @@ -90,105 +88,212 @@ func runCommand() *cobra.Command { "id": getBenchmarkID(), }) - // start - start := time.Now() - cnt := int64(0) + if clientProtocol == "grpc" { + sendWithGRPC(cmd) + } else { + sendWithHTTP(cmd) + } + }, + } + cmd.Flags().StringArrayVar(&eventbusList, "eventbus", []string{}, "the eventbus name used to") + cmd.Flags().Int64Var(&number, "number", 100000, "the event number") + cmd.Flags().IntVar(¶llelism, "parallelism", 1, "") + cmd.Flags().IntVar(&payloadSize, "payload-size", 64, "byte") + cmd.Flags().StringVar(&clientProtocol, "protocol", "grpc", "") + cmd.Flags().IntVar(&batchSize, "batch-size", 1, "") + return cmd +} + +func sendWithGRPC(cmd *cobra.Command) { + endpoint := mustGetGatewayEndpoint(cmd) + + // start + start := time.Now() + ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second) + defer cancel() + + opts := []grpc.DialOption{ + grpc.WithBlock(), + grpc.WithTransportCredentials(insecure.NewCredentials()), + } + conn, err := grpc.DialContext(ctx, endpoint, opts...) + if err != nil { + cmdFailedf(cmd, "failed to connect to gateway") + } + + batchClient := cloudevents.NewCloudEventsClient(conn) + + var success int64 + wg := sync.WaitGroup{} + latency := hdrhistogram.New(1, 1000000, 10000) + for _, eb := range eventbusList { + for idx := 0; idx < parallelism; idx++ { + wg.Add(1) go func() { - for atomic.LoadInt64(&cnt) < number { - for idx := 0; idx < len(eventbusList); idx++ { - ebCh <- eventbusList[idx] - atomic.AddInt64(&cnt, 1) + for atomic.LoadInt64(&success) < number { + s := time.Now() + events := generateEvents() + _, err := batchClient.Send(context.Background(), &cloudevents.BatchEvent{ + EventbusName: eb, + Events: &cloudevents.CloudEventBatch{Events: events}, + }) + if err != nil { + log.Warning(context.Background(), "failed to send events", map[string]interface{}{ + log.KeyError: err, + }) + } else { + atomic.AddInt64(&success, int64(len(events))) + if err := latency.RecordValue(time.Now().Sub(s).Microseconds()); err != nil { + panic(err) + } } } - close(ebCh) - log.Info(context.Background(), "all events were made", map[string]interface{}{ - "num": number, - }) + wg.Done() }() + } + } - p, err := ce.NewHTTP() - if err != nil { - cmdFailedf(cmd, "init ce protocol error: %s\n", err) - } - c, err := ce.NewClient(p, ce.WithTimeNow(), ce.WithUUIDs()) - if err != nil { - cmdFailedf(cmd, "create ce client error: %s\n", err) + ctx, can := context.WithCancel(context.Background()) + m := make(map[int]int, 0) + wg2 := sync.WaitGroup{} + wg2.Add(1) + go func() { + var prev int64 + tick := time.NewTicker(time.Second) + c := 1 + defer func() { + tick.Stop() + tps := success - prev + log.Info(nil, fmt.Sprintf("Sent: %d, TPS: %d\n", success, tps), nil) + m[c] = int(tps) + wg2.Done() + }() + for prev < number { + select { + case <-tick.C: + cur := atomic.LoadInt64(&success) + tps := cur - prev + m[c] = int(tps) + log.Info(nil, fmt.Sprintf("Sent: %d, TPS: %d\n", cur, tps), nil) + prev = cur + c++ + case <-ctx.Done(): + return } + } + }() + wg.Wait() + can() + wg2.Wait() + saveTPS(m, "produce") + res := latency.CumulativeDistribution() + unit := "us" + result := map[string]map[string]interface{}{} + for _, v := range res { + if v.Count == 0 { + continue + } - var success int64 - wg := sync.WaitGroup{} - for idx := 0; idx < parallelism; idx++ { - wg.Add(1) - go func() { - for { - eb, ok := <-ebCh - if !ok && eb == "" { - break - } - var target string - if strings.HasPrefix(endpoint, httpPrefix) { - target = fmt.Sprintf("%s/gateway/%s", endpoint, eb) - } else { - target = fmt.Sprintf("%s%s/gateway/%s", httpPrefix, endpoint, eb) - } - r, e := send(c, target) - if e != nil { - panic(e) - } - if r { - atomic.AddInt64(&success, 1) - } - } - wg.Done() - }() - } + result[fmt.Sprintf("%.2f", v.Quantile)] = map[string]interface{}{ + "value": v.ValueAt, + "unit": unit, + "count": v.Count, + } + fmt.Printf("%.2f pct - %d %s, count: %d\n", v.Quantile, v.ValueAt, unit, v.Count) + } + + fmt.Printf("Total: %d\n", latency.TotalCount()) + fmt.Printf("Latency Mean: %.2f %s\n", latency.Mean(), unit) + fmt.Printf("Latency StdDev: %.2f\n", latency.StdDev()) + fmt.Printf("Latency Max: %d %s, Latency Min: %d %s\n", latency.Max(), unit, latency.Min(), "ms") + fmt.Println() + log.Info(nil, "all message were sent", map[string]interface{}{ + "success": success, + "failed": number - success, + "used": time.Now().Sub(start), + }) + _ = rdb.Close() +} - ctx, can := context.WithCancel(context.Background()) - m := make(map[int]int, 0) - wg2 := sync.WaitGroup{} - wg2.Add(1) +func sendWithHTTP(cmd *cobra.Command) { + endpoint := mustGetGatewayEndpoint(cmd) + + // start + start := time.Now() + p, err := ce.NewHTTP() + if err != nil { + cmdFailedf(cmd, "init ce protocol error: %s\n", err) + } + c, err := ce.NewClient(p, ce.WithTimeNow(), ce.WithUUIDs()) + if err != nil { + cmdFailedf(cmd, "create ce client error: %s\n", err) + } + + var success int64 + wg := sync.WaitGroup{} + for _, eb := range eventbusList { + for idx := 0; idx < parallelism; idx++ { + wg.Add(1) go func() { - var prev int64 - tick := time.NewTicker(time.Second) - c := 1 - defer func() { - tick.Stop() - tps := success - prev - log.Info(nil, fmt.Sprintf("Sent: %d, TPS: %d\n", success, tps), nil) - m[c] = int(tps) - wg2.Done() - }() - for prev < number { - select { - case <-tick.C: - cur := atomic.LoadInt64(&success) - tps := cur - prev - m[c] = int(tps) - log.Info(nil, fmt.Sprintf("Sent: %d, TPS: %d\n", cur, tps), nil) - prev = cur - c++ - case <-ctx.Done(): - return + for atomic.LoadInt64(&success) < number { + var target string + if strings.HasPrefix(endpoint, httpPrefix) { + target = fmt.Sprintf("%s/gateway/%s", endpoint, eb) + } else { + target = fmt.Sprintf("%s%s/gateway/%s", httpPrefix, endpoint, eb) + } + r, e := send(c, target) + if e != nil { + panic(e) + } + if r { + atomic.AddInt64(&success, 1) } } + wg.Done() }() - wg.Wait() - can() - wg2.Wait() - saveTPS(m, "produce") - log.Info(nil, "all message were sent", map[string]interface{}{ - "success": success, - "failed": number - success, - "used": time.Now().Sub(start), - }) - _ = rdb.Close() - }, + } } - cmd.Flags().StringArrayVar(&eventbusList, "eventbus", []string{}, "the eventbus name used to") - cmd.Flags().Int64Var(&number, "number", 100000, "the event number") - cmd.Flags().IntVar(¶llelism, "parallelism", 1, "") - cmd.Flags().IntVar(&payloadSize, "payload-size", 64, "byte") - return cmd + + ctx, can := context.WithCancel(context.Background()) + m := make(map[int]int, 0) + wg2 := sync.WaitGroup{} + wg2.Add(1) + go func() { + var prev int64 + tick := time.NewTicker(time.Second) + c := 1 + defer func() { + tick.Stop() + tps := success - prev + log.Info(nil, fmt.Sprintf("Sent: %d, TPS: %d\n", success, tps), nil) + m[c] = int(tps) + wg2.Done() + }() + for prev < number { + select { + case <-tick.C: + cur := atomic.LoadInt64(&success) + tps := cur - prev + m[c] = int(tps) + log.Info(nil, fmt.Sprintf("Sent: %d, TPS: %d\n", cur, tps), nil) + prev = cur + c++ + case <-ctx.Done(): + return + } + } + }() + wg.Wait() + can() + wg2.Wait() + saveTPS(m, "produce") + log.Info(nil, "all message were sent", map[string]interface{}{ + "success": success, + "failed": number - success, + "used": time.Now().Sub(start), + }) + _ = rdb.Close() } func saveTPS(m map[int]int, t string) { @@ -243,15 +348,18 @@ func receiveCommand() *cobra.Command { cmdFailedf(cmd, "init network error: %s", err) } - c, err := client.NewHTTP(cehttp.WithListener(ls), cehttp.WithRequestDataAtContextMiddleware()) - if err != nil { - cmdFailedf(cmd, "init ce http error: %s", err) - } + grpcServer := grpc.NewServer() + + cloudevents.RegisterCloudEventsServer(grpcServer, &testReceiver{}) + log.Info(context.TODO(), fmt.Sprintf("the receiver ready to work at %d", port), map[string]interface{}{ "benchmark_id": getBenchmarkID(), }) - if err := c.StartReceiver(context.Background(), receive); err != nil { - cmdFailedf(cmd, "start cloudevents receiver error: %s", err) + err = grpcServer.Serve(ls) + if err != nil { + log.Error(nil, "grpc server occurred an error", map[string]interface{}{ + log.KeyError: err, + }) } }, } @@ -259,6 +367,20 @@ func receiveCommand() *cobra.Command { return cmd } +type testReceiver struct{} + +func (t testReceiver) Send(ctx context.Context, event *cloudevents.BatchEvent) (*emptypb.Empty, error) { + for idx := range event.Events.GetEvents() { + e := event.Events.GetEvents()[idx] + attr := e.GetAttributes()["time"] + + if err := receive(ctx, e.Id, attr.GetCeTimestamp().AsTime()); err != nil { + return nil, err + } + } + return &emptypb.Empty{}, nil +} + func analyseCommand() *cobra.Command { cmd := &cobra.Command{ Use: "analyse", @@ -295,7 +417,6 @@ func analyseCommand() *cobra.Command { r := &BenchmarkResult{ ID: primitive.NewObjectID(), TaskID: taskID, - CaseName: name, RType: ResultLatency, Values: result, Mean: his.Mean(), @@ -339,7 +460,6 @@ func analyseCommand() *cobra.Command { r = &BenchmarkResult{ ID: primitive.NewObjectID(), TaskID: taskID, - CaseName: name, RType: ResultThroughput, Values: result, Mean: tps.Mean(), @@ -399,10 +519,13 @@ func analyseCommand() *cobra.Command { return cmd } -var receiveOnce = sync.Once{} -var consumingCnt = int64(0) +var ( + receiveOnce = sync.Once{} + consumingCnt = int64(0) + totalTime = int64(0) +) -func receive(_ context.Context, event ce.Event) protocol.Result { +func receive(_ context.Context, id string, t time.Time) error { receiveOnce.Do(func() { prev := int64(0) go func() { @@ -410,20 +533,21 @@ func receive(_ context.Context, event ce.Event) protocol.Result { cur := atomic.LoadInt64(&consumingCnt) tps := cur - prev prev = cur - log.Info(nil, fmt.Sprintf("Received: %d, TPS: %d\n", cur, tps), nil) + log.Info(nil, fmt.Sprintf("Received: %d, TPS: %d, Average Latency: %d us\n", cur, tps, + atomic.LoadInt64(&totalTime)/atomic.LoadInt64(&consumingCnt)), nil) time.Sleep(time.Second) } }() }) - event.SetExtension(eventReceivedAt, time.Now()) + atomic.AddInt64(&totalTime, time.Now().Sub(t).Microseconds()) r := &Record{ - ID: event.ID(), - BornAt: event.Time(), + ID: id, + BornAt: t, ReceivedAt: time.Now(), } cache(r, "receive") atomic.AddInt64(&consumingCnt, 1) - return ce.ResultACK + return nil } func isOutputFormatJSON(cmd *cobra.Command) bool { @@ -435,14 +559,14 @@ func isOutputFormatJSON(cmd *cobra.Command) bool { } func cache(r *Record, key string) { - key = path.Join(redisKey, key, getBenchmarkID()) - data, _ := json.Marshal(r) - cmd := rdb.LPush(context.Background(), key, data) - if cmd.Err() != nil { - log.Warning(context.Background(), "set event to redis failed", map[string]interface{}{ - log.KeyError: cmd.Err(), - }) - } + //key = path.Join(redisKey, key, getBenchmarkID()) + //data, _ := json.Marshal(r) + //cmd := rdb.LPush(context.Background(), key, data) + //if cmd.Err() != nil { + // log.Warning(context.Background(), "set event to redis failed", map[string]interface{}{ + // log.KeyError: cmd.Err(), + // }) + //} } func analyseProduction(ch <-chan *Record, f func(his *hdrhistogram.Histogram, unit string)) { @@ -533,6 +657,13 @@ func cmdFailedf(cmd *cobra.Command, format string, a ...interface{}) { os.Exit(-1) } +var ( + tmpID = uuid.NewString() +) + func getBenchmarkID() string { + if taskID.IsZero() { + return tmpID + } return taskID.Hex() } diff --git a/test/benchmark/main.go b/test/benchmark/main.go index 52a50b297..e629cac79 100644 --- a/test/benchmark/main.go +++ b/test/benchmark/main.go @@ -53,10 +53,6 @@ func main() { rootCmd.AddCommand(command.E2ECommand()) rootCmd.AddCommand(command.ComponentCommand()) rootCmd.PersistentPreRun = func(_ *cobra.Command, _ []string) { - if !caseNames[name] { - panic("invalid case name: " + name) - } - command.SetCaseName(name) command.InitDatabase(redisAddr, fmt.Sprintf(defaultMongoDBURI, mongodbPass), begin, withMongoDB) } rootCmd.PersistentPostRun = func(_ *cobra.Command, _ []string) { diff --git a/test/infra/Dockerfile b/test/infra/Dockerfile index 6f9cdba49..99e2d3656 100644 --- a/test/infra/Dockerfile +++ b/test/infra/Dockerfile @@ -2,7 +2,7 @@ FROM golang:1.18 as builder WORKDIR /workspace COPY . . -RUN go mod download +RUN go mod tidy RUN GOOS=linux GOARCH=amd64 go build -o bin/vanus-bench ./test/benchmark @@ -14,10 +14,7 @@ COPY --from=builder /workspace/bin/vanus-bench /usr/bin/vanus-bench WORKDIR /vanus-bench RUN apt-get update && apt-get install -y curl -RUN curl -O https://download.linkall.com/vsctl/v0.5.1/linux-amd64/vsctl && \ +RUN curl -O https://download.linkall.com/vsctl/v0.5.4/linux-amd64/vsctl && \ mv vsctl /usr/bin/vsctl RUN chmod a+x /usr/bin/vsctl -RUN chmod a+x /usr/bin/vanus-bench -RUN chmod a+x /vanus-bench/run.sh - -ENTRYPOINT ["/vanus-bench/run.sh"] \ No newline at end of file +RUN chmod a+x /usr/bin/vanus-bench \ No newline at end of file diff --git a/test/infra/benchmark.yml b/test/infra/benchmark.yml index 31b5ca8a0..ad25058ad 100644 --- a/test/infra/benchmark.yml +++ b/test/infra/benchmark.yml @@ -1,7 +1,7 @@ apiVersion: v1 kind: Namespace metadata: - name: vanus-bench + name: vanus --- apiVersion: apps/v1 kind: Deployment @@ -9,7 +9,7 @@ metadata: labels: app: redis name: redis - namespace: vanus-bench + namespace: vanus spec: replicas: 1 selector: @@ -31,7 +31,7 @@ metadata: labels: app: redis name: redis - namespace: vanus-bench + namespace: vanus spec: ports: - port: 6379 diff --git a/test/infra/case1/job-a.yml b/test/infra/case1/job-a.yml index d5d7b7300..480aacdc2 100644 --- a/test/infra/case1/job-a.yml +++ b/test/infra/case1/job-a.yml @@ -4,7 +4,7 @@ metadata: labels: app: vanus-bench-case1-a name: vanus-bench-case1-a - namespace: vanus-bench + namespace: vanus spec: parallelism: 1 completions: 1 @@ -17,7 +17,7 @@ spec: spec: restartPolicy: Never containers: - - image: linkall.tencentcloudcr.com/vanus/test-infra:dev + - image: public.ecr.aws/vanus/test-infra:dev imagePullPolicy: Always name: vanus-bench-case1-a env: @@ -30,7 +30,7 @@ spec: - name: VANUS_GATEWAY value: "vanus-gateway.vanus:8080" - name: REDIS_ADDR - value: "redis.vanus-bench:6379" + value: "redis.vanus:6379" - name: MONGODB_PASSWORD valueFrom: secretKeyRef: diff --git a/test/infra/case1/job-b.yml b/test/infra/case1/job-b.yml index a67307088..e7e995dc0 100644 --- a/test/infra/case1/job-b.yml +++ b/test/infra/case1/job-b.yml @@ -4,7 +4,7 @@ metadata: labels: app: vanus-bench-case1-b name: vanus-bench-case1-b - namespace: vanus-bench + namespace: vanus spec: parallelism: 1 completions: 1 @@ -17,7 +17,7 @@ spec: spec: restartPolicy: Never containers: - - image: linkall.tencentcloudcr.com/vanus/test-infra:dev + - image: public.ecr.aws/vanus/test-infra:dev imagePullPolicy: Always name: vanus-bench-case1-b env: @@ -30,7 +30,7 @@ spec: - name: VANUS_GATEWAY value: "vanus-gateway.vanus:8080" - name: REDIS_ADDR - value: "redis.vanus-bench:6379" + value: "redis.vanus:6379" - name: MONGODB_PASSWORD valueFrom: secretKeyRef: diff --git a/test/infra/case1/job-c.yml b/test/infra/case1/job-c.yml index 1cfd20e18..fcefc0d6f 100644 --- a/test/infra/case1/job-c.yml +++ b/test/infra/case1/job-c.yml @@ -4,12 +4,12 @@ metadata: labels: app: vanus-bench-case1-c name: vanus-bench-case1-c - namespace: vanus-bench + namespace: vanus spec: parallelism: 1 completions: 1 backoffLimit: 3 - activeDeadlineSeconds: 3600 + activeDeadlineSeconds: 10000 template: metadata: labels: @@ -17,9 +17,10 @@ spec: spec: restartPolicy: Never containers: - - image: linkall.tencentcloudcr.com/vanus/test-infra:dev + - image: public.ecr.aws/vanus/test-infra:dev imagePullPolicy: Always name: vanus-bench-case1-c + command: ["bash", "-c", "sleep 10000"] env: - name: CASE_NAME value: "case1" @@ -30,7 +31,7 @@ spec: - name: VANUS_GATEWAY value: "vanus-gateway.vanus:8080" - name: REDIS_ADDR - value: "redis.vanus-bench:6379" + value: "redis.vanus:6379" - name: MONGODB_PASSWORD valueFrom: secretKeyRef: diff --git a/test/infra/case2/play.sh b/test/infra/case2/play.sh index 5f5a4c21a..d5ec116ee 100644 --- a/test/infra/case2/play.sh +++ b/test/infra/case2/play.sh @@ -10,10 +10,10 @@ nohup vanus-bench component store run --volume-id 3 --name e2e-component-store-3 # The maximum TPS between 12K to 14K with high shake, and TPS isn't sensitive with payload-size [10B, 1024B] # BUG: it need to wait a seconds to make sure write successfully after a Segment is activated -# vanus-bench component store create-block --name e2e-component-store-1replicas --without-mongodb --replicas 1 --block-size 512 --store-address 127.0.0.1:2149 --number 24 +# vanus-bench component store create-block --name e2e-component-store-1replicas --without-mongodb --replicas 1 --block-size 512 --store-address 127.0.0.1:2149 --number 16 # vanus-bench component store send --name e2e-component-store-1replicas --without-mongodb --no-clean-cache --total-number 1000000 --parallelism 1 --payload-size 1024 # vanus-bench component store create-block --name e2e-component-store-3replicas --without-mongodb --replicas 3 --block-size 512 --number 16 -# vanus-bench component store send --total-number 100000 --parallelism 16 --no-clean-cache --payload-size 10240 --name e2e-component-store-3replicas --without-mongodb +# vanus-bench component store send --total-number 1000000 --parallelism 16 --no-clean-cache --payload-size 1024 --name e2e-component-store-3replicas --without-mongodb --batch-size 32 # ps -ef | grep bench | grep -v "auto" | awk '{print $2}' | xargs kill && rm -rf /Users/wenfeng/tmp/data/test/vanus && rm *.log # redis-cli LTRIM /vanus/test/store/block_records 1 0 \ No newline at end of file diff --git a/test/infra/consumer/consumer.yml b/test/infra/consumer/consumer.yml new file mode 100644 index 000000000..d9c66a444 --- /dev/null +++ b/test/infra/consumer/consumer.yml @@ -0,0 +1,51 @@ +apiVersion: v1 +kind: Service +metadata: + name: vanus-benchmark-consumer + namespace: vanus +spec: + ports: + - name: receiver + port: 8080 + targetPort: 8080 + selector: + app: vanus-benchmark-consumer + type: ClusterIP +--- +apiVersion: apps/v1 +kind: Deployment +metadata: + labels: + app: vanus-benchmark-consumer + name: vanus-benchmark-consumer + namespace: vanus +spec: + replicas: 1 + selector: + matchLabels: + app: vanus-benchmark-consumer + template: + metadata: + annotations: + vanus.dev/metrics.port: "2112" + labels: + app: vanus-benchmark-consumer + spec: + containers: + - image: public.ecr.aws/vanus/test-infra:dev + imagePullPolicy: Always + name: benchmark-consumer +# command: ["sh", "/vanus-bench/consumer/play.sh"] + command: ["bash", "-c", "sleep 1000000"] + env: + - name: JOB_NAME + value: "e2e-benchmark-consumer" + - name: VANUS_GATEWAY + value: "vanus-gateway.vanus:8080" + - name: REDIS_ADDR + value: "redis.vanus:6379" + - name: MONGODB_PASSWORD + valueFrom: + secretKeyRef: + name: benchmark-credentials + key: mongodb_password \ No newline at end of file diff --git a/test/infra/consumer/play.sh b/test/infra/consumer/play.sh new file mode 100644 index 000000000..145edd0c9 --- /dev/null +++ b/test/infra/consumer/play.sh @@ -0,0 +1,7 @@ +#!/bin/bash +set -ex + +vanus-bench e2e receive \ + --name "${JOB_NAME}" \ + --redis-addr "${REDIS_ADDR}" \ + --mongodb-password "${MONGODB_PASSWORD}" \ No newline at end of file diff --git a/test/infra/producer/play.sh b/test/infra/producer/play.sh new file mode 100644 index 000000000..05cec917b --- /dev/null +++ b/test/infra/producer/play.sh @@ -0,0 +1,14 @@ +#!/bin/bash +set -ex + +vanus-bench e2e run \ + --name "${JOB_NAME}" \ + --eventbus "${JOB_NAME}" \ + --number "${TOTAL_NUMBER}" \ + --parallelism "${PARALLELISM}" \ + --endpoint "${VANUS_GATEWAY}" \ + --payload-size "${PAYLOAD_SIZE}" \ + --redis-addr "${REDIS_ADDR}" \ + --mongodb-password "${MONGODB_PASSWORD}" \ + --batch-size "${BATCH_SIZE}" \ + --begin diff --git a/test/infra/producer/producer.yml b/test/infra/producer/producer.yml new file mode 100644 index 000000000..ed2624f6d --- /dev/null +++ b/test/infra/producer/producer.yml @@ -0,0 +1,47 @@ +apiVersion: apps/v1 +kind: Deployment +metadata: + labels: + app: vanus-benchmark-producer + name: vanus-benchmark-producer + namespace: vanus +spec: + replicas: 1 + selector: + matchLabels: + app: vanus-benchmark-producer + template: + metadata: + annotations: + vanus.dev/metrics.port: "2112" + labels: + app: vanus-benchmark-producer + spec: + containers: + - image: public.ecr.aws/vanus/test-infra:dev + imagePullPolicy: Always + name: benchmark-producer +# command: ["sh", "/vanus-bench/producer/play.sh"] + command: ["bash", "-c", "sleep 1000000"] + env: + - name: JOB_NAME + value: "e2e-benchmark-producer" + - name: PAYLOAD_SIZE + value: "1024" + - name: VANUS_GATEWAY + value: "vanus-gateway.vanus:8080" + - name: REDIS_ADDR + value: "redis.vanus:6379" + - name: MONGODB_PASSWORD + valueFrom: + secretKeyRef: + name: benchmark-credentials + key: mongodb_password + - name: EVENTLOG_NUMBER + value: "16" + - name: PARALLELISM + value: "16" + - name: BATCH_SIZE + value: "16" + - name: TOTAL_NUMBER + value: "1000000" \ No newline at end of file diff --git a/test/infra/run.sh b/test/infra/run.sh deleted file mode 100644 index 91d948ede..000000000 --- a/test/infra/run.sh +++ /dev/null @@ -1,4 +0,0 @@ -#!/bin/bash -set -ex - -sh /vanus-bench/"${CASE_NAME}"/play.sh \ No newline at end of file diff --git a/test/infra/sc.yml b/test/infra/sc.yml new file mode 100644 index 000000000..8b110c245 --- /dev/null +++ b/test/infra/sc.yml @@ -0,0 +1,259 @@ +apiVersion: storage.k8s.io/v1 +kind: StorageClass +metadata: + name: local-storage + namespace: vanus +provisioner: kubernetes.io/no-provisioner +volumeBindingMode: WaitForFirstConsumer +--- +apiVersion: v1 +kind: PersistentVolume +metadata: + name: controller-pv1 + namespace: vanus +spec: + capacity: + storage: 20Gi + accessModes: + - ReadWriteOnce + persistentVolumeReclaimPolicy: Retain + storageClassName: local-storage + local: + path: /var/lib/docker/data/volumes/controller-pv1 + nodeAffinity: + required: + nodeSelectorTerms: + - matchExpressions: + - key: kubernetes.io/hostname + operator: In + values: + - 10.0.32.6 +--- +apiVersion: v1 +kind: PersistentVolume +metadata: + name: controller-pv2 + namespace: vanus +spec: + capacity: + storage: 20Gi + accessModes: + - ReadWriteOnce + persistentVolumeReclaimPolicy: Retain + storageClassName: local-storage + local: + path: /var/lib/docker/data/volumes/controller-pv2 + nodeAffinity: + required: + nodeSelectorTerms: + - matchExpressions: + - key: kubernetes.io/hostname + operator: In + values: + - 10.0.32.6 +--- +apiVersion: v1 +kind: PersistentVolume +metadata: + name: controller-pv3 + namespace: vanus +spec: + capacity: + storage: 20Gi + accessModes: + - ReadWriteOnce + persistentVolumeReclaimPolicy: Retain + storageClassName: local-storage + local: + path: /var/lib/docker/data/volumes/controller-pv3 + nodeAffinity: + required: + nodeSelectorTerms: + - matchExpressions: + - key: kubernetes.io/hostname + operator: In + values: + - 10.0.32.6 +--- +kind: PersistentVolumeClaim +apiVersion: v1 +metadata: + name: data-vanus-controller-0 + namespace: vanus +spec: + accessModes: + - ReadWriteOnce + storageClassName: local-storage + resources: + requests: + storage: 20Gi +--- +kind: PersistentVolumeClaim +apiVersion: v1 +metadata: + name: data-vanus-controller-1 + namespace: vanus +spec: + accessModes: + - ReadWriteOnce + storageClassName: local-storage + resources: + requests: + storage: 20Gi +--- +kind: PersistentVolumeClaim +apiVersion: v1 +metadata: + name: data-vanus-controller-2 + namespace: vanus +spec: + accessModes: + - ReadWriteOnce + storageClassName: local-storage + resources: + requests: + storage: 20Gi +--- +apiVersion: v1 +kind: PersistentVolume +metadata: + name: store-pv1 + namespace: vanus +spec: + capacity: + storage: 200Gi + accessModes: + - ReadWriteOnce + persistentVolumeReclaimPolicy: Retain + storageClassName: local-storage + local: + path: /var/lib/docker/data/volumes/store-pv1 + nodeAffinity: + required: + nodeSelectorTerms: + - matchExpressions: + - key: kubernetes.io/hostname + operator: In + values: + - 10.0.32.6 +--- +apiVersion: v1 +kind: PersistentVolume +metadata: + name: store-pv2 + namespace: vanus +spec: + capacity: + storage: 200Gi + accessModes: + - ReadWriteOnce + persistentVolumeReclaimPolicy: Retain + storageClassName: local-storage + local: + path: /var/lib/docker/data/volumes/store-pv2 + nodeAffinity: + required: + nodeSelectorTerms: + - matchExpressions: + - key: kubernetes.io/hostname + operator: In + values: + - 10.0.32.6 +--- +apiVersion: v1 +kind: PersistentVolume +metadata: + name: store-pv3 + namespace: vanus +spec: + capacity: + storage: 200Gi + accessModes: + - ReadWriteOnce + persistentVolumeReclaimPolicy: Retain + storageClassName: local-storage + local: + path: /var/lib/docker/data/volumes/store-pv3 + nodeAffinity: + required: + nodeSelectorTerms: + - matchExpressions: + - key: kubernetes.io/hostname + operator: In + values: + - 10.0.32.6 +--- +apiVersion: v1 +kind: PersistentVolume +metadata: + name: store-pv4 + namespace: vanus +spec: + capacity: + storage: 200Gi + accessModes: + - ReadWriteOnce + persistentVolumeReclaimPolicy: Retain + storageClassName: local-storage + local: + path: /var/lib/docker/data/volumes/store-pv4 + nodeAffinity: + required: + nodeSelectorTerms: + - matchExpressions: + - key: kubernetes.io/hostname + operator: In + values: + - 10.0.32.6 +--- +kind: PersistentVolumeClaim +apiVersion: v1 +metadata: + name: data-vanus-store-0 + namespace: vanus +spec: + accessModes: + - ReadWriteOnce + storageClassName: local-storage + resources: + requests: + storage: 200Gi +--- +kind: PersistentVolumeClaim +apiVersion: v1 +metadata: + name: data-vanus-store-1 + namespace: vanus +spec: + accessModes: + - ReadWriteOnce + storageClassName: local-storage + resources: + requests: + storage: 200Gi +--- +kind: PersistentVolumeClaim +apiVersion: v1 +metadata: + name: data-vanus-store-2 + namespace: vanus +spec: + accessModes: + - ReadWriteOnce + storageClassName: local-storage + resources: + requests: + storage: 200Gi +--- +kind: PersistentVolumeClaim +apiVersion: v1 +metadata: + name: data-vanus-store-3 + namespace: vanus +spec: + accessModes: + - ReadWriteOnce + storageClassName: local-storage + resources: + requests: + storage: 200Gi diff --git a/test/infra/secret.yml.example b/test/infra/secret.yml.example index 5440afa23..d4ea9051b 100644 --- a/test/infra/secret.yml.example +++ b/test/infra/secret.yml.example @@ -2,7 +2,7 @@ apiVersion: v1 kind: Secret metadata: name: benchmark-credentials - namespace: vanus-bench + namespace: vanus type: Opaque data: # echo "your_mongodb_password" | base64 diff --git a/test/infra/vanus.yml b/test/infra/vanus.yml index 6b80d0bc7..98801b6a5 100644 --- a/test/infra/vanus.yml +++ b/test/infra/vanus.yml @@ -1,9 +1,4 @@ apiVersion: v1 -kind: Namespace -metadata: - name: vanus ---- -apiVersion: v1 data: controller.yaml: |- node_id: ${NODE_ID} @@ -15,6 +10,7 @@ data: - vanus-controller-1.vanus-controller:2379 - vanus-controller-2.vanus-controller:2379 data_dir: /data + segment_capacity: 268435456 replicas: 3 metadata: key_prefix: /vanus @@ -74,8 +70,16 @@ data: engine: psync raft: wal: + block_size: 16384 io: engine: psync + parallel: 16 + vsb: + flush_batch_size: 16384 + io: + engine: psync + parallel: 16 + kind: ConfigMap metadata: name: config-store @@ -165,7 +169,7 @@ metadata: name: vanus-gateway namespace: vanus spec: - replicas: 1 + replicas: 3 selector: matchLabels: app: vanus-gateway @@ -177,16 +181,21 @@ spec: app: vanus-gateway spec: containers: - - image: linkall.tencentcloudcr.com/vanus/gateway:dev + - image: public.ecr.aws/vanus/gateway:2621ed7 imagePullPolicy: IfNotPresent name: gateway + resources: + limits: + cpu: 2000m + memory: 8000Mi + requests: + cpu: 2000m + memory: 8000Mi ports: - containerPort: 8080 - name: httpput + name: proxy - containerPort: 8081 - name: httpget - - containerPort: 8082 - name: ctrl-proxy + name: cloudevents volumeMounts: - mountPath: /vanus/config name: config-gateway @@ -203,7 +212,7 @@ metadata: name: vanus-timer namespace: vanus spec: - replicas: 2 + replicas: 1 selector: matchLabels: app: vanus-timer @@ -222,7 +231,7 @@ spec: valueFrom: fieldRef: fieldPath: status.podIP - image: linkall.tencentcloudcr.com/vanus/timer:dev + image: public.ecr.aws/vanus/timer:2621ed7 imagePullPolicy: IfNotPresent name: timer volumeMounts: @@ -252,17 +261,31 @@ spec: labels: app: vanus-trigger spec: + initContainers: + - name: set-system-parameters + image: busybox + securityContext: + capabilities: { } + privileged: true + command: ["/bin/sh", "-c", "sysctl -w net.ipv4.tcp_tw_reuse=1 && sysctl -w net.ipv4.tcp_fin_timeout=10 && sysctl -w net.ipv4.ip_local_port_range='4096 65000' && sysctl -p"] containers: - env: - name: VANUS_LOG_LEVEL - value: DEBUG + value: INFO - name: POD_IP valueFrom: fieldRef: fieldPath: status.podIP - image: linkall.tencentcloudcr.com/vanus/trigger:dev + image: public.ecr.aws/vanus/trigger:2621ed7 imagePullPolicy: IfNotPresent name: trigger + resources: + limits: + cpu: 8000m + memory: 8000Mi + requests: + cpu: 8000m + memory: 8000Mi ports: - containerPort: 2148 name: grpc @@ -310,9 +333,16 @@ spec: valueFrom: fieldRef: fieldPath: status.podIP - image: linkall.tencentcloudcr.com/vanus/controller:dev + image: public.ecr.aws/vanus/controller:2621ed7 imagePullPolicy: IfNotPresent name: controller + resources: + limits: + cpu: 1000m + memory: 4000Mi + requests: + cpu: 1000m + memory: 4000Mi ports: - containerPort: 2048 name: grpc @@ -339,7 +369,8 @@ spec: spec: accessModes: - ReadWriteOnce - storageClassName: ssd +# storageClassName: ssd + storageClassName: local-storage resources: requests: storage: 20Gi @@ -352,7 +383,7 @@ metadata: name: vanus-store namespace: vanus spec: - replicas: 3 + replicas: 4 selector: matchLabels: app: vanus-store @@ -371,14 +402,21 @@ spec: - VOLUME_ID=${HOSTNAME##*-} /vanus/bin/store env: - name: VANUS_LOG_LEVEL - value: DEBUG + value: INFO - name: POD_IP valueFrom: fieldRef: fieldPath: status.podIP - image: linkall.tencentcloudcr.com/vanus/store:dev + image: public.ecr.aws/vanus/store:2621ed7 imagePullPolicy: IfNotPresent name: store + resources: + limits: + cpu: 4000m + memory: 8000Mi + requests: + cpu: 4000m + memory: 8000Mi ports: - containerPort: 11811 name: grpc @@ -399,7 +437,8 @@ spec: spec: accessModes: - ReadWriteOnce - storageClassName: ssd +# storageClassName: ssd + storageClassName: local-storage resources: requests: - storage: 20Gi + storage: 200Gi From 1ea3797a1c30c632ca985132962a693c8b6d1fe2 Mon Sep 17 00:00:00 2001 From: James Yin Date: Thu, 5 Jan 2023 16:58:00 +0800 Subject: [PATCH 5/9] fix(store): recursive calls cause io worker exhaustion (#385) Signed-off-by: James Yin Signed-off-by: James Yin --- internal/store/vsb/block_append.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/store/vsb/block_append.go b/internal/store/vsb/block_append.go index 9534889dd..91d7b9880 100644 --- a/internal/store/vsb/block_append.go +++ b/internal/store/vsb/block_append.go @@ -204,7 +204,7 @@ func (b *vsBlock) CommitAppend(ctx context.Context, frag block.Fragment, cb bloc m, i := makeSnapshot(b.actx, b.indexes) - b.appendIndexEntry(ctx, i, func(n int, err error) { + go b.appendIndexEntry(ctx, i, func(n int, err error) { defer b.wg.Done() b.indexOffset = m.writeOffset b.indexLength = n From e84055c3b7c296efb6c37062ebd4e14bb0d920fe Mon Sep 17 00:00:00 2001 From: James Yin Date: Thu, 5 Jan 2023 19:08:11 +0800 Subject: [PATCH 6/9] fix(store): broken wal format (#386) Signed-off-by: James Yin Signed-off-by: James Yin --- internal/store/wal/record/packing.go | 4 ++-- internal/store/wal/record/packing_test.go | 2 +- internal/store/wal/record/record.go | 12 ++++++------ 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/internal/store/wal/record/packing.go b/internal/store/wal/record/packing.go index 04b4d5487..b8530fe9c 100644 --- a/internal/store/wal/record/packing.go +++ b/internal/store/wal/record/packing.go @@ -19,7 +19,7 @@ func Pack(entry []byte, firstSize, otherSize int) ([]Record, int) { if num == 1 { packet := makePacket(Full, entry) padding := firstSize - packet.Size() - if padding > HeaderSize { + if padding >= HeaderSize { padding = 0 } return []Record{packet}, padding @@ -43,7 +43,7 @@ func Pack(entry []byte, firstSize, otherSize int) ([]Record, int) { packets = append(packets, last) padding := otherSize - last.Size() - if padding > HeaderSize { + if padding >= HeaderSize { padding = 0 } diff --git a/internal/store/wal/record/packing_test.go b/internal/store/wal/record/packing_test.go index 0a5e1a3c2..1223beb69 100644 --- a/internal/store/wal/record/packing_test.go +++ b/internal/store/wal/record/packing_test.go @@ -41,7 +41,7 @@ func TestPack(t *testing.T) { Convey("pack with just enough space in first block", t, func() { records, padding := Pack(rawData, HeaderSize*2+len(rawData), blockSize) So(records, ShouldHaveLength, 1) - So(padding, ShouldEqual, HeaderSize) + So(padding, ShouldEqual, 0) r0 := &records[0] So(r0.Type, ShouldEqual, Full) So(r0.Length, ShouldEqual, len(rawData)) diff --git a/internal/store/wal/record/record.go b/internal/store/wal/record/record.go index 6c675f10f..08c35d4fe 100644 --- a/internal/store/wal/record/record.go +++ b/internal/store/wal/record/record.go @@ -23,15 +23,15 @@ import ( const ( crcFieldSO = 0 - crcFieldEO = crcFieldSO + 4 // [0,4) - lengthFieldSO = crcFieldEO + crcFieldEO = crcFieldSO + 4 // [0,4) + lengthFieldSO = crcFieldEO // 4 lengthFieldEO = lengthFieldSO + 2 // [4,6) - typeFieldSO = lengthFieldEO - typeFieldEO = typeFieldSO + 1 // [6,7) - dataFieldSO = typeFieldEO // [7,n) + typeFieldSO = lengthFieldEO // 6 + typeFieldEO = typeFieldSO + 1 // [6,7) + dataFieldSO = typeFieldEO // [7,n) ) -const HeaderSize = dataFieldSO +const HeaderSize = dataFieldSO // 7 var crc32q = crc32.MakeTable(crc32.Castagnoli) From 394c0e9e83eacae3a9866d12afdb78a5f6d9eeb1 Mon Sep 17 00:00:00 2001 From: James Yin Date: Thu, 5 Jan 2023 20:07:04 +0800 Subject: [PATCH 7/9] fix(store): missing data in async store (#387) Signed-off-by: James Yin Signed-off-by: James Yin --- internal/store/meta/async.go | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/internal/store/meta/async.go b/internal/store/meta/async.go index 41642c296..1d70b1d6b 100644 --- a/internal/store/meta/async.go +++ b/internal/store/meta/async.go @@ -178,28 +178,34 @@ func (s *AsyncStore) commit() { span.End() }() + s.mu.Lock() + if s.pending.Len() == 0 { + s.mu.Unlock() return } - // Write WAL. - s.mu.RLock() + // Marshal changed data. data, err := s.marshaler.Marshal(SkiplistRange(s.pending)) - s.mu.RUnlock() if err != nil { panic(err) } + + // Update state. + merge(s.committed, s.pending) + s.pending.Init() + + s.mu.Unlock() + + // Write WAL. r, err := s.wal.AppendOne(ctx, data, walog.WithoutBatching()).Wait() if err != nil { panic(err) } - // Update state. s.mu.Lock() defer s.mu.Unlock() - merge(s.committed, s.pending) s.version = r.EO - s.pending.Init() } func merge(dst, src *skiplist.SkipList) { From 303e88adbf4687285b5c08424fcff75d40c24ec3 Mon Sep 17 00:00:00 2001 From: James Yin Date: Thu, 5 Jan 2023 20:07:41 +0800 Subject: [PATCH 8/9] fix(store): break wal format after recover (#389) Signed-off-by: James Yin Signed-off-by: James Yin --- internal/store/block/raft/appender.go | 3 +-- internal/store/io/stream/scheduler.go | 1 - internal/store/io/stream/stream.go | 3 ++- internal/store/wal/wal.go | 6 ++++++ 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/internal/store/block/raft/appender.go b/internal/store/block/raft/appender.go index 2f84a226e..96c9e1a2a 100644 --- a/internal/store/block/raft/appender.go +++ b/internal/store/block/raft/appender.go @@ -277,7 +277,6 @@ func (a *appender) applyEntries(ctx context.Context, committedEntries []raftpb.E span.AddEvent("store.block.raft.appender.applyEntries() Start") defer span.AddEvent("store.block.raft.appender.applyEntries() End") - var cs *raftpb.ConfState for i := 0; i < len(committedEntries); i++ { pbEntry := &committedEntries[i] index := pbEntry.Index @@ -299,7 +298,7 @@ func (a *appender) applyEntries(ctx context.Context, committedEntries []raftpb.E } // Change membership. - cs = a.applyConfChange(ctx, pbEntry) + cs := a.applyConfChange(ctx, pbEntry) ch := make(chan struct{}) go func() { if err := a.log.SetConfState(ctx, *cs); err != nil { diff --git a/internal/store/io/stream/scheduler.go b/internal/store/io/stream/scheduler.go index e5060428c..19ef1428e 100644 --- a/internal/store/io/stream/scheduler.go +++ b/internal/store/io/stream/scheduler.go @@ -72,7 +72,6 @@ func (s *scheduler) Register(z zone.Interface, wo int64) Stream { if err := buf.RecoverFromFile(f, off, int(so)); err != nil { panic(err) } - // FIXME(james.yin): switch buf if it is full. } ss := &stream{ diff --git a/internal/store/io/stream/stream.go b/internal/store/io/stream/stream.go index ab73842f5..c5c30f994 100644 --- a/internal/store/io/stream/stream.go +++ b/internal/store/io/stream/stream.go @@ -118,9 +118,10 @@ func (s *stream) Append(r stdio.Reader, cb io.WriteCallback) { if empty { s.waiting = append(s.waiting, cb) - if last == nil && !s.dirty { + if last == nil { s.dirty = true s.startFlushTimer() + return } } diff --git a/internal/store/wal/wal.go b/internal/store/wal/wal.go index e187b4615..ef361789c 100644 --- a/internal/store/wal/wal.go +++ b/internal/store/wal/wal.go @@ -31,6 +31,7 @@ import ( "github.com/linkall-labs/vanus/internal/store/io/engine" "github.com/linkall-labs/vanus/internal/store/io/stream" "github.com/linkall-labs/vanus/internal/store/io/zone/segmentedfile" + "github.com/linkall-labs/vanus/internal/store/wal/record" ) const ( @@ -132,6 +133,11 @@ func open(ctx context.Context, dir string, cfg config) (*WAL, error) { return nil, err } + // Skip padding. + if padding := int64(cfg.blockSize) - off%int64(cfg.blockSize); padding < record.HeaderSize { + off += padding + } + scheduler := stream.NewScheduler(cfg.engine, cfg.blockSize, cfg.flushTimeout) s := scheduler.Register(sf, off) From 1395ee8899259fa1f780d8f1297d7abfee31a1b1 Mon Sep 17 00:00:00 2001 From: delu Date: Thu, 5 Jan 2023 20:10:06 +0800 Subject: [PATCH 9/9] feat: add trigger config (#388) * feat: add trigger config Signed-off-by: xdlbdy * feat: add trigger config Signed-off-by: xdlbdy * feat: add trigger config Signed-off-by: xdlbdy * feat: add trigger config Signed-off-by: xdlbdy Signed-off-by: xdlbdy --- internal/trigger/config.go | 10 +++- internal/trigger/offset/offset.go | 76 +++++++++++++++--------- internal/trigger/offset/offset_test.go | 9 +-- internal/trigger/reader/reader.go | 13 ++-- internal/trigger/reader/reader_test.go | 6 +- internal/trigger/trigger/config.go | 68 ++++++++++++++------- internal/trigger/trigger/trigger.go | 50 ++++++++-------- internal/trigger/trigger/trigger_test.go | 4 -- internal/trigger/worker.go | 6 +- 9 files changed, 151 insertions(+), 91 deletions(-) diff --git a/internal/trigger/config.go b/internal/trigger/config.go index ae930dbee..e58ed6228 100644 --- a/internal/trigger/config.go +++ b/internal/trigger/config.go @@ -30,7 +30,15 @@ type Config struct { ControllerAddr []string `yaml:"controllers"` Observability observability.Config `yaml:"observability"` - HeartbeatInterval time.Duration + HeartbeatInterval time.Duration `yaml:"heartbeat_interval"` + // send event goroutine size + SendEventGoroutineSize int `yaml:"send_event_goroutine_size"` + // push event batch size when use grpc + SendEventBatchSize int `yaml:"send_event_batch_size"` + // var client read event from segment batch size. + PullEventBatchSize int `yaml:"pull_event_batch_size"` + // max uack event number + MaxUACKEventNumber int `yaml:"max_uack_event_number"` } func InitConfig(filename string) (*Config, error) { diff --git a/internal/trigger/offset/offset.go b/internal/trigger/offset/offset.go index 7b36d0ed5..8b17c42d8 100644 --- a/internal/trigger/offset/offset.go +++ b/internal/trigger/offset/offset.go @@ -15,7 +15,6 @@ package offset import ( - "math" "sync" "github.com/huandu/skiplist" @@ -23,55 +22,82 @@ import ( "github.com/linkall-labs/vanus/internal/primitive/vanus" ) -func NewSubscriptionOffset(id vanus.ID) *SubscriptionOffset { - return &SubscriptionOffset{ +func NewSubscriptionOffset(id vanus.ID, maxUACKNumber int, initOffsets info.ListOffsetInfo) *SubscriptionOffset { + sub := &SubscriptionOffset{ subscriptionID: id, + cond: sync.NewCond(&sync.Mutex{}), + maxUACKNumber: maxUACKNumber, + elOffsets: make(map[vanus.ID]*offsetTracker, len(initOffsets)), } + for _, offset := range initOffsets { + sub.elOffsets[offset.EventLogID] = initOffset(offset.Offset) + } + return sub } type SubscriptionOffset struct { subscriptionID vanus.ID - elOffset sync.Map + cond *sync.Cond + maxUACKNumber int + uACKNumber int + elOffsets map[vanus.ID]*offsetTracker + closed bool } -func (offset *SubscriptionOffset) Clear() { - offset.elOffset.Range(func(key, value interface{}) bool { - offset.elOffset.Delete(key) - return true - }) +func (offset *SubscriptionOffset) Close() { + offset.cond.L.Lock() + defer offset.cond.L.Unlock() + offset.closed = true + offset.cond.Broadcast() } func (offset *SubscriptionOffset) EventReceive(info info.OffsetInfo) { - o, exist := offset.elOffset.Load(info.EventLogID) + offset.cond.L.Lock() + defer offset.cond.L.Unlock() + for offset.uACKNumber >= offset.maxUACKNumber && !offset.closed { + offset.cond.Wait() + } + if offset.closed { + return + } + offset.uACKNumber++ + tracker, exist := offset.elOffsets[info.EventLogID] if !exist { - o, _ = offset.elOffset.LoadOrStore(info.EventLogID, initOffset(info.Offset)) + tracker = initOffset(info.Offset) + offset.elOffsets[info.EventLogID] = tracker } - o.(*offsetTracker).putOffset(info.Offset) + tracker.putOffset(info.Offset) } func (offset *SubscriptionOffset) EventCommit(info info.OffsetInfo) { - o, exist := offset.elOffset.Load(info.EventLogID) + offset.cond.L.Lock() + defer offset.cond.L.Unlock() + if offset.closed { + return + } + tracker, exist := offset.elOffsets[info.EventLogID] if !exist { return } - o.(*offsetTracker).commitOffset(info.Offset) + offset.uACKNumber-- + offset.cond.Signal() + tracker.commitOffset(info.Offset) } func (offset *SubscriptionOffset) GetCommit() info.ListOffsetInfo { + offset.cond.L.Lock() + defer offset.cond.L.Unlock() var commit info.ListOffsetInfo - offset.elOffset.Range(func(key, value interface{}) bool { - tracker, _ := value.(*offsetTracker) + for id, tracker := range offset.elOffsets { commit = append(commit, info.OffsetInfo{ - EventLogID: key.(vanus.ID), + EventLogID: id, Offset: tracker.offsetToCommit(), }) - return true - }) + } return commit } type offsetTracker struct { - mutex sync.Mutex maxOffset uint64 initOffset uint64 list *skiplist.SkipList @@ -80,7 +106,7 @@ type offsetTracker struct { func initOffset(initOffset uint64) *offsetTracker { return &offsetTracker{ initOffset: initOffset, - maxOffset: math.MaxUint64, + maxOffset: initOffset, list: skiplist.New(skiplist.GreaterThanFunc(func(lhs, rhs interface{}) int { v1, _ := lhs.(uint64) v2, _ := rhs.(uint64) @@ -95,23 +121,17 @@ func initOffset(initOffset uint64) *offsetTracker { } func (o *offsetTracker) putOffset(offset uint64) { - o.mutex.Lock() - defer o.mutex.Unlock() o.list.Set(offset, offset) o.maxOffset, _ = o.list.Back().Key().(uint64) } func (o *offsetTracker) commitOffset(offset uint64) { - o.mutex.Lock() - defer o.mutex.Unlock() o.list.Remove(offset) } func (o *offsetTracker) offsetToCommit() uint64 { - o.mutex.Lock() - defer o.mutex.Unlock() if o.list.Len() == 0 { - if o.maxOffset == math.MaxUint64 { + if o.maxOffset == o.initOffset { return o.initOffset } return o.maxOffset + 1 diff --git a/internal/trigger/offset/offset_test.go b/internal/trigger/offset/offset_test.go index bc7e2fd8e..b76e167ed 100644 --- a/internal/trigger/offset/offset_test.go +++ b/internal/trigger/offset/offset_test.go @@ -26,8 +26,8 @@ import ( func TestSubscriptionOffset(t *testing.T) { Convey("subscription offset", t, func() { eventLogID := vanus.NewTestID() - subOffset := NewSubscriptionOffset(vanus.NewTestID()) - Convey("commit with no receive", func() { + Convey("commit with no exist eventlog", func() { + subOffset := NewSubscriptionOffset(vanus.NewTestID(), 100, info.ListOffsetInfo{}) offsetBegin := uint64(1) commitEnd := offsetBegin + 10 for offset := offsetBegin; offset < commitEnd; offset++ { @@ -40,6 +40,7 @@ func TestSubscriptionOffset(t *testing.T) { So(0, ShouldEqual, len(commits)) }) Convey("commit with receive", func() { + subOffset := NewSubscriptionOffset(vanus.NewTestID(), 100, info.ListOffsetInfo{}) offsetBegin := uint64(1) offsetEnd := uint64(100) var wg sync.WaitGroup @@ -88,9 +89,9 @@ func TestSubscriptionOffset(t *testing.T) { commits = subOffset.GetCommit() So(1, ShouldEqual, len(commits)) So(offsetEnd, ShouldEqual, commits[0].Offset) - subOffset.Clear() + subOffset.Close() commits = subOffset.GetCommit() - So(0, ShouldEqual, len(commits)) + So(1, ShouldEqual, len(commits)) }) }) } diff --git a/internal/trigger/reader/reader.go b/internal/trigger/reader/reader.go index 7aa9f3623..4bdb2f26c 100644 --- a/internal/trigger/reader/reader.go +++ b/internal/trigger/reader/reader.go @@ -52,6 +52,7 @@ type Config struct { SubscriptionID vanus.ID SubscriptionIDStr string Offset EventLogOffset + BatchSize int CheckEventLogInterval time.Duration } @@ -255,7 +256,7 @@ func (elReader *eventLogReader) run(ctx context.Context) { } func (elReader *eventLogReader) readEvent(ctx context.Context, lr api.BusReader) error { - events, err := readEvents(ctx, lr, elReader.policy) + events, err := readEvents(ctx, lr) if err != nil { return err } @@ -272,13 +273,14 @@ func (elReader *eventLogReader) readEvent(ctx context.Context, lr api.BusReader) return err } elReader.offset = offset - elReader.policy.Forward(1) } + elReader.policy.Forward(len(events)) metrics.TriggerPullEventCounter.WithLabelValues( elReader.config.SubscriptionIDStr, elReader.config.EventBusName, elReader.eventLogIDStr). Add(float64(len(events))) return nil } + func (elReader *eventLogReader) putEvent(ctx context.Context, event info.EventRecord) error { select { case elReader.events <- event: @@ -288,14 +290,15 @@ func (elReader *eventLogReader) putEvent(ctx context.Context, event info.EventRe } } -func readEvents(ctx context.Context, lr api.BusReader, p api.ReadPolicy) ([]*ce.Event, error) { +func readEvents(ctx context.Context, lr api.BusReader) ([]*ce.Event, error) { timeout, cancel := context.WithTimeout(ctx, readEventTimeout) defer cancel() - events, _, _, err := lr.Read(timeout, option.WithReadPolicy(p), option.WithBatchSize(int(readSize))) + events, _, _, err := lr.Read(timeout) return events, err } func (elReader *eventLogReader) init(ctx context.Context) (api.BusReader, error) { - lr := elReader.config.Client.Eventbus(ctx, elReader.config.EventBusName).Reader() + lr := elReader.config.Client.Eventbus(ctx, elReader.config.EventBusName).Reader( + option.WithReadPolicy(elReader.policy), option.WithBatchSize(elReader.config.BatchSize)) return lr, nil } diff --git a/internal/trigger/reader/reader_test.go b/internal/trigger/reader/reader_test.go index be888b4f8..54476d8dd 100644 --- a/internal/trigger/reader/reader_test.go +++ b/internal/trigger/reader/reader_test.go @@ -41,7 +41,7 @@ func TestReaderStart(t *testing.T) { mockBusReader := api.NewMockBusReader(mockCtrl) mockClient.EXPECT().Eventbus(Any(), Any()).AnyTimes().Return(mockEventbus) mockEventbus.EXPECT().Writer().AnyTimes().Return(mockBusWriter) - mockEventbus.EXPECT().Reader(Any()).AnyTimes().Return(mockBusReader) + mockEventbus.EXPECT().Reader(Any(), Any()).AnyTimes().Return(mockBusReader) mockEventbus.EXPECT().GetLog(Any(), Any()).AnyTimes().Return(mockEventlog, nil) mockEventbus.EXPECT().ListLog(Any()).AnyTimes().Return([]api.Eventlog{mockEventlog}, nil) mockEventlog.EXPECT().ID().AnyTimes().Return(uint64(0)) @@ -51,7 +51,7 @@ func TestReaderStart(t *testing.T) { index := uint64(offset) mockEventlog.EXPECT().LatestOffset(Any()).AnyTimes().Return(offset, nil) mockEventlog.EXPECT().EarliestOffset(Any()).AnyTimes().Return(offset, nil) - mockBusReader.EXPECT().Read(Any(), Any(), Any()).AnyTimes().DoAndReturn( + mockBusReader.EXPECT().Read(Any()).AnyTimes().DoAndReturn( func(ctx context.Context, opts ...api.ReadOption) ([]*ce.Event, int64, uint64, error) { time.Sleep(time.Millisecond) e := ce.NewEvent() @@ -63,7 +63,7 @@ func TestReaderStart(t *testing.T) { return []*ce.Event{&e}, int64(0), uint64(0), nil }) eventCh := make(chan info.EventRecord, 100) - r := NewReader(Config{EventBusName: "test"}, eventCh).(*reader) + r := NewReader(Config{EventBusName: "test", BatchSize: 1}, eventCh).(*reader) r.config.Client = mockClient r.Start() var wg sync.WaitGroup diff --git a/internal/trigger/trigger/config.go b/internal/trigger/trigger/config.go index 0d1377263..2a9f5a77a 100644 --- a/internal/trigger/trigger/config.go +++ b/internal/trigger/trigger/config.go @@ -23,16 +23,15 @@ import ( ) const ( - defaultBufferSize = 1 << 10 - defaultFilterProcessSize = 2 - defaultDeliveryTimeout = 5 * time.Second - defaultMaxWriteAttempt = 3 - defaultGoroutineSize = 10000 - defaultBatchSize = 32 + defaultBufferSize = 1 << 10 + defaultDeliveryTimeout = 5 * time.Second + defaultMaxWriteAttempt = 3 + defaultGoroutineSize = 10000 + defaultMaxUACKNumber = 10000 + defaultBatchSize = 32 ) type Config struct { - FilterProcessSize int BufferSize int MaxRetryAttempts int32 DeliveryTimeout time.Duration @@ -43,34 +42,28 @@ type Config struct { Ordered bool GoroutineSize int - BatchSize int + SendBatchSize int + PullBatchSize int + MaxUACKNumber int } func defaultConfig() Config { c := Config{ - FilterProcessSize: defaultFilterProcessSize, BufferSize: defaultBufferSize, MaxRetryAttempts: primitive.MaxRetryAttempts, DeliveryTimeout: defaultDeliveryTimeout, DeadLetterEventbus: primitive.DeadLetterEventbusName, MaxWriteAttempt: defaultMaxWriteAttempt, GoroutineSize: defaultGoroutineSize, - BatchSize: defaultBatchSize, + SendBatchSize: defaultBatchSize, + MaxUACKNumber: defaultMaxUACKNumber, + PullBatchSize: defaultBatchSize, } return c } type Option func(t *trigger) -func WithFilterProcessSize(size int) Option { - return func(t *trigger) { - if size <= 0 { - return - } - t.config.FilterProcessSize = size - } -} - func WithBufferSize(size int) Option { return func(t *trigger) { if size <= 0 { @@ -102,7 +95,6 @@ func WithDeliveryTimeout(timeout uint32) Option { func WithOrdered(ordered bool) Option { return func(t *trigger) { t.config.Ordered = ordered - t.config.FilterProcessSize = 1 } } @@ -131,3 +123,39 @@ func WithDeadLetterEventbus(eventbus string) Option { t.config.DeadLetterEventbus = eventbus } } + +func WithGoroutineSize(size int) Option { + return func(t *trigger) { + if size <= 0 { + return + } + t.config.GoroutineSize = size + } +} + +func WithSendBatchSize(batchSize int) Option { + return func(t *trigger) { + if batchSize <= 0 { + return + } + t.config.SendBatchSize = batchSize + } +} + +func WithPullBatchSize(batchSize int) Option { + return func(t *trigger) { + if batchSize <= 0 { + return + } + t.config.PullBatchSize = batchSize + } +} + +func WithMaxUACKNumber(maxUACKNumber int) Option { + return func(t *trigger) { + if maxUACKNumber <= 0 { + return + } + t.config.MaxUACKNumber = maxUACKNumber + } +} diff --git a/internal/trigger/trigger/trigger.go b/internal/trigger/trigger/trigger.go index 12b71e10e..56d71cc1b 100644 --- a/internal/trigger/trigger/trigger.go +++ b/internal/trigger/trigger/trigger.go @@ -101,7 +101,6 @@ func NewTrigger(subscription *primitive.Subscription, opts ...Option) Trigger { config: defaultConfig(), state: TriggerCreated, filter: filter.GetFilter(subscription.Filters), - offsetManager: offset.NewSubscriptionOffset(subscription.ID), subscription: subscription, subscriptionIDStr: subscription.ID.String(), transformer: transform.NewTransformer(subscription.Transformer), @@ -113,6 +112,7 @@ func NewTrigger(subscription *primitive.Subscription, opts ...Option) Trigger { if t.rateLimiter == nil { t.rateLimiter = ratelimit.NewUnlimited() } + t.offsetManager = offset.NewSubscriptionOffset(subscription.ID, t.config.MaxUACKNumber, subscription.Offsets) t.pool, _ = ants.NewPool(t.config.GoroutineSize) return t } @@ -239,8 +239,8 @@ func (t *trigger) runRetryEventFilterTransform(ctx context.Context) { if !ok { return } + t.offsetManager.EventReceive(record.OffsetInfo) _ = t.pool.Submit(func() { - t.offsetManager.EventReceive(record.OffsetInfo) ec, _ := record.Event.Context.(*ce.EventContextV1) if len(ec.Extensions) == 0 { t.offsetManager.EventCommit(record.OffsetInfo) @@ -280,8 +280,8 @@ func (t *trigger) runEventFilterTransform(ctx context.Context) { if !ok { return } + t.offsetManager.EventReceive(record.OffsetInfo) _ = t.pool.Submit(func() { - t.offsetManager.EventReceive(record.OffsetInfo) startTime := time.Now() res := filter.Run(t.getFilter(), *record.Event) metrics.TriggerFilterCostSecond.WithLabelValues(t.subscriptionIDStr).Observe(time.Since(startTime).Seconds()) @@ -326,7 +326,7 @@ func (t *trigger) runEventToBatch(ctx context.Context) { } lock.Lock() events = append(events, event) - if !t.batch || len(events) >= t.config.BatchSize { + if !t.batch || len(events) >= t.config.SendBatchSize { e := make([]*toSendEvent, len(events)) copy(e, events) t.batchSendCh <- e @@ -358,13 +358,20 @@ func (t *trigger) runEventSend(ctx context.Context) { } func (t *trigger) processEvent(ctx context.Context, events ...*toSendEvent) { + defer func() { + // commit offset + for _, event := range events { + t.offsetManager.EventCommit(event.record.OffsetInfo) + } + }() es := make([]*ce.Event, len(events)) for i := range events { es[i] = events[i].transform } code, err := t.sendEvent(ctx, es...) if err != nil { - metrics.TriggerPushEventCounter.WithLabelValues(t.subscriptionIDStr, metrics.LabelValuePushEventFail).Inc() + metrics.TriggerPushEventCounter.WithLabelValues(t.subscriptionIDStr, metrics.LabelValuePushEventFail). + Add(float64(len(es))) log.Info(ctx, "send event fail", map[string]interface{}{ log.KeyError: err, "count": len(es), @@ -377,14 +384,12 @@ func (t *trigger) processEvent(ctx context.Context, events ...*toSendEvent) { t.writeFailEvent(ctx, event.record.Event, code, err) } } else { - metrics.TriggerPushEventCounter.WithLabelValues(t.subscriptionIDStr, metrics.LabelValuePushEventSuccess).Inc() + metrics.TriggerPushEventCounter.WithLabelValues(t.subscriptionIDStr, metrics.LabelValuePushEventSuccess). + Add(float64(len(es))) log.Debug(ctx, "send event success", map[string]interface{}{ "count": len(es), }) } - for _, event := range events { - t.offsetManager.EventCommit(event.record.OffsetInfo) - } } func (t *trigger) writeFailEvent(ctx context.Context, e *ce.Event, code int, err error) { @@ -493,38 +498,33 @@ func (t *trigger) writeEventToDeadLetter(ctx context.Context, e *ce.Event, reaso } func (t *trigger) getReaderConfig() reader.Config { - sub := t.subscription return reader.Config{ - EventBusName: sub.EventBus, + EventBusName: t.subscription.EventBus, Client: t.client, - SubscriptionID: sub.ID, - Offset: getOffset(t.offsetManager, sub), + SubscriptionID: t.subscription.ID, + BatchSize: t.config.PullBatchSize, + Offset: getOffset(t.subscription), } } func (t *trigger) getRetryEventReaderConfig() reader.Config { - sub := t.subscription ebName := primitive.RetryEventbusName return reader.Config{ EventBusName: ebName, Client: t.client, - SubscriptionID: sub.ID, - Offset: getOffset(t.offsetManager, sub), + SubscriptionID: t.subscription.ID, + BatchSize: t.config.PullBatchSize, + Offset: getOffset(t.subscription), } } -// getOffset from subscription,if subscriptionOffset exist,use subscriptionOffset. -func getOffset(subscriptionOffset *offset.SubscriptionOffset, sub *primitive.Subscription) map[vanus.ID]uint64 { +// getOffset from subscription. +func getOffset(sub *primitive.Subscription) map[vanus.ID]uint64 { // get offset from subscription offsetMap := make(map[vanus.ID]uint64) for _, o := range sub.Offsets { offsetMap[o.EventLogID] = o.Offset } - // get offset from offset manager - offsets := subscriptionOffset.GetCommit() - for _, offset := range offsets { - offsetMap[offset.EventLogID] = offset.Offset - } return offsetMap } @@ -540,7 +540,6 @@ func (t *trigger) Init(ctx context.Context) error { t.reader = reader.NewReader(t.getReaderConfig(), t.eventCh) t.retryEventCh = make(chan info.EventRecord, t.config.BufferSize) t.retryEventReader = reader.NewReader(t.getRetryEventReaderConfig(), t.retryEventCh) - t.offsetManager.Clear() return nil } @@ -575,12 +574,13 @@ func (t *trigger) Stop(ctx context.Context) error { t.stop() t.reader.Close() t.retryEventReader.Close() - t.wg.Wait() close(t.eventCh) close(t.retryEventCh) close(t.sendCh) close(t.batchSendCh) + t.wg.Wait() t.pool.Release() + t.offsetManager.Close() t.state = TriggerStopped log.Info(ctx, "trigger stopped", map[string]interface{}{ log.KeySubscriptionID: t.subscription.ID, diff --git a/internal/trigger/trigger/trigger_test.go b/internal/trigger/trigger/trigger_test.go index 6178f25f3..ab131f578 100644 --- a/internal/trigger/trigger/trigger_test.go +++ b/internal/trigger/trigger/trigger_test.go @@ -40,11 +40,7 @@ import ( func TestTrigger_Options(t *testing.T) { Convey("test trigger option", t, func() { tg := &trigger{} - WithFilterProcessSize(0)(tg) - So(tg.config.FilterProcessSize, ShouldEqual, 0) size := rand.Intn(1000) + 1 - WithFilterProcessSize(size)(tg) - So(tg.config.FilterProcessSize, ShouldEqual, size) WithDeliveryTimeout(0)(tg) So(tg.config.DeliveryTimeout, ShouldEqual, defaultDeliveryTimeout) size = rand.Intn(1000) + size diff --git a/internal/trigger/worker.go b/internal/trigger/worker.go index 10357e739..ea7c46732 100644 --- a/internal/trigger/worker.go +++ b/internal/trigger/worker.go @@ -255,6 +255,10 @@ func (w *worker) getTriggerOptions(subscription *primitive.Subscription) []trigg trigger.WithDeliveryTimeout(config.DeliveryTimeout), trigger.WithMaxRetryAttempts(config.GetMaxRetryAttempts()), trigger.WithDeadLetterEventbus(config.DeadLetterEventbus), - trigger.WithOrdered(config.OrderedEvent)) + trigger.WithOrdered(config.OrderedEvent), + trigger.WithGoroutineSize(w.config.SendEventGoroutineSize), + trigger.WithSendBatchSize(w.config.SendEventBatchSize), + trigger.WithPullBatchSize(w.config.PullEventBatchSize), + trigger.WithMaxUACKNumber(w.config.MaxUACKEventNumber)) return opts }