From 82eac0987ec61f4642fca9f01831fa233e69691c Mon Sep 17 00:00:00 2001 From: Wei Liu Date: Thu, 3 Apr 2025 18:05:10 +0800 Subject: [PATCH] enhance: Optimize shard serviceable mechanism - Merge leader view and channel management into ChannelDistManager, allowing a channel to have multiple delegators. - Improve shard leader switching to ensure a single replica only has one shard leader per channel. The shard leader handles all resource loading and query requests. - Refine the serviceable mechanism: after QC completes loading, sync the query view to the delegator. The delegator then determines its serviceable status based on the query view. - When a delegator encounters forwarding query or deletion failures, mark the corresponding segment as offline and transition it to an unserviceable state. Signed-off-by: Wei Liu --- .../channel_level_score_balancer_test.go | 40 +- internal/querycoordv2/balance/report.go | 2 +- .../balance/rowcount_based_balancer.go | 6 +- .../balance/rowcount_based_balancer_test.go | 104 +- .../balance/score_based_balancer.go | 20 +- .../balance/score_based_balancer_test.go | 82 +- .../querycoordv2/checkers/balance_checker.go | 6 +- .../querycoordv2/checkers/channel_checker.go | 46 +- .../checkers/channel_checker_test.go | 141 +- .../querycoordv2/checkers/controller_test.go | 38 +- .../querycoordv2/checkers/leader_checker.go | 12 +- .../checkers/leader_checker_test.go | 201 +- .../querycoordv2/checkers/segment_checker.go | 75 +- .../checkers/segment_checker_test.go | 190 +- internal/querycoordv2/dist/dist_controller.go | 35 +- .../querycoordv2/dist/dist_controller_test.go | 4 +- internal/querycoordv2/dist/dist_handler.go | 225 +- .../querycoordv2/dist/dist_handler_test.go | 86 +- internal/querycoordv2/handlers.go | 10 +- internal/querycoordv2/job/job_test.go | 37 +- .../querycoordv2/meta/channel_dist_manager.go | 197 +- .../meta/channel_dist_manager_test.go | 316 +- internal/querycoordv2/meta/dist_manager.go | 12 +- .../querycoordv2/meta/dist_manager_test.go | 62 +- .../querycoordv2/meta/leader_view_manager.go | 377 -- .../meta/leader_view_manager_test.go | 400 --- internal/querycoordv2/meta/mock_replica.go | 1035 ++++++ internal/querycoordv2/meta/replica.go | 36 + internal/querycoordv2/meta/replica_manager.go | 32 + .../querycoordv2/meta/segment_dist_manager.go | 6 + .../observers/collection_observer.go | 17 +- .../observers/collection_observer_test.go | 161 +- .../observers/replica_observer_test.go | 14 +- .../querycoordv2/observers/target_observer.go | 43 +- .../observers/target_observer_test.go | 67 +- internal/querycoordv2/server.go | 8 +- internal/querycoordv2/server_test.go | 6 +- internal/querycoordv2/services_test.go | 57 +- internal/querycoordv2/task/action.go | 6 +- internal/querycoordv2/task/executor.go | 21 +- internal/querycoordv2/task/scheduler.go | 77 +- internal/querycoordv2/task/task_test.go | 296 +- internal/querycoordv2/task/utils.go | 2 + internal/querycoordv2/utils/util.go | 43 +- internal/querynodev2/delegator/delegator.go | 27 +- .../querynodev2/delegator/delegator_data.go | 6 +- .../delegator/delegator_data_test.go | 9 +- .../querynodev2/delegator/delegator_test.go | 10 +- .../querynodev2/delegator/distribution.go | 148 +- .../delegator/distribution_test.go | 26 +- .../querynodev2/delegator/mock_delegator.go | 92 +- internal/querynodev2/services.go | 6 +- internal/querynodev2/services_test.go | 1 + pkg/proto/query_coord.proto | 7 +- pkg/proto/querypb/query_coord.pb.go | 3048 +++++++++-------- 55 files changed, 4693 insertions(+), 3338 deletions(-) delete mode 100644 internal/querycoordv2/meta/leader_view_manager.go delete mode 100644 internal/querycoordv2/meta/leader_view_manager_test.go create mode 100644 internal/querycoordv2/meta/mock_replica.go diff --git a/internal/querycoordv2/balance/channel_level_score_balancer_test.go b/internal/querycoordv2/balance/channel_level_score_balancer_test.go index c1f92ac75f79d..cb12d4571f61e 100644 --- a/internal/querycoordv2/balance/channel_level_score_balancer_test.go +++ b/internal/querycoordv2/balance/channel_level_score_balancer_test.go @@ -291,11 +291,18 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestAssignSegmentWithGrowing() defer paramtable.Get().Reset(paramtable.Get().QueryCoordCfg.DelegatorMemoryOverloadFactor.Key) // mock 50 growing row count in node 1, which is delegator, expect all segment assign to node 2 - leaderView := &meta.LeaderView{ - ID: 1, - CollectionID: 1, - } - suite.balancer.dist.LeaderViewManager.Update(1, leaderView) + suite.balancer.dist.ChannelDistManager.Update(1, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "v1", + }, + Node: 1, + View: &meta.LeaderView{ + ID: 1, + CollectionID: 1, + NumOfGrowingRows: 50, + }, + }) plans := balancer.AssignSegment(ctx, 1, toAssign, lo.Keys(distributions), false) for _, p := range plans { suite.Equal(int64(2), p.To) @@ -752,12 +759,12 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestMultiReplicaBalance() { }, channelDist: map[int64][]*meta.DmChannel{ 1: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 1}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 1, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, }, 3: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 3}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 3, View: &meta.LeaderView{ID: 4, CollectionID: 1}}, }, }, expectPlans: []SegmentAssignPlan{}, @@ -815,10 +822,10 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestMultiReplicaBalance() { suite.Len(channelPlans, 2) // mock new distribution after channel balance - balancer.dist.ChannelDistManager.Update(1, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1}) - balancer.dist.ChannelDistManager.Update(2, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 2}) - balancer.dist.ChannelDistManager.Update(3, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3}) - balancer.dist.ChannelDistManager.Update(4, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 4}) + balancer.dist.ChannelDistManager.Update(1, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1}}) + balancer.dist.ChannelDistManager.Update(2, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}) + balancer.dist.ChannelDistManager.Update(3, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3, View: &meta.LeaderView{ID: 3, CollectionID: 1}}) + balancer.dist.ChannelDistManager.Update(4, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 4, View: &meta.LeaderView{ID: 4, CollectionID: 1}}) // expected to balance segment segmentPlans, channelPlans = suite.getCollectionBalancePlans(balancer, c.collectionID) @@ -909,6 +916,7 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestExclusiveChannelBalance_Cha ChannelName: "channel2", }, Node: ch1Nodes[0], + View: &meta.LeaderView{ID: 2, CollectionID: collectionID}, }, }...) @@ -984,6 +992,7 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestExclusiveChannelBalance_Seg ChannelName: "channel1", }, Node: ch1Nodes[0], + View: &meta.LeaderView{ID: ch1Nodes[0], CollectionID: collectionID}, }, }...) @@ -994,6 +1003,7 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestExclusiveChannelBalance_Seg ChannelName: "channel2", }, Node: ch2Nodes[0], + View: &meta.LeaderView{ID: ch2Nodes[0], CollectionID: collectionID}, }, }...) @@ -1082,6 +1092,7 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestExclusiveChannelBalance_Nod ChannelName: "channel1", }, Node: ch1Nodes[0], + View: &meta.LeaderView{ID: ch1Nodes[0], CollectionID: collectionID}, }, }...) @@ -1092,6 +1103,7 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestExclusiveChannelBalance_Nod ChannelName: "channel2", }, Node: ch2Nodes[0], + View: &meta.LeaderView{ID: ch2Nodes[0], CollectionID: collectionID}, }, }...) @@ -1207,6 +1219,7 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestExclusiveChannelBalance_Seg ChannelName: "channel1", }, Node: ch1Nodes[0], + View: &meta.LeaderView{ID: ch1Nodes[0], CollectionID: collectionID}, }, }...) @@ -1217,6 +1230,7 @@ func (suite *ChannelLevelScoreBalancerTestSuite) TestExclusiveChannelBalance_Seg ChannelName: "channel2", }, Node: ch2Nodes[0], + View: &meta.LeaderView{ID: ch2Nodes[0], CollectionID: collectionID}, }, }...) diff --git a/internal/querycoordv2/balance/report.go b/internal/querycoordv2/balance/report.go index ce3f2c1cf766e..1c4fa9e69f370 100644 --- a/internal/querycoordv2/balance/report.go +++ b/internal/querycoordv2/balance/report.go @@ -55,7 +55,7 @@ func (br *balanceReport) SetMemoryFactor(node int64, memoryFactor float64) { } } -func (br *balanceReport) SetDeletagorScore(node int64, delegatorScore float64) { +func (br *balanceReport) SetDelegatorScore(node int64, delegatorScore float64) { nodeItem, ok := br.nodeItems[node] if ok { nodeItem.delegatorScore = delegatorScore diff --git a/internal/querycoordv2/balance/rowcount_based_balancer.go b/internal/querycoordv2/balance/rowcount_based_balancer.go index 7e41e31048220..4f8a6b79b9a0d 100644 --- a/internal/querycoordv2/balance/rowcount_based_balancer.go +++ b/internal/querycoordv2/balance/rowcount_based_balancer.go @@ -150,9 +150,9 @@ func (b *RowCountBasedBalancer) convertToNodeItemsBySegment(nodeIDs []int64) []* } // calculate growing segment row count on node - views := b.dist.LeaderViewManager.GetByFilter(meta.WithNodeID2LeaderView(node)) - for _, view := range views { - rowcnt += int(view.NumOfGrowingRows) + channels := b.dist.ChannelDistManager.GetByFilter(meta.WithNodeID2Channel(node)) + for _, channel := range channels { + rowcnt += int(channel.View.NumOfGrowingRows) } // calculate executing task cost in scheduler diff --git a/internal/querycoordv2/balance/rowcount_based_balancer_test.go b/internal/querycoordv2/balance/rowcount_based_balancer_test.go index 9302903e0f247..a3e24d4f7635b 100644 --- a/internal/querycoordv2/balance/rowcount_based_balancer_test.go +++ b/internal/querycoordv2/balance/rowcount_based_balancer_test.go @@ -250,10 +250,10 @@ func (suite *RowCountBasedBalancerTestSuite) TestBalance() { }, distributionChannels: map[int64][]*meta.DmChannel{ 2: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, }, 3: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 3}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 3, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, }, }, expectPlans: []SegmentAssignPlan{}, @@ -280,10 +280,10 @@ func (suite *RowCountBasedBalancerTestSuite) TestBalance() { }, distributionChannels: map[int64][]*meta.DmChannel{ 2: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, }, 1: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 1}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 1, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, }, }, expectPlans: []SegmentAssignPlan{ @@ -301,8 +301,8 @@ func (suite *RowCountBasedBalancerTestSuite) TestBalance() { distributions: map[int64][]*meta.Segment{}, distributionChannels: map[int64][]*meta.DmChannel{ 2: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 2}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 2, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, }, 3: {}, }, @@ -320,14 +320,14 @@ func (suite *RowCountBasedBalancerTestSuite) TestBalance() { distributions: map[int64][]*meta.Segment{}, distributionChannels: map[int64][]*meta.DmChannel{ 1: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v1"}, Node: 1}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 1}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v1"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1, Channel: "v1", Status: &querypb.LeaderViewStatus{Serviceable: true}}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 1, View: &meta.LeaderView{ID: 2, CollectionID: 1, Channel: "v2", Status: &querypb.LeaderViewStatus{Serviceable: true}}}, }, 2: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 2}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 2, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, }, 3: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v4"}, Node: 3}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v4"}, Node: 3, View: &meta.LeaderView{ID: 4, CollectionID: 1}}, }, }, expectPlans: []SegmentAssignPlan{}, @@ -342,8 +342,8 @@ func (suite *RowCountBasedBalancerTestSuite) TestBalance() { distributions: map[int64][]*meta.Segment{}, distributionChannels: map[int64][]*meta.DmChannel{ 1: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v1"}, Node: 1}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 1}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v1"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1, Channel: "v1", Status: &querypb.LeaderViewStatus{Serviceable: true}}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 1, View: &meta.LeaderView{ID: 2, CollectionID: 1, Channel: "v2", Status: &querypb.LeaderViewStatus{Serviceable: true}}}, }, 2: {}, 3: {}, @@ -443,8 +443,6 @@ func (suite *RowCountBasedBalancerTestSuite) TestBalance() { assertChannelAssignPlanElementMatch(&suite.Suite, c.expectChannelPlans, channelPlans, true) } - // clear distribution - for _, node := range c.nodes { balancer.meta.ResourceManager.HandleNodeDown(ctx, node) balancer.nodeManager.Remove(node) @@ -535,7 +533,7 @@ func (suite *RowCountBasedBalancerTestSuite) TestBalanceOnPartStopping() { }, distributionChannels: map[int64][]*meta.DmChannel{ 2: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, }, }, expectPlans: []SegmentAssignPlan{ @@ -595,10 +593,10 @@ func (suite *RowCountBasedBalancerTestSuite) TestBalanceOnPartStopping() { }, distributionChannels: map[int64][]*meta.DmChannel{ 2: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, }, 3: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 3}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 3, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, }, }, expectPlans: []SegmentAssignPlan{}, @@ -688,10 +686,10 @@ func (suite *RowCountBasedBalancerTestSuite) TestBalanceOutboundNodes() { }, distributionChannels: map[int64][]*meta.DmChannel{ 2: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, }, 3: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 3}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 3, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, }, }, expectPlans: []SegmentAssignPlan{}, @@ -718,10 +716,10 @@ func (suite *RowCountBasedBalancerTestSuite) TestBalanceOutboundNodes() { }, distributionChannels: map[int64][]*meta.DmChannel{ 2: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, }, 1: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 1}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 1, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, }, }, expectPlans: []SegmentAssignPlan{ @@ -897,12 +895,18 @@ func (suite *RowCountBasedBalancerTestSuite) TestAssignSegmentWithGrowing() { } // mock 50 growing row count in node 1, which is delegator, expect all segment assign to node 2 - leaderView := &meta.LeaderView{ - ID: 1, - CollectionID: 1, - NumOfGrowingRows: 50, - } - suite.balancer.dist.LeaderViewManager.Update(1, leaderView) + suite.balancer.dist.ChannelDistManager.Update(1, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "v1", + }, + Node: 1, + View: &meta.LeaderView{ + ID: 1, + CollectionID: 1, + NumOfGrowingRows: 50, + }, + }) plans := balancer.AssignSegment(ctx, 1, toAssign, lo.Keys(distributions), false) for _, p := range plans { suite.Equal(int64(2), p.To) @@ -934,14 +938,18 @@ func (suite *RowCountBasedBalancerTestSuite) TestDisableBalanceChannel() { distributions: map[int64][]*meta.Segment{}, distributionChannels: map[int64][]*meta.DmChannel{ 2: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 2}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, }, 3: {}, }, expectPlans: []SegmentAssignPlan{}, expectChannelPlans: []ChannelAssignPlan{ - {Channel: &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 2}, From: 2, To: 3, Replica: newReplicaDefaultRG(1)}, + {Channel: &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, + Node: 2, + View: &meta.LeaderView{ID: 3, CollectionID: 1}, + }, From: 2, To: 3, Replica: newReplicaDefaultRG(1)}, }, enableBalanceChannel: true, }, @@ -955,8 +963,8 @@ func (suite *RowCountBasedBalancerTestSuite) TestDisableBalanceChannel() { distributions: map[int64][]*meta.Segment{}, distributionChannels: map[int64][]*meta.DmChannel{ 2: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 2}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v2"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v3"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}, }, 3: {}, }, @@ -1099,12 +1107,12 @@ func (suite *RowCountBasedBalancerTestSuite) TestMultiReplicaBalance() { }, channelDist: map[int64][]*meta.DmChannel{ 1: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 1}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1}}, }, 3: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 3}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 3, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, }, }, expectPlans: []SegmentAssignPlan{}, @@ -1162,10 +1170,26 @@ func (suite *RowCountBasedBalancerTestSuite) TestMultiReplicaBalance() { suite.Len(channelPlans, 2) // mock new distribution after channel balance - balancer.dist.ChannelDistManager.Update(1, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1}) - balancer.dist.ChannelDistManager.Update(2, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 2}) - balancer.dist.ChannelDistManager.Update(3, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3}) - balancer.dist.ChannelDistManager.Update(4, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 4}) + balancer.dist.ChannelDistManager.Update(1, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, + Node: 1, + View: &meta.LeaderView{ID: 1, CollectionID: 1}, + }) + balancer.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, + Node: 2, + View: &meta.LeaderView{ID: 2, CollectionID: 1}, + }) + balancer.dist.ChannelDistManager.Update(3, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, + Node: 3, + View: &meta.LeaderView{ID: 3, CollectionID: 1}, + }) + balancer.dist.ChannelDistManager.Update(4, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, + Node: 4, + View: &meta.LeaderView{ID: 4, CollectionID: 1}, + }) // expected to balance segment segmentPlans, channelPlans = suite.getCollectionBalancePlans(balancer, c.collectionID) diff --git a/internal/querycoordv2/balance/score_based_balancer.go b/internal/querycoordv2/balance/score_based_balancer.go index 77a46e4b91f75..b35907e28ab46 100644 --- a/internal/querycoordv2/balance/score_based_balancer.go +++ b/internal/querycoordv2/balance/score_based_balancer.go @@ -311,11 +311,11 @@ func (b *ScoreBasedBalancer) convertToNodeItemsBySegment(br *balanceReport, coll nodeScoreMap[node].setAssignedScore(average) } // use assignedScore * delegatorOverloadFactor * delegator_num, to preserve fixed memory size for delegator - collectionViews := b.dist.LeaderViewManager.GetByFilter(meta.WithCollectionID2LeaderView(collectionID), meta.WithNodeID2LeaderView(node)) - if len(collectionViews) > 0 { - delegatorDelta := nodeScoreMap[node].getAssignedScore() * delegatorOverloadFactor * float64(len(collectionViews)) + collDelegator := b.dist.ChannelDistManager.GetByFilter(meta.WithCollectionID2Channel(collectionID), meta.WithNodeID2Channel(node)) + if len(collDelegator) > 0 { + delegatorDelta := nodeScoreMap[node].getAssignedScore() * delegatorOverloadFactor * float64(len(collDelegator)) nodeScoreMap[node].AddCurrentScoreDelta(delegatorDelta) - br.SetDeletagorScore(node, delegatorDelta) + br.SetDelegatorScore(node, delegatorDelta) } } return nodeScoreMap @@ -376,9 +376,9 @@ func (b *ScoreBasedBalancer) calculateScoreBySegment(br *balanceReport, collecti } // calculate global growing segment row count - views := b.dist.LeaderViewManager.GetByFilter(meta.WithNodeID2LeaderView(nodeID)) - for _, view := range views { - nodeRowCount += int(float64(view.NumOfGrowingRows)) + delegatorList := b.dist.ChannelDistManager.GetByFilter(meta.WithNodeID2Channel(nodeID)) + for _, d := range delegatorList { + nodeRowCount += int(float64(d.View.NumOfGrowingRows)) } // calculate executing task cost in scheduler @@ -392,9 +392,9 @@ func (b *ScoreBasedBalancer) calculateScoreBySegment(br *balanceReport, collecti } // calculate collection growing segment row count - collectionViews := b.dist.LeaderViewManager.GetByFilter(meta.WithCollectionID2LeaderView(collectionID), meta.WithNodeID2LeaderView(nodeID)) - for _, view := range collectionViews { - collectionRowCount += int(float64(view.NumOfGrowingRows)) + collDelegatorList := b.dist.ChannelDistManager.GetByFilter(meta.WithCollectionID2Channel(collectionID), meta.WithNodeID2Channel(nodeID)) + for _, d := range collDelegatorList { + collectionRowCount += int(float64(d.View.NumOfGrowingRows)) } // calculate executing task cost in scheduler diff --git a/internal/querycoordv2/balance/score_based_balancer_test.go b/internal/querycoordv2/balance/score_based_balancer_test.go index 70638086b1b84..b0bd235cc1290 100644 --- a/internal/querycoordv2/balance/score_based_balancer_test.go +++ b/internal/querycoordv2/balance/score_based_balancer_test.go @@ -300,11 +300,18 @@ func (suite *ScoreBasedBalancerTestSuite) TestAssignSegmentWithGrowing() { } // mock 50 growing row count in node 1, which is delegator, expect all segment assign to node 2 - leaderView := &meta.LeaderView{ - ID: 1, - CollectionID: 1, - } - suite.balancer.dist.LeaderViewManager.Update(1, leaderView) + suite.balancer.dist.ChannelDistManager.Update(1, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "v1", + }, + Node: 1, + View: &meta.LeaderView{ + ID: 1, + CollectionID: 1, + NumOfGrowingRows: 50, + }, + }) plans := balancer.AssignSegment(ctx, 1, toAssign, lo.Keys(distributions), false) for _, p := range plans { suite.Equal(int64(2), p.To) @@ -452,6 +459,19 @@ func (suite *ScoreBasedBalancerTestSuite) TestDelegatorPreserveMemory() { {SegmentInfo: &datapb.SegmentInfo{ID: 5, CollectionID: 1, NumOfRows: 10}, Node: 2}, }, }, + distributionChannels: map[int64][]*meta.DmChannel{ + 1: { + { + VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "v1"}, + Node: 1, + View: &meta.LeaderView{ + ID: 1, + CollectionID: 1, + NumOfGrowingRows: 10, + }, + }, + }, + }, expectPlans: []SegmentAssignPlan{}, expectChannelPlans: []ChannelAssignPlan{}, }, @@ -485,12 +505,6 @@ func (suite *ScoreBasedBalancerTestSuite) TestDelegatorPreserveMemory() { balancer.dist.ChannelDistManager.Update(node, v...) } - leaderView := &meta.LeaderView{ - ID: 1, - CollectionID: 1, - } - suite.balancer.dist.LeaderViewManager.Update(1, leaderView) - // 3. set up nodes info and resourceManager for balancer for i := range c.nodes { nodeInfo := session.NewNodeInfo(session.ImmutableNodeInfo{ @@ -940,12 +954,12 @@ func (suite *ScoreBasedBalancerTestSuite) TestMultiReplicaBalance() { }, channelDist: map[int64][]*meta.DmChannel{ 1: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 1}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1}}, }, 3: { - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 3}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 3, View: &meta.LeaderView{ID: 3, CollectionID: 1}}, }, }, expectPlans: []SegmentAssignPlan{}, @@ -1003,10 +1017,10 @@ func (suite *ScoreBasedBalancerTestSuite) TestMultiReplicaBalance() { suite.Len(channelPlans, 2) // mock new distribution after channel balance - balancer.dist.ChannelDistManager.Update(1, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1}) - balancer.dist.ChannelDistManager.Update(2, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 2}) - balancer.dist.ChannelDistManager.Update(3, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3}) - balancer.dist.ChannelDistManager.Update(4, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 4}) + balancer.dist.ChannelDistManager.Update(1, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1}}) + balancer.dist.ChannelDistManager.Update(2, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 2, View: &meta.LeaderView{ID: 2, CollectionID: 1}}) + balancer.dist.ChannelDistManager.Update(3, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 3, View: &meta.LeaderView{ID: 3, CollectionID: 1}}) + balancer.dist.ChannelDistManager.Update(4, &meta.DmChannel{VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel4"}, Node: 4, View: &meta.LeaderView{ID: 4, CollectionID: 1}}) // expected to balance segment segmentPlans, channelPlans = suite.getCollectionBalancePlans(balancer, c.collectionID) @@ -1200,9 +1214,9 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceSegmentAndChannel() { // set unbalance channel distribution balancer.dist.ChannelDistManager.Update(1, []*meta.DmChannel{ - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 1}, - {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 1}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel1"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel2"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1}}, + {VchannelInfo: &datapb.VchannelInfo{CollectionID: 1, ChannelName: "channel3"}, Node: 1, View: &meta.LeaderView{ID: 1, CollectionID: 1}}, }...) // expect to generate 2 balance segment task @@ -1269,7 +1283,9 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnMultiCollections() collectionID := int64(i) for i := 0; i < channelNum; i++ { channelDist = append(channelDist, &meta.DmChannel{ - VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, Node: 1, + VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, + Node: 1, + View: &meta.LeaderView{ID: 1, CollectionID: collectionID}, }) } } @@ -1351,7 +1367,9 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnDifferentQN() { channelDist := make([]*meta.DmChannel, 0) for i := 0; i < channelNum; i++ { channelDist = append(channelDist, &meta.DmChannel{ - VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, Node: 1, + VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, + Node: 1, + View: &meta.LeaderView{ID: 1, CollectionID: collectionID}, }) } balancer.dist.ChannelDistManager.Update(1, channelDist...) @@ -1433,7 +1451,9 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnChannelExclusive() channelDist1 := make([]*meta.DmChannel, 0) for i := 0; i < channelNum; i++ { channelDist1 = append(channelDist1, &meta.DmChannel{ - VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, Node: 1, + VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, + Node: 1, + View: &meta.LeaderView{ID: 1, CollectionID: collectionID}, }) } balancer.dist.ChannelDistManager.Update(1, channelDist1...) @@ -1442,7 +1462,9 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnChannelExclusive() channelDist2 := make([]*meta.DmChannel, 0) for i := 0; i < channelNum; i++ { channelDist2 = append(channelDist2, &meta.DmChannel{ - VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, Node: 1, + VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, + Node: 2, + View: &meta.LeaderView{ID: 2, CollectionID: collectionID}, }) } balancer.dist.ChannelDistManager.Update(2, channelDist2...) @@ -1451,7 +1473,9 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnChannelExclusive() channelDist3 := make([]*meta.DmChannel, 0) for i := 0; i < channelNum; i++ { channelDist3 = append(channelDist3, &meta.DmChannel{ - VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, Node: 1, + VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, + Node: 3, + View: &meta.LeaderView{ID: 3, CollectionID: collectionID}, }) } balancer.dist.ChannelDistManager.Update(3, channelDist3...) @@ -1517,7 +1541,9 @@ func (suite *ScoreBasedBalancerTestSuite) TestBalanceChannelOnStoppingNode() { collectionID := int64(i) for i := 0; i < channelNum; i++ { channelDist = append(channelDist, &meta.DmChannel{ - VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, Node: 1, + VchannelInfo: &datapb.VchannelInfo{CollectionID: collectionID, ChannelName: fmt.Sprintf("channel-%d-%d", collectionID, i)}, + Node: 1, + View: &meta.LeaderView{ID: 1, CollectionID: collectionID}, }) } } diff --git a/internal/querycoordv2/checkers/balance_checker.go b/internal/querycoordv2/checkers/balance_checker.go index 5c2aa04944ab0..ec9ff10c31c22 100644 --- a/internal/querycoordv2/checkers/balance_checker.go +++ b/internal/querycoordv2/checkers/balance_checker.go @@ -239,10 +239,8 @@ func (b *BalanceChecker) Check(ctx context.Context) []task.Task { // if balance on multiple collections is disabled, and there are already some tasks, break break } - if len(channelTasks) < channelBatchSize { - replicasToBalance := b.getReplicaForStoppingBalance(ctx) - generateBalanceTaskForReplicas(replicasToBalance) - } + replicasToBalance := b.getReplicaForStoppingBalance(ctx) + generateBalanceTaskForReplicas(replicasToBalance) } } else { // then check for auto balance diff --git a/internal/querycoordv2/checkers/channel_checker.go b/internal/querycoordv2/checkers/channel_checker.go index 0745dbdfaa380..86bdf3c3745c5 100644 --- a/internal/querycoordv2/checkers/channel_checker.go +++ b/internal/querycoordv2/checkers/channel_checker.go @@ -181,49 +181,23 @@ func (c *ChannelChecker) getDmChannelDiff(ctx context.Context, collectionID int6 func (c *ChannelChecker) findRepeatedChannels(ctx context.Context, replicaID int64) []*meta.DmChannel { log := log.Ctx(ctx).WithRateGroup("ChannelChecker.findRepeatedChannels", 1, 60) replica := c.meta.Get(ctx, replicaID) - ret := make([]*meta.DmChannel, 0) + dupChannels := make([]*meta.DmChannel, 0) if replica == nil { log.Info("replica does not exist, skip it") - return ret + return dupChannels } - dist := c.dist.ChannelDistManager.GetByCollectionAndFilter(replica.GetCollectionID(), meta.WithReplica2Channel(replica)) - - versionsMap := make(map[string]*meta.DmChannel) - for _, ch := range dist { - leaderView := c.dist.LeaderViewManager.GetLeaderShardView(ch.Node, ch.GetChannelName()) - if leaderView == nil { - log.Info("shard leader view is not ready, skip", - zap.Int64("collectionID", replica.GetCollectionID()), - zap.Int64("replicaID", replicaID), - zap.Int64("leaderID", ch.Node), - zap.String("channel", ch.GetChannelName())) - continue - } - - if leaderView.UnServiceableError != nil { - log.RatedInfo(10, "replica has unavailable shard leader", - zap.Int64("collectionID", replica.GetCollectionID()), - zap.Int64("replicaID", replicaID), - zap.Int64("leaderID", ch.Node), - zap.String("channel", ch.GetChannelName()), - zap.Error(leaderView.UnServiceableError)) - continue - } - maxVer, ok := versionsMap[ch.GetChannelName()] - if !ok { - versionsMap[ch.GetChannelName()] = ch - continue - } - if maxVer.Version <= ch.Version { - ret = append(ret, maxVer) - versionsMap[ch.GetChannelName()] = ch - } else { - ret = append(ret, ch) + delegatorList := c.dist.ChannelDistManager.GetByCollectionAndFilter(replica.GetCollectionID(), meta.WithReplica2Channel(replica)) + for _, delegator := range delegatorList { + leader := c.dist.ChannelDistManager.GetShardLeader(delegator.GetChannelName(), replica) + // if channel's version is smaller than shard leader's version, it means that the channel is not up to date + if delegator.Version < leader.Version && delegator.Node != leader.Node { + dupChannels = append(dupChannels, delegator) } } - return ret + + return dupChannels } func (c *ChannelChecker) createChannelLoadTask(ctx context.Context, channels []*meta.DmChannel, replica *meta.Replica) []task.Task { diff --git a/internal/querycoordv2/checkers/channel_checker_test.go b/internal/querycoordv2/checkers/channel_checker_test.go index 1619a27ed5399..f2e03f2f41166 100644 --- a/internal/querycoordv2/checkers/channel_checker_test.go +++ b/internal/querycoordv2/checkers/channel_checker_test.go @@ -34,6 +34,7 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/v2/kv" "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/querypb" "github.com/milvus-io/milvus/pkg/v2/util/etcd" "github.com/milvus-io/milvus/pkg/v2/util/paramtable" ) @@ -169,10 +170,38 @@ func (suite *ChannelCheckerTestSuite) TestReduceChannel() { checker.targetMgr.UpdateCollectionNextTarget(ctx, int64(1)) checker.targetMgr.UpdateCollectionCurrentTarget(ctx, int64(1)) - checker.dist.ChannelDistManager.Update(1, utils.CreateTestChannel(1, 1, 1, "test-insert-channel1")) - checker.dist.LeaderViewManager.Update(1, &meta.LeaderView{ID: 1, Channel: "test-insert-channel1"}) - checker.dist.ChannelDistManager.Update(1, utils.CreateTestChannel(1, 1, 1, "test-insert-channel2")) - checker.dist.LeaderViewManager.Update(1, &meta.LeaderView{ID: 1, Channel: "test-insert-channel2"}) + checker.dist.ChannelDistManager.Update(1, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel1", + }, + Node: 1, + Version: 1, + View: &meta.LeaderView{ + ID: 1, + Channel: "test-insert-channel1", + Version: 1, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + }, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel2", + }, + Node: 1, + Version: 1, + View: &meta.LeaderView{ + ID: 1, + Channel: "test-insert-channel2", + Version: 1, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + }) + suite.setNodeAvailable(1) tasks := checker.Check(context.TODO()) suite.Len(tasks, 1) @@ -210,15 +239,59 @@ func (suite *ChannelCheckerTestSuite) TestRepeatedChannels() { suite.broker.EXPECT().GetRecoveryInfoV2(mock.Anything, int64(1)).Return( channels, segments, nil) checker.targetMgr.UpdateCollectionNextTarget(ctx, int64(1)) - checker.dist.ChannelDistManager.Update(1, utils.CreateTestChannel(1, 1, 1, "test-insert-channel")) - checker.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 2, "test-insert-channel")) + checker.dist.ChannelDistManager.Update(1, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 1, + Version: 1, + View: &meta.LeaderView{ + ID: 1, + Channel: "test-insert-channel", + Version: 1, + Status: &querypb.LeaderViewStatus{Serviceable: true}, + }, + }) + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 2, + View: &meta.LeaderView{ + ID: 2, + Channel: "test-insert-channel", + Version: 2, + Status: &querypb.LeaderViewStatus{ + Serviceable: false, + }, + }, + }) tasks := checker.Check(context.TODO()) suite.Len(tasks, 0) suite.setNodeAvailable(1, 2) - checker.dist.LeaderViewManager.Update(1, &meta.LeaderView{ID: 1, Channel: "test-insert-channel"}) - checker.dist.LeaderViewManager.Update(2, &meta.LeaderView{ID: 2, Channel: "test-insert-channel"}) + + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 2, + View: &meta.LeaderView{ + ID: 2, + Channel: "test-insert-channel", + Version: 2, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + }) + tasks = checker.Check(context.TODO()) suite.Len(tasks, 1) suite.EqualValues(1, tasks[0].ReplicaID()) @@ -239,19 +312,6 @@ func (suite *ChannelCheckerTestSuite) TestReleaseDirtyChannels() { err = checker.meta.ReplicaManager.Put(ctx, utils.CreateTestReplica(1, 1, []int64{1})) suite.NoError(err) - segments := []*datapb.SegmentInfo{ - { - ID: 1, - InsertChannel: "test-insert-channel", - }, - } - - channels := []*datapb.VchannelInfo{ - { - CollectionID: 1, - ChannelName: "test-insert-channel", - }, - } suite.nodeMgr.Add(session.NewNodeInfo(session.ImmutableNodeInfo{ NodeID: 1, Address: "localhost", @@ -263,13 +323,38 @@ func (suite *ChannelCheckerTestSuite) TestReleaseDirtyChannels() { Hostname: "localhost", })) - suite.broker.EXPECT().GetRecoveryInfoV2(mock.Anything, int64(1)).Return( - channels, segments, nil) - checker.targetMgr.UpdateCollectionNextTarget(ctx, int64(1)) - checker.dist.ChannelDistManager.Update(1, utils.CreateTestChannel(1, 1, 2, "test-insert-channel")) - checker.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 2, "test-insert-channel")) - checker.dist.LeaderViewManager.Update(1, &meta.LeaderView{ID: 1, Channel: "test-insert-channel"}) - checker.dist.LeaderViewManager.Update(2, &meta.LeaderView{ID: 2, Channel: "test-insert-channel"}) + checker.dist.ChannelDistManager.Update(1, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 1, + Version: 1, + View: &meta.LeaderView{ + ID: 1, + Channel: "test-insert-channel", + Version: 1, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + }) + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 2, + View: &meta.LeaderView{ + ID: 2, + Channel: "test-insert-channel", + Version: 2, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + }) tasks := checker.Check(context.TODO()) suite.Len(tasks, 1) diff --git a/internal/querycoordv2/checkers/controller_test.go b/internal/querycoordv2/checkers/controller_test.go index 420e69ae65b30..254ab7114cdc7 100644 --- a/internal/querycoordv2/checkers/controller_test.go +++ b/internal/querycoordv2/checkers/controller_test.go @@ -35,6 +35,7 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/v2/kv" "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/querypb" "github.com/milvus-io/milvus/pkg/v2/util/etcd" "github.com/milvus-io/milvus/pkg/v2/util/paramtable" ) @@ -124,8 +125,23 @@ func (suite *CheckerControllerSuite) TestBasic() { suite.targetManager.UpdateCollectionNextTarget(ctx, int64(1)) // set dist - suite.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - suite.dist.LeaderViewManager.Update(2, utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{1: 2}, map[int64]*meta.Segment{})) + suite.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + // View: utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{1: 2}, map[int64]*meta.Segment{}), + View: &meta.LeaderView{ + ID: 2, + Channel: "test-insert-channel", + Version: 1, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + }) counter := atomic.NewInt64(0) suite.scheduler.EXPECT().Add(mock.Anything).Run(func(task task.Task) { @@ -154,8 +170,22 @@ func (suite *CheckerControllerSuite) TestBasic() { }, 3*time.Second, 1*time.Millisecond) // until new channel has been subscribed - suite.dist.ChannelDistManager.Update(1, utils.CreateTestChannel(1, 1, 1, "test-insert-channel2")) - suite.dist.LeaderViewManager.Update(1, utils.CreateTestLeaderView(1, 1, "test-insert-channel2", map[int64]int64{}, map[int64]*meta.Segment{})) + suite.dist.ChannelDistManager.Update(1, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel2", + }, + Node: 1, + Version: 1, + View: &meta.LeaderView{ + ID: 1, + Channel: "test-insert-channel2", + Version: 1, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + }) // expect assign segment after channel has been subscribed suite.Eventually(func() bool { diff --git a/internal/querycoordv2/checkers/leader_checker.go b/internal/querycoordv2/checkers/leader_checker.go index c6c617ff24268..57ed269415ad4 100644 --- a/internal/querycoordv2/checkers/leader_checker.go +++ b/internal/querycoordv2/checkers/leader_checker.go @@ -97,12 +97,12 @@ func (c *LeaderChecker) Check(ctx context.Context) []task.Task { nodes = replica.GetRWSQNodes() } for _, node := range nodes { - leaderViews := c.dist.LeaderViewManager.GetByFilter(meta.WithCollectionID2LeaderView(replica.GetCollectionID()), meta.WithNodeID2LeaderView(node)) - for _, leaderView := range leaderViews { - dist := c.dist.SegmentDistManager.GetByFilter(meta.WithChannel(leaderView.Channel), meta.WithReplica(replica)) - tasks = append(tasks, c.findNeedLoadedSegments(ctx, replica, leaderView, dist)...) - tasks = append(tasks, c.findNeedRemovedSegments(ctx, replica, leaderView, dist)...) - tasks = append(tasks, c.findNeedSyncPartitionStats(ctx, replica, leaderView, node)...) + delegatorList := c.dist.ChannelDistManager.GetByFilter(meta.WithCollectionID2Channel(replica.GetCollectionID()), meta.WithNodeID2Channel(node)) + for _, d := range delegatorList { + dist := c.dist.SegmentDistManager.GetByFilter(meta.WithChannel(d.View.Channel), meta.WithReplica(replica)) + tasks = append(tasks, c.findNeedLoadedSegments(ctx, replica, d.View, dist)...) + tasks = append(tasks, c.findNeedRemovedSegments(ctx, replica, d.View, dist)...) + tasks = append(tasks, c.findNeedSyncPartitionStats(ctx, replica, d.View, node)...) } } } diff --git a/internal/querycoordv2/checkers/leader_checker_test.go b/internal/querycoordv2/checkers/leader_checker_test.go index 52618b0d2f467..fe313175e2e8f 100644 --- a/internal/querycoordv2/checkers/leader_checker_test.go +++ b/internal/querycoordv2/checkers/leader_checker_test.go @@ -124,10 +124,20 @@ func (suite *LeaderCheckerTestSuite) TestSyncLoadedSegments() { observer.target.UpdateCollectionCurrentTarget(ctx, 1) loadVersion := time.Now().UnixMilli() observer.dist.SegmentDistManager.Update(1, utils.CreateTestSegment(1, 1, 1, 2, loadVersion, "test-insert-channel")) - observer.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - view := utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{}, map[int64]*meta.Segment{}) - view.TargetVersion = observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget) - observer.dist.LeaderViewManager.Update(2, view) + observer.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget), + }, + }) tasks = suite.checker.Check(context.TODO()) suite.Len(tasks, 1) @@ -145,13 +155,21 @@ func (suite *LeaderCheckerTestSuite) TestSyncLoadedSegments() { version1, version2 := int64(1), int64(2) observer.dist.SegmentDistManager.Update(node1) observer.dist.SegmentDistManager.Update(node2, utils.CreateTestSegment(1, 1, 1, node2, version2, "test-insert-channel")) - view = utils.CreateTestLeaderView(node2, 1, "test-insert-channel", map[int64]int64{}, map[int64]*meta.Segment{}) - view.TargetVersion = observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget) - view.Segments[1] = &querypb.SegmentDist{ - NodeID: node1, - Version: version1, - } - observer.dist.LeaderViewManager.Update(node2, view) + observer.dist.ChannelDistManager.Update(node2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: node2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget), + Segments: map[int64]*querypb.SegmentDist{1: {NodeID: node1, Version: version1}}, + }, + }) tasks = suite.checker.Check(context.TODO()) suite.Len(tasks, 1) @@ -199,10 +217,20 @@ func (suite *LeaderCheckerTestSuite) TestActivation() { observer.target.UpdateCollectionNextTarget(ctx, int64(1)) observer.target.UpdateCollectionCurrentTarget(ctx, 1) observer.dist.SegmentDistManager.Update(1, utils.CreateTestSegment(1, 1, 1, 2, 1, "test-insert-channel")) - observer.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - view := utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{}, map[int64]*meta.Segment{}) - view.TargetVersion = observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget) - observer.dist.LeaderViewManager.Update(2, view) + observer.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget), + }, + }) suite.checker.Deactivate() tasks := suite.checker.Check(context.TODO()) @@ -243,10 +271,20 @@ func (suite *LeaderCheckerTestSuite) TestStoppingNode() { observer.target.UpdateCollectionNextTarget(ctx, int64(1)) observer.target.UpdateCollectionCurrentTarget(ctx, 1) observer.dist.SegmentDistManager.Update(1, utils.CreateTestSegment(1, 1, 1, 2, 1, "test-insert-channel")) - observer.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - view := utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{}, map[int64]*meta.Segment{}) - view.TargetVersion = observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget) - observer.dist.LeaderViewManager.Update(2, view) + observer.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget), + }, + }) mutableReplica := replica.CopyForWrite() mutableReplica.AddRONode(2) @@ -293,11 +331,20 @@ func (suite *LeaderCheckerTestSuite) TestIgnoreSyncLoadedSegments() { observer.target.UpdateCollectionNextTarget(ctx, int64(1)) observer.dist.SegmentDistManager.Update(1, utils.CreateTestSegment(1, 1, 1, 2, 1, "test-insert-channel"), utils.CreateTestSegment(1, 1, 2, 2, 1, "test-insert-channel")) - observer.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - view := utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{}, map[int64]*meta.Segment{}) - view.TargetVersion = observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget) - observer.dist.LeaderViewManager.Update(2, view) - + observer.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget), + }, + }) tasks := suite.checker.Check(context.TODO()) suite.Len(tasks, 1) suite.Equal(tasks[0].Source(), utils.LeaderChecker) @@ -346,14 +393,35 @@ func (suite *LeaderCheckerTestSuite) TestSyncLoadedSegmentsWithReplicas() { observer.target.UpdateCollectionCurrentTarget(ctx, 1) observer.dist.SegmentDistManager.Update(1, utils.CreateTestSegment(1, 1, 1, 1, 0, "test-insert-channel")) observer.dist.SegmentDistManager.Update(4, utils.CreateTestSegment(1, 1, 1, 4, 0, "test-insert-channel")) - observer.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - observer.dist.ChannelDistManager.Update(4, utils.CreateTestChannel(1, 4, 2, "test-insert-channel")) - view := utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{}, map[int64]*meta.Segment{}) - view.TargetVersion = observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget) - observer.dist.LeaderViewManager.Update(2, view) - view2 := utils.CreateTestLeaderView(4, 1, "test-insert-channel", map[int64]int64{1: 4}, map[int64]*meta.Segment{}) - view.TargetVersion = observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget) - observer.dist.LeaderViewManager.Update(4, view2) + observer.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget), + }, + }) + observer.dist.ChannelDistManager.Update(4, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 4, + Version: 2, + View: &meta.LeaderView{ + ID: 4, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget), + Segments: map[int64]*querypb.SegmentDist{1: {NodeID: 4}}, + }, + }) tasks := suite.checker.Check(context.TODO()) suite.Len(tasks, 1) @@ -385,10 +453,21 @@ func (suite *LeaderCheckerTestSuite) TestSyncRemovedSegments() { observer.target.UpdateCollectionNextTarget(ctx, int64(1)) observer.target.UpdateCollectionCurrentTarget(ctx, 1) - observer.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - view := utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{3: 1}, map[int64]*meta.Segment{}) - view.TargetVersion = observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget) - observer.dist.LeaderViewManager.Update(2, view) + observer.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget), + Segments: map[int64]*querypb.SegmentDist{3: {NodeID: 1}}, + }, + }) tasks := suite.checker.Check(context.TODO()) suite.Len(tasks, 1) @@ -426,8 +505,21 @@ func (suite *LeaderCheckerTestSuite) TestIgnoreSyncRemovedSegments() { channels, segments, nil) observer.target.UpdateCollectionNextTarget(ctx, int64(1)) - observer.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - observer.dist.LeaderViewManager.Update(2, utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{3: 2, 2: 2}, map[int64]*meta.Segment{})) + observer.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget), + Segments: map[int64]*querypb.SegmentDist{3: {NodeID: 2}, 2: {NodeID: 2}}, + }, + }) tasks := suite.checker.Check(context.TODO()) suite.Len(tasks, 1) @@ -443,7 +535,7 @@ func (suite *LeaderCheckerTestSuite) TestIgnoreSyncRemovedSegments() { func (suite *LeaderCheckerTestSuite) TestUpdatePartitionStats() { ctx := context.Background() testChannel := "test-insert-channel" - leaderID := int64(2) + // leaderID := int64(2) observer := suite.checker observer.meta.CollectionManager.PutCollection(ctx, utils.CreateTestCollection(1, 1)) observer.meta.CollectionManager.PutPartition(ctx, utils.CreateTestPartition(1, 1)) @@ -477,14 +569,31 @@ func (suite *LeaderCheckerTestSuite) TestUpdatePartitionStats() { observer.target.UpdateCollectionCurrentTarget(ctx, 1) loadVersion := time.Now().UnixMilli() observer.dist.SegmentDistManager.Update(1, utils.CreateTestSegment(1, 1, 2, 1, loadVersion, testChannel)) - observer.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, testChannel)) - view := utils.CreateTestLeaderView(2, 1, testChannel, map[int64]int64{2: 1}, map[int64]*meta.Segment{}) - view.PartitionStatsVersions = map[int64]int64{ - 1: 100, - } + // observer.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, testChannel)) + // view := utils.CreateTestLeaderView(2, 1, testChannel, map[int64]int64{2: 1}, map[int64]*meta.Segment{}) + // view.PartitionStatsVersions = map[int64]int64{ + // 1: 100, + // } // current partition stat version in leader view is version100 for partition1 - view.TargetVersion = observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget) - observer.dist.LeaderViewManager.Update(leaderID, view) + // view.TargetVersion = observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget) + // observer.dist.ShardLeaderManager.Update(leaderID, view) + observer.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: observer.target.GetCollectionTargetVersion(ctx, 1, meta.CurrentTarget), + PartitionStatsVersions: map[int64]int64{ + 1: 100, + }, + }, + }) tasks = suite.checker.Check(context.TODO()) suite.Len(tasks, 1) diff --git a/internal/querycoordv2/checkers/segment_checker.go b/internal/querycoordv2/checkers/segment_checker.go index 16270cf54ed36..2a8eeb4a0e96d 100644 --- a/internal/querycoordv2/checkers/segment_checker.go +++ b/internal/querycoordv2/checkers/segment_checker.go @@ -128,13 +128,11 @@ func (c *SegmentChecker) checkReplica(ctx context.Context, replica *meta.Replica // compare with targets to find the lack and redundancy of segments lacks, redundancies := c.getSealedSegmentDiff(ctx, replica.GetCollectionID(), replica.GetID()) - // loadCtx := trace.ContextWithSpan(context.Background(), c.meta.GetCollection(replica.CollectionID).LoadSpan) tasks := c.createSegmentLoadTasks(c.getTraceCtx(ctx, replica.GetCollectionID()), lacks, replica) task.SetReason("lacks of segment", tasks...) task.SetPriority(task.TaskPriorityNormal, tasks...) ret = append(ret, tasks...) - redundancies = c.filterSegmentInUse(ctx, replica, redundancies) tasks = c.createSegmentReduceTasks(c.getTraceCtx(ctx, replica.GetCollectionID()), redundancies, replica, querypb.DataScope_Historical) task.SetReason("segment not exists in target", tasks...) task.SetPriority(task.TaskPriorityNormal, tasks...) @@ -142,7 +140,7 @@ func (c *SegmentChecker) checkReplica(ctx context.Context, replica *meta.Replica // compare inner dists to find repeated loaded segments redundancies = c.findRepeatedSealedSegments(ctx, replica.GetID()) - redundancies = c.filterExistedOnLeader(replica, redundancies) + redundancies = c.filterInUsedByDelegator(replica, redundancies) tasks = c.createSegmentReduceTasks(c.getTraceCtx(ctx, replica.GetCollectionID()), redundancies, replica, querypb.DataScope_Historical) task.SetReason("redundancies of segment", tasks...) // set deduplicate task priority to low, to avoid deduplicate task cancel balance task @@ -173,19 +171,15 @@ func (c *SegmentChecker) getGrowingSegmentDiff(ctx context.Context, collectionID zap.Int64("collectionID", collectionID), zap.Int64("replicaID", replica.GetID())) - leaders := c.dist.ChannelDistManager.GetShardLeadersByReplica(replica) - for channelName, node := range leaders { - view := c.dist.LeaderViewManager.GetLeaderShardView(node, channelName) - if view == nil { - log.Info("leaderView is not ready, skip", zap.String("channelName", channelName), zap.Int64("node", node)) - continue - } + delegatorList := c.dist.ChannelDistManager.GetByFilter(meta.WithReplica2Channel(replica)) + for _, d := range delegatorList { + view := d.View targetVersion := c.targetMgr.GetCollectionTargetVersion(ctx, collectionID, meta.CurrentTarget) if view.TargetVersion != targetVersion { // before shard delegator update it's readable version, skip release segment log.RatedInfo(20, "before shard delegator update it's readable version, skip release segment", - zap.String("channelName", channelName), - zap.Int64("nodeID", node), + zap.String("channelName", view.Channel), + zap.Int64("nodeID", view.ID), zap.Int64("leaderVersion", view.TargetVersion), zap.Int64("currentVersion", targetVersion), ) @@ -300,51 +294,32 @@ func (c *SegmentChecker) findRepeatedSealedSegments(ctx context.Context, replica return segments } -func (c *SegmentChecker) filterExistedOnLeader(replica *meta.Replica, segments []*meta.Segment) []*meta.Segment { +func (c *SegmentChecker) filterInUsedByDelegator(replica *meta.Replica, segments []*meta.Segment) []*meta.Segment { filtered := make([]*meta.Segment, 0, len(segments)) - for _, s := range segments { - leaderID, ok := c.dist.ChannelDistManager.GetShardLeader(replica, s.GetInsertChannel()) - if !ok { - continue - } - - view := c.dist.LeaderViewManager.GetLeaderShardView(leaderID, s.GetInsertChannel()) - if view == nil { - continue - } - seg, ok := view.Segments[s.GetID()] - if ok && seg.NodeID == s.Node { - // if this segment is serving on leader, do not remove it for search available - continue - } - filtered = append(filtered, s) - } - return filtered -} + delegatorList := c.dist.ChannelDistManager.GetByFilter(meta.WithReplica2Channel(replica)) + ch2DelegatorList := lo.GroupBy(delegatorList, func(d *meta.DmChannel) string { + return d.View.Channel + }) -func (c *SegmentChecker) filterSegmentInUse(ctx context.Context, replica *meta.Replica, segments []*meta.Segment) []*meta.Segment { - filtered := make([]*meta.Segment, 0, len(segments)) for _, s := range segments { - leaderID, ok := c.dist.ChannelDistManager.GetShardLeader(replica, s.GetInsertChannel()) - if !ok { + delegatorList := ch2DelegatorList[s.GetInsertChannel()] + if len(delegatorList) == 0 { + // skip deduplication if delegator is not found continue } - view := c.dist.LeaderViewManager.GetLeaderShardView(leaderID, s.GetInsertChannel()) - if view == nil { - continue + usedByDelegator := false + for _, delegator := range delegatorList { + seg, ok := delegator.View.Segments[s.GetID()] + if ok && seg.NodeID == s.Node { + // if this segment is serving on leader, do not remove it for search available + usedByDelegator = true + break + } } - currentTargetVersion := c.targetMgr.GetCollectionTargetVersion(ctx, s.CollectionID, meta.CurrentTarget) - partition := c.meta.CollectionManager.GetPartition(ctx, s.PartitionID) - - // if delegator has valid target version, and before it update to latest readable version, skip release it's sealed segment - // Notice: if syncTargetVersion stuck, segment on delegator won't be released - readableVersionNotUpdate := view.TargetVersion != initialTargetVersion && view.TargetVersion < currentTargetVersion - if partition != nil && readableVersionNotUpdate { - // leader view version hasn't been updated, segment maybe still in use - continue + if !usedByDelegator { + filtered = append(filtered, s) } - filtered = append(filtered, s) } return filtered } @@ -361,7 +336,7 @@ func (c *SegmentChecker) createSegmentLoadTasks(ctx context.Context, segments [] plans := make([]balance.SegmentAssignPlan, 0) for shard, segments := range shardSegments { // if channel is not subscribed yet, skip load segments - leader := c.dist.LeaderViewManager.GetLatestShardLeaderByFilter(meta.WithReplica2LeaderView(replica), meta.WithChannelName2LeaderView(shard)) + leader := c.dist.ChannelDistManager.GetShardLeader(shard, replica) if leader == nil { continue } diff --git a/internal/querycoordv2/checkers/segment_checker_test.go b/internal/querycoordv2/checkers/segment_checker_test.go index d775082a03bb6..42ad673756f06 100644 --- a/internal/querycoordv2/checkers/segment_checker_test.go +++ b/internal/querycoordv2/checkers/segment_checker_test.go @@ -35,6 +35,7 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/pkg/v2/kv" "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/querypb" "github.com/milvus-io/milvus/pkg/v2/util/etcd" "github.com/milvus-io/milvus/pkg/v2/util/paramtable" ) @@ -144,8 +145,15 @@ func (suite *SegmentCheckerTestSuite) TestLoadSegments() { checker.targetMgr.UpdateCollectionNextTarget(ctx, int64(1)) // set dist - checker.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - checker.dist.LeaderViewManager.Update(2, utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{}, map[int64]*meta.Segment{})) + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ID: 2, CollectionID: 1, Channel: "test-insert-channel", Version: 1, Status: &querypb.LeaderViewStatus{Serviceable: true}}, + }) tasks := checker.Check(context.TODO()) suite.Len(tasks, 1) @@ -234,9 +242,16 @@ func (suite *SegmentCheckerTestSuite) TestReleaseSegments() { checker.targetMgr.UpdateCollectionNextTarget(ctx, int64(1)) // set dist - checker.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - checker.dist.LeaderViewManager.Update(2, utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{}, map[int64]*meta.Segment{})) checker.dist.SegmentDistManager.Update(1, utils.CreateTestSegment(1, 1, 2, 1, 1, "test-insert-channel")) + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ID: 2, CollectionID: 1, Channel: "test-insert-channel", Version: 1, Status: &querypb.LeaderViewStatus{Serviceable: true}}, + }) tasks := checker.Check(context.TODO()) suite.Len(tasks, 1) @@ -276,10 +291,17 @@ func (suite *SegmentCheckerTestSuite) TestReleaseRepeatedSegments() { checker.targetMgr.UpdateCollectionNextTarget(ctx, int64(1)) // set dist - checker.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - checker.dist.LeaderViewManager.Update(2, utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{1: 2}, map[int64]*meta.Segment{})) checker.dist.SegmentDistManager.Update(1, utils.CreateTestSegment(1, 1, 1, 1, 1, "test-insert-channel")) checker.dist.SegmentDistManager.Update(2, utils.CreateTestSegment(1, 1, 1, 1, 2, "test-insert-channel")) + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{1: 2}, map[int64]*meta.Segment{}), + }) tasks := checker.Check(context.TODO()) suite.Len(tasks, 1) @@ -293,7 +315,15 @@ func (suite *SegmentCheckerTestSuite) TestReleaseRepeatedSegments() { suite.Equal(tasks[0].Priority(), task.TaskPriorityLow) // test less version exist on leader - checker.dist.LeaderViewManager.Update(2, utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{1: 1}, map[int64]*meta.Segment{})) + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{1: 1}, map[int64]*meta.Segment{}), + }) tasks = checker.Check(context.TODO()) suite.Len(tasks, 0) } @@ -335,9 +365,16 @@ func (suite *SegmentCheckerTestSuite) TestReleaseDirtySegments() { checker.targetMgr.UpdateCollectionNextTarget(ctx, int64(1)) // set dist - checker.dist.ChannelDistManager.Update(2, utils.CreateTestChannel(1, 2, 1, "test-insert-channel")) - checker.dist.LeaderViewManager.Update(2, utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{1: 2}, map[int64]*meta.Segment{})) checker.dist.SegmentDistManager.Update(2, utils.CreateTestSegment(1, 1, 1, 1, 1, "test-insert-channel")) + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{1: 2}, map[int64]*meta.Segment{}), + }) tasks := checker.Check(context.TODO()) suite.Len(tasks, 1) @@ -379,22 +416,27 @@ func (suite *SegmentCheckerTestSuite) TestSkipReleaseSealedSegments() { checker.targetMgr.UpdateCollectionNextTarget(ctx, collectionID) readableVersion := checker.targetMgr.GetCollectionTargetVersion(ctx, collectionID, meta.CurrentTarget) - // test less target version exist on leader,meet segment doesn't exit in target, segment should be released + // test less target version exist on leader,meet segment doesn't exit in target, segment shouldn't be released nodeID := int64(2) segmentID := int64(1) - checker.dist.ChannelDistManager.Update(nodeID, utils.CreateTestChannel(collectionID, nodeID, segmentID, "test-insert-channel")) - view := utils.CreateTestLeaderView(nodeID, collectionID, "test-insert-channel", map[int64]int64{segmentID: 2}, map[int64]*meta.Segment{}) - view.TargetVersion = readableVersion - 1 - checker.dist.LeaderViewManager.Update(nodeID, view) checker.dist.SegmentDistManager.Update(nodeID, utils.CreateTestSegment(collectionID, partitionID, segmentID, nodeID, 2, "test-insert-channel")) - tasks := checker.Check(context.TODO()) - suite.Len(tasks, 0) + checker.dist.ChannelDistManager.Update(nodeID, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: nodeID, + CollectionID: collectionID, + Channel: "test-insert-channel", + TargetVersion: readableVersion - 1, + Segments: map[int64]*querypb.SegmentDist{segmentID: {NodeID: nodeID}}, + }, + }) - // test leader's target version update to latest,meet segment doesn't exit in target, segment should be released - view = utils.CreateTestLeaderView(nodeID, collectionID, "test-insert-channel", map[int64]int64{1: 3}, map[int64]*meta.Segment{}) - view.TargetVersion = readableVersion - checker.dist.LeaderViewManager.Update(2, view) - tasks = checker.Check(context.TODO()) + tasks := checker.Check(context.TODO()) suite.Len(tasks, 1) suite.Len(tasks[0].Actions(), 1) action, ok := tasks[0].Actions()[0].(*task.SegmentAction) @@ -404,21 +446,6 @@ func (suite *SegmentCheckerTestSuite) TestSkipReleaseSealedSegments() { suite.EqualValues(segmentID, action.GetSegmentID()) suite.EqualValues(nodeID, action.Node()) suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal) - - // test leader with initialTargetVersion, meet segment doesn't exit in target, segment should be released - view = utils.CreateTestLeaderView(nodeID, collectionID, "test-insert-channel", map[int64]int64{1: 3}, map[int64]*meta.Segment{}) - view.TargetVersion = initialTargetVersion - checker.dist.LeaderViewManager.Update(2, view) - tasks = checker.Check(context.TODO()) - suite.Len(tasks, 1) - suite.Len(tasks[0].Actions(), 1) - action, ok = tasks[0].Actions()[0].(*task.SegmentAction) - suite.True(ok) - suite.EqualValues(1, tasks[0].ReplicaID()) - suite.Equal(task.ActionTypeReduce, action.Type()) - suite.EqualValues(segmentID, action.GetSegmentID()) - suite.EqualValues(nodeID, action.Node()) - suite.Equal(tasks[0].Priority(), task.TaskPriorityNormal) } func (suite *SegmentCheckerTestSuite) TestReleaseGrowingSegments() { @@ -458,13 +485,25 @@ func (suite *SegmentCheckerTestSuite) TestReleaseGrowingSegments() { growingSegments[4] = utils.CreateTestSegment(1, 1, 4, 2, 1, "test-insert-channel") growingSegments[4].SegmentInfo.StartPosition = &msgpb.MsgPosition{Timestamp: 11} - dmChannel := utils.CreateTestChannel(1, 2, 1, "test-insert-channel") - dmChannel.UnflushedSegmentIds = []int64{2, 3} - checker.dist.ChannelDistManager.Update(2, dmChannel) - view := utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{3: 2}, growingSegments) - view.TargetVersion = checker.targetMgr.GetCollectionTargetVersion(ctx, int64(1), meta.CurrentTarget) - checker.dist.LeaderViewManager.Update(2, view) checker.dist.SegmentDistManager.Update(2, utils.CreateTestSegment(1, 1, 3, 2, 2, "test-insert-channel")) + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + UnflushedSegmentIds: []int64{2, 3}, + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: checker.targetMgr.GetCollectionTargetVersion(ctx, int64(1), meta.CurrentTarget), + Segments: map[int64]*querypb.SegmentDist{3: {NodeID: 2}}, + GrowingSegments: growingSegments, + Status: &querypb.LeaderViewStatus{Serviceable: true}, + }, + }) tasks := checker.Check(context.TODO()) suite.Len(tasks, 2) @@ -524,13 +563,25 @@ func (suite *SegmentCheckerTestSuite) TestReleaseCompactedGrowingSegments() { growingSegments[4] = utils.CreateTestSegment(1, 1, 4, 2, 1, "test-insert-channel") growingSegments[4].SegmentInfo.StartPosition = &msgpb.MsgPosition{Timestamp: 11} - dmChannel := utils.CreateTestChannel(1, 2, 1, "test-insert-channel") - dmChannel.UnflushedSegmentIds = []int64{2, 3} - checker.dist.ChannelDistManager.Update(2, dmChannel) - view := utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{3: 2}, growingSegments) - view.TargetVersion = checker.targetMgr.GetCollectionTargetVersion(ctx, int64(1), meta.CurrentTarget) - checker.dist.LeaderViewManager.Update(2, view) checker.dist.SegmentDistManager.Update(2, utils.CreateTestSegment(1, 1, 3, 2, 2, "test-insert-channel")) + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + UnflushedSegmentIds: []int64{2, 3}, + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: checker.targetMgr.GetCollectionTargetVersion(ctx, int64(1), meta.CurrentTarget), + Segments: map[int64]*querypb.SegmentDist{3: {NodeID: 2}}, + GrowingSegments: growingSegments, + Status: &querypb.LeaderViewStatus{Serviceable: true}, + }, + }) tasks := checker.Check(context.TODO()) suite.Len(tasks, 1) @@ -572,18 +623,45 @@ func (suite *SegmentCheckerTestSuite) TestSkipReleaseGrowingSegments() { growingSegments[2] = utils.CreateTestSegment(1, 1, 2, 2, 0, "test-insert-channel") growingSegments[2].SegmentInfo.StartPosition = &msgpb.MsgPosition{Timestamp: 2} - dmChannel := utils.CreateTestChannel(1, 2, 1, "test-insert-channel") - dmChannel.UnflushedSegmentIds = []int64{2, 3} - checker.dist.ChannelDistManager.Update(2, dmChannel) - view := utils.CreateTestLeaderView(2, 1, "test-insert-channel", map[int64]int64{}, growingSegments) - view.TargetVersion = checker.targetMgr.GetCollectionTargetVersion(ctx, int64(1), meta.CurrentTarget) - 1 - checker.dist.LeaderViewManager.Update(2, view) + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + UnflushedSegmentIds: []int64{2, 3}, + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: checker.targetMgr.GetCollectionTargetVersion(ctx, int64(1), meta.CurrentTarget) - 1, + Segments: map[int64]*querypb.SegmentDist{3: {NodeID: 2}}, + GrowingSegments: growingSegments, + Status: &querypb.LeaderViewStatus{Serviceable: true}, + }, + }) tasks := checker.Check(context.TODO()) suite.Len(tasks, 0) - view.TargetVersion = checker.targetMgr.GetCollectionTargetVersion(ctx, int64(1), meta.CurrentTarget) - checker.dist.LeaderViewManager.Update(2, view) + checker.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 1, + ChannelName: "test-insert-channel", + UnflushedSegmentIds: []int64{2, 3}, + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 1, + Channel: "test-insert-channel", + TargetVersion: checker.targetMgr.GetCollectionTargetVersion(ctx, int64(1), meta.CurrentTarget), + Segments: map[int64]*querypb.SegmentDist{3: {NodeID: 2}}, + GrowingSegments: growingSegments, + }, + }) tasks = checker.Check(context.TODO()) suite.Len(tasks, 1) suite.Len(tasks[0].Actions(), 1) diff --git a/internal/querycoordv2/dist/dist_controller.go b/internal/querycoordv2/dist/dist_controller.go index 484713094db39..e8685b2ad6a50 100644 --- a/internal/querycoordv2/dist/dist_controller.go +++ b/internal/querycoordv2/dist/dist_controller.go @@ -36,14 +36,15 @@ type Controller interface { } type ControllerImpl struct { - mu sync.RWMutex - handlers map[int64]*distHandler - client session.Cluster - nodeManager *session.NodeManager - dist *meta.DistributionManager - targetMgr meta.TargetManagerInterface - scheduler task.Scheduler - syncTargetVersionFn TriggerUpdateTargetVersion + mu sync.RWMutex + handlers map[int64]*distHandler + client session.Cluster + nodeManager *session.NodeManager + dist *meta.DistributionManager + targetMgr meta.TargetManagerInterface + scheduler task.Scheduler + + notifyFunc NotifyDelegatorChanges } func (dc *ControllerImpl) StartDistInstance(ctx context.Context, nodeID int64) { @@ -53,7 +54,7 @@ func (dc *ControllerImpl) StartDistInstance(ctx context.Context, nodeID int64) { log.Info("node has started", zap.Int64("nodeID", nodeID)) return } - h := newDistHandler(ctx, nodeID, dc.client, dc.nodeManager, dc.scheduler, dc.dist, dc.targetMgr, dc.syncTargetVersionFn) + h := newDistHandler(ctx, nodeID, dc.client, dc.nodeManager, dc.scheduler, dc.dist, dc.targetMgr, dc.notifyFunc) dc.handlers[nodeID] = h } @@ -101,15 +102,15 @@ func NewDistController( dist *meta.DistributionManager, targetMgr meta.TargetManagerInterface, scheduler task.Scheduler, - syncTargetVersionFn TriggerUpdateTargetVersion, + notifyFunc NotifyDelegatorChanges, ) *ControllerImpl { return &ControllerImpl{ - handlers: make(map[int64]*distHandler), - client: client, - nodeManager: nodeManager, - dist: dist, - targetMgr: targetMgr, - scheduler: scheduler, - syncTargetVersionFn: syncTargetVersionFn, + handlers: make(map[int64]*distHandler), + client: client, + nodeManager: nodeManager, + dist: dist, + targetMgr: targetMgr, + scheduler: scheduler, + notifyFunc: notifyFunc, } } diff --git a/internal/querycoordv2/dist/dist_controller_test.go b/internal/querycoordv2/dist/dist_controller_test.go index f05041edf39da..06a335adc391f 100644 --- a/internal/querycoordv2/dist/dist_controller_test.go +++ b/internal/querycoordv2/dist/dist_controller_test.go @@ -81,8 +81,8 @@ func (suite *DistControllerTestSuite) SetupTest() { targetManager := meta.NewTargetManager(suite.broker, suite.meta) suite.mockScheduler = task.NewMockScheduler(suite.T()) suite.mockScheduler.EXPECT().GetExecutedFlag(mock.Anything).Return(nil).Maybe() - syncTargetVersionFn := func(collectionID int64) {} - suite.controller = NewDistController(suite.mockCluster, suite.nodeMgr, distManager, targetManager, suite.mockScheduler, syncTargetVersionFn) + + suite.controller = NewDistController(suite.mockCluster, suite.nodeMgr, distManager, targetManager, suite.mockScheduler, func(collectionID ...int64) {}) } func (suite *DistControllerTestSuite) TearDownSuite() { diff --git a/internal/querycoordv2/dist/dist_handler.go b/internal/querycoordv2/dist/dist_handler.go index 00b8cdf31fa64..47d7bd3e92416 100644 --- a/internal/querycoordv2/dist/dist_handler.go +++ b/internal/querycoordv2/dist/dist_handler.go @@ -27,6 +27,7 @@ import ( "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/querycoordv2/meta" . "github.com/milvus-io/milvus/internal/querycoordv2/params" "github.com/milvus-io/milvus/internal/querycoordv2/session" @@ -44,6 +45,8 @@ import ( type TriggerUpdateTargetVersion = func(collectionID int64) +type NotifyDelegatorChanges = func(collectionID ...int64) + type distHandler struct { nodeID int64 c chan struct{} @@ -57,7 +60,7 @@ type distHandler struct { stopOnce sync.Once lastUpdateTs int64 - syncTargetVersionFn TriggerUpdateTargetVersion + notifyFunc NotifyDelegatorChanges } func (dh *distHandler) start(ctx context.Context) { @@ -140,7 +143,6 @@ func (dh *distHandler) handleDistResp(ctx context.Context, resp *querypb.GetData ) dh.updateSegmentsDistribution(ctx, resp) dh.updateChannelsDistribution(ctx, resp) - dh.updateLeaderView(ctx, resp) } if dispatchTask { @@ -148,6 +150,10 @@ func (dh *distHandler) handleDistResp(ctx context.Context, resp *querypb.GetData } } +func (dh *distHandler) SetNotifyFunc(notifyFunc NotifyDelegatorChanges) { + dh.notifyFunc = notifyFunc +} + func (dh *distHandler) updateSegmentsDistribution(ctx context.Context, resp *querypb.GetDataDistributionResponse) { updates := make([]*meta.Segment, 0, len(resp.GetSegments())) for _, s := range resp.GetSegments() { @@ -181,120 +187,129 @@ func (dh *distHandler) updateSegmentsDistribution(ctx context.Context, resp *que } func (dh *distHandler) updateChannelsDistribution(ctx context.Context, resp *querypb.GetDataDistributionResponse) { + channelMap := lo.SliceToMap(resp.GetChannels(), func(ch *querypb.ChannelVersionInfo) (string, *querypb.ChannelVersionInfo) { + return ch.GetChannel(), ch + }) + updates := make([]*meta.DmChannel, 0, len(resp.GetChannels())) - for _, ch := range resp.GetChannels() { - channelInfo := dh.target.GetDmChannel(ctx, ch.GetCollection(), ch.GetChannel(), meta.CurrentTarget) - var channel *meta.DmChannel - if channelInfo == nil { - channel = &meta.DmChannel{ - VchannelInfo: &datapb.VchannelInfo{ - ChannelName: ch.GetChannel(), - CollectionID: ch.GetCollection(), - }, - Node: resp.GetNodeID(), - Version: ch.GetVersion(), - } + for _, lview := range resp.GetLeaderViews() { + channel, ok := channelMap[lview.GetChannel()] + if !ok { + // unreachable path, querynode should return leader view and channel dist at same time + log.Ctx(ctx).WithRateGroup("distHandler.updateChannelsDistribution", 1, 60). + RatedInfo(30, "channel not found in distribution", + zap.Int64("collectionID", lview.GetCollection()), + zap.String("channel", lview.GetChannel())) + continue + } + delegatorVersion := channel.GetVersion() + + // Get or create channel info + collectionID := lview.GetCollection() + channelName := lview.GetChannel() + channelInfo := dh.target.GetDmChannel(ctx, collectionID, channelName, meta.CurrentTarget) + var vChannelInfo *datapb.VchannelInfo + if channelInfo != nil { + vChannelInfo = channelInfo.VchannelInfo } else { - channel = &meta.DmChannel{ - VchannelInfo: channelInfo.VchannelInfo, - Node: resp.GetNodeID(), - Version: ch.GetVersion(), + vChannelInfo = &datapb.VchannelInfo{ + ChannelName: channelName, + CollectionID: collectionID, } } - updates = append(updates, channel) - } - - dh.dist.ChannelDistManager.Update(resp.GetNodeID(), updates...) -} -func (dh *distHandler) updateLeaderView(ctx context.Context, resp *querypb.GetDataDistributionResponse) { - updates := make([]*meta.LeaderView, 0, len(resp.GetLeaderViews())) - - channels := lo.SliceToMap(resp.GetChannels(), func(channel *querypb.ChannelVersionInfo) (string, *querypb.ChannelVersionInfo) { - return channel.GetChannel(), channel - }) - - collectionsToSync := typeutil.NewUniqueSet() - for _, lview := range resp.GetLeaderViews() { - segments := make(map[int64]*meta.Segment) - for ID, position := range lview.GrowingSegments { - // To maintain compatibility with older versions of QueryNode, - // QueryCoord should neither process nor interact with L0 segments. - segmentInfo := dh.target.GetSealedSegment(ctx, lview.GetCollection(), ID, meta.CurrentTargetFirst) - if segmentInfo != nil && segmentInfo.GetLevel() == datapb.SegmentLevel_L0 { - continue - } - segments[ID] = &meta.Segment{ + // Pre-allocate growing segments map + growings := lo.MapValues(lview.GetGrowingSegments(), func(position *msgpb.MsgPosition, id int64) *meta.Segment { + return &meta.Segment{ SegmentInfo: &datapb.SegmentInfo{ - ID: ID, - CollectionID: lview.GetCollection(), + ID: id, + CollectionID: collectionID, StartPosition: position, - InsertChannel: lview.GetChannel(), + InsertChannel: channelName, }, - Node: resp.NodeID, + Node: resp.GetNodeID(), } - } + }) - var version int64 - channel, ok := channels[lview.GetChannel()] - if ok { - version = channel.GetVersion() + // Update DmChannel and register shard leader in same loop + dmChannel := &meta.DmChannel{ + VchannelInfo: vChannelInfo, + Node: resp.NodeID, + Version: delegatorVersion, + View: &meta.LeaderView{ + ID: resp.NodeID, + CollectionID: collectionID, + Channel: channelName, + Version: delegatorVersion, + Segments: lview.GetSegmentDist(), + GrowingSegments: growings, + NumOfGrowingRows: lview.GetNumOfGrowingRows(), + PartitionStatsVersions: lview.PartitionStatsVersions, + TargetVersion: lview.GetTargetVersion(), + Status: lview.GetStatus(), + }, } + updates = append(updates, dmChannel) - view := &meta.LeaderView{ - ID: resp.GetNodeID(), - CollectionID: lview.GetCollection(), - Channel: lview.GetChannel(), - Version: version, - Segments: lview.GetSegmentDist(), - GrowingSegments: segments, - TargetVersion: lview.TargetVersion, - NumOfGrowingRows: lview.GetNumOfGrowingRows(), - PartitionStatsVersions: lview.PartitionStatsVersions, + serviceable := checkDelegatorServiceable(ctx, dh, dmChannel.View) + // trigger pull next target until shard leader is ready + if !serviceable { + dh.lastUpdateTs = 0 } - updates = append(updates, view) - - // check leader serviceable - if err := utils.CheckDelegatorDataReady(dh.nodeManager, dh.target, view, meta.CurrentTarget); err != nil { - view.UnServiceableError = err - log.Ctx(ctx). - WithRateGroup(fmt.Sprintf("distHandler.updateLeaderView.%s", view.Channel), 1, 60). - RatedInfo(10, "leader is not available due to distribution not ready", - zap.Int64("collectionID", view.CollectionID), - zap.Int64("nodeID", view.ID), - zap.String("channel", view.Channel), - zap.Error(err)) - continue + } + + newLeaderOnNode := dh.dist.ChannelDistManager.Update(resp.GetNodeID(), updates...) + if dh.notifyFunc != nil { + collectionIDs := typeutil.NewUniqueSet() + for _, ch := range newLeaderOnNode { + collectionIDs.Insert(ch.VchannelInfo.CollectionID) } + dh.notifyFunc(collectionIDs.Collect()...) + } +} - // if target version hasn't been synced, delegator will get empty readable segment list - // so shard leader should be unserviceable until target version is synced - currentTargetVersion := dh.target.GetCollectionTargetVersion(ctx, lview.GetCollection(), meta.CurrentTarget) - if lview.TargetVersion <= 0 { - err := merr.WrapErrServiceInternal(fmt.Sprintf("target version mismatch, collection: %d, channel: %s, current target version: %v, leader version: %v", - lview.GetCollection(), lview.GetChannel(), currentTargetVersion, lview.TargetVersion)) +func checkDelegatorServiceable(ctx context.Context, dh *distHandler, view *meta.LeaderView) bool { + log := log.Ctx(ctx). + WithRateGroup(fmt.Sprintf("distHandler.updateChannelsDistribution.%s", view.Channel), 1, 60). + With( + zap.Int64("nodeID", view.ID), + zap.String("channel", view.Channel), + ) - view.UnServiceableError = err - // make dist handler pull next distribution until all delegator is serviceable - dh.lastUpdateTs = 0 - collectionsToSync.Insert(lview.Collection) - log.Ctx(ctx). - WithRateGroup(fmt.Sprintf("distHandler.updateLeaderView.%s", view.Channel), 1, 60). - RatedInfo(10, "leader is not available due to target version not ready", - zap.Int64("collectionID", view.CollectionID), - zap.Int64("nodeID", view.ID), - zap.String("channel", view.Channel), - zap.Error(err)) + if status := view.Status; status != nil { + if !status.GetServiceable() { + log.RatedInfo(10, "delegator is not serviceable", zap.Int64("queryViewVersion", view.TargetVersion)) + return false } + return true } - dh.dist.LeaderViewManager.Update(resp.GetNodeID(), updates...) + // check leader data ready for version before 2.5.8 + if err := utils.CheckDelegatorDataReady(dh.nodeManager, dh.target, view, meta.CurrentTarget); err != nil { + log.RatedInfo(10, "delegator is not serviceable due to distribution not ready", zap.Error(err)) + view.Status = &querypb.LeaderViewStatus{ + Serviceable: false, + } + return false + } - // segment and channel already loaded, trigger target observer to update - collectionsToSync.Range(func(collection int64) bool { - dh.syncTargetVersionFn(collection) - return true - }) + // if target version hasn't been synced, delegator will get empty readable segment list + // so shard leader should be unserviceable until target version is synced + currentTargetVersion := dh.target.GetCollectionTargetVersion(ctx, view.CollectionID, meta.CurrentTarget) + if view.TargetVersion <= 0 { + log.RatedInfo(10, "delegator is not serviceable due to target version not ready", + zap.Int64("currentTargetVersion", currentTargetVersion), + zap.Int64("leaderTargetVersion", view.TargetVersion)) + view.Status = &querypb.LeaderViewStatus{ + Serviceable: false, + } + return false + } + + view.Status = &querypb.LeaderViewStatus{ + Serviceable: true, + } + return true } func (dh *distHandler) getDistribution(ctx context.Context) (*querypb.GetDataDistributionResponse, error) { @@ -337,17 +352,17 @@ func newDistHandler( scheduler task.Scheduler, dist *meta.DistributionManager, targetMgr meta.TargetManagerInterface, - syncTargetVersionFn TriggerUpdateTargetVersion, + notifyFunc NotifyDelegatorChanges, ) *distHandler { h := &distHandler{ - nodeID: nodeID, - c: make(chan struct{}), - client: client, - nodeManager: nodeManager, - scheduler: scheduler, - dist: dist, - target: targetMgr, - syncTargetVersionFn: syncTargetVersionFn, + nodeID: nodeID, + c: make(chan struct{}), + client: client, + nodeManager: nodeManager, + scheduler: scheduler, + dist: dist, + target: targetMgr, + notifyFunc: notifyFunc, } h.wg.Add(1) go h.start(ctx) diff --git a/internal/querycoordv2/dist/dist_handler_test.go b/internal/querycoordv2/dist/dist_handler_test.go index c34da497ff623..05fcd76d3bdc3 100644 --- a/internal/querycoordv2/dist/dist_handler_test.go +++ b/internal/querycoordv2/dist/dist_handler_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/errors" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" + "go.uber.org/atomic" "github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/session" @@ -49,8 +50,7 @@ type DistHandlerSuite struct { executedFlagChan chan struct{} dist *meta.DistributionManager target *meta.MockTargetManager - - handler *distHandler + handler *distHandler } func (suite *DistHandlerSuite) SetupSuite() { @@ -114,8 +114,7 @@ func (suite *DistHandlerSuite) TestBasic() { LastModifyTs: 1, }, nil) - syncTargetVersionFn := func(collectionID int64) {} - suite.handler = newDistHandler(suite.ctx, suite.nodeID, suite.client, suite.nodeManager, suite.scheduler, suite.dist, suite.target, syncTargetVersionFn) + suite.handler = newDistHandler(suite.ctx, suite.nodeID, suite.client, suite.nodeManager, suite.scheduler, suite.dist, suite.target, func(collectionID ...int64) {}) defer suite.handler.stop() time.Sleep(3 * time.Second) @@ -135,8 +134,7 @@ func (suite *DistHandlerSuite) TestGetDistributionFailed() { })) suite.client.EXPECT().GetDataDistribution(mock.Anything, mock.Anything, mock.Anything).Return(nil, errors.New("fake error")) - syncTargetVersionFn := func(collectionID int64) {} - suite.handler = newDistHandler(suite.ctx, suite.nodeID, suite.client, suite.nodeManager, suite.scheduler, suite.dist, suite.target, syncTargetVersionFn) + suite.handler = newDistHandler(suite.ctx, suite.nodeID, suite.client, suite.nodeManager, suite.scheduler, suite.dist, suite.target, func(collectionID ...int64) {}) defer suite.handler.stop() time.Sleep(3 * time.Second) @@ -184,13 +182,85 @@ func (suite *DistHandlerSuite) TestForcePullDist() { LastModifyTs: 1, }, nil) suite.executedFlagChan <- struct{}{} - syncTargetVersionFn := func(collectionID int64) {} - suite.handler = newDistHandler(suite.ctx, suite.nodeID, suite.client, suite.nodeManager, suite.scheduler, suite.dist, suite.target, syncTargetVersionFn) + suite.handler = newDistHandler(suite.ctx, suite.nodeID, suite.client, suite.nodeManager, suite.scheduler, suite.dist, suite.target, func(collectionID ...int64) {}) defer suite.handler.stop() time.Sleep(300 * time.Millisecond) } +func (suite *DistHandlerSuite) TestHandlerWithSyncDelegatorChanges() { + if suite.dispatchMockCall != nil { + suite.dispatchMockCall.Unset() + suite.dispatchMockCall = nil + } + + suite.target.EXPECT().GetSealedSegmentsByChannel(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(map[int64]*datapb.SegmentInfo{}).Maybe() + suite.dispatchMockCall = suite.scheduler.EXPECT().Dispatch(mock.Anything).Maybe() + suite.nodeManager.Add(session.NewNodeInfo(session.ImmutableNodeInfo{ + NodeID: 1, + Address: "localhost", + Hostname: "localhost", + })) + + // Test scenario: update segments and channels distribution without replicaMgr + suite.client.EXPECT().GetDataDistribution(mock.Anything, mock.Anything, mock.Anything).Return(&querypb.GetDataDistributionResponse{ + Status: merr.Success(), + NodeID: 1, + Channels: []*querypb.ChannelVersionInfo{ + { + Channel: "test-channel-1", + Collection: 1, + Version: 1, + }, + }, + Segments: []*querypb.SegmentVersionInfo{ + { + ID: 1, + Collection: 1, + Partition: 1, + Channel: "test-channel-1", + Version: 1, + }, + }, + LeaderViews: []*querypb.LeaderView{ + { + Collection: 1, + Channel: "test-channel-1", + TargetVersion: 1011, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + }, + LastModifyTs: 2, // Different from previous test to ensure update happens + }, nil) + + notifyCounter := atomic.NewInt32(0) + notifyFunc := func(collectionID ...int64) { + suite.Require().Equal(1, len(collectionID)) + suite.Require().Equal(int64(1), collectionID[0]) + notifyCounter.Inc() + } + + suite.handler = newDistHandler(suite.ctx, suite.nodeID, suite.client, suite.nodeManager, suite.scheduler, suite.dist, suite.target, notifyFunc) + defer suite.handler.stop() + + // Wait for distribution to be processed + time.Sleep(1000 * time.Millisecond) + + // Verify that the distributions were updated correctly + segments := suite.dist.SegmentDistManager.GetByFilter(meta.WithNodeID(1)) + suite.Require().Equal(1, len(segments)) + suite.Require().Equal(int64(1), segments[0].SegmentInfo.ID) + + channels := suite.dist.ChannelDistManager.GetByFilter(meta.WithNodeID2Channel(1)) + suite.Require().Equal(1, len(channels)) + suite.Require().Equal("test-channel-1", channels[0].VchannelInfo.ChannelName) + + // Verify that the notification was called + suite.Require().Greater(notifyCounter.Load(), int32(0)) +} + func TestDistHandlerSuite(t *testing.T) { suite.Run(t, new(DistHandlerSuite)) } diff --git a/internal/querycoordv2/handlers.go b/internal/querycoordv2/handlers.go index e13eb273eb87e..bd2132acbf1eb 100644 --- a/internal/querycoordv2/handlers.go +++ b/internal/querycoordv2/handlers.go @@ -487,10 +487,10 @@ func (s *Server) fillReplicaInfo(ctx context.Context, replica *meta.Replica, wit } for _, channel := range channels { - leader, ok := s.dist.ChannelDistManager.GetShardLeader(replica, channel.GetChannelName()) + leader := s.dist.ChannelDistManager.GetShardLeader(channel.ChannelName, replica) var leaderInfo *session.NodeInfo - if ok { - leaderInfo = s.nodeMgr.Get(leader) + if leader != nil { + leaderInfo = s.nodeMgr.Get(leader.Node) } if leaderInfo == nil { log.Warn("failed to get shard leader for shard", @@ -501,10 +501,10 @@ func (s *Server) fillReplicaInfo(ctx context.Context, replica *meta.Replica, wit } shard := &milvuspb.ShardReplica{ - LeaderID: leader, + LeaderID: leader.Node, LeaderAddr: leaderInfo.Addr(), DmChannelName: channel.GetChannelName(), - NodeIds: []int64{leader}, + NodeIds: []int64{leader.Node}, } if withShardNodes { shardNodes := lo.FilterMap(segments, func(segment *meta.Segment, _ int) (int64, bool) { diff --git a/internal/querycoordv2/job/job_test.go b/internal/querycoordv2/job/job_test.go index 1a0146d790ff5..432ed7595e71a 100644 --- a/internal/querycoordv2/job/job_test.go +++ b/internal/querycoordv2/job/job_test.go @@ -1604,24 +1604,32 @@ func (suite *JobSuite) updateChannelDist(ctx context.Context, collection int64, segments := lo.Flatten(lo.Values(suite.segments[collection])) replicas := suite.meta.ReplicaManager.GetByCollection(ctx, collection) + targetVersion := suite.targetMgr.GetCollectionTargetVersion(ctx, collection, meta.CurrentTargetFirst) for _, replica := range replicas { if loaded { i := 0 for _, node := range replica.GetNodes() { - suite.dist.ChannelDistManager.Update(node, meta.DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: collection, - ChannelName: channels[i], - })) - suite.dist.LeaderViewManager.Update(node, &meta.LeaderView{ - ID: node, - CollectionID: collection, - Channel: channels[i], - Segments: lo.SliceToMap(segments, func(segment int64) (int64, *querypb.SegmentDist) { - return segment, &querypb.SegmentDist{ - NodeID: node, - Version: time.Now().Unix(), - } - }), + suite.dist.ChannelDistManager.Update(node, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: collection, + ChannelName: channels[i], + }, + Node: node, + View: &meta.LeaderView{ + ID: node, + CollectionID: collection, + Channel: channels[i], + Segments: lo.SliceToMap(segments, func(segment int64) (int64, *querypb.SegmentDist) { + return segment, &querypb.SegmentDist{ + NodeID: node, + Version: time.Now().Unix(), + } + }), + TargetVersion: targetVersion, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, }) i++ if i >= len(channels) { @@ -1631,7 +1639,6 @@ func (suite *JobSuite) updateChannelDist(ctx context.Context, collection int64, } else { for _, node := range replica.GetNodes() { suite.dist.ChannelDistManager.Update(node) - suite.dist.LeaderViewManager.Update(node) } } } diff --git a/internal/querycoordv2/meta/channel_dist_manager.go b/internal/querycoordv2/meta/channel_dist_manager.go index 64bb240aee9fe..96515e998f525 100644 --- a/internal/querycoordv2/meta/channel_dist_manager.go +++ b/internal/querycoordv2/meta/channel_dist_manager.go @@ -24,10 +24,48 @@ import ( "github.com/milvus-io/milvus/internal/util/metrics" "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/querypb" "github.com/milvus-io/milvus/pkg/v2/util/metricsinfo" "github.com/milvus-io/milvus/pkg/v2/util/typeutil" ) +type LeaderView struct { + ID int64 + CollectionID int64 + Channel string + Version int64 + Segments map[int64]*querypb.SegmentDist + GrowingSegments map[int64]*Segment + TargetVersion int64 + NumOfGrowingRows int64 + PartitionStatsVersions map[int64]int64 + Status *querypb.LeaderViewStatus +} + +func (view *LeaderView) Clone() *LeaderView { + segments := make(map[int64]*querypb.SegmentDist) + for k, v := range view.Segments { + segments[k] = v + } + + growings := make(map[int64]*Segment) + for k, v := range view.GrowingSegments { + growings[k] = v + } + + return &LeaderView{ + ID: view.ID, + CollectionID: view.CollectionID, + Channel: view.Channel, + Version: view.Version, + Segments: segments, + GrowingSegments: growings, + TargetVersion: view.TargetVersion, + NumOfGrowingRows: view.NumOfGrowingRows, + PartitionStatsVersions: view.PartitionStatsVersions, + } +} + type channelDistCriterion struct { nodeIDs typeutil.Set[int64] collectionID int64 @@ -116,6 +154,7 @@ type DmChannel struct { *datapb.VchannelInfo Node int64 Version int64 + View *LeaderView } func DmChannelFromVChannel(channel *datapb.VchannelInfo) *DmChannel { @@ -129,9 +168,23 @@ func (channel *DmChannel) Clone() *DmChannel { VchannelInfo: proto.Clone(channel.VchannelInfo).(*datapb.VchannelInfo), Node: channel.Node, Version: channel.Version, + View: &LeaderView{ + ID: channel.View.ID, + CollectionID: channel.View.CollectionID, + Channel: channel.View.Channel, + Version: channel.View.Version, + Status: proto.Clone(channel.View.Status).(*querypb.LeaderViewStatus), + }, } } +func (channel *DmChannel) IsServiceable() bool { + if channel.View == nil { + return false + } + return channel.View.Status.GetServiceable() +} + func newDmChannelMetricsFrom(channel *DmChannel) *metricsinfo.DmChannel { dmChannel := metrics.NewDMChannelFrom(channel.VchannelInfo) dmChannel.NodeID = channel.Node @@ -171,6 +224,15 @@ func composeNodeChannels(channels ...*DmChannel) nodeChannels { } } +type ChannelDistManagerInterface interface { + GetByFilter(filters ...ChannelDistFilter) []*DmChannel + GetByCollectionAndFilter(collectionID int64, filters ...ChannelDistFilter) []*DmChannel + Update(nodeID typeutil.UniqueID, channels ...*DmChannel) []*DmChannel + GetShardLeader(channelName string, replica *Replica) *DmChannel + GetChannelDist(collectionID int64) []*metricsinfo.DmChannel + GetLeaderView(collectionID int64) []*metricsinfo.LeaderView +} + type ChannelDistManager struct { rwmutex sync.RWMutex @@ -188,39 +250,6 @@ func NewChannelDistManager() *ChannelDistManager { } } -// todo by liuwei: should consider the case of duplicate leader exists -// GetShardLeader returns the node whthin the given replicaNodes and subscribing the given shard, -// returns (0, false) if not found. -func (m *ChannelDistManager) GetShardLeader(replica *Replica, shard string) (int64, bool) { - m.rwmutex.RLock() - defer m.rwmutex.RUnlock() - - for _, node := range replica.GetNodes() { - channels := m.channels[node] - _, ok := channels.nameChannel[shard] - if ok { - return node, true - } - } - - return 0, false -} - -// todo by liuwei: should consider the case of duplicate leader exists -func (m *ChannelDistManager) GetShardLeadersByReplica(replica *Replica) map[string]int64 { - m.rwmutex.RLock() - defer m.rwmutex.RUnlock() - - ret := make(map[string]int64) - for _, node := range replica.GetNodes() { - channels := m.channels[node] - for _, dmc := range channels.collChannels[replica.GetCollectionID()] { - ret[dmc.GetChannelName()] = node - } - } - return ret -} - // return all channels in list which match all given filters func (m *ChannelDistManager) GetByFilter(filters ...ChannelDistFilter) []*DmChannel { m.rwmutex.RLock() @@ -272,17 +301,23 @@ func (m *ChannelDistManager) GetByCollectionAndFilter(collectionID int64, filter return ret } -func (m *ChannelDistManager) Update(nodeID typeutil.UniqueID, channels ...*DmChannel) { +func (m *ChannelDistManager) Update(nodeID typeutil.UniqueID, channels ...*DmChannel) []*DmChannel { m.rwmutex.Lock() defer m.rwmutex.Unlock() + newServiceableChannels := make([]*DmChannel, 0) for _, channel := range channels { channel.Node = nodeID + + old, ok := m.channels[nodeID].nameChannel[channel.GetChannelName()] + if channel.IsServiceable() && (!ok || !old.IsServiceable()) { + newServiceableChannels = append(newServiceableChannels, channel) + } } m.channels[nodeID] = composeNodeChannels(channels...) - m.updateCollectionIndex() + return newServiceableChannels } // update secondary index for channel distribution @@ -300,6 +335,45 @@ func (m *ChannelDistManager) updateCollectionIndex() { } } +// GetShardLeader return the only one delegator leader which has the highest version in given replica +// if there is no serviceable leader, return the highest version leader +// With specific channel name and replica, return the only one delegator leader +func (m *ChannelDistManager) GetShardLeader(channelName string, replica *Replica) *DmChannel { + m.rwmutex.RLock() + defer m.rwmutex.RUnlock() + + channels := m.collectionIndex[replica.GetCollectionID()] + + var candidates *DmChannel + for _, channel := range channels { + if channel.GetChannelName() == channelName && replica.Contains(channel.Node) { + if candidates == nil { + candidates = channel + } else { + // Prioritize serviceability first, then version number + candidatesServiceable := candidates.IsServiceable() + channelServiceable := channel.IsServiceable() + + updateNeeded := false + switch { + case !candidatesServiceable && channelServiceable: + // Current candidate is not serviceable but new channel is + updateNeeded = true + case candidatesServiceable == channelServiceable && channel.Version > candidates.Version: + // Same service status but higher version + updateNeeded = true + } + + if updateNeeded { + candidates = channel + } + } + } + } + + return candidates +} + func (m *ChannelDistManager) GetChannelDist(collectionID int64) []*metricsinfo.DmChannel { m.rwmutex.RLock() defer m.rwmutex.RUnlock() @@ -321,3 +395,56 @@ func (m *ChannelDistManager) GetChannelDist(collectionID int64) []*metricsinfo.D } return ret } + +// GetLeaderView returns a slice of LeaderView objects, each representing the state of a leader node. +// It traverses the views map, converts each LeaderView to a metricsinfo.LeaderView, and collects them into a slice. +// The method locks the views map for reading to ensure thread safety. +func (m *ChannelDistManager) GetLeaderView(collectionID int64) []*metricsinfo.LeaderView { + m.rwmutex.RLock() + defer m.rwmutex.RUnlock() + + var ret []*metricsinfo.LeaderView + if collectionID > 0 { + if channels, ok := m.collectionIndex[collectionID]; ok { + for _, channel := range channels { + ret = append(ret, newMetricsLeaderViewFrom(channel.View)) + } + } + return ret + } + + for _, channels := range m.collectionIndex { + for _, channel := range channels { + ret = append(ret, newMetricsLeaderViewFrom(channel.View)) + } + } + return ret +} + +func newMetricsLeaderViewFrom(lv *LeaderView) *metricsinfo.LeaderView { + leaderView := &metricsinfo.LeaderView{ + LeaderID: lv.ID, + CollectionID: lv.CollectionID, + Channel: lv.Channel, + Version: lv.Version, + SealedSegments: make([]*metricsinfo.Segment, 0, len(lv.Segments)), + GrowingSegments: make([]*metricsinfo.Segment, 0, len(lv.GrowingSegments)), + TargetVersion: lv.TargetVersion, + NumOfGrowingRows: lv.NumOfGrowingRows, + } + + for segID, seg := range lv.Segments { + leaderView.SealedSegments = append(leaderView.SealedSegments, &metricsinfo.Segment{ + SegmentID: segID, + NodeID: seg.NodeID, + }) + } + + for _, seg := range lv.GrowingSegments { + leaderView.GrowingSegments = append(leaderView.GrowingSegments, &metricsinfo.Segment{ + SegmentID: seg.ID, + NodeID: seg.Node, + }) + } + return leaderView +} diff --git a/internal/querycoordv2/meta/channel_dist_manager_test.go b/internal/querycoordv2/meta/channel_dist_manager_test.go index 29431b3503f70..851e510770144 100644 --- a/internal/querycoordv2/meta/channel_dist_manager_test.go +++ b/internal/querycoordv2/meta/channel_dist_manager_test.go @@ -25,7 +25,6 @@ import ( "github.com/milvus-io/milvus/pkg/v2/proto/datapb" "github.com/milvus-io/milvus/pkg/v2/proto/querypb" "github.com/milvus-io/milvus/pkg/v2/util/metricsinfo" - "github.com/milvus-io/milvus/pkg/v2/util/typeutil" ) type ChannelDistManagerSuite struct { @@ -42,14 +41,40 @@ func (suite *ChannelDistManagerSuite) SetupSuite() { suite.collection = 10 suite.nodes = []int64{0, 1, 2} suite.channels = map[string]*DmChannel{ - "dmc0": DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: suite.collection, - ChannelName: "dmc0", - }), - "dmc1": DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: suite.collection, - ChannelName: "dmc1", - }), + "dmc0": { + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: "dmc0", + }, + Node: 0, + Version: 1, + View: &LeaderView{ + ID: 1, + CollectionID: suite.collection, + Channel: "dmc0", + Version: 1, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + }, + "dmc1": { + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: "dmc1", + }, + Node: 1, + Version: 1, + View: &LeaderView{ + ID: 1, + CollectionID: suite.collection, + Channel: "dmc1", + Version: 1, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + }, } } @@ -101,55 +126,6 @@ func (suite *ChannelDistManagerSuite) TestGetBy() { suite.Len(channels, 0) } -func (suite *ChannelDistManagerSuite) TestGetShardLeader() { - replicas := []*Replica{ - NewReplica( - &querypb.Replica{ - CollectionID: suite.collection, - }, - typeutil.NewUniqueSet(suite.nodes[0], suite.nodes[2]), - ), - NewReplica( - &querypb.Replica{ - CollectionID: suite.collection, - }, - typeutil.NewUniqueSet(suite.nodes[1]), - ), - } - - // Test on replica 0 - leader0, ok := suite.dist.GetShardLeader(replicas[0], "dmc0") - suite.True(ok) - suite.Equal(suite.nodes[0], leader0) - leader1, ok := suite.dist.GetShardLeader(replicas[0], "dmc1") - suite.True(ok) - suite.Equal(suite.nodes[2], leader1) - - // Test on replica 1 - leader0, ok = suite.dist.GetShardLeader(replicas[1], "dmc0") - suite.True(ok) - suite.Equal(suite.nodes[1], leader0) - leader1, ok = suite.dist.GetShardLeader(replicas[1], "dmc1") - suite.True(ok) - suite.Equal(suite.nodes[1], leader1) - - // Test no shard leader for given channel - _, ok = suite.dist.GetShardLeader(replicas[0], "invalid-shard") - suite.False(ok) - - // Test on replica 0 - leaders := suite.dist.GetShardLeadersByReplica(replicas[0]) - suite.Len(leaders, 2) - suite.Equal(leaders["dmc0"], suite.nodes[0]) - suite.Equal(leaders["dmc1"], suite.nodes[2]) - - // Test on replica 1 - leaders = suite.dist.GetShardLeadersByReplica(replicas[1]) - suite.Len(leaders, 2) - suite.Equal(leaders["dmc0"], suite.nodes[1]) - suite.Equal(leaders["dmc1"], suite.nodes[1]) -} - func (suite *ChannelDistManagerSuite) AssertNames(channels []*DmChannel, names ...string) bool { for _, channel := range channels { hasChannel := false @@ -188,22 +164,218 @@ func TestChannelDistManager(t *testing.T) { suite.Run(t, new(ChannelDistManagerSuite)) } +func TestDmChannelClone(t *testing.T) { + // Test that Clone properly copies the View field including Status + originalChannel := &DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 100, + ChannelName: "test-channel", + }, + Node: 1, + Version: 10, + View: &LeaderView{ + ID: 5, + CollectionID: 100, + Channel: "test-channel", + Version: 20, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + } + + clonedChannel := originalChannel.Clone() + + // Check all fields were properly cloned + assert.Equal(t, originalChannel.GetCollectionID(), clonedChannel.GetCollectionID()) + assert.Equal(t, originalChannel.GetChannelName(), clonedChannel.GetChannelName()) + assert.Equal(t, originalChannel.Node, clonedChannel.Node) + assert.Equal(t, originalChannel.Version, clonedChannel.Version) + + // Check that View was properly cloned + assert.NotNil(t, clonedChannel.View) + assert.Equal(t, originalChannel.View.ID, clonedChannel.View.ID) + assert.Equal(t, originalChannel.View.CollectionID, clonedChannel.View.CollectionID) + assert.Equal(t, originalChannel.View.Channel, clonedChannel.View.Channel) + assert.Equal(t, originalChannel.View.Version, clonedChannel.View.Version) + + // Check that Status was properly cloned + assert.NotNil(t, clonedChannel.View.Status) + assert.Equal(t, originalChannel.View.Status.GetServiceable(), clonedChannel.View.Status.GetServiceable()) + + // Verify that modifying the clone doesn't affect the original + clonedChannel.View.Status.Serviceable = false + assert.True(t, originalChannel.View.Status.GetServiceable()) + assert.False(t, clonedChannel.View.Status.GetServiceable()) +} + +func TestDmChannelIsServiceable(t *testing.T) { + // Test serviceable channel + serviceableChannel := &DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 100, + ChannelName: "serviceable", + }, + View: &LeaderView{ + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + } + assert.True(t, serviceableChannel.IsServiceable()) + + // Test non-serviceable channel + nonServiceableChannel := &DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 100, + ChannelName: "non-serviceable", + }, + View: &LeaderView{ + Status: &querypb.LeaderViewStatus{ + Serviceable: false, + }, + }, + } + assert.False(t, nonServiceableChannel.IsServiceable()) +} + +func (suite *ChannelDistManagerSuite) TestUpdateReturnsNewServiceableChannels() { + dist := NewChannelDistManager() + + // Create a non-serviceable channel + nonServiceableChannel := suite.channels["dmc0"].Clone() + nonServiceableChannel.View.Status.Serviceable = false + + // Update with non-serviceable channel first + newServiceableChannels := dist.Update(suite.nodes[0], nonServiceableChannel) + suite.Len(newServiceableChannels, 0, "No new serviceable channels should be returned") + + // Now update with a serviceable channel + serviceableChannel := nonServiceableChannel.Clone() + serviceableChannel.View.Status.Serviceable = true + + newServiceableChannels = dist.Update(suite.nodes[0], serviceableChannel) + suite.Len(newServiceableChannels, 1, "One new serviceable channel should be returned") + suite.Equal("dmc0", newServiceableChannels[0].GetChannelName()) + + // Update with same serviceable channel should not return it again + newServiceableChannels = dist.Update(suite.nodes[0], serviceableChannel) + suite.Len(newServiceableChannels, 0, "Already serviceable channel should not be returned") + + // Add a different channel that's serviceable + newChannel := suite.channels["dmc1"].Clone() + newChannel.View.Status.Serviceable = true + + newServiceableChannels = dist.Update(suite.nodes[0], serviceableChannel, newChannel) + suite.Len(newServiceableChannels, 1, "Only the new serviceable channel should be returned") + suite.Equal("dmc1", newServiceableChannels[0].GetChannelName()) +} + +func (suite *ChannelDistManagerSuite) TestGetShardLeader() { + dist := NewChannelDistManager() + + // Create a replica + replicaPB := &querypb.Replica{ + ID: 1, + CollectionID: suite.collection, + Nodes: []int64{0, 2}, + } + replica := NewReplica(replicaPB) + + // Create channels with different versions and serviceability + channel1Node0 := suite.channels["dmc0"].Clone() + channel1Node0.Version = 1 + channel1Node0.View.Status.Serviceable = false + + channel1Node2 := suite.channels["dmc0"].Clone() + channel1Node2.Node = 2 + channel1Node2.Version = 2 + channel1Node2.View.Status.Serviceable = false + + // Update with non-serviceable channels + dist.Update(0, channel1Node0) + dist.Update(2, channel1Node2) + + // Test getting leader with no serviceable channels - should return highest version + leader := dist.GetShardLeader("dmc0", replica) + suite.NotNil(leader) + suite.Equal(int64(2), leader.Node) + suite.Equal(int64(2), leader.Version) + + // Now make one channel serviceable + channel1Node0.View.Status.Serviceable = true + dist.Update(0, channel1Node0) + + // Test that serviceable channel is preferred even with lower version + leader = dist.GetShardLeader("dmc0", replica) + suite.NotNil(leader) + suite.Equal(int64(0), leader.Node) + suite.Equal(int64(1), leader.Version) + suite.True(leader.IsServiceable()) + + // Make both channels serviceable but with different versions + channel1Node2.View.Status.Serviceable = true + dist.Update(2, channel1Node2) + + // Test that highest version is chosen among serviceable channels + leader = dist.GetShardLeader("dmc0", replica) + suite.NotNil(leader) + suite.Equal(int64(2), leader.Node) + suite.Equal(int64(2), leader.Version) + suite.True(leader.IsServiceable()) + + // Test channel not in replica + // Create a new replica with different nodes + replicaPB = &querypb.Replica{ + ID: 1, + CollectionID: suite.collection, + Nodes: []int64{1}, + } + replicaWithDifferentNodes := NewReplica(replicaPB) + leader = dist.GetShardLeader("dmc0", replicaWithDifferentNodes) + suite.Nil(leader) + + // Test nonexistent channel + leader = dist.GetShardLeader("nonexistent", replica) + suite.Nil(leader) +} + func TestGetChannelDistJSON(t *testing.T) { manager := NewChannelDistManager() - channel1 := DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: 100, - ChannelName: "channel-1", - }) - channel1.Node = 1 - channel1.Version = 1 - - channel2 := DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: 200, - ChannelName: "channel-2", - }) - channel2.Node = 2 - channel2.Version = 1 - + channel1 := &DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 100, + ChannelName: "channel-1", + }, + Node: 1, + Version: 1, + View: &LeaderView{ + ID: 1, + CollectionID: 100, + Channel: "channel-1", + Version: 1, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + } + channel2 := &DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 200, + ChannelName: "channel-2", + }, + Node: 2, + Version: 1, + View: &LeaderView{ + ID: 1, + CollectionID: 200, + Channel: "channel-2", + Version: 1, + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, + } manager.Update(1, channel1) manager.Update(2, channel2) diff --git a/internal/querycoordv2/meta/dist_manager.go b/internal/querycoordv2/meta/dist_manager.go index 6ce7a54243068..b9879582ef1c3 100644 --- a/internal/querycoordv2/meta/dist_manager.go +++ b/internal/querycoordv2/meta/dist_manager.go @@ -25,16 +25,14 @@ import ( ) type DistributionManager struct { - *SegmentDistManager - *ChannelDistManager - *LeaderViewManager + SegmentDistManager SegmentDistManagerInterface + ChannelDistManager ChannelDistManagerInterface } func NewDistributionManager() *DistributionManager { return &DistributionManager{ SegmentDistManager: NewSegmentDistManager(), ChannelDistManager: NewChannelDistManager(), - LeaderViewManager: NewLeaderViewManager(), } } @@ -43,9 +41,9 @@ func NewDistributionManager() *DistributionManager { // If there are no segments, channels, or leader views, it returns an empty string. // In case of an error during JSON marshaling, it returns the error. func (dm *DistributionManager) GetDistributionJSON(collectionID int64) string { - segments := dm.GetSegmentDist(collectionID) - channels := dm.GetChannelDist(collectionID) - leaderView := dm.GetLeaderView(collectionID) + segments := dm.SegmentDistManager.GetSegmentDist(collectionID) + channels := dm.ChannelDistManager.GetChannelDist(collectionID) + leaderView := dm.ChannelDistManager.GetLeaderView(collectionID) dist := &metricsinfo.QueryCoordDist{ Segments: segments, diff --git a/internal/querycoordv2/meta/dist_manager_test.go b/internal/querycoordv2/meta/dist_manager_test.go index f7e2bae77daae..3cb479a24bc19 100644 --- a/internal/querycoordv2/meta/dist_manager_test.go +++ b/internal/querycoordv2/meta/dist_manager_test.go @@ -43,42 +43,36 @@ func TestGetDistributionJSON(t *testing.T) { manager.SegmentDistManager.Update(2, segment2) // Add some channels to the ChannelDistManager - channel1 := DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: 100, - ChannelName: "channel-1", + manager.ChannelDistManager.Update(1, &DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 100, + ChannelName: "channel-1", + }, + Node: 1, + Version: 1, + View: &LeaderView{ + ID: 1, + CollectionID: 100, + Channel: "channel-1", + Version: 1, + Segments: map[int64]*querypb.SegmentDist{1: {NodeID: 1}}, + }, }) - channel1.Node = 1 - channel1.Version = 1 - - channel2 := DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: 200, - ChannelName: "channel-2", + manager.ChannelDistManager.Update(2, &DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 200, + ChannelName: "channel-2", + }, + Node: 2, + Version: 1, + View: &LeaderView{ + ID: 2, + CollectionID: 200, + Channel: "channel-2", + Version: 1, + Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2}}, + }, }) - channel2.Node = 2 - channel2.Version = 1 - - manager.ChannelDistManager.Update(1, channel1) - manager.ChannelDistManager.Update(2, channel2) - - // Add some leader views to the LeaderViewManager - leaderView1 := &LeaderView{ - ID: 1, - CollectionID: 100, - Channel: "channel-1", - Version: 1, - Segments: map[int64]*querypb.SegmentDist{1: {NodeID: 1}}, - } - - leaderView2 := &LeaderView{ - ID: 2, - CollectionID: 200, - Channel: "channel-2", - Version: 1, - Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2}}, - } - - manager.LeaderViewManager.Update(1, leaderView1) - manager.LeaderViewManager.Update(2, leaderView2) // Call GetDistributionJSON jsonOutput := manager.GetDistributionJSON(0) diff --git a/internal/querycoordv2/meta/leader_view_manager.go b/internal/querycoordv2/meta/leader_view_manager.go deleted file mode 100644 index af4e02a08062a..0000000000000 --- a/internal/querycoordv2/meta/leader_view_manager.go +++ /dev/null @@ -1,377 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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 meta - -import ( - "sync" - - "github.com/samber/lo" - - "github.com/milvus-io/milvus/pkg/v2/proto/querypb" - "github.com/milvus-io/milvus/pkg/v2/util/metricsinfo" - "github.com/milvus-io/milvus/pkg/v2/util/typeutil" -) - -type lvCriterion struct { - nodeID int64 - channelName string - collectionID int64 - hasOtherFilter bool -} - -type LeaderViewFilter interface { - Match(*LeaderView) bool - AddFilter(*lvCriterion) -} - -type lvFilterFunc func(view *LeaderView) bool - -func (f lvFilterFunc) Match(view *LeaderView) bool { - return f(view) -} - -func (f lvFilterFunc) AddFilter(c *lvCriterion) { - c.hasOtherFilter = true -} - -type lvChannelNameFilter string - -func (f lvChannelNameFilter) Match(v *LeaderView) bool { - return v.Channel == string(f) -} - -func (f lvChannelNameFilter) AddFilter(c *lvCriterion) { - c.channelName = string(f) -} - -type lvNodeFilter int64 - -func (f lvNodeFilter) Match(v *LeaderView) bool { - return v.ID == int64(f) -} - -func (f lvNodeFilter) AddFilter(c *lvCriterion) { - c.nodeID = int64(f) -} - -type lvCollectionFilter int64 - -func (f lvCollectionFilter) Match(v *LeaderView) bool { - return v.CollectionID == int64(f) -} - -func (f lvCollectionFilter) AddFilter(c *lvCriterion) { - c.collectionID = int64(f) -} - -func WithNodeID2LeaderView(nodeID int64) LeaderViewFilter { - return lvNodeFilter(nodeID) -} - -func WithChannelName2LeaderView(channelName string) LeaderViewFilter { - return lvChannelNameFilter(channelName) -} - -func WithCollectionID2LeaderView(collectionID int64) LeaderViewFilter { - return lvCollectionFilter(collectionID) -} - -func WithReplica2LeaderView(replica *Replica) LeaderViewFilter { - return lvFilterFunc(func(view *LeaderView) bool { - if replica == nil { - return false - } - return replica.GetCollectionID() == view.CollectionID && replica.Contains(view.ID) - }) -} - -func WithSegment2LeaderView(segmentID int64, isGrowing bool) LeaderViewFilter { - return lvFilterFunc(func(view *LeaderView) bool { - if isGrowing { - _, ok := view.GrowingSegments[segmentID] - return ok - } - _, ok := view.Segments[segmentID] - return ok - }) -} - -func WithServiceable() LeaderViewFilter { - return lvFilterFunc(func(view *LeaderView) bool { - return view.UnServiceableError == nil - }) -} - -type LeaderView struct { - ID int64 - CollectionID int64 - Channel string - Version int64 - Segments map[int64]*querypb.SegmentDist - GrowingSegments map[int64]*Segment - TargetVersion int64 - NumOfGrowingRows int64 - PartitionStatsVersions map[int64]int64 - UnServiceableError error -} - -func (view *LeaderView) Clone() *LeaderView { - segments := make(map[int64]*querypb.SegmentDist) - for k, v := range view.Segments { - segments[k] = v - } - - growings := make(map[int64]*Segment) - for k, v := range view.GrowingSegments { - growings[k] = v - } - - return &LeaderView{ - ID: view.ID, - CollectionID: view.CollectionID, - Channel: view.Channel, - Version: view.Version, - Segments: segments, - GrowingSegments: growings, - TargetVersion: view.TargetVersion, - NumOfGrowingRows: view.NumOfGrowingRows, - PartitionStatsVersions: view.PartitionStatsVersions, - UnServiceableError: view.UnServiceableError, - } -} - -type nodeViews struct { - views []*LeaderView - // channel name => LeaderView - channelView map[string]*LeaderView - // collection id => leader views - collectionViews map[int64][]*LeaderView -} - -func (v nodeViews) Filter(criterion *lvCriterion, filters ...LeaderViewFilter) []*LeaderView { - mergedFilter := func(view *LeaderView) bool { - for _, filter := range filters { - if !filter.Match(view) { - return false - } - } - return true - } - - var views []*LeaderView - switch { - case criterion.channelName != "": - if view, ok := v.channelView[criterion.channelName]; ok { - views = append(views, view) - } - case criterion.collectionID != 0: - views = v.collectionViews[criterion.collectionID] - default: - views = v.views - } - - if criterion.hasOtherFilter { - views = lo.Filter(views, func(view *LeaderView, _ int) bool { - return mergedFilter(view) - }) - } - return views -} - -func composeNodeViews(views ...*LeaderView) nodeViews { - return nodeViews{ - views: views, - channelView: lo.SliceToMap(views, func(view *LeaderView) (string, *LeaderView) { - return view.Channel, view - }), - collectionViews: lo.GroupBy(views, func(view *LeaderView) int64 { - return view.CollectionID - }), - } -} - -type NotifyDelegatorChanges = func(collectionID ...int64) - -type LeaderViewManager struct { - rwmutex sync.RWMutex - views map[int64]nodeViews // LeaderID -> Views (one per shard) - notifyFunc NotifyDelegatorChanges -} - -func NewLeaderViewManager() *LeaderViewManager { - return &LeaderViewManager{ - views: make(map[int64]nodeViews), - } -} - -func (mgr *LeaderViewManager) SetNotifyFunc(notifyFunc NotifyDelegatorChanges) { - mgr.notifyFunc = notifyFunc -} - -// Update updates the leader's views, all views have to be with the same leader ID -func (mgr *LeaderViewManager) Update(leaderID int64, views ...*LeaderView) { - mgr.rwmutex.Lock() - defer mgr.rwmutex.Unlock() - - oldViews := make(map[string]*LeaderView, 0) - if _, ok := mgr.views[leaderID]; ok { - oldViews = mgr.views[leaderID].channelView - } - - newViews := lo.SliceToMap(views, func(v *LeaderView) (string, *LeaderView) { - return v.Channel, v - }) - - // update leader views - mgr.views[leaderID] = composeNodeViews(views...) - - // compute leader location change, find it's correspond collection - // 1. leader has been released from node - // 2. leader has been loaded to node - // 3. leader serviceable status changed - if mgr.notifyFunc != nil { - viewChanges := typeutil.NewUniqueSet() - for channel, oldView := range oldViews { - // if channel released from current node - if _, ok := newViews[channel]; !ok { - viewChanges.Insert(oldView.CollectionID) - } - } - - serviceableChange := func(old, new *LeaderView) bool { - if old == nil || new == nil { - return true - } - - return (old.UnServiceableError == nil) != (new.UnServiceableError == nil) - } - - for channel, newView := range newViews { - // if channel loaded to current node - if oldView, ok := oldViews[channel]; !ok || serviceableChange(oldView, newView) { - viewChanges.Insert(newView.CollectionID) - } - } - mgr.notifyFunc(viewChanges.Collect()...) - } -} - -func (mgr *LeaderViewManager) GetLeaderShardView(id int64, shard string) *LeaderView { - mgr.rwmutex.RLock() - defer mgr.rwmutex.RUnlock() - - return mgr.views[id].channelView[shard] -} - -func (mgr *LeaderViewManager) GetByFilter(filters ...LeaderViewFilter) []*LeaderView { - mgr.rwmutex.RLock() - defer mgr.rwmutex.RUnlock() - - return mgr.getByFilter(filters...) -} - -func (mgr *LeaderViewManager) getByFilter(filters ...LeaderViewFilter) []*LeaderView { - criterion := &lvCriterion{} - for _, filter := range filters { - filter.AddFilter(criterion) - } - - var candidates []nodeViews - if criterion.nodeID > 0 { - nodeView, ok := mgr.views[criterion.nodeID] - if ok { - candidates = append(candidates, nodeView) - } - } else { - candidates = lo.Values(mgr.views) - } - - var result []*LeaderView - for _, candidate := range candidates { - result = append(result, candidate.Filter(criterion, filters...)...) - } - return result -} - -func (mgr *LeaderViewManager) GetLatestShardLeaderByFilter(filters ...LeaderViewFilter) *LeaderView { - mgr.rwmutex.RLock() - defer mgr.rwmutex.RUnlock() - views := mgr.getByFilter(filters...) - - return lo.MaxBy(views, func(v1, v2 *LeaderView) bool { - return v1.Version > v2.Version - }) -} - -// GetLeaderView returns a slice of LeaderView objects, each representing the state of a leader node. -// It traverses the views map, converts each LeaderView to a metricsinfo.LeaderView, and collects them into a slice. -// The method locks the views map for reading to ensure thread safety. -func (mgr *LeaderViewManager) GetLeaderView(collectionID int64) []*metricsinfo.LeaderView { - mgr.rwmutex.RLock() - defer mgr.rwmutex.RUnlock() - - var leaderViews []*metricsinfo.LeaderView - for _, nodeViews := range mgr.views { - var filteredViews []*LeaderView - if collectionID > 0 { - if lv, ok := nodeViews.collectionViews[collectionID]; ok { - filteredViews = lv - } else { - // skip if collectionID is not found - continue - } - } else { - // if collectionID is not set, return all leader views - filteredViews = nodeViews.views - } - - for _, lv := range filteredViews { - errString := "" - if lv.UnServiceableError != nil { - errString = lv.UnServiceableError.Error() - } - leaderView := &metricsinfo.LeaderView{ - LeaderID: lv.ID, - CollectionID: lv.CollectionID, - Channel: lv.Channel, - Version: lv.Version, - SealedSegments: make([]*metricsinfo.Segment, 0, len(lv.Segments)), - GrowingSegments: make([]*metricsinfo.Segment, 0, len(lv.GrowingSegments)), - TargetVersion: lv.TargetVersion, - NumOfGrowingRows: lv.NumOfGrowingRows, - UnServiceableError: errString, - } - - for segID, seg := range lv.Segments { - leaderView.SealedSegments = append(leaderView.SealedSegments, &metricsinfo.Segment{ - SegmentID: segID, - NodeID: seg.NodeID, - }) - } - - for _, seg := range lv.GrowingSegments { - leaderView.GrowingSegments = append(leaderView.GrowingSegments, &metricsinfo.Segment{ - SegmentID: seg.ID, - NodeID: seg.Node, - }) - } - - leaderViews = append(leaderViews, leaderView) - } - } - return leaderViews -} diff --git a/internal/querycoordv2/meta/leader_view_manager_test.go b/internal/querycoordv2/meta/leader_view_manager_test.go deleted file mode 100644 index 9335dedc3828b..0000000000000 --- a/internal/querycoordv2/meta/leader_view_manager_test.go +++ /dev/null @@ -1,400 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you 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 meta - -import ( - "testing" - - "github.com/cockroachdb/errors" - "github.com/samber/lo" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/suite" - - "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - "github.com/milvus-io/milvus/internal/json" - "github.com/milvus-io/milvus/pkg/v2/proto/datapb" - "github.com/milvus-io/milvus/pkg/v2/proto/querypb" - "github.com/milvus-io/milvus/pkg/v2/util/metricsinfo" - "github.com/milvus-io/milvus/pkg/v2/util/typeutil" -) - -type LeaderViewManagerSuite struct { - suite.Suite - // Data - collections []int64 - channels map[int64][]string - growingSegments map[int64]map[string]int64 - segments map[int64]map[string][]int64 - nodes []int64 - leaders map[int64]map[string]*LeaderView - - // Test object - mgr *LeaderViewManager -} - -func (suite *LeaderViewManagerSuite) SetupSuite() { - suite.collections = []int64{100, 101} - suite.channels = map[int64][]string{ - 100: {"100-dmc0", "100-dmc1"}, - 101: {"101-dmc0", "101-dmc1"}, - } - suite.growingSegments = map[int64]map[string]int64{ - 100: { - "100-dmc0": 10, - "100-dmc1": 11, - }, - 101: { - "101-dmc0": 12, - "101-dmc1": 13, - }, - } - suite.segments = map[int64]map[string][]int64{ - 100: { - "100-dmc0": []int64{1, 2}, - "100-dmc1": {3, 4}, - }, - 101: { - "101-dmc0": {5, 6}, - "101-dmc1": {7, 8}, - }, - } - suite.nodes = []int64{1, 2} - - // Leaders: 1, 2 - suite.leaders = make(map[int64]map[string]*LeaderView) - for _, collection := range suite.collections { - for j := 0; j < 2; j++ { - channel := suite.channels[collection][j] - node := suite.nodes[j] - view := &LeaderView{ - ID: node, - CollectionID: collection, - Channel: channel, - GrowingSegments: map[int64]*Segment{suite.growingSegments[collection][channel]: nil}, - Segments: make(map[int64]*querypb.SegmentDist), - } - - for _, segment := range suite.segments[collection][channel] { - view.Segments[segment] = &querypb.SegmentDist{ - NodeID: node, - Version: 0, - } - } - if suite.leaders[node] == nil { - suite.leaders[node] = map[string]*LeaderView{ - channel: view, - } - } else { - suite.leaders[node][channel] = view - } - } - } -} - -func (suite *LeaderViewManagerSuite) SetupTest() { - suite.mgr = NewLeaderViewManager() - for id, views := range suite.leaders { - suite.mgr.Update(id, lo.Values(views)...) - } -} - -func (suite *LeaderViewManagerSuite) TestGetByFilter() { - // Test WithChannelName - for collectionID, channels := range suite.channels { - for _, channel := range channels { - views := suite.mgr.GetByFilter(WithChannelName2LeaderView(channel)) - suite.Len(views, 1) - suite.Equal(collectionID, views[0].CollectionID) - } - } - - // Test WithCollection - for _, collectionID := range suite.collections { - views := suite.mgr.GetByFilter(WithCollectionID2LeaderView(collectionID)) - suite.Len(views, 2) - suite.Equal(collectionID, views[0].CollectionID) - } - - // Test WithNodeID - for _, nodeID := range suite.nodes { - views := suite.mgr.GetByFilter(WithNodeID2LeaderView(nodeID)) - suite.Len(views, 2) - for _, view := range views { - suite.Equal(nodeID, view.ID) - } - } - - // Test WithReplica - for i, collectionID := range suite.collections { - replica := newReplica(&querypb.Replica{ - ID: int64(i), - CollectionID: collectionID, - Nodes: suite.nodes, - }) - views := suite.mgr.GetByFilter(WithReplica2LeaderView(replica)) - suite.Len(views, 2) - } - - // Test WithSegment - for _, leaders := range suite.leaders { - for _, leader := range leaders { - for sid := range leader.Segments { - views := suite.mgr.GetByFilter(WithSegment2LeaderView(sid, false)) - suite.Len(views, 1) - suite.Equal(views[0].ID, leader.ID) - suite.Equal(views[0].Channel, leader.Channel) - } - - for sid := range leader.GrowingSegments { - views := suite.mgr.GetByFilter(WithSegment2LeaderView(sid, true)) - suite.Len(views, 1) - suite.Equal(views[0].ID, leader.ID) - suite.Equal(views[0].Channel, leader.Channel) - } - - view := suite.mgr.GetLeaderShardView(leader.ID, leader.Channel) - suite.Equal(view.ID, leader.ID) - suite.Equal(view.Channel, leader.Channel) - } - } -} - -func (suite *LeaderViewManagerSuite) TestGetLatestShardLeader() { - nodeID := int64(1001) - collectionID := suite.collections[0] - channel := suite.channels[collectionID][0] - // add duplicate shard leader - view := &LeaderView{ - ID: nodeID, - CollectionID: collectionID, - Channel: channel, - GrowingSegments: map[int64]*Segment{suite.growingSegments[collectionID][channel]: nil}, - Segments: make(map[int64]*querypb.SegmentDist), - } - - for _, segment := range suite.segments[collectionID][channel] { - view.Segments[segment] = &querypb.SegmentDist{ - NodeID: nodeID, - Version: 1000, - } - } - view.Version = 1000 - - suite.mgr.Update(nodeID, view) - - leader := suite.mgr.GetLatestShardLeaderByFilter(WithChannelName2LeaderView(channel)) - suite.Equal(nodeID, leader.ID) - - // test replica is nil - leader = suite.mgr.GetLatestShardLeaderByFilter(WithReplica2LeaderView(nil)) - suite.Nil(leader) -} - -func (suite *LeaderViewManagerSuite) TestClone() { - for _, leaders := range suite.leaders { - for _, leader := range leaders { - clone := leader.Clone() - suite.Equal(leader.ID, clone.ID) - suite.Equal(leader.Channel, clone.Channel) - suite.Equal(leader.CollectionID, clone.CollectionID) - } - } -} - -func (suite *LeaderViewManagerSuite) TestNotifyDelegatorChanges() { - mgr := NewLeaderViewManager() - - oldViews := []*LeaderView{ - { - ID: 1, - CollectionID: 100, - Channel: "test-channel-1", - }, - { - ID: 1, - CollectionID: 101, - Channel: "test-channel-2", - }, - { - ID: 1, - CollectionID: 102, - Channel: "test-channel-3", - }, - } - mgr.Update(1, oldViews...) - - newViews := []*LeaderView{ - { - ID: 1, - CollectionID: 101, - Channel: "test-channel-2", - }, - { - ID: 1, - CollectionID: 102, - Channel: "test-channel-3", - }, - { - ID: 1, - CollectionID: 103, - Channel: "test-channel-4", - }, - } - - retSet := typeutil.NewUniqueSet() - mgr.SetNotifyFunc(func(collectionIDs ...int64) { - retSet.Insert(collectionIDs...) - }) - - mgr.Update(1, newViews...) - suite.Equal(2, retSet.Len()) - suite.True(retSet.Contain(100)) - suite.True(retSet.Contain(103)) - - newViews1 := []*LeaderView{ - { - ID: 1, - CollectionID: 101, - Channel: "test-channel-2", - UnServiceableError: errors.New("test error"), - }, - { - ID: 1, - CollectionID: 102, - Channel: "test-channel-3", - UnServiceableError: errors.New("test error"), - }, - { - ID: 1, - CollectionID: 103, - Channel: "test-channel-4", - UnServiceableError: errors.New("test error"), - }, - } - - retSet.Clear() - mgr.Update(1, newViews1...) - suite.Equal(3, len(retSet)) - suite.True(retSet.Contain(101)) - suite.True(retSet.Contain(102)) - suite.True(retSet.Contain(103)) - - newViews2 := []*LeaderView{ - { - ID: 1, - CollectionID: 101, - Channel: "test-channel-2", - UnServiceableError: errors.New("test error"), - }, - { - ID: 1, - CollectionID: 102, - Channel: "test-channel-3", - }, - { - ID: 1, - CollectionID: 103, - Channel: "test-channel-4", - }, - } - - retSet.Clear() - mgr.Update(1, newViews2...) - suite.Equal(2, len(retSet)) - suite.True(retSet.Contain(102)) - suite.True(retSet.Contain(103)) -} - -func TestLeaderViewManager(t *testing.T) { - suite.Run(t, new(LeaderViewManagerSuite)) -} - -func TestGetLeaderView(t *testing.T) { - manager := NewLeaderViewManager() - leaderView1 := &LeaderView{ - ID: 1, - CollectionID: 100, - Channel: "channel-1", - Version: 1, - Segments: map[int64]*querypb.SegmentDist{1: {NodeID: 1}}, - GrowingSegments: map[int64]*Segment{ - 1: {SegmentInfo: &datapb.SegmentInfo{ID: 1, CollectionID: 100, PartitionID: 10, InsertChannel: "channel-1", NumOfRows: 1000, State: commonpb.SegmentState_Growing}, Node: 1}, - }, - TargetVersion: 1, - NumOfGrowingRows: 1000, - UnServiceableError: nil, - } - - leaderView2 := &LeaderView{ - ID: 2, - CollectionID: 200, - Channel: "channel-2", - Version: 1, - Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2}}, - GrowingSegments: map[int64]*Segment{ - 2: {SegmentInfo: &datapb.SegmentInfo{ID: 2, CollectionID: 200, PartitionID: 20, InsertChannel: "channel-2", NumOfRows: 2000, State: commonpb.SegmentState_Growing}, Node: 2}, - }, - TargetVersion: 1, - NumOfGrowingRows: 2000, - UnServiceableError: nil, - } - - manager.Update(1, leaderView1) - manager.Update(2, leaderView2) - - // Call GetLeaderView - leaderViews := manager.GetLeaderView(0) - jsonOutput, err := json.Marshal(leaderViews) - assert.NoError(t, err) - - var result []*metricsinfo.LeaderView - err = json.Unmarshal(jsonOutput, &result) - assert.NoError(t, err) - assert.Len(t, result, 2) - - checkResult := func(lv *metricsinfo.LeaderView) { - if lv.LeaderID == 1 { - assert.Equal(t, int64(100), lv.CollectionID) - assert.Equal(t, "channel-1", lv.Channel) - assert.Equal(t, int64(1), lv.Version) - assert.Len(t, lv.SealedSegments, 1) - assert.Len(t, lv.GrowingSegments, 1) - assert.Equal(t, int64(1), lv.SealedSegments[0].SegmentID) - assert.Equal(t, int64(1), lv.GrowingSegments[0].SegmentID) - } else if lv.LeaderID == 2 { - assert.Equal(t, int64(200), lv.CollectionID) - assert.Equal(t, "channel-2", lv.Channel) - assert.Equal(t, int64(1), lv.Version) - assert.Len(t, lv.SealedSegments, 1) - assert.Len(t, lv.GrowingSegments, 1) - assert.Equal(t, int64(2), lv.SealedSegments[0].SegmentID) - assert.Equal(t, int64(2), lv.GrowingSegments[0].SegmentID) - } else { - assert.Failf(t, "unexpected leader id", "unexpected leader id %d", lv.LeaderID) - } - } - - for _, lv := range result { - checkResult(lv) - } - - leaderViews = manager.GetLeaderView(1) - assert.Len(t, leaderViews, 0) - - leaderViews = manager.GetLeaderView(100) - assert.Len(t, leaderViews, 1) -} diff --git a/internal/querycoordv2/meta/mock_replica.go b/internal/querycoordv2/meta/mock_replica.go new file mode 100644 index 0000000000000..44d220c44e0ba --- /dev/null +++ b/internal/querycoordv2/meta/mock_replica.go @@ -0,0 +1,1035 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package meta + +import mock "github.com/stretchr/testify/mock" + +// MockReplica is an autogenerated mock type for the ReplicaInterface type +type MockReplica struct { + mock.Mock +} + +type MockReplica_Expecter struct { + mock *mock.Mock +} + +func (_m *MockReplica) EXPECT() *MockReplica_Expecter { + return &MockReplica_Expecter{mock: &_m.Mock} +} + +// ContainRONode provides a mock function with given fields: node +func (_m *MockReplica) ContainRONode(node int64) bool { + ret := _m.Called(node) + + if len(ret) == 0 { + panic("no return value specified for ContainRONode") + } + + var r0 bool + if rf, ok := ret.Get(0).(func(int64) bool); ok { + r0 = rf(node) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// MockReplica_ContainRONode_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ContainRONode' +type MockReplica_ContainRONode_Call struct { + *mock.Call +} + +// ContainRONode is a helper method to define mock.On call +// - node int64 +func (_e *MockReplica_Expecter) ContainRONode(node interface{}) *MockReplica_ContainRONode_Call { + return &MockReplica_ContainRONode_Call{Call: _e.mock.On("ContainRONode", node)} +} + +func (_c *MockReplica_ContainRONode_Call) Run(run func(node int64)) *MockReplica_ContainRONode_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64)) + }) + return _c +} + +func (_c *MockReplica_ContainRONode_Call) Return(_a0 bool) *MockReplica_ContainRONode_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_ContainRONode_Call) RunAndReturn(run func(int64) bool) *MockReplica_ContainRONode_Call { + _c.Call.Return(run) + return _c +} + +// ContainROSQNode provides a mock function with given fields: node +func (_m *MockReplica) ContainROSQNode(node int64) bool { + ret := _m.Called(node) + + if len(ret) == 0 { + panic("no return value specified for ContainROSQNode") + } + + var r0 bool + if rf, ok := ret.Get(0).(func(int64) bool); ok { + r0 = rf(node) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// MockReplica_ContainROSQNode_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ContainROSQNode' +type MockReplica_ContainROSQNode_Call struct { + *mock.Call +} + +// ContainROSQNode is a helper method to define mock.On call +// - node int64 +func (_e *MockReplica_Expecter) ContainROSQNode(node interface{}) *MockReplica_ContainROSQNode_Call { + return &MockReplica_ContainROSQNode_Call{Call: _e.mock.On("ContainROSQNode", node)} +} + +func (_c *MockReplica_ContainROSQNode_Call) Run(run func(node int64)) *MockReplica_ContainROSQNode_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64)) + }) + return _c +} + +func (_c *MockReplica_ContainROSQNode_Call) Return(_a0 bool) *MockReplica_ContainROSQNode_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_ContainROSQNode_Call) RunAndReturn(run func(int64) bool) *MockReplica_ContainROSQNode_Call { + _c.Call.Return(run) + return _c +} + +// ContainRWNode provides a mock function with given fields: node +func (_m *MockReplica) ContainRWNode(node int64) bool { + ret := _m.Called(node) + + if len(ret) == 0 { + panic("no return value specified for ContainRWNode") + } + + var r0 bool + if rf, ok := ret.Get(0).(func(int64) bool); ok { + r0 = rf(node) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// MockReplica_ContainRWNode_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ContainRWNode' +type MockReplica_ContainRWNode_Call struct { + *mock.Call +} + +// ContainRWNode is a helper method to define mock.On call +// - node int64 +func (_e *MockReplica_Expecter) ContainRWNode(node interface{}) *MockReplica_ContainRWNode_Call { + return &MockReplica_ContainRWNode_Call{Call: _e.mock.On("ContainRWNode", node)} +} + +func (_c *MockReplica_ContainRWNode_Call) Run(run func(node int64)) *MockReplica_ContainRWNode_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64)) + }) + return _c +} + +func (_c *MockReplica_ContainRWNode_Call) Return(_a0 bool) *MockReplica_ContainRWNode_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_ContainRWNode_Call) RunAndReturn(run func(int64) bool) *MockReplica_ContainRWNode_Call { + _c.Call.Return(run) + return _c +} + +// ContainRWSQNode provides a mock function with given fields: node +func (_m *MockReplica) ContainRWSQNode(node int64) bool { + ret := _m.Called(node) + + if len(ret) == 0 { + panic("no return value specified for ContainRWSQNode") + } + + var r0 bool + if rf, ok := ret.Get(0).(func(int64) bool); ok { + r0 = rf(node) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// MockReplica_ContainRWSQNode_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ContainRWSQNode' +type MockReplica_ContainRWSQNode_Call struct { + *mock.Call +} + +// ContainRWSQNode is a helper method to define mock.On call +// - node int64 +func (_e *MockReplica_Expecter) ContainRWSQNode(node interface{}) *MockReplica_ContainRWSQNode_Call { + return &MockReplica_ContainRWSQNode_Call{Call: _e.mock.On("ContainRWSQNode", node)} +} + +func (_c *MockReplica_ContainRWSQNode_Call) Run(run func(node int64)) *MockReplica_ContainRWSQNode_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64)) + }) + return _c +} + +func (_c *MockReplica_ContainRWSQNode_Call) Return(_a0 bool) *MockReplica_ContainRWSQNode_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_ContainRWSQNode_Call) RunAndReturn(run func(int64) bool) *MockReplica_ContainRWSQNode_Call { + _c.Call.Return(run) + return _c +} + +// ContainSQNode provides a mock function with given fields: node +func (_m *MockReplica) ContainSQNode(node int64) bool { + ret := _m.Called(node) + + if len(ret) == 0 { + panic("no return value specified for ContainSQNode") + } + + var r0 bool + if rf, ok := ret.Get(0).(func(int64) bool); ok { + r0 = rf(node) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// MockReplica_ContainSQNode_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ContainSQNode' +type MockReplica_ContainSQNode_Call struct { + *mock.Call +} + +// ContainSQNode is a helper method to define mock.On call +// - node int64 +func (_e *MockReplica_Expecter) ContainSQNode(node interface{}) *MockReplica_ContainSQNode_Call { + return &MockReplica_ContainSQNode_Call{Call: _e.mock.On("ContainSQNode", node)} +} + +func (_c *MockReplica_ContainSQNode_Call) Run(run func(node int64)) *MockReplica_ContainSQNode_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64)) + }) + return _c +} + +func (_c *MockReplica_ContainSQNode_Call) Return(_a0 bool) *MockReplica_ContainSQNode_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_ContainSQNode_Call) RunAndReturn(run func(int64) bool) *MockReplica_ContainSQNode_Call { + _c.Call.Return(run) + return _c +} + +// Contains provides a mock function with given fields: node +func (_m *MockReplica) Contains(node int64) bool { + ret := _m.Called(node) + + if len(ret) == 0 { + panic("no return value specified for Contains") + } + + var r0 bool + if rf, ok := ret.Get(0).(func(int64) bool); ok { + r0 = rf(node) + } else { + r0 = ret.Get(0).(bool) + } + + return r0 +} + +// MockReplica_Contains_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Contains' +type MockReplica_Contains_Call struct { + *mock.Call +} + +// Contains is a helper method to define mock.On call +// - node int64 +func (_e *MockReplica_Expecter) Contains(node interface{}) *MockReplica_Contains_Call { + return &MockReplica_Contains_Call{Call: _e.mock.On("Contains", node)} +} + +func (_c *MockReplica_Contains_Call) Run(run func(node int64)) *MockReplica_Contains_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64)) + }) + return _c +} + +func (_c *MockReplica_Contains_Call) Return(_a0 bool) *MockReplica_Contains_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_Contains_Call) RunAndReturn(run func(int64) bool) *MockReplica_Contains_Call { + _c.Call.Return(run) + return _c +} + +// GetCollectionID provides a mock function with given fields: +func (_m *MockReplica) GetCollectionID() int64 { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetCollectionID") + } + + var r0 int64 + if rf, ok := ret.Get(0).(func() int64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int64) + } + + return r0 +} + +// MockReplica_GetCollectionID_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetCollectionID' +type MockReplica_GetCollectionID_Call struct { + *mock.Call +} + +// GetCollectionID is a helper method to define mock.On call +func (_e *MockReplica_Expecter) GetCollectionID() *MockReplica_GetCollectionID_Call { + return &MockReplica_GetCollectionID_Call{Call: _e.mock.On("GetCollectionID")} +} + +func (_c *MockReplica_GetCollectionID_Call) Run(run func()) *MockReplica_GetCollectionID_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_GetCollectionID_Call) Return(_a0 int64) *MockReplica_GetCollectionID_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_GetCollectionID_Call) RunAndReturn(run func() int64) *MockReplica_GetCollectionID_Call { + _c.Call.Return(run) + return _c +} + +// GetID provides a mock function with given fields: +func (_m *MockReplica) GetID() int64 { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetID") + } + + var r0 int64 + if rf, ok := ret.Get(0).(func() int64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int64) + } + + return r0 +} + +// MockReplica_GetID_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetID' +type MockReplica_GetID_Call struct { + *mock.Call +} + +// GetID is a helper method to define mock.On call +func (_e *MockReplica_Expecter) GetID() *MockReplica_GetID_Call { + return &MockReplica_GetID_Call{Call: _e.mock.On("GetID")} +} + +func (_c *MockReplica_GetID_Call) Run(run func()) *MockReplica_GetID_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_GetID_Call) Return(_a0 int64) *MockReplica_GetID_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_GetID_Call) RunAndReturn(run func() int64) *MockReplica_GetID_Call { + _c.Call.Return(run) + return _c +} + +// GetNodes provides a mock function with given fields: +func (_m *MockReplica) GetNodes() []int64 { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetNodes") + } + + var r0 []int64 + if rf, ok := ret.Get(0).(func() []int64); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]int64) + } + } + + return r0 +} + +// MockReplica_GetNodes_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetNodes' +type MockReplica_GetNodes_Call struct { + *mock.Call +} + +// GetNodes is a helper method to define mock.On call +func (_e *MockReplica_Expecter) GetNodes() *MockReplica_GetNodes_Call { + return &MockReplica_GetNodes_Call{Call: _e.mock.On("GetNodes")} +} + +func (_c *MockReplica_GetNodes_Call) Run(run func()) *MockReplica_GetNodes_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_GetNodes_Call) Return(_a0 []int64) *MockReplica_GetNodes_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_GetNodes_Call) RunAndReturn(run func() []int64) *MockReplica_GetNodes_Call { + _c.Call.Return(run) + return _c +} + +// GetRONodes provides a mock function with given fields: +func (_m *MockReplica) GetRONodes() []int64 { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetRONodes") + } + + var r0 []int64 + if rf, ok := ret.Get(0).(func() []int64); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]int64) + } + } + + return r0 +} + +// MockReplica_GetRONodes_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetRONodes' +type MockReplica_GetRONodes_Call struct { + *mock.Call +} + +// GetRONodes is a helper method to define mock.On call +func (_e *MockReplica_Expecter) GetRONodes() *MockReplica_GetRONodes_Call { + return &MockReplica_GetRONodes_Call{Call: _e.mock.On("GetRONodes")} +} + +func (_c *MockReplica_GetRONodes_Call) Run(run func()) *MockReplica_GetRONodes_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_GetRONodes_Call) Return(_a0 []int64) *MockReplica_GetRONodes_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_GetRONodes_Call) RunAndReturn(run func() []int64) *MockReplica_GetRONodes_Call { + _c.Call.Return(run) + return _c +} + +// GetROSQNodes provides a mock function with given fields: +func (_m *MockReplica) GetROSQNodes() []int64 { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetROSQNodes") + } + + var r0 []int64 + if rf, ok := ret.Get(0).(func() []int64); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]int64) + } + } + + return r0 +} + +// MockReplica_GetROSQNodes_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetROSQNodes' +type MockReplica_GetROSQNodes_Call struct { + *mock.Call +} + +// GetROSQNodes is a helper method to define mock.On call +func (_e *MockReplica_Expecter) GetROSQNodes() *MockReplica_GetROSQNodes_Call { + return &MockReplica_GetROSQNodes_Call{Call: _e.mock.On("GetROSQNodes")} +} + +func (_c *MockReplica_GetROSQNodes_Call) Run(run func()) *MockReplica_GetROSQNodes_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_GetROSQNodes_Call) Return(_a0 []int64) *MockReplica_GetROSQNodes_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_GetROSQNodes_Call) RunAndReturn(run func() []int64) *MockReplica_GetROSQNodes_Call { + _c.Call.Return(run) + return _c +} + +// GetRWNodes provides a mock function with given fields: +func (_m *MockReplica) GetRWNodes() []int64 { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetRWNodes") + } + + var r0 []int64 + if rf, ok := ret.Get(0).(func() []int64); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]int64) + } + } + + return r0 +} + +// MockReplica_GetRWNodes_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetRWNodes' +type MockReplica_GetRWNodes_Call struct { + *mock.Call +} + +// GetRWNodes is a helper method to define mock.On call +func (_e *MockReplica_Expecter) GetRWNodes() *MockReplica_GetRWNodes_Call { + return &MockReplica_GetRWNodes_Call{Call: _e.mock.On("GetRWNodes")} +} + +func (_c *MockReplica_GetRWNodes_Call) Run(run func()) *MockReplica_GetRWNodes_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_GetRWNodes_Call) Return(_a0 []int64) *MockReplica_GetRWNodes_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_GetRWNodes_Call) RunAndReturn(run func() []int64) *MockReplica_GetRWNodes_Call { + _c.Call.Return(run) + return _c +} + +// GetRWSQNodes provides a mock function with given fields: +func (_m *MockReplica) GetRWSQNodes() []int64 { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetRWSQNodes") + } + + var r0 []int64 + if rf, ok := ret.Get(0).(func() []int64); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]int64) + } + } + + return r0 +} + +// MockReplica_GetRWSQNodes_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetRWSQNodes' +type MockReplica_GetRWSQNodes_Call struct { + *mock.Call +} + +// GetRWSQNodes is a helper method to define mock.On call +func (_e *MockReplica_Expecter) GetRWSQNodes() *MockReplica_GetRWSQNodes_Call { + return &MockReplica_GetRWSQNodes_Call{Call: _e.mock.On("GetRWSQNodes")} +} + +func (_c *MockReplica_GetRWSQNodes_Call) Run(run func()) *MockReplica_GetRWSQNodes_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_GetRWSQNodes_Call) Return(_a0 []int64) *MockReplica_GetRWSQNodes_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_GetRWSQNodes_Call) RunAndReturn(run func() []int64) *MockReplica_GetRWSQNodes_Call { + _c.Call.Return(run) + return _c +} + +// GetResourceGroup provides a mock function with given fields: +func (_m *MockReplica) GetResourceGroup() string { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetResourceGroup") + } + + var r0 string + if rf, ok := ret.Get(0).(func() string); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(string) + } + + return r0 +} + +// MockReplica_GetResourceGroup_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetResourceGroup' +type MockReplica_GetResourceGroup_Call struct { + *mock.Call +} + +// GetResourceGroup is a helper method to define mock.On call +func (_e *MockReplica_Expecter) GetResourceGroup() *MockReplica_GetResourceGroup_Call { + return &MockReplica_GetResourceGroup_Call{Call: _e.mock.On("GetResourceGroup")} +} + +func (_c *MockReplica_GetResourceGroup_Call) Run(run func()) *MockReplica_GetResourceGroup_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_GetResourceGroup_Call) Return(_a0 string) *MockReplica_GetResourceGroup_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_GetResourceGroup_Call) RunAndReturn(run func() string) *MockReplica_GetResourceGroup_Call { + _c.Call.Return(run) + return _c +} + +// NodesCount provides a mock function with given fields: +func (_m *MockReplica) NodesCount() int { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for NodesCount") + } + + var r0 int + if rf, ok := ret.Get(0).(func() int); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int) + } + + return r0 +} + +// MockReplica_NodesCount_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'NodesCount' +type MockReplica_NodesCount_Call struct { + *mock.Call +} + +// NodesCount is a helper method to define mock.On call +func (_e *MockReplica_Expecter) NodesCount() *MockReplica_NodesCount_Call { + return &MockReplica_NodesCount_Call{Call: _e.mock.On("NodesCount")} +} + +func (_c *MockReplica_NodesCount_Call) Run(run func()) *MockReplica_NodesCount_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_NodesCount_Call) Return(_a0 int) *MockReplica_NodesCount_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_NodesCount_Call) RunAndReturn(run func() int) *MockReplica_NodesCount_Call { + _c.Call.Return(run) + return _c +} + +// RONodesCount provides a mock function with given fields: +func (_m *MockReplica) RONodesCount() int { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for RONodesCount") + } + + var r0 int + if rf, ok := ret.Get(0).(func() int); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int) + } + + return r0 +} + +// MockReplica_RONodesCount_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RONodesCount' +type MockReplica_RONodesCount_Call struct { + *mock.Call +} + +// RONodesCount is a helper method to define mock.On call +func (_e *MockReplica_Expecter) RONodesCount() *MockReplica_RONodesCount_Call { + return &MockReplica_RONodesCount_Call{Call: _e.mock.On("RONodesCount")} +} + +func (_c *MockReplica_RONodesCount_Call) Run(run func()) *MockReplica_RONodesCount_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_RONodesCount_Call) Return(_a0 int) *MockReplica_RONodesCount_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_RONodesCount_Call) RunAndReturn(run func() int) *MockReplica_RONodesCount_Call { + _c.Call.Return(run) + return _c +} + +// ROSQNodesCount provides a mock function with given fields: +func (_m *MockReplica) ROSQNodesCount() int { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for ROSQNodesCount") + } + + var r0 int + if rf, ok := ret.Get(0).(func() int); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int) + } + + return r0 +} + +// MockReplica_ROSQNodesCount_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ROSQNodesCount' +type MockReplica_ROSQNodesCount_Call struct { + *mock.Call +} + +// ROSQNodesCount is a helper method to define mock.On call +func (_e *MockReplica_Expecter) ROSQNodesCount() *MockReplica_ROSQNodesCount_Call { + return &MockReplica_ROSQNodesCount_Call{Call: _e.mock.On("ROSQNodesCount")} +} + +func (_c *MockReplica_ROSQNodesCount_Call) Run(run func()) *MockReplica_ROSQNodesCount_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_ROSQNodesCount_Call) Return(_a0 int) *MockReplica_ROSQNodesCount_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_ROSQNodesCount_Call) RunAndReturn(run func() int) *MockReplica_ROSQNodesCount_Call { + _c.Call.Return(run) + return _c +} + +// RWNodesCount provides a mock function with given fields: +func (_m *MockReplica) RWNodesCount() int { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for RWNodesCount") + } + + var r0 int + if rf, ok := ret.Get(0).(func() int); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int) + } + + return r0 +} + +// MockReplica_RWNodesCount_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RWNodesCount' +type MockReplica_RWNodesCount_Call struct { + *mock.Call +} + +// RWNodesCount is a helper method to define mock.On call +func (_e *MockReplica_Expecter) RWNodesCount() *MockReplica_RWNodesCount_Call { + return &MockReplica_RWNodesCount_Call{Call: _e.mock.On("RWNodesCount")} +} + +func (_c *MockReplica_RWNodesCount_Call) Run(run func()) *MockReplica_RWNodesCount_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_RWNodesCount_Call) Return(_a0 int) *MockReplica_RWNodesCount_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_RWNodesCount_Call) RunAndReturn(run func() int) *MockReplica_RWNodesCount_Call { + _c.Call.Return(run) + return _c +} + +// RWSQNodesCount provides a mock function with given fields: +func (_m *MockReplica) RWSQNodesCount() int { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for RWSQNodesCount") + } + + var r0 int + if rf, ok := ret.Get(0).(func() int); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int) + } + + return r0 +} + +// MockReplica_RWSQNodesCount_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RWSQNodesCount' +type MockReplica_RWSQNodesCount_Call struct { + *mock.Call +} + +// RWSQNodesCount is a helper method to define mock.On call +func (_e *MockReplica_Expecter) RWSQNodesCount() *MockReplica_RWSQNodesCount_Call { + return &MockReplica_RWSQNodesCount_Call{Call: _e.mock.On("RWSQNodesCount")} +} + +func (_c *MockReplica_RWSQNodesCount_Call) Run(run func()) *MockReplica_RWSQNodesCount_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockReplica_RWSQNodesCount_Call) Return(_a0 int) *MockReplica_RWSQNodesCount_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockReplica_RWSQNodesCount_Call) RunAndReturn(run func() int) *MockReplica_RWSQNodesCount_Call { + _c.Call.Return(run) + return _c +} + +// RangeOverRONodes provides a mock function with given fields: f +func (_m *MockReplica) RangeOverRONodes(f func(int64) bool) { + _m.Called(f) +} + +// MockReplica_RangeOverRONodes_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RangeOverRONodes' +type MockReplica_RangeOverRONodes_Call struct { + *mock.Call +} + +// RangeOverRONodes is a helper method to define mock.On call +// - f func(int64) bool +func (_e *MockReplica_Expecter) RangeOverRONodes(f interface{}) *MockReplica_RangeOverRONodes_Call { + return &MockReplica_RangeOverRONodes_Call{Call: _e.mock.On("RangeOverRONodes", f)} +} + +func (_c *MockReplica_RangeOverRONodes_Call) Run(run func(f func(int64) bool)) *MockReplica_RangeOverRONodes_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(func(int64) bool)) + }) + return _c +} + +func (_c *MockReplica_RangeOverRONodes_Call) Return() *MockReplica_RangeOverRONodes_Call { + _c.Call.Return() + return _c +} + +func (_c *MockReplica_RangeOverRONodes_Call) RunAndReturn(run func(func(int64) bool)) *MockReplica_RangeOverRONodes_Call { + _c.Call.Return(run) + return _c +} + +// RangeOverROSQNodes provides a mock function with given fields: f +func (_m *MockReplica) RangeOverROSQNodes(f func(int64) bool) { + _m.Called(f) +} + +// MockReplica_RangeOverROSQNodes_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RangeOverROSQNodes' +type MockReplica_RangeOverROSQNodes_Call struct { + *mock.Call +} + +// RangeOverROSQNodes is a helper method to define mock.On call +// - f func(int64) bool +func (_e *MockReplica_Expecter) RangeOverROSQNodes(f interface{}) *MockReplica_RangeOverROSQNodes_Call { + return &MockReplica_RangeOverROSQNodes_Call{Call: _e.mock.On("RangeOverROSQNodes", f)} +} + +func (_c *MockReplica_RangeOverROSQNodes_Call) Run(run func(f func(int64) bool)) *MockReplica_RangeOverROSQNodes_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(func(int64) bool)) + }) + return _c +} + +func (_c *MockReplica_RangeOverROSQNodes_Call) Return() *MockReplica_RangeOverROSQNodes_Call { + _c.Call.Return() + return _c +} + +func (_c *MockReplica_RangeOverROSQNodes_Call) RunAndReturn(run func(func(int64) bool)) *MockReplica_RangeOverROSQNodes_Call { + _c.Call.Return(run) + return _c +} + +// RangeOverRWNodes provides a mock function with given fields: f +func (_m *MockReplica) RangeOverRWNodes(f func(int64) bool) { + _m.Called(f) +} + +// MockReplica_RangeOverRWNodes_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RangeOverRWNodes' +type MockReplica_RangeOverRWNodes_Call struct { + *mock.Call +} + +// RangeOverRWNodes is a helper method to define mock.On call +// - f func(int64) bool +func (_e *MockReplica_Expecter) RangeOverRWNodes(f interface{}) *MockReplica_RangeOverRWNodes_Call { + return &MockReplica_RangeOverRWNodes_Call{Call: _e.mock.On("RangeOverRWNodes", f)} +} + +func (_c *MockReplica_RangeOverRWNodes_Call) Run(run func(f func(int64) bool)) *MockReplica_RangeOverRWNodes_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(func(int64) bool)) + }) + return _c +} + +func (_c *MockReplica_RangeOverRWNodes_Call) Return() *MockReplica_RangeOverRWNodes_Call { + _c.Call.Return() + return _c +} + +func (_c *MockReplica_RangeOverRWNodes_Call) RunAndReturn(run func(func(int64) bool)) *MockReplica_RangeOverRWNodes_Call { + _c.Call.Return(run) + return _c +} + +// RangeOverRWSQNodes provides a mock function with given fields: f +func (_m *MockReplica) RangeOverRWSQNodes(f func(int64) bool) { + _m.Called(f) +} + +// MockReplica_RangeOverRWSQNodes_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RangeOverRWSQNodes' +type MockReplica_RangeOverRWSQNodes_Call struct { + *mock.Call +} + +// RangeOverRWSQNodes is a helper method to define mock.On call +// - f func(int64) bool +func (_e *MockReplica_Expecter) RangeOverRWSQNodes(f interface{}) *MockReplica_RangeOverRWSQNodes_Call { + return &MockReplica_RangeOverRWSQNodes_Call{Call: _e.mock.On("RangeOverRWSQNodes", f)} +} + +func (_c *MockReplica_RangeOverRWSQNodes_Call) Run(run func(f func(int64) bool)) *MockReplica_RangeOverRWSQNodes_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(func(int64) bool)) + }) + return _c +} + +func (_c *MockReplica_RangeOverRWSQNodes_Call) Return() *MockReplica_RangeOverRWSQNodes_Call { + _c.Call.Return() + return _c +} + +func (_c *MockReplica_RangeOverRWSQNodes_Call) RunAndReturn(run func(func(int64) bool)) *MockReplica_RangeOverRWSQNodes_Call { + _c.Call.Return(run) + return _c +} + +// NewMockReplica creates a new instance of MockReplica. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewMockReplica(t interface { + mock.TestingT + Cleanup(func()) +}) *MockReplica { + mock := &MockReplica{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/querycoordv2/meta/replica.go b/internal/querycoordv2/meta/replica.go index 76761677f1548..659cb7fbfa1e7 100644 --- a/internal/querycoordv2/meta/replica.go +++ b/internal/querycoordv2/meta/replica.go @@ -8,6 +8,42 @@ import ( "github.com/milvus-io/milvus/pkg/v2/util/typeutil" ) +// ReplicaInterface defines read operations for replica metadata +type ReplicaInterface interface { + // Basic information + GetID() typeutil.UniqueID + GetCollectionID() typeutil.UniqueID + GetResourceGroup() string + + // Node access + GetNodes() []int64 + GetRONodes() []int64 + GetRWNodes() []int64 + GetROSQNodes() []int64 + GetRWSQNodes() []int64 + + // Node iteration + RangeOverRWNodes(f func(node int64) bool) + RangeOverRONodes(f func(node int64) bool) + RangeOverRWSQNodes(f func(node int64) bool) + RangeOverROSQNodes(f func(node int64) bool) + + // Node counting + RWNodesCount() int + RONodesCount() int + RWSQNodesCount() int + ROSQNodesCount() int + NodesCount() int + + // Node existence checks + Contains(node int64) bool + ContainRONode(node int64) bool + ContainRWNode(node int64) bool + ContainSQNode(node int64) bool + ContainROSQNode(node int64) bool + ContainRWSQNode(node int64) bool +} + // NilReplica is used to represent a nil replica. var NilReplica = newReplica(&querypb.Replica{ ID: -1, diff --git a/internal/querycoordv2/meta/replica_manager.go b/internal/querycoordv2/meta/replica_manager.go index 390be030307b5..947a3c1cd3e9d 100644 --- a/internal/querycoordv2/meta/replica_manager.go +++ b/internal/querycoordv2/meta/replica_manager.go @@ -37,6 +37,38 @@ import ( "github.com/milvus-io/milvus/pkg/v2/util/typeutil" ) +// ReplicaManagerInterface defines core operations for replica management +type ReplicaManagerInterface interface { + // Basic operations + Recover(ctx context.Context, collections []int64) error + Get(ctx context.Context, id typeutil.UniqueID) *Replica + Spawn(ctx context.Context, collection int64, replicaNumInRG map[string]int, channels []string) ([]*Replica, error) + + // Replica manipulation + TransferReplica(ctx context.Context, collectionID typeutil.UniqueID, srcRGName string, dstRGName string, replicaNum int) error + MoveReplica(ctx context.Context, dstRGName string, toMove []*Replica) error + RemoveCollection(ctx context.Context, collectionID typeutil.UniqueID) error + RemoveReplicas(ctx context.Context, collectionID typeutil.UniqueID, replicas ...typeutil.UniqueID) error + + // Query operations + GetByCollection(ctx context.Context, collectionID typeutil.UniqueID) []*Replica + GetByCollectionAndNode(ctx context.Context, collectionID, nodeID typeutil.UniqueID) *Replica + GetByNode(ctx context.Context, nodeID typeutil.UniqueID) []*Replica + GetByResourceGroup(ctx context.Context, rgName string) []*Replica + + // Node management + RecoverNodesInCollection(ctx context.Context, collectionID typeutil.UniqueID, rgs map[string]typeutil.UniqueSet) error + RemoveNode(ctx context.Context, replicaID typeutil.UniqueID, nodes ...typeutil.UniqueID) error + RemoveSQNode(ctx context.Context, replicaID typeutil.UniqueID, nodes ...typeutil.UniqueID) error + + // Metadata access + GetResourceGroupByCollection(ctx context.Context, collection typeutil.UniqueID) typeutil.Set[string] + GetReplicasJSON(ctx context.Context, meta *Meta) string +} + +// Add the interface implementation assertion +var _ ReplicaManagerInterface = (*ReplicaManager)(nil) + type ReplicaManager struct { rwmutex sync.RWMutex diff --git a/internal/querycoordv2/meta/segment_dist_manager.go b/internal/querycoordv2/meta/segment_dist_manager.go index cfd7f66495bda..bf6c1dffd207a 100644 --- a/internal/querycoordv2/meta/segment_dist_manager.go +++ b/internal/querycoordv2/meta/segment_dist_manager.go @@ -158,6 +158,12 @@ func (segment *Segment) Clone() *Segment { } } +type SegmentDistManagerInterface interface { + Update(nodeID typeutil.UniqueID, segments ...*Segment) + GetByFilter(filters ...SegmentDistFilter) []*Segment + GetSegmentDist(collectionID int64) []*metricsinfo.Segment +} + type SegmentDistManager struct { rwmutex sync.RWMutex diff --git a/internal/querycoordv2/observers/collection_observer.go b/internal/querycoordv2/observers/collection_observer.go index ef44d8f548957..a688b38dfc63a 100644 --- a/internal/querycoordv2/observers/collection_observer.go +++ b/internal/querycoordv2/observers/collection_observer.go @@ -325,8 +325,8 @@ func (ob *CollectionObserver) observeChannelStatus(ctx context.Context, collecti subChannelCount := 0 for _, channel := range channelTargets { - views := ob.dist.LeaderViewManager.GetByFilter(meta.WithChannelName2LeaderView(channel.GetChannelName())) - nodes := lo.Map(views, func(v *meta.LeaderView, _ int) int64 { return v.ID }) + delegatorList := ob.dist.ChannelDistManager.GetByFilter(meta.WithChannelName2Channel(channel.GetChannelName())) + nodes := lo.Map(delegatorList, func(v *meta.DmChannel, _ int) int64 { return v.Node }) group := utils.GroupNodesByReplica(ctx, ob.meta.ReplicaManager, collectionID, nodes) subChannelCount += len(group) } @@ -354,11 +354,14 @@ func (ob *CollectionObserver) observePartitionLoadStatus(ctx context.Context, pa loadPercentage := int32(0) for _, segment := range segmentTargets { - views := ob.dist.LeaderViewManager.GetByFilter( - meta.WithChannelName2LeaderView(segment.GetInsertChannel()), - meta.WithSegment2LeaderView(segment.GetID(), false)) - nodes := lo.Map(views, func(view *meta.LeaderView, _ int) int64 { return view.ID }) - group := utils.GroupNodesByReplica(ctx, ob.meta.ReplicaManager, partition.GetCollectionID(), nodes) + delegatorList := ob.dist.ChannelDistManager.GetByFilter(meta.WithChannelName2Channel(segment.GetInsertChannel())) + loadedSegmentNodes := make([]int64, 0) + for _, delegator := range delegatorList { + if delegator.View.Segments[segment.GetID()] != nil { + loadedSegmentNodes = append(loadedSegmentNodes, delegator.Node) + } + } + group := utils.GroupNodesByReplica(ctx, ob.meta.ReplicaManager, partition.GetCollectionID(), loadedSegmentNodes) loadedCount += len(group) } loadPercentage = int32(loadedCount * 100 / (targetNum * int(replicaNum))) diff --git a/internal/querycoordv2/observers/collection_observer_test.go b/internal/querycoordv2/observers/collection_observer_test.go index a89bcd4650de1..ca7eb9d1c43b6 100644 --- a/internal/querycoordv2/observers/collection_observer_test.go +++ b/internal/querycoordv2/observers/collection_observer_test.go @@ -258,49 +258,90 @@ func (suite *CollectionObserverSuite) TestObserve() { // Collection 100 loaded before timeout, // collection 101 timeout - suite.dist.LeaderViewManager.Update(1, &meta.LeaderView{ - ID: 1, - CollectionID: 100, - Channel: "100-dmc0", - Segments: map[int64]*querypb.SegmentDist{1: {NodeID: 1, Version: 0}}, - }) - view := &meta.LeaderView{ - ID: 2, - CollectionID: 103, - Channel: "103-dmc0", - Segments: make(map[int64]*querypb.SegmentDist), + ch1 := &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 100, + ChannelName: "100-dmc0", + }, + Node: 1, + View: &meta.LeaderView{ + ID: 1, + CollectionID: 100, + Channel: "100-dmc0", + Segments: map[int64]*querypb.SegmentDist{1: {NodeID: 1, Version: 0}}, + }, + } + suite.dist.ChannelDistManager.Update(1, ch1) + + ch2 := &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 103, + ChannelName: "103-dmc0", + }, + Node: 2, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 103, + Channel: "103-dmc0", + Segments: make(map[int64]*querypb.SegmentDist), + }, + } + + ch3 := &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 100, + ChannelName: "100-dmc1", + }, + Node: 2, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 100, + Channel: "100-dmc1", + Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2, Version: 0}}, + }, } - suite.dist.LeaderViewManager.Update(2, &meta.LeaderView{ - ID: 2, - CollectionID: 100, - Channel: "100-dmc1", - Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2, Version: 0}}, - }, view) - - view1 := &meta.LeaderView{ - ID: 3, - CollectionID: 102, - Channel: "102-dmc0", - Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 5, Version: 0}}, + suite.dist.ChannelDistManager.Update(2, ch2, ch3) + + ch4 := &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 102, + ChannelName: "102-dmc0", + }, + Node: 3, + View: &meta.LeaderView{ + ID: 3, + CollectionID: 102, + Channel: "102-dmc0", + Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 5, Version: 0}}, + }, + } + + ch5 := &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 103, + ChannelName: "103-dmc0", + }, + Node: 3, + View: &meta.LeaderView{ + ID: 3, + CollectionID: 103, + Channel: "103-dmc0", + Segments: make(map[int64]*querypb.SegmentDist), + }, } segmentsInfo, ok := suite.segments[103] suite.True(ok) - view2 := &meta.LeaderView{ - ID: 3, - CollectionID: 103, - Channel: "103-dmc0", - Segments: make(map[int64]*querypb.SegmentDist), - } for _, segment := range segmentsInfo { - view2.Segments[segment.GetID()] = &querypb.SegmentDist{ - NodeID: 3, Version: 0, - } - view.Segments[segment.GetID()] = &querypb.SegmentDist{ + ch2.View.Segments[segment.GetID()] = &querypb.SegmentDist{ NodeID: 2, Version: 0, } + ch5.View.Segments[segment.GetID()] = &querypb.SegmentDist{ + NodeID: 3, Version: 0, + } } - suite.dist.LeaderViewManager.Update(3, view1, view2) + + suite.dist.ChannelDistManager.Update(3, ch4, ch5) suite.broker.EXPECT().DescribeCollection(mock.Anything, mock.Anything).Return(nil, nil).Maybe() suite.broker.EXPECT().ListIndexes(mock.Anything, mock.Anything).Return(nil, nil).Maybe() @@ -331,23 +372,43 @@ func (suite *CollectionObserverSuite) TestObservePartition() { // Partition 10 loaded // Partition 11 timeout - suite.dist.LeaderViewManager.Update(1, &meta.LeaderView{ - ID: 1, - CollectionID: 100, - Channel: "100-dmc0", - Segments: map[int64]*querypb.SegmentDist{1: {NodeID: 1, Version: 0}}, - }, &meta.LeaderView{ - ID: 1, - CollectionID: 101, - Channel: "", - - Segments: map[int64]*querypb.SegmentDist{}, + suite.dist.ChannelDistManager.Update(1, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 100, + ChannelName: "100-dmc0", + }, + Node: 1, + View: &meta.LeaderView{ + ID: 1, + CollectionID: 100, + Channel: "100-dmc0", + Segments: map[int64]*querypb.SegmentDist{1: {NodeID: 1, Version: 0}}, + }, + }, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 101, + ChannelName: "101-dmc0", + }, + Node: 1, + View: &meta.LeaderView{ + ID: 1, + CollectionID: 101, + Channel: "101-dmc0", + }, }) - suite.dist.LeaderViewManager.Update(2, &meta.LeaderView{ - ID: 2, - CollectionID: 100, - Channel: "100-dmc1", - Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2, Version: 0}}, + + suite.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: 100, + ChannelName: "100-dmc1", + }, + Node: 2, + View: &meta.LeaderView{ + ID: 2, + CollectionID: 100, + Channel: "100-dmc1", + Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2, Version: 0}}, + }, }) suite.Eventually(func() bool { diff --git a/internal/querycoordv2/observers/replica_observer_test.go b/internal/querycoordv2/observers/replica_observer_test.go index 80c1f0c0ff288..d1dcbd9457bd4 100644 --- a/internal/querycoordv2/observers/replica_observer_test.go +++ b/internal/querycoordv2/observers/replica_observer_test.go @@ -34,6 +34,8 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/utils" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/v2/kv" + "github.com/milvus-io/milvus/pkg/v2/proto/datapb" + "github.com/milvus-io/milvus/pkg/v2/proto/querypb" "github.com/milvus-io/milvus/pkg/v2/streaming/util/types" "github.com/milvus-io/milvus/pkg/v2/util/etcd" "github.com/milvus-io/milvus/pkg/v2/util/paramtable" @@ -152,9 +154,15 @@ func (suite *ReplicaObserverSuite) TestCheckNodesInReplica() { // Add some segment on nodes. for nodeID := int64(1); nodeID <= 4; nodeID++ { - suite.distMgr.ChannelDistManager.Update( - nodeID, - utils.CreateTestChannel(suite.collectionID, nodeID, 1, "test-insert-channel1")) + suite.distMgr.ChannelDistManager.Update(nodeID, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collectionID, + ChannelName: "test-insert-channel1", + }, + Node: nodeID, + Version: 1, + View: &meta.LeaderView{ID: nodeID, CollectionID: suite.collectionID, Channel: "test-insert-channel1", Status: &querypb.LeaderViewStatus{Serviceable: true}}, + }) suite.distMgr.SegmentDistManager.Update( nodeID, utils.CreateTestSegment(suite.collectionID, suite.partitionID, 1, nodeID, 1, "test-insert-channel1")) diff --git a/internal/querycoordv2/observers/target_observer.go b/internal/querycoordv2/observers/target_observer.go index f1f33e89f10c8..1fef6831f98ff 100644 --- a/internal/querycoordv2/observers/target_observer.go +++ b/internal/querycoordv2/observers/target_observer.go @@ -393,35 +393,36 @@ func (ob *TargetObserver) shouldUpdateCurrentTarget(ctx context.Context, collect return false } - collectionReadyLeaders := make([]*meta.LeaderView, 0) + newVersion := ob.targetMgr.GetCollectionTargetVersion(ctx, collectionID, meta.NextTarget) + collReadyDelegatorList := make([]*meta.DmChannel, 0) for channel := range channelNames { - channelReadyLeaders := lo.Filter(ob.distMgr.LeaderViewManager.GetByFilter(meta.WithChannelName2LeaderView(channel)), func(leader *meta.LeaderView, _ int) bool { - return utils.CheckDelegatorDataReady(ob.nodeMgr, ob.targetMgr, leader, meta.NextTarget) == nil + chReadyDelegatorList := lo.Filter(ob.distMgr.ChannelDistManager.GetByFilter(meta.WithChannelName2Channel(channel)), func(ch *meta.DmChannel, _ int) bool { + return (newVersion == ch.View.TargetVersion && ch.IsServiceable()) || + utils.CheckDelegatorDataReady(ob.nodeMgr, ob.targetMgr, ch.View, meta.NextTarget) == nil }) // to avoid stuck here in dynamic increase replica case, we just check available delegator number - if int32(len(channelReadyLeaders)) < replicaNum { + if int32(len(chReadyDelegatorList)) < replicaNum { log.RatedInfo(10, "channel not ready", - zap.Int("readyReplicaNum", len(channelReadyLeaders)), + zap.Int("readyReplicaNum", len(chReadyDelegatorList)), zap.String("channelName", channel), ) return false } - collectionReadyLeaders = append(collectionReadyLeaders, channelReadyLeaders...) + collReadyDelegatorList = append(collReadyDelegatorList, chReadyDelegatorList...) } var partitions []int64 var indexInfo []*indexpb.IndexInfo var err error - newVersion := ob.targetMgr.GetCollectionTargetVersion(ctx, collectionID, meta.NextTarget) - for _, leader := range collectionReadyLeaders { - updateVersionAction := ob.checkNeedUpdateTargetVersion(ctx, leader, newVersion) + for _, d := range collReadyDelegatorList { + updateVersionAction := ob.checkNeedUpdateTargetVersion(ctx, d.View, newVersion) if updateVersionAction == nil { continue } - replica := ob.meta.ReplicaManager.GetByCollectionAndNode(ctx, collectionID, leader.ID) + replica := ob.meta.ReplicaManager.GetByCollectionAndNode(ctx, collectionID, d.Node) if replica == nil { - log.Warn("replica not found", zap.Int64("nodeID", leader.ID), zap.Int64("collectionID", collectionID)) + log.Warn("replica not found", zap.Int64("nodeID", d.Node), zap.Int64("collectionID", collectionID)) continue } // init all the meta information @@ -440,14 +441,14 @@ func (ob *TargetObserver) shouldUpdateCurrentTarget(ctx context.Context, collect } } - if !ob.sync(ctx, replica, leader, []*querypb.SyncAction{updateVersionAction}, partitions, indexInfo) { + if !ob.sync(ctx, replica, d.View, []*querypb.SyncAction{updateVersionAction}, partitions, indexInfo) { return false } } return true } -func (ob *TargetObserver) sync(ctx context.Context, replica *meta.Replica, leaderView *meta.LeaderView, diffs []*querypb.SyncAction, +func (ob *TargetObserver) sync(ctx context.Context, replica *meta.Replica, LeaderView *meta.LeaderView, diffs []*querypb.SyncAction, partitions []int64, indexInfo []*indexpb.IndexInfo, ) bool { if len(diffs) == 0 { @@ -456,22 +457,22 @@ func (ob *TargetObserver) sync(ctx context.Context, replica *meta.Replica, leade replicaID := replica.GetID() log := log.With( - zap.Int64("leaderID", leaderView.ID), - zap.Int64("collectionID", leaderView.CollectionID), - zap.String("channel", leaderView.Channel), + zap.Int64("leaderID", LeaderView.ID), + zap.Int64("collectionID", LeaderView.CollectionID), + zap.String("channel", LeaderView.Channel), ) req := &querypb.SyncDistributionRequest{ Base: commonpbutil.NewMsgBase( commonpbutil.WithMsgType(commonpb.MsgType_SyncDistribution), ), - CollectionID: leaderView.CollectionID, + CollectionID: LeaderView.CollectionID, ReplicaID: replicaID, - Channel: leaderView.Channel, + Channel: LeaderView.Channel, Actions: diffs, LoadMeta: &querypb.LoadMetaInfo{ - LoadType: ob.meta.GetLoadType(ctx, leaderView.CollectionID), - CollectionID: leaderView.CollectionID, + LoadType: ob.meta.GetLoadType(ctx, LeaderView.CollectionID), + CollectionID: LeaderView.CollectionID, PartitionIDs: partitions, ResourceGroup: replica.GetResourceGroup(), }, @@ -481,7 +482,7 @@ func (ob *TargetObserver) sync(ctx context.Context, replica *meta.Replica, leade ctx, cancel := context.WithTimeout(ctx, paramtable.Get().QueryCoordCfg.BrokerTimeout.GetAsDuration(time.Millisecond)) defer cancel() - resp, err := ob.cluster.SyncDistribution(ctx, leaderView.ID, req) + resp, err := ob.cluster.SyncDistribution(ctx, LeaderView.ID, req) if err != nil { log.Warn("failed to sync distribution", zap.Error(err)) return false diff --git a/internal/querycoordv2/observers/target_observer_test.go b/internal/querycoordv2/observers/target_observer_test.go index d18e86371b827..7d88753fa5282 100644 --- a/internal/querycoordv2/observers/target_observer_test.go +++ b/internal/querycoordv2/observers/target_observer_test.go @@ -156,8 +156,13 @@ func (suite *TargetObserverSuite) TestTriggerUpdateTarget() { len(suite.targetMgr.GetDmChannelsByCollection(ctx, suite.collectionID, meta.NextTarget)) == 2 }, 5*time.Second, 1*time.Second) - suite.distMgr.LeaderViewManager.Update(2, - &meta.LeaderView{ + suite.distMgr.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collectionID, + ChannelName: "channel-1", + }, + Node: 2, + View: &meta.LeaderView{ ID: 2, CollectionID: suite.collectionID, Channel: "channel-1", @@ -165,7 +170,13 @@ func (suite *TargetObserverSuite) TestTriggerUpdateTarget() { 11: {NodeID: 2}, }, }, - &meta.LeaderView{ + }, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collectionID, + ChannelName: "channel-2", + }, + Node: 2, + View: &meta.LeaderView{ ID: 2, CollectionID: suite.collectionID, Channel: "channel-2", @@ -173,7 +184,7 @@ func (suite *TargetObserverSuite) TestTriggerUpdateTarget() { 12: {NodeID: 2}, }, }, - ) + }) // Never update current target if it's empty, even the next target is ready suite.Eventually(func() bool { @@ -203,27 +214,36 @@ func (suite *TargetObserverSuite) TestTriggerUpdateTarget() { // Manually update next target ready, err := suite.observer.UpdateNextTarget(suite.collectionID) suite.NoError(err) - - ch1View := &meta.LeaderView{ - ID: 2, - CollectionID: suite.collectionID, - Channel: "channel-1", - Segments: map[int64]*querypb.SegmentDist{ - 11: {NodeID: 2}, - 13: {NodeID: 2}, + suite.distMgr.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collectionID, + ChannelName: "channel-1", }, - } - - ch2View := &meta.LeaderView{ - ID: 2, - CollectionID: suite.collectionID, - Channel: "channel-2", - Segments: map[int64]*querypb.SegmentDist{ - 12: {NodeID: 2}, + Node: 2, + View: &meta.LeaderView{ + ID: 2, + CollectionID: suite.collectionID, + Channel: "channel-1", + Segments: map[int64]*querypb.SegmentDist{ + 11: {NodeID: 2}, + 13: {NodeID: 2}, + }, }, - } - - suite.distMgr.LeaderViewManager.Update(2, ch1View, ch2View) + }, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collectionID, + ChannelName: "channel-2", + }, + Node: 2, + View: &meta.LeaderView{ + ID: 2, + CollectionID: suite.collectionID, + Channel: "channel-2", + Segments: map[int64]*querypb.SegmentDist{ + 12: {NodeID: 2}, + }, + }, + }) suite.broker.EXPECT().DescribeCollection(mock.Anything, mock.Anything).Return(nil, nil).Maybe() suite.broker.EXPECT().ListIndexes(mock.Anything, mock.Anything).Return(nil, nil).Maybe() @@ -242,6 +262,7 @@ func (suite *TargetObserverSuite) TestTriggerUpdateTarget() { len(suite.targetMgr.GetDmChannelsByCollection(ctx, suite.collectionID, meta.CurrentTarget)) == 2 }, 7*time.Second, 1*time.Second) + ch1View := suite.distMgr.ChannelDistManager.GetByFilter(meta.WithChannelName2Channel("channel-1"))[0].View action := suite.observer.checkNeedUpdateTargetVersion(ctx, ch1View, 100) suite.Equal(action.GetDeleteCP().Timestamp, uint64(200)) } diff --git a/internal/querycoordv2/server.go b/internal/querycoordv2/server.go index a072667d398b2..4a723660757b3 100644 --- a/internal/querycoordv2/server.go +++ b/internal/querycoordv2/server.go @@ -357,9 +357,6 @@ func (s *Server) initQueryCoord() error { s.initObserver() // Init heartbeat - syncTargetVersionFn := func(collectionID int64) { - s.targetObserver.TriggerUpdateCurrentTarget(collectionID) - } log.Info("init dist controller") s.distController = dist.NewDistController( s.cluster, @@ -367,7 +364,7 @@ func (s *Server) initQueryCoord() error { s.dist, s.targetMgr, s.taskScheduler, - syncTargetVersionFn, + s.leaderCacheObserver.RegisterEvent, ) // Init load status cache @@ -418,7 +415,6 @@ func (s *Server) initMeta() error { s.dist = &meta.DistributionManager{ SegmentDistManager: meta.NewSegmentDistManager(), ChannelDistManager: meta.NewChannelDistManager(), - LeaderViewManager: meta.NewLeaderViewManager(), } s.targetMgr = meta.NewTargetManager(s.broker, s.meta) err = s.targetMgr.Recover(s.ctx, s.store) @@ -459,7 +455,6 @@ func (s *Server) initObserver() { s.leaderCacheObserver = observers.NewLeaderCacheObserver( s.proxyClientManager, ) - s.dist.LeaderViewManager.SetNotifyFunc(s.leaderCacheObserver.RegisterEvent) } func (s *Server) afterStart() {} @@ -774,7 +769,6 @@ func (s *Server) handleNodeDown(node int64) { s.distController.Remove(node) // Clear dist - s.dist.LeaderViewManager.Update(node) s.dist.ChannelDistManager.Update(node) s.dist.SegmentDistManager.Update(node) diff --git a/internal/querycoordv2/server_test.go b/internal/querycoordv2/server_test.go index 5b2f86efd3ecb..bc2772aa09ed0 100644 --- a/internal/querycoordv2/server_test.go +++ b/internal/querycoordv2/server_test.go @@ -43,7 +43,6 @@ import ( "github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/internal/util/sessionutil" - "github.com/milvus-io/milvus/pkg/v2/common" "github.com/milvus-io/milvus/pkg/v2/log" "github.com/milvus-io/milvus/pkg/v2/proto/datapb" "github.com/milvus-io/milvus/pkg/v2/proto/querypb" @@ -577,16 +576,13 @@ func (suite *ServerSuite) hackServer() { suite.server.nodeMgr, ) - syncTargetVersionFn := func(collectionID int64) { - suite.server.targetObserver.Check(context.Background(), collectionID, common.AllPartitionsID) - } suite.server.distController = dist.NewDistController( suite.server.cluster, suite.server.nodeMgr, suite.server.dist, suite.server.targetMgr, suite.server.taskScheduler, - syncTargetVersionFn, + suite.server.leaderCacheObserver.RegisterEvent, ) suite.server.checkerController = checkers.NewCheckerController( suite.server.meta, diff --git a/internal/querycoordv2/services_test.go b/internal/querycoordv2/services_test.go index 65c23fc9d2ceb..fe57db2f5dbb1 100644 --- a/internal/querycoordv2/services_test.go +++ b/internal/querycoordv2/services_test.go @@ -1701,7 +1701,6 @@ func (suite *ServiceSuite) TestGetShardLeadersFailed() { for _, node := range suite.nodes { suite.dist.SegmentDistManager.Update(node) suite.dist.ChannelDistManager.Update(node) - suite.dist.LeaderViewManager.Update(node) } suite.updateChannelDistWithoutSegment(ctx, collection) suite.fetchHeartbeats(time.Now()) @@ -1712,7 +1711,7 @@ func (suite *ServiceSuite) TestGetShardLeadersFailed() { // channel not subscribed for _, node := range suite.nodes { - suite.dist.LeaderViewManager.Update(node) + suite.dist.ChannelDistManager.Update(node) } for _, collection := range suite.collections { req := &querypb.GetShardLeadersRequest{ @@ -1955,20 +1954,26 @@ func (suite *ServiceSuite) updateChannelDist(ctx context.Context, collection int for _, replica := range replicas { i := 0 for _, node := range suite.sortInt64(replica.GetNodes()) { - suite.dist.ChannelDistManager.Update(node, meta.DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: collection, - ChannelName: channels[i], - })) - suite.dist.LeaderViewManager.Update(node, &meta.LeaderView{ - ID: node, - CollectionID: collection, - Channel: channels[i], - Segments: lo.SliceToMap(segments, func(segment int64) (int64, *querypb.SegmentDist) { - return segment, &querypb.SegmentDist{ - NodeID: node, - Version: time.Now().Unix(), - } - }), + suite.dist.ChannelDistManager.Update(node, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: collection, + ChannelName: channels[i], + }, + Node: node, + View: &meta.LeaderView{ + ID: node, + CollectionID: collection, + Channel: channels[i], + Segments: lo.SliceToMap(segments, func(segment int64) (int64, *querypb.SegmentDist) { + return segment, &querypb.SegmentDist{ + NodeID: node, + Version: time.Now().Unix(), + } + }), + Status: &querypb.LeaderViewStatus{ + Serviceable: true, + }, + }, }) i++ if i >= len(channels) { @@ -1992,15 +1997,17 @@ func (suite *ServiceSuite) updateChannelDistWithoutSegment(ctx context.Context, for _, replica := range replicas { i := 0 for _, node := range suite.sortInt64(replica.GetNodes()) { - suite.dist.ChannelDistManager.Update(node, meta.DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: collection, - ChannelName: channels[i], - })) - suite.dist.LeaderViewManager.Update(node, &meta.LeaderView{ - ID: node, - CollectionID: collection, - Channel: channels[i], - UnServiceableError: merr.ErrSegmentLack, + suite.dist.ChannelDistManager.Update(node, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: collection, + ChannelName: channels[i], + }, + Node: node, + View: &meta.LeaderView{ + ID: node, + CollectionID: collection, + Channel: channels[i], + }, }) i++ if i >= len(channels) { diff --git a/internal/querycoordv2/task/action.go b/internal/querycoordv2/task/action.go index 5e755619d543e..f3f2df8c69be8 100644 --- a/internal/querycoordv2/task/action.go +++ b/internal/querycoordv2/task/action.go @@ -176,9 +176,9 @@ func (action *ChannelAction) Desc() string { } func (action *ChannelAction) IsFinished(distMgr *meta.DistributionManager) bool { - views := distMgr.LeaderViewManager.GetByFilter(meta.WithChannelName2LeaderView(action.ChannelName())) - _, hasNode := lo.Find(views, func(v *meta.LeaderView) bool { - return v.ID == action.Node() + delegator := distMgr.ChannelDistManager.GetByFilter(meta.WithChannelName2Channel(action.ChannelName())) + _, hasNode := lo.Find(delegator, func(v *meta.DmChannel) bool { + return v.Node == action.Node() }) isGrow := action.Type() == ActionTypeGrow diff --git a/internal/querycoordv2/task/executor.go b/internal/querycoordv2/task/executor.go index faca7776fbfad..321481964d21c 100644 --- a/internal/querycoordv2/task/executor.go +++ b/internal/querycoordv2/task/executor.go @@ -18,6 +18,7 @@ package task import ( "context" + "fmt" "sync" "time" @@ -215,18 +216,18 @@ func (ex *Executor) loadSegment(task *SegmentTask, step int) error { log.Warn(msg, zap.Error(err)) return err } - view := ex.dist.LeaderViewManager.GetLatestShardLeaderByFilter(meta.WithReplica2LeaderView(replica), meta.WithChannelName2LeaderView(action.Shard)) + view := ex.dist.ChannelDistManager.GetShardLeader(task.Shard(), replica) if view == nil { msg := "no shard leader for the segment to execute loading" err = merr.WrapErrChannelNotFound(task.Shard(), "shard delegator not found") log.Warn(msg, zap.Error(err)) return err } - log = log.With(zap.Int64("shardLeader", view.ID)) + log = log.With(zap.Int64("shardLeader", view.Node)) startTs := time.Now() log.Info("load segments...") - status, err := ex.cluster.LoadSegments(task.Context(), view.ID, req) + status, err := ex.cluster.LoadSegments(task.Context(), view.Node, req) err = merr.CheckRPCCall(status, err) if err != nil { log.Warn("failed to load segment", zap.Error(err)) @@ -282,15 +283,15 @@ func (ex *Executor) releaseSegment(task *SegmentTask, step int) { dstNode = action.Node() req.NeedTransfer = false } else { - view := ex.dist.LeaderViewManager.GetLatestShardLeaderByFilter(meta.WithReplica2LeaderView(replica), meta.WithChannelName2LeaderView(action.Shard)) + view := ex.dist.ChannelDistManager.GetShardLeader(task.Shard(), replica) if view == nil { msg := "no shard leader for the segment to execute releasing" err := merr.WrapErrChannelNotFound(task.Shard(), "shard delegator not found") log.Warn(msg, zap.Error(err)) return } - dstNode = view.ID - log = log.With(zap.Int64("shardLeader", view.ID)) + dstNode = view.Node + log = log.With(zap.Int64("shardLeader", view.Node)) req.NeedTransfer = true } } @@ -376,6 +377,13 @@ func (ex *Executor) subscribeChannel(task *ChannelTask, step int) error { log.Warn(msg, zap.String("channelName", action.ChannelName())) return merr.WrapErrChannelReduplicate(action.ChannelName()) } + + partitions, err = utils.GetPartitions(ctx, ex.targetMgr, task.collectionID) + if err != nil { + log.Warn("failed to get partitions", zap.Error(err)) + return merr.WrapErrServiceInternal(fmt.Sprintf("failed to get partitions for collection=%d", task.CollectionID())) + } + req := packSubChannelRequest( task, action, @@ -383,6 +391,7 @@ func (ex *Executor) subscribeChannel(task *ChannelTask, step int) error { loadMeta, dmChannel, indexInfo, + partitions, ) err = fillSubChannelRequest(ctx, req, ex.broker, ex.shouldIncludeFlushedSegmentInfo(action.Node())) if err != nil { diff --git a/internal/querycoordv2/task/scheduler.go b/internal/querycoordv2/task/scheduler.go index 46b353325a56e..85f92896efd98 100644 --- a/internal/querycoordv2/task/scheduler.go +++ b/internal/querycoordv2/task/scheduler.go @@ -498,11 +498,9 @@ func (scheduler *taskScheduler) preAdd(task Task) error { taskType := GetTaskType(task) if taskType == TaskTypeMove { - views := scheduler.distMgr.LeaderViewManager.GetByFilter( - meta.WithChannelName2LeaderView(task.Shard()), - meta.WithSegment2LeaderView(task.SegmentID(), false)) - if len(views) == 0 { - return merr.WrapErrServiceInternal("segment's delegator not found, stop balancing") + leader := scheduler.distMgr.ChannelDistManager.GetShardLeader(task.Shard(), task.replica) + if leader == nil { + return merr.WrapErrServiceInternal("segment's delegator leader not found, stop balancing") } segmentInTargetNode := scheduler.distMgr.SegmentDistManager.GetByFilter(meta.WithNodeID(task.Actions()[1].Node()), meta.WithSegmentID(task.SegmentID())) if len(segmentInTargetNode) == 0 { @@ -530,15 +528,15 @@ func (scheduler *taskScheduler) preAdd(task Task) error { taskType := GetTaskType(task) if taskType == TaskTypeGrow { - views := scheduler.distMgr.LeaderViewManager.GetByFilter(meta.WithChannelName2LeaderView(task.Channel())) - nodesWithChannel := lo.Map(views, func(v *meta.LeaderView, _ int) UniqueID { return v.ID }) + delegatorList := scheduler.distMgr.ChannelDistManager.GetByFilter(meta.WithChannelName2Channel(task.Channel())) + nodesWithChannel := lo.Map(delegatorList, func(v *meta.DmChannel, _ int) UniqueID { return v.Node }) replicaNodeMap := utils.GroupNodesByReplica(task.ctx, scheduler.meta.ReplicaManager, task.CollectionID(), nodesWithChannel) if _, ok := replicaNodeMap[task.ReplicaID()]; ok { return merr.WrapErrServiceInternal("channel subscribed, it can be only balanced") } } else if taskType == TaskTypeMove { - views := scheduler.distMgr.LeaderViewManager.GetByFilter(meta.WithChannelName2LeaderView(task.Channel())) - _, ok := lo.Find(views, func(v *meta.LeaderView) bool { return v.ID == task.Actions()[1].Node() }) + delegatorList := scheduler.distMgr.ChannelDistManager.GetByFilter(meta.WithChannelName2Channel(task.Channel())) + _, ok := lo.Find(delegatorList, func(v *meta.DmChannel) bool { return v.Node == task.Actions()[1].Node() }) if !ok { return merr.WrapErrServiceInternal("source channel unsubscribed, stop balancing") } @@ -827,15 +825,14 @@ func (scheduler *taskScheduler) isRelated(task Task, node int64) bool { if segment == nil { continue } - replica := scheduler.meta.ReplicaManager.GetByCollectionAndNode(task.ctx, task.CollectionID(), action.Node()) - if replica == nil { + if task.replica == nil { continue } - leader, ok := scheduler.distMgr.GetShardLeader(replica, segment.GetInsertChannel()) - if !ok { + leader := scheduler.distMgr.ChannelDistManager.GetShardLeader(task.Shard(), task.replica) + if leader == nil { continue } - if leader == node { + if leader.Node == node { return true } } @@ -855,24 +852,23 @@ func (scheduler *taskScheduler) preProcess(task Task) bool { actions, step := task.Actions(), task.Step() for step < len(actions) && actions[step].IsFinished(scheduler.distMgr) { if GetTaskType(task) == TaskTypeMove && actions[step].Type() == ActionTypeGrow { - var ready bool - switch actions[step].(type) { + var newDelegatorReady bool + switch action := actions[step].(type) { case *ChannelAction: - // if balance channel task has finished grow action, block reduce action until - // segment distribution has been sync to new delegator, cause new delegator may - // causes a few time to load delta log, if reduce the old delegator in advance, - // new delegator can't service search and query, will got no available channel error - channelAction := actions[step].(*ChannelAction) - leader := scheduler.distMgr.LeaderViewManager.GetLeaderShardView(channelAction.Node(), channelAction.Shard) - ready = leader.UnServiceableError == nil + // wait for new delegator becomes leader, then try to remove old leader + task := task.(*ChannelTask) + delegator := scheduler.distMgr.ChannelDistManager.GetShardLeader(task.Shard(), task.replica) + newDelegatorReady = delegator != nil && delegator.Node == action.Node() default: - ready = true + newDelegatorReady = true } - - if !ready { - log.Ctx(scheduler.ctx).WithRateGroup("qcv2.taskScheduler", 1, 60).RatedInfo(30, "Blocking reduce action in balance channel task", - zap.Int64("collectionID", task.CollectionID()), - zap.Int64("taskID", task.ID())) + if !newDelegatorReady { + log.Ctx(scheduler.ctx). + WithRateGroup("qcv2.preProcess", 1, 60). + RatedInfo(30, "Blocking reduce action in balance channel task", + zap.Int64("collectionID", task.CollectionID()), + zap.String("channelName", task.Shard()), + zap.Int64("taskID", task.ID())) break } } @@ -1112,13 +1108,8 @@ func (scheduler *taskScheduler) checkSegmentTaskStale(task *SegmentTask) error { return merr.WrapErrSegmentReduplicate(task.SegmentID(), "target doesn't contain this segment") } - replica := scheduler.meta.ReplicaManager.GetByCollectionAndNode(task.ctx, task.CollectionID(), action.Node()) - if replica == nil { - log.Ctx(task.Context()).Warn("task stale due to replica not found", WrapTaskLog(task)...) - return merr.WrapErrReplicaNotFound(task.CollectionID(), "by collectionID") - } - _, ok := scheduler.distMgr.GetShardLeader(replica, segment.GetInsertChannel()) - if !ok { + leader := scheduler.distMgr.ChannelDistManager.GetShardLeader(task.Shard(), task.replica) + if leader == nil { log.Ctx(task.Context()).Warn("task stale due to leader not found", WrapTaskLog(task)...) return merr.WrapErrChannelNotFound(segment.GetInsertChannel(), "failed to get shard delegator") } @@ -1171,21 +1162,15 @@ func (scheduler *taskScheduler) checkLeaderTaskStale(task *LeaderTask) error { return merr.WrapErrSegmentReduplicate(task.SegmentID(), "target doesn't contain this segment") } - replica := scheduler.meta.ReplicaManager.GetByCollectionAndNode(task.ctx, task.CollectionID(), action.Node()) - if replica == nil { - log.Ctx(task.Context()).Warn("task stale due to replica not found", WrapTaskLog(task, zap.Int64("leaderID", task.leaderID))...) - return merr.WrapErrReplicaNotFound(task.CollectionID(), "by collectionID") - } - - view := scheduler.distMgr.GetLeaderShardView(task.leaderID, task.Shard()) - if view == nil { + leader := scheduler.distMgr.ChannelDistManager.GetShardLeader(task.Shard(), task.replica) + if leader == nil { log.Ctx(task.Context()).Warn("task stale due to leader not found", WrapTaskLog(task, zap.Int64("leaderID", task.leaderID))...) return merr.WrapErrChannelNotFound(task.Shard(), "failed to get shard delegator") } case ActionTypeReduce: - view := scheduler.distMgr.GetLeaderShardView(task.leaderID, task.Shard()) - if view == nil { + leader := scheduler.distMgr.ChannelDistManager.GetShardLeader(task.Shard(), task.replica) + if leader == nil { log.Ctx(task.Context()).Warn("task stale due to leader not found", WrapTaskLog(task, zap.Int64("leaderID", task.leaderID))...) return merr.WrapErrChannelNotFound(task.Shard(), "failed to get shard delegator") } diff --git a/internal/querycoordv2/task/task_test.go b/internal/querycoordv2/task/task_test.go index ec1429a1aa335..cd743bc895ed0 100644 --- a/internal/querycoordv2/task/task_test.go +++ b/internal/querycoordv2/task/task_test.go @@ -98,7 +98,13 @@ func (suite *TaskSuite) SetupSuite() { params.Save(params.EtcdCfg.Endpoints.Key, strings.Join(addressList, ",")) suite.collection = 1000 - suite.replica = newReplicaDefaultRG(10) + suite.replica = meta.NewReplica(&querypb.Replica{ + CollectionID: suite.collection, + ID: 10, + ResourceGroup: meta.DefaultResourceGroupName, + Nodes: []int64{1, 2, 3}, + }, typeutil.NewUniqueSet(1, 2, 3)) + suite.subChannels = []string{ "sub-0", "sub-1", @@ -289,15 +295,23 @@ func (suite *TaskSuite) TestSubscribeChannelTask() { // Process tasks done // Dist contains channels - views := make([]*meta.LeaderView, 0) + channels := []*meta.DmChannel{} for _, channel := range suite.subChannels { - views = append(views, &meta.LeaderView{ - ID: targetNode, - CollectionID: suite.collection, - Channel: channel, + channels = append(channels, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: channel, + }, + Node: targetNode, + Version: 1, + View: &meta.LeaderView{ + ID: targetNode, + CollectionID: suite.collection, + Channel: channel, + }, }) } - suite.dist.LeaderViewManager.Update(targetNode, views...) + suite.dist.ChannelDistManager.Update(targetNode, channels...) suite.dispatchAndWait(targetNode) suite.AssertTaskNum(0, 0, 0, 0) @@ -327,15 +341,23 @@ func (suite *TaskSuite) TestSubmitDuplicateSubscribeChannelTask() { tasks = append(tasks, task) } - views := make([]*meta.LeaderView, 0) + channels := []*meta.DmChannel{} for _, channel := range suite.subChannels { - views = append(views, &meta.LeaderView{ - ID: targetNode, - CollectionID: suite.collection, - Channel: channel, + channels = append(channels, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: channel, + }, + Node: targetNode, + Version: 1, + View: &meta.LeaderView{ + ID: targetNode, + CollectionID: suite.collection, + Channel: channel, + }, }) } - suite.dist.LeaderViewManager.Update(targetNode, views...) + suite.dist.ChannelDistManager.Update(targetNode, channels...) for _, task := range tasks { err := suite.scheduler.Add(task) @@ -378,10 +400,18 @@ func (suite *TaskSuite) TestUnsubscribeChannelTask() { suite.target.UpdateCollectionNextTarget(ctx, suite.collection) // Only first channel exists - suite.dist.LeaderViewManager.Update(targetNode, &meta.LeaderView{ - ID: targetNode, - CollectionID: suite.collection, - Channel: suite.unsubChannels[0], + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: suite.unsubChannels[0], + }, + Node: targetNode, + Version: 1, + View: &meta.LeaderView{ + ID: targetNode, + CollectionID: suite.collection, + Channel: suite.unsubChannels[0], + }, }) suite.AssertTaskNum(0, len(suite.unsubChannels), len(suite.unsubChannels), 0) @@ -390,7 +420,7 @@ func (suite *TaskSuite) TestUnsubscribeChannelTask() { suite.AssertTaskNum(1, 0, 1, 0) // Update dist - suite.dist.LeaderViewManager.Update(targetNode) + suite.dist.ChannelDistManager.Update(targetNode) suite.dispatchAndWait(targetNode) suite.AssertTaskNum(0, 0, 0, 0) @@ -440,11 +470,17 @@ func (suite *TaskSuite) TestLoadSegmentTask() { suite.cluster.EXPECT().LoadSegments(mock.Anything, targetNode, mock.Anything).Return(merr.Success(), nil) // Test load segment task - suite.dist.ChannelDistManager.Update(targetNode, meta.DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: suite.collection, - ChannelName: channel.ChannelName, - })) - suite.dist.LeaderViewManager.Update(targetNode, utils.CreateTestLeaderView(targetNode, suite.collection, channel.ChannelName, map[int64]int64{}, map[int64]*meta.Segment{})) + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: channel, + Node: targetNode, + Version: 1, + View: &meta.LeaderView{ + ID: targetNode, + CollectionID: suite.collection, + Channel: channel.ChannelName, + Status: &querypb.LeaderViewStatus{Serviceable: true}, + }, + }) tasks := []Task{} segments := make([]*datapb.SegmentInfo, 0) for _, segment := range suite.loadSegments { @@ -490,7 +526,15 @@ func (suite *TaskSuite) TestLoadSegmentTask() { distSegments := lo.Map(segments, func(info *datapb.SegmentInfo, _ int) *meta.Segment { return meta.SegmentFromInfo(info) }) - suite.dist.LeaderViewManager.Update(targetNode, view) + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: channel.ChannelName, + }, + Node: targetNode, + Version: 1, + View: view, + }) suite.dist.SegmentDistManager.Update(targetNode, distSegments...) suite.dispatchAndWait(targetNode) suite.AssertTaskNum(0, 0, 0, 0) @@ -541,11 +585,15 @@ func (suite *TaskSuite) TestLoadSegmentTaskNotIndex() { suite.cluster.EXPECT().LoadSegments(mock.Anything, targetNode, mock.Anything).Return(merr.Success(), nil) // Test load segment task - suite.dist.ChannelDistManager.Update(targetNode, meta.DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: suite.collection, - ChannelName: channel.ChannelName, - })) - suite.dist.LeaderViewManager.Update(targetNode, utils.CreateTestLeaderView(targetNode, suite.collection, channel.ChannelName, map[int64]int64{}, map[int64]*meta.Segment{})) + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: channel.ChannelName, + }, + Node: targetNode, + Version: 1, + View: &meta.LeaderView{ID: targetNode, CollectionID: suite.collection, Channel: channel.ChannelName, Status: &querypb.LeaderViewStatus{Serviceable: true}}, + }) tasks := []Task{} segments := make([]*datapb.SegmentInfo, 0) for _, segment := range suite.loadSegments { @@ -590,7 +638,6 @@ func (suite *TaskSuite) TestLoadSegmentTaskNotIndex() { distSegments := lo.Map(segments, func(info *datapb.SegmentInfo, _ int) *meta.Segment { return meta.SegmentFromInfo(info) }) - suite.dist.LeaderViewManager.Update(targetNode, view) suite.dist.SegmentDistManager.Update(targetNode, distSegments...) suite.dispatchAndWait(targetNode) suite.AssertTaskNum(0, 0, 0, 0) @@ -635,11 +682,16 @@ func (suite *TaskSuite) TestLoadSegmentTaskFailed() { } // Test load segment task - suite.dist.ChannelDistManager.Update(targetNode, meta.DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: suite.collection, - ChannelName: channel.ChannelName, - })) - suite.dist.LeaderViewManager.Update(targetNode, utils.CreateTestLeaderView(targetNode, suite.collection, channel.ChannelName, map[int64]int64{}, map[int64]*meta.Segment{})) + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: channel.ChannelName, + }, + Node: targetNode, + Version: 1, + View: &meta.LeaderView{ID: targetNode, CollectionID: suite.collection, Channel: channel.ChannelName, Status: &querypb.LeaderViewStatus{Serviceable: true}}, + }) + tasks := []Task{} segments := make([]*datapb.SegmentInfo, 0) for _, segment := range suite.loadSegments { @@ -727,7 +779,6 @@ func (suite *TaskSuite) TestReleaseSegmentTask() { suite.NoError(err) } suite.dist.SegmentDistManager.Update(targetNode, segments...) - suite.dist.LeaderViewManager.Update(targetNode, view) segmentsNum := len(suite.releaseSegments) suite.AssertTaskNum(0, segmentsNum, 0, segmentsNum) @@ -737,7 +788,6 @@ func (suite *TaskSuite) TestReleaseSegmentTask() { suite.AssertTaskNum(segmentsNum, 0, 0, segmentsNum) // Process tasks done - suite.dist.LeaderViewManager.Update(targetNode) suite.dispatchAndWait(targetNode) suite.AssertTaskNum(0, 0, 0, 0) @@ -833,12 +883,23 @@ func (suite *TaskSuite) TestMoveSegmentTask() { CollectionID: suite.collection, ChannelName: channel.ChannelName, } - suite.dist.ChannelDistManager.Update(leader, meta.DmChannelFromVChannel(vchannel)) + suite.dist.ChannelDistManager.Update(leader, &meta.DmChannel{ + VchannelInfo: vchannel, + Node: leader, + Version: 1, + View: &meta.LeaderView{ + ID: leader, + CollectionID: suite.collection, + Channel: channel.ChannelName, + Status: &querypb.LeaderViewStatus{Serviceable: true}, + }, + }) view := &meta.LeaderView{ ID: leader, CollectionID: suite.collection, Channel: channel.ChannelName, Segments: make(map[int64]*querypb.SegmentDist), + Status: &querypb.LeaderViewStatus{Serviceable: true}, } tasks := []Task{} segments := make([]*meta.Segment, 0) @@ -869,7 +930,14 @@ func (suite *TaskSuite) TestMoveSegmentTask() { suite.target.UpdateCollectionNextTarget(ctx, suite.collection) suite.target.UpdateCollectionCurrentTarget(ctx, suite.collection) suite.dist.SegmentDistManager.Update(sourceNode, segments...) - suite.dist.LeaderViewManager.Update(leader, view) + suite.dist.ChannelDistManager.Update(leader, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: channel.ChannelName, + }, + Node: leader, + View: view, + }) for _, task := range tasks { err := suite.scheduler.Add(task) suite.NoError(err) @@ -891,7 +959,6 @@ func (suite *TaskSuite) TestMoveSegmentTask() { return meta.SegmentFromInfo(info) }) - suite.dist.LeaderViewManager.Update(leader, view) suite.dist.SegmentDistManager.Update(targetNode, distSegments...) // First action done, execute the second action suite.dispatchAndWait(leader) @@ -920,7 +987,17 @@ func (suite *TaskSuite) TestMoveSegmentTaskStale() { CollectionID: suite.collection, ChannelName: channel.ChannelName, } - suite.dist.ChannelDistManager.Update(leader, meta.DmChannelFromVChannel(vchannel)) + suite.dist.ChannelDistManager.Update(leader, &meta.DmChannel{ + VchannelInfo: vchannel, + Node: leader, + Version: 1, + View: &meta.LeaderView{ + ID: leader, + CollectionID: suite.collection, + Channel: channel.ChannelName, + Status: &querypb.LeaderViewStatus{Serviceable: true}, + }, + }) view := &meta.LeaderView{ ID: leader, CollectionID: suite.collection, @@ -952,7 +1029,6 @@ func (suite *TaskSuite) TestMoveSegmentTaskStale() { suite.broker.EXPECT().GetRecoveryInfoV2(mock.Anything, suite.collection).Return([]*datapb.VchannelInfo{vchannel}, segmentInfos, nil) suite.target.UpdateCollectionNextTarget(ctx, suite.collection) suite.target.UpdateCollectionCurrentTarget(ctx, suite.collection) - suite.dist.LeaderViewManager.Update(leader, view) for _, task := range tasks { err := suite.scheduler.Add(task) suite.Error(err) @@ -1002,11 +1078,16 @@ func (suite *TaskSuite) TestTaskCanceled() { suite.cluster.EXPECT().LoadSegments(mock.Anything, targetNode, mock.Anything).Return(merr.Success(), nil) // Test load segment task - suite.dist.ChannelDistManager.Update(targetNode, meta.DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: suite.collection, - ChannelName: channel.ChannelName, - })) - suite.dist.LeaderViewManager.Update(targetNode, utils.CreateTestLeaderView(targetNode, suite.collection, channel.ChannelName, map[int64]int64{}, map[int64]*meta.Segment{})) + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: channel.ChannelName, + }, + Node: targetNode, + Version: 1, + View: &meta.LeaderView{ID: targetNode, CollectionID: suite.collection, Channel: channel.ChannelName, Status: &querypb.LeaderViewStatus{Serviceable: true}}, + }) + tasks := []Task{} segmentInfos := []*datapb.SegmentInfo{} for _, segment := range suite.loadSegments { @@ -1092,12 +1173,15 @@ func (suite *TaskSuite) TestSegmentTaskStale() { suite.cluster.EXPECT().LoadSegments(mock.Anything, targetNode, mock.Anything).Return(merr.Success(), nil) // Test load segment task - suite.meta.ReplicaManager.Put(ctx, createReplica(suite.collection, targetNode)) - suite.dist.ChannelDistManager.Update(targetNode, meta.DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: suite.collection, - ChannelName: channel.ChannelName, - })) - suite.dist.LeaderViewManager.Update(targetNode, utils.CreateTestLeaderView(targetNode, suite.collection, channel.ChannelName, map[int64]int64{}, map[int64]*meta.Segment{})) + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: channel.ChannelName, + }, + Node: targetNode, + Version: 1, + View: &meta.LeaderView{ID: targetNode, CollectionID: suite.collection, Channel: channel.ChannelName, Status: &querypb.LeaderViewStatus{Serviceable: true}}, + }) tasks := []Task{} for _, segment := range suite.loadSegments { task, err := NewSegmentTask( @@ -1244,10 +1328,12 @@ func (suite *TaskSuite) TestLeaderTaskSet() { suite.cluster.EXPECT().SyncDistribution(mock.Anything, targetNode, mock.Anything).Return(merr.Success(), nil) // Test load segment task - suite.dist.ChannelDistManager.Update(targetNode, meta.DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: suite.collection, - ChannelName: channel.ChannelName, - })) + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: channel, + Node: targetNode, + Version: 1, + View: &meta.LeaderView{ID: targetNode, CollectionID: suite.collection, Channel: channel.ChannelName, Status: &querypb.LeaderViewStatus{Serviceable: true}}, + }) tasks := []Task{} segments := make([]*datapb.SegmentInfo, 0) for _, segment := range suite.loadSegments { @@ -1278,8 +1364,16 @@ func (suite *TaskSuite) TestLeaderTaskSet() { CollectionID: suite.collection, Channel: channel.GetChannelName(), Segments: map[int64]*querypb.SegmentDist{}, + Status: &querypb.LeaderViewStatus{Serviceable: true}, } - suite.dist.LeaderViewManager.Update(targetNode, view) + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: channel.GetChannelName(), + }, + Node: targetNode, + View: view, + }) // Process tasks suite.dispatchAndWait(targetNode) @@ -1299,7 +1393,14 @@ func (suite *TaskSuite) TestLeaderTaskSet() { distSegments := lo.Map(segments, func(info *datapb.SegmentInfo, _ int) *meta.Segment { return meta.SegmentFromInfo(info) }) - suite.dist.LeaderViewManager.Update(targetNode, view) + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: channel.GetChannelName(), + }, + Node: targetNode, + View: view, + }) suite.dist.SegmentDistManager.Update(targetNode, distSegments...) suite.dispatchAndWait(targetNode) suite.AssertTaskNum(0, 0, 0, 0) @@ -1418,10 +1519,12 @@ func (suite *TaskSuite) TestNoExecutor() { suite.meta.ReplicaManager.Put(ctx, utils.CreateTestReplica(suite.replica.GetID(), suite.collection, []int64{1, 2, 3, -1})) // Test load segment task - suite.dist.ChannelDistManager.Update(targetNode, meta.DmChannelFromVChannel(&datapb.VchannelInfo{ - CollectionID: suite.collection, - ChannelName: channel.ChannelName, - })) + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: channel, + Node: targetNode, + Version: 1, + View: &meta.LeaderView{ID: targetNode, CollectionID: suite.collection, Channel: channel.ChannelName, Status: &querypb.LeaderViewStatus{Serviceable: true}}, + }) segments := make([]*datapb.SegmentInfo, 0) for _, segment := range suite.loadSegments { segments = append(segments, &datapb.SegmentInfo{ @@ -1517,6 +1620,7 @@ func (suite *TaskSuite) TestLeaderTaskRemove() { CollectionID: suite.collection, Channel: channel.ChannelName, Segments: make(map[int64]*querypb.SegmentDist), + Status: &querypb.LeaderViewStatus{Serviceable: true}, } segments := make([]*meta.Segment, 0) tasks := []Task{} @@ -1543,7 +1647,15 @@ func (suite *TaskSuite) TestLeaderTaskRemove() { suite.NoError(err) } suite.dist.SegmentDistManager.Update(targetNode, segments...) - suite.dist.LeaderViewManager.Update(targetNode, view) + suite.dist.ChannelDistManager.Update(targetNode, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: suite.collection, + ChannelName: channel.ChannelName, + }, + Node: targetNode, + Version: 1, + View: view, + }) segmentsNum := len(suite.releaseSegments) suite.AssertTaskNum(0, segmentsNum, 0, segmentsNum) @@ -1552,10 +1664,9 @@ func (suite *TaskSuite) TestLeaderTaskRemove() { suite.dispatchAndWait(targetNode) suite.AssertTaskNum(segmentsNum, 0, 0, segmentsNum) + // mock leader view which has removed all segments view.Segments = make(map[int64]*querypb.SegmentDist) - suite.dist.LeaderViewManager.Update(targetNode, view) // Process tasks done - // suite.dist.LeaderViewManager.Update(targetNode) suite.dispatchAndWait(targetNode) suite.AssertTaskNum(0, 0, 0, 0) @@ -1590,7 +1701,7 @@ func createReplica(collection int64, nodes ...int64) *meta.Replica { func (suite *TaskSuite) TestBalanceChannelTask() { ctx := context.Background() - collectionID := int64(1) + collectionID := suite.collection partitionID := int64(1) channel := "channel-1" vchannel := &datapb.VchannelInfo{ @@ -1625,27 +1736,25 @@ func (suite *TaskSuite) TestBalanceChannelTask() { suite.target.UpdateCollectionCurrentTarget(ctx, collectionID) suite.target.UpdateCollectionNextTarget(ctx, collectionID) - suite.dist.LeaderViewManager.Update(2, &meta.LeaderView{ - ID: 2, - CollectionID: collectionID, - Channel: channel, - Segments: map[int64]*querypb.SegmentDist{ - 1: {}, - 2: {}, - 3: {}, + suite.dist.ChannelDistManager.Update(2, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: collectionID, + ChannelName: channel, + }, + Node: 2, + Version: 1, + View: &meta.LeaderView{ + ID: 2, + CollectionID: collectionID, + Channel: channel, + Status: &querypb.LeaderViewStatus{Serviceable: true}, }, - }) - suite.dist.LeaderViewManager.Update(1, &meta.LeaderView{ - ID: 1, - CollectionID: collectionID, - Channel: channel, - UnServiceableError: merr.ErrSegmentLack, }) task, err := NewChannelTask(context.Background(), 10*time.Second, WrapIDSource(2), collectionID, - meta.NilReplica, + suite.replica, NewChannelAction(1, ActionTypeGrow, channel), NewChannelAction(2, ActionTypeReduce, channel), ) @@ -1655,14 +1764,19 @@ func (suite *TaskSuite) TestBalanceChannelTask() { suite.scheduler.preProcess(task) suite.Equal(0, task.step) - suite.dist.LeaderViewManager.Update(1, &meta.LeaderView{ - ID: 1, - CollectionID: collectionID, - Channel: channel, - Segments: map[int64]*querypb.SegmentDist{ - 1: {}, - 2: {}, - 3: {}, + suite.dist.ChannelDistManager.Update(1, &meta.DmChannel{ + VchannelInfo: &datapb.VchannelInfo{ + CollectionID: collectionID, + ChannelName: channel, + }, + Node: 1, + Version: 2, + View: &meta.LeaderView{ + ID: 1, + CollectionID: collectionID, + Channel: channel, + Version: 2, + Status: &querypb.LeaderViewStatus{Serviceable: true}, }, }) @@ -1670,7 +1784,7 @@ func (suite *TaskSuite) TestBalanceChannelTask() { suite.scheduler.preProcess(task) suite.Equal(1, task.step) - suite.dist.LeaderViewManager.Update(2) + suite.dist.ChannelDistManager.Update(2) // old delegator removed suite.scheduler.preProcess(task) suite.Equal(2, task.step) diff --git a/internal/querycoordv2/task/utils.go b/internal/querycoordv2/task/utils.go index 815a450b658eb..cef042e949a33 100644 --- a/internal/querycoordv2/task/utils.go +++ b/internal/querycoordv2/task/utils.go @@ -207,6 +207,7 @@ func packSubChannelRequest( loadMeta *querypb.LoadMetaInfo, channel *meta.DmChannel, indexInfo []*indexpb.IndexInfo, + partitions []int64, ) *querypb.WatchDmChannelsRequest { return &querypb.WatchDmChannelsRequest{ Base: commonpbutil.NewMsgBase( @@ -215,6 +216,7 @@ func packSubChannelRequest( ), NodeID: action.Node(), CollectionID: task.CollectionID(), + PartitionIDs: partitions, Infos: []*datapb.VchannelInfo{channel.VchannelInfo}, Schema: schema, // assign it for compatibility of rolling upgrade from 2.2.x to 2.3 LoadMeta: loadMeta, // assign it for compatibility of rolling upgrade from 2.2.x to 2.3 diff --git a/internal/querycoordv2/utils/util.go b/internal/querycoordv2/utils/util.go index 93d20ef68f402..afd17490b1f22 100644 --- a/internal/querycoordv2/utils/util.go +++ b/internal/querycoordv2/utils/util.go @@ -21,7 +21,6 @@ import ( "fmt" "time" - "go.uber.org/multierr" "go.uber.org/zap" "github.com/milvus-io/milvus/internal/querycoordv2/meta" @@ -105,36 +104,21 @@ func GetShardLeadersWithChannels(ctx context.Context, m *meta.Meta, targetMgr me nodeMgr *session.NodeManager, collectionID int64, channels map[string]*meta.DmChannel, ) ([]*querypb.ShardLeadersList, error) { ret := make([]*querypb.ShardLeadersList, 0) + + replicas := m.ReplicaManager.GetByCollection(ctx, collectionID) for _, channel := range channels { log := log.Ctx(ctx).With(zap.String("channel", channel.GetChannelName())) - var channelErr error - leaders := dist.LeaderViewManager.GetByFilter(meta.WithChannelName2LeaderView(channel.GetChannelName())) - if len(leaders) == 0 { - channelErr = merr.WrapErrChannelLack(channel.GetChannelName(), "channel not subscribed") - } - - readableLeaders := make(map[int64]*meta.LeaderView) - for _, leader := range leaders { - if leader.UnServiceableError != nil { - multierr.AppendInto(&channelErr, leader.UnServiceableError) + ids := make([]int64, 0, len(replicas)) + addrs := make([]string, 0, len(replicas)) + for _, replica := range replicas { + leader := dist.ChannelDistManager.GetShardLeader(channel.GetChannelName(), replica) + if leader == nil || !leader.IsServiceable() { + log.WithRateGroup("util.GetShardLeaders", 1, 60). + Warn("leader is not available in replica", zap.String("channel", channel.GetChannelName()), zap.Int64("replicaID", replica.GetID())) continue } - readableLeaders[leader.ID] = leader - } - - if len(readableLeaders) == 0 { - msg := fmt.Sprintf("channel %s is not available in any replica", channel.GetChannelName()) - log.Warn(msg, zap.Error(channelErr)) - err := merr.WrapErrChannelNotAvailable(channel.GetChannelName(), channelErr.Error()) - return nil, err - } - - readableLeaders = filterDupLeaders(ctx, m.ReplicaManager, readableLeaders) - ids := make([]int64, 0, len(leaders)) - addrs := make([]string, 0, len(leaders)) - for _, leader := range readableLeaders { - info := nodeMgr.Get(leader.ID) + info := nodeMgr.Get(leader.Node) if info != nil { ids = append(ids, info.ID()) addrs = append(addrs, info.Addr()) @@ -143,12 +127,9 @@ func GetShardLeadersWithChannels(ctx context.Context, m *meta.Meta, targetMgr me // to avoid node down during GetShardLeaders if len(ids) == 0 { - if channelErr == nil { - channelErr = merr.WrapErrChannelNotAvailable(channel.GetChannelName()) - } + err := merr.WrapErrChannelNotAvailable(channel.GetChannelName()) msg := fmt.Sprintf("channel %s is not available in any replica", channel.GetChannelName()) - log.Warn(msg, zap.Error(channelErr)) - err := merr.WrapErrChannelNotAvailable(channel.GetChannelName(), channelErr.Error()) + log.Warn(msg, zap.Error(err)) return nil, err } diff --git a/internal/querynodev2/delegator/delegator.go b/internal/querynodev2/delegator/delegator.go index 13c340518cbf8..972a473da2ec9 100644 --- a/internal/querynodev2/delegator/delegator.go +++ b/internal/querynodev2/delegator/delegator.go @@ -30,6 +30,8 @@ import ( "go.opentelemetry.io/otel" "go.uber.org/atomic" "go.uber.org/zap" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/status" "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" @@ -87,7 +89,7 @@ type ShardDelegator interface { LoadSegments(ctx context.Context, req *querypb.LoadSegmentsRequest) error ReleaseSegments(ctx context.Context, req *querypb.ReleaseSegmentsRequest, force bool) error SyncTargetVersion(newVersion int64, partitions []int64, growingInTarget []int64, sealedInTarget []int64, droppedInTarget []int64, checkpoint *msgpb.MsgPosition, deleteSeekPos *msgpb.MsgPosition) - GetTargetVersion() int64 + GetQueryView() *channelQueryView GetDeleteBufferSize() (entryNum int64, memorySize int64) // manage exclude segments @@ -327,7 +329,12 @@ func (sd *shardDelegator) search(ctx context.Context, req *querypb.SearchRequest return nil, err } results, err := executeSubTasks(ctx, tasks, func(ctx context.Context, req *querypb.SearchRequest, worker cluster.Worker) (*internalpb.SearchResults, error) { - return worker.SearchSegments(ctx, req) + resp, err := worker.SearchSegments(ctx, req) + status, ok := status.FromError(err) + if ok && status.Code() == codes.Unavailable { + sd.markSegmentOffline(req.GetSegmentIDs()...) + } + return resp, err }, "Search", log) if err != nil { log.Warn("Delegator search failed", zap.Error(err)) @@ -515,7 +522,12 @@ func (sd *shardDelegator) QueryStream(ctx context.Context, req *querypb.QueryReq } _, err = executeSubTasks(ctx, tasks, func(ctx context.Context, req *querypb.QueryRequest, worker cluster.Worker) (*internalpb.RetrieveResults, error) { - return nil, worker.QueryStreamSegments(ctx, req, srv) + err := worker.QueryStreamSegments(ctx, req, srv) + status, ok := status.FromError(err) + if ok && status.Code() == codes.Unavailable { + sd.markSegmentOffline(req.GetSegmentIDs()...) + } + return nil, err }, "Query", log) if err != nil { log.Warn("Delegator query failed", zap.Error(err)) @@ -595,7 +607,12 @@ func (sd *shardDelegator) Query(ctx context.Context, req *querypb.QueryRequest) } results, err := executeSubTasks(ctx, tasks, func(ctx context.Context, req *querypb.QueryRequest, worker cluster.Worker) (*internalpb.RetrieveResults, error) { - return worker.QuerySegments(ctx, req) + resp, err := worker.QuerySegments(ctx, req) + status, ok := status.FromError(err) + if ok && status.Code() == codes.Unavailable { + sd.markSegmentOffline(req.GetSegmentIDs()...) + } + return resp, err }, "Query", log) if err != nil { log.Warn("Delegator query failed", zap.Error(err)) @@ -1019,7 +1036,7 @@ func NewShardDelegator(ctx context.Context, collectionID UniqueID, replicaID Uni segmentManager: manager.Segment, workerManager: workerManager, lifetime: lifetime.NewLifetime(lifetime.Initializing), - distribution: NewDistribution(), + distribution: NewDistribution(channel), deleteBuffer: deletebuffer.NewListDeleteBuffer[*deletebuffer.Item](startTs, sizePerBlock, []string{fmt.Sprint(paramtable.GetNodeID()), channel}), pkOracle: pkoracle.NewPkOracle(), diff --git a/internal/querynodev2/delegator/delegator_data.go b/internal/querynodev2/delegator/delegator_data.go index a28a400275720..a612abec28193 100644 --- a/internal/querynodev2/delegator/delegator_data.go +++ b/internal/querynodev2/delegator/delegator_data.go @@ -329,7 +329,7 @@ func (sd *shardDelegator) applyDelete(ctx context.Context, // markSegmentOffline makes segment go offline and waits for QueryCoord to fix. func (sd *shardDelegator) markSegmentOffline(segmentIDs ...int64) { - sd.distribution.AddOfflines(segmentIDs...) + sd.distribution.MarkOfflineSegments(segmentIDs...) } // addGrowing add growing segment record for delegator. @@ -1026,8 +1026,8 @@ func (sd *shardDelegator) SyncTargetVersion( sd.RefreshLevel0DeletionStats() } -func (sd *shardDelegator) GetTargetVersion() int64 { - return sd.distribution.getTargetVersion() +func (sd *shardDelegator) GetQueryView() *channelQueryView { + return sd.distribution.queryView } func (sd *shardDelegator) AddExcludedSegments(excludeInfo map[int64]uint64) { diff --git a/internal/querynodev2/delegator/delegator_data_test.go b/internal/querynodev2/delegator/delegator_data_test.go index 4afc329b2444d..9baf57cf44790 100644 --- a/internal/querynodev2/delegator/delegator_data_test.go +++ b/internal/querynodev2/delegator/delegator_data_test.go @@ -414,9 +414,12 @@ func (s *DelegatorDataSuite) TestProcessDelete() { InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", s.collectionID), }, }, + Version: time.Now().UnixNano(), }) s.Require().NoError(err) + // sync target version, make delegator serviceable + s.delegator.SyncTargetVersion(time.Now().UnixNano(), []int64{500}, []int64{1001}, []int64{1000}, nil, &msgpb.MsgPosition{}, &msgpb.MsgPosition{}) s.delegator.ProcessDelete([]*DeleteData{ { PartitionID: 500, @@ -471,7 +474,7 @@ func (s *DelegatorDataSuite) TestProcessDelete() { InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", s.collectionID), }, }, - Version: 1, + Version: time.Now().UnixNano(), }) s.Require().NoError(err) s.True(s.delegator.distribution.Serviceable()) @@ -506,7 +509,7 @@ func (s *DelegatorDataSuite) TestProcessDelete() { InsertChannel: fmt.Sprintf("by-dev-rootcoord-dml_0_%dv0", s.collectionID), }, }, - Version: 2, + Version: time.Now().UnixNano(), }) s.Require().NoError(err) s.True(s.delegator.distribution.Serviceable()) @@ -1409,7 +1412,7 @@ func (s *DelegatorDataSuite) TestSyncTargetVersion() { } s.delegator.SyncTargetVersion(int64(5), []int64{1}, []int64{1}, []int64{2}, []int64{3, 4}, &msgpb.MsgPosition{}, &msgpb.MsgPosition{}) - s.Equal(int64(5), s.delegator.GetTargetVersion()) + s.Equal(int64(5), s.delegator.GetQueryView().GetVersion()) } func (s *DelegatorDataSuite) TestLevel0Deletions() { diff --git a/internal/querynodev2/delegator/delegator_test.go b/internal/querynodev2/delegator/delegator_test.go index 8303bb1c86e71..6db0b7456ed20 100644 --- a/internal/querynodev2/delegator/delegator_test.go +++ b/internal/querynodev2/delegator/delegator_test.go @@ -522,7 +522,7 @@ func (s *DelegatorSuite) TestSearch() { sd, ok := s.delegator.(*shardDelegator) s.Require().True(ok) - sd.distribution.AddOfflines(1001) + sd.distribution.MarkOfflineSegments(1001) _, err := s.delegator.Search(ctx, &querypb.SearchRequest{ Req: &internalpb.SearchRequest{ @@ -709,7 +709,7 @@ func (s *DelegatorSuite) TestQuery() { sd, ok := s.delegator.(*shardDelegator) s.Require().True(ok) - sd.distribution.AddOfflines(1001) + sd.distribution.MarkOfflineSegments(1001) _, err := s.delegator.Query(ctx, &querypb.QueryRequest{ Req: &internalpb.RetrieveRequest{Base: commonpbutil.NewMsgBase()}, @@ -987,7 +987,7 @@ func (s *DelegatorSuite) TestQueryStream() { sd, ok := s.delegator.(*shardDelegator) s.Require().True(ok) - sd.distribution.AddOfflines(1001) + sd.distribution.MarkOfflineSegments(1001) client := streamrpc.NewLocalQueryClient(ctx) server := client.CreateServer() @@ -1164,7 +1164,7 @@ func (s *DelegatorSuite) TestGetStats() { sd, ok := s.delegator.(*shardDelegator) s.Require().True(ok) - sd.distribution.AddOfflines(1001) + sd.distribution.MarkOfflineSegments(1001) _, err := s.delegator.GetStatistics(ctx, &querypb.GetStatisticsRequest{ Req: &internalpb.GetStatisticsRequest{Base: commonpbutil.NewMsgBase()}, @@ -1265,7 +1265,7 @@ func (s *DelegatorSuite) TestUpdateSchema() { sd, ok := s.delegator.(*shardDelegator) s.Require().True(ok) - sd.distribution.AddOfflines(1001) + sd.distribution.MarkOfflineSegments(1001) err := s.delegator.UpdateSchema(ctx, &schemapb.CollectionSchema{}, 100) s.Error(err) diff --git a/internal/querynodev2/delegator/distribution.go b/internal/querynodev2/delegator/distribution.go index 4a0729074996e..1d5bb231813b7 100644 --- a/internal/querynodev2/delegator/distribution.go +++ b/internal/querynodev2/delegator/distribution.go @@ -55,22 +55,34 @@ func getClosedCh() chan struct{} { return closedCh } +// channelQueryView maintains the sealed segment list which should be used for search/query. +type channelQueryView struct { + sealedSegments []int64 // sealed segment list which should be used for search/query + partitions typeutil.UniqueSet // partitions list which sealed segments belong to + version int64 // version of current query view, same as targetVersion in qc + + serviceable *atomic.Bool +} + +func (q *channelQueryView) GetVersion() int64 { + return q.version +} + +func (q *channelQueryView) Serviceable() bool { + return q.serviceable.Load() +} + // distribution is the struct to store segment distribution. // it contains both growing and sealed segments. type distribution struct { // segments information // map[SegmentID]=>segmentEntry - targetVersion *atomic.Int64 growingSegments map[UniqueID]SegmentEntry sealedSegments map[UniqueID]SegmentEntry // snapshotVersion indicator snapshotVersion int64 - // quick flag for current snapshot is serviceable - serviceable *atomic.Bool - offlines typeutil.Set[int64] - - snapshots *typeutil.ConcurrentMap[int64, *snapshot] + snapshots *typeutil.ConcurrentMap[int64, *snapshot] // current is the snapshot for quick usage for search/query // generated for each change of distribution current *atomic.Pointer[snapshot] @@ -78,6 +90,10 @@ type distribution struct { idfOracle IDFOracle // protects current & segments mut sync.RWMutex + + // distribution info + channelName string + queryView *channelQueryView } // SegmentEntry stores the segment meta information. @@ -88,18 +104,22 @@ type SegmentEntry struct { Version int64 TargetVersion int64 Level datapb.SegmentLevel + Offline bool // if delegator failed to execute forwardDelete/Query/Search on segment, it will be offline } // NewDistribution creates a new distribution instance with all field initialized. -func NewDistribution() *distribution { +func NewDistribution(channelName string) *distribution { dist := &distribution{ - serviceable: atomic.NewBool(false), + channelName: channelName, growingSegments: make(map[UniqueID]SegmentEntry), sealedSegments: make(map[UniqueID]SegmentEntry), snapshots: typeutil.NewConcurrentMap[int64, *snapshot](), current: atomic.NewPointer[snapshot](nil), - offlines: typeutil.NewSet[int64](), - targetVersion: atomic.NewInt64(initialTargetVersion), + queryView: &channelQueryView{ + serviceable: atomic.NewBool(false), + partitions: typeutil.NewSet[int64](), + version: initialTargetVersion, + }, } dist.genSnapshot() @@ -142,13 +162,11 @@ func (d *distribution) PinOnlineSegments(partitions ...int64) (sealed []Snapshot current := d.current.Load() sealed, growing = current.Get(partitions...) - version = current.version - filterOnline := func(entry SegmentEntry, _ int) bool { - return !d.offlines.Contain(entry.SegmentID) + return !entry.Offline } sealed, growing = d.filterSegments(sealed, growing, filterOnline) - + version = current.version return } @@ -195,7 +213,26 @@ func (d *distribution) getTargetVersion() int64 { // Serviceable returns wether current snapshot is serviceable. func (d *distribution) Serviceable() bool { - return d.serviceable.Load() + return d.queryView.serviceable.Load() +} + +func (d *distribution) updateServiceable(triggerAction string) { + if d.queryView.version != initialTargetVersion { + serviceable := true + for _, s := range d.queryView.sealedSegments { + if entry, ok := d.sealedSegments[s]; !ok || entry.Offline { + serviceable = false + break + } + } + if serviceable != d.queryView.serviceable.Load() { + d.queryView.serviceable.Store(serviceable) + log.Info("channel distribution serviceable changed", + zap.String("channel", d.channelName), + zap.Bool("serviceable", serviceable), + zap.String("action", triggerAction)) + } + } } // AddDistributions add multiple segment entries. @@ -224,10 +261,10 @@ func (d *distribution) AddDistributions(entries ...SegmentEntry) { entry.TargetVersion = unreadableTargetVersion } d.sealedSegments[entry.SegmentID] = entry - d.offlines.Remove(entry.SegmentID) } d.genSnapshot() + d.updateServiceable("AddDistributions") } // AddGrowing adds growing segment distribution. @@ -243,7 +280,7 @@ func (d *distribution) AddGrowing(entries ...SegmentEntry) { } // AddOffline set segmentIDs to offlines. -func (d *distribution) AddOfflines(segmentIDs ...int64) { +func (d *distribution) MarkOfflineSegments(segmentIDs ...int64) { d.mut.Lock() defer d.mut.Unlock() @@ -253,17 +290,19 @@ func (d *distribution) AddOfflines(segmentIDs ...int64) { if !ok { continue } - // FIXME: remove offlie logic later - // mark segment distribution as offline, set verion to unreadable - entry.NodeID = wildcardNodeID + updated = true + entry.Offline = true entry.Version = unreadableTargetVersion + entry.NodeID = -1 d.sealedSegments[segmentID] = entry - updated = true - d.offlines.Insert(segmentID) } if updated { + log.Info("mark sealed segment offline from distribution", + zap.String("channelName", d.channelName), + zap.Int64s("segmentIDs", segmentIDs)) d.genSnapshot() + d.updateServiceable("MarkOfflineSegments") } } @@ -292,25 +331,30 @@ func (d *distribution) SyncTargetVersion(newVersion int64, partitions []int64, g d.growingSegments[segmentID] = entry } - available := true for _, segmentID := range sealedInTarget { entry, ok := d.sealedSegments[segmentID] if !ok { - log.Warn("readable sealed segment lost, make it unserviceable", zap.Int64("segmentID", segmentID)) - available = false continue } entry.TargetVersion = newVersion d.sealedSegments[segmentID] = entry } - oldValue := d.targetVersion.Load() - d.targetVersion.Store(newVersion) + oldValue := d.queryView.version + d.queryView = &channelQueryView{ + sealedSegments: sealedInTarget, + partitions: typeutil.NewUniqueSet(partitions...), + version: newVersion, + serviceable: d.queryView.serviceable, + } + // update working partition list - d.genSnapshot(WithPartitions(partitions)) + d.genSnapshot() // if sealed segment in leader view is less than sealed segment in target, set delegator to unserviceable - d.serviceable.Store(available) - log.Info("Update readable segment version", + d.updateServiceable("SyncTargetVersion") + + log.Info("Update channel query view", + zap.String("channel", d.channelName), zap.Int64s("partitions", partitions), zap.Int64("oldVersion", oldValue), zap.Int64("newVersion", newVersion), @@ -325,9 +369,6 @@ func (d *distribution) RemoveDistributions(sealedSegments []SegmentEntry, growin defer d.mut.Unlock() for _, sealed := range sealedSegments { - if d.offlines.Contain(sealed.SegmentID) { - d.offlines.Remove(sealed.SegmentID) - } entry, ok := d.sealedSegments[sealed.SegmentID] if !ok { continue @@ -346,6 +387,13 @@ func (d *distribution) RemoveDistributions(sealedSegments []SegmentEntry, growin delete(d.growingSegments, growing.SegmentID) } + log.Info("remove segments from distribution", + zap.String("channelName", d.channelName), + zap.Int64s("growing", lo.Map(growingSegments, func(s SegmentEntry, _ int) int64 { return s.SegmentID })), + zap.Int64s("sealed", lo.Map(sealedSegments, func(s SegmentEntry, _ int) int64 { return s.SegmentID })), + ) + + d.updateServiceable("RemoveDistributions") // wait previous read even not distribution changed // in case of segment balance caused segment lost track return d.genSnapshot() @@ -354,20 +402,10 @@ func (d *distribution) RemoveDistributions(sealedSegments []SegmentEntry, growin // getSnapshot converts current distribution to snapshot format. // in which, user could use found nodeID=>segmentID list. // mutex RLock is required before calling this method. -func (d *distribution) genSnapshot(opts ...genSnapshotOpt) chan struct{} { +func (d *distribution) genSnapshot() chan struct{} { // stores last snapshot // ok to be nil last := d.current.Load() - option := &genSnapshotOption{ - partitions: typeutil.NewSet[int64](), // if no working list provided, snapshot shall have no item - } - // use last snapshot working parition list by default - if last != nil { - option.partitions = last.partitions - } - for _, opt := range opts { - opt(option) - } nodeSegments := make(map[int64][]SegmentEntry) for _, entry := range d.sealedSegments { @@ -380,7 +418,7 @@ func (d *distribution) genSnapshot(opts ...genSnapshotOpt) chan struct{} { dist = append(dist, SnapshotItem{ NodeID: nodeID, Segments: lo.Map(items, func(entry SegmentEntry, _ int) SegmentEntry { - if !option.partitions.Contain(entry.PartitionID) { + if !d.queryView.partitions.Contain(entry.PartitionID) { entry.TargetVersion = unreadableTargetVersion } return entry @@ -390,18 +428,16 @@ func (d *distribution) genSnapshot(opts ...genSnapshotOpt) chan struct{} { growing := make([]SegmentEntry, 0, len(d.growingSegments)) for _, entry := range d.growingSegments { - if !option.partitions.Contain(entry.PartitionID) { + if !d.queryView.partitions.Contain(entry.PartitionID) { entry.TargetVersion = unreadableTargetVersion } growing = append(growing, entry) } - d.serviceable.Store(d.offlines.Len() == 0) - // update snapshot version d.snapshotVersion++ - newSnapShot := NewSnapshot(dist, growing, last, d.snapshotVersion, d.targetVersion.Load()) - newSnapShot.partitions = option.partitions + newSnapShot := NewSnapshot(dist, growing, last, d.snapshotVersion, d.queryView.GetVersion()) + newSnapShot.partitions = d.queryView.partitions d.current.Store(newSnapShot) // shall be a new one @@ -435,15 +471,3 @@ func (d *distribution) getCleanup(version int64) snapshotCleanup { d.snapshots.GetAndRemove(version) } } - -type genSnapshotOption struct { - partitions typeutil.Set[int64] -} - -type genSnapshotOpt func(*genSnapshotOption) - -func WithPartitions(partitions []int64) genSnapshotOpt { - return func(opt *genSnapshotOption) { - opt.partitions = typeutil.NewSet(partitions...) - } -} diff --git a/internal/querynodev2/delegator/distribution_test.go b/internal/querynodev2/delegator/distribution_test.go index 6b19e2fb3d9c6..afea4f76aad54 100644 --- a/internal/querynodev2/delegator/distribution_test.go +++ b/internal/querynodev2/delegator/distribution_test.go @@ -20,6 +20,7 @@ import ( "testing" "time" + "github.com/samber/lo" "github.com/stretchr/testify/suite" ) @@ -29,8 +30,7 @@ type DistributionSuite struct { } func (s *DistributionSuite) SetupTest() { - s.dist = NewDistribution() - s.Equal(initialTargetVersion, s.dist.getTargetVersion()) + s.dist = NewDistribution("channel-1") } func (s *DistributionSuite) TearDownTest() { @@ -177,6 +177,7 @@ func (s *DistributionSuite) TestAddDistribution() { s.SetupTest() defer s.TearDownTest() s.dist.AddGrowing(tc.growing...) + s.dist.SyncTargetVersion(1000, nil, nil, nil, nil) _, _, version, err := s.dist.PinReadableSegments() s.Require().NoError(err) s.dist.AddDistributions(tc.input...) @@ -447,6 +448,15 @@ func (s *DistributionSuite) TestRemoveDistribution() { s.dist.AddGrowing(tc.presetGrowing...) s.dist.AddDistributions(tc.presetSealed...) + // update target version, make delegator serviceable + growingIDs := lo.Map(tc.presetGrowing, func(item SegmentEntry, idx int) int64 { + return item.SegmentID + }) + sealedIDs := lo.Map(tc.presetSealed, func(item SegmentEntry, idx int) int64 { + return item.SegmentID + }) + s.dist.SyncTargetVersion(time.Now().Unix(), nil, growingIDs, sealedIDs, nil) + var version int64 if tc.withMockRead { var err error @@ -614,7 +624,7 @@ func (s *DistributionSuite) TestPeek() { } } -func (s *DistributionSuite) TestAddOfflines() { +func (s *DistributionSuite) TestMarkOfflineSegments() { type testCase struct { tag string input []SegmentEntry @@ -665,12 +675,14 @@ func (s *DistributionSuite) TestAddOfflines() { defer s.TearDownTest() s.dist.AddDistributions(tc.input...) - s.dist.AddOfflines(tc.offlines...) + sealedSegmentID := lo.Map(tc.input, func(t SegmentEntry, _ int) int64 { + return t.SegmentID + }) + s.dist.SyncTargetVersion(1000, nil, nil, sealedSegmentID, nil) + s.dist.MarkOfflineSegments(tc.offlines...) s.Equal(tc.serviceable, s.dist.Serviceable()) - // current := s.dist.current.Load() for _, offline := range tc.offlines { - // current. s.dist.mut.RLock() entry, ok := s.dist.sealedSegments[offline] s.dist.mut.RUnlock() @@ -739,7 +751,7 @@ func (s *DistributionSuite) Test_SyncTargetVersion() { s.Len(s1[0].Segments, 3) s.Len(s2, 3) - s.dist.serviceable.Store(true) + s.dist.queryView.serviceable.Store(true) s.dist.SyncTargetVersion(2, []int64{1}, []int64{222}, []int64{}, []int64{}) s.True(s.dist.Serviceable()) diff --git a/internal/querynodev2/delegator/mock_delegator.go b/internal/querynodev2/delegator/mock_delegator.go index 236c499b7dae6..bad4e98b3f701 100644 --- a/internal/querynodev2/delegator/mock_delegator.go +++ b/internal/querynodev2/delegator/mock_delegator.go @@ -243,6 +243,53 @@ func (_c *MockShardDelegator_GetPartitionStatsVersions_Call) RunAndReturn(run fu return _c } +// GetQueryView provides a mock function with no fields +func (_m *MockShardDelegator) GetQueryView() *channelQueryView { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetQueryView") + } + + var r0 *channelQueryView + if rf, ok := ret.Get(0).(func() *channelQueryView); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*channelQueryView) + } + } + + return r0 +} + +// MockShardDelegator_GetQueryView_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetQueryView' +type MockShardDelegator_GetQueryView_Call struct { + *mock.Call +} + +// GetQueryView is a helper method to define mock.On call +func (_e *MockShardDelegator_Expecter) GetQueryView() *MockShardDelegator_GetQueryView_Call { + return &MockShardDelegator_GetQueryView_Call{Call: _e.mock.On("GetQueryView")} +} + +func (_c *MockShardDelegator_GetQueryView_Call) Run(run func()) *MockShardDelegator_GetQueryView_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockShardDelegator_GetQueryView_Call) Return(_a0 *channelQueryView) *MockShardDelegator_GetQueryView_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockShardDelegator_GetQueryView_Call) RunAndReturn(run func() *channelQueryView) *MockShardDelegator_GetQueryView_Call { + _c.Call.Return(run) + return _c +} + // GetSegmentInfo provides a mock function with given fields: readable func (_m *MockShardDelegator) GetSegmentInfo(readable bool) ([]SnapshotItem, []SegmentEntry) { ret := _m.Called(readable) @@ -407,51 +454,6 @@ func (_c *MockShardDelegator_GetTSafe_Call) RunAndReturn(run func() uint64) *Moc return _c } -// GetTargetVersion provides a mock function with no fields -func (_m *MockShardDelegator) GetTargetVersion() int64 { - ret := _m.Called() - - if len(ret) == 0 { - panic("no return value specified for GetTargetVersion") - } - - var r0 int64 - if rf, ok := ret.Get(0).(func() int64); ok { - r0 = rf() - } else { - r0 = ret.Get(0).(int64) - } - - return r0 -} - -// MockShardDelegator_GetTargetVersion_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetTargetVersion' -type MockShardDelegator_GetTargetVersion_Call struct { - *mock.Call -} - -// GetTargetVersion is a helper method to define mock.On call -func (_e *MockShardDelegator_Expecter) GetTargetVersion() *MockShardDelegator_GetTargetVersion_Call { - return &MockShardDelegator_GetTargetVersion_Call{Call: _e.mock.On("GetTargetVersion")} -} - -func (_c *MockShardDelegator_GetTargetVersion_Call) Run(run func()) *MockShardDelegator_GetTargetVersion_Call { - _c.Call.Run(func(args mock.Arguments) { - run() - }) - return _c -} - -func (_c *MockShardDelegator_GetTargetVersion_Call) Return(_a0 int64) *MockShardDelegator_GetTargetVersion_Call { - _c.Call.Return(_a0) - return _c -} - -func (_c *MockShardDelegator_GetTargetVersion_Call) RunAndReturn(run func() int64) *MockShardDelegator_GetTargetVersion_Call { - _c.Call.Return(run) - return _c -} - // LoadGrowing provides a mock function with given fields: ctx, infos, version func (_m *MockShardDelegator) LoadGrowing(ctx context.Context, infos []*querypb.SegmentLoadInfo, version int64) error { ret := _m.Called(ctx, infos, version) diff --git a/internal/querynodev2/services.go b/internal/querynodev2/services.go index 1bcb311cbc418..bc1810a43e3bc 100644 --- a/internal/querynodev2/services.go +++ b/internal/querynodev2/services.go @@ -1253,14 +1253,18 @@ func (node *QueryNode) GetDataDistribution(ctx context.Context, req *querypb.Get numOfGrowingRows += segment.InsertCount() } + queryView := delegator.GetQueryView() leaderViews = append(leaderViews, &querypb.LeaderView{ Collection: delegator.Collection(), Channel: key, SegmentDist: sealedSegments, GrowingSegments: growingSegments, - TargetVersion: delegator.GetTargetVersion(), NumOfGrowingRows: numOfGrowingRows, PartitionStatsVersions: delegator.GetPartitionStatsVersions(ctx), + TargetVersion: queryView.GetVersion(), + Status: &querypb.LeaderViewStatus{ + Serviceable: queryView.Serviceable(), + }, }) return true }) diff --git a/internal/querynodev2/services_test.go b/internal/querynodev2/services_test.go index accd2db0832da..f908bc81e75f4 100644 --- a/internal/querynodev2/services_test.go +++ b/internal/querynodev2/services_test.go @@ -212,6 +212,7 @@ func (suite *ServiceSuite) TestGetStatistics_Normal() { ctx := context.Background() suite.TestWatchDmChannelsInt64() suite.TestLoadSegments_Int64() + suite.syncDistribution(context.TODO()) req := &querypb.GetStatisticsRequest{ Req: &internalpb.GetStatisticsRequest{ diff --git a/pkg/proto/query_coord.proto b/pkg/proto/query_coord.proto index 04a86afcbf379..eb06e4f72f7b8 100644 --- a/pkg/proto/query_coord.proto +++ b/pkg/proto/query_coord.proto @@ -618,9 +618,14 @@ message LeaderView { map segment_dist = 3; repeated int64 growing_segmentIDs = 4; map growing_segments = 5; - int64 TargetVersion = 6; + int64 TargetVersion = 6; // deprecated int64 num_of_growing_rows = 7; map partition_stats_versions = 8; + LeaderViewStatus status = 9; +} + +message LeaderViewStatus { + bool serviceable = 10; } message SegmentDist { diff --git a/pkg/proto/querypb/query_coord.pb.go b/pkg/proto/querypb/query_coord.pb.go index ac70e4461918e..80d11b34b967d 100644 --- a/pkg/proto/querypb/query_coord.pb.go +++ b/pkg/proto/querypb/query_coord.pb.go @@ -4149,9 +4149,10 @@ type LeaderView struct { SegmentDist map[int64]*SegmentDist `protobuf:"bytes,3,rep,name=segment_dist,json=segmentDist,proto3" json:"segment_dist,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` GrowingSegmentIDs []int64 `protobuf:"varint,4,rep,packed,name=growing_segmentIDs,json=growingSegmentIDs,proto3" json:"growing_segmentIDs,omitempty"` GrowingSegments map[int64]*msgpb.MsgPosition `protobuf:"bytes,5,rep,name=growing_segments,json=growingSegments,proto3" json:"growing_segments,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` - TargetVersion int64 `protobuf:"varint,6,opt,name=TargetVersion,proto3" json:"TargetVersion,omitempty"` + TargetVersion int64 `protobuf:"varint,6,opt,name=TargetVersion,proto3" json:"TargetVersion,omitempty"` // deprecated NumOfGrowingRows int64 `protobuf:"varint,7,opt,name=num_of_growing_rows,json=numOfGrowingRows,proto3" json:"num_of_growing_rows,omitempty"` PartitionStatsVersions map[int64]int64 `protobuf:"bytes,8,rep,name=partition_stats_versions,json=partitionStatsVersions,proto3" json:"partition_stats_versions,omitempty" protobuf_key:"varint,1,opt,name=key,proto3" protobuf_val:"varint,2,opt,name=value,proto3"` + Status *LeaderViewStatus `protobuf:"bytes,9,opt,name=status,proto3" json:"status,omitempty"` } func (x *LeaderView) Reset() { @@ -4242,6 +4243,60 @@ func (x *LeaderView) GetPartitionStatsVersions() map[int64]int64 { return nil } +func (x *LeaderView) GetStatus() *LeaderViewStatus { + if x != nil { + return x.Status + } + return nil +} + +type LeaderViewStatus struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Serviceable bool `protobuf:"varint,10,opt,name=serviceable,proto3" json:"serviceable,omitempty"` +} + +func (x *LeaderViewStatus) Reset() { + *x = LeaderViewStatus{} + if protoimpl.UnsafeEnabled { + mi := &file_query_coord_proto_msgTypes[45] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LeaderViewStatus) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LeaderViewStatus) ProtoMessage() {} + +func (x *LeaderViewStatus) ProtoReflect() protoreflect.Message { + mi := &file_query_coord_proto_msgTypes[45] + 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 LeaderViewStatus.ProtoReflect.Descriptor instead. +func (*LeaderViewStatus) Descriptor() ([]byte, []int) { + return file_query_coord_proto_rawDescGZIP(), []int{45} +} + +func (x *LeaderViewStatus) GetServiceable() bool { + if x != nil { + return x.Serviceable + } + return false +} + type SegmentDist struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -4254,7 +4309,7 @@ type SegmentDist struct { func (x *SegmentDist) Reset() { *x = SegmentDist{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[45] + mi := &file_query_coord_proto_msgTypes[46] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4267,7 +4322,7 @@ func (x *SegmentDist) String() string { func (*SegmentDist) ProtoMessage() {} func (x *SegmentDist) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[45] + mi := &file_query_coord_proto_msgTypes[46] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4280,7 +4335,7 @@ func (x *SegmentDist) ProtoReflect() protoreflect.Message { // Deprecated: Use SegmentDist.ProtoReflect.Descriptor instead. func (*SegmentDist) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{45} + return file_query_coord_proto_rawDescGZIP(), []int{46} } func (x *SegmentDist) GetNodeID() int64 { @@ -4317,7 +4372,7 @@ type SegmentVersionInfo struct { func (x *SegmentVersionInfo) Reset() { *x = SegmentVersionInfo{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[46] + mi := &file_query_coord_proto_msgTypes[47] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4330,7 +4385,7 @@ func (x *SegmentVersionInfo) String() string { func (*SegmentVersionInfo) ProtoMessage() {} func (x *SegmentVersionInfo) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[46] + mi := &file_query_coord_proto_msgTypes[47] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4343,7 +4398,7 @@ func (x *SegmentVersionInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use SegmentVersionInfo.ProtoReflect.Descriptor instead. func (*SegmentVersionInfo) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{46} + return file_query_coord_proto_rawDescGZIP(), []int{47} } func (x *SegmentVersionInfo) GetID() int64 { @@ -4429,7 +4484,7 @@ type ChannelVersionInfo struct { func (x *ChannelVersionInfo) Reset() { *x = ChannelVersionInfo{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[47] + mi := &file_query_coord_proto_msgTypes[48] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4442,7 +4497,7 @@ func (x *ChannelVersionInfo) String() string { func (*ChannelVersionInfo) ProtoMessage() {} func (x *ChannelVersionInfo) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[47] + mi := &file_query_coord_proto_msgTypes[48] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4455,7 +4510,7 @@ func (x *ChannelVersionInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use ChannelVersionInfo.ProtoReflect.Descriptor instead. func (*ChannelVersionInfo) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{47} + return file_query_coord_proto_rawDescGZIP(), []int{48} } func (x *ChannelVersionInfo) GetChannel() string { @@ -4498,7 +4553,7 @@ type CollectionLoadInfo struct { func (x *CollectionLoadInfo) Reset() { *x = CollectionLoadInfo{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[48] + mi := &file_query_coord_proto_msgTypes[49] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4511,7 +4566,7 @@ func (x *CollectionLoadInfo) String() string { func (*CollectionLoadInfo) ProtoMessage() {} func (x *CollectionLoadInfo) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[48] + mi := &file_query_coord_proto_msgTypes[49] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4524,7 +4579,7 @@ func (x *CollectionLoadInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use CollectionLoadInfo.ProtoReflect.Descriptor instead. func (*CollectionLoadInfo) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{48} + return file_query_coord_proto_rawDescGZIP(), []int{49} } func (x *CollectionLoadInfo) GetCollectionID() int64 { @@ -4606,7 +4661,7 @@ type PartitionLoadInfo struct { func (x *PartitionLoadInfo) Reset() { *x = PartitionLoadInfo{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[49] + mi := &file_query_coord_proto_msgTypes[50] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4619,7 +4674,7 @@ func (x *PartitionLoadInfo) String() string { func (*PartitionLoadInfo) ProtoMessage() {} func (x *PartitionLoadInfo) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[49] + mi := &file_query_coord_proto_msgTypes[50] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4632,7 +4687,7 @@ func (x *PartitionLoadInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use PartitionLoadInfo.ProtoReflect.Descriptor instead. func (*PartitionLoadInfo) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{49} + return file_query_coord_proto_rawDescGZIP(), []int{50} } func (x *PartitionLoadInfo) GetCollectionID() int64 { @@ -4688,7 +4743,7 @@ type ChannelNodeInfo struct { func (x *ChannelNodeInfo) Reset() { *x = ChannelNodeInfo{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[50] + mi := &file_query_coord_proto_msgTypes[51] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4701,7 +4756,7 @@ func (x *ChannelNodeInfo) String() string { func (*ChannelNodeInfo) ProtoMessage() {} func (x *ChannelNodeInfo) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[50] + mi := &file_query_coord_proto_msgTypes[51] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4714,7 +4769,7 @@ func (x *ChannelNodeInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use ChannelNodeInfo.ProtoReflect.Descriptor instead. func (*ChannelNodeInfo) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{50} + return file_query_coord_proto_rawDescGZIP(), []int{51} } func (x *ChannelNodeInfo) GetRwNodes() []int64 { @@ -4747,7 +4802,7 @@ type Replica struct { func (x *Replica) Reset() { *x = Replica{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[51] + mi := &file_query_coord_proto_msgTypes[52] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4760,7 +4815,7 @@ func (x *Replica) String() string { func (*Replica) ProtoMessage() {} func (x *Replica) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[51] + mi := &file_query_coord_proto_msgTypes[52] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4773,7 +4828,7 @@ func (x *Replica) ProtoReflect() protoreflect.Message { // Deprecated: Use Replica.ProtoReflect.Descriptor instead. func (*Replica) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{51} + return file_query_coord_proto_rawDescGZIP(), []int{52} } func (x *Replica) GetID() int64 { @@ -4855,7 +4910,7 @@ type SyncAction struct { func (x *SyncAction) Reset() { *x = SyncAction{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[52] + mi := &file_query_coord_proto_msgTypes[53] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -4868,7 +4923,7 @@ func (x *SyncAction) String() string { func (*SyncAction) ProtoMessage() {} func (x *SyncAction) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[52] + mi := &file_query_coord_proto_msgTypes[53] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -4881,7 +4936,7 @@ func (x *SyncAction) ProtoReflect() protoreflect.Message { // Deprecated: Use SyncAction.ProtoReflect.Descriptor instead. func (*SyncAction) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{52} + return file_query_coord_proto_rawDescGZIP(), []int{53} } func (x *SyncAction) GetType() SyncType { @@ -4994,7 +5049,7 @@ type SyncDistributionRequest struct { func (x *SyncDistributionRequest) Reset() { *x = SyncDistributionRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[53] + mi := &file_query_coord_proto_msgTypes[54] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5007,7 +5062,7 @@ func (x *SyncDistributionRequest) String() string { func (*SyncDistributionRequest) ProtoMessage() {} func (x *SyncDistributionRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[53] + mi := &file_query_coord_proto_msgTypes[54] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5020,7 +5075,7 @@ func (x *SyncDistributionRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SyncDistributionRequest.ProtoReflect.Descriptor instead. func (*SyncDistributionRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{53} + return file_query_coord_proto_rawDescGZIP(), []int{54} } func (x *SyncDistributionRequest) GetBase() *commonpb.MsgBase { @@ -5101,7 +5156,7 @@ type ResourceGroup struct { func (x *ResourceGroup) Reset() { *x = ResourceGroup{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[54] + mi := &file_query_coord_proto_msgTypes[55] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5114,7 +5169,7 @@ func (x *ResourceGroup) String() string { func (*ResourceGroup) ProtoMessage() {} func (x *ResourceGroup) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[54] + mi := &file_query_coord_proto_msgTypes[55] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5127,7 +5182,7 @@ func (x *ResourceGroup) ProtoReflect() protoreflect.Message { // Deprecated: Use ResourceGroup.ProtoReflect.Descriptor instead. func (*ResourceGroup) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{54} + return file_query_coord_proto_rawDescGZIP(), []int{55} } func (x *ResourceGroup) GetName() string { @@ -5175,7 +5230,7 @@ type TransferReplicaRequest struct { func (x *TransferReplicaRequest) Reset() { *x = TransferReplicaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[55] + mi := &file_query_coord_proto_msgTypes[56] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5188,7 +5243,7 @@ func (x *TransferReplicaRequest) String() string { func (*TransferReplicaRequest) ProtoMessage() {} func (x *TransferReplicaRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[55] + mi := &file_query_coord_proto_msgTypes[56] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5201,7 +5256,7 @@ func (x *TransferReplicaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use TransferReplicaRequest.ProtoReflect.Descriptor instead. func (*TransferReplicaRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{55} + return file_query_coord_proto_rawDescGZIP(), []int{56} } func (x *TransferReplicaRequest) GetBase() *commonpb.MsgBase { @@ -5251,7 +5306,7 @@ type DescribeResourceGroupRequest struct { func (x *DescribeResourceGroupRequest) Reset() { *x = DescribeResourceGroupRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[56] + mi := &file_query_coord_proto_msgTypes[57] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5264,7 +5319,7 @@ func (x *DescribeResourceGroupRequest) String() string { func (*DescribeResourceGroupRequest) ProtoMessage() {} func (x *DescribeResourceGroupRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[56] + mi := &file_query_coord_proto_msgTypes[57] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5277,7 +5332,7 @@ func (x *DescribeResourceGroupRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DescribeResourceGroupRequest.ProtoReflect.Descriptor instead. func (*DescribeResourceGroupRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{56} + return file_query_coord_proto_rawDescGZIP(), []int{57} } func (x *DescribeResourceGroupRequest) GetBase() *commonpb.MsgBase { @@ -5306,7 +5361,7 @@ type DescribeResourceGroupResponse struct { func (x *DescribeResourceGroupResponse) Reset() { *x = DescribeResourceGroupResponse{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[57] + mi := &file_query_coord_proto_msgTypes[58] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5319,7 +5374,7 @@ func (x *DescribeResourceGroupResponse) String() string { func (*DescribeResourceGroupResponse) ProtoMessage() {} func (x *DescribeResourceGroupResponse) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[57] + mi := &file_query_coord_proto_msgTypes[58] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5332,7 +5387,7 @@ func (x *DescribeResourceGroupResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use DescribeResourceGroupResponse.ProtoReflect.Descriptor instead. func (*DescribeResourceGroupResponse) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{57} + return file_query_coord_proto_rawDescGZIP(), []int{58} } func (x *DescribeResourceGroupResponse) GetStatus() *commonpb.Status { @@ -5372,7 +5427,7 @@ type ResourceGroupInfo struct { func (x *ResourceGroupInfo) Reset() { *x = ResourceGroupInfo{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[58] + mi := &file_query_coord_proto_msgTypes[59] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5385,7 +5440,7 @@ func (x *ResourceGroupInfo) String() string { func (*ResourceGroupInfo) ProtoMessage() {} func (x *ResourceGroupInfo) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[58] + mi := &file_query_coord_proto_msgTypes[59] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5398,7 +5453,7 @@ func (x *ResourceGroupInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use ResourceGroupInfo.ProtoReflect.Descriptor instead. func (*ResourceGroupInfo) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{58} + return file_query_coord_proto_rawDescGZIP(), []int{59} } func (x *ResourceGroupInfo) GetName() string { @@ -5477,7 +5532,7 @@ type DeleteRequest struct { func (x *DeleteRequest) Reset() { *x = DeleteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[59] + mi := &file_query_coord_proto_msgTypes[60] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5490,7 +5545,7 @@ func (x *DeleteRequest) String() string { func (*DeleteRequest) ProtoMessage() {} func (x *DeleteRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[59] + mi := &file_query_coord_proto_msgTypes[60] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5503,7 +5558,7 @@ func (x *DeleteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteRequest.ProtoReflect.Descriptor instead. func (*DeleteRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{59} + return file_query_coord_proto_rawDescGZIP(), []int{60} } func (x *DeleteRequest) GetBase() *commonpb.MsgBase { @@ -5587,7 +5642,7 @@ type DeleteBatchRequest struct { func (x *DeleteBatchRequest) Reset() { *x = DeleteBatchRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[60] + mi := &file_query_coord_proto_msgTypes[61] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5600,7 +5655,7 @@ func (x *DeleteBatchRequest) String() string { func (*DeleteBatchRequest) ProtoMessage() {} func (x *DeleteBatchRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[60] + mi := &file_query_coord_proto_msgTypes[61] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5613,7 +5668,7 @@ func (x *DeleteBatchRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteBatchRequest.ProtoReflect.Descriptor instead. func (*DeleteBatchRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{60} + return file_query_coord_proto_rawDescGZIP(), []int{61} } func (x *DeleteBatchRequest) GetBase() *commonpb.MsgBase { @@ -5687,7 +5742,7 @@ type DeleteBatchResponse struct { func (x *DeleteBatchResponse) Reset() { *x = DeleteBatchResponse{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[61] + mi := &file_query_coord_proto_msgTypes[62] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5700,7 +5755,7 @@ func (x *DeleteBatchResponse) String() string { func (*DeleteBatchResponse) ProtoMessage() {} func (x *DeleteBatchResponse) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[61] + mi := &file_query_coord_proto_msgTypes[62] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5713,7 +5768,7 @@ func (x *DeleteBatchResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use DeleteBatchResponse.ProtoReflect.Descriptor instead. func (*DeleteBatchResponse) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{61} + return file_query_coord_proto_rawDescGZIP(), []int{62} } func (x *DeleteBatchResponse) GetStatus() *commonpb.Status { @@ -5749,7 +5804,7 @@ type ActivateCheckerRequest struct { func (x *ActivateCheckerRequest) Reset() { *x = ActivateCheckerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[62] + mi := &file_query_coord_proto_msgTypes[63] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5762,7 +5817,7 @@ func (x *ActivateCheckerRequest) String() string { func (*ActivateCheckerRequest) ProtoMessage() {} func (x *ActivateCheckerRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[62] + mi := &file_query_coord_proto_msgTypes[63] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5775,7 +5830,7 @@ func (x *ActivateCheckerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ActivateCheckerRequest.ProtoReflect.Descriptor instead. func (*ActivateCheckerRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{62} + return file_query_coord_proto_rawDescGZIP(), []int{63} } func (x *ActivateCheckerRequest) GetBase() *commonpb.MsgBase { @@ -5804,7 +5859,7 @@ type DeactivateCheckerRequest struct { func (x *DeactivateCheckerRequest) Reset() { *x = DeactivateCheckerRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[63] + mi := &file_query_coord_proto_msgTypes[64] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5817,7 +5872,7 @@ func (x *DeactivateCheckerRequest) String() string { func (*DeactivateCheckerRequest) ProtoMessage() {} func (x *DeactivateCheckerRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[63] + mi := &file_query_coord_proto_msgTypes[64] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5830,7 +5885,7 @@ func (x *DeactivateCheckerRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use DeactivateCheckerRequest.ProtoReflect.Descriptor instead. func (*DeactivateCheckerRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{63} + return file_query_coord_proto_rawDescGZIP(), []int{64} } func (x *DeactivateCheckerRequest) GetBase() *commonpb.MsgBase { @@ -5859,7 +5914,7 @@ type ListCheckersRequest struct { func (x *ListCheckersRequest) Reset() { *x = ListCheckersRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[64] + mi := &file_query_coord_proto_msgTypes[65] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5872,7 +5927,7 @@ func (x *ListCheckersRequest) String() string { func (*ListCheckersRequest) ProtoMessage() {} func (x *ListCheckersRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[64] + mi := &file_query_coord_proto_msgTypes[65] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5885,7 +5940,7 @@ func (x *ListCheckersRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListCheckersRequest.ProtoReflect.Descriptor instead. func (*ListCheckersRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{64} + return file_query_coord_proto_rawDescGZIP(), []int{65} } func (x *ListCheckersRequest) GetBase() *commonpb.MsgBase { @@ -5914,7 +5969,7 @@ type ListCheckersResponse struct { func (x *ListCheckersResponse) Reset() { *x = ListCheckersResponse{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[65] + mi := &file_query_coord_proto_msgTypes[66] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5927,7 +5982,7 @@ func (x *ListCheckersResponse) String() string { func (*ListCheckersResponse) ProtoMessage() {} func (x *ListCheckersResponse) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[65] + mi := &file_query_coord_proto_msgTypes[66] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5940,7 +5995,7 @@ func (x *ListCheckersResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListCheckersResponse.ProtoReflect.Descriptor instead. func (*ListCheckersResponse) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{65} + return file_query_coord_proto_rawDescGZIP(), []int{66} } func (x *ListCheckersResponse) GetStatus() *commonpb.Status { @@ -5971,7 +6026,7 @@ type CheckerInfo struct { func (x *CheckerInfo) Reset() { *x = CheckerInfo{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[66] + mi := &file_query_coord_proto_msgTypes[67] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -5984,7 +6039,7 @@ func (x *CheckerInfo) String() string { func (*CheckerInfo) ProtoMessage() {} func (x *CheckerInfo) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[66] + mi := &file_query_coord_proto_msgTypes[67] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -5997,7 +6052,7 @@ func (x *CheckerInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use CheckerInfo.ProtoReflect.Descriptor instead. func (*CheckerInfo) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{66} + return file_query_coord_proto_rawDescGZIP(), []int{67} } func (x *CheckerInfo) GetId() int32 { @@ -6041,7 +6096,7 @@ type SegmentTarget struct { func (x *SegmentTarget) Reset() { *x = SegmentTarget{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[67] + mi := &file_query_coord_proto_msgTypes[68] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6054,7 +6109,7 @@ func (x *SegmentTarget) String() string { func (*SegmentTarget) ProtoMessage() {} func (x *SegmentTarget) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[67] + mi := &file_query_coord_proto_msgTypes[68] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6067,7 +6122,7 @@ func (x *SegmentTarget) ProtoReflect() protoreflect.Message { // Deprecated: Use SegmentTarget.ProtoReflect.Descriptor instead. func (*SegmentTarget) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{67} + return file_query_coord_proto_rawDescGZIP(), []int{68} } func (x *SegmentTarget) GetID() int64 { @@ -6103,7 +6158,7 @@ type PartitionTarget struct { func (x *PartitionTarget) Reset() { *x = PartitionTarget{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[68] + mi := &file_query_coord_proto_msgTypes[69] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6116,7 +6171,7 @@ func (x *PartitionTarget) String() string { func (*PartitionTarget) ProtoMessage() {} func (x *PartitionTarget) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[68] + mi := &file_query_coord_proto_msgTypes[69] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6129,7 +6184,7 @@ func (x *PartitionTarget) ProtoReflect() protoreflect.Message { // Deprecated: Use PartitionTarget.ProtoReflect.Descriptor instead. func (*PartitionTarget) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{68} + return file_query_coord_proto_rawDescGZIP(), []int{69} } func (x *PartitionTarget) GetPartitionID() int64 { @@ -6162,7 +6217,7 @@ type ChannelTarget struct { func (x *ChannelTarget) Reset() { *x = ChannelTarget{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[69] + mi := &file_query_coord_proto_msgTypes[70] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6175,7 +6230,7 @@ func (x *ChannelTarget) String() string { func (*ChannelTarget) ProtoMessage() {} func (x *ChannelTarget) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[69] + mi := &file_query_coord_proto_msgTypes[70] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6188,7 +6243,7 @@ func (x *ChannelTarget) ProtoReflect() protoreflect.Message { // Deprecated: Use ChannelTarget.ProtoReflect.Descriptor instead. func (*ChannelTarget) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{69} + return file_query_coord_proto_rawDescGZIP(), []int{70} } func (x *ChannelTarget) GetChannelName() string { @@ -6246,7 +6301,7 @@ type CollectionTarget struct { func (x *CollectionTarget) Reset() { *x = CollectionTarget{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[70] + mi := &file_query_coord_proto_msgTypes[71] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6259,7 +6314,7 @@ func (x *CollectionTarget) String() string { func (*CollectionTarget) ProtoMessage() {} func (x *CollectionTarget) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[70] + mi := &file_query_coord_proto_msgTypes[71] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6272,7 +6327,7 @@ func (x *CollectionTarget) ProtoReflect() protoreflect.Message { // Deprecated: Use CollectionTarget.ProtoReflect.Descriptor instead. func (*CollectionTarget) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{70} + return file_query_coord_proto_rawDescGZIP(), []int{71} } func (x *CollectionTarget) GetCollectionID() int64 { @@ -6309,7 +6364,7 @@ type NodeInfo struct { func (x *NodeInfo) Reset() { *x = NodeInfo{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[71] + mi := &file_query_coord_proto_msgTypes[72] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6322,7 +6377,7 @@ func (x *NodeInfo) String() string { func (*NodeInfo) ProtoMessage() {} func (x *NodeInfo) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[71] + mi := &file_query_coord_proto_msgTypes[72] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6335,7 +6390,7 @@ func (x *NodeInfo) ProtoReflect() protoreflect.Message { // Deprecated: Use NodeInfo.ProtoReflect.Descriptor instead. func (*NodeInfo) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{71} + return file_query_coord_proto_rawDescGZIP(), []int{72} } func (x *NodeInfo) GetID() int64 { @@ -6370,7 +6425,7 @@ type ListQueryNodeRequest struct { func (x *ListQueryNodeRequest) Reset() { *x = ListQueryNodeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[72] + mi := &file_query_coord_proto_msgTypes[73] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6383,7 +6438,7 @@ func (x *ListQueryNodeRequest) String() string { func (*ListQueryNodeRequest) ProtoMessage() {} func (x *ListQueryNodeRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[72] + mi := &file_query_coord_proto_msgTypes[73] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6396,7 +6451,7 @@ func (x *ListQueryNodeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ListQueryNodeRequest.ProtoReflect.Descriptor instead. func (*ListQueryNodeRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{72} + return file_query_coord_proto_rawDescGZIP(), []int{73} } func (x *ListQueryNodeRequest) GetBase() *commonpb.MsgBase { @@ -6418,7 +6473,7 @@ type ListQueryNodeResponse struct { func (x *ListQueryNodeResponse) Reset() { *x = ListQueryNodeResponse{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[73] + mi := &file_query_coord_proto_msgTypes[74] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6431,7 +6486,7 @@ func (x *ListQueryNodeResponse) String() string { func (*ListQueryNodeResponse) ProtoMessage() {} func (x *ListQueryNodeResponse) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[73] + mi := &file_query_coord_proto_msgTypes[74] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6444,7 +6499,7 @@ func (x *ListQueryNodeResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ListQueryNodeResponse.ProtoReflect.Descriptor instead. func (*ListQueryNodeResponse) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{73} + return file_query_coord_proto_rawDescGZIP(), []int{74} } func (x *ListQueryNodeResponse) GetStatus() *commonpb.Status { @@ -6473,7 +6528,7 @@ type GetQueryNodeDistributionRequest struct { func (x *GetQueryNodeDistributionRequest) Reset() { *x = GetQueryNodeDistributionRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[74] + mi := &file_query_coord_proto_msgTypes[75] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6486,7 +6541,7 @@ func (x *GetQueryNodeDistributionRequest) String() string { func (*GetQueryNodeDistributionRequest) ProtoMessage() {} func (x *GetQueryNodeDistributionRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[74] + mi := &file_query_coord_proto_msgTypes[75] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6499,7 +6554,7 @@ func (x *GetQueryNodeDistributionRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GetQueryNodeDistributionRequest.ProtoReflect.Descriptor instead. func (*GetQueryNodeDistributionRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{74} + return file_query_coord_proto_rawDescGZIP(), []int{75} } func (x *GetQueryNodeDistributionRequest) GetBase() *commonpb.MsgBase { @@ -6530,7 +6585,7 @@ type GetQueryNodeDistributionResponse struct { func (x *GetQueryNodeDistributionResponse) Reset() { *x = GetQueryNodeDistributionResponse{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[75] + mi := &file_query_coord_proto_msgTypes[76] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6543,7 +6598,7 @@ func (x *GetQueryNodeDistributionResponse) String() string { func (*GetQueryNodeDistributionResponse) ProtoMessage() {} func (x *GetQueryNodeDistributionResponse) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[75] + mi := &file_query_coord_proto_msgTypes[76] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6556,7 +6611,7 @@ func (x *GetQueryNodeDistributionResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GetQueryNodeDistributionResponse.ProtoReflect.Descriptor instead. func (*GetQueryNodeDistributionResponse) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{75} + return file_query_coord_proto_rawDescGZIP(), []int{76} } func (x *GetQueryNodeDistributionResponse) GetStatus() *commonpb.Status { @@ -6598,7 +6653,7 @@ type SuspendBalanceRequest struct { func (x *SuspendBalanceRequest) Reset() { *x = SuspendBalanceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[76] + mi := &file_query_coord_proto_msgTypes[77] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6611,7 +6666,7 @@ func (x *SuspendBalanceRequest) String() string { func (*SuspendBalanceRequest) ProtoMessage() {} func (x *SuspendBalanceRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[76] + mi := &file_query_coord_proto_msgTypes[77] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6624,7 +6679,7 @@ func (x *SuspendBalanceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SuspendBalanceRequest.ProtoReflect.Descriptor instead. func (*SuspendBalanceRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{76} + return file_query_coord_proto_rawDescGZIP(), []int{77} } func (x *SuspendBalanceRequest) GetBase() *commonpb.MsgBase { @@ -6645,7 +6700,7 @@ type ResumeBalanceRequest struct { func (x *ResumeBalanceRequest) Reset() { *x = ResumeBalanceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[77] + mi := &file_query_coord_proto_msgTypes[78] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6658,7 +6713,7 @@ func (x *ResumeBalanceRequest) String() string { func (*ResumeBalanceRequest) ProtoMessage() {} func (x *ResumeBalanceRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[77] + mi := &file_query_coord_proto_msgTypes[78] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6671,7 +6726,7 @@ func (x *ResumeBalanceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ResumeBalanceRequest.ProtoReflect.Descriptor instead. func (*ResumeBalanceRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{77} + return file_query_coord_proto_rawDescGZIP(), []int{78} } func (x *ResumeBalanceRequest) GetBase() *commonpb.MsgBase { @@ -6692,7 +6747,7 @@ type CheckBalanceStatusRequest struct { func (x *CheckBalanceStatusRequest) Reset() { *x = CheckBalanceStatusRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[78] + mi := &file_query_coord_proto_msgTypes[79] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6705,7 +6760,7 @@ func (x *CheckBalanceStatusRequest) String() string { func (*CheckBalanceStatusRequest) ProtoMessage() {} func (x *CheckBalanceStatusRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[78] + mi := &file_query_coord_proto_msgTypes[79] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6718,7 +6773,7 @@ func (x *CheckBalanceStatusRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use CheckBalanceStatusRequest.ProtoReflect.Descriptor instead. func (*CheckBalanceStatusRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{78} + return file_query_coord_proto_rawDescGZIP(), []int{79} } func (x *CheckBalanceStatusRequest) GetBase() *commonpb.MsgBase { @@ -6740,7 +6795,7 @@ type CheckBalanceStatusResponse struct { func (x *CheckBalanceStatusResponse) Reset() { *x = CheckBalanceStatusResponse{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[79] + mi := &file_query_coord_proto_msgTypes[80] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6753,7 +6808,7 @@ func (x *CheckBalanceStatusResponse) String() string { func (*CheckBalanceStatusResponse) ProtoMessage() {} func (x *CheckBalanceStatusResponse) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[79] + mi := &file_query_coord_proto_msgTypes[80] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6766,7 +6821,7 @@ func (x *CheckBalanceStatusResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use CheckBalanceStatusResponse.ProtoReflect.Descriptor instead. func (*CheckBalanceStatusResponse) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{79} + return file_query_coord_proto_rawDescGZIP(), []int{80} } func (x *CheckBalanceStatusResponse) GetStatus() *commonpb.Status { @@ -6795,7 +6850,7 @@ type SuspendNodeRequest struct { func (x *SuspendNodeRequest) Reset() { *x = SuspendNodeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[80] + mi := &file_query_coord_proto_msgTypes[81] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6808,7 +6863,7 @@ func (x *SuspendNodeRequest) String() string { func (*SuspendNodeRequest) ProtoMessage() {} func (x *SuspendNodeRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[80] + mi := &file_query_coord_proto_msgTypes[81] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6821,7 +6876,7 @@ func (x *SuspendNodeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use SuspendNodeRequest.ProtoReflect.Descriptor instead. func (*SuspendNodeRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{80} + return file_query_coord_proto_rawDescGZIP(), []int{81} } func (x *SuspendNodeRequest) GetBase() *commonpb.MsgBase { @@ -6850,7 +6905,7 @@ type ResumeNodeRequest struct { func (x *ResumeNodeRequest) Reset() { *x = ResumeNodeRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[81] + mi := &file_query_coord_proto_msgTypes[82] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6863,7 +6918,7 @@ func (x *ResumeNodeRequest) String() string { func (*ResumeNodeRequest) ProtoMessage() {} func (x *ResumeNodeRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[81] + mi := &file_query_coord_proto_msgTypes[82] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6876,7 +6931,7 @@ func (x *ResumeNodeRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ResumeNodeRequest.ProtoReflect.Descriptor instead. func (*ResumeNodeRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{81} + return file_query_coord_proto_rawDescGZIP(), []int{82} } func (x *ResumeNodeRequest) GetBase() *commonpb.MsgBase { @@ -6910,7 +6965,7 @@ type TransferSegmentRequest struct { func (x *TransferSegmentRequest) Reset() { *x = TransferSegmentRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[82] + mi := &file_query_coord_proto_msgTypes[83] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -6923,7 +6978,7 @@ func (x *TransferSegmentRequest) String() string { func (*TransferSegmentRequest) ProtoMessage() {} func (x *TransferSegmentRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[82] + mi := &file_query_coord_proto_msgTypes[83] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -6936,7 +6991,7 @@ func (x *TransferSegmentRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use TransferSegmentRequest.ProtoReflect.Descriptor instead. func (*TransferSegmentRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{82} + return file_query_coord_proto_rawDescGZIP(), []int{83} } func (x *TransferSegmentRequest) GetBase() *commonpb.MsgBase { @@ -7005,7 +7060,7 @@ type TransferChannelRequest struct { func (x *TransferChannelRequest) Reset() { *x = TransferChannelRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[83] + mi := &file_query_coord_proto_msgTypes[84] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7018,7 +7073,7 @@ func (x *TransferChannelRequest) String() string { func (*TransferChannelRequest) ProtoMessage() {} func (x *TransferChannelRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[83] + mi := &file_query_coord_proto_msgTypes[84] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7031,7 +7086,7 @@ func (x *TransferChannelRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use TransferChannelRequest.ProtoReflect.Descriptor instead. func (*TransferChannelRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{83} + return file_query_coord_proto_rawDescGZIP(), []int{84} } func (x *TransferChannelRequest) GetBase() *commonpb.MsgBase { @@ -7096,7 +7151,7 @@ type CheckQueryNodeDistributionRequest struct { func (x *CheckQueryNodeDistributionRequest) Reset() { *x = CheckQueryNodeDistributionRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[84] + mi := &file_query_coord_proto_msgTypes[85] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7109,7 +7164,7 @@ func (x *CheckQueryNodeDistributionRequest) String() string { func (*CheckQueryNodeDistributionRequest) ProtoMessage() {} func (x *CheckQueryNodeDistributionRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[84] + mi := &file_query_coord_proto_msgTypes[85] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7122,7 +7177,7 @@ func (x *CheckQueryNodeDistributionRequest) ProtoReflect() protoreflect.Message // Deprecated: Use CheckQueryNodeDistributionRequest.ProtoReflect.Descriptor instead. func (*CheckQueryNodeDistributionRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{84} + return file_query_coord_proto_rawDescGZIP(), []int{85} } func (x *CheckQueryNodeDistributionRequest) GetBase() *commonpb.MsgBase { @@ -7161,7 +7216,7 @@ type UpdateLoadConfigRequest struct { func (x *UpdateLoadConfigRequest) Reset() { *x = UpdateLoadConfigRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[85] + mi := &file_query_coord_proto_msgTypes[86] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7174,7 +7229,7 @@ func (x *UpdateLoadConfigRequest) String() string { func (*UpdateLoadConfigRequest) ProtoMessage() {} func (x *UpdateLoadConfigRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[85] + mi := &file_query_coord_proto_msgTypes[86] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7187,7 +7242,7 @@ func (x *UpdateLoadConfigRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateLoadConfigRequest.ProtoReflect.Descriptor instead. func (*UpdateLoadConfigRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{85} + return file_query_coord_proto_rawDescGZIP(), []int{86} } func (x *UpdateLoadConfigRequest) GetBase() *commonpb.MsgBase { @@ -7239,7 +7294,7 @@ type UpdateSchemaRequest struct { func (x *UpdateSchemaRequest) Reset() { *x = UpdateSchemaRequest{} if protoimpl.UnsafeEnabled { - mi := &file_query_coord_proto_msgTypes[86] + mi := &file_query_coord_proto_msgTypes[87] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -7252,7 +7307,7 @@ func (x *UpdateSchemaRequest) String() string { func (*UpdateSchemaRequest) ProtoMessage() {} func (x *UpdateSchemaRequest) ProtoReflect() protoreflect.Message { - mi := &file_query_coord_proto_msgTypes[86] + mi := &file_query_coord_proto_msgTypes[87] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -7265,7 +7320,7 @@ func (x *UpdateSchemaRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use UpdateSchemaRequest.ProtoReflect.Descriptor instead. func (*UpdateSchemaRequest) Descriptor() ([]byte, []int) { - return file_query_coord_proto_rawDescGZIP(), []int{86} + return file_query_coord_proto_rawDescGZIP(), []int{87} } func (x *UpdateSchemaRequest) GetBase() *commonpb.MsgBase { @@ -8125,7 +8180,7 @@ var file_query_coord_proto_rawDesc = []byte{ 0x6f, 0x64, 0x69, 0x66, 0x79, 0x54, 0x73, 0x12, 0x28, 0x0a, 0x0f, 0x6d, 0x65, 0x6d, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x49, 0x6e, 0x4d, 0x42, 0x18, 0x07, 0x20, 0x01, 0x28, 0x01, 0x52, 0x0f, 0x6d, 0x65, 0x6d, 0x43, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x49, 0x6e, 0x4d, - 0x42, 0x22, 0x83, 0x06, 0x0a, 0x0a, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, 0x69, 0x65, 0x77, + 0x42, 0x22, 0xc1, 0x06, 0x0a, 0x0a, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, 0x69, 0x65, 0x77, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, @@ -8156,353 +8211,336 @@ var file_query_coord_proto_rawDesc = []byte{ 0x56, 0x69, 0x65, 0x77, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, - 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x5f, 0x0a, 0x10, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x35, - 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x61, 0x0a, 0x14, 0x47, 0x72, 0x6f, - 0x77, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x33, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x49, 0x0a, 0x1b, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3f, 0x0a, 0x0b, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x18, - 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x89, 0x04, 0x0a, 0x12, 0x53, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, - 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, - 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x1c, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, - 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, - 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, - 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, - 0x6e, 0x12, 0x30, 0x0a, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x64, 0x65, 0x6c, 0x74, 0x61, 0x5f, - 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, - 0x12, 0x6c, 0x61, 0x73, 0x74, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x12, 0x54, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, - 0x6f, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x2e, - 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x09, - 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x35, 0x0a, 0x05, 0x6c, 0x65, 0x76, - 0x65, 0x6c, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, - 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x18, 0x09, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x12, 0x33, 0x0a, - 0x16, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x03, 0x52, 0x13, 0x66, - 0x69, 0x65, 0x6c, 0x64, 0x4a, 0x73, 0x6f, 0x6e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, - 0x74, 0x73, 0x1a, 0x60, 0x0a, 0x0e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x38, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, - 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x22, 0x68, 0x0a, 0x12, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x56, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, - 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0xfd, - 0x03, 0x0a, 0x12, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x61, - 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x2f, 0x0a, 0x13, 0x72, 0x65, 0x6c, - 0x65, 0x61, 0x73, 0x65, 0x64, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x12, 0x72, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x64, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, 0x75, 0x6d, 0x62, 0x65, - 0x72, 0x12, 0x36, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x5d, 0x0a, 0x0d, 0x66, 0x69, 0x65, - 0x6c, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x38, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, - 0x64, 0x65, 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x66, 0x69, 0x65, 0x6c, - 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x39, 0x0a, 0x09, 0x6c, 0x6f, 0x61, 0x64, - 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x08, 0x6c, 0x6f, 0x61, 0x64, 0x54, - 0x79, 0x70, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x5f, 0x74, - 0x69, 0x6d, 0x65, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, 0x72, 0x65, 0x63, 0x6f, - 0x76, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x6c, 0x6f, 0x61, 0x64, - 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x6c, - 0x6f, 0x61, 0x64, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, - 0x44, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x1a, 0x3f, 0x0a, - 0x11, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xfc, - 0x02, 0x0a, 0x11, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x61, 0x64, - 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, 0x75, 0x6d, 0x62, 0x65, - 0x72, 0x12, 0x36, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x5c, 0x0a, 0x0d, 0x66, 0x69, 0x65, - 0x6c, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4c, - 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x66, 0x69, 0x65, 0x6c, 0x64, - 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x63, 0x6f, 0x76, - 0x65, 0x72, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, - 0x72, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x1a, 0x3f, 0x0a, 0x11, - 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x2c, 0x0a, - 0x0f, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, - 0x12, 0x19, 0x0a, 0x08, 0x72, 0x77, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, - 0x28, 0x03, 0x52, 0x07, 0x72, 0x77, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x22, 0xa0, 0x03, 0x0a, 0x07, - 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x6e, - 0x6f, 0x64, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x05, 0x6e, 0x6f, 0x64, 0x65, - 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, - 0x6f, 0x75, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x6f, 0x5f, 0x6e, - 0x6f, 0x64, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x72, 0x6f, 0x4e, 0x6f, - 0x64, 0x65, 0x73, 0x12, 0x5f, 0x0a, 0x12, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, - 0x6f, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x2e, 0x43, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x10, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x49, - 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x72, 0x77, 0x5f, 0x73, 0x71, 0x5f, 0x6e, 0x6f, - 0x64, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x72, 0x77, 0x53, 0x71, 0x4e, - 0x6f, 0x64, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x72, 0x6f, 0x5f, 0x73, 0x71, 0x5f, 0x6e, 0x6f, - 0x64, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x72, 0x6f, 0x53, 0x71, 0x4e, - 0x6f, 0x64, 0x65, 0x73, 0x1a, 0x68, 0x0a, 0x15, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, - 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, - 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, - 0x39, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x49, - 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xc6, - 0x05, 0x0a, 0x0a, 0x53, 0x79, 0x6e, 0x63, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x30, 0x0a, - 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, 0x6d, 0x69, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3c, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x56, 0x69, 0x65, 0x77, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x1a, 0x5f, 0x0a, 0x10, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x44, 0x69, 0x73, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x35, 0x0a, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, - 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, - 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, - 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, - 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, - 0x6e, 0x12, 0x37, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x61, 0x64, - 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x12, 0x28, 0x0a, 0x0f, 0x67, 0x72, - 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x07, 0x20, - 0x03, 0x28, 0x03, 0x52, 0x0f, 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x54, 0x61, - 0x72, 0x67, 0x65, 0x74, 0x12, 0x26, 0x0a, 0x0e, 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x49, 0x6e, - 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x08, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e, 0x73, 0x65, - 0x61, 0x6c, 0x65, 0x64, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x24, 0x0a, 0x0d, - 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x09, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x0d, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, - 0x6f, 0x6e, 0x12, 0x28, 0x0a, 0x0f, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x49, 0x6e, 0x54, - 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0f, 0x64, 0x72, 0x6f, - 0x70, 0x70, 0x65, 0x64, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x3d, 0x0a, 0x0a, - 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, + 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, 0x52, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x61, 0x0a, 0x14, 0x47, 0x72, 0x6f, 0x77, 0x69, + 0x6e, 0x67, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x33, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x74, 0x0a, 0x18, 0x70, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x5f, 0x76, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x50, 0x61, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x49, 0x0a, 0x1b, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, - 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, - 0x73, 0x12, 0x39, 0x0a, 0x08, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x50, 0x18, 0x0d, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x08, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x50, 0x1a, 0x49, 0x0a, 0x1b, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xc0, 0x03, 0x0a, 0x17, 0x53, 0x79, 0x6e, 0x63, - 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x34, 0x0a, 0x10, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, + 0x69, 0x65, 0x77, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x20, 0x0a, 0x0b, 0x73, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, + 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x61, 0x62, 0x6c, 0x65, 0x22, 0x3f, 0x0a, 0x0b, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x44, 0x69, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, + 0x64, 0x65, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, + 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x89, 0x04, 0x0a, + 0x12, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x02, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x18, 0x0a, 0x07, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x30, 0x0a, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x64, 0x65, + 0x6c, 0x74, 0x61, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x04, 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x54, 0x0a, 0x0a, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, + 0x6e, 0x66, 0x6f, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x09, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x35, 0x0a, + 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x05, 0x6c, + 0x65, 0x76, 0x65, 0x6c, 0x12, 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65, + 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x53, 0x6f, 0x72, 0x74, 0x65, + 0x64, 0x12, 0x33, 0x0a, 0x16, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x6a, 0x73, 0x6f, 0x6e, 0x5f, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x13, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x4a, 0x73, 0x6f, 0x6e, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x53, 0x74, 0x61, 0x74, 0x73, 0x1a, 0x60, 0x0a, 0x0e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, + 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x38, 0x0a, 0x05, 0x76, 0x61, + 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x46, + 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x68, 0x0a, 0x12, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x18, + 0x0a, 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x22, 0xfd, 0x03, 0x0a, 0x12, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x2f, 0x0a, + 0x13, 0x72, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x64, 0x5f, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x12, 0x72, 0x65, 0x6c, 0x65, + 0x61, 0x73, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x25, + 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, + 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x36, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x5d, 0x0a, + 0x0d, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x05, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x46, 0x69, 0x65, + 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, + 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x39, 0x0a, 0x09, + 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x08, 0x6c, + 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x72, 0x65, 0x63, 0x6f, 0x76, + 0x65, 0x72, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, + 0x72, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x12, 0x1f, 0x0a, 0x0b, + 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x0a, 0x6c, 0x6f, 0x61, 0x64, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x12, 0x0a, + 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, + 0x44, 0x1a, 0x3f, 0x0a, 0x11, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, + 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0xfc, 0x02, 0x0a, 0x11, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x25, + 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, + 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x36, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x5c, 0x0a, + 0x0d, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x18, 0x05, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x46, 0x69, 0x65, 0x6c, + 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x66, + 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x72, + 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x0c, 0x72, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x1a, 0x3f, 0x0a, 0x11, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x44, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x2c, 0x0a, 0x0f, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x6f, 0x64, 0x65, + 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x77, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x73, + 0x18, 0x06, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, 0x72, 0x77, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x22, + 0xa0, 0x03, 0x0a, 0x07, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x12, 0x0e, 0x0a, 0x02, 0x49, + 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, + 0x14, 0x0a, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x05, + 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x19, 0x0a, 0x08, + 0x72, 0x6f, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x07, + 0x72, 0x6f, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x5f, 0x0a, 0x12, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x06, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, + 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x72, 0x77, 0x5f, 0x73, + 0x71, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x72, + 0x77, 0x53, 0x71, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x72, 0x6f, 0x5f, 0x73, + 0x71, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x72, + 0x6f, 0x53, 0x71, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x1a, 0x68, 0x0a, 0x15, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, + 0x6b, 0x65, 0x79, 0x12, 0x39, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, + 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, + 0x38, 0x01, 0x22, 0xc6, 0x05, 0x0a, 0x0a, 0x53, 0x79, 0x6e, 0x63, 0x41, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x30, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, + 0x79, 0x70, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x37, 0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x4c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x12, 0x28, + 0x0a, 0x0f, 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, + 0x74, 0x18, 0x07, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0f, 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, + 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x26, 0x0a, 0x0e, 0x73, 0x65, 0x61, 0x6c, + 0x65, 0x64, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x08, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x0e, 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, + 0x12, 0x24, 0x0a, 0x0d, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x28, 0x0a, 0x0f, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, + 0x64, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x0f, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x49, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, + 0x12, 0x3d, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x0b, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, + 0x74, 0x0a, 0x18, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x74, 0x61, + 0x74, 0x73, 0x5f, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0c, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x41, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, + 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x16, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x39, 0x0a, 0x08, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, + 0x50, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, + 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x50, + 0x1a, 0x49, 0x0a, 0x1b, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x73, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, + 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xc0, 0x03, 0x0a, 0x17, + 0x53, 0x79, 0x6e, 0x63, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, + 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, + 0x07, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x38, 0x0a, 0x07, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, + 0x6e, 0x63, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x12, 0x3d, 0x0a, 0x09, 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x74, + 0x61, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x12, + 0x1c, 0x0a, 0x09, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x09, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x44, 0x12, 0x18, 0x0a, + 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, + 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x45, 0x0a, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x0d, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x69, 0x73, 0x74, 0x22, 0x97, + 0x01, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, + 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x42, 0x02, 0x18, 0x01, 0x52, 0x08, 0x63, 0x61, 0x70, 0x61, + 0x63, 0x69, 0x74, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x03, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x06, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x67, 0x2e, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x22, 0xf7, 0x01, 0x0a, 0x16, 0x54, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x65, 0x72, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, - 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, - 0x6e, 0x65, 0x6c, 0x12, 0x38, 0x0a, 0x07, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x41, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x3d, 0x0a, - 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, - 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, - 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x3d, 0x0a, 0x09, - 0x6c, 0x6f, 0x61, 0x64, 0x5f, 0x6d, 0x65, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x49, 0x6e, 0x66, - 0x6f, 0x52, 0x08, 0x6c, 0x6f, 0x61, 0x64, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x1c, 0x0a, 0x09, 0x72, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, - 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, - 0x69, 0x6f, 0x6e, 0x12, 0x45, 0x0a, 0x0f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x69, 0x6e, 0x66, - 0x6f, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x2e, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0d, 0x69, 0x6e, 0x64, - 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x69, 0x73, 0x74, 0x22, 0x97, 0x01, 0x0a, 0x0d, 0x52, - 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x12, 0x0a, 0x04, - 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, - 0x12, 0x1e, 0x0a, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x05, 0x42, 0x02, 0x18, 0x01, 0x52, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, - 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, - 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x67, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x22, 0xf7, 0x01, 0x0a, 0x16, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, - 0x72, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, + 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x32, 0x0a, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, + 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x32, 0x0a, 0x15, 0x74, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, + 0x75, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, + 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x22, 0x0a, + 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x12, 0x1f, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x6e, 0x75, 0x6d, 0x52, 0x65, 0x70, 0x6c, 0x69, + 0x63, 0x61, 0x22, 0x77, 0x0a, 0x1c, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, + 0x62, 0x61, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x22, 0xa2, 0x01, 0x0a, 0x1d, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, + 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, - 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, - 0x65, 0x12, 0x32, 0x0a, 0x15, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x13, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x32, 0x0a, 0x15, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, - 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1f, 0x0a, - 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x0a, 0x6e, 0x75, 0x6d, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x22, 0x77, - 0x0a, 0x1c, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, - 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, - 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, - 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, - 0x75, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x22, 0xa2, 0x01, 0x0a, 0x1d, 0x44, 0x65, 0x73, 0x63, - 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, - 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, - 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x4c, - 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0d, 0x72, - 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x22, 0xf0, 0x05, 0x0a, - 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, - 0x66, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, - 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x42, 0x02, 0x18, 0x01, 0x52, 0x08, 0x63, 0x61, - 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x12, 0x2c, 0x0a, 0x12, 0x6e, 0x75, 0x6d, 0x5f, 0x61, 0x76, - 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x05, 0x52, 0x10, 0x6e, 0x75, 0x6d, 0x41, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, - 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x69, 0x0a, 0x12, 0x6e, 0x75, 0x6d, 0x5f, 0x6c, 0x6f, 0x61, 0x64, - 0x65, 0x64, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x3b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, - 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4e, 0x75, 0x6d, 0x4c, 0x6f, 0x61, 0x64, 0x65, - 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x6e, - 0x75, 0x6d, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x12, - 0x66, 0x0a, 0x11, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x5f, - 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, + 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x12, 0x4c, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, + 0x72, 0x6f, 0x75, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x66, - 0x6f, 0x2e, 0x4e, 0x75, 0x6d, 0x4f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, - 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x6e, 0x75, 0x6d, 0x4f, 0x75, 0x74, 0x67, 0x6f, - 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x66, 0x0a, 0x11, 0x6e, 0x75, 0x6d, 0x5f, 0x69, - 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x06, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4e, 0x75, 0x6d, 0x49, 0x6e, 0x63, - 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, - 0x6e, 0x75, 0x6d, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x12, - 0x3c, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, - 0x67, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x33, 0x0a, - 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, - 0x6f, 0x6e, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x6e, 0x6f, 0x64, - 0x65, 0x73, 0x1a, 0x43, 0x0a, 0x15, 0x4e, 0x75, 0x6d, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x52, - 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x42, 0x0a, 0x14, 0x4e, 0x75, 0x6d, 0x4f, 0x75, - 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, + 0x6f, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, + 0x22, 0xf0, 0x05, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, + 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1e, 0x0a, 0x08, 0x63, 0x61, + 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x42, 0x02, 0x18, 0x01, + 0x52, 0x08, 0x63, 0x61, 0x70, 0x61, 0x63, 0x69, 0x74, 0x79, 0x12, 0x2c, 0x0a, 0x12, 0x6e, 0x75, + 0x6d, 0x5f, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x6e, 0x75, 0x6d, 0x41, 0x76, 0x61, 0x69, 0x6c, + 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x69, 0x0a, 0x12, 0x6e, 0x75, 0x6d, 0x5f, + 0x6c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4e, 0x75, 0x6d, 0x4c, + 0x6f, 0x61, 0x64, 0x65, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x45, 0x6e, 0x74, 0x72, + 0x79, 0x52, 0x10, 0x6e, 0x75, 0x6d, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x52, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x12, 0x66, 0x0a, 0x11, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, 0x75, 0x74, 0x67, 0x6f, + 0x69, 0x6e, 0x67, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4e, 0x75, 0x6d, 0x4f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, + 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0f, 0x6e, 0x75, 0x6d, 0x4f, + 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x66, 0x0a, 0x11, 0x6e, + 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x5f, 0x6e, 0x6f, 0x64, 0x65, + 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4e, 0x75, + 0x6d, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x0f, 0x6e, 0x75, 0x6d, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x4e, + 0x6f, 0x64, 0x65, 0x12, 0x3c, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x07, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x72, 0x67, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, + 0x6f, 0x75, 0x70, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x12, 0x33, 0x0a, 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x05, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x1a, 0x43, 0x0a, 0x15, 0x4e, 0x75, 0x6d, 0x4c, 0x6f, 0x61, + 0x64, 0x65, 0x64, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x42, 0x0a, 0x14, 0x4e, - 0x75, 0x6d, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x6e, + 0x75, 0x6d, 0x4f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x4e, 0x6f, 0x64, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, - 0xfa, 0x02, 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, - 0x61, 0x73, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, - 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x76, - 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0c, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, - 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x05, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x12, - 0x3b, 0x0a, 0x0c, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x49, 0x44, 0x73, 0x52, - 0x0b, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x73, 0x12, 0x1e, 0x0a, 0x0a, - 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x04, - 0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x73, 0x12, 0x33, 0x0a, 0x05, - 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, - 0x2e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, - 0x65, 0x12, 0x19, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x5f, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x20, - 0x01, 0x28, 0x08, 0x52, 0x07, 0x75, 0x73, 0x65, 0x4c, 0x6f, 0x61, 0x64, 0x22, 0xe6, 0x02, 0x0a, - 0x12, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, - 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x23, 0x0a, - 0x0d, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, - 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x49, 0x64, 0x73, 0x12, 0x3b, 0x0a, 0x0c, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, + 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, + 0x42, 0x0a, 0x14, 0x4e, 0x75, 0x6d, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x4e, 0x6f, + 0x64, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0xfa, 0x02, 0x0a, 0x0d, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, + 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, + 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, + 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, + 0x23, 0x0a, 0x0d, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x64, 0x12, 0x3b, 0x0a, 0x0c, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x49, 0x44, 0x73, 0x52, 0x0b, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x4b, 0x65, 0x79, 0x73, @@ -8511,176 +8549,218 @@ var file_query_coord_proto_rawDesc = []byte{ 0x12, 0x33, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x52, 0x05, - 0x73, 0x63, 0x6f, 0x70, 0x65, 0x22, 0x8a, 0x01, 0x0a, 0x13, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, - 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, - 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x5f, 0x69, 0x64, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x49, 0x64, - 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x64, 0x73, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x49, - 0x64, 0x73, 0x22, 0x68, 0x0a, 0x16, 0x41, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, + 0x73, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x75, 0x73, 0x65, 0x5f, 0x6c, 0x6f, 0x61, + 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x75, 0x73, 0x65, 0x4c, 0x6f, 0x61, 0x64, + 0x22, 0xe6, 0x02, 0x0a, 0x12, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, + 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, + 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x64, 0x12, 0x23, 0x0a, 0x0d, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x12, 0x3b, 0x0a, 0x0c, 0x70, 0x72, 0x69, 0x6d, 0x61, + 0x72, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x18, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x2e, 0x49, 0x44, 0x73, 0x52, 0x0b, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, + 0x4b, 0x65, 0x79, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x04, 0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, + 0x61, 0x6d, 0x70, 0x73, 0x12, 0x33, 0x0a, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x18, 0x08, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x61, 0x74, 0x61, 0x53, 0x63, 0x6f, + 0x70, 0x65, 0x52, 0x05, 0x73, 0x63, 0x6f, 0x70, 0x65, 0x22, 0x8a, 0x01, 0x0a, 0x13, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, + 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x09, 0x66, 0x61, 0x69, 0x6c, + 0x65, 0x64, 0x49, 0x64, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6e, 0x67, + 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x6d, 0x69, 0x73, 0x73, + 0x69, 0x6e, 0x67, 0x49, 0x64, 0x73, 0x22, 0x68, 0x0a, 0x16, 0x41, 0x63, 0x74, 0x69, 0x76, 0x61, + 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, + 0x22, 0x6a, 0x0a, 0x18, 0x44, 0x65, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x05, 0x52, 0x09, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, 0x22, 0x6a, 0x0a, 0x18, - 0x44, 0x65, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, - 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, - 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x68, - 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x63, - 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, 0x22, 0x67, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, - 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, - 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, - 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, - 0x73, 0x22, 0x90, 0x01, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, - 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, - 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, - 0x43, 0x0a, 0x0c, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, - 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0c, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, - 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x65, 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, - 0x6e, 0x66, 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, - 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x65, 0x73, 0x63, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x04, 0x64, 0x65, 0x73, 0x63, 0x12, 0x1c, 0x0a, 0x09, 0x61, 0x63, 0x74, 0x69, 0x76, - 0x61, 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, 0x61, 0x63, 0x74, 0x69, - 0x76, 0x61, 0x74, 0x65, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x22, 0x76, 0x0a, 0x0d, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x0e, 0x0a, 0x02, - 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, 0x35, 0x0a, 0x05, - 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1f, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x05, 0x6c, 0x65, - 0x76, 0x65, 0x6c, 0x12, 0x1e, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, 0x66, 0x5f, 0x72, 0x6f, - 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x75, 0x6d, 0x4f, 0x66, 0x52, - 0x6f, 0x77, 0x73, 0x22, 0x72, 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x3d, 0x0a, 0x08, 0x73, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x08, 0x73, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xf1, 0x02, 0x0a, 0x0d, 0x43, 0x68, 0x61, 0x6e, - 0x6e, 0x65, 0x6c, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x63, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, - 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2d, 0x0a, 0x12, 0x64, - 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, - 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x11, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x2d, 0x0a, 0x12, 0x67, 0x72, - 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x11, 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x50, 0x0a, 0x11, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, 0x18, 0x04, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x10, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, 0x12, 0x42, 0x0a, 0x0d, 0x73, - 0x65, 0x65, 0x6b, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x0c, 0x73, 0x65, 0x65, 0x6b, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x4a, 0x0a, 0x11, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x70, - 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, - 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x64, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x22, 0x9c, 0x01, 0x0a, 0x10, - 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, - 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x4a, 0x0a, 0x0f, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, - 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, - 0x52, 0x0e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, - 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x4a, 0x0a, 0x08, 0x4e, 0x6f, - 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, - 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, - 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x48, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, - 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, - 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, - 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, - 0x22, 0x88, 0x01, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, - 0x64, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, + 0x05, 0x52, 0x09, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x44, 0x22, 0x67, 0x0a, 0x13, + 0x4c, 0x69, 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, + 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, + 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x05, 0x52, 0x0a, 0x63, 0x68, 0x65, 0x63, 0x6b, + 0x65, 0x72, 0x49, 0x44, 0x73, 0x22, 0x90, 0x01, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x68, + 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x12, 0x43, 0x0a, 0x0c, 0x63, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x6e, + 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, + 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0c, 0x63, 0x68, 0x65, 0x63, + 0x6b, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x65, 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, + 0x6b, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x65, 0x73, 0x63, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x64, 0x65, 0x73, 0x63, 0x12, 0x1c, 0x0a, 0x09, 0x61, + 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x09, + 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x6f, 0x75, + 0x6e, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x05, 0x66, 0x6f, 0x75, 0x6e, 0x64, 0x22, + 0x76, 0x0a, 0x0d, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, + 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, + 0x12, 0x35, 0x0a, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, + 0x52, 0x05, 0x6c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x1e, 0x0a, 0x0b, 0x6e, 0x75, 0x6d, 0x5f, 0x6f, + 0x66, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x75, + 0x6d, 0x4f, 0x66, 0x52, 0x6f, 0x77, 0x73, 0x22, 0x72, 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x3d, 0x0a, 0x08, + 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x61, 0x72, 0x67, 0x65, + 0x74, 0x52, 0x08, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xf1, 0x02, 0x0a, 0x0d, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, 0x20, 0x0a, + 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x2d, 0x0a, 0x12, 0x64, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, 0x11, 0x64, 0x72, 0x6f, + 0x70, 0x70, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x2d, + 0x0a, 0x12, 0x67, 0x72, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x11, 0x67, 0x72, 0x6f, 0x77, + 0x69, 0x6e, 0x67, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x50, 0x0a, + 0x11, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x61, 0x72, 0x67, 0x65, + 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x52, 0x10, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, 0x12, + 0x42, 0x0a, 0x0d, 0x73, 0x65, 0x65, 0x6b, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x73, 0x65, 0x65, 0x6b, 0x50, 0x6f, 0x73, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x4a, 0x0a, 0x11, 0x64, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x63, 0x68, + 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x73, + 0x67, 0x2e, 0x4d, 0x73, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x10, 0x64, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x22, + 0x9c, 0x01, 0x0a, 0x10, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x4a, 0x0a, 0x0f, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x5f, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x52, 0x0e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x61, 0x72, + 0x67, 0x65, 0x74, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x4a, + 0x0a, 0x08, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x61, 0x64, + 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x61, 0x64, 0x64, + 0x72, 0x65, 0x73, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x22, 0x48, 0x0a, 0x14, 0x4c, 0x69, + 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, + 0x62, 0x61, 0x73, 0x65, 0x22, 0x88, 0x01, 0x0a, 0x15, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, + 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x12, 0x3a, 0x0a, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4e, 0x6f, 0x64, 0x65, + 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, + 0x6b, 0x0a, 0x1f, 0x47, 0x65, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, + 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, + 0x62, 0x61, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, 0xb9, 0x01, 0x0a, + 0x20, 0x47, 0x65, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, + 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x02, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x73, + 0x65, 0x61, 0x6c, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, + 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x10, 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x22, 0x49, 0x0a, 0x15, 0x53, 0x75, 0x73, 0x70, + 0x65, 0x6e, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, + 0x61, 0x73, 0x65, 0x22, 0x48, 0x0a, 0x14, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x42, 0x61, 0x6c, + 0x61, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, + 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, 0x4d, 0x0a, + 0x19, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, + 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, + 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, 0x6e, 0x0a, 0x1a, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, - 0x3a, 0x0a, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, - 0x52, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x22, 0x6b, 0x0a, 0x1f, 0x47, - 0x65, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, - 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, - 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, - 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, - 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, 0xb9, 0x01, 0x0a, 0x20, 0x47, 0x65, 0x74, - 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, - 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, - 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, - 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, - 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x2b, 0x0a, 0x11, 0x73, 0x65, 0x61, 0x6c, 0x65, - 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, - 0x28, 0x03, 0x52, 0x10, 0x73, 0x65, 0x61, 0x6c, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x49, 0x44, 0x73, 0x22, 0x49, 0x0a, 0x15, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x42, - 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, - 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, - 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, - 0x48, 0x0a, 0x14, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, - 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, 0x4d, 0x0a, 0x19, 0x43, 0x68, 0x65, - 0x63, 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, + 0x1b, 0x0a, 0x09, 0x69, 0x73, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x08, 0x69, 0x73, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x22, 0x5e, 0x0a, 0x12, + 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, + 0x62, 0x61, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, 0x5d, 0x0a, 0x11, + 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, + 0x61, 0x73, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, 0x94, 0x02, 0x0a, 0x16, + 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, - 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x22, 0x6e, 0x0a, 0x1a, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x33, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, - 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x69, - 0x73, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, - 0x69, 0x73, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x22, 0x5e, 0x0a, 0x12, 0x53, 0x75, 0x73, 0x70, - 0x65, 0x6e, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, - 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, - 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, - 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, 0x5d, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x75, - 0x6d, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, + 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x74, + 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0c, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, + 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x5f, 0x61, 0x6c, 0x6c, + 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, + 0x41, 0x6c, 0x6c, 0x12, 0x20, 0x0a, 0x0c, 0x74, 0x6f, 0x5f, 0x61, 0x6c, 0x6c, 0x5f, 0x6e, 0x6f, + 0x64, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x74, 0x6f, 0x41, 0x6c, 0x6c, + 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6d, 0x6f, + 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x63, 0x6f, 0x70, 0x79, 0x4d, 0x6f, + 0x64, 0x65, 0x22, 0x99, 0x02, 0x0a, 0x16, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, - 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, 0x94, 0x02, 0x0a, 0x16, 0x54, 0x72, 0x61, 0x6e, - 0x73, 0x66, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, - 0x62, 0x61, 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, - 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6e, 0x6f, 0x64, + 0x21, 0x0a, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, + 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, @@ -8690,493 +8770,475 @@ var file_query_coord_proto_rawDesc = []byte{ 0x20, 0x0a, 0x0c, 0x74, 0x6f, 0x5f, 0x61, 0x6c, 0x6c, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x74, 0x6f, 0x41, 0x6c, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x07, - 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x63, 0x6f, 0x70, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x22, 0x99, - 0x02, 0x0a, 0x16, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, - 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x63, - 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0b, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x23, - 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4e, 0x6f, 0x64, - 0x65, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6e, 0x6f, - 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x74, 0x61, 0x72, 0x67, - 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, - 0x73, 0x66, 0x65, 0x72, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0b, - 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x41, 0x6c, 0x6c, 0x12, 0x20, 0x0a, 0x0c, 0x74, - 0x6f, 0x5f, 0x61, 0x6c, 0x6c, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0a, 0x74, 0x6f, 0x41, 0x6c, 0x6c, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x1b, 0x0a, - 0x09, 0x63, 0x6f, 0x70, 0x79, 0x5f, 0x6d, 0x6f, 0x64, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x08, 0x63, 0x6f, 0x70, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x22, 0x9f, 0x01, 0x0a, 0x21, 0x43, - 0x68, 0x65, 0x63, 0x6b, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, - 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x63, 0x6f, 0x70, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x22, 0x9f, + 0x01, 0x0a, 0x21, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, + 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, + 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x74, + 0x61, 0x72, 0x67, 0x65, 0x74, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x0c, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, + 0x22, 0xd5, 0x01, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4c, 0x6f, 0x61, 0x64, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, + 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x12, + 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, + 0x49, 0x44, 0x12, 0x24, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6c, + 0x69, 0x63, 0x61, 0x5f, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x0d, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, + 0x27, 0x0a, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, + 0x70, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x13, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, - 0x73, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6e, 0x6f, 0x64, - 0x65, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x23, 0x0a, 0x0d, 0x74, 0x61, 0x72, 0x67, 0x65, - 0x74, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, - 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x22, 0xd5, 0x01, 0x0a, - 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, - 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, - 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x24, - 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, - 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x49, 0x44, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x5f, - 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x72, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x4e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0f, 0x72, - 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x18, 0x05, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, - 0x6f, 0x75, 0x70, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x13, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, - 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, - 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, - 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x22, - 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x49, 0x44, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, - 0x61, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2a, 0x36, 0x0a, 0x09, 0x4c, - 0x6f, 0x61, 0x64, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x46, 0x75, 0x6c, 0x6c, - 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x10, 0x01, 0x12, 0x09, 0x0a, - 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x09, 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, - 0x73, 0x10, 0x03, 0x2a, 0x40, 0x0a, 0x09, 0x44, 0x61, 0x74, 0x61, 0x53, 0x63, 0x6f, 0x70, 0x65, - 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x4b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x07, 0x0a, - 0x03, 0x41, 0x6c, 0x6c, 0x10, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x48, 0x69, 0x73, 0x74, 0x6f, 0x72, 0x69, - 0x63, 0x61, 0x6c, 0x10, 0x03, 0x2a, 0x7a, 0x0a, 0x0e, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x6f, 0x74, 0x45, 0x78, - 0x69, 0x73, 0x74, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4e, 0x6f, 0x74, 0x50, 0x72, 0x65, 0x73, - 0x65, 0x6e, 0x74, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x4f, 0x6e, 0x44, 0x69, 0x73, 0x6b, 0x10, - 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x4d, 0x65, - 0x6d, 0x6f, 0x72, 0x79, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, - 0x72, 0x79, 0x10, 0x04, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, 0x49, - 0x6e, 0x47, 0x50, 0x55, 0x10, 0x05, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x6e, 0x47, 0x50, 0x55, 0x10, - 0x06, 0x2a, 0x64, 0x0a, 0x10, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x43, 0x6f, 0x6e, 0x64, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x13, 0x0a, 0x0f, 0x55, 0x6e, 0x4b, 0x6e, 0x6f, 0x77, 0x43, - 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x48, 0x61, - 0x6e, 0x64, 0x6f, 0x66, 0x66, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x4c, 0x6f, 0x61, 0x64, 0x42, - 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x47, 0x72, 0x70, 0x63, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x6f, 0x64, - 0x65, 0x44, 0x6f, 0x77, 0x6e, 0x10, 0x04, 0x2a, 0x42, 0x0a, 0x08, 0x4c, 0x6f, 0x61, 0x64, 0x54, - 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x6e, 0x4b, 0x6e, 0x6f, 0x77, 0x6e, 0x54, 0x79, - 0x70, 0x65, 0x10, 0x00, 0x12, 0x11, 0x0a, 0x0d, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x12, 0x12, 0x0a, 0x0e, 0x4c, 0x6f, 0x61, 0x64, 0x43, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x2a, 0x32, 0x0a, 0x0a, 0x4c, - 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, 0x07, 0x49, 0x6e, 0x76, - 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x4c, 0x6f, 0x61, 0x64, 0x69, 0x6e, - 0x67, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x4c, 0x6f, 0x61, 0x64, 0x65, 0x64, 0x10, 0x02, 0x2a, - 0x57, 0x0a, 0x08, 0x53, 0x79, 0x6e, 0x63, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x52, - 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x53, 0x65, 0x74, 0x10, 0x01, - 0x12, 0x09, 0x0a, 0x05, 0x41, 0x6d, 0x65, 0x6e, 0x64, 0x10, 0x02, 0x12, 0x11, 0x0a, 0x0d, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x12, 0x18, - 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x10, 0x04, 0x32, 0x87, 0x1d, 0x0a, 0x0a, 0x51, 0x75, 0x65, - 0x72, 0x79, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x12, 0x70, 0x0a, 0x13, 0x53, 0x68, 0x6f, 0x77, 0x4c, - 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2a, + 0x73, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, + 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x2a, + 0x36, 0x0a, 0x09, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, 0x08, 0x0a, 0x04, + 0x46, 0x75, 0x6c, 0x6c, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x10, + 0x01, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x09, 0x0a, 0x05, + 0x53, 0x74, 0x61, 0x74, 0x73, 0x10, 0x03, 0x2a, 0x40, 0x0a, 0x09, 0x44, 0x61, 0x74, 0x61, 0x53, + 0x63, 0x6f, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x4b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, + 0x00, 0x12, 0x07, 0x0a, 0x03, 0x41, 0x6c, 0x6c, 0x10, 0x01, 0x12, 0x0d, 0x0a, 0x09, 0x53, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, 0x48, 0x69, 0x73, + 0x74, 0x6f, 0x72, 0x69, 0x63, 0x61, 0x6c, 0x10, 0x03, 0x2a, 0x7a, 0x0a, 0x0e, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0c, 0x0a, 0x08, 0x4e, + 0x6f, 0x74, 0x45, 0x78, 0x69, 0x73, 0x74, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x4e, 0x6f, 0x74, + 0x50, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x4f, 0x6e, 0x44, + 0x69, 0x73, 0x6b, 0x10, 0x02, 0x12, 0x13, 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x74, 0x69, 0x61, 0x6c, + 0x49, 0x6e, 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x49, 0x6e, + 0x4d, 0x65, 0x6d, 0x6f, 0x72, 0x79, 0x10, 0x04, 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x47, 0x50, 0x55, 0x10, 0x05, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x6e, + 0x47, 0x50, 0x55, 0x10, 0x06, 0x2a, 0x64, 0x0a, 0x10, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, + 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x13, 0x0a, 0x0f, 0x55, 0x6e, 0x4b, + 0x6e, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x0b, + 0x0a, 0x07, 0x48, 0x61, 0x6e, 0x64, 0x6f, 0x66, 0x66, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x4c, + 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, + 0x47, 0x72, 0x70, 0x63, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x10, 0x03, 0x12, 0x0c, 0x0a, + 0x08, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x6f, 0x77, 0x6e, 0x10, 0x04, 0x2a, 0x42, 0x0a, 0x08, 0x4c, + 0x6f, 0x61, 0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x6e, 0x4b, 0x6e, 0x6f, + 0x77, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x10, 0x00, 0x12, 0x11, 0x0a, 0x0d, 0x4c, 0x6f, 0x61, 0x64, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x12, 0x12, 0x0a, 0x0e, 0x4c, + 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x2a, + 0x32, 0x0a, 0x0a, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x0b, 0x0a, + 0x07, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x4c, 0x6f, + 0x61, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x4c, 0x6f, 0x61, 0x64, 0x65, + 0x64, 0x10, 0x02, 0x2a, 0x57, 0x0a, 0x08, 0x53, 0x79, 0x6e, 0x63, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x0a, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x10, 0x00, 0x12, 0x07, 0x0a, 0x03, 0x53, + 0x65, 0x74, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x6d, 0x65, 0x6e, 0x64, 0x10, 0x02, 0x12, + 0x11, 0x0a, 0x0d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x10, 0x03, 0x12, 0x18, 0x0a, 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x10, 0x04, 0x32, 0x87, 0x1d, 0x0a, + 0x0a, 0x51, 0x75, 0x65, 0x72, 0x79, 0x43, 0x6f, 0x6f, 0x72, 0x64, 0x12, 0x70, 0x0a, 0x13, 0x53, + 0x68, 0x6f, 0x77, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x12, 0x53, 0x68, 0x6f, - 0x77, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, - 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x4c, 0x6f, 0x61, 0x64, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, - 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, + 0x12, 0x53, 0x68, 0x6f, 0x77, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, + 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x29, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, + 0x61, 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, + 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x4c, 0x6f, + 0x61, 0x64, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, - 0x61, 0x64, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, - 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x17, 0x53, 0x79, 0x6e, 0x63, 0x4e, 0x65, 0x77, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x4e, 0x65, 0x77, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, + 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x17, 0x53, 0x79, 0x6e, 0x63, + 0x4e, 0x65, 0x77, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x4e, 0x65, 0x77, + 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2d, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, + 0x12, 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, + 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, + 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, - 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, + 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x62, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, + 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, + 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x6c, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, + 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x12, 0x47, 0x65, 0x74, - 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, - 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, - 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, 0x4c, 0x6f, 0x61, 0x64, - 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, - 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x7b, - 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, - 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, - 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, - 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, - 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x0b, - 0x47, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x6c, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, - 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x68, 0x61, 0x72, 0x64, 0x4c, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, - 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x12, 0x27, 0x2e, + 0x64, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x62, 0x0a, 0x0b, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, + 0x68, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, + 0x6c, 0x74, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x65, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x65, - 0x63, 0x6b, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x65, 0x0a, 0x13, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, - 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, - 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x14, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, - 0x73, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, + 0x14, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, + 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x61, 0x0a, 0x11, 0x44, 0x72, 0x6f, 0x70, 0x52, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, + 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0c, 0x54, 0x72, 0x61, 0x6e, + 0x73, 0x66, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x54, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, - 0x00, 0x12, 0x61, 0x0a, 0x11, 0x44, 0x72, 0x6f, 0x70, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, 0x6f, - 0x70, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, - 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, - 0x66, 0x65, 0x72, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, - 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5c, 0x0a, - 0x0f, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, + 0x00, 0x12, 0x5c, 0x0a, 0x0f, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, + 0x65, 0x72, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, + 0x77, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, + 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, + 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, + 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x15, 0x44, 0x65, 0x73, 0x63, + 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, + 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x0c, 0x4c, 0x69, 0x73, 0x74, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, 0x6b, + 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5c, 0x0a, + 0x0f, 0x41, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x52, 0x65, - 0x70, 0x6c, 0x69, 0x63, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, + 0x65, 0x63, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, - 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x77, 0x0a, 0x12, 0x4c, - 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, - 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x15, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, - 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x30, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, - 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, 0x0c, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x65, 0x72, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x68, - 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x0f, 0x41, 0x63, 0x74, - 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x12, 0x2a, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, - 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x44, 0x65, 0x61, 0x63, 0x74, - 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x12, 0x2c, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x2e, 0x44, 0x65, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, - 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x4c, 0x69, 0x73, - 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, - 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x87, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, - 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x33, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x44, + 0x65, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, + 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x61, 0x63, 0x74, 0x69, 0x76, 0x61, 0x74, 0x65, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x66, 0x0a, + 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, - 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x51, 0x75, 0x65, 0x72, - 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x53, - 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x29, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x58, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x75, 0x6d, - 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, - 0x73, 0x75, 0x6d, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, - 0x00, 0x12, 0x75, 0x0a, 0x12, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, - 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, 0x68, 0x65, - 0x63, 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, 0x53, 0x75, 0x73, 0x70, - 0x65, 0x6e, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x75, 0x73, - 0x70, 0x65, 0x6e, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x52, - 0x0a, 0x0a, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x25, 0x2e, 0x6d, + 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x87, 0x01, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x33, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x34, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, + 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x5a, 0x0a, 0x0e, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, + 0x65, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x42, 0x61, + 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x58, 0x0a, 0x0d, 0x52, + 0x65, 0x73, 0x75, 0x6d, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x79, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x61, + 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x54, 0x0a, 0x0b, + 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x53, 0x75, 0x73, 0x70, 0x65, 0x6e, 0x64, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x0f, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, - 0x66, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, - 0x12, 0x5c, 0x0a, 0x0f, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, - 0x6e, 0x65, 0x6c, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, - 0x72, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x72, - 0x0a, 0x1a, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, - 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x35, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, - 0x79, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, - 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0a, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x4e, 0x6f, 0x64, 0x65, + 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x4e, 0x6f, 0x64, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x0f, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, + 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, + 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x0f, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x65, 0x72, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x54, 0x72, 0x61, + 0x6e, 0x73, 0x66, 0x65, 0x72, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x10, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4c, 0x6f, 0x61, 0x64, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x65, 0x71, 0x75, + 0x22, 0x00, 0x12, 0x72, 0x0a, 0x1a, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x35, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x4e, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x10, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4c, 0x6f, 0x61, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x32, 0x8b, 0x14, 0x0a, 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x4e, 0x6f, 0x64, 0x65, 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, + 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, + 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, + 0x6b, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x2e, 0x47, 0x65, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x71, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, + 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x0f, 0x57, 0x61, 0x74, 0x63, 0x68, 0x44, 0x6d, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x57, 0x61, 0x74, + 0x63, 0x68, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x22, 0x00, 0x32, 0x8b, 0x14, 0x0a, 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4e, 0x6f, 0x64, 0x65, - 0x12, 0x6c, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, - 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x6f, 0x6d, - 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x22, 0x00, 0x12, 0x6d, - 0x0a, 0x12, 0x47, 0x65, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x43, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, - 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x72, - 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, - 0x14, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, - 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, - 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x43, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x5c, 0x0a, 0x0f, 0x57, 0x61, 0x74, 0x63, 0x68, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x44, 0x6d, - 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x44, 0x6d, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x44, + 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, - 0x0a, 0x0e, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, - 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x55, 0x6e, 0x73, 0x75, 0x62, 0x44, 0x6d, 0x43, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, - 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x56, 0x0a, 0x0c, 0x4c, 0x6f, - 0x61, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x56, + 0x0a, 0x0c, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x27, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, + 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, + 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x4c, 0x6f, 0x61, 0x64, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, - 0x4c, 0x6f, 0x61, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, - 0x65, 0x61, 0x73, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, + 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x22, 0x00, 0x12, 0x5a, 0x0a, 0x0e, 0x4c, 0x6f, 0x61, 0x64, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, - 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, - 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x0f, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x53, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x6c, 0x65, 0x61, - 0x73, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, - 0x12, 0x69, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, - 0x66, 0x6f, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, - 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x13, 0x53, - 0x79, 0x6e, 0x63, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x52, 0x65, 0x70, 0x6c, - 0x69, 0x63, 0x61, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, - 0x00, 0x12, 0x69, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, - 0x63, 0x73, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, - 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, - 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, - 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x53, 0x0a, 0x06, - 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x12, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x61, 0x72, - 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, - 0x6c, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, - 0x00, 0x12, 0x5b, 0x0a, 0x0e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x73, 0x12, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, - 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x12, 0x53, - 0x0a, 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, - 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, - 0x6c, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, - 0x73, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0b, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x65, 0x74, - 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x30, 0x01, - 0x12, 0x5b, 0x0a, 0x0d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x73, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x65, 0x74, 0x72, - 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x12, 0x63, 0x0a, - 0x13, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x73, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, - 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, - 0x30, 0x01, 0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x11, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, + 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x0f, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, + 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x52, + 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x64, 0x0a, 0x13, 0x53, 0x79, 0x6e, 0x63, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, 0x63, + 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x53, 0x0a, 0x06, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x12, 0x21, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, + 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x73, 0x75, + 0x6c, 0x74, 0x73, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x61, + 0x72, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x2e, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, + 0x22, 0x00, 0x12, 0x53, 0x0a, 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x12, 0x20, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, + 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0b, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, + 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, - 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, + 0x2e, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, + 0x22, 0x00, 0x30, 0x01, 0x12, 0x5b, 0x0a, 0x0d, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, + 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x22, + 0x00, 0x12, 0x63, 0x0a, 0x13, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, - 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x26, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, - 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x78, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x44, 0x69, 0x73, 0x74, 0x72, - 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, - 0x44, 0x61, 0x74, 0x61, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x47, 0x65, 0x74, - 0x44, 0x61, 0x74, 0x61, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x10, 0x53, 0x79, - 0x6e, 0x63, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, - 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, + 0x61, 0x6c, 0x2e, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x65, 0x52, 0x65, 0x73, 0x75, 0x6c, + 0x74, 0x73, 0x22, 0x00, 0x30, 0x01, 0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, + 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, + 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x78, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x44, + 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, + 0x2e, 0x47, 0x65, 0x74, 0x44, 0x61, 0x74, 0x61, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, + 0x0a, 0x10, 0x53, 0x79, 0x6e, 0x63, 0x44, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x53, 0x79, 0x6e, 0x63, 0x44, 0x69, 0x73, 0x74, + 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x4a, + 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, - 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x4a, 0x0a, 0x06, 0x44, 0x65, - 0x6c, 0x65, 0x74, 0x65, 0x12, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0b, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x56, 0x0a, 0x0c, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, - 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2f, - 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x71, 0x75, 0x65, - 0x72, 0x79, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0b, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x71, 0x75, 0x65, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x42, 0x61, 0x74, + 0x63, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x56, 0x0a, 0x0c, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x27, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x22, 0x00, 0x42, 0x32, 0x5a, 0x30, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, + 0x6f, 0x6d, 0x2f, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2d, 0x69, 0x6f, 0x2f, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x76, 0x32, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2f, 0x71, 0x75, 0x65, 0x72, 0x79, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -9192,7 +9254,7 @@ func file_query_coord_proto_rawDescGZIP() []byte { } var file_query_coord_proto_enumTypes = make([]protoimpl.EnumInfo, 7) -var file_query_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 105) +var file_query_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 106) var file_query_coord_proto_goTypes = []interface{}{ (LoadScope)(0), // 0: milvus.proto.query.LoadScope (DataScope)(0), // 1: milvus.proto.query.DataScope @@ -9246,406 +9308,408 @@ var file_query_coord_proto_goTypes = []interface{}{ (*GetDataDistributionRequest)(nil), // 49: milvus.proto.query.GetDataDistributionRequest (*GetDataDistributionResponse)(nil), // 50: milvus.proto.query.GetDataDistributionResponse (*LeaderView)(nil), // 51: milvus.proto.query.LeaderView - (*SegmentDist)(nil), // 52: milvus.proto.query.SegmentDist - (*SegmentVersionInfo)(nil), // 53: milvus.proto.query.SegmentVersionInfo - (*ChannelVersionInfo)(nil), // 54: milvus.proto.query.ChannelVersionInfo - (*CollectionLoadInfo)(nil), // 55: milvus.proto.query.CollectionLoadInfo - (*PartitionLoadInfo)(nil), // 56: milvus.proto.query.PartitionLoadInfo - (*ChannelNodeInfo)(nil), // 57: milvus.proto.query.ChannelNodeInfo - (*Replica)(nil), // 58: milvus.proto.query.Replica - (*SyncAction)(nil), // 59: milvus.proto.query.SyncAction - (*SyncDistributionRequest)(nil), // 60: milvus.proto.query.SyncDistributionRequest - (*ResourceGroup)(nil), // 61: milvus.proto.query.ResourceGroup - (*TransferReplicaRequest)(nil), // 62: milvus.proto.query.TransferReplicaRequest - (*DescribeResourceGroupRequest)(nil), // 63: milvus.proto.query.DescribeResourceGroupRequest - (*DescribeResourceGroupResponse)(nil), // 64: milvus.proto.query.DescribeResourceGroupResponse - (*ResourceGroupInfo)(nil), // 65: milvus.proto.query.ResourceGroupInfo - (*DeleteRequest)(nil), // 66: milvus.proto.query.DeleteRequest - (*DeleteBatchRequest)(nil), // 67: milvus.proto.query.DeleteBatchRequest - (*DeleteBatchResponse)(nil), // 68: milvus.proto.query.DeleteBatchResponse - (*ActivateCheckerRequest)(nil), // 69: milvus.proto.query.ActivateCheckerRequest - (*DeactivateCheckerRequest)(nil), // 70: milvus.proto.query.DeactivateCheckerRequest - (*ListCheckersRequest)(nil), // 71: milvus.proto.query.ListCheckersRequest - (*ListCheckersResponse)(nil), // 72: milvus.proto.query.ListCheckersResponse - (*CheckerInfo)(nil), // 73: milvus.proto.query.CheckerInfo - (*SegmentTarget)(nil), // 74: milvus.proto.query.SegmentTarget - (*PartitionTarget)(nil), // 75: milvus.proto.query.PartitionTarget - (*ChannelTarget)(nil), // 76: milvus.proto.query.ChannelTarget - (*CollectionTarget)(nil), // 77: milvus.proto.query.CollectionTarget - (*NodeInfo)(nil), // 78: milvus.proto.query.NodeInfo - (*ListQueryNodeRequest)(nil), // 79: milvus.proto.query.ListQueryNodeRequest - (*ListQueryNodeResponse)(nil), // 80: milvus.proto.query.ListQueryNodeResponse - (*GetQueryNodeDistributionRequest)(nil), // 81: milvus.proto.query.GetQueryNodeDistributionRequest - (*GetQueryNodeDistributionResponse)(nil), // 82: milvus.proto.query.GetQueryNodeDistributionResponse - (*SuspendBalanceRequest)(nil), // 83: milvus.proto.query.SuspendBalanceRequest - (*ResumeBalanceRequest)(nil), // 84: milvus.proto.query.ResumeBalanceRequest - (*CheckBalanceStatusRequest)(nil), // 85: milvus.proto.query.CheckBalanceStatusRequest - (*CheckBalanceStatusResponse)(nil), // 86: milvus.proto.query.CheckBalanceStatusResponse - (*SuspendNodeRequest)(nil), // 87: milvus.proto.query.SuspendNodeRequest - (*ResumeNodeRequest)(nil), // 88: milvus.proto.query.ResumeNodeRequest - (*TransferSegmentRequest)(nil), // 89: milvus.proto.query.TransferSegmentRequest - (*TransferChannelRequest)(nil), // 90: milvus.proto.query.TransferChannelRequest - (*CheckQueryNodeDistributionRequest)(nil), // 91: milvus.proto.query.CheckQueryNodeDistributionRequest - (*UpdateLoadConfigRequest)(nil), // 92: milvus.proto.query.UpdateLoadConfigRequest - (*UpdateSchemaRequest)(nil), // 93: milvus.proto.query.UpdateSchemaRequest - nil, // 94: milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry - nil, // 95: milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry - nil, // 96: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry - nil, // 97: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry - nil, // 98: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry - nil, // 99: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry - nil, // 100: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry - nil, // 101: milvus.proto.query.LeaderView.SegmentDistEntry - nil, // 102: milvus.proto.query.LeaderView.GrowingSegmentsEntry - nil, // 103: milvus.proto.query.LeaderView.PartitionStatsVersionsEntry - nil, // 104: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry - nil, // 105: milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry - nil, // 106: milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry - nil, // 107: milvus.proto.query.Replica.ChannelNodeInfosEntry - nil, // 108: milvus.proto.query.SyncAction.PartitionStatsVersionsEntry - nil, // 109: milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry - nil, // 110: milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry - nil, // 111: milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry - (*commonpb.MsgBase)(nil), // 112: milvus.proto.common.MsgBase - (*commonpb.Status)(nil), // 113: milvus.proto.common.Status - (*schemapb.LongArray)(nil), // 114: milvus.proto.schema.LongArray - (*schemapb.CollectionSchema)(nil), // 115: milvus.proto.schema.CollectionSchema - (*internalpb.GetStatisticsRequest)(nil), // 116: milvus.proto.internal.GetStatisticsRequest - (*indexpb.IndexInfo)(nil), // 117: milvus.proto.index.IndexInfo - (*commonpb.KeyValuePair)(nil), // 118: milvus.proto.common.KeyValuePair - (*datapb.VchannelInfo)(nil), // 119: milvus.proto.data.VchannelInfo - (*datapb.SegmentInfo)(nil), // 120: milvus.proto.data.SegmentInfo - (*datapb.FieldBinlog)(nil), // 121: milvus.proto.data.FieldBinlog - (*msgpb.MsgPosition)(nil), // 122: milvus.proto.msg.MsgPosition - (datapb.SegmentLevel)(0), // 123: milvus.proto.data.SegmentLevel - (*internalpb.SearchRequest)(nil), // 124: milvus.proto.internal.SearchRequest - (*internalpb.RetrieveRequest)(nil), // 125: milvus.proto.internal.RetrieveRequest - (commonpb.SegmentState)(0), // 126: milvus.proto.common.SegmentState - (*rgpb.ResourceGroupConfig)(nil), // 127: milvus.proto.rg.ResourceGroupConfig - (*commonpb.NodeInfo)(nil), // 128: milvus.proto.common.NodeInfo - (*schemapb.IDs)(nil), // 129: milvus.proto.schema.IDs - (*datapb.TextIndexStats)(nil), // 130: milvus.proto.data.TextIndexStats - (*datapb.JsonKeyStats)(nil), // 131: milvus.proto.data.JsonKeyStats - (*internalpb.ShowConfigurationsRequest)(nil), // 132: milvus.proto.internal.ShowConfigurationsRequest - (*milvuspb.GetMetricsRequest)(nil), // 133: milvus.proto.milvus.GetMetricsRequest - (*milvuspb.GetReplicasRequest)(nil), // 134: milvus.proto.milvus.GetReplicasRequest - (*milvuspb.CheckHealthRequest)(nil), // 135: milvus.proto.milvus.CheckHealthRequest - (*milvuspb.CreateResourceGroupRequest)(nil), // 136: milvus.proto.milvus.CreateResourceGroupRequest - (*milvuspb.DropResourceGroupRequest)(nil), // 137: milvus.proto.milvus.DropResourceGroupRequest - (*milvuspb.TransferNodeRequest)(nil), // 138: milvus.proto.milvus.TransferNodeRequest - (*milvuspb.ListResourceGroupsRequest)(nil), // 139: milvus.proto.milvus.ListResourceGroupsRequest - (*milvuspb.GetComponentStatesRequest)(nil), // 140: milvus.proto.milvus.GetComponentStatesRequest - (*internalpb.GetTimeTickChannelRequest)(nil), // 141: milvus.proto.internal.GetTimeTickChannelRequest - (*internalpb.GetStatisticsChannelRequest)(nil), // 142: milvus.proto.internal.GetStatisticsChannelRequest - (*internalpb.ShowConfigurationsResponse)(nil), // 143: milvus.proto.internal.ShowConfigurationsResponse - (*milvuspb.GetMetricsResponse)(nil), // 144: milvus.proto.milvus.GetMetricsResponse - (*milvuspb.GetReplicasResponse)(nil), // 145: milvus.proto.milvus.GetReplicasResponse - (*milvuspb.CheckHealthResponse)(nil), // 146: milvus.proto.milvus.CheckHealthResponse - (*milvuspb.ListResourceGroupsResponse)(nil), // 147: milvus.proto.milvus.ListResourceGroupsResponse - (*milvuspb.ComponentStates)(nil), // 148: milvus.proto.milvus.ComponentStates - (*milvuspb.StringResponse)(nil), // 149: milvus.proto.milvus.StringResponse - (*internalpb.GetStatisticsResponse)(nil), // 150: milvus.proto.internal.GetStatisticsResponse - (*internalpb.SearchResults)(nil), // 151: milvus.proto.internal.SearchResults - (*internalpb.RetrieveResults)(nil), // 152: milvus.proto.internal.RetrieveResults + (*LeaderViewStatus)(nil), // 52: milvus.proto.query.LeaderViewStatus + (*SegmentDist)(nil), // 53: milvus.proto.query.SegmentDist + (*SegmentVersionInfo)(nil), // 54: milvus.proto.query.SegmentVersionInfo + (*ChannelVersionInfo)(nil), // 55: milvus.proto.query.ChannelVersionInfo + (*CollectionLoadInfo)(nil), // 56: milvus.proto.query.CollectionLoadInfo + (*PartitionLoadInfo)(nil), // 57: milvus.proto.query.PartitionLoadInfo + (*ChannelNodeInfo)(nil), // 58: milvus.proto.query.ChannelNodeInfo + (*Replica)(nil), // 59: milvus.proto.query.Replica + (*SyncAction)(nil), // 60: milvus.proto.query.SyncAction + (*SyncDistributionRequest)(nil), // 61: milvus.proto.query.SyncDistributionRequest + (*ResourceGroup)(nil), // 62: milvus.proto.query.ResourceGroup + (*TransferReplicaRequest)(nil), // 63: milvus.proto.query.TransferReplicaRequest + (*DescribeResourceGroupRequest)(nil), // 64: milvus.proto.query.DescribeResourceGroupRequest + (*DescribeResourceGroupResponse)(nil), // 65: milvus.proto.query.DescribeResourceGroupResponse + (*ResourceGroupInfo)(nil), // 66: milvus.proto.query.ResourceGroupInfo + (*DeleteRequest)(nil), // 67: milvus.proto.query.DeleteRequest + (*DeleteBatchRequest)(nil), // 68: milvus.proto.query.DeleteBatchRequest + (*DeleteBatchResponse)(nil), // 69: milvus.proto.query.DeleteBatchResponse + (*ActivateCheckerRequest)(nil), // 70: milvus.proto.query.ActivateCheckerRequest + (*DeactivateCheckerRequest)(nil), // 71: milvus.proto.query.DeactivateCheckerRequest + (*ListCheckersRequest)(nil), // 72: milvus.proto.query.ListCheckersRequest + (*ListCheckersResponse)(nil), // 73: milvus.proto.query.ListCheckersResponse + (*CheckerInfo)(nil), // 74: milvus.proto.query.CheckerInfo + (*SegmentTarget)(nil), // 75: milvus.proto.query.SegmentTarget + (*PartitionTarget)(nil), // 76: milvus.proto.query.PartitionTarget + (*ChannelTarget)(nil), // 77: milvus.proto.query.ChannelTarget + (*CollectionTarget)(nil), // 78: milvus.proto.query.CollectionTarget + (*NodeInfo)(nil), // 79: milvus.proto.query.NodeInfo + (*ListQueryNodeRequest)(nil), // 80: milvus.proto.query.ListQueryNodeRequest + (*ListQueryNodeResponse)(nil), // 81: milvus.proto.query.ListQueryNodeResponse + (*GetQueryNodeDistributionRequest)(nil), // 82: milvus.proto.query.GetQueryNodeDistributionRequest + (*GetQueryNodeDistributionResponse)(nil), // 83: milvus.proto.query.GetQueryNodeDistributionResponse + (*SuspendBalanceRequest)(nil), // 84: milvus.proto.query.SuspendBalanceRequest + (*ResumeBalanceRequest)(nil), // 85: milvus.proto.query.ResumeBalanceRequest + (*CheckBalanceStatusRequest)(nil), // 86: milvus.proto.query.CheckBalanceStatusRequest + (*CheckBalanceStatusResponse)(nil), // 87: milvus.proto.query.CheckBalanceStatusResponse + (*SuspendNodeRequest)(nil), // 88: milvus.proto.query.SuspendNodeRequest + (*ResumeNodeRequest)(nil), // 89: milvus.proto.query.ResumeNodeRequest + (*TransferSegmentRequest)(nil), // 90: milvus.proto.query.TransferSegmentRequest + (*TransferChannelRequest)(nil), // 91: milvus.proto.query.TransferChannelRequest + (*CheckQueryNodeDistributionRequest)(nil), // 92: milvus.proto.query.CheckQueryNodeDistributionRequest + (*UpdateLoadConfigRequest)(nil), // 93: milvus.proto.query.UpdateLoadConfigRequest + (*UpdateSchemaRequest)(nil), // 94: milvus.proto.query.UpdateSchemaRequest + nil, // 95: milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry + nil, // 96: milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry + nil, // 97: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry + nil, // 98: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry + nil, // 99: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry + nil, // 100: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry + nil, // 101: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry + nil, // 102: milvus.proto.query.LeaderView.SegmentDistEntry + nil, // 103: milvus.proto.query.LeaderView.GrowingSegmentsEntry + nil, // 104: milvus.proto.query.LeaderView.PartitionStatsVersionsEntry + nil, // 105: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry + nil, // 106: milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry + nil, // 107: milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry + nil, // 108: milvus.proto.query.Replica.ChannelNodeInfosEntry + nil, // 109: milvus.proto.query.SyncAction.PartitionStatsVersionsEntry + nil, // 110: milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry + nil, // 111: milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry + nil, // 112: milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry + (*commonpb.MsgBase)(nil), // 113: milvus.proto.common.MsgBase + (*commonpb.Status)(nil), // 114: milvus.proto.common.Status + (*schemapb.LongArray)(nil), // 115: milvus.proto.schema.LongArray + (*schemapb.CollectionSchema)(nil), // 116: milvus.proto.schema.CollectionSchema + (*internalpb.GetStatisticsRequest)(nil), // 117: milvus.proto.internal.GetStatisticsRequest + (*indexpb.IndexInfo)(nil), // 118: milvus.proto.index.IndexInfo + (*commonpb.KeyValuePair)(nil), // 119: milvus.proto.common.KeyValuePair + (*datapb.VchannelInfo)(nil), // 120: milvus.proto.data.VchannelInfo + (*datapb.SegmentInfo)(nil), // 121: milvus.proto.data.SegmentInfo + (*datapb.FieldBinlog)(nil), // 122: milvus.proto.data.FieldBinlog + (*msgpb.MsgPosition)(nil), // 123: milvus.proto.msg.MsgPosition + (datapb.SegmentLevel)(0), // 124: milvus.proto.data.SegmentLevel + (*internalpb.SearchRequest)(nil), // 125: milvus.proto.internal.SearchRequest + (*internalpb.RetrieveRequest)(nil), // 126: milvus.proto.internal.RetrieveRequest + (commonpb.SegmentState)(0), // 127: milvus.proto.common.SegmentState + (*rgpb.ResourceGroupConfig)(nil), // 128: milvus.proto.rg.ResourceGroupConfig + (*commonpb.NodeInfo)(nil), // 129: milvus.proto.common.NodeInfo + (*schemapb.IDs)(nil), // 130: milvus.proto.schema.IDs + (*datapb.TextIndexStats)(nil), // 131: milvus.proto.data.TextIndexStats + (*datapb.JsonKeyStats)(nil), // 132: milvus.proto.data.JsonKeyStats + (*internalpb.ShowConfigurationsRequest)(nil), // 133: milvus.proto.internal.ShowConfigurationsRequest + (*milvuspb.GetMetricsRequest)(nil), // 134: milvus.proto.milvus.GetMetricsRequest + (*milvuspb.GetReplicasRequest)(nil), // 135: milvus.proto.milvus.GetReplicasRequest + (*milvuspb.CheckHealthRequest)(nil), // 136: milvus.proto.milvus.CheckHealthRequest + (*milvuspb.CreateResourceGroupRequest)(nil), // 137: milvus.proto.milvus.CreateResourceGroupRequest + (*milvuspb.DropResourceGroupRequest)(nil), // 138: milvus.proto.milvus.DropResourceGroupRequest + (*milvuspb.TransferNodeRequest)(nil), // 139: milvus.proto.milvus.TransferNodeRequest + (*milvuspb.ListResourceGroupsRequest)(nil), // 140: milvus.proto.milvus.ListResourceGroupsRequest + (*milvuspb.GetComponentStatesRequest)(nil), // 141: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetTimeTickChannelRequest)(nil), // 142: milvus.proto.internal.GetTimeTickChannelRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 143: milvus.proto.internal.GetStatisticsChannelRequest + (*internalpb.ShowConfigurationsResponse)(nil), // 144: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 145: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.GetReplicasResponse)(nil), // 146: milvus.proto.milvus.GetReplicasResponse + (*milvuspb.CheckHealthResponse)(nil), // 147: milvus.proto.milvus.CheckHealthResponse + (*milvuspb.ListResourceGroupsResponse)(nil), // 148: milvus.proto.milvus.ListResourceGroupsResponse + (*milvuspb.ComponentStates)(nil), // 149: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 150: milvus.proto.milvus.StringResponse + (*internalpb.GetStatisticsResponse)(nil), // 151: milvus.proto.internal.GetStatisticsResponse + (*internalpb.SearchResults)(nil), // 152: milvus.proto.internal.SearchResults + (*internalpb.RetrieveResults)(nil), // 153: milvus.proto.internal.RetrieveResults } var file_query_coord_proto_depIdxs = []int32{ - 112, // 0: milvus.proto.query.ShowCollectionsRequest.base:type_name -> milvus.proto.common.MsgBase - 113, // 1: milvus.proto.query.ShowCollectionsResponse.status:type_name -> milvus.proto.common.Status - 114, // 2: milvus.proto.query.ShowCollectionsResponse.load_fields:type_name -> milvus.proto.schema.LongArray - 112, // 3: milvus.proto.query.ShowPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase - 113, // 4: milvus.proto.query.ShowPartitionsResponse.status:type_name -> milvus.proto.common.Status - 112, // 5: milvus.proto.query.LoadCollectionRequest.base:type_name -> milvus.proto.common.MsgBase - 115, // 6: milvus.proto.query.LoadCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 94, // 7: milvus.proto.query.LoadCollectionRequest.field_indexID:type_name -> milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry - 112, // 8: milvus.proto.query.ReleaseCollectionRequest.base:type_name -> milvus.proto.common.MsgBase - 116, // 9: milvus.proto.query.GetStatisticsRequest.req:type_name -> milvus.proto.internal.GetStatisticsRequest + 113, // 0: milvus.proto.query.ShowCollectionsRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 1: milvus.proto.query.ShowCollectionsResponse.status:type_name -> milvus.proto.common.Status + 115, // 2: milvus.proto.query.ShowCollectionsResponse.load_fields:type_name -> milvus.proto.schema.LongArray + 113, // 3: milvus.proto.query.ShowPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 4: milvus.proto.query.ShowPartitionsResponse.status:type_name -> milvus.proto.common.Status + 113, // 5: milvus.proto.query.LoadCollectionRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 6: milvus.proto.query.LoadCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 95, // 7: milvus.proto.query.LoadCollectionRequest.field_indexID:type_name -> milvus.proto.query.LoadCollectionRequest.FieldIndexIDEntry + 113, // 8: milvus.proto.query.ReleaseCollectionRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 9: milvus.proto.query.GetStatisticsRequest.req:type_name -> milvus.proto.internal.GetStatisticsRequest 1, // 10: milvus.proto.query.GetStatisticsRequest.scope:type_name -> milvus.proto.query.DataScope - 112, // 11: milvus.proto.query.LoadPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase - 115, // 12: milvus.proto.query.LoadPartitionsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 95, // 13: milvus.proto.query.LoadPartitionsRequest.field_indexID:type_name -> milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry - 117, // 14: milvus.proto.query.LoadPartitionsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 112, // 15: milvus.proto.query.ReleasePartitionsRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 16: milvus.proto.query.GetPartitionStatesRequest.base:type_name -> milvus.proto.common.MsgBase - 113, // 17: milvus.proto.query.GetPartitionStatesResponse.status:type_name -> milvus.proto.common.Status + 113, // 11: milvus.proto.query.LoadPartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 12: milvus.proto.query.LoadPartitionsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 96, // 13: milvus.proto.query.LoadPartitionsRequest.field_indexID:type_name -> milvus.proto.query.LoadPartitionsRequest.FieldIndexIDEntry + 118, // 14: milvus.proto.query.LoadPartitionsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 113, // 15: milvus.proto.query.ReleasePartitionsRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 16: milvus.proto.query.GetPartitionStatesRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 17: milvus.proto.query.GetPartitionStatesResponse.status:type_name -> milvus.proto.common.Status 42, // 18: milvus.proto.query.GetPartitionStatesResponse.partition_descriptions:type_name -> milvus.proto.query.PartitionStates - 112, // 19: milvus.proto.query.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 113, // 20: milvus.proto.query.GetSegmentInfoResponse.status:type_name -> milvus.proto.common.Status + 113, // 19: milvus.proto.query.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 20: milvus.proto.query.GetSegmentInfoResponse.status:type_name -> milvus.proto.common.Status 43, // 21: milvus.proto.query.GetSegmentInfoResponse.infos:type_name -> milvus.proto.query.SegmentInfo - 112, // 22: milvus.proto.query.GetShardLeadersRequest.base:type_name -> milvus.proto.common.MsgBase - 113, // 23: milvus.proto.query.GetShardLeadersResponse.status:type_name -> milvus.proto.common.Status + 113, // 22: milvus.proto.query.GetShardLeadersRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 23: milvus.proto.query.GetShardLeadersResponse.status:type_name -> milvus.proto.common.Status 23, // 24: milvus.proto.query.GetShardLeadersResponse.shards:type_name -> milvus.proto.query.ShardLeadersList - 112, // 25: milvus.proto.query.UpdateResourceGroupsRequest.base:type_name -> milvus.proto.common.MsgBase - 96, // 26: milvus.proto.query.UpdateResourceGroupsRequest.resource_groups:type_name -> milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry - 112, // 27: milvus.proto.query.SyncNewCreatedPartitionRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 25: milvus.proto.query.UpdateResourceGroupsRequest.base:type_name -> milvus.proto.common.MsgBase + 97, // 26: milvus.proto.query.UpdateResourceGroupsRequest.resource_groups:type_name -> milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry + 113, // 27: milvus.proto.query.SyncNewCreatedPartitionRequest.base:type_name -> milvus.proto.common.MsgBase 4, // 28: milvus.proto.query.LoadMetaInfo.load_type:type_name -> milvus.proto.query.LoadType - 118, // 29: milvus.proto.query.LoadMetaInfo.db_properties:type_name -> milvus.proto.common.KeyValuePair - 112, // 30: milvus.proto.query.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase - 119, // 31: milvus.proto.query.WatchDmChannelsRequest.infos:type_name -> milvus.proto.data.VchannelInfo - 115, // 32: milvus.proto.query.WatchDmChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 120, // 33: milvus.proto.query.WatchDmChannelsRequest.exclude_infos:type_name -> milvus.proto.data.SegmentInfo + 119, // 29: milvus.proto.query.LoadMetaInfo.db_properties:type_name -> milvus.proto.common.KeyValuePair + 113, // 30: milvus.proto.query.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase + 120, // 31: milvus.proto.query.WatchDmChannelsRequest.infos:type_name -> milvus.proto.data.VchannelInfo + 116, // 32: milvus.proto.query.WatchDmChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 121, // 33: milvus.proto.query.WatchDmChannelsRequest.exclude_infos:type_name -> milvus.proto.data.SegmentInfo 25, // 34: milvus.proto.query.WatchDmChannelsRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo - 97, // 35: milvus.proto.query.WatchDmChannelsRequest.segment_infos:type_name -> milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry - 117, // 36: milvus.proto.query.WatchDmChannelsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 112, // 37: milvus.proto.query.UnsubDmChannelRequest.base:type_name -> milvus.proto.common.MsgBase - 121, // 38: milvus.proto.query.SegmentLoadInfo.binlog_paths:type_name -> milvus.proto.data.FieldBinlog - 121, // 39: milvus.proto.query.SegmentLoadInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog - 121, // 40: milvus.proto.query.SegmentLoadInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 98, // 35: milvus.proto.query.WatchDmChannelsRequest.segment_infos:type_name -> milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry + 118, // 36: milvus.proto.query.WatchDmChannelsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 113, // 37: milvus.proto.query.UnsubDmChannelRequest.base:type_name -> milvus.proto.common.MsgBase + 122, // 38: milvus.proto.query.SegmentLoadInfo.binlog_paths:type_name -> milvus.proto.data.FieldBinlog + 122, // 39: milvus.proto.query.SegmentLoadInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog + 122, // 40: milvus.proto.query.SegmentLoadInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog 29, // 41: milvus.proto.query.SegmentLoadInfo.index_infos:type_name -> milvus.proto.query.FieldIndexInfo - 122, // 42: milvus.proto.query.SegmentLoadInfo.start_position:type_name -> milvus.proto.msg.MsgPosition - 122, // 43: milvus.proto.query.SegmentLoadInfo.delta_position:type_name -> milvus.proto.msg.MsgPosition - 123, // 44: milvus.proto.query.SegmentLoadInfo.level:type_name -> milvus.proto.data.SegmentLevel - 98, // 45: milvus.proto.query.SegmentLoadInfo.textStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry - 121, // 46: milvus.proto.query.SegmentLoadInfo.bm25logs:type_name -> milvus.proto.data.FieldBinlog - 99, // 47: milvus.proto.query.SegmentLoadInfo.jsonKeyStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry - 118, // 48: milvus.proto.query.FieldIndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair - 112, // 49: milvus.proto.query.LoadSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 123, // 42: milvus.proto.query.SegmentLoadInfo.start_position:type_name -> milvus.proto.msg.MsgPosition + 123, // 43: milvus.proto.query.SegmentLoadInfo.delta_position:type_name -> milvus.proto.msg.MsgPosition + 124, // 44: milvus.proto.query.SegmentLoadInfo.level:type_name -> milvus.proto.data.SegmentLevel + 99, // 45: milvus.proto.query.SegmentLoadInfo.textStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry + 122, // 46: milvus.proto.query.SegmentLoadInfo.bm25logs:type_name -> milvus.proto.data.FieldBinlog + 100, // 47: milvus.proto.query.SegmentLoadInfo.jsonKeyStatsLogs:type_name -> milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry + 119, // 48: milvus.proto.query.FieldIndexInfo.index_params:type_name -> milvus.proto.common.KeyValuePair + 113, // 49: milvus.proto.query.LoadSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 28, // 50: milvus.proto.query.LoadSegmentsRequest.infos:type_name -> milvus.proto.query.SegmentLoadInfo - 115, // 51: milvus.proto.query.LoadSegmentsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 116, // 51: milvus.proto.query.LoadSegmentsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema 25, // 52: milvus.proto.query.LoadSegmentsRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo - 122, // 53: milvus.proto.query.LoadSegmentsRequest.delta_positions:type_name -> milvus.proto.msg.MsgPosition + 123, // 53: milvus.proto.query.LoadSegmentsRequest.delta_positions:type_name -> milvus.proto.msg.MsgPosition 0, // 54: milvus.proto.query.LoadSegmentsRequest.load_scope:type_name -> milvus.proto.query.LoadScope - 117, // 55: milvus.proto.query.LoadSegmentsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 112, // 56: milvus.proto.query.ReleaseSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 118, // 55: milvus.proto.query.LoadSegmentsRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 113, // 56: milvus.proto.query.ReleaseSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 1, // 57: milvus.proto.query.ReleaseSegmentsRequest.scope:type_name -> milvus.proto.query.DataScope - 122, // 58: milvus.proto.query.ReleaseSegmentsRequest.checkpoint:type_name -> milvus.proto.msg.MsgPosition - 124, // 59: milvus.proto.query.SearchRequest.req:type_name -> milvus.proto.internal.SearchRequest + 123, // 58: milvus.proto.query.ReleaseSegmentsRequest.checkpoint:type_name -> milvus.proto.msg.MsgPosition + 125, // 59: milvus.proto.query.SearchRequest.req:type_name -> milvus.proto.internal.SearchRequest 1, // 60: milvus.proto.query.SearchRequest.scope:type_name -> milvus.proto.query.DataScope - 125, // 61: milvus.proto.query.QueryRequest.req:type_name -> milvus.proto.internal.RetrieveRequest + 126, // 61: milvus.proto.query.QueryRequest.req:type_name -> milvus.proto.internal.RetrieveRequest 1, // 62: milvus.proto.query.QueryRequest.scope:type_name -> milvus.proto.query.DataScope - 112, // 63: milvus.proto.query.SyncReplicaSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 63: milvus.proto.query.SyncReplicaSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 35, // 64: milvus.proto.query.SyncReplicaSegmentsRequest.replica_segments:type_name -> milvus.proto.query.ReplicaSegmentsInfo - 112, // 65: milvus.proto.query.GetLoadInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 113, // 66: milvus.proto.query.GetLoadInfoResponse.status:type_name -> milvus.proto.common.Status - 115, // 67: milvus.proto.query.GetLoadInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema + 113, // 65: milvus.proto.query.GetLoadInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 66: milvus.proto.query.GetLoadInfoResponse.status:type_name -> milvus.proto.common.Status + 116, // 67: milvus.proto.query.GetLoadInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema 4, // 68: milvus.proto.query.GetLoadInfoResponse.load_type:type_name -> milvus.proto.query.LoadType - 112, // 69: milvus.proto.query.HandoffSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 69: milvus.proto.query.HandoffSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 43, // 70: milvus.proto.query.HandoffSegmentsRequest.segmentInfos:type_name -> milvus.proto.query.SegmentInfo - 112, // 71: milvus.proto.query.LoadBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 71: milvus.proto.query.LoadBalanceRequest.base:type_name -> milvus.proto.common.MsgBase 3, // 72: milvus.proto.query.LoadBalanceRequest.balance_reason:type_name -> milvus.proto.query.TriggerCondition 43, // 73: milvus.proto.query.QueryChannelInfo.global_sealed_segments:type_name -> milvus.proto.query.SegmentInfo - 122, // 74: milvus.proto.query.QueryChannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition + 123, // 74: milvus.proto.query.QueryChannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition 2, // 75: milvus.proto.query.PartitionStates.state:type_name -> milvus.proto.query.PartitionState - 126, // 76: milvus.proto.query.SegmentInfo.segment_state:type_name -> milvus.proto.common.SegmentState + 127, // 76: milvus.proto.query.SegmentInfo.segment_state:type_name -> milvus.proto.common.SegmentState 29, // 77: milvus.proto.query.SegmentInfo.index_infos:type_name -> milvus.proto.query.FieldIndexInfo - 123, // 78: milvus.proto.query.SegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel + 124, // 78: milvus.proto.query.SegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel 42, // 79: milvus.proto.query.CollectionInfo.partition_states:type_name -> milvus.proto.query.PartitionStates 4, // 80: milvus.proto.query.CollectionInfo.load_type:type_name -> milvus.proto.query.LoadType - 115, // 81: milvus.proto.query.CollectionInfo.schema:type_name -> milvus.proto.schema.CollectionSchema + 116, // 81: milvus.proto.query.CollectionInfo.schema:type_name -> milvus.proto.schema.CollectionSchema 45, // 82: milvus.proto.query.UnsubscribeChannelInfo.collection_channels:type_name -> milvus.proto.query.UnsubscribeChannels 43, // 83: milvus.proto.query.SegmentChangeInfo.online_segments:type_name -> milvus.proto.query.SegmentInfo 43, // 84: milvus.proto.query.SegmentChangeInfo.offline_segments:type_name -> milvus.proto.query.SegmentInfo - 112, // 85: milvus.proto.query.SealedSegmentsChangeInfo.base:type_name -> milvus.proto.common.MsgBase + 113, // 85: milvus.proto.query.SealedSegmentsChangeInfo.base:type_name -> milvus.proto.common.MsgBase 47, // 86: milvus.proto.query.SealedSegmentsChangeInfo.infos:type_name -> milvus.proto.query.SegmentChangeInfo - 112, // 87: milvus.proto.query.GetDataDistributionRequest.base:type_name -> milvus.proto.common.MsgBase - 100, // 88: milvus.proto.query.GetDataDistributionRequest.checkpoints:type_name -> milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry - 113, // 89: milvus.proto.query.GetDataDistributionResponse.status:type_name -> milvus.proto.common.Status - 53, // 90: milvus.proto.query.GetDataDistributionResponse.segments:type_name -> milvus.proto.query.SegmentVersionInfo - 54, // 91: milvus.proto.query.GetDataDistributionResponse.channels:type_name -> milvus.proto.query.ChannelVersionInfo + 113, // 87: milvus.proto.query.GetDataDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 101, // 88: milvus.proto.query.GetDataDistributionRequest.checkpoints:type_name -> milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry + 114, // 89: milvus.proto.query.GetDataDistributionResponse.status:type_name -> milvus.proto.common.Status + 54, // 90: milvus.proto.query.GetDataDistributionResponse.segments:type_name -> milvus.proto.query.SegmentVersionInfo + 55, // 91: milvus.proto.query.GetDataDistributionResponse.channels:type_name -> milvus.proto.query.ChannelVersionInfo 51, // 92: milvus.proto.query.GetDataDistributionResponse.leader_views:type_name -> milvus.proto.query.LeaderView - 101, // 93: milvus.proto.query.LeaderView.segment_dist:type_name -> milvus.proto.query.LeaderView.SegmentDistEntry - 102, // 94: milvus.proto.query.LeaderView.growing_segments:type_name -> milvus.proto.query.LeaderView.GrowingSegmentsEntry - 103, // 95: milvus.proto.query.LeaderView.partition_stats_versions:type_name -> milvus.proto.query.LeaderView.PartitionStatsVersionsEntry - 104, // 96: milvus.proto.query.SegmentVersionInfo.index_info:type_name -> milvus.proto.query.SegmentVersionInfo.IndexInfoEntry - 123, // 97: milvus.proto.query.SegmentVersionInfo.level:type_name -> milvus.proto.data.SegmentLevel - 5, // 98: milvus.proto.query.CollectionLoadInfo.status:type_name -> milvus.proto.query.LoadStatus - 105, // 99: milvus.proto.query.CollectionLoadInfo.field_indexID:type_name -> milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry - 4, // 100: milvus.proto.query.CollectionLoadInfo.load_type:type_name -> milvus.proto.query.LoadType - 5, // 101: milvus.proto.query.PartitionLoadInfo.status:type_name -> milvus.proto.query.LoadStatus - 106, // 102: milvus.proto.query.PartitionLoadInfo.field_indexID:type_name -> milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry - 107, // 103: milvus.proto.query.Replica.channel_node_infos:type_name -> milvus.proto.query.Replica.ChannelNodeInfosEntry - 6, // 104: milvus.proto.query.SyncAction.type:type_name -> milvus.proto.query.SyncType - 28, // 105: milvus.proto.query.SyncAction.info:type_name -> milvus.proto.query.SegmentLoadInfo - 122, // 106: milvus.proto.query.SyncAction.checkpoint:type_name -> milvus.proto.msg.MsgPosition - 108, // 107: milvus.proto.query.SyncAction.partition_stats_versions:type_name -> milvus.proto.query.SyncAction.PartitionStatsVersionsEntry - 122, // 108: milvus.proto.query.SyncAction.deleteCP:type_name -> milvus.proto.msg.MsgPosition - 112, // 109: milvus.proto.query.SyncDistributionRequest.base:type_name -> milvus.proto.common.MsgBase - 59, // 110: milvus.proto.query.SyncDistributionRequest.actions:type_name -> milvus.proto.query.SyncAction - 115, // 111: milvus.proto.query.SyncDistributionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 25, // 112: milvus.proto.query.SyncDistributionRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo - 117, // 113: milvus.proto.query.SyncDistributionRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo - 127, // 114: milvus.proto.query.ResourceGroup.config:type_name -> milvus.proto.rg.ResourceGroupConfig - 112, // 115: milvus.proto.query.TransferReplicaRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 116: milvus.proto.query.DescribeResourceGroupRequest.base:type_name -> milvus.proto.common.MsgBase - 113, // 117: milvus.proto.query.DescribeResourceGroupResponse.status:type_name -> milvus.proto.common.Status - 65, // 118: milvus.proto.query.DescribeResourceGroupResponse.resource_group:type_name -> milvus.proto.query.ResourceGroupInfo - 109, // 119: milvus.proto.query.ResourceGroupInfo.num_loaded_replica:type_name -> milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry - 110, // 120: milvus.proto.query.ResourceGroupInfo.num_outgoing_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry - 111, // 121: milvus.proto.query.ResourceGroupInfo.num_incoming_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry - 127, // 122: milvus.proto.query.ResourceGroupInfo.config:type_name -> milvus.proto.rg.ResourceGroupConfig - 128, // 123: milvus.proto.query.ResourceGroupInfo.nodes:type_name -> milvus.proto.common.NodeInfo - 112, // 124: milvus.proto.query.DeleteRequest.base:type_name -> milvus.proto.common.MsgBase - 129, // 125: milvus.proto.query.DeleteRequest.primary_keys:type_name -> milvus.proto.schema.IDs - 1, // 126: milvus.proto.query.DeleteRequest.scope:type_name -> milvus.proto.query.DataScope - 112, // 127: milvus.proto.query.DeleteBatchRequest.base:type_name -> milvus.proto.common.MsgBase - 129, // 128: milvus.proto.query.DeleteBatchRequest.primary_keys:type_name -> milvus.proto.schema.IDs - 1, // 129: milvus.proto.query.DeleteBatchRequest.scope:type_name -> milvus.proto.query.DataScope - 113, // 130: milvus.proto.query.DeleteBatchResponse.status:type_name -> milvus.proto.common.Status - 112, // 131: milvus.proto.query.ActivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 132: milvus.proto.query.DeactivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 133: milvus.proto.query.ListCheckersRequest.base:type_name -> milvus.proto.common.MsgBase - 113, // 134: milvus.proto.query.ListCheckersResponse.status:type_name -> milvus.proto.common.Status - 73, // 135: milvus.proto.query.ListCheckersResponse.checkerInfos:type_name -> milvus.proto.query.CheckerInfo - 123, // 136: milvus.proto.query.SegmentTarget.level:type_name -> milvus.proto.data.SegmentLevel - 74, // 137: milvus.proto.query.PartitionTarget.segments:type_name -> milvus.proto.query.SegmentTarget - 75, // 138: milvus.proto.query.ChannelTarget.partition_targets:type_name -> milvus.proto.query.PartitionTarget - 122, // 139: milvus.proto.query.ChannelTarget.seek_position:type_name -> milvus.proto.msg.MsgPosition - 122, // 140: milvus.proto.query.ChannelTarget.delete_checkpoint:type_name -> milvus.proto.msg.MsgPosition - 76, // 141: milvus.proto.query.CollectionTarget.Channel_targets:type_name -> milvus.proto.query.ChannelTarget - 112, // 142: milvus.proto.query.ListQueryNodeRequest.base:type_name -> milvus.proto.common.MsgBase - 113, // 143: milvus.proto.query.ListQueryNodeResponse.status:type_name -> milvus.proto.common.Status - 78, // 144: milvus.proto.query.ListQueryNodeResponse.nodeInfos:type_name -> milvus.proto.query.NodeInfo - 112, // 145: milvus.proto.query.GetQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase - 113, // 146: milvus.proto.query.GetQueryNodeDistributionResponse.status:type_name -> milvus.proto.common.Status - 112, // 147: milvus.proto.query.SuspendBalanceRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 148: milvus.proto.query.ResumeBalanceRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 149: milvus.proto.query.CheckBalanceStatusRequest.base:type_name -> milvus.proto.common.MsgBase - 113, // 150: milvus.proto.query.CheckBalanceStatusResponse.status:type_name -> milvus.proto.common.Status - 112, // 151: milvus.proto.query.SuspendNodeRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 152: milvus.proto.query.ResumeNodeRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 153: milvus.proto.query.TransferSegmentRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 154: milvus.proto.query.TransferChannelRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 155: milvus.proto.query.CheckQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 156: milvus.proto.query.UpdateLoadConfigRequest.base:type_name -> milvus.proto.common.MsgBase - 112, // 157: milvus.proto.query.UpdateSchemaRequest.base:type_name -> milvus.proto.common.MsgBase - 115, // 158: milvus.proto.query.UpdateSchemaRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 127, // 159: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry.value:type_name -> milvus.proto.rg.ResourceGroupConfig - 120, // 160: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SegmentInfo - 130, // 161: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats - 131, // 162: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry.value:type_name -> milvus.proto.data.JsonKeyStats - 122, // 163: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry.value:type_name -> milvus.proto.msg.MsgPosition - 52, // 164: milvus.proto.query.LeaderView.SegmentDistEntry.value:type_name -> milvus.proto.query.SegmentDist - 122, // 165: milvus.proto.query.LeaderView.GrowingSegmentsEntry.value:type_name -> milvus.proto.msg.MsgPosition - 29, // 166: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry.value:type_name -> milvus.proto.query.FieldIndexInfo - 57, // 167: milvus.proto.query.Replica.ChannelNodeInfosEntry.value:type_name -> milvus.proto.query.ChannelNodeInfo - 7, // 168: milvus.proto.query.QueryCoord.ShowLoadCollections:input_type -> milvus.proto.query.ShowCollectionsRequest - 9, // 169: milvus.proto.query.QueryCoord.ShowLoadPartitions:input_type -> milvus.proto.query.ShowPartitionsRequest - 14, // 170: milvus.proto.query.QueryCoord.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest - 15, // 171: milvus.proto.query.QueryCoord.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest - 11, // 172: milvus.proto.query.QueryCoord.LoadCollection:input_type -> milvus.proto.query.LoadCollectionRequest - 12, // 173: milvus.proto.query.QueryCoord.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest - 24, // 174: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:input_type -> milvus.proto.query.SyncNewCreatedPartitionRequest - 16, // 175: milvus.proto.query.QueryCoord.GetPartitionStates:input_type -> milvus.proto.query.GetPartitionStatesRequest - 18, // 176: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest - 39, // 177: milvus.proto.query.QueryCoord.LoadBalance:input_type -> milvus.proto.query.LoadBalanceRequest - 132, // 178: milvus.proto.query.QueryCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest - 133, // 179: milvus.proto.query.QueryCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest - 134, // 180: milvus.proto.query.QueryCoord.GetReplicas:input_type -> milvus.proto.milvus.GetReplicasRequest - 20, // 181: milvus.proto.query.QueryCoord.GetShardLeaders:input_type -> milvus.proto.query.GetShardLeadersRequest - 135, // 182: milvus.proto.query.QueryCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest - 136, // 183: milvus.proto.query.QueryCoord.CreateResourceGroup:input_type -> milvus.proto.milvus.CreateResourceGroupRequest - 22, // 184: milvus.proto.query.QueryCoord.UpdateResourceGroups:input_type -> milvus.proto.query.UpdateResourceGroupsRequest - 137, // 185: milvus.proto.query.QueryCoord.DropResourceGroup:input_type -> milvus.proto.milvus.DropResourceGroupRequest - 138, // 186: milvus.proto.query.QueryCoord.TransferNode:input_type -> milvus.proto.milvus.TransferNodeRequest - 62, // 187: milvus.proto.query.QueryCoord.TransferReplica:input_type -> milvus.proto.query.TransferReplicaRequest - 139, // 188: milvus.proto.query.QueryCoord.ListResourceGroups:input_type -> milvus.proto.milvus.ListResourceGroupsRequest - 63, // 189: milvus.proto.query.QueryCoord.DescribeResourceGroup:input_type -> milvus.proto.query.DescribeResourceGroupRequest - 71, // 190: milvus.proto.query.QueryCoord.ListCheckers:input_type -> milvus.proto.query.ListCheckersRequest - 69, // 191: milvus.proto.query.QueryCoord.ActivateChecker:input_type -> milvus.proto.query.ActivateCheckerRequest - 70, // 192: milvus.proto.query.QueryCoord.DeactivateChecker:input_type -> milvus.proto.query.DeactivateCheckerRequest - 79, // 193: milvus.proto.query.QueryCoord.ListQueryNode:input_type -> milvus.proto.query.ListQueryNodeRequest - 81, // 194: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:input_type -> milvus.proto.query.GetQueryNodeDistributionRequest - 83, // 195: milvus.proto.query.QueryCoord.SuspendBalance:input_type -> milvus.proto.query.SuspendBalanceRequest - 84, // 196: milvus.proto.query.QueryCoord.ResumeBalance:input_type -> milvus.proto.query.ResumeBalanceRequest - 85, // 197: milvus.proto.query.QueryCoord.CheckBalanceStatus:input_type -> milvus.proto.query.CheckBalanceStatusRequest - 87, // 198: milvus.proto.query.QueryCoord.SuspendNode:input_type -> milvus.proto.query.SuspendNodeRequest - 88, // 199: milvus.proto.query.QueryCoord.ResumeNode:input_type -> milvus.proto.query.ResumeNodeRequest - 89, // 200: milvus.proto.query.QueryCoord.TransferSegment:input_type -> milvus.proto.query.TransferSegmentRequest - 90, // 201: milvus.proto.query.QueryCoord.TransferChannel:input_type -> milvus.proto.query.TransferChannelRequest - 91, // 202: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:input_type -> milvus.proto.query.CheckQueryNodeDistributionRequest - 92, // 203: milvus.proto.query.QueryCoord.UpdateLoadConfig:input_type -> milvus.proto.query.UpdateLoadConfigRequest - 140, // 204: milvus.proto.query.QueryNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest - 141, // 205: milvus.proto.query.QueryNode.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest - 142, // 206: milvus.proto.query.QueryNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest - 26, // 207: milvus.proto.query.QueryNode.WatchDmChannels:input_type -> milvus.proto.query.WatchDmChannelsRequest - 27, // 208: milvus.proto.query.QueryNode.UnsubDmChannel:input_type -> milvus.proto.query.UnsubDmChannelRequest - 30, // 209: milvus.proto.query.QueryNode.LoadSegments:input_type -> milvus.proto.query.LoadSegmentsRequest - 12, // 210: milvus.proto.query.QueryNode.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest - 14, // 211: milvus.proto.query.QueryNode.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest - 15, // 212: milvus.proto.query.QueryNode.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest - 31, // 213: milvus.proto.query.QueryNode.ReleaseSegments:input_type -> milvus.proto.query.ReleaseSegmentsRequest - 18, // 214: milvus.proto.query.QueryNode.GetSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest - 34, // 215: milvus.proto.query.QueryNode.SyncReplicaSegments:input_type -> milvus.proto.query.SyncReplicaSegmentsRequest - 13, // 216: milvus.proto.query.QueryNode.GetStatistics:input_type -> milvus.proto.query.GetStatisticsRequest - 32, // 217: milvus.proto.query.QueryNode.Search:input_type -> milvus.proto.query.SearchRequest - 32, // 218: milvus.proto.query.QueryNode.SearchSegments:input_type -> milvus.proto.query.SearchRequest - 33, // 219: milvus.proto.query.QueryNode.Query:input_type -> milvus.proto.query.QueryRequest - 33, // 220: milvus.proto.query.QueryNode.QueryStream:input_type -> milvus.proto.query.QueryRequest - 33, // 221: milvus.proto.query.QueryNode.QuerySegments:input_type -> milvus.proto.query.QueryRequest - 33, // 222: milvus.proto.query.QueryNode.QueryStreamSegments:input_type -> milvus.proto.query.QueryRequest - 132, // 223: milvus.proto.query.QueryNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest - 133, // 224: milvus.proto.query.QueryNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest - 49, // 225: milvus.proto.query.QueryNode.GetDataDistribution:input_type -> milvus.proto.query.GetDataDistributionRequest - 60, // 226: milvus.proto.query.QueryNode.SyncDistribution:input_type -> milvus.proto.query.SyncDistributionRequest - 66, // 227: milvus.proto.query.QueryNode.Delete:input_type -> milvus.proto.query.DeleteRequest - 67, // 228: milvus.proto.query.QueryNode.DeleteBatch:input_type -> milvus.proto.query.DeleteBatchRequest - 93, // 229: milvus.proto.query.QueryNode.UpdateSchema:input_type -> milvus.proto.query.UpdateSchemaRequest - 8, // 230: milvus.proto.query.QueryCoord.ShowLoadCollections:output_type -> milvus.proto.query.ShowCollectionsResponse - 10, // 231: milvus.proto.query.QueryCoord.ShowLoadPartitions:output_type -> milvus.proto.query.ShowPartitionsResponse - 113, // 232: milvus.proto.query.QueryCoord.LoadPartitions:output_type -> milvus.proto.common.Status - 113, // 233: milvus.proto.query.QueryCoord.ReleasePartitions:output_type -> milvus.proto.common.Status - 113, // 234: milvus.proto.query.QueryCoord.LoadCollection:output_type -> milvus.proto.common.Status - 113, // 235: milvus.proto.query.QueryCoord.ReleaseCollection:output_type -> milvus.proto.common.Status - 113, // 236: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:output_type -> milvus.proto.common.Status - 17, // 237: milvus.proto.query.QueryCoord.GetPartitionStates:output_type -> milvus.proto.query.GetPartitionStatesResponse - 19, // 238: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse - 113, // 239: milvus.proto.query.QueryCoord.LoadBalance:output_type -> milvus.proto.common.Status - 143, // 240: milvus.proto.query.QueryCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 144, // 241: milvus.proto.query.QueryCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 145, // 242: milvus.proto.query.QueryCoord.GetReplicas:output_type -> milvus.proto.milvus.GetReplicasResponse - 21, // 243: milvus.proto.query.QueryCoord.GetShardLeaders:output_type -> milvus.proto.query.GetShardLeadersResponse - 146, // 244: milvus.proto.query.QueryCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse - 113, // 245: milvus.proto.query.QueryCoord.CreateResourceGroup:output_type -> milvus.proto.common.Status - 113, // 246: milvus.proto.query.QueryCoord.UpdateResourceGroups:output_type -> milvus.proto.common.Status - 113, // 247: milvus.proto.query.QueryCoord.DropResourceGroup:output_type -> milvus.proto.common.Status - 113, // 248: milvus.proto.query.QueryCoord.TransferNode:output_type -> milvus.proto.common.Status - 113, // 249: milvus.proto.query.QueryCoord.TransferReplica:output_type -> milvus.proto.common.Status - 147, // 250: milvus.proto.query.QueryCoord.ListResourceGroups:output_type -> milvus.proto.milvus.ListResourceGroupsResponse - 64, // 251: milvus.proto.query.QueryCoord.DescribeResourceGroup:output_type -> milvus.proto.query.DescribeResourceGroupResponse - 72, // 252: milvus.proto.query.QueryCoord.ListCheckers:output_type -> milvus.proto.query.ListCheckersResponse - 113, // 253: milvus.proto.query.QueryCoord.ActivateChecker:output_type -> milvus.proto.common.Status - 113, // 254: milvus.proto.query.QueryCoord.DeactivateChecker:output_type -> milvus.proto.common.Status - 80, // 255: milvus.proto.query.QueryCoord.ListQueryNode:output_type -> milvus.proto.query.ListQueryNodeResponse - 82, // 256: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:output_type -> milvus.proto.query.GetQueryNodeDistributionResponse - 113, // 257: milvus.proto.query.QueryCoord.SuspendBalance:output_type -> milvus.proto.common.Status - 113, // 258: milvus.proto.query.QueryCoord.ResumeBalance:output_type -> milvus.proto.common.Status - 86, // 259: milvus.proto.query.QueryCoord.CheckBalanceStatus:output_type -> milvus.proto.query.CheckBalanceStatusResponse - 113, // 260: milvus.proto.query.QueryCoord.SuspendNode:output_type -> milvus.proto.common.Status - 113, // 261: milvus.proto.query.QueryCoord.ResumeNode:output_type -> milvus.proto.common.Status - 113, // 262: milvus.proto.query.QueryCoord.TransferSegment:output_type -> milvus.proto.common.Status - 113, // 263: milvus.proto.query.QueryCoord.TransferChannel:output_type -> milvus.proto.common.Status - 113, // 264: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:output_type -> milvus.proto.common.Status - 113, // 265: milvus.proto.query.QueryCoord.UpdateLoadConfig:output_type -> milvus.proto.common.Status - 148, // 266: milvus.proto.query.QueryNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 149, // 267: milvus.proto.query.QueryNode.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse - 149, // 268: milvus.proto.query.QueryNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse - 113, // 269: milvus.proto.query.QueryNode.WatchDmChannels:output_type -> milvus.proto.common.Status - 113, // 270: milvus.proto.query.QueryNode.UnsubDmChannel:output_type -> milvus.proto.common.Status - 113, // 271: milvus.proto.query.QueryNode.LoadSegments:output_type -> milvus.proto.common.Status - 113, // 272: milvus.proto.query.QueryNode.ReleaseCollection:output_type -> milvus.proto.common.Status - 113, // 273: milvus.proto.query.QueryNode.LoadPartitions:output_type -> milvus.proto.common.Status - 113, // 274: milvus.proto.query.QueryNode.ReleasePartitions:output_type -> milvus.proto.common.Status - 113, // 275: milvus.proto.query.QueryNode.ReleaseSegments:output_type -> milvus.proto.common.Status - 19, // 276: milvus.proto.query.QueryNode.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse - 113, // 277: milvus.proto.query.QueryNode.SyncReplicaSegments:output_type -> milvus.proto.common.Status - 150, // 278: milvus.proto.query.QueryNode.GetStatistics:output_type -> milvus.proto.internal.GetStatisticsResponse - 151, // 279: milvus.proto.query.QueryNode.Search:output_type -> milvus.proto.internal.SearchResults - 151, // 280: milvus.proto.query.QueryNode.SearchSegments:output_type -> milvus.proto.internal.SearchResults - 152, // 281: milvus.proto.query.QueryNode.Query:output_type -> milvus.proto.internal.RetrieveResults - 152, // 282: milvus.proto.query.QueryNode.QueryStream:output_type -> milvus.proto.internal.RetrieveResults - 152, // 283: milvus.proto.query.QueryNode.QuerySegments:output_type -> milvus.proto.internal.RetrieveResults - 152, // 284: milvus.proto.query.QueryNode.QueryStreamSegments:output_type -> milvus.proto.internal.RetrieveResults - 143, // 285: milvus.proto.query.QueryNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 144, // 286: milvus.proto.query.QueryNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 50, // 287: milvus.proto.query.QueryNode.GetDataDistribution:output_type -> milvus.proto.query.GetDataDistributionResponse - 113, // 288: milvus.proto.query.QueryNode.SyncDistribution:output_type -> milvus.proto.common.Status - 113, // 289: milvus.proto.query.QueryNode.Delete:output_type -> milvus.proto.common.Status - 68, // 290: milvus.proto.query.QueryNode.DeleteBatch:output_type -> milvus.proto.query.DeleteBatchResponse - 113, // 291: milvus.proto.query.QueryNode.UpdateSchema:output_type -> milvus.proto.common.Status - 230, // [230:292] is the sub-list for method output_type - 168, // [168:230] is the sub-list for method input_type - 168, // [168:168] is the sub-list for extension type_name - 168, // [168:168] is the sub-list for extension extendee - 0, // [0:168] is the sub-list for field type_name + 102, // 93: milvus.proto.query.LeaderView.segment_dist:type_name -> milvus.proto.query.LeaderView.SegmentDistEntry + 103, // 94: milvus.proto.query.LeaderView.growing_segments:type_name -> milvus.proto.query.LeaderView.GrowingSegmentsEntry + 104, // 95: milvus.proto.query.LeaderView.partition_stats_versions:type_name -> milvus.proto.query.LeaderView.PartitionStatsVersionsEntry + 52, // 96: milvus.proto.query.LeaderView.status:type_name -> milvus.proto.query.LeaderViewStatus + 105, // 97: milvus.proto.query.SegmentVersionInfo.index_info:type_name -> milvus.proto.query.SegmentVersionInfo.IndexInfoEntry + 124, // 98: milvus.proto.query.SegmentVersionInfo.level:type_name -> milvus.proto.data.SegmentLevel + 5, // 99: milvus.proto.query.CollectionLoadInfo.status:type_name -> milvus.proto.query.LoadStatus + 106, // 100: milvus.proto.query.CollectionLoadInfo.field_indexID:type_name -> milvus.proto.query.CollectionLoadInfo.FieldIndexIDEntry + 4, // 101: milvus.proto.query.CollectionLoadInfo.load_type:type_name -> milvus.proto.query.LoadType + 5, // 102: milvus.proto.query.PartitionLoadInfo.status:type_name -> milvus.proto.query.LoadStatus + 107, // 103: milvus.proto.query.PartitionLoadInfo.field_indexID:type_name -> milvus.proto.query.PartitionLoadInfo.FieldIndexIDEntry + 108, // 104: milvus.proto.query.Replica.channel_node_infos:type_name -> milvus.proto.query.Replica.ChannelNodeInfosEntry + 6, // 105: milvus.proto.query.SyncAction.type:type_name -> milvus.proto.query.SyncType + 28, // 106: milvus.proto.query.SyncAction.info:type_name -> milvus.proto.query.SegmentLoadInfo + 123, // 107: milvus.proto.query.SyncAction.checkpoint:type_name -> milvus.proto.msg.MsgPosition + 109, // 108: milvus.proto.query.SyncAction.partition_stats_versions:type_name -> milvus.proto.query.SyncAction.PartitionStatsVersionsEntry + 123, // 109: milvus.proto.query.SyncAction.deleteCP:type_name -> milvus.proto.msg.MsgPosition + 113, // 110: milvus.proto.query.SyncDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 60, // 111: milvus.proto.query.SyncDistributionRequest.actions:type_name -> milvus.proto.query.SyncAction + 116, // 112: milvus.proto.query.SyncDistributionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 25, // 113: milvus.proto.query.SyncDistributionRequest.load_meta:type_name -> milvus.proto.query.LoadMetaInfo + 118, // 114: milvus.proto.query.SyncDistributionRequest.index_info_list:type_name -> milvus.proto.index.IndexInfo + 128, // 115: milvus.proto.query.ResourceGroup.config:type_name -> milvus.proto.rg.ResourceGroupConfig + 113, // 116: milvus.proto.query.TransferReplicaRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 117: milvus.proto.query.DescribeResourceGroupRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 118: milvus.proto.query.DescribeResourceGroupResponse.status:type_name -> milvus.proto.common.Status + 66, // 119: milvus.proto.query.DescribeResourceGroupResponse.resource_group:type_name -> milvus.proto.query.ResourceGroupInfo + 110, // 120: milvus.proto.query.ResourceGroupInfo.num_loaded_replica:type_name -> milvus.proto.query.ResourceGroupInfo.NumLoadedReplicaEntry + 111, // 121: milvus.proto.query.ResourceGroupInfo.num_outgoing_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumOutgoingNodeEntry + 112, // 122: milvus.proto.query.ResourceGroupInfo.num_incoming_node:type_name -> milvus.proto.query.ResourceGroupInfo.NumIncomingNodeEntry + 128, // 123: milvus.proto.query.ResourceGroupInfo.config:type_name -> milvus.proto.rg.ResourceGroupConfig + 129, // 124: milvus.proto.query.ResourceGroupInfo.nodes:type_name -> milvus.proto.common.NodeInfo + 113, // 125: milvus.proto.query.DeleteRequest.base:type_name -> milvus.proto.common.MsgBase + 130, // 126: milvus.proto.query.DeleteRequest.primary_keys:type_name -> milvus.proto.schema.IDs + 1, // 127: milvus.proto.query.DeleteRequest.scope:type_name -> milvus.proto.query.DataScope + 113, // 128: milvus.proto.query.DeleteBatchRequest.base:type_name -> milvus.proto.common.MsgBase + 130, // 129: milvus.proto.query.DeleteBatchRequest.primary_keys:type_name -> milvus.proto.schema.IDs + 1, // 130: milvus.proto.query.DeleteBatchRequest.scope:type_name -> milvus.proto.query.DataScope + 114, // 131: milvus.proto.query.DeleteBatchResponse.status:type_name -> milvus.proto.common.Status + 113, // 132: milvus.proto.query.ActivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 133: milvus.proto.query.DeactivateCheckerRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 134: milvus.proto.query.ListCheckersRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 135: milvus.proto.query.ListCheckersResponse.status:type_name -> milvus.proto.common.Status + 74, // 136: milvus.proto.query.ListCheckersResponse.checkerInfos:type_name -> milvus.proto.query.CheckerInfo + 124, // 137: milvus.proto.query.SegmentTarget.level:type_name -> milvus.proto.data.SegmentLevel + 75, // 138: milvus.proto.query.PartitionTarget.segments:type_name -> milvus.proto.query.SegmentTarget + 76, // 139: milvus.proto.query.ChannelTarget.partition_targets:type_name -> milvus.proto.query.PartitionTarget + 123, // 140: milvus.proto.query.ChannelTarget.seek_position:type_name -> milvus.proto.msg.MsgPosition + 123, // 141: milvus.proto.query.ChannelTarget.delete_checkpoint:type_name -> milvus.proto.msg.MsgPosition + 77, // 142: milvus.proto.query.CollectionTarget.Channel_targets:type_name -> milvus.proto.query.ChannelTarget + 113, // 143: milvus.proto.query.ListQueryNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 144: milvus.proto.query.ListQueryNodeResponse.status:type_name -> milvus.proto.common.Status + 79, // 145: milvus.proto.query.ListQueryNodeResponse.nodeInfos:type_name -> milvus.proto.query.NodeInfo + 113, // 146: milvus.proto.query.GetQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 147: milvus.proto.query.GetQueryNodeDistributionResponse.status:type_name -> milvus.proto.common.Status + 113, // 148: milvus.proto.query.SuspendBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 149: milvus.proto.query.ResumeBalanceRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 150: milvus.proto.query.CheckBalanceStatusRequest.base:type_name -> milvus.proto.common.MsgBase + 114, // 151: milvus.proto.query.CheckBalanceStatusResponse.status:type_name -> milvus.proto.common.Status + 113, // 152: milvus.proto.query.SuspendNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 153: milvus.proto.query.ResumeNodeRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 154: milvus.proto.query.TransferSegmentRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 155: milvus.proto.query.TransferChannelRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 156: milvus.proto.query.CheckQueryNodeDistributionRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 157: milvus.proto.query.UpdateLoadConfigRequest.base:type_name -> milvus.proto.common.MsgBase + 113, // 158: milvus.proto.query.UpdateSchemaRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 159: milvus.proto.query.UpdateSchemaRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 128, // 160: milvus.proto.query.UpdateResourceGroupsRequest.ResourceGroupsEntry.value:type_name -> milvus.proto.rg.ResourceGroupConfig + 121, // 161: milvus.proto.query.WatchDmChannelsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SegmentInfo + 131, // 162: milvus.proto.query.SegmentLoadInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 132, // 163: milvus.proto.query.SegmentLoadInfo.JsonKeyStatsLogsEntry.value:type_name -> milvus.proto.data.JsonKeyStats + 123, // 164: milvus.proto.query.GetDataDistributionRequest.CheckpointsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 53, // 165: milvus.proto.query.LeaderView.SegmentDistEntry.value:type_name -> milvus.proto.query.SegmentDist + 123, // 166: milvus.proto.query.LeaderView.GrowingSegmentsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 29, // 167: milvus.proto.query.SegmentVersionInfo.IndexInfoEntry.value:type_name -> milvus.proto.query.FieldIndexInfo + 58, // 168: milvus.proto.query.Replica.ChannelNodeInfosEntry.value:type_name -> milvus.proto.query.ChannelNodeInfo + 7, // 169: milvus.proto.query.QueryCoord.ShowLoadCollections:input_type -> milvus.proto.query.ShowCollectionsRequest + 9, // 170: milvus.proto.query.QueryCoord.ShowLoadPartitions:input_type -> milvus.proto.query.ShowPartitionsRequest + 14, // 171: milvus.proto.query.QueryCoord.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest + 15, // 172: milvus.proto.query.QueryCoord.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest + 11, // 173: milvus.proto.query.QueryCoord.LoadCollection:input_type -> milvus.proto.query.LoadCollectionRequest + 12, // 174: milvus.proto.query.QueryCoord.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest + 24, // 175: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:input_type -> milvus.proto.query.SyncNewCreatedPartitionRequest + 16, // 176: milvus.proto.query.QueryCoord.GetPartitionStates:input_type -> milvus.proto.query.GetPartitionStatesRequest + 18, // 177: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest + 39, // 178: milvus.proto.query.QueryCoord.LoadBalance:input_type -> milvus.proto.query.LoadBalanceRequest + 133, // 179: milvus.proto.query.QueryCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 134, // 180: milvus.proto.query.QueryCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 135, // 181: milvus.proto.query.QueryCoord.GetReplicas:input_type -> milvus.proto.milvus.GetReplicasRequest + 20, // 182: milvus.proto.query.QueryCoord.GetShardLeaders:input_type -> milvus.proto.query.GetShardLeadersRequest + 136, // 183: milvus.proto.query.QueryCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest + 137, // 184: milvus.proto.query.QueryCoord.CreateResourceGroup:input_type -> milvus.proto.milvus.CreateResourceGroupRequest + 22, // 185: milvus.proto.query.QueryCoord.UpdateResourceGroups:input_type -> milvus.proto.query.UpdateResourceGroupsRequest + 138, // 186: milvus.proto.query.QueryCoord.DropResourceGroup:input_type -> milvus.proto.milvus.DropResourceGroupRequest + 139, // 187: milvus.proto.query.QueryCoord.TransferNode:input_type -> milvus.proto.milvus.TransferNodeRequest + 63, // 188: milvus.proto.query.QueryCoord.TransferReplica:input_type -> milvus.proto.query.TransferReplicaRequest + 140, // 189: milvus.proto.query.QueryCoord.ListResourceGroups:input_type -> milvus.proto.milvus.ListResourceGroupsRequest + 64, // 190: milvus.proto.query.QueryCoord.DescribeResourceGroup:input_type -> milvus.proto.query.DescribeResourceGroupRequest + 72, // 191: milvus.proto.query.QueryCoord.ListCheckers:input_type -> milvus.proto.query.ListCheckersRequest + 70, // 192: milvus.proto.query.QueryCoord.ActivateChecker:input_type -> milvus.proto.query.ActivateCheckerRequest + 71, // 193: milvus.proto.query.QueryCoord.DeactivateChecker:input_type -> milvus.proto.query.DeactivateCheckerRequest + 80, // 194: milvus.proto.query.QueryCoord.ListQueryNode:input_type -> milvus.proto.query.ListQueryNodeRequest + 82, // 195: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:input_type -> milvus.proto.query.GetQueryNodeDistributionRequest + 84, // 196: milvus.proto.query.QueryCoord.SuspendBalance:input_type -> milvus.proto.query.SuspendBalanceRequest + 85, // 197: milvus.proto.query.QueryCoord.ResumeBalance:input_type -> milvus.proto.query.ResumeBalanceRequest + 86, // 198: milvus.proto.query.QueryCoord.CheckBalanceStatus:input_type -> milvus.proto.query.CheckBalanceStatusRequest + 88, // 199: milvus.proto.query.QueryCoord.SuspendNode:input_type -> milvus.proto.query.SuspendNodeRequest + 89, // 200: milvus.proto.query.QueryCoord.ResumeNode:input_type -> milvus.proto.query.ResumeNodeRequest + 90, // 201: milvus.proto.query.QueryCoord.TransferSegment:input_type -> milvus.proto.query.TransferSegmentRequest + 91, // 202: milvus.proto.query.QueryCoord.TransferChannel:input_type -> milvus.proto.query.TransferChannelRequest + 92, // 203: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:input_type -> milvus.proto.query.CheckQueryNodeDistributionRequest + 93, // 204: milvus.proto.query.QueryCoord.UpdateLoadConfig:input_type -> milvus.proto.query.UpdateLoadConfigRequest + 141, // 205: milvus.proto.query.QueryNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 142, // 206: milvus.proto.query.QueryNode.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest + 143, // 207: milvus.proto.query.QueryNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 26, // 208: milvus.proto.query.QueryNode.WatchDmChannels:input_type -> milvus.proto.query.WatchDmChannelsRequest + 27, // 209: milvus.proto.query.QueryNode.UnsubDmChannel:input_type -> milvus.proto.query.UnsubDmChannelRequest + 30, // 210: milvus.proto.query.QueryNode.LoadSegments:input_type -> milvus.proto.query.LoadSegmentsRequest + 12, // 211: milvus.proto.query.QueryNode.ReleaseCollection:input_type -> milvus.proto.query.ReleaseCollectionRequest + 14, // 212: milvus.proto.query.QueryNode.LoadPartitions:input_type -> milvus.proto.query.LoadPartitionsRequest + 15, // 213: milvus.proto.query.QueryNode.ReleasePartitions:input_type -> milvus.proto.query.ReleasePartitionsRequest + 31, // 214: milvus.proto.query.QueryNode.ReleaseSegments:input_type -> milvus.proto.query.ReleaseSegmentsRequest + 18, // 215: milvus.proto.query.QueryNode.GetSegmentInfo:input_type -> milvus.proto.query.GetSegmentInfoRequest + 34, // 216: milvus.proto.query.QueryNode.SyncReplicaSegments:input_type -> milvus.proto.query.SyncReplicaSegmentsRequest + 13, // 217: milvus.proto.query.QueryNode.GetStatistics:input_type -> milvus.proto.query.GetStatisticsRequest + 32, // 218: milvus.proto.query.QueryNode.Search:input_type -> milvus.proto.query.SearchRequest + 32, // 219: milvus.proto.query.QueryNode.SearchSegments:input_type -> milvus.proto.query.SearchRequest + 33, // 220: milvus.proto.query.QueryNode.Query:input_type -> milvus.proto.query.QueryRequest + 33, // 221: milvus.proto.query.QueryNode.QueryStream:input_type -> milvus.proto.query.QueryRequest + 33, // 222: milvus.proto.query.QueryNode.QuerySegments:input_type -> milvus.proto.query.QueryRequest + 33, // 223: milvus.proto.query.QueryNode.QueryStreamSegments:input_type -> milvus.proto.query.QueryRequest + 133, // 224: milvus.proto.query.QueryNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 134, // 225: milvus.proto.query.QueryNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 49, // 226: milvus.proto.query.QueryNode.GetDataDistribution:input_type -> milvus.proto.query.GetDataDistributionRequest + 61, // 227: milvus.proto.query.QueryNode.SyncDistribution:input_type -> milvus.proto.query.SyncDistributionRequest + 67, // 228: milvus.proto.query.QueryNode.Delete:input_type -> milvus.proto.query.DeleteRequest + 68, // 229: milvus.proto.query.QueryNode.DeleteBatch:input_type -> milvus.proto.query.DeleteBatchRequest + 94, // 230: milvus.proto.query.QueryNode.UpdateSchema:input_type -> milvus.proto.query.UpdateSchemaRequest + 8, // 231: milvus.proto.query.QueryCoord.ShowLoadCollections:output_type -> milvus.proto.query.ShowCollectionsResponse + 10, // 232: milvus.proto.query.QueryCoord.ShowLoadPartitions:output_type -> milvus.proto.query.ShowPartitionsResponse + 114, // 233: milvus.proto.query.QueryCoord.LoadPartitions:output_type -> milvus.proto.common.Status + 114, // 234: milvus.proto.query.QueryCoord.ReleasePartitions:output_type -> milvus.proto.common.Status + 114, // 235: milvus.proto.query.QueryCoord.LoadCollection:output_type -> milvus.proto.common.Status + 114, // 236: milvus.proto.query.QueryCoord.ReleaseCollection:output_type -> milvus.proto.common.Status + 114, // 237: milvus.proto.query.QueryCoord.SyncNewCreatedPartition:output_type -> milvus.proto.common.Status + 17, // 238: milvus.proto.query.QueryCoord.GetPartitionStates:output_type -> milvus.proto.query.GetPartitionStatesResponse + 19, // 239: milvus.proto.query.QueryCoord.GetLoadSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse + 114, // 240: milvus.proto.query.QueryCoord.LoadBalance:output_type -> milvus.proto.common.Status + 144, // 241: milvus.proto.query.QueryCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 145, // 242: milvus.proto.query.QueryCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 146, // 243: milvus.proto.query.QueryCoord.GetReplicas:output_type -> milvus.proto.milvus.GetReplicasResponse + 21, // 244: milvus.proto.query.QueryCoord.GetShardLeaders:output_type -> milvus.proto.query.GetShardLeadersResponse + 147, // 245: milvus.proto.query.QueryCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 114, // 246: milvus.proto.query.QueryCoord.CreateResourceGroup:output_type -> milvus.proto.common.Status + 114, // 247: milvus.proto.query.QueryCoord.UpdateResourceGroups:output_type -> milvus.proto.common.Status + 114, // 248: milvus.proto.query.QueryCoord.DropResourceGroup:output_type -> milvus.proto.common.Status + 114, // 249: milvus.proto.query.QueryCoord.TransferNode:output_type -> milvus.proto.common.Status + 114, // 250: milvus.proto.query.QueryCoord.TransferReplica:output_type -> milvus.proto.common.Status + 148, // 251: milvus.proto.query.QueryCoord.ListResourceGroups:output_type -> milvus.proto.milvus.ListResourceGroupsResponse + 65, // 252: milvus.proto.query.QueryCoord.DescribeResourceGroup:output_type -> milvus.proto.query.DescribeResourceGroupResponse + 73, // 253: milvus.proto.query.QueryCoord.ListCheckers:output_type -> milvus.proto.query.ListCheckersResponse + 114, // 254: milvus.proto.query.QueryCoord.ActivateChecker:output_type -> milvus.proto.common.Status + 114, // 255: milvus.proto.query.QueryCoord.DeactivateChecker:output_type -> milvus.proto.common.Status + 81, // 256: milvus.proto.query.QueryCoord.ListQueryNode:output_type -> milvus.proto.query.ListQueryNodeResponse + 83, // 257: milvus.proto.query.QueryCoord.GetQueryNodeDistribution:output_type -> milvus.proto.query.GetQueryNodeDistributionResponse + 114, // 258: milvus.proto.query.QueryCoord.SuspendBalance:output_type -> milvus.proto.common.Status + 114, // 259: milvus.proto.query.QueryCoord.ResumeBalance:output_type -> milvus.proto.common.Status + 87, // 260: milvus.proto.query.QueryCoord.CheckBalanceStatus:output_type -> milvus.proto.query.CheckBalanceStatusResponse + 114, // 261: milvus.proto.query.QueryCoord.SuspendNode:output_type -> milvus.proto.common.Status + 114, // 262: milvus.proto.query.QueryCoord.ResumeNode:output_type -> milvus.proto.common.Status + 114, // 263: milvus.proto.query.QueryCoord.TransferSegment:output_type -> milvus.proto.common.Status + 114, // 264: milvus.proto.query.QueryCoord.TransferChannel:output_type -> milvus.proto.common.Status + 114, // 265: milvus.proto.query.QueryCoord.CheckQueryNodeDistribution:output_type -> milvus.proto.common.Status + 114, // 266: milvus.proto.query.QueryCoord.UpdateLoadConfig:output_type -> milvus.proto.common.Status + 149, // 267: milvus.proto.query.QueryNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 150, // 268: milvus.proto.query.QueryNode.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse + 150, // 269: milvus.proto.query.QueryNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 114, // 270: milvus.proto.query.QueryNode.WatchDmChannels:output_type -> milvus.proto.common.Status + 114, // 271: milvus.proto.query.QueryNode.UnsubDmChannel:output_type -> milvus.proto.common.Status + 114, // 272: milvus.proto.query.QueryNode.LoadSegments:output_type -> milvus.proto.common.Status + 114, // 273: milvus.proto.query.QueryNode.ReleaseCollection:output_type -> milvus.proto.common.Status + 114, // 274: milvus.proto.query.QueryNode.LoadPartitions:output_type -> milvus.proto.common.Status + 114, // 275: milvus.proto.query.QueryNode.ReleasePartitions:output_type -> milvus.proto.common.Status + 114, // 276: milvus.proto.query.QueryNode.ReleaseSegments:output_type -> milvus.proto.common.Status + 19, // 277: milvus.proto.query.QueryNode.GetSegmentInfo:output_type -> milvus.proto.query.GetSegmentInfoResponse + 114, // 278: milvus.proto.query.QueryNode.SyncReplicaSegments:output_type -> milvus.proto.common.Status + 151, // 279: milvus.proto.query.QueryNode.GetStatistics:output_type -> milvus.proto.internal.GetStatisticsResponse + 152, // 280: milvus.proto.query.QueryNode.Search:output_type -> milvus.proto.internal.SearchResults + 152, // 281: milvus.proto.query.QueryNode.SearchSegments:output_type -> milvus.proto.internal.SearchResults + 153, // 282: milvus.proto.query.QueryNode.Query:output_type -> milvus.proto.internal.RetrieveResults + 153, // 283: milvus.proto.query.QueryNode.QueryStream:output_type -> milvus.proto.internal.RetrieveResults + 153, // 284: milvus.proto.query.QueryNode.QuerySegments:output_type -> milvus.proto.internal.RetrieveResults + 153, // 285: milvus.proto.query.QueryNode.QueryStreamSegments:output_type -> milvus.proto.internal.RetrieveResults + 144, // 286: milvus.proto.query.QueryNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 145, // 287: milvus.proto.query.QueryNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 50, // 288: milvus.proto.query.QueryNode.GetDataDistribution:output_type -> milvus.proto.query.GetDataDistributionResponse + 114, // 289: milvus.proto.query.QueryNode.SyncDistribution:output_type -> milvus.proto.common.Status + 114, // 290: milvus.proto.query.QueryNode.Delete:output_type -> milvus.proto.common.Status + 69, // 291: milvus.proto.query.QueryNode.DeleteBatch:output_type -> milvus.proto.query.DeleteBatchResponse + 114, // 292: milvus.proto.query.QueryNode.UpdateSchema:output_type -> milvus.proto.common.Status + 231, // [231:293] is the sub-list for method output_type + 169, // [169:231] is the sub-list for method input_type + 169, // [169:169] is the sub-list for extension type_name + 169, // [169:169] is the sub-list for extension extendee + 0, // [0:169] is the sub-list for field type_name } func init() { file_query_coord_proto_init() } @@ -10195,7 +10259,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SegmentDist); i { + switch v := v.(*LeaderViewStatus); i { case 0: return &v.state case 1: @@ -10207,7 +10271,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SegmentVersionInfo); i { + switch v := v.(*SegmentDist); i { case 0: return &v.state case 1: @@ -10219,7 +10283,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ChannelVersionInfo); i { + switch v := v.(*SegmentVersionInfo); i { case 0: return &v.state case 1: @@ -10231,7 +10295,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CollectionLoadInfo); i { + switch v := v.(*ChannelVersionInfo); i { case 0: return &v.state case 1: @@ -10243,7 +10307,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PartitionLoadInfo); i { + switch v := v.(*CollectionLoadInfo); i { case 0: return &v.state case 1: @@ -10255,7 +10319,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ChannelNodeInfo); i { + switch v := v.(*PartitionLoadInfo); i { case 0: return &v.state case 1: @@ -10267,7 +10331,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Replica); i { + switch v := v.(*ChannelNodeInfo); i { case 0: return &v.state case 1: @@ -10279,7 +10343,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SyncAction); i { + switch v := v.(*Replica); i { case 0: return &v.state case 1: @@ -10291,7 +10355,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SyncDistributionRequest); i { + switch v := v.(*SyncAction); i { case 0: return &v.state case 1: @@ -10303,7 +10367,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResourceGroup); i { + switch v := v.(*SyncDistributionRequest); i { case 0: return &v.state case 1: @@ -10315,7 +10379,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TransferReplicaRequest); i { + switch v := v.(*ResourceGroup); i { case 0: return &v.state case 1: @@ -10327,7 +10391,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DescribeResourceGroupRequest); i { + switch v := v.(*TransferReplicaRequest); i { case 0: return &v.state case 1: @@ -10339,7 +10403,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DescribeResourceGroupResponse); i { + switch v := v.(*DescribeResourceGroupRequest); i { case 0: return &v.state case 1: @@ -10351,7 +10415,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResourceGroupInfo); i { + switch v := v.(*DescribeResourceGroupResponse); i { case 0: return &v.state case 1: @@ -10363,7 +10427,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeleteRequest); i { + switch v := v.(*ResourceGroupInfo); i { case 0: return &v.state case 1: @@ -10375,7 +10439,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeleteBatchRequest); i { + switch v := v.(*DeleteRequest); i { case 0: return &v.state case 1: @@ -10387,7 +10451,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[61].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeleteBatchResponse); i { + switch v := v.(*DeleteBatchRequest); i { case 0: return &v.state case 1: @@ -10399,7 +10463,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ActivateCheckerRequest); i { + switch v := v.(*DeleteBatchResponse); i { case 0: return &v.state case 1: @@ -10411,7 +10475,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*DeactivateCheckerRequest); i { + switch v := v.(*ActivateCheckerRequest); i { case 0: return &v.state case 1: @@ -10423,7 +10487,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[64].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListCheckersRequest); i { + switch v := v.(*DeactivateCheckerRequest); i { case 0: return &v.state case 1: @@ -10435,7 +10499,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[65].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListCheckersResponse); i { + switch v := v.(*ListCheckersRequest); i { case 0: return &v.state case 1: @@ -10447,7 +10511,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[66].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckerInfo); i { + switch v := v.(*ListCheckersResponse); i { case 0: return &v.state case 1: @@ -10459,7 +10523,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[67].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SegmentTarget); i { + switch v := v.(*CheckerInfo); i { case 0: return &v.state case 1: @@ -10471,7 +10535,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[68].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PartitionTarget); i { + switch v := v.(*SegmentTarget); i { case 0: return &v.state case 1: @@ -10483,7 +10547,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ChannelTarget); i { + switch v := v.(*PartitionTarget); i { case 0: return &v.state case 1: @@ -10495,7 +10559,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CollectionTarget); i { + switch v := v.(*ChannelTarget); i { case 0: return &v.state case 1: @@ -10507,7 +10571,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*NodeInfo); i { + switch v := v.(*CollectionTarget); i { case 0: return &v.state case 1: @@ -10519,7 +10583,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListQueryNodeRequest); i { + switch v := v.(*NodeInfo); i { case 0: return &v.state case 1: @@ -10531,7 +10595,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ListQueryNodeResponse); i { + switch v := v.(*ListQueryNodeRequest); i { case 0: return &v.state case 1: @@ -10543,7 +10607,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[74].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetQueryNodeDistributionRequest); i { + switch v := v.(*ListQueryNodeResponse); i { case 0: return &v.state case 1: @@ -10555,7 +10619,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GetQueryNodeDistributionResponse); i { + switch v := v.(*GetQueryNodeDistributionRequest); i { case 0: return &v.state case 1: @@ -10567,7 +10631,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SuspendBalanceRequest); i { + switch v := v.(*GetQueryNodeDistributionResponse); i { case 0: return &v.state case 1: @@ -10579,7 +10643,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResumeBalanceRequest); i { + switch v := v.(*SuspendBalanceRequest); i { case 0: return &v.state case 1: @@ -10591,7 +10655,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckBalanceStatusRequest); i { + switch v := v.(*ResumeBalanceRequest); i { case 0: return &v.state case 1: @@ -10603,7 +10667,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckBalanceStatusResponse); i { + switch v := v.(*CheckBalanceStatusRequest); i { case 0: return &v.state case 1: @@ -10615,7 +10679,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SuspendNodeRequest); i { + switch v := v.(*CheckBalanceStatusResponse); i { case 0: return &v.state case 1: @@ -10627,7 +10691,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResumeNodeRequest); i { + switch v := v.(*SuspendNodeRequest); i { case 0: return &v.state case 1: @@ -10639,7 +10703,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TransferSegmentRequest); i { + switch v := v.(*ResumeNodeRequest); i { case 0: return &v.state case 1: @@ -10651,7 +10715,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TransferChannelRequest); i { + switch v := v.(*TransferSegmentRequest); i { case 0: return &v.state case 1: @@ -10663,7 +10727,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*CheckQueryNodeDistributionRequest); i { + switch v := v.(*TransferChannelRequest); i { case 0: return &v.state case 1: @@ -10675,7 +10739,7 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*UpdateLoadConfigRequest); i { + switch v := v.(*CheckQueryNodeDistributionRequest); i { case 0: return &v.state case 1: @@ -10687,6 +10751,18 @@ func file_query_coord_proto_init() { } } file_query_coord_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*UpdateLoadConfigRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_query_coord_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*UpdateSchemaRequest); i { case 0: return &v.state @@ -10705,7 +10781,7 @@ func file_query_coord_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_query_coord_proto_rawDesc, NumEnums: 7, - NumMessages: 105, + NumMessages: 106, NumExtensions: 0, NumServices: 2, },