From 047254665d8560af17ea35664092abefa89f9506 Mon Sep 17 00:00:00 2001 From: SimFG Date: Sun, 16 Feb 2025 00:08:13 +0800 Subject: [PATCH] feat: support to replicate import msg (#39171) - issue: #39849 --------- Signed-off-by: SimFG Signed-off-by: chyezh Co-authored-by: chyezh --- .github/workflows/main.yaml | 2 +- go.mod | 6 +- go.sum | 4 +- internal/.mockery.yaml | 7 +- internal/datacoord/compaction.go | 44 +- internal/datacoord/compaction_policy_l0.go | 42 +- .../datacoord/compaction_policy_l0_test.go | 24 + internal/datacoord/compaction_test.go | 58 + internal/datacoord/compaction_trigger_test.go | 5 + internal/datacoord/compaction_trigger_v2.go | 163 +- .../datacoord/compaction_trigger_v2_test.go | 85 +- internal/datacoord/import_checker.go | 103 +- internal/datacoord/import_checker_test.go | 217 +- internal/datacoord/import_job.go | 2 + internal/datacoord/import_meta.go | 8 + internal/datacoord/import_meta_test.go | 56 +- internal/datacoord/import_task.go | 17 + internal/datacoord/import_util.go | 22 +- internal/datacoord/import_util_test.go | 127 + .../datacoord/mock_compaction_plan_context.go | 59 + internal/datacoord/mock_trigger_manager.go | 98 + internal/datacoord/server.go | 12 +- internal/datacoord/services.go | 48 +- internal/datacoord/services_test.go | 26 +- internal/datanode/data_node.go | 2 + .../msghandlerimpl/msg_handler_impl.go | 96 + .../msghandlerimpl/msg_handler_impl_test.go | 61 + .../distributed/streaming/test_streaming.go | 7 + internal/flushcommon/broker/broker.go | 2 + internal/flushcommon/broker/datacoord.go | 11 + internal/flushcommon/broker/datacoord_test.go | 23 + internal/flushcommon/broker/mock_broker.go | 61 + .../pipeline/flow_graph_dd_node.go | 20 +- .../pipeline/flow_graph_dd_node_test.go | 4 +- .../util}/msg_handler.go | 13 +- internal/flushcommon/util/util.go | 3 +- .../mock_streaming/mock_Broadcast.go | 239 ++ .../flushcommon/mock_util/mock_MsgHandler.go | 230 ++ .../mock_flusher/mock_FlushMsgHandler.go | 180 -- internal/proxy/impl.go | 173 +- internal/proxy/impl_test.go | 46 +- internal/proxy/proxy_test.go | 64 +- internal/proxy/task_import.go | 273 ++ .../server/broadcaster/broadcast_manager.go | 32 +- .../flusher/flusherimpl/msg_handler_impl.go | 49 + pkg/go.mod | 2 +- pkg/go.sum | 4 +- pkg/metrics/metrics.go | 11 +- pkg/mq/msgstream/msg_for_import.go | 78 + pkg/mq/msgstream/msg_for_import_test.go | 64 + pkg/mq/msgstream/msg_for_index_test.go | 2 +- pkg/mq/msgstream/msg_for_replicate_test.go | 64 + pkg/mq/msgstream/unmarshal.go | 2 + pkg/proto/data_coord.proto | 10 + pkg/proto/datapb/data_coord.pb.go | 2384 +++++++++-------- pkg/proto/internal.proto | 4 +- pkg/proto/internalpb/internal.pb.go | 312 ++- pkg/proto/messages.proto | 3 + pkg/proto/messagespb/messages.pb.go | 362 +-- pkg/streaming/util/message/adaptor/message.go | 11 + .../util/message/adaptor/message_type.go | 1 + pkg/streaming/util/message/builder.go | 1 + pkg/streaming/util/message/message_type.go | 2 + .../util/message/specialized_message.go | 8 +- scripts/run_intergration_test.sh | 2 + .../compaction/l0_compaction_test.go | 2 +- .../integration/import/dynamic_field_test.go | 2 +- tests/integration/import/import_test.go | 2 +- .../integration/import/partition_key_test.go | 2 +- 69 files changed, 4260 insertions(+), 1899 deletions(-) create mode 100644 internal/datanode/msghandlerimpl/msg_handler_impl.go create mode 100644 internal/datanode/msghandlerimpl/msg_handler_impl_test.go rename internal/{streamingnode/server/flusher => flushcommon/util}/msg_handler.go (75%) create mode 100644 internal/mocks/distributed/mock_streaming/mock_Broadcast.go create mode 100644 internal/mocks/flushcommon/mock_util/mock_MsgHandler.go delete mode 100644 internal/mocks/streamingnode/server/mock_flusher/mock_FlushMsgHandler.go create mode 100644 internal/proxy/task_import.go create mode 100644 pkg/mq/msgstream/msg_for_import.go create mode 100644 pkg/mq/msgstream/msg_for_import_test.go create mode 100644 pkg/mq/msgstream/msg_for_replicate_test.go diff --git a/.github/workflows/main.yaml b/.github/workflows/main.yaml index fa97b75bf5..5ef4a3da52 100644 --- a/.github/workflows/main.yaml +++ b/.github/workflows/main.yaml @@ -221,7 +221,7 @@ jobs: name: Integration Test needs: Build runs-on: ubuntu-latest - timeout-minutes: 120 + timeout-minutes: 150 steps: - name: Maximize build space uses: easimon/maximize-build-space@master diff --git a/go.mod b/go.mod index d439c52811..762afb11fd 100644 --- a/go.mod +++ b/go.mod @@ -18,12 +18,12 @@ require ( github.com/gin-gonic/gin v1.9.1 github.com/go-playground/validator/v10 v10.14.0 github.com/gofrs/flock v0.8.1 - github.com/golang/protobuf v1.5.4 + github.com/golang/protobuf v1.5.4 // indirect github.com/google/btree v1.1.2 github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 github.com/klauspost/compress v1.17.9 github.com/mgutz/ansi v0.0.0-20200706080929-d51e80ef957d - github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559 + github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b github.com/minio/minio-go/v7 v7.0.73 github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81 github.com/prometheus/client_golang v1.14.0 @@ -72,6 +72,7 @@ require ( github.com/greatroar/blobloom v0.0.0-00010101000000-000000000000 github.com/hashicorp/golang-lru/v2 v2.0.7 github.com/jolestar/go-commons-pool/v2 v2.1.2 + github.com/magiconair/properties v1.8.5 github.com/milvus-io/milvus/pkg v0.0.2-0.20241126032235-cb6542339e84 github.com/pkg/errors v0.9.1 github.com/remeh/sizedwaitgroup v1.0.0 @@ -181,7 +182,6 @@ require ( github.com/leodido/go-urn v1.2.4 // indirect github.com/linkedin/goavro/v2 v2.11.1 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect - github.com/magiconair/properties v1.8.5 // indirect github.com/mattn/go-colorable v0.1.11 // indirect github.com/mattn/go-isatty v0.0.19 // indirect github.com/matttproud/golang_protobuf_extensions v1.0.4 // indirect diff --git a/go.sum b/go.sum index 352cefd651..2816f3b51a 100644 --- a/go.sum +++ b/go.sum @@ -660,8 +660,8 @@ github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119 h1:9VXijWu github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119/go.mod h1:DvXTE/K/RtHehxU8/GtDs4vFtfw64jJ3PaCnFri8CRg= github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b h1:TfeY0NxYxZzUfIfYe5qYDBzt4ZYRqzUjTR6CvUzjat8= github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b/go.mod h1:iwW+9cWfIzzDseEBCCeDSN5SD16Tidvy8cwQ7ZY8Qj4= -github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559 h1:c8n10eBkYU/HYaDUNAaKog4aIA3ZHO+GL7bHN2Ug/MA= -github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs= +github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b h1:s3gdV+iYJMvo9mydBJVAEA2Uaz29eIuUnQK867U3G8I= +github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs= github.com/milvus-io/pulsar-client-go v0.12.1 h1:O2JZp1tsYiO7C0MQ4hrUY/aJXnn2Gry6hpm7UodghmE= github.com/milvus-io/pulsar-client-go v0.12.1/go.mod h1:dkutuH4oS2pXiGm+Ti7fQZ4MRjrMPZ8IJeEGAWMeckk= github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 h1:AMFGa4R4MiIpspGNG7Z948v4n35fFGB3RR3G/ry4FWs= diff --git a/internal/.mockery.yaml b/internal/.mockery.yaml index cd53168bad..2ffeab9d7e 100644 --- a/internal/.mockery.yaml +++ b/internal/.mockery.yaml @@ -9,6 +9,7 @@ packages: interfaces: WALAccesser: Utility: + Broadcast: github.com/milvus-io/milvus/internal/streamingcoord/server/balancer: interfaces: Balancer: @@ -38,9 +39,6 @@ packages: github.com/milvus-io/milvus/internal/streamingnode/client/handler/consumer: interfaces: Consumer: - github.com/milvus-io/milvus/internal/streamingnode/server/flusher: - interfaces: - FlushMsgHandler: github.com/milvus-io/milvus/internal/streamingnode/server/wal: interfaces: OpenerBuilder: @@ -91,6 +89,9 @@ packages: github.com/milvus-io/milvus/internal/util/searchutil/optimizers: interfaces: QueryHook: +# github.com/milvus-io/milvus/internal/flushcommon/util: +# interfaces: +# MsgHandler: google.golang.org/grpc/resolver: interfaces: ClientConn: diff --git a/internal/datacoord/compaction.go b/internal/datacoord/compaction.go index 5923d9b150..b37ab7c501 100644 --- a/internal/datacoord/compaction.go +++ b/internal/datacoord/compaction.go @@ -61,6 +61,7 @@ type compactionPlanContext interface { removeTasksByChannel(channel string) setTaskScheduler(scheduler *taskScheduler) checkAndSetSegmentStating(channel string, segmentID int64) bool + getCompactionTasksNum(filters ...compactionTaskFilter) int } var ( @@ -146,7 +147,7 @@ func (sna *SlotBasedNodeAssigner) pickAnyNode(task CompactionTask) (nodeID int64 } type compactionPlanHandler struct { - queueTasks CompactionQueue + queueTasks *CompactionQueue executingGuard lock.RWMutex executingTasks map[int64]CompactionTask // planID -> task @@ -271,7 +272,7 @@ func newCompactionPlanHandler(cluster Cluster, sessions session.DataNodeManager, // TODO[GOOSE]: Higher capacity makes tasks waiting longer, which need to be get rid of. capacity := paramtable.Get().DataCoordCfg.CompactionTaskQueueCapacity.GetAsInt() return &compactionPlanHandler{ - queueTasks: *NewCompactionQueue(capacity, getPrioritizer()), + queueTasks: NewCompactionQueue(capacity, getPrioritizer()), meta: meta, sessions: sessions, allocator: allocator, @@ -806,6 +807,45 @@ func (c *compactionPlanHandler) checkDelay(t CompactionTask) { } } +func (c *compactionPlanHandler) getCompactionTasksNum(filters ...compactionTaskFilter) int { + cnt := 0 + isMatch := func(task CompactionTask) bool { + for _, f := range filters { + if !f(task) { + return false + } + } + return true + } + c.queueTasks.ForEach(func(task CompactionTask) { + if isMatch(task) { + cnt += 1 + } + }) + c.executingGuard.RLock() + for _, t := range c.executingTasks { + if isMatch(t) { + cnt += 1 + } + } + c.executingGuard.RUnlock() + return cnt +} + +type compactionTaskFilter func(task CompactionTask) bool + +func CollectionIDCompactionTaskFilter(collectionID int64) compactionTaskFilter { + return func(task CompactionTask) bool { + return task.GetTaskProto().GetCollectionID() == collectionID + } +} + +func L0CompactionCompactionTaskFilter() compactionTaskFilter { + return func(task CompactionTask) bool { + return task.GetTaskProto().GetType() == datapb.CompactionType_Level0DeleteCompaction + } +} + var ( ioPool *conc.Pool[any] ioPoolInitOnce sync.Once diff --git a/internal/datacoord/compaction_policy_l0.go b/internal/datacoord/compaction_policy_l0.go index c0fe9e66fd..493f32e9fa 100644 --- a/internal/datacoord/compaction_policy_l0.go +++ b/internal/datacoord/compaction_policy_l0.go @@ -18,12 +18,17 @@ type l0CompactionPolicy struct { meta *meta activeCollections *activeCollections + + // key: collectionID, value: reference count + skipCompactionCollections map[int64]int + skipLocker sync.RWMutex } func newL0CompactionPolicy(meta *meta) *l0CompactionPolicy { return &l0CompactionPolicy{ - meta: meta, - activeCollections: newActiveCollections(), + meta: meta, + activeCollections: newActiveCollections(), + skipCompactionCollections: make(map[int64]int), } } @@ -31,6 +36,34 @@ func (policy *l0CompactionPolicy) Enable() bool { return Params.DataCoordCfg.EnableAutoCompaction.GetAsBool() } +func (policy *l0CompactionPolicy) AddSkipCollection(collectionID UniqueID) { + policy.skipLocker.Lock() + defer policy.skipLocker.Unlock() + + if _, ok := policy.skipCompactionCollections[collectionID]; !ok { + policy.skipCompactionCollections[collectionID] = 1 + } else { + policy.skipCompactionCollections[collectionID]++ + } +} + +func (policy *l0CompactionPolicy) RemoveSkipCollection(collectionID UniqueID) { + policy.skipLocker.Lock() + defer policy.skipLocker.Unlock() + refCount := policy.skipCompactionCollections[collectionID] + if refCount > 1 { + policy.skipCompactionCollections[collectionID]-- + } else { + delete(policy.skipCompactionCollections, collectionID) + } +} + +func (policy *l0CompactionPolicy) isSkipCollection(collectionID UniqueID) bool { + policy.skipLocker.RLock() + defer policy.skipLocker.RUnlock() + return policy.skipCompactionCollections[collectionID] > 0 +} + // Notify policy to record the active updated(when adding a new L0 segment) collections. func (policy *l0CompactionPolicy) OnCollectionUpdate(collectionID int64) { policy.activeCollections.Record(collectionID) @@ -50,8 +83,11 @@ func (policy *l0CompactionPolicy) Trigger() (events map[CompactionTriggerType][] idleCollsSet := typeutil.NewUniqueSet(idleColls...) activeL0Views, idleL0Views := []CompactionView{}, []CompactionView{} for collID, segments := range latestCollSegs { - policy.activeCollections.Read(collID) + if policy.isSkipCollection(collID) { + continue + } + policy.activeCollections.Read(collID) levelZeroSegments := lo.Filter(segments, func(info *SegmentInfo, _ int) bool { return info.GetLevel() == datapb.SegmentLevel_L0 }) diff --git a/internal/datacoord/compaction_policy_l0_test.go b/internal/datacoord/compaction_policy_l0_test.go index a10eebcc76..94a0806b4c 100644 --- a/internal/datacoord/compaction_policy_l0_test.go +++ b/internal/datacoord/compaction_policy_l0_test.go @@ -113,6 +113,30 @@ func (s *L0CompactionPolicySuite) TestTriggerIdle() { for _, view := range cView.GetSegmentsView() { s.Equal(datapb.SegmentLevel_L0, view.Level) } + + // test for skip collection + s.l0_policy.AddSkipCollection(1) + s.l0_policy.AddSkipCollection(1) + // Test for skip collection + events, err = s.l0_policy.Trigger() + s.NoError(err) + s.Empty(events) + + // Test for skip collection with ref count + s.l0_policy.RemoveSkipCollection(1) + events, err = s.l0_policy.Trigger() + s.NoError(err) + s.Empty(events) + + s.l0_policy.RemoveSkipCollection(1) + events, err = s.l0_policy.Trigger() + s.NoError(err) + s.Equal(1, len(events)) + gotViews, ok = events[TriggerTypeLevelZeroViewIDLE] + s.True(ok) + s.NotNil(gotViews) + s.Equal(1, len(gotViews)) + log.Info("cView", zap.String("string", cView.String())) } diff --git a/internal/datacoord/compaction_test.go b/internal/datacoord/compaction_test.go index e67965c0bb..a0b039dcaa 100644 --- a/internal/datacoord/compaction_test.go +++ b/internal/datacoord/compaction_test.go @@ -22,6 +22,7 @@ import ( "time" "github.com/cockroachdb/errors" + "github.com/magiconair/properties/assert" "github.com/samber/lo" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" @@ -1187,3 +1188,60 @@ func TestCheckDelay(t *testing.T) { }, nil, nil, nil, nil, nil) handler.checkDelay(t3) } + +func TestGetCompactionTasksNum(t *testing.T) { + queueTasks := NewCompactionQueue(10, DefaultPrioritizer) + queueTasks.Enqueue( + newMixCompactionTask(&datapb.CompactionTask{ + StartTime: time.Now().Add(-100 * time.Minute).Unix(), + CollectionID: 1, + Type: datapb.CompactionType_MixCompaction, + }, nil, nil, nil), + ) + queueTasks.Enqueue( + newL0CompactionTask(&datapb.CompactionTask{ + StartTime: time.Now().Add(-100 * time.Minute).Unix(), + CollectionID: 1, + Type: datapb.CompactionType_Level0DeleteCompaction, + }, nil, nil, nil), + ) + queueTasks.Enqueue( + newClusteringCompactionTask(&datapb.CompactionTask{ + StartTime: time.Now().Add(-100 * time.Minute).Unix(), + CollectionID: 10, + Type: datapb.CompactionType_ClusteringCompaction, + }, nil, nil, nil, nil, nil), + ) + executingTasks := make(map[int64]CompactionTask, 0) + executingTasks[1] = newMixCompactionTask(&datapb.CompactionTask{ + StartTime: time.Now().Add(-100 * time.Minute).Unix(), + CollectionID: 1, + Type: datapb.CompactionType_MixCompaction, + }, nil, nil, nil) + executingTasks[2] = newL0CompactionTask(&datapb.CompactionTask{ + StartTime: time.Now().Add(-100 * time.Minute).Unix(), + CollectionID: 10, + Type: datapb.CompactionType_Level0DeleteCompaction, + }, nil, nil, nil) + + handler := &compactionPlanHandler{ + queueTasks: queueTasks, + executingTasks: executingTasks, + } + t.Run("no filter", func(t *testing.T) { + i := handler.getCompactionTasksNum() + assert.Equal(t, 5, i) + }) + t.Run("collection id filter", func(t *testing.T) { + i := handler.getCompactionTasksNum(CollectionIDCompactionTaskFilter(1)) + assert.Equal(t, 3, i) + }) + t.Run("l0 compaction filter", func(t *testing.T) { + i := handler.getCompactionTasksNum(L0CompactionCompactionTaskFilter()) + assert.Equal(t, 2, i) + }) + t.Run("collection id and l0 compaction filter", func(t *testing.T) { + i := handler.getCompactionTasksNum(CollectionIDCompactionTaskFilter(1), L0CompactionCompactionTaskFilter()) + assert.Equal(t, 1, i) + }) +} diff --git a/internal/datacoord/compaction_trigger_test.go b/internal/datacoord/compaction_trigger_test.go index 723cc85b72..94e7e798e5 100644 --- a/internal/datacoord/compaction_trigger_test.go +++ b/internal/datacoord/compaction_trigger_test.go @@ -50,6 +50,11 @@ type spyCompactionHandler struct { meta *meta } +// getCompactionTasksNum implements compactionPlanContext. +func (h *spyCompactionHandler) getCompactionTasksNum(filters ...compactionTaskFilter) int { + return 0 +} + func (h *spyCompactionHandler) getCompactionTasksNumBySignalID(signalID int64) int { return 0 } diff --git a/internal/datacoord/compaction_trigger_v2.go b/internal/datacoord/compaction_trigger_v2.go index 1a48dac610..4ebf3cbc76 100644 --- a/internal/datacoord/compaction_trigger_v2.go +++ b/internal/datacoord/compaction_trigger_v2.go @@ -27,6 +27,7 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/logutil" ) @@ -62,6 +63,8 @@ type TriggerManager interface { Stop() OnCollectionUpdate(collectionID int64) ManualTrigger(ctx context.Context, collectionID int64, clusteringCompaction bool) (UniqueID, error) + GetPauseCompactionChan(jobID, collectionID int64) <-chan struct{} + GetResumeCompactionChan(jobID, collectionID int64) <-chan struct{} } var _ TriggerManager = (*CompactionTriggerManager)(nil) @@ -82,21 +85,35 @@ type CompactionTriggerManager struct { allocator allocator.Allocator meta *meta + imeta ImportMeta l0Policy *l0CompactionPolicy clusteringPolicy *clusteringCompactionPolicy singlePolicy *singleCompactionPolicy cancel context.CancelFunc closeWg sync.WaitGroup + + l0Triggering bool + l0SigLock *sync.Mutex + l0TickSig *sync.Cond + + pauseCompactionChanMap map[int64]chan struct{} + resumeCompactionChanMap map[int64]chan struct{} + compactionChanLock sync.Mutex } -func NewCompactionTriggerManager(alloc allocator.Allocator, handler Handler, compactionHandler compactionPlanContext, meta *meta) *CompactionTriggerManager { +func NewCompactionTriggerManager(alloc allocator.Allocator, handler Handler, compactionHandler compactionPlanContext, meta *meta, imeta ImportMeta) *CompactionTriggerManager { m := &CompactionTriggerManager{ - allocator: alloc, - handler: handler, - compactionHandler: compactionHandler, - meta: meta, + allocator: alloc, + handler: handler, + compactionHandler: compactionHandler, + meta: meta, + imeta: imeta, + pauseCompactionChanMap: make(map[int64]chan struct{}), + resumeCompactionChanMap: make(map[int64]chan struct{}), } + m.l0SigLock = &sync.Mutex{} + m.l0TickSig = sync.NewCond(m.l0SigLock) m.l0Policy = newL0CompactionPolicy(meta) m.clusteringPolicy = newClusteringCompactionPolicy(meta, m.allocator, m.handler) m.singlePolicy = newSingleCompactionPolicy(meta, m.allocator, m.handler) @@ -126,6 +143,64 @@ func (m *CompactionTriggerManager) Stop() { m.closeWg.Wait() } +func (m *CompactionTriggerManager) pauseL0SegmentCompacting(jobID, collectionID int64) { + m.l0Policy.AddSkipCollection(collectionID) + m.l0SigLock.Lock() + for m.l0Triggering { + m.l0TickSig.Wait() + } + m.l0SigLock.Unlock() + m.compactionChanLock.Lock() + if ch, ok := m.pauseCompactionChanMap[jobID]; ok { + close(ch) + } + m.compactionChanLock.Unlock() +} + +func (m *CompactionTriggerManager) resumeL0SegmentCompacting(jobID, collectionID int64) { + m.compactionChanLock.Lock() + m.l0Policy.RemoveSkipCollection(collectionID) + if ch, ok := m.resumeCompactionChanMap[jobID]; ok { + close(ch) + delete(m.pauseCompactionChanMap, jobID) + delete(m.resumeCompactionChanMap, jobID) + } + m.compactionChanLock.Unlock() +} + +func (m *CompactionTriggerManager) GetPauseCompactionChan(jobID, collectionID int64) <-chan struct{} { + m.compactionChanLock.Lock() + defer m.compactionChanLock.Unlock() + if ch, ok := m.pauseCompactionChanMap[jobID]; ok { + return ch + } + ch := make(chan struct{}) + m.pauseCompactionChanMap[jobID] = ch + go m.pauseL0SegmentCompacting(jobID, collectionID) + return ch +} + +func (m *CompactionTriggerManager) GetResumeCompactionChan(jobID, collectionID int64) <-chan struct{} { + m.compactionChanLock.Lock() + defer m.compactionChanLock.Unlock() + if ch, ok := m.resumeCompactionChanMap[jobID]; ok { + return ch + } + ch := make(chan struct{}) + m.resumeCompactionChanMap[jobID] = ch + go m.resumeL0SegmentCompacting(jobID, collectionID) + return ch +} + +func (m *CompactionTriggerManager) setL0Triggering(b bool) { + m.l0SigLock.Lock() + defer m.l0SigLock.Unlock() + m.l0Triggering = b + if !b { + m.l0TickSig.Broadcast() + } +} + func (m *CompactionTriggerManager) loop(ctx context.Context) { defer logutil.LogPanic() @@ -150,9 +225,11 @@ func (m *CompactionTriggerManager) loop(ctx context.Context) { log.RatedInfo(10, "Skip trigger l0 compaction since compactionHandler is full") continue } + m.setL0Triggering(true) events, err := m.l0Policy.Trigger() if err != nil { log.Warn("Fail to trigger L0 policy", zap.Error(err)) + m.setL0Triggering(false) continue } if len(events) > 0 { @@ -160,6 +237,7 @@ func (m *CompactionTriggerManager) loop(ctx context.Context) { m.notify(ctx, triggerType, views) } } + m.setL0Triggering(false) case <-clusteringTicker.C: if !m.clusteringPolicy.Enable() { continue @@ -262,6 +340,12 @@ func (m *CompactionTriggerManager) SubmitL0ViewToScheduler(ctx context.Context, return } + err = m.addL0ImportTaskForImport(ctx, collection, view) + if err != nil { + log.Warn("Failed to submit compaction view to scheduler because add l0 import task fail", zap.Error(err)) + return + } + task := &datapb.CompactionTask{ TriggerID: taskID, // inner trigger, use task id as trigger id PlanID: taskID, @@ -294,6 +378,75 @@ func (m *CompactionTriggerManager) SubmitL0ViewToScheduler(ctx context.Context, ) } +func (m *CompactionTriggerManager) addL0ImportTaskForImport(ctx context.Context, collection *collectionInfo, view CompactionView) error { + // add l0 import task for the collection if the collection is importing + importJobs := m.imeta.GetJobBy(ctx, WithCollectionID(collection.ID), WithoutJobStates(internalpb.ImportJobState_Completed, internalpb.ImportJobState_Failed)) + if len(importJobs) > 0 { + partitionID := view.GetGroupLabel().PartitionID + var ( + fileSize int64 = 0 + totalRows int64 = 0 + totalMemorySize int64 = 0 + importPaths []string + ) + idStart := time.Now().UnixMilli() + for _, segmentView := range view.GetSegmentsView() { + segInfo := m.meta.GetSegment(ctx, segmentView.ID) + if segInfo == nil { + continue + } + totalRows += int64(segmentView.DeltaRowCount) + totalMemorySize += int64(segmentView.DeltaSize) + for _, deltaLogs := range segInfo.GetDeltalogs() { + for _, binlog := range deltaLogs.GetBinlogs() { + fileSize += binlog.GetLogSize() + importPaths = append(importPaths, binlog.GetLogPath()) + } + } + } + + for i, job := range importJobs { + newTasks, err := NewImportTasks([][]*datapb.ImportFileStats{ + { + { + ImportFile: &internalpb.ImportFile{ + Id: idStart + int64(i), + Paths: importPaths, + }, + FileSize: fileSize, + TotalRows: totalRows, + TotalMemorySize: totalMemorySize, + HashedStats: map[string]*datapb.PartitionImportStats{ + // which is vchannel + view.GetGroupLabel().Channel: { + PartitionRows: map[int64]int64{ + partitionID: totalRows, + }, + PartitionDataSize: map[int64]int64{ + partitionID: totalMemorySize, + }, + }, + }, + }, + }, + }, job, m.allocator, m.meta) + if err != nil { + log.Warn("new import tasks failed", zap.Error(err)) + return err + } + for _, t := range newTasks { + err = m.imeta.AddTask(ctx, t) + if err != nil { + log.Warn("add new l0 import task from l0 compaction failed", WrapTaskLog(t, zap.Error(err))...) + return err + } + log.Info("add new l0 import task from l0 compaction", WrapTaskLog(t)...) + } + } + } + return nil +} + func (m *CompactionTriggerManager) SubmitClusteringViewToScheduler(ctx context.Context, view CompactionView) { log := log.Ctx(ctx).With(zap.String("view", view.String())) taskID, _, err := m.allocator.AllocN(2) diff --git a/internal/datacoord/compaction_trigger_v2_test.go b/internal/datacoord/compaction_trigger_v2_test.go index 3bb670aa09..18ff9c7143 100644 --- a/internal/datacoord/compaction_trigger_v2_test.go +++ b/internal/datacoord/compaction_trigger_v2_test.go @@ -4,8 +4,10 @@ import ( "context" "strconv" "testing" + "time" "github.com/samber/lo" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" "go.uber.org/zap" @@ -13,10 +15,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datacoord/allocator" + "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/paramtable" ) @@ -32,6 +36,7 @@ type CompactionTriggerManagerSuite struct { mockPlanContext *MockCompactionPlanContext testLabel *CompactionGroupLabel meta *meta + imeta ImportMeta triggerManager *CompactionTriggerManager } @@ -51,8 +56,14 @@ func (s *CompactionTriggerManagerSuite) SetupTest() { for id, segment := range segments { s.meta.segments.SetSegment(id, segment) } - - s.triggerManager = NewCompactionTriggerManager(s.mockAlloc, s.handler, s.mockPlanContext, s.meta) + catalog := mocks.NewDataCoordCatalog(s.T()) + catalog.EXPECT().ListPreImportTasks(mock.Anything).Return([]*datapb.PreImportTask{}, nil) + catalog.EXPECT().ListImportTasks(mock.Anything).Return([]*datapb.ImportTaskV2{}, nil) + catalog.EXPECT().ListImportJobs(mock.Anything).Return([]*datapb.ImportJob{}, nil) + importMeta, err := NewImportMeta(context.TODO(), catalog) + s.Require().NoError(err) + s.imeta = importMeta + s.triggerManager = NewCompactionTriggerManager(s.mockAlloc, s.handler, s.mockPlanContext, s.meta, s.imeta) } func (s *CompactionTriggerManagerSuite) TestNotifyByViewIDLE() { @@ -308,3 +319,73 @@ func (s *CompactionTriggerManagerSuite) TestGetExpectedSegmentSize() { s.Equal(int64(100*1024*1024), getExpectedSegmentSize(s.triggerManager.meta, collection)) }) } + +func TestCompactionAndImport(t *testing.T) { + paramtable.Init() + mockAlloc := allocator.NewMockAllocator(t) + handler := NewNMockHandler(t) + handler.EXPECT().GetCollection(mock.Anything, mock.Anything).Return(&collectionInfo{ + ID: 1, + }, nil) + mockPlanContext := NewMockCompactionPlanContext(t) + mockPlanContext.EXPECT().isFull().Return(false) + + testLabel := &CompactionGroupLabel{ + CollectionID: 1, + PartitionID: 10, + Channel: "ch-1", + } + segments := genSegmentsForMeta(testLabel) + catelog := mocks.NewDataCoordCatalog(t) + catelog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil) + meta := &meta{ + segments: NewSegmentsInfo(), + catalog: catelog, + } + for id, segment := range segments { + meta.segments.SetSegment(id, segment) + } + catalog := mocks.NewDataCoordCatalog(t) + catalog.EXPECT().ListPreImportTasks(mock.Anything).Return([]*datapb.PreImportTask{}, nil) + catalog.EXPECT().ListImportTasks(mock.Anything).Return([]*datapb.ImportTaskV2{}, nil) + catalog.EXPECT().ListImportJobs(mock.Anything).Return([]*datapb.ImportJob{ + { + JobID: 100, + CollectionID: 1, + State: internalpb.ImportJobState_Importing, + }, + }, nil).Once() + catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil) + importMeta, err := NewImportMeta(context.TODO(), catalog) + assert.NoError(t, err) + imeta := importMeta + triggerManager := NewCompactionTriggerManager(mockAlloc, handler, mockPlanContext, meta, imeta) + + Params.Save(Params.DataCoordCfg.L0CompactionTriggerInterval.Key, "1") + defer Params.Reset(Params.DataCoordCfg.L0CompactionTriggerInterval.Key) + Params.Save(Params.DataCoordCfg.ClusteringCompactionTriggerInterval.Key, "6000000") + defer Params.Reset(Params.DataCoordCfg.ClusteringCompactionTriggerInterval.Key) + Params.Save(Params.DataCoordCfg.MixCompactionTriggerInterval.Key, "6000000") + defer Params.Reset(Params.DataCoordCfg.MixCompactionTriggerInterval.Key) + + mockAlloc.EXPECT().AllocID(mock.Anything).Return(1, nil) + mockAlloc.EXPECT().AllocN(mock.Anything).Return(195300, 195300, nil) + mockAlloc.EXPECT().AllocTimestamp(mock.Anything).Return(30000, nil) + mockPlanContext.EXPECT().enqueueCompaction(mock.Anything). + RunAndReturn(func(task *datapb.CompactionTask) error { + assert.Equal(t, datapb.CompactionType_Level0DeleteCompaction, task.GetType()) + expectedSegs := []int64{100, 101, 102} + assert.ElementsMatch(t, expectedSegs, task.GetInputSegments()) + return nil + }).Return(nil) + mockAlloc.EXPECT().AllocID(mock.Anything).Return(19530, nil).Maybe() + + <-triggerManager.GetPauseCompactionChan(100, 10) + defer func() { + <-triggerManager.GetResumeCompactionChan(100, 10) + }() + + triggerManager.Start() + defer triggerManager.Stop() + time.Sleep(3 * time.Second) +} diff --git a/internal/datacoord/import_checker.go b/internal/datacoord/import_checker.go index 628fee027c..7909151205 100644 --- a/internal/datacoord/import_checker.go +++ b/internal/datacoord/import_checker.go @@ -33,6 +33,7 @@ import ( "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) @@ -42,12 +43,13 @@ type ImportChecker interface { } type importChecker struct { - meta *meta - broker broker.Broker - cluster Cluster - alloc allocator.Allocator - imeta ImportMeta - sjm StatsJobManager + meta *meta + broker broker.Broker + cluster Cluster + alloc allocator.Allocator + imeta ImportMeta + sjm StatsJobManager + l0CompactionTrigger TriggerManager closeOnce sync.Once closeChan chan struct{} @@ -59,15 +61,17 @@ func NewImportChecker(meta *meta, alloc allocator.Allocator, imeta ImportMeta, sjm StatsJobManager, + l0CompactionTrigger TriggerManager, ) ImportChecker { return &importChecker{ - meta: meta, - broker: broker, - cluster: cluster, - alloc: alloc, - imeta: imeta, - sjm: sjm, - closeChan: make(chan struct{}), + meta: meta, + broker: broker, + cluster: cluster, + alloc: alloc, + imeta: imeta, + sjm: sjm, + l0CompactionTrigger: l0CompactionTrigger, + closeChan: make(chan struct{}), } } @@ -87,6 +91,14 @@ func (c *importChecker) Start() { case <-ticker1.C: jobs := c.imeta.GetJobBy(context.TODO()) for _, job := range jobs { + if !funcutil.SliceSetEqual[string](job.GetVchannels(), job.GetReadyVchannels()) { + // wait for all channels to send signals + log.Info("waiting for all channels to send signals", + zap.Strings("vchannels", job.GetVchannels()), + zap.Strings("readyVchannels", job.GetReadyVchannels()), + zap.Int64("jobID", job.GetJobID())) + continue + } switch job.GetState() { case internalpb.ImportJobState_Pending: c.checkPendingJob(job) @@ -280,7 +292,7 @@ func (c *importChecker) checkPreImportingJob(job ImportJob) { func (c *importChecker) checkImportingJob(job ImportJob) { log := log.With(zap.Int64("jobID", job.GetJobID())) - tasks := c.imeta.GetTaskBy(context.TODO(), WithType(ImportTaskType), WithJob(job.GetJobID())) + tasks := c.imeta.GetTaskBy(context.TODO(), WithType(ImportTaskType), WithJob(job.GetJobID()), WithRequestSource()) for _, t := range tasks { if t.GetState() != datapb.ImportTaskStateV2_Completed { return @@ -380,11 +392,54 @@ func (c *importChecker) checkIndexBuildingJob(job ImportJob) { log.Debug("waiting for import segments building index...", zap.Int64s("unindexed", unindexed)) return } - buildIndexDuration := job.GetTR().RecordSpan() metrics.ImportJobLatency.WithLabelValues(metrics.ImportStageBuildIndex).Observe(float64(buildIndexDuration.Milliseconds())) log.Info("import job build index done", zap.Duration("jobTimeCost/buildIndex", buildIndexDuration)) + // wait l0 segment import and block l0 compaction + log.Info("start to pause l0 segment compacting", zap.Int64("jobID", job.GetJobID())) + <-c.l0CompactionTrigger.GetPauseCompactionChan(job.GetJobID(), job.GetCollectionID()) + log.Info("l0 segment compacting paused", zap.Int64("jobID", job.GetJobID())) + + if c.waitL0ImortTaskDone(job) { + return + } + waitL0ImportDuration := job.GetTR().RecordSpan() + metrics.ImportJobLatency.WithLabelValues(metrics.ImportStageWaitL0Import).Observe(float64(buildIndexDuration.Milliseconds())) + log.Info("import job l0 import done", zap.Duration("jobTimeCost/l0Import", waitL0ImportDuration)) + + if c.updateSegmentState(job, originSegmentIDs, statsSegmentIDs) { + return + } + // all finished, update import job state to `Completed`. + completeTime := time.Now().Format("2006-01-02T15:04:05Z07:00") + err := c.imeta.UpdateJob(context.TODO(), job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Completed), UpdateJobCompleteTime(completeTime)) + if err != nil { + log.Warn("failed to update job state to Completed", zap.Error(err)) + return + } + totalDuration := job.GetTR().ElapseSpan() + metrics.ImportJobLatency.WithLabelValues(metrics.TotalLabel).Observe(float64(totalDuration.Milliseconds())) + <-c.l0CompactionTrigger.GetResumeCompactionChan(job.GetJobID(), job.GetCollectionID()) + log.Info("import job all completed", zap.Duration("jobTimeCost/total", totalDuration)) +} + +func (c *importChecker) waitL0ImortTaskDone(job ImportJob) bool { + // wait all lo import tasks to be completed + l0ImportTasks := c.imeta.GetTaskBy(context.TODO(), WithType(ImportTaskType), WithJob(job.GetJobID()), WithL0CompactionSource()) + for _, t := range l0ImportTasks { + if t.GetState() != datapb.ImportTaskStateV2_Completed { + log.Info("waiting for l0 import task...", + zap.Int64s("taskIDs", lo.Map(l0ImportTasks, func(t ImportTask, _ int) int64 { + return t.GetTaskID() + }))) + return true + } + } + return false +} + +func (c *importChecker) updateSegmentState(job ImportJob, originSegmentIDs, statsSegmentIDs []int64) bool { // Here, all segment indexes have been successfully built, try unset isImporting flag for all segments. isImportingSegments := lo.Filter(append(originSegmentIDs, statsSegmentIDs...), func(segmentID int64, _ int) bool { segment := c.meta.GetSegment(context.TODO(), segmentID) @@ -397,13 +452,13 @@ func (c *importChecker) checkIndexBuildingJob(job ImportJob) { channels, err := c.meta.GetSegmentsChannels(isImportingSegments) if err != nil { log.Warn("get segments channels failed", zap.Error(err)) - return + return true } for _, segmentID := range isImportingSegments { channelCP := c.meta.GetChannelCheckpoint(channels[segmentID]) if channelCP == nil { log.Warn("nil channel checkpoint") - return + return true } op1 := UpdateStartPosition([]*datapb.SegmentStartPosition{{StartPosition: channelCP, SegmentID: segmentID}}) op2 := UpdateDmlPosition(segmentID, channelCP) @@ -411,20 +466,10 @@ func (c *importChecker) checkIndexBuildingJob(job ImportJob) { err = c.meta.UpdateSegmentsInfo(context.TODO(), op1, op2, op3) if err != nil { log.Warn("update import segment failed", zap.Error(err)) - return + return true } } - - // all finished, update import job state to `Completed`. - completeTime := time.Now().Format("2006-01-02T15:04:05Z07:00") - err = c.imeta.UpdateJob(context.TODO(), job.GetJobID(), UpdateJobState(internalpb.ImportJobState_Completed), UpdateJobCompleteTime(completeTime)) - if err != nil { - log.Warn("failed to update job state to Completed", zap.Error(err)) - return - } - totalDuration := job.GetTR().ElapseSpan() - metrics.ImportJobLatency.WithLabelValues(metrics.TotalLabel).Observe(float64(totalDuration.Milliseconds())) - log.Info("import job all completed", zap.Duration("jobTimeCost/total", totalDuration)) + return false } func (c *importChecker) checkFailedJob(job ImportJob) { diff --git a/internal/datacoord/import_checker_test.go b/internal/datacoord/import_checker_test.go index acc6301e47..0f33718979 100644 --- a/internal/datacoord/import_checker_test.go +++ b/internal/datacoord/import_checker_test.go @@ -23,17 +23,22 @@ import ( "time" "github.com/cockroachdb/errors" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" + "go.uber.org/zap" "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/datacoord/allocator" broker2 "github.com/milvus-io/milvus/internal/datacoord/broker" "github.com/milvus-io/milvus/internal/metastore/mocks" + "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/timerecord" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) @@ -74,8 +79,13 @@ func (s *ImportCheckerSuite) SetupTest() { s.NoError(err) sjm := NewMockStatsJobManager(s.T()) + l0CompactionTrigger := NewMockTriggerManager(s.T()) + compactionChan := make(chan struct{}, 1) + close(compactionChan) + l0CompactionTrigger.EXPECT().GetPauseCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe() + l0CompactionTrigger.EXPECT().GetResumeCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe() - checker := NewImportChecker(meta, broker, cluster, s.alloc, imeta, sjm).(*importChecker) + checker := NewImportChecker(meta, broker, cluster, s.alloc, imeta, sjm, l0CompactionTrigger).(*importChecker) s.checker = checker job := &importJob{ @@ -508,3 +518,208 @@ func (s *ImportCheckerSuite) TestCheckCollection() { func TestImportChecker(t *testing.T) { suite.Run(t, new(ImportCheckerSuite)) } + +func TestImportCheckerCompaction(t *testing.T) { + paramtable.Init() + Params.Save(Params.DataCoordCfg.ImportCheckIntervalHigh.Key, "1") + defer Params.Reset(Params.DataCoordCfg.ImportCheckIntervalHigh.Key) + Params.Save(Params.DataCoordCfg.ImportCheckIntervalLow.Key, "10000") + defer Params.Reset(Params.DataCoordCfg.ImportCheckIntervalLow.Key) + + // prepare objects + catalog := mocks.NewDataCoordCatalog(t) + catalog.EXPECT().ListImportJobs(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListPreImportTasks(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListImportTasks(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListChannelCheckpoint(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListIndexes(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) + + cluster := NewMockCluster(t) + alloc := allocator.NewMockAllocator(t) + + imeta, err := NewImportMeta(context.TODO(), catalog) + assert.NoError(t, err) + + broker := broker2.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{}, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) + sjm := NewMockStatsJobManager(t) + l0CompactionTrigger := NewMockTriggerManager(t) + compactionChan := make(chan struct{}, 1) + close(compactionChan) + l0CompactionTrigger.EXPECT().GetPauseCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe() + l0CompactionTrigger.EXPECT().GetResumeCompactionChan(mock.Anything, mock.Anything).Return(compactionChan).Maybe() + + checker := NewImportChecker(meta, broker, cluster, alloc, imeta, sjm, l0CompactionTrigger).(*importChecker) + + job := &importJob{ + ImportJob: &datapb.ImportJob{ + JobID: 1001, + CollectionID: 1, + PartitionIDs: []int64{2}, + ReadyVchannels: []string{"ch0"}, + Vchannels: []string{"ch0", "ch1"}, + State: internalpb.ImportJobState_Pending, + TimeoutTs: tsoutil.ComposeTSByTime(time.Now().Add(time.Hour), 0), + CleanupTs: tsoutil.ComposeTSByTime(time.Now().Add(time.Hour), 0), + Files: []*internalpb.ImportFile{ + { + Id: 1, + Paths: []string{"a.json"}, + }, + { + Id: 2, + Paths: []string{"b.json"}, + }, + { + Id: 3, + Paths: []string{"c.json"}, + }, + }, + }, + tr: timerecord.NewTimeRecorder("import job"), + } + catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once() + err = imeta.AddJob(context.TODO(), job) + assert.NoError(t, err) + jobID := job.GetJobID() + + // start check + go checker.Start() + + // sleep 1.5s and ready the job, go to pending stats + time.Sleep(1500 * time.Millisecond) + catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once() + job2 := &importJob{ + ImportJob: &datapb.ImportJob{ + JobID: 1001, + CollectionID: 1, + PartitionIDs: []int64{2}, + ReadyVchannels: []string{"ch1"}, + Vchannels: []string{"ch0", "ch1"}, + State: internalpb.ImportJobState_Pending, + TimeoutTs: tsoutil.ComposeTSByTime(time.Now().Add(time.Hour), 0), + CleanupTs: tsoutil.ComposeTSByTime(time.Now().Add(time.Hour), 0), + Files: []*internalpb.ImportFile{ + { + Id: 1, + Paths: []string{"a.json"}, + }, + { + Id: 2, + Paths: []string{"b.json"}, + }, + { + Id: 3, + Paths: []string{"c.json"}, + }, + }, + }, + tr: timerecord.NewTimeRecorder("import job"), + } + err = imeta.AddJob(context.TODO(), job2) + assert.NoError(t, err) + log.Info("job ready") + + // check pending + alloc.EXPECT().AllocN(mock.Anything).RunAndReturn(func(n int64) (int64, int64, error) { + id := rand.Int63() + return id, id + n, nil + }).Maybe() + alloc.EXPECT().AllocID(mock.Anything).Return(rand.Int63(), nil).Maybe() + catalog.EXPECT().SavePreImportTask(mock.Anything, mock.Anything).Return(nil).Twice() + catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once() + assert.Eventually(t, func() bool { + job := imeta.GetJob(context.TODO(), jobID) + preimportTasks := imeta.GetTaskBy(context.TODO(), WithJob(job.GetJobID()), WithType(PreImportTaskType)) + taskLen := len(preimportTasks) + log.Info("job pre-importing", zap.Any("taskLen", taskLen), zap.Any("jobState", job.GetState())) + return taskLen == 2 && job.GetState() == internalpb.ImportJobState_PreImporting + }, 2*time.Second, 500*time.Millisecond) + log.Info("job pre-importing") + + // check pre-importing + catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil).Once() + catalog.EXPECT().SavePreImportTask(mock.Anything, mock.Anything).Return(nil).Twice() + catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once() + preimportTasks := imeta.GetTaskBy(context.TODO(), WithJob(job.GetJobID()), WithType(PreImportTaskType)) + for _, pt := range preimportTasks { + err := imeta.UpdateTask(context.TODO(), pt.GetTaskID(), UpdateState(datapb.ImportTaskStateV2_Completed)) + assert.NoError(t, err) + } + assert.Eventually(t, func() bool { + job := imeta.GetJob(context.TODO(), jobID) + importTasks := imeta.GetTaskBy(context.TODO(), WithJob(job.GetJobID()), WithType(ImportTaskType)) + return len(importTasks) == 1 && job.GetState() == internalpb.ImportJobState_Importing + }, 2*time.Second, 100*time.Millisecond) + log.Info("job importing") + + // check importing + catalog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil).Once() + catalog.EXPECT().AlterSegments(mock.Anything, mock.Anything).Return(nil).Once() + catalog.EXPECT().SaveChannelCheckpoint(mock.Anything, mock.Anything, mock.Anything).Return(nil).Once() + catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once() + catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil).Once() + importTasks := imeta.GetTaskBy(context.TODO(), WithJob(job.GetJobID()), WithType(ImportTaskType)) + for _, it := range importTasks { + segment := &SegmentInfo{ + SegmentInfo: &datapb.SegmentInfo{ + ID: rand.Int63(), + State: commonpb.SegmentState_Flushed, + IsImporting: true, + InsertChannel: "ch0", + }, + } + err := checker.meta.AddSegment(context.Background(), segment) + assert.NoError(t, err) + err = imeta.UpdateTask(context.TODO(), it.GetTaskID(), UpdateState(datapb.ImportTaskStateV2_Completed), + UpdateSegmentIDs([]int64{segment.GetID()}), UpdateStatsSegmentIDs([]int64{rand.Int63()})) + assert.NoError(t, err) + err = checker.meta.UpdateChannelCheckpoint(context.TODO(), segment.GetInsertChannel(), &msgpb.MsgPosition{MsgID: []byte{0}}) + assert.NoError(t, err) + } + assert.Eventually(t, func() bool { + job := imeta.GetJob(context.TODO(), jobID) + return job.GetState() == internalpb.ImportJobState_Stats + }, 2*time.Second, 100*time.Millisecond) + log.Info("job stats") + + // check stats + catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once() + sjm.EXPECT().GetStatsTask(mock.Anything, mock.Anything).Return(&indexpb.StatsTask{ + State: indexpb.JobState_JobStateFinished, + }).Once() + assert.Eventually(t, func() bool { + job := imeta.GetJob(context.TODO(), jobID) + return job.GetState() == internalpb.ImportJobState_IndexBuilding + }, 2*time.Second, 100*time.Millisecond) + log.Info("job index building") + + // wait l0 import task + catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil).Once() + imeta.AddTask(context.TODO(), &importTask{ + ImportTaskV2: &datapb.ImportTaskV2{ + JobID: jobID, + TaskID: 100000, + Source: datapb.ImportTaskSourceV2_L0Compaction, + State: datapb.ImportTaskStateV2_InProgress, + }, + }) + time.Sleep(1200 * time.Millisecond) + catalog.EXPECT().SaveImportTask(mock.Anything, mock.Anything).Return(nil).Once() + imeta.UpdateTask(context.TODO(), 100000, UpdateState(datapb.ImportTaskStateV2_Completed)) + log.Info("job l0 compaction") + + // check index building + catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil).Once() + assert.Eventually(t, func() bool { + job := imeta.GetJob(context.TODO(), jobID) + return job.GetState() == internalpb.ImportJobState_Completed + }, 2*time.Second, 100*time.Millisecond) + log.Info("job completed") +} diff --git a/internal/datacoord/import_job.go b/internal/datacoord/import_job.go index e9826b6f0d..20d697345e 100644 --- a/internal/datacoord/import_job.go +++ b/internal/datacoord/import_job.go @@ -104,6 +104,7 @@ type ImportJob interface { GetCollectionName() string GetPartitionIDs() []int64 GetVchannels() []string + GetReadyVchannels() []string GetSchema() *schemapb.CollectionSchema GetTimeoutTs() uint64 GetCleanupTs() uint64 @@ -115,6 +116,7 @@ type ImportJob interface { GetFiles() []*internalpb.ImportFile GetOptions() []*commonpb.KeyValuePair GetTR() *timerecord.TimeRecorder + GetDataTs() uint64 Clone() ImportJob } diff --git a/internal/datacoord/import_meta.go b/internal/datacoord/import_meta.go index 1453c0afaa..4d547896d2 100644 --- a/internal/datacoord/import_meta.go +++ b/internal/datacoord/import_meta.go @@ -21,6 +21,7 @@ import ( "time" "github.com/hashicorp/golang-lru/v2/expirable" + "github.com/samber/lo" "golang.org/x/exp/maps" "github.com/milvus-io/milvus/internal/json" @@ -140,6 +141,13 @@ func NewImportMeta(ctx context.Context, catalog metastore.DataCoordCatalog) (Imp func (m *importMeta) AddJob(ctx context.Context, job ImportJob) error { m.mu.Lock() defer m.mu.Unlock() + originJob := m.jobs[job.GetJobID()] + if originJob != nil { + originJob := originJob.Clone() + internalJob := originJob.(*importJob).ImportJob + internalJob.ReadyVchannels = lo.Union(originJob.GetReadyVchannels(), job.GetReadyVchannels()) + job = originJob + } err := m.catalog.SaveImportJob(ctx, job.(*importJob).ImportJob) if err != nil { return err diff --git a/internal/datacoord/import_meta_test.go b/internal/datacoord/import_meta_test.go index 8f59624e7e..a51992d04e 100644 --- a/internal/datacoord/import_meta_test.go +++ b/internal/datacoord/import_meta_test.go @@ -90,13 +90,15 @@ func TestImportMeta_Job(t *testing.T) { jobIDs := []int64{1000, 2000, 3000} for i, jobID := range jobIDs { + channel := fmt.Sprintf("ch-%d", rand.Int63()) var job ImportJob = &importJob{ ImportJob: &datapb.ImportJob{ - JobID: jobID, - CollectionID: rand.Int63(), - PartitionIDs: []int64{rand.Int63()}, - Vchannels: []string{fmt.Sprintf("ch-%d", rand.Int63())}, - State: internalpb.ImportJobState_Pending, + JobID: jobID, + CollectionID: rand.Int63(), + PartitionIDs: []int64{rand.Int63()}, + Vchannels: []string{channel}, + ReadyVchannels: []string{channel}, + State: internalpb.ImportJobState_Pending, }, } err = im.AddJob(context.TODO(), job) @@ -110,7 +112,7 @@ func TestImportMeta_Job(t *testing.T) { err = im.AddJob(context.TODO(), job) assert.NoError(t, err) ret = im.GetJob(context.TODO(), jobID) - assert.Equal(t, job, ret) + assert.EqualValues(t, job, ret) jobs = im.GetJobBy(context.TODO()) assert.Equal(t, i+1, len(jobs)) } @@ -149,6 +151,48 @@ func TestImportMeta_Job(t *testing.T) { assert.Equal(t, 2, count) } +func TestImportMetaAddJob(t *testing.T) { + catalog := mocks.NewDataCoordCatalog(t) + catalog.EXPECT().ListImportJobs(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListPreImportTasks(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListImportTasks(mock.Anything).Return(nil, nil) + catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil) + + im, err := NewImportMeta(context.TODO(), catalog) + assert.NoError(t, err) + + var job ImportJob = &importJob{ + ImportJob: &datapb.ImportJob{ + JobID: 10000, + CollectionID: rand.Int63(), + PartitionIDs: []int64{rand.Int63()}, + Vchannels: []string{"ch-1", "ch-2"}, + ReadyVchannels: []string{"ch-1"}, + State: internalpb.ImportJobState_Pending, + }, + } + err = im.AddJob(context.TODO(), job) + assert.NoError(t, err) + + job = &importJob{ + ImportJob: &datapb.ImportJob{ + JobID: 10000, + CollectionID: rand.Int63(), + PartitionIDs: []int64{rand.Int63()}, + Vchannels: []string{"ch-1", "ch-2"}, + ReadyVchannels: []string{"ch-2"}, + State: internalpb.ImportJobState_Pending, + }, + } + err = im.AddJob(context.TODO(), job) + assert.NoError(t, err) + + job = im.GetJob(context.TODO(), 10000) + assert.NotNil(t, job) + assert.Equal(t, []string{"ch-1", "ch-2"}, job.GetVchannels()) + assert.Equal(t, []string{"ch-1", "ch-2"}, job.GetReadyVchannels()) +} + func TestImportMeta_ImportTask(t *testing.T) { catalog := mocks.NewDataCoordCatalog(t) catalog.EXPECT().ListImportJobs(mock.Anything).Return(nil, nil) diff --git a/internal/datacoord/import_task.go b/internal/datacoord/import_task.go index e4a1a0b67c..a8c0877205 100644 --- a/internal/datacoord/import_task.go +++ b/internal/datacoord/import_task.go @@ -68,6 +68,18 @@ func WithStates(states ...datapb.ImportTaskStateV2) ImportTaskFilter { } } +func WithRequestSource() ImportTaskFilter { + return func(task ImportTask) bool { + return task.GetSource() == datapb.ImportTaskSourceV2_Request + } +} + +func WithL0CompactionSource() ImportTaskFilter { + return func(task ImportTask) bool { + return task.GetSource() == datapb.ImportTaskSourceV2_L0Compaction + } +} + type UpdateAction func(task ImportTask) func UpdateState(state datapb.ImportTaskStateV2) UpdateAction { @@ -150,6 +162,7 @@ type ImportTask interface { GetTR() *timerecord.TimeRecorder GetSlots() int64 Clone() ImportTask + GetSource() datapb.ImportTaskSourceV2 } type preImportTask struct { @@ -176,6 +189,10 @@ func (p *preImportTask) Clone() ImportTask { } } +func (p *preImportTask) GetSource() datapb.ImportTaskSourceV2 { + return datapb.ImportTaskSourceV2_Request +} + func (p *preImportTask) MarshalJSON() ([]byte, error) { importTask := metricsinfo.ImportTask{ JobID: p.GetJobID(), diff --git a/internal/datacoord/import_util.go b/internal/datacoord/import_util.go index 25ee278654..0f49bcbfe3 100644 --- a/internal/datacoord/import_util.go +++ b/internal/datacoord/import_util.go @@ -156,7 +156,7 @@ func AssignSegments(job ImportJob, task ImportTask, alloc allocator.Allocator, m defer cancel() for size > 0 { segmentInfo, err := AllocImportSegment(ctx, alloc, meta, - task.GetJobID(), task.GetTaskID(), task.GetCollectionID(), partitionID, vchannel, segmentLevel) + task.GetJobID(), task.GetTaskID(), task.GetCollectionID(), partitionID, vchannel, job.GetDataTs(), segmentLevel) if err != nil { return err } @@ -183,6 +183,7 @@ func AllocImportSegment(ctx context.Context, jobID int64, taskID int64, collectionID UniqueID, partitionID UniqueID, channelName string, + dataTimestamp uint64, level datapb.SegmentLevel, ) (*SegmentInfo, error) { log := log.Ctx(ctx) @@ -191,9 +192,12 @@ func AllocImportSegment(ctx context.Context, log.Error("failed to alloc id for import segment", zap.Error(err)) return nil, err } - ts, err := alloc.AllocTimestamp(ctx) - if err != nil { - return nil, err + ts := dataTimestamp + if ts == 0 { + ts, err = alloc.AllocTimestamp(ctx) + if err != nil { + return nil, err + } } position := &msgpb.MsgPosition{ ChannelName: channelName, @@ -263,9 +267,13 @@ func AssembleImportRequest(task ImportTask, job ImportJob, meta *meta, alloc all } ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second) defer cancel() - ts, err := alloc.AllocTimestamp(ctx) - if err != nil { - return nil, err + ts := job.GetDataTs() + var err error + if ts == 0 { + ts, err = alloc.AllocTimestamp(ctx) + if err != nil { + return nil, err + } } totalRows := lo.SumBy(task.GetFileStats(), func(stat *datapb.ImportFileStats) int64 { diff --git a/internal/datacoord/import_util_test.go b/internal/datacoord/import_util_test.go index 6ee847adcc..73ea2dc7f8 100644 --- a/internal/datacoord/import_util_test.go +++ b/internal/datacoord/import_util_test.go @@ -33,6 +33,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/broker" + broker2 "github.com/milvus-io/milvus/internal/datacoord/broker" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore/mocks" mocks2 "github.com/milvus-io/milvus/internal/mocks" @@ -41,6 +42,7 @@ import ( "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -130,6 +132,64 @@ func TestImportUtil_NewImportTasks(t *testing.T) { } } +func TestImportUtil_NewImportTasksWithDataTt(t *testing.T) { + dataSize := paramtable.Get().DataCoordCfg.SegmentMaxSize.GetAsInt64() * 1024 * 1024 + fileGroups := [][]*datapb.ImportFileStats{ + { + { + ImportFile: &internalpb.ImportFile{Id: 0, Paths: []string{"a.json"}}, + HashedStats: map[string]*datapb.PartitionImportStats{"c0": {PartitionDataSize: map[int64]int64{100: dataSize}}}, + }, + { + ImportFile: &internalpb.ImportFile{Id: 1, Paths: []string{"b.json"}}, + HashedStats: map[string]*datapb.PartitionImportStats{"c0": {PartitionDataSize: map[int64]int64{100: dataSize * 2}}}, + }, + }, + { + { + ImportFile: &internalpb.ImportFile{Id: 2, Paths: []string{"c.npy", "d.npy"}}, + HashedStats: map[string]*datapb.PartitionImportStats{"c0": {PartitionDataSize: map[int64]int64{100: dataSize}}}, + }, + { + ImportFile: &internalpb.ImportFile{Id: 3, Paths: []string{"e.npy", "f.npy"}}, + HashedStats: map[string]*datapb.PartitionImportStats{"c0": {PartitionDataSize: map[int64]int64{100: dataSize * 2}}}, + }, + }, + } + job := &importJob{ + ImportJob: &datapb.ImportJob{JobID: 1, CollectionID: 2, DataTs: 100}, + } + alloc := allocator.NewMockAllocator(t) + alloc.EXPECT().AllocN(mock.Anything).RunAndReturn(func(n int64) (int64, int64, error) { + id := rand.Int63() + return id, id + n, nil + }) + alloc.EXPECT().AllocID(mock.Anything).Return(rand.Int63(), nil) + + catalog := mocks.NewDataCoordCatalog(t) + catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListChannelCheckpoint(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListIndexes(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return(nil, nil) + catalog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil) + catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) + + broker := broker2.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{}, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) + assert.NoError(t, err) + + tasks, err := NewImportTasks(fileGroups, job, alloc, meta) + assert.NoError(t, err) + assert.Equal(t, 2, len(tasks)) + for _, task := range tasks { + segmentIDs := task.(*importTask).GetSegmentIDs() + assert.Equal(t, 3, len(segmentIDs)) + } +} + func TestImportUtil_AssembleRequest(t *testing.T) { var job ImportJob = &importJob{ ImportJob: &datapb.ImportJob{JobID: 0, CollectionID: 1, PartitionIDs: []int64{2}, Vchannels: []string{"v0"}}, @@ -198,6 +258,73 @@ func TestImportUtil_AssembleRequest(t *testing.T) { assert.Equal(t, job.GetVchannels(), importReq.GetVchannels()) } +func TestImportUtil_AssembleRequestWithDataTt(t *testing.T) { + var job ImportJob = &importJob{ + ImportJob: &datapb.ImportJob{JobID: 0, CollectionID: 1, PartitionIDs: []int64{2}, Vchannels: []string{"v0"}, DataTs: 100}, + } + + var pt ImportTask = &preImportTask{ + PreImportTask: &datapb.PreImportTask{ + JobID: 0, + TaskID: 3, + CollectionID: 1, + State: datapb.ImportTaskStateV2_Pending, + }, + } + preimportReq := AssemblePreImportRequest(pt, job) + assert.Equal(t, pt.GetJobID(), preimportReq.GetJobID()) + assert.Equal(t, pt.GetTaskID(), preimportReq.GetTaskID()) + assert.Equal(t, pt.GetCollectionID(), preimportReq.GetCollectionID()) + assert.Equal(t, job.GetPartitionIDs(), preimportReq.GetPartitionIDs()) + assert.Equal(t, job.GetVchannels(), preimportReq.GetVchannels()) + + var task ImportTask = &importTask{ + ImportTaskV2: &datapb.ImportTaskV2{ + JobID: 0, + TaskID: 4, + CollectionID: 1, + SegmentIDs: []int64{5, 6}, + }, + } + + catalog := mocks.NewDataCoordCatalog(t) + catalog.EXPECT().ListChannelCheckpoint(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListIndexes(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return(nil, nil) + catalog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil) + catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) + catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) + + alloc := allocator.NewMockAllocator(t) + alloc.EXPECT().AllocN(mock.Anything).RunAndReturn(func(n int64) (int64, int64, error) { + id := rand.Int63() + return id, id + n, nil + }) + + broker := broker2.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{}, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) + assert.NoError(t, err) + segment := &SegmentInfo{ + SegmentInfo: &datapb.SegmentInfo{ID: 5, IsImporting: true}, + } + err = meta.AddSegment(context.Background(), segment) + assert.NoError(t, err) + segment.ID = 6 + err = meta.AddSegment(context.Background(), segment) + assert.NoError(t, err) + + importReq, err := AssembleImportRequest(task, job, meta, alloc) + assert.NoError(t, err) + assert.Equal(t, task.GetJobID(), importReq.GetJobID()) + assert.Equal(t, task.GetTaskID(), importReq.GetTaskID()) + assert.Equal(t, task.GetCollectionID(), importReq.GetCollectionID()) + assert.Equal(t, job.GetPartitionIDs(), importReq.GetPartitionIDs()) + assert.Equal(t, job.GetVchannels(), importReq.GetVchannels()) +} + func TestImportUtil_RegroupImportFiles(t *testing.T) { fileNum := 4096 dataSize := paramtable.Get().DataCoordCfg.SegmentMaxSize.GetAsInt64() * 1024 * 1024 diff --git a/internal/datacoord/mock_compaction_plan_context.go b/internal/datacoord/mock_compaction_plan_context.go index ce73b0cdef..16e45b2146 100644 --- a/internal/datacoord/mock_compaction_plan_context.go +++ b/internal/datacoord/mock_compaction_plan_context.go @@ -164,6 +164,65 @@ func (_c *MockCompactionPlanContext_getCompactionInfo_Call) RunAndReturn(run fun return _c } +// getCompactionTasksNum provides a mock function with given fields: filters +func (_m *MockCompactionPlanContext) getCompactionTasksNum(filters ...compactionTaskFilter) int { + _va := make([]interface{}, len(filters)) + for _i := range filters { + _va[_i] = filters[_i] + } + var _ca []interface{} + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + if len(ret) == 0 { + panic("no return value specified for getCompactionTasksNum") + } + + var r0 int + if rf, ok := ret.Get(0).(func(...compactionTaskFilter) int); ok { + r0 = rf(filters...) + } else { + r0 = ret.Get(0).(int) + } + + return r0 +} + +// MockCompactionPlanContext_getCompactionTasksNum_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'getCompactionTasksNum' +type MockCompactionPlanContext_getCompactionTasksNum_Call struct { + *mock.Call +} + +// getCompactionTasksNum is a helper method to define mock.On call +// - filters ...compactionTaskFilter +func (_e *MockCompactionPlanContext_Expecter) getCompactionTasksNum(filters ...interface{}) *MockCompactionPlanContext_getCompactionTasksNum_Call { + return &MockCompactionPlanContext_getCompactionTasksNum_Call{Call: _e.mock.On("getCompactionTasksNum", + append([]interface{}{}, filters...)...)} +} + +func (_c *MockCompactionPlanContext_getCompactionTasksNum_Call) Run(run func(filters ...compactionTaskFilter)) *MockCompactionPlanContext_getCompactionTasksNum_Call { + _c.Call.Run(func(args mock.Arguments) { + variadicArgs := make([]compactionTaskFilter, len(args)-0) + for i, a := range args[0:] { + if a != nil { + variadicArgs[i] = a.(compactionTaskFilter) + } + } + run(variadicArgs...) + }) + return _c +} + +func (_c *MockCompactionPlanContext_getCompactionTasksNum_Call) Return(_a0 int) *MockCompactionPlanContext_getCompactionTasksNum_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockCompactionPlanContext_getCompactionTasksNum_Call) RunAndReturn(run func(...compactionTaskFilter) int) *MockCompactionPlanContext_getCompactionTasksNum_Call { + _c.Call.Return(run) + return _c +} + // getCompactionTasksNumBySignalID provides a mock function with given fields: signalID func (_m *MockCompactionPlanContext) getCompactionTasksNumBySignalID(signalID int64) int { ret := _m.Called(signalID) diff --git a/internal/datacoord/mock_trigger_manager.go b/internal/datacoord/mock_trigger_manager.go index fdb093a8e4..bf5d80c4f9 100644 --- a/internal/datacoord/mock_trigger_manager.go +++ b/internal/datacoord/mock_trigger_manager.go @@ -21,6 +21,104 @@ func (_m *MockTriggerManager) EXPECT() *MockTriggerManager_Expecter { return &MockTriggerManager_Expecter{mock: &_m.Mock} } +// GetPauseCompactionChan provides a mock function with given fields: jobID, collectionID +func (_m *MockTriggerManager) GetPauseCompactionChan(jobID int64, collectionID int64) <-chan struct{} { + ret := _m.Called(jobID, collectionID) + + if len(ret) == 0 { + panic("no return value specified for GetPauseCompactionChan") + } + + var r0 <-chan struct{} + if rf, ok := ret.Get(0).(func(int64, int64) <-chan struct{}); ok { + r0 = rf(jobID, collectionID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(<-chan struct{}) + } + } + + return r0 +} + +// MockTriggerManager_GetPauseCompactionChan_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetPauseCompactionChan' +type MockTriggerManager_GetPauseCompactionChan_Call struct { + *mock.Call +} + +// GetPauseCompactionChan is a helper method to define mock.On call +// - jobID int64 +// - collectionID int64 +func (_e *MockTriggerManager_Expecter) GetPauseCompactionChan(jobID interface{}, collectionID interface{}) *MockTriggerManager_GetPauseCompactionChan_Call { + return &MockTriggerManager_GetPauseCompactionChan_Call{Call: _e.mock.On("GetPauseCompactionChan", jobID, collectionID)} +} + +func (_c *MockTriggerManager_GetPauseCompactionChan_Call) Run(run func(jobID int64, collectionID int64)) *MockTriggerManager_GetPauseCompactionChan_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64), args[1].(int64)) + }) + return _c +} + +func (_c *MockTriggerManager_GetPauseCompactionChan_Call) Return(_a0 <-chan struct{}) *MockTriggerManager_GetPauseCompactionChan_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockTriggerManager_GetPauseCompactionChan_Call) RunAndReturn(run func(int64, int64) <-chan struct{}) *MockTriggerManager_GetPauseCompactionChan_Call { + _c.Call.Return(run) + return _c +} + +// GetResumeCompactionChan provides a mock function with given fields: jobID, collectionID +func (_m *MockTriggerManager) GetResumeCompactionChan(jobID int64, collectionID int64) <-chan struct{} { + ret := _m.Called(jobID, collectionID) + + if len(ret) == 0 { + panic("no return value specified for GetResumeCompactionChan") + } + + var r0 <-chan struct{} + if rf, ok := ret.Get(0).(func(int64, int64) <-chan struct{}); ok { + r0 = rf(jobID, collectionID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(<-chan struct{}) + } + } + + return r0 +} + +// MockTriggerManager_GetResumeCompactionChan_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetResumeCompactionChan' +type MockTriggerManager_GetResumeCompactionChan_Call struct { + *mock.Call +} + +// GetResumeCompactionChan is a helper method to define mock.On call +// - jobID int64 +// - collectionID int64 +func (_e *MockTriggerManager_Expecter) GetResumeCompactionChan(jobID interface{}, collectionID interface{}) *MockTriggerManager_GetResumeCompactionChan_Call { + return &MockTriggerManager_GetResumeCompactionChan_Call{Call: _e.mock.On("GetResumeCompactionChan", jobID, collectionID)} +} + +func (_c *MockTriggerManager_GetResumeCompactionChan_Call) Run(run func(jobID int64, collectionID int64)) *MockTriggerManager_GetResumeCompactionChan_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(int64), args[1].(int64)) + }) + return _c +} + +func (_c *MockTriggerManager_GetResumeCompactionChan_Call) Return(_a0 <-chan struct{}) *MockTriggerManager_GetResumeCompactionChan_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockTriggerManager_GetResumeCompactionChan_Call) RunAndReturn(run func(int64, int64) <-chan struct{}) *MockTriggerManager_GetResumeCompactionChan_Call { + _c.Call.Return(run) + return _c +} + // ManualTrigger provides a mock function with given fields: ctx, collectionID, clusteringCompaction func (_m *MockTriggerManager) ManualTrigger(ctx context.Context, collectionID int64, clusteringCompaction bool) (int64, error) { ret := _m.Called(ctx, collectionID, clusteringCompaction) diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 648b8796a9..61928a0b6a 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -382,6 +382,10 @@ func (s *Server) initDataCoord() error { } log.Info("init service discovery done") + s.importMeta, err = NewImportMeta(s.ctx, s.meta.catalog) + if err != nil { + return err + } s.initCompaction() log.Info("init compaction done") @@ -398,12 +402,8 @@ func (s *Server) initDataCoord() error { s.initGarbageCollection(storageCli) - s.importMeta, err = NewImportMeta(s.ctx, s.meta.catalog) - if err != nil { - return err - } s.importScheduler = NewImportScheduler(s.meta, s.cluster, s.allocator, s.importMeta) - s.importChecker = NewImportChecker(s.meta, s.broker, s.cluster, s.allocator, s.importMeta, s.jobManager) + s.importChecker = NewImportChecker(s.meta, s.broker, s.cluster, s.allocator, s.importMeta, s.jobManager, s.compactionTriggerManager) s.syncSegmentsScheduler = newSyncSegmentsScheduler(s.meta, s.channelManager, s.sessionManager) @@ -706,7 +706,7 @@ func (s *Server) initCompaction() { cph := newCompactionPlanHandler(s.cluster, s.sessionManager, s.meta, s.allocator, s.handler) cph.loadMeta() s.compactionHandler = cph - s.compactionTriggerManager = NewCompactionTriggerManager(s.allocator, s.handler, s.compactionHandler, s.meta) + s.compactionTriggerManager = NewCompactionTriggerManager(s.allocator, s.handler, s.compactionHandler, s.meta, s.importMeta) s.compactionTrigger = newCompactionTrigger(s.meta, s.compactionHandler, s.allocator, s.handler, s.indexEngineVersionManager) } diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index 58030679ec..6b2ddb0597 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -1723,16 +1723,20 @@ func (s *Server) ImportV2(ctx context.Context, in *internalpb.ImportRequestInter log.Info("list binlogs prefixes for import", zap.Any("binlog_prefixes", files)) } + // The import task does not need to be controlled for the time being, and additional development is required later. + // Here is a comment, because the current importv2 communicates through messages and needs to ensure idempotence. + // Adding this part of the logic will cause importv2 to retry infinitely until the previous import task is completed. + // Check if the number of jobs exceeds the limit. - maxNum := paramtable.Get().DataCoordCfg.MaxImportJobNum.GetAsInt() - executingNum := s.importMeta.CountJobBy(ctx, WithoutJobStates(internalpb.ImportJobState_Completed, internalpb.ImportJobState_Failed)) - if executingNum >= maxNum { - resp.Status = merr.Status(merr.WrapErrImportFailed( - fmt.Sprintf("The number of jobs has reached the limit, please try again later. " + - "If your request is set to only import a single file, " + - "please consider importing multiple files in one request for better efficiency."))) - return resp, nil - } + // maxNum := paramtable.Get().DataCoordCfg.MaxImportJobNum.GetAsInt() + // executingNum := s.importMeta.CountJobBy(ctx, WithoutJobStates(internalpb.ImportJobState_Completed, internalpb.ImportJobState_Failed)) + // if executingNum >= maxNum { + // resp.Status = merr.Status(merr.WrapErrImportFailed( + // fmt.Sprintf("The number of jobs has reached the limit, please try again later. " + + // "If your request is set to only import a single file, " + + // "please consider importing multiple files in one request for better efficiency."))) + // return resp, nil + // } // Allocate file ids. idStart, _, err := s.allocator.AllocN(int64(len(files)) + 1) @@ -1744,15 +1748,28 @@ func (s *Server) ImportV2(ctx context.Context, in *internalpb.ImportRequestInter importFile.Id = idStart + int64(i) + 1 return importFile }) + importCollectionInfo, err := s.handler.GetCollection(ctx, in.GetCollectionID()) + if err != nil { + resp.Status = merr.Status(merr.WrapErrImportFailed(fmt.Sprint("get collection failed, err=%w", err))) + return resp, nil + } + if importCollectionInfo == nil { + resp.Status = merr.Status(merr.WrapErrCollectionNotFound(in.GetCollectionID())) + return resp, nil + } + jobID := in.GetJobID() + if jobID == 0 { + jobID = idStart + } startTime := time.Now() job := &importJob{ ImportJob: &datapb.ImportJob{ - JobID: idStart, + JobID: jobID, CollectionID: in.GetCollectionID(), CollectionName: in.GetCollectionName(), PartitionIDs: in.GetPartitionIDs(), - Vchannels: in.GetChannelNames(), + Vchannels: importCollectionInfo.VChannelNames, Schema: in.GetSchema(), TimeoutTs: timeoutTs, CleanupTs: math.MaxUint64, @@ -1760,6 +1777,8 @@ func (s *Server) ImportV2(ctx context.Context, in *internalpb.ImportRequestInter Files: files, Options: in.GetOptions(), StartTime: startTime.Format("2006-01-02T15:04:05Z07:00"), + ReadyVchannels: in.GetChannelNames(), + DataTs: in.GetDataTimestamp(), }, tr: timerecord.NewTimeRecorder("import job"), } @@ -1770,7 +1789,11 @@ func (s *Server) ImportV2(ctx context.Context, in *internalpb.ImportRequestInter } resp.JobID = fmt.Sprint(job.GetJobID()) - log.Info("add import job done", zap.Int64("jobID", job.GetJobID()), zap.Any("files", files)) + log.Info("add import job done", + zap.Int64("jobID", job.GetJobID()), + zap.Any("files", files), + zap.Strings("readyChannels", in.GetChannelNames()), + ) return resp, nil } @@ -1790,6 +1813,7 @@ func (s *Server) GetImportProgress(ctx context.Context, in *internalpb.GetImport resp.Status = merr.Status(merr.WrapErrImportFailed(fmt.Sprint("parse job id failed, err=%w", err))) return resp, nil } + job := s.importMeta.GetJob(ctx, jobID) if job == nil { resp.Status = merr.Status(merr.WrapErrImportFailed(fmt.Sprintf("import job does not exist, jobID=%d", jobID))) diff --git a/internal/datacoord/services_test.go b/internal/datacoord/services_test.go index 9ee282c7b9..f3ebb030f4 100644 --- a/internal/datacoord/services_test.go +++ b/internal/datacoord/services_test.go @@ -22,9 +22,11 @@ import ( "github.com/milvus-io/milvus/internal/datacoord/allocator" "github.com/milvus-io/milvus/internal/datacoord/broker" "github.com/milvus-io/milvus/internal/datacoord/session" + "github.com/milvus-io/milvus/internal/distributed/streaming" "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" mocks2 "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/mq/msgstream" @@ -1313,6 +1315,12 @@ func TestImportV2(t *testing.T) { assert.NoError(t, err) assert.NotEqual(t, int32(0), resp.GetStatus().GetCode()) s.stateCode.Store(commonpb.StateCode_Healthy) + mockHandler := NewNMockHandler(t) + mockHandler.EXPECT().GetCollection(mock.Anything, mock.Anything).Return(&collectionInfo{ + ID: 1000, + VChannelNames: []string{"foo_1v1"}, + }, nil).Maybe() + s.handler = mockHandler // parse timeout failed resp, err = s.ImportV2(ctx, &internalpb.ImportRequestInternal{ @@ -1397,6 +1405,7 @@ func TestImportV2(t *testing.T) { Paths: []string{"a.json"}, }, }, + ChannelNames: []string{"foo_1v1"}, }) assert.NoError(t, err) assert.Equal(t, int32(0), resp.GetStatus().GetCode()) @@ -1404,11 +1413,11 @@ func TestImportV2(t *testing.T) { assert.Equal(t, 1, len(jobs)) // number of jobs reached the limit - Params.Save(paramtable.Get().DataCoordCfg.MaxImportJobNum.Key, "1") - resp, err = s.ImportV2(ctx, &internalpb.ImportRequestInternal{}) - assert.NoError(t, err) - assert.True(t, errors.Is(merr.Error(resp.GetStatus()), merr.ErrImportFailed)) - Params.Reset(paramtable.Get().DataCoordCfg.MaxImportJobNum.Key) + // Params.Save(paramtable.Get().DataCoordCfg.MaxImportJobNum.Key, "1") + // resp, err = s.ImportV2(ctx, &internalpb.ImportRequestInternal{}) + // assert.NoError(t, err) + // assert.True(t, errors.Is(merr.Error(resp.GetStatus()), merr.ErrImportFailed)) + // Params.Reset(paramtable.Get().DataCoordCfg.MaxImportJobNum.Key) }) t.Run("GetImportProgress", func(t *testing.T) { @@ -1433,6 +1442,13 @@ func TestImportV2(t *testing.T) { catalog.EXPECT().ListPreImportTasks(mock.Anything).Return(nil, nil) catalog.EXPECT().ListImportTasks(mock.Anything).Return(nil, nil) catalog.EXPECT().SaveImportJob(mock.Anything, mock.Anything).Return(nil) + wal := mock_streaming.NewMockWALAccesser(t) + b := mock_streaming.NewMockBroadcast(t) + wal.EXPECT().Broadcast().Return(b).Maybe() + b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe() + streaming.SetWALForTest(wal) + defer streaming.RecoverWALForTest() + s.importMeta, err = NewImportMeta(context.TODO(), catalog) assert.NoError(t, err) resp, err = s.GetImportProgress(ctx, &internalpb.GetImportProgressRequest{ diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index bba865fbaf..5e9b40dd87 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -40,6 +40,7 @@ import ( "github.com/milvus-io/milvus/internal/datanode/channel" "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/datanode/importv2" + "github.com/milvus-io/milvus/internal/datanode/msghandlerimpl" "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/pipeline" @@ -470,5 +471,6 @@ func getPipelineParams(node *DataNode) *util2.PipelineParams { WriteBufferManager: node.writeBufferManager, CheckpointUpdater: node.channelCheckpointUpdater, Allocator: node.allocator, + MsgHandler: msghandlerimpl.NewMsgHandlerImpl(node.broker), } } diff --git a/internal/datanode/msghandlerimpl/msg_handler_impl.go b/internal/datanode/msghandlerimpl/msg_handler_impl.go new file mode 100644 index 0000000000..c8c6c3a681 --- /dev/null +++ b/internal/datanode/msghandlerimpl/msg_handler_impl.go @@ -0,0 +1,96 @@ +/* + * 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 msghandlerimpl + +import ( + "context" + + "github.com/cockroachdb/errors" + "github.com/samber/lo" + "go.uber.org/zap" + + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus/internal/distributed/streaming" + "github.com/milvus-io/milvus/internal/flushcommon/broker" + "github.com/milvus-io/milvus/internal/flushcommon/util" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/util/funcutil" + "github.com/milvus-io/milvus/pkg/util/merr" + "github.com/milvus-io/milvus/pkg/util/retry" +) + +type msgHandlerImpl struct { + broker broker.Broker +} + +func (m *msgHandlerImpl) HandleCreateSegment(ctx context.Context, vchannel string, createSegmentMsg message.ImmutableCreateSegmentMessageV2) error { + panic("unreachable code") +} + +func (m *msgHandlerImpl) HandleFlush(vchannel string, flushMsg message.ImmutableFlushMessageV2) error { + panic("unreachable code") +} + +func (m *msgHandlerImpl) HandleManualFlush(vchannel string, flushMsg message.ImmutableManualFlushMessageV2) error { + panic("unreachable code") +} + +func (m *msgHandlerImpl) HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error { + return retry.Do(ctx, func() (err error) { + defer func() { + if err == nil { + err = streaming.WAL().Broadcast().Ack(ctx, types.BroadcastAckRequest{ + BroadcastID: uint64(importMsg.GetJobID()), + VChannel: vchannel, + }) + } + }() + importResp, err := m.broker.ImportV2(ctx, &internalpb.ImportRequestInternal{ + CollectionID: importMsg.GetCollectionID(), + CollectionName: importMsg.GetCollectionName(), + PartitionIDs: importMsg.GetPartitionIDs(), + ChannelNames: []string{vchannel}, + Schema: importMsg.GetSchema(), + Files: lo.Map(importMsg.GetFiles(), util.ConvertInternalImportFile), + Options: funcutil.Map2KeyValuePair(importMsg.GetOptions()), + DataTimestamp: importMsg.GetBase().GetTimestamp(), + JobID: importMsg.GetJobID(), + }) + err = merr.CheckRPCCall(importResp, err) + if errors.Is(err, merr.ErrCollectionNotFound) { + log.Ctx(ctx).Warn("import message failed because of collection not found, skip it", zap.String("job_id", importResp.GetJobID()), zap.Error(err)) + return nil + } + if err != nil { + log.Ctx(ctx).Warn("import message failed", zap.String("job_id", importResp.GetJobID()), zap.Error(err)) + return err + } + log.Ctx(ctx).Info("import message handled", zap.String("job_id", importResp.GetJobID())) + return nil + }, retry.AttemptAlways()) +} + +func NewMsgHandlerImpl(broker broker.Broker) *msgHandlerImpl { + return &msgHandlerImpl{ + broker: broker, + } +} diff --git a/internal/datanode/msghandlerimpl/msg_handler_impl_test.go b/internal/datanode/msghandlerimpl/msg_handler_impl_test.go new file mode 100644 index 0000000000..5b7367b4d4 --- /dev/null +++ b/internal/datanode/msghandlerimpl/msg_handler_impl_test.go @@ -0,0 +1,61 @@ +/* + * 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 msghandlerimpl + +import ( + "context" + "testing" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + + "github.com/milvus-io/milvus/internal/distributed/streaming" + "github.com/milvus-io/milvus/internal/flushcommon/broker" + "github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming" + "github.com/milvus-io/milvus/pkg/util/paramtable" +) + +func TestMsgHandlerImpl(t *testing.T) { + paramtable.Init() + ctx := context.Background() + b := broker.NewMockBroker(t) + m := NewMsgHandlerImpl(b) + assert.Panics(t, func() { + m.HandleCreateSegment(nil, "", nil) + }) + assert.Panics(t, func() { + m.HandleFlush("", nil) + }) + assert.Panics(t, func() { + m.HandleManualFlush("", nil) + }) + t.Run("HandleImport success", func(t *testing.T) { + wal := mock_streaming.NewMockWALAccesser(t) + bo := mock_streaming.NewMockBroadcast(t) + wal.EXPECT().Broadcast().Return(bo) + bo.EXPECT().Ack(mock.Anything, mock.Anything).Return(nil) + streaming.SetWALForTest(wal) + defer streaming.RecoverWALForTest() + + b.EXPECT().ImportV2(mock.Anything, mock.Anything).Return(nil, assert.AnError).Once() + b.EXPECT().ImportV2(mock.Anything, mock.Anything).Return(nil, nil).Once() + err := m.HandleImport(ctx, "", nil) + assert.NoError(t, err) + }) +} diff --git a/internal/distributed/streaming/test_streaming.go b/internal/distributed/streaming/test_streaming.go index bf878d86fa..ae045ca9f9 100644 --- a/internal/distributed/streaming/test_streaming.go +++ b/internal/distributed/streaming/test_streaming.go @@ -18,7 +18,14 @@ package streaming +import kvfactory "github.com/milvus-io/milvus/internal/util/dependency/kv" + // SetWALForTest initializes the singleton of wal for test. func SetWALForTest(w WALAccesser) { singleton = w } + +func RecoverWALForTest() { + c, _ := kvfactory.GetEtcdAndPath() + singleton = newWALAccesser(c) +} diff --git a/internal/flushcommon/broker/broker.go b/internal/flushcommon/broker/broker.go index 41091c6323..5af668ea29 100644 --- a/internal/flushcommon/broker/broker.go +++ b/internal/flushcommon/broker/broker.go @@ -6,6 +6,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -38,4 +39,5 @@ type DataCoord interface { SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPathsRequest) error DropVirtualChannel(ctx context.Context, req *datapb.DropVirtualChannelRequest) (*datapb.DropVirtualChannelResponse, error) UpdateSegmentStatistics(ctx context.Context, req *datapb.UpdateSegmentStatisticsRequest) error + ImportV2(ctx context.Context, in *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error) } diff --git a/internal/flushcommon/broker/datacoord.go b/internal/flushcommon/broker/datacoord.go index 070c88de75..3be9e374e0 100644 --- a/internal/flushcommon/broker/datacoord.go +++ b/internal/flushcommon/broker/datacoord.go @@ -14,6 +14,7 @@ import ( "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -166,3 +167,13 @@ func (dc *dataCoordBroker) UpdateSegmentStatistics(ctx context.Context, req *dat return nil } + +func (dc *dataCoordBroker) ImportV2(ctx context.Context, in *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error) { + resp, err := dc.client.ImportV2(ctx, in) + if err := merr.CheckRPCCall(resp, err); err != nil { + log.Ctx(ctx).Warn("failed to ImportV2", zap.Error(err)) + return resp, err + } + + return resp, nil +} diff --git a/internal/flushcommon/broker/datacoord_test.go b/internal/flushcommon/broker/datacoord_test.go index 5b7aca1af9..31d6df389e 100644 --- a/internal/flushcommon/broker/datacoord_test.go +++ b/internal/flushcommon/broker/datacoord_test.go @@ -15,6 +15,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tsoutil" @@ -328,6 +329,28 @@ func (s *dataCoordSuite) TestUpdateSegmentStatistics() { }) } +func (s *dataCoordSuite) TestImportV2() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + req := &internalpb.ImportRequestInternal{} + + s.Run("normal_case", func() { + s.dc.EXPECT().ImportV2(mock.Anything, mock.Anything). + Return(&internalpb.ImportResponse{Status: merr.Status(nil), JobID: "1000"}, nil) + resp, err := s.broker.ImportV2(ctx, req) + s.NoError(err) + s.Equal("1000", resp.GetJobID()) + s.resetMock() + }) + s.Run("datacoord_return_error", func() { + s.dc.EXPECT().ImportV2(mock.Anything, mock.Anything). + Return(nil, errors.New("mock")) + _, err := s.broker.ImportV2(ctx, req) + s.Error(err) + s.resetMock() + }) +} + func TestDataCoordBroker(t *testing.T) { suite.Run(t, new(dataCoordSuite)) } diff --git a/internal/flushcommon/broker/mock_broker.go b/internal/flushcommon/broker/mock_broker.go index c55a031770..97ea094914 100644 --- a/internal/flushcommon/broker/mock_broker.go +++ b/internal/flushcommon/broker/mock_broker.go @@ -6,6 +6,8 @@ import ( context "context" datapb "github.com/milvus-io/milvus/pkg/proto/datapb" + internalpb "github.com/milvus-io/milvus/pkg/proto/internalpb" + mock "github.com/stretchr/testify/mock" msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" @@ -215,6 +217,65 @@ func (_c *MockBroker_GetSegmentInfo_Call) RunAndReturn(run func(context.Context, return _c } +// ImportV2 provides a mock function with given fields: ctx, in +func (_m *MockBroker) ImportV2(ctx context.Context, in *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error) { + ret := _m.Called(ctx, in) + + if len(ret) == 0 { + panic("no return value specified for ImportV2") + } + + var r0 *internalpb.ImportResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error)); ok { + return rf(ctx, in) + } + if rf, ok := ret.Get(0).(func(context.Context, *internalpb.ImportRequestInternal) *internalpb.ImportResponse); ok { + r0 = rf(ctx, in) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*internalpb.ImportResponse) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *internalpb.ImportRequestInternal) error); ok { + r1 = rf(ctx, in) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockBroker_ImportV2_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ImportV2' +type MockBroker_ImportV2_Call struct { + *mock.Call +} + +// ImportV2 is a helper method to define mock.On call +// - ctx context.Context +// - in *internalpb.ImportRequestInternal +func (_e *MockBroker_Expecter) ImportV2(ctx interface{}, in interface{}) *MockBroker_ImportV2_Call { + return &MockBroker_ImportV2_Call{Call: _e.mock.On("ImportV2", ctx, in)} +} + +func (_c *MockBroker_ImportV2_Call) Run(run func(ctx context.Context, in *internalpb.ImportRequestInternal)) *MockBroker_ImportV2_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*internalpb.ImportRequestInternal)) + }) + return _c +} + +func (_c *MockBroker_ImportV2_Call) Return(_a0 *internalpb.ImportResponse, _a1 error) *MockBroker_ImportV2_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockBroker_ImportV2_Call) RunAndReturn(run func(context.Context, *internalpb.ImportRequestInternal) (*internalpb.ImportResponse, error)) *MockBroker_ImportV2_Call { + _c.Call.Return(run) + return _c +} + // ReportTimeTick provides a mock function with given fields: ctx, msgs func (_m *MockBroker) ReportTimeTick(ctx context.Context, msgs []*msgpb.DataNodeTtMsg) error { ret := _m.Called(ctx, msgs) diff --git a/internal/flushcommon/pipeline/flow_graph_dd_node.go b/internal/flushcommon/pipeline/flow_graph_dd_node.go index 523cfcdb29..f9cd966868 100644 --- a/internal/flushcommon/pipeline/flow_graph_dd_node.go +++ b/internal/flushcommon/pipeline/flow_graph_dd_node.go @@ -30,7 +30,6 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/flushcommon/util" - "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/log" @@ -71,7 +70,7 @@ type ddNode struct { dropMode atomic.Value compactionExecutor compaction.Executor - msgHandler flusher.MsgHandler + msgHandler util.MsgHandler // for recovery growingSegInfo map[typeutil.UniqueID]*datapb.SegmentInfo // segmentID @@ -277,6 +276,21 @@ func (ddn *ddNode) Operate(in []Msg) []Msg { } else { logger.Info("handle manual flush message success") } + case commonpb.MsgType_Import: + importMsg := msg.(*msgstream.ImportMsg) + if importMsg.GetCollectionID() != ddn.collectionID { + continue + } + logger := log.With( + zap.String("vchannel", ddn.Name()), + zap.Int32("msgType", int32(msg.Type())), + ) + logger.Info("receive import message") + if err := ddn.msgHandler.HandleImport(context.Background(), ddn.vChannelName, importMsg.ImportMsg); err != nil { + logger.Warn("handle import message failed", zap.Error(err)) + } else { + logger.Info("handle import message success") + } } } @@ -332,7 +346,7 @@ func (ddn *ddNode) Close() { } func newDDNode(ctx context.Context, collID typeutil.UniqueID, vChannelName string, droppedSegmentIDs []typeutil.UniqueID, - sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, executor compaction.Executor, handler flusher.MsgHandler, + sealedSegments []*datapb.SegmentInfo, growingSegments []*datapb.SegmentInfo, executor compaction.Executor, handler util.MsgHandler, ) *ddNode { baseNode := BaseNode{} baseNode.SetMaxQueueLength(paramtable.Get().DataNodeCfg.FlowGraphMaxQueueLength.GetAsInt32()) diff --git a/internal/flushcommon/pipeline/flow_graph_dd_node_test.go b/internal/flushcommon/pipeline/flow_graph_dd_node_test.go index 053b5e9df5..61e18a6127 100644 --- a/internal/flushcommon/pipeline/flow_graph_dd_node_test.go +++ b/internal/flushcommon/pipeline/flow_graph_dd_node_test.go @@ -28,7 +28,7 @@ import ( "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/datanode/compaction" - "github.com/milvus-io/milvus/internal/mocks/streamingnode/server/mock_flusher" + "github.com/milvus-io/milvus/internal/mocks/flushcommon/mock_util" "github.com/milvus-io/milvus/internal/util/flowgraph" "github.com/milvus-io/milvus/pkg/mocks/streaming/util/mock_message" "github.com/milvus-io/milvus/pkg/mq/msgstream" @@ -97,7 +97,7 @@ func TestFlowGraph_DDNode_newDDNode(t *testing.T) { } func TestFlowGraph_DDNode_OperateFlush(t *testing.T) { - h := mock_flusher.NewMockFlushMsgHandler(t) + h := mock_util.NewMockMsgHandler(t) h.EXPECT().HandleCreateSegment(mock.Anything, mock.Anything, mock.Anything).Return(nil) h.EXPECT().HandleFlush(mock.Anything, mock.Anything).Return(nil) h.EXPECT().HandleManualFlush(mock.Anything, mock.Anything).Return(nil) diff --git a/internal/streamingnode/server/flusher/msg_handler.go b/internal/flushcommon/util/msg_handler.go similarity index 75% rename from internal/streamingnode/server/flusher/msg_handler.go rename to internal/flushcommon/util/msg_handler.go index 784b32235c..d976ebac7b 100644 --- a/internal/streamingnode/server/flusher/msg_handler.go +++ b/internal/flushcommon/util/msg_handler.go @@ -14,11 +14,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -package flusher +package util import ( "context" + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" ) @@ -28,4 +30,13 @@ type MsgHandler interface { HandleFlush(vchannel string, flushMsg message.ImmutableFlushMessageV2) error HandleManualFlush(vchannel string, flushMsg message.ImmutableManualFlushMessageV2) error + + HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error +} + +func ConvertInternalImportFile(file *msgpb.ImportFile, _ int) *internalpb.ImportFile { + return &internalpb.ImportFile{ + Id: file.GetId(), + Paths: file.GetPaths(), + } } diff --git a/internal/flushcommon/util/util.go b/internal/flushcommon/util/util.go index 65765e8146..273586a11c 100644 --- a/internal/flushcommon/util/util.go +++ b/internal/flushcommon/util/util.go @@ -28,7 +28,6 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/storage" - "github.com/milvus-io/milvus/internal/streamingnode/server/flusher" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" @@ -49,7 +48,7 @@ type PipelineParams struct { WriteBufferManager writebuffer.BufferManager CheckpointUpdater *ChannelCheckpointUpdater Allocator allocator.Interface - MsgHandler flusher.MsgHandler + MsgHandler MsgHandler } // TimeRange is a range of timestamp contains the min-timestamp and max-timestamp diff --git a/internal/mocks/distributed/mock_streaming/mock_Broadcast.go b/internal/mocks/distributed/mock_streaming/mock_Broadcast.go new file mode 100644 index 0000000000..6cb94f17a0 --- /dev/null +++ b/internal/mocks/distributed/mock_streaming/mock_Broadcast.go @@ -0,0 +1,239 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package mock_streaming + +import ( + context "context" + + message "github.com/milvus-io/milvus/pkg/streaming/util/message" + mock "github.com/stretchr/testify/mock" + + types "github.com/milvus-io/milvus/pkg/streaming/util/types" +) + +// MockBroadcast is an autogenerated mock type for the Broadcast type +type MockBroadcast struct { + mock.Mock +} + +type MockBroadcast_Expecter struct { + mock *mock.Mock +} + +func (_m *MockBroadcast) EXPECT() *MockBroadcast_Expecter { + return &MockBroadcast_Expecter{mock: &_m.Mock} +} + +// Ack provides a mock function with given fields: ctx, req +func (_m *MockBroadcast) Ack(ctx context.Context, req types.BroadcastAckRequest) error { + ret := _m.Called(ctx, req) + + if len(ret) == 0 { + panic("no return value specified for Ack") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, types.BroadcastAckRequest) error); ok { + r0 = rf(ctx, req) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockBroadcast_Ack_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Ack' +type MockBroadcast_Ack_Call struct { + *mock.Call +} + +// Ack is a helper method to define mock.On call +// - ctx context.Context +// - req types.BroadcastAckRequest +func (_e *MockBroadcast_Expecter) Ack(ctx interface{}, req interface{}) *MockBroadcast_Ack_Call { + return &MockBroadcast_Ack_Call{Call: _e.mock.On("Ack", ctx, req)} +} + +func (_c *MockBroadcast_Ack_Call) Run(run func(ctx context.Context, req types.BroadcastAckRequest)) *MockBroadcast_Ack_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(types.BroadcastAckRequest)) + }) + return _c +} + +func (_c *MockBroadcast_Ack_Call) Return(_a0 error) *MockBroadcast_Ack_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockBroadcast_Ack_Call) RunAndReturn(run func(context.Context, types.BroadcastAckRequest) error) *MockBroadcast_Ack_Call { + _c.Call.Return(run) + return _c +} + +// Append provides a mock function with given fields: ctx, msg +func (_m *MockBroadcast) Append(ctx context.Context, msg message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error) { + ret := _m.Called(ctx, msg) + + if len(ret) == 0 { + panic("no return value specified for Append") + } + + var r0 *types.BroadcastAppendResult + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)); ok { + return rf(ctx, msg) + } + if rf, ok := ret.Get(0).(func(context.Context, message.BroadcastMutableMessage) *types.BroadcastAppendResult); ok { + r0 = rf(ctx, msg) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*types.BroadcastAppendResult) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, message.BroadcastMutableMessage) error); ok { + r1 = rf(ctx, msg) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockBroadcast_Append_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Append' +type MockBroadcast_Append_Call struct { + *mock.Call +} + +// Append is a helper method to define mock.On call +// - ctx context.Context +// - msg message.BroadcastMutableMessage +func (_e *MockBroadcast_Expecter) Append(ctx interface{}, msg interface{}) *MockBroadcast_Append_Call { + return &MockBroadcast_Append_Call{Call: _e.mock.On("Append", ctx, msg)} +} + +func (_c *MockBroadcast_Append_Call) Run(run func(ctx context.Context, msg message.BroadcastMutableMessage)) *MockBroadcast_Append_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(message.BroadcastMutableMessage)) + }) + return _c +} + +func (_c *MockBroadcast_Append_Call) Return(_a0 *types.BroadcastAppendResult, _a1 error) *MockBroadcast_Append_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockBroadcast_Append_Call) RunAndReturn(run func(context.Context, message.BroadcastMutableMessage) (*types.BroadcastAppendResult, error)) *MockBroadcast_Append_Call { + _c.Call.Return(run) + return _c +} + +// BlockUntilResourceKeyAckAll provides a mock function with given fields: ctx, rk +func (_m *MockBroadcast) BlockUntilResourceKeyAckAll(ctx context.Context, rk message.ResourceKey) error { + ret := _m.Called(ctx, rk) + + if len(ret) == 0 { + panic("no return value specified for BlockUntilResourceKeyAckAll") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, message.ResourceKey) error); ok { + r0 = rf(ctx, rk) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockBroadcast_BlockUntilResourceKeyAckAll_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BlockUntilResourceKeyAckAll' +type MockBroadcast_BlockUntilResourceKeyAckAll_Call struct { + *mock.Call +} + +// BlockUntilResourceKeyAckAll is a helper method to define mock.On call +// - ctx context.Context +// - rk message.ResourceKey +func (_e *MockBroadcast_Expecter) BlockUntilResourceKeyAckAll(ctx interface{}, rk interface{}) *MockBroadcast_BlockUntilResourceKeyAckAll_Call { + return &MockBroadcast_BlockUntilResourceKeyAckAll_Call{Call: _e.mock.On("BlockUntilResourceKeyAckAll", ctx, rk)} +} + +func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) Run(run func(ctx context.Context, rk message.ResourceKey)) *MockBroadcast_BlockUntilResourceKeyAckAll_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(message.ResourceKey)) + }) + return _c +} + +func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) Return(_a0 error) *MockBroadcast_BlockUntilResourceKeyAckAll_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockBroadcast_BlockUntilResourceKeyAckAll_Call) RunAndReturn(run func(context.Context, message.ResourceKey) error) *MockBroadcast_BlockUntilResourceKeyAckAll_Call { + _c.Call.Return(run) + return _c +} + +// BlockUntilResourceKeyAckOnce provides a mock function with given fields: ctx, rk +func (_m *MockBroadcast) BlockUntilResourceKeyAckOnce(ctx context.Context, rk message.ResourceKey) error { + ret := _m.Called(ctx, rk) + + if len(ret) == 0 { + panic("no return value specified for BlockUntilResourceKeyAckOnce") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, message.ResourceKey) error); ok { + r0 = rf(ctx, rk) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockBroadcast_BlockUntilResourceKeyAckOnce_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'BlockUntilResourceKeyAckOnce' +type MockBroadcast_BlockUntilResourceKeyAckOnce_Call struct { + *mock.Call +} + +// BlockUntilResourceKeyAckOnce is a helper method to define mock.On call +// - ctx context.Context +// - rk message.ResourceKey +func (_e *MockBroadcast_Expecter) BlockUntilResourceKeyAckOnce(ctx interface{}, rk interface{}) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call { + return &MockBroadcast_BlockUntilResourceKeyAckOnce_Call{Call: _e.mock.On("BlockUntilResourceKeyAckOnce", ctx, rk)} +} + +func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) Run(run func(ctx context.Context, rk message.ResourceKey)) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(message.ResourceKey)) + }) + return _c +} + +func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) Return(_a0 error) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockBroadcast_BlockUntilResourceKeyAckOnce_Call) RunAndReturn(run func(context.Context, message.ResourceKey) error) *MockBroadcast_BlockUntilResourceKeyAckOnce_Call { + _c.Call.Return(run) + return _c +} + +// NewMockBroadcast creates a new instance of MockBroadcast. 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 NewMockBroadcast(t interface { + mock.TestingT + Cleanup(func()) +}) *MockBroadcast { + mock := &MockBroadcast{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/mocks/flushcommon/mock_util/mock_MsgHandler.go b/internal/mocks/flushcommon/mock_util/mock_MsgHandler.go new file mode 100644 index 0000000000..8103238813 --- /dev/null +++ b/internal/mocks/flushcommon/mock_util/mock_MsgHandler.go @@ -0,0 +1,230 @@ +// Code generated by mockery v2.46.0. DO NOT EDIT. + +package mock_util + +import ( + context "context" + + message "github.com/milvus-io/milvus/pkg/streaming/util/message" + + mock "github.com/stretchr/testify/mock" + + msgpb "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" +) + +// MockMsgHandler is an autogenerated mock type for the MsgHandler type +type MockMsgHandler struct { + mock.Mock +} + +type MockMsgHandler_Expecter struct { + mock *mock.Mock +} + +func (_m *MockMsgHandler) EXPECT() *MockMsgHandler_Expecter { + return &MockMsgHandler_Expecter{mock: &_m.Mock} +} + +// HandleCreateSegment provides a mock function with given fields: ctx, vchannel, createSegmentMsg +func (_m *MockMsgHandler) HandleCreateSegment(ctx context.Context, vchannel string, createSegmentMsg message.ImmutableCreateSegmentMessageV2) error { + ret := _m.Called(ctx, vchannel, createSegmentMsg) + + if len(ret) == 0 { + panic("no return value specified for HandleCreateSegment") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, string, message.ImmutableCreateSegmentMessageV2) error); ok { + r0 = rf(ctx, vchannel, createSegmentMsg) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockMsgHandler_HandleCreateSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleCreateSegment' +type MockMsgHandler_HandleCreateSegment_Call struct { + *mock.Call +} + +// HandleCreateSegment is a helper method to define mock.On call +// - ctx context.Context +// - vchannel string +// - createSegmentMsg message.specializedImmutableMessage[*messagespb.CreateSegmentMessageHeader,*messagespb.CreateSegmentMessageBody] +func (_e *MockMsgHandler_Expecter) HandleCreateSegment(ctx interface{}, vchannel interface{}, createSegmentMsg interface{}) *MockMsgHandler_HandleCreateSegment_Call { + return &MockMsgHandler_HandleCreateSegment_Call{Call: _e.mock.On("HandleCreateSegment", ctx, vchannel, createSegmentMsg)} +} + +func (_c *MockMsgHandler_HandleCreateSegment_Call) Run(run func(ctx context.Context, vchannel string, createSegmentMsg message.ImmutableCreateSegmentMessageV2)) *MockMsgHandler_HandleCreateSegment_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string), args[2].(message.ImmutableCreateSegmentMessageV2)) + }) + return _c +} + +func (_c *MockMsgHandler_HandleCreateSegment_Call) Return(_a0 error) *MockMsgHandler_HandleCreateSegment_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockMsgHandler_HandleCreateSegment_Call) RunAndReturn(run func(context.Context, string, message.ImmutableCreateSegmentMessageV2) error) *MockMsgHandler_HandleCreateSegment_Call { + _c.Call.Return(run) + return _c +} + +// HandleFlush provides a mock function with given fields: vchannel, flushMsg +func (_m *MockMsgHandler) HandleFlush(vchannel string, flushMsg message.ImmutableFlushMessageV2) error { + ret := _m.Called(vchannel, flushMsg) + + if len(ret) == 0 { + panic("no return value specified for HandleFlush") + } + + var r0 error + if rf, ok := ret.Get(0).(func(string, message.ImmutableFlushMessageV2) error); ok { + r0 = rf(vchannel, flushMsg) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockMsgHandler_HandleFlush_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleFlush' +type MockMsgHandler_HandleFlush_Call struct { + *mock.Call +} + +// HandleFlush is a helper method to define mock.On call +// - vchannel string +// - flushMsg message.specializedImmutableMessage[*messagespb.FlushMessageHeader,*messagespb.FlushMessageBody] +func (_e *MockMsgHandler_Expecter) HandleFlush(vchannel interface{}, flushMsg interface{}) *MockMsgHandler_HandleFlush_Call { + return &MockMsgHandler_HandleFlush_Call{Call: _e.mock.On("HandleFlush", vchannel, flushMsg)} +} + +func (_c *MockMsgHandler_HandleFlush_Call) Run(run func(vchannel string, flushMsg message.ImmutableFlushMessageV2)) *MockMsgHandler_HandleFlush_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(string), args[1].(message.ImmutableFlushMessageV2)) + }) + return _c +} + +func (_c *MockMsgHandler_HandleFlush_Call) Return(_a0 error) *MockMsgHandler_HandleFlush_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockMsgHandler_HandleFlush_Call) RunAndReturn(run func(string, message.ImmutableFlushMessageV2) error) *MockMsgHandler_HandleFlush_Call { + _c.Call.Return(run) + return _c +} + +// HandleImport provides a mock function with given fields: ctx, vchannel, importMsg +func (_m *MockMsgHandler) HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error { + ret := _m.Called(ctx, vchannel, importMsg) + + if len(ret) == 0 { + panic("no return value specified for HandleImport") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, string, *msgpb.ImportMsg) error); ok { + r0 = rf(ctx, vchannel, importMsg) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockMsgHandler_HandleImport_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleImport' +type MockMsgHandler_HandleImport_Call struct { + *mock.Call +} + +// HandleImport is a helper method to define mock.On call +// - ctx context.Context +// - vchannel string +// - importMsg *msgpb.ImportMsg +func (_e *MockMsgHandler_Expecter) HandleImport(ctx interface{}, vchannel interface{}, importMsg interface{}) *MockMsgHandler_HandleImport_Call { + return &MockMsgHandler_HandleImport_Call{Call: _e.mock.On("HandleImport", ctx, vchannel, importMsg)} +} + +func (_c *MockMsgHandler_HandleImport_Call) Run(run func(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg)) *MockMsgHandler_HandleImport_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string), args[2].(*msgpb.ImportMsg)) + }) + return _c +} + +func (_c *MockMsgHandler_HandleImport_Call) Return(_a0 error) *MockMsgHandler_HandleImport_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockMsgHandler_HandleImport_Call) RunAndReturn(run func(context.Context, string, *msgpb.ImportMsg) error) *MockMsgHandler_HandleImport_Call { + _c.Call.Return(run) + return _c +} + +// HandleManualFlush provides a mock function with given fields: vchannel, flushMsg +func (_m *MockMsgHandler) HandleManualFlush(vchannel string, flushMsg message.ImmutableManualFlushMessageV2) error { + ret := _m.Called(vchannel, flushMsg) + + if len(ret) == 0 { + panic("no return value specified for HandleManualFlush") + } + + var r0 error + if rf, ok := ret.Get(0).(func(string, message.ImmutableManualFlushMessageV2) error); ok { + r0 = rf(vchannel, flushMsg) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// MockMsgHandler_HandleManualFlush_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleManualFlush' +type MockMsgHandler_HandleManualFlush_Call struct { + *mock.Call +} + +// HandleManualFlush is a helper method to define mock.On call +// - vchannel string +// - flushMsg message.specializedImmutableMessage[*messagespb.ManualFlushMessageHeader,*messagespb.ManualFlushMessageBody] +func (_e *MockMsgHandler_Expecter) HandleManualFlush(vchannel interface{}, flushMsg interface{}) *MockMsgHandler_HandleManualFlush_Call { + return &MockMsgHandler_HandleManualFlush_Call{Call: _e.mock.On("HandleManualFlush", vchannel, flushMsg)} +} + +func (_c *MockMsgHandler_HandleManualFlush_Call) Run(run func(vchannel string, flushMsg message.ImmutableManualFlushMessageV2)) *MockMsgHandler_HandleManualFlush_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(string), args[1].(message.ImmutableManualFlushMessageV2)) + }) + return _c +} + +func (_c *MockMsgHandler_HandleManualFlush_Call) Return(_a0 error) *MockMsgHandler_HandleManualFlush_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockMsgHandler_HandleManualFlush_Call) RunAndReturn(run func(string, message.ImmutableManualFlushMessageV2) error) *MockMsgHandler_HandleManualFlush_Call { + _c.Call.Return(run) + return _c +} + +// NewMockMsgHandler creates a new instance of MockMsgHandler. 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 NewMockMsgHandler(t interface { + mock.TestingT + Cleanup(func()) +}) *MockMsgHandler { + mock := &MockMsgHandler{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/mocks/streamingnode/server/mock_flusher/mock_FlushMsgHandler.go b/internal/mocks/streamingnode/server/mock_flusher/mock_FlushMsgHandler.go deleted file mode 100644 index b2e6662497..0000000000 --- a/internal/mocks/streamingnode/server/mock_flusher/mock_FlushMsgHandler.go +++ /dev/null @@ -1,180 +0,0 @@ -// Code generated by mockery v2.46.0. DO NOT EDIT. - -package mock_flusher - -import ( - context "context" - - "github.com/milvus-io/milvus/pkg/streaming/util/message" - - mock "github.com/stretchr/testify/mock" -) - -// MockFlushMsgHandler is an autogenerated mock type for the FlushMsgHandler type -type MockFlushMsgHandler struct { - mock.Mock -} - -type MockFlushMsgHandler_Expecter struct { - mock *mock.Mock -} - -func (_m *MockFlushMsgHandler) EXPECT() *MockFlushMsgHandler_Expecter { - return &MockFlushMsgHandler_Expecter{mock: &_m.Mock} -} - -// HandleCreateSegment provides a mock function with given fields: ctx, vchannel, createSegmentMsg -func (_m *MockFlushMsgHandler) HandleCreateSegment(ctx context.Context, vchannel string, createSegmentMsg message.ImmutableCreateSegmentMessageV2) error { - ret := _m.Called(ctx, vchannel, createSegmentMsg) - - if len(ret) == 0 { - panic("no return value specified for HandleCreateSegment") - } - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, string, message.ImmutableCreateSegmentMessageV2) error); ok { - r0 = rf(ctx, vchannel, createSegmentMsg) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// MockFlushMsgHandler_HandleCreateSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleCreateSegment' -type MockFlushMsgHandler_HandleCreateSegment_Call struct { - *mock.Call -} - -// HandleCreateSegment is a helper method to define mock.On call -// - ctx context.Context -// - vchannel string -// - createSegmentMsg message.specializedImmutableMessage[*messagespb.CreateSegmentMessageHeader,*messagespb.CreateSegmentMessageBody] -func (_e *MockFlushMsgHandler_Expecter) HandleCreateSegment(ctx interface{}, vchannel interface{}, createSegmentMsg interface{}) *MockFlushMsgHandler_HandleCreateSegment_Call { - return &MockFlushMsgHandler_HandleCreateSegment_Call{Call: _e.mock.On("HandleCreateSegment", ctx, vchannel, createSegmentMsg)} -} - -func (_c *MockFlushMsgHandler_HandleCreateSegment_Call) Run(run func(ctx context.Context, vchannel string, createSegmentMsg message.ImmutableCreateSegmentMessageV2)) *MockFlushMsgHandler_HandleCreateSegment_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(string), args[2].(message.ImmutableCreateSegmentMessageV2)) - }) - return _c -} - -func (_c *MockFlushMsgHandler_HandleCreateSegment_Call) Return(_a0 error) *MockFlushMsgHandler_HandleCreateSegment_Call { - _c.Call.Return(_a0) - return _c -} - -func (_c *MockFlushMsgHandler_HandleCreateSegment_Call) RunAndReturn(run func(context.Context, string, message.ImmutableCreateSegmentMessageV2) error) *MockFlushMsgHandler_HandleCreateSegment_Call { - _c.Call.Return(run) - return _c -} - -// HandleFlush provides a mock function with given fields: vchannel, flushMsg -func (_m *MockFlushMsgHandler) HandleFlush(vchannel string, flushMsg message.ImmutableFlushMessageV2) error { - ret := _m.Called(vchannel, flushMsg) - - if len(ret) == 0 { - panic("no return value specified for HandleFlush") - } - - var r0 error - if rf, ok := ret.Get(0).(func(string, message.ImmutableFlushMessageV2) error); ok { - r0 = rf(vchannel, flushMsg) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// MockFlushMsgHandler_HandleFlush_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleFlush' -type MockFlushMsgHandler_HandleFlush_Call struct { - *mock.Call -} - -// HandleFlush is a helper method to define mock.On call -// - vchannel string -// - flushMsg message.specializedImmutableMessage[*messagespb.FlushMessageHeader,*messagespb.FlushMessageBody] -func (_e *MockFlushMsgHandler_Expecter) HandleFlush(vchannel interface{}, flushMsg interface{}) *MockFlushMsgHandler_HandleFlush_Call { - return &MockFlushMsgHandler_HandleFlush_Call{Call: _e.mock.On("HandleFlush", vchannel, flushMsg)} -} - -func (_c *MockFlushMsgHandler_HandleFlush_Call) Run(run func(vchannel string, flushMsg message.ImmutableFlushMessageV2)) *MockFlushMsgHandler_HandleFlush_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(string), args[1].(message.ImmutableFlushMessageV2)) - }) - return _c -} - -func (_c *MockFlushMsgHandler_HandleFlush_Call) Return(_a0 error) *MockFlushMsgHandler_HandleFlush_Call { - _c.Call.Return(_a0) - return _c -} - -func (_c *MockFlushMsgHandler_HandleFlush_Call) RunAndReturn(run func(string, message.ImmutableFlushMessageV2) error) *MockFlushMsgHandler_HandleFlush_Call { - _c.Call.Return(run) - return _c -} - -// HandleManualFlush provides a mock function with given fields: vchannel, flushMsg -func (_m *MockFlushMsgHandler) HandleManualFlush(vchannel string, flushMsg message.ImmutableManualFlushMessageV2) error { - ret := _m.Called(vchannel, flushMsg) - - if len(ret) == 0 { - panic("no return value specified for HandleManualFlush") - } - - var r0 error - if rf, ok := ret.Get(0).(func(string, message.ImmutableManualFlushMessageV2) error); ok { - r0 = rf(vchannel, flushMsg) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// MockFlushMsgHandler_HandleManualFlush_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'HandleManualFlush' -type MockFlushMsgHandler_HandleManualFlush_Call struct { - *mock.Call -} - -// HandleManualFlush is a helper method to define mock.On call -// - vchannel string -// - flushMsg message.specializedImmutableMessage[*messagespb.ManualFlushMessageHeader,*messagespb.ManualFlushMessageBody] -func (_e *MockFlushMsgHandler_Expecter) HandleManualFlush(vchannel interface{}, flushMsg interface{}) *MockFlushMsgHandler_HandleManualFlush_Call { - return &MockFlushMsgHandler_HandleManualFlush_Call{Call: _e.mock.On("HandleManualFlush", vchannel, flushMsg)} -} - -func (_c *MockFlushMsgHandler_HandleManualFlush_Call) Run(run func(vchannel string, flushMsg message.ImmutableManualFlushMessageV2)) *MockFlushMsgHandler_HandleManualFlush_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(string), args[1].(message.ImmutableManualFlushMessageV2)) - }) - return _c -} - -func (_c *MockFlushMsgHandler_HandleManualFlush_Call) Return(_a0 error) *MockFlushMsgHandler_HandleManualFlush_Call { - _c.Call.Return(_a0) - return _c -} - -func (_c *MockFlushMsgHandler_HandleManualFlush_Call) RunAndReturn(run func(string, message.ImmutableManualFlushMessageV2) error) *MockFlushMsgHandler_HandleManualFlush_Call { - _c.Call.Return(run) - return _c -} - -// NewMockFlushMsgHandler creates a new instance of MockFlushMsgHandler. 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 NewMockFlushMsgHandler(t interface { - mock.TestingT - Cleanup(func()) -}) *MockFlushMsgHandler { - mock := &MockFlushMsgHandler{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index cf3f95c6dc..a95c1d62b0 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -41,12 +41,12 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/distributed/streaming" "github.com/milvus-io/milvus/internal/http" "github.com/milvus-io/milvus/internal/proxy/connection" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/ctokenizer" "github.com/milvus-io/milvus/internal/util/hookutil" - "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/internal/util/streamingutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" @@ -56,6 +56,7 @@ import ( "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/proto/querypb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/crypto" @@ -6535,143 +6536,61 @@ func (node *Proxy) ImportV2(ctx context.Context, req *internalpb.ImportRequest) method := "ImportV2" tr := timerecord.NewTimeRecorder(method) log.Info(rpcReceived(method)) - nodeID := fmt.Sprint(paramtable.GetNodeID()) - defer func() { - metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.TotalLabel, req.GetDbName(), req.GetCollectionName()).Inc() - if resp.GetStatus().GetCode() != 0 { - log.Warn("import failed", zap.String("err", resp.GetStatus().GetReason())) - metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.FailLabel, req.GetDbName(), req.GetCollectionName()).Inc() - } else { - metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.SuccessLabel, req.GetDbName(), req.GetCollectionName()).Inc() - } - }() + metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.TotalLabel, req.GetDbName(), req.GetCollectionName()).Inc() - collectionID, err := globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.GetCollectionName()) - if err != nil { - resp.Status = merr.Status(err) - return resp, nil + it := &importTask{ + ctx: ctx, + Condition: NewTaskCondition(ctx), + req: req, + node: node, + dataCoord: node.dataCoord, + resp: resp, } - schema, err := globalMetaCache.GetCollectionSchema(ctx, req.GetDbName(), req.GetCollectionName()) - if err != nil { - resp.Status = merr.Status(err) - return resp, nil - } - channels, err := node.chMgr.getVChannels(collectionID) - if err != nil { + + if err := node.sched.dmQueue.Enqueue(it); err != nil { + log.Warn( + rpcFailedToEnqueue(method), + zap.Error(err)) + metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.AbandonLabel, req.GetDbName(), req.GetCollectionName()).Inc() resp.Status = merr.Status(err) return resp, nil } - isBackup := importutilv2.IsBackup(req.GetOptions()) - isL0Import := importutilv2.IsL0Import(req.GetOptions()) - hasPartitionKey := typeutil.HasPartitionKey(schema.CollectionSchema) + log.Info( + rpcEnqueued(method), + zap.Uint64("BeginTs", it.BeginTs()), + zap.Uint64("EndTs", it.EndTs())) - var partitionIDs []int64 - if isBackup { - if req.GetPartitionName() == "" { - resp.Status = merr.Status(merr.WrapErrParameterInvalidMsg("partition not specified")) - return resp, nil - } - // Currently, Backup tool call import must with a partition name, each time restore a partition - partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.GetPartitionName()) - if err != nil { - resp.Status = merr.Status(err) - return resp, nil - } - partitionIDs = []UniqueID{partitionID} - } else if isL0Import { - if req.GetPartitionName() == "" { - partitionIDs = []UniqueID{common.AllPartitionsID} - } else { - partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.PartitionName) - if err != nil { - resp.Status = merr.Status(err) - return resp, nil - } - partitionIDs = []UniqueID{partitionID} - } - // Currently, querynodes first load L0 segments and then load L1 segments. - // Therefore, to ensure the deletes from L0 import take effect, - // the collection needs to be in an unloaded state, - // and then all L0 and L1 segments should be loaded at once. - // We will remove this restriction after querynode supported to load L0 segments dynamically. - loaded, err := isCollectionLoaded(ctx, node.queryCoord, collectionID) - if err != nil { - resp.Status = merr.Status(err) - return resp, nil - } - if loaded { - resp.Status = merr.Status(merr.WrapErrImportFailed("for l0 import, collection cannot be loaded, please release it first")) - return resp, nil - } - } else { - if hasPartitionKey { - if req.GetPartitionName() != "" { - resp.Status = merr.Status(merr.WrapErrImportFailed("not allow to set partition name for collection with partition key")) - return resp, nil - } - partitions, err := globalMetaCache.GetPartitions(ctx, req.GetDbName(), req.GetCollectionName()) - if err != nil { - resp.Status = merr.Status(err) - return resp, nil - } - _, partitionIDs, err = typeutil.RearrangePartitionsForPartitionKey(partitions) - if err != nil { - resp.Status = merr.Status(err) - return resp, nil - } - } else { - if req.GetPartitionName() == "" { - req.PartitionName = Params.CommonCfg.DefaultPartitionName.GetValue() - } - partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.PartitionName) - if err != nil { - resp.Status = merr.Status(err) - return resp, nil - } - partitionIDs = []UniqueID{partitionID} - } - } - - req.Files = lo.Filter(req.GetFiles(), func(file *internalpb.ImportFile, _ int) bool { - return len(file.GetPaths()) > 0 - }) - if len(req.Files) == 0 { - resp.Status = merr.Status(merr.WrapErrParameterInvalidMsg("import request is empty")) - return resp, nil - } - if len(req.Files) > Params.DataCoordCfg.MaxFilesPerImportReq.GetAsInt() { - resp.Status = merr.Status(merr.WrapErrImportFailed(fmt.Sprintf("The max number of import files should not exceed %d, but got %d", - Params.DataCoordCfg.MaxFilesPerImportReq.GetAsInt(), len(req.Files)))) - return resp, nil - } - if !isBackup && !isL0Import { - // check file type - for _, file := range req.GetFiles() { - _, err = importutilv2.GetFileType(file) - if err != nil { - resp.Status = merr.Status(err) - return resp, nil - } - } - } - importRequest := &internalpb.ImportRequestInternal{ - CollectionID: collectionID, - CollectionName: req.GetCollectionName(), - PartitionIDs: partitionIDs, - ChannelNames: channels, - Schema: schema.CollectionSchema, - Files: req.GetFiles(), - Options: req.GetOptions(), - } - resp, err = node.dataCoord.ImportV2(ctx, importRequest) - if err != nil { - log.Warn("import failed", zap.Error(err)) + if err := it.WaitToFinish(); err != nil { + log.Warn( + rpcFailedToWaitToFinish(method), + zap.Error(err), + zap.Uint64("BeginTs", it.BeginTs()), + zap.Uint64("EndTs", it.EndTs())) metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.FailLabel, req.GetDbName(), req.GetCollectionName()).Inc() + resp.Status = merr.Status(err) + return resp, nil } + + // Import is asynchronous consumed from the wal, so we need to wait for the wal to release the resource key. + // The job can be seen by the user after the resource key is acked once at any vchannel. + jobID, err := strconv.ParseInt(resp.GetJobID(), 10, 64) + if err != nil { + return &internalpb.ImportResponse{ + Status: merr.Status(merr.WrapErrServiceInternal("invalid job ID")), + }, nil + } + resourceKey := message.NewImportJobIDResourceKey(jobID) + if err := streaming.WAL().Broadcast().BlockUntilResourceKeyAckOnce(ctx, resourceKey); err != nil { + log.Warn("failed to wait for resource key ack", zap.Error(err)) + return &internalpb.ImportResponse{ + Status: merr.Status(merr.WrapErrServiceInternal("failed to wait for resource key ack")), + }, nil + } + metrics.ProxyFunctionCall.WithLabelValues(nodeID, method, metrics.SuccessLabel, req.GetDbName(), req.GetCollectionName()).Inc() metrics.ProxyReqLatency.WithLabelValues(nodeID, method).Observe(float64(tr.ElapseSpan().Milliseconds())) - return resp, err + return resp, nil } func (node *Proxy) GetImportProgress(ctx context.Context, req *internalpb.GetImportProgressRequest) (*internalpb.GetImportProgressResponse, error) { diff --git a/internal/proxy/impl_test.go b/internal/proxy/impl_test.go index 7f55da7cfa..a4735c8b86 100644 --- a/internal/proxy/impl_test.go +++ b/internal/proxy/impl_test.go @@ -20,6 +20,7 @@ import ( "context" "encoding/base64" "fmt" + "math/rand" "net/http" "net/http/httptest" "testing" @@ -40,8 +41,10 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/allocator" + "github.com/milvus-io/milvus/internal/distributed/streaming" mhttp "github.com/milvus-io/milvus/internal/http" "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" @@ -53,6 +56,7 @@ import ( "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -1610,6 +1614,13 @@ func TestProxy_ReplicateMessage(t *testing.T) { } func TestProxy_ImportV2(t *testing.T) { + wal := mock_streaming.NewMockWALAccesser(t) + b := mock_streaming.NewMockBroadcast(t) + wal.EXPECT().Broadcast().Return(b).Maybe() + b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe() + b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil).Maybe() + streaming.SetWALForTest(wal) + defer streaming.RecoverWALForTest() ctx := context.Background() mockErr := errors.New("mock error") @@ -1625,6 +1636,22 @@ func TestProxy_ImportV2(t *testing.T) { assert.NotEqual(t, int32(0), rsp.GetStatus().GetCode()) node.UpdateStateCode(commonpb.StateCode_Healthy) + factory := dependency.NewDefaultFactory(true) + node, err = NewProxy(ctx, factory) + assert.NoError(t, err) + node.UpdateStateCode(commonpb.StateCode_Healthy) + node.tsoAllocator = ×tampAllocator{ + tso: newMockTimestampAllocatorInterface(), + } + scheduler, err := newTaskScheduler(ctx, node.tsoAllocator, factory) + assert.NoError(t, err) + node.sched = scheduler + err = node.sched.Start() + assert.NoError(t, err) + chMgr := NewMockChannelsMgr(t) + chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"p1"}, nil) + node.chMgr = chMgr + // no such collection mc := NewMockCache(t) mc.EXPECT().GetCollectionID(mock.Anything, mock.Anything, mock.Anything).Return(0, mockErr) @@ -1651,9 +1678,7 @@ func TestProxy_ImportV2(t *testing.T) { }}, }, nil) globalMetaCache = mc - chMgr := NewMockChannelsMgr(t) - chMgr.EXPECT().getVChannels(mock.Anything).Return(nil, mockErr) - node.chMgr = chMgr + chMgr.EXPECT().getVChannels(mock.Anything).Return(nil, mockErr).Once() rsp, err = node.ImportV2(ctx, &internalpb.ImportRequest{CollectionName: "aaa"}) assert.NoError(t, err) assert.NotEqual(t, int32(0), rsp.GetStatus().GetCode()) @@ -1661,6 +1686,7 @@ func TestProxy_ImportV2(t *testing.T) { // set partition name and with partition key chMgr = NewMockChannelsMgr(t) chMgr.EXPECT().getVChannels(mock.Anything).Return([]string{"ch0"}, nil) + chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"p1"}, nil) node.chMgr = chMgr rsp, err = node.ImportV2(ctx, &internalpb.ImportRequest{CollectionName: "aaa", PartitionName: "bbb"}) assert.NoError(t, err) @@ -1717,9 +1743,17 @@ func TestProxy_ImportV2(t *testing.T) { assert.NotEqual(t, int32(0), rsp.GetStatus().GetCode()) // normal case - dataCoord := mocks.NewMockDataCoordClient(t) - dataCoord.EXPECT().ImportV2(mock.Anything, mock.Anything).Return(nil, nil) - node.dataCoord = dataCoord + rc := mocks.NewMockRootCoordClient(t) + rc.EXPECT().AllocID(mock.Anything, mock.Anything).Return(&rootcoordpb.AllocIDResponse{ + ID: rand.Int63(), + Count: 1, + }, nil).Once() + idAllocator, err := allocator.NewIDAllocator(ctx, rc, 0) + assert.NoError(t, err) + node.rowIDAllocator = idAllocator + err = idAllocator.Start() + assert.NoError(t, err) + rsp, err = node.ImportV2(ctx, &internalpb.ImportRequest{ CollectionName: "aaa", Files: []*internalpb.ImportFile{{ diff --git a/internal/proxy/proxy_test.go b/internal/proxy/proxy_test.go index 6d2c6b8d26..ec43a557a2 100644 --- a/internal/proxy/proxy_test.go +++ b/internal/proxy/proxy_test.go @@ -43,6 +43,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/coordinator/coordclient" grpcdatacoordclient "github.com/milvus-io/milvus/internal/distributed/datacoord" grpcdatacoordclient2 "github.com/milvus-io/milvus/internal/distributed/datacoord/client" @@ -53,8 +54,10 @@ import ( grpcquerynode "github.com/milvus-io/milvus/internal/distributed/querynode" grpcrootcoord "github.com/milvus-io/milvus/internal/distributed/rootcoord" rcc "github.com/milvus-io/milvus/internal/distributed/rootcoord/client" + "github.com/milvus-io/milvus/internal/distributed/streaming" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/internal/mocks/distributed/mock_streaming" "github.com/milvus-io/milvus/internal/util/componentutil" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" @@ -65,6 +68,7 @@ import ( "github.com/milvus-io/milvus/pkg/proto/proxypb" "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" + "github.com/milvus-io/milvus/pkg/streaming/util/types" "github.com/milvus-io/milvus/pkg/tracer" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/crypto" @@ -2020,22 +2024,6 @@ func TestProxy(t *testing.T) { rateCol.Register(internalpb.RateType_DMLInsert.String()) }) - wg.Add(1) - t.Run("test import", func(t *testing.T) { - defer wg.Done() - req := &milvuspb.ImportRequest{ - DbName: dbName, - CollectionName: collectionName, - Files: []string{"f1.json"}, - } - proxy.UpdateStateCode(commonpb.StateCode_Healthy) - resp, err := proxy.Import(context.TODO(), req) - assert.EqualValues(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) - assert.NoError(t, err) - // Wait a bit for complete import to start. - time.Sleep(2 * time.Second) - }) - wg.Add(1) t.Run("release collection", func(t *testing.T) { defer wg.Done() @@ -4669,6 +4657,13 @@ func TestProxy_Import(t *testing.T) { cache := globalMetaCache defer func() { globalMetaCache = cache }() + wal := mock_streaming.NewMockWALAccesser(t) + b := mock_streaming.NewMockBroadcast(t) + wal.EXPECT().Broadcast().Return(b).Maybe() + b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe() + streaming.SetWALForTest(wal) + defer streaming.RecoverWALForTest() + t.Run("Import failed", func(t *testing.T) { proxy := &Proxy{} proxy.UpdateStateCode(commonpb.StateCode_Abnormal) @@ -4680,6 +4675,7 @@ func TestProxy_Import(t *testing.T) { }) t.Run("Import", func(t *testing.T) { + ctx := context.Background() proxy := &Proxy{} proxy.UpdateStateCode(commonpb.StateCode_Healthy) @@ -4692,15 +4688,37 @@ func TestProxy_Import(t *testing.T) { globalMetaCache = mc chMgr := NewMockChannelsMgr(t) - chMgr.EXPECT().getVChannels(mock.Anything).Return(nil, nil) + chMgr.EXPECT().getVChannels(mock.Anything).Return([]string{"foo"}, nil) + chMgr.EXPECT().getChannels(mock.Anything).Return([]string{"foo_v1"}, nil) proxy.chMgr = chMgr - dataCoord := mocks.NewMockDataCoordClient(t) - dataCoord.EXPECT().ImportV2(mock.Anything, mock.Anything).Return(&internalpb.ImportResponse{ - Status: merr.Success(), - JobID: "100", - }, nil) - proxy.dataCoord = dataCoord + factory := dependency.NewDefaultFactory(true) + rc := mocks.NewMockRootCoordClient(t) + rc.EXPECT().AllocID(mock.Anything, mock.Anything).Return(&rootcoordpb.AllocIDResponse{ + ID: rand.Int63(), + Count: 1, + }, nil).Once() + idAllocator, err := allocator.NewIDAllocator(ctx, rc, 0) + assert.NoError(t, err) + err = idAllocator.Start() + assert.NoError(t, err) + proxy.rowIDAllocator = idAllocator + proxy.tsoAllocator = ×tampAllocator{ + tso: newMockTimestampAllocatorInterface(), + } + scheduler, err := newTaskScheduler(ctx, proxy.tsoAllocator, factory) + assert.NoError(t, err) + proxy.sched = scheduler + err = proxy.sched.Start() + assert.NoError(t, err) + + wal := mock_streaming.NewMockWALAccesser(t) + b := mock_streaming.NewMockBroadcast(t) + wal.EXPECT().Broadcast().Return(b) + b.EXPECT().Append(mock.Anything, mock.Anything).Return(&types.BroadcastAppendResult{}, nil) + b.EXPECT().BlockUntilResourceKeyAckOnce(mock.Anything, mock.Anything).Return(nil).Maybe() + streaming.SetWALForTest(wal) + defer streaming.RecoverWALForTest() req := &milvuspb.ImportRequest{ CollectionName: "dummy", diff --git a/internal/proxy/task_import.go b/internal/proxy/task_import.go new file mode 100644 index 0000000000..82c9fe9b81 --- /dev/null +++ b/internal/proxy/task_import.go @@ -0,0 +1,273 @@ +/* + * 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 proxy + +import ( + "context" + "fmt" + "strconv" + + "github.com/samber/lo" + "go.uber.org/zap" + + "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/distributed/streaming" + "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/importutilv2" + "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" + "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/util/funcutil" + "github.com/milvus-io/milvus/pkg/util/merr" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +type importTask struct { + baseTask + Condition + req *internalpb.ImportRequest + ctx context.Context + node *Proxy + dataCoord types.DataCoordClient + + msgID UniqueID + taskTS Timestamp + vchannels []string + pchannels []string + partitionIDs []int64 + collectionID UniqueID + schema *schemaInfo + resp *internalpb.ImportResponse +} + +func (it *importTask) TraceCtx() context.Context { + return it.ctx +} + +func (it *importTask) ID() UniqueID { + return it.msgID +} + +func (it *importTask) SetID(uid UniqueID) { + it.msgID = uid +} + +func (it *importTask) Name() string { + return "ImportTask" +} + +func (it *importTask) Type() commonpb.MsgType { + return commonpb.MsgType_Import +} + +func (it *importTask) BeginTs() Timestamp { + return it.taskTS +} + +func (it *importTask) EndTs() Timestamp { + return it.taskTS +} + +func (it *importTask) SetTs(ts Timestamp) { + it.taskTS = ts +} + +func (it *importTask) OnEnqueue() error { + return nil +} + +func (it *importTask) PreExecute(ctx context.Context) error { + req := it.req + node := it.node + collectionID, err := globalMetaCache.GetCollectionID(ctx, req.GetDbName(), req.GetCollectionName()) + if err != nil { + return err + } + it.collectionID = collectionID + schema, err := globalMetaCache.GetCollectionSchema(ctx, req.GetDbName(), req.GetCollectionName()) + if err != nil { + return err + } + it.schema = schema + channels, err := node.chMgr.getVChannels(collectionID) + if err != nil { + return err + } + it.vchannels = channels + + isBackup := importutilv2.IsBackup(req.GetOptions()) + isL0Import := importutilv2.IsL0Import(req.GetOptions()) + hasPartitionKey := typeutil.HasPartitionKey(schema.CollectionSchema) + + var partitionIDs []int64 + if isBackup { + if req.GetPartitionName() == "" { + return merr.WrapErrParameterInvalidMsg("partition not specified") + } + // Currently, Backup tool call import must with a partition name, each time restore a partition + partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.GetPartitionName()) + if err != nil { + return err + } + partitionIDs = []UniqueID{partitionID} + } else if isL0Import { + if req.GetPartitionName() == "" { + partitionIDs = []UniqueID{common.AllPartitionsID} + } else { + partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.PartitionName) + if err != nil { + return err + } + partitionIDs = []UniqueID{partitionID} + } + // Currently, querynodes first load L0 segments and then load L1 segments. + // Therefore, to ensure the deletes from L0 import take effect, + // the collection needs to be in an unloaded state, + // and then all L0 and L1 segments should be loaded at once. + // We will remove this restriction after querynode supported to load L0 segments dynamically. + loaded, err := isCollectionLoaded(ctx, node.queryCoord, collectionID) + if err != nil { + return err + } + if loaded { + return merr.WrapErrImportFailed("for l0 import, collection cannot be loaded, please release it first") + } + } else { + if hasPartitionKey { + if req.GetPartitionName() != "" { + return merr.WrapErrImportFailed("not allow to set partition name for collection with partition key") + } + partitions, err := globalMetaCache.GetPartitions(ctx, req.GetDbName(), req.GetCollectionName()) + if err != nil { + return err + } + _, partitionIDs, err = typeutil.RearrangePartitionsForPartitionKey(partitions) + if err != nil { + return err + } + } else { + if req.GetPartitionName() == "" { + req.PartitionName = Params.CommonCfg.DefaultPartitionName.GetValue() + } + partitionID, err := globalMetaCache.GetPartitionID(ctx, req.GetDbName(), req.GetCollectionName(), req.PartitionName) + if err != nil { + return err + } + partitionIDs = []UniqueID{partitionID} + } + } + + req.Files = lo.Filter(req.GetFiles(), func(file *internalpb.ImportFile, _ int) bool { + return len(file.GetPaths()) > 0 + }) + if len(req.Files) == 0 { + return merr.WrapErrParameterInvalidMsg("import request is empty") + } + if len(req.Files) > Params.DataCoordCfg.MaxFilesPerImportReq.GetAsInt() { + return merr.WrapErrImportFailed(fmt.Sprintf("The max number of import files should not exceed %d, but got %d", + Params.DataCoordCfg.MaxFilesPerImportReq.GetAsInt(), len(req.Files))) + } + if !isBackup && !isL0Import { + // check file type + for _, file := range req.GetFiles() { + _, err = importutilv2.GetFileType(file) + if err != nil { + return err + } + } + } + it.partitionIDs = partitionIDs + return nil +} + +func (it *importTask) setChannels() error { + collID, err := globalMetaCache.GetCollectionID(it.ctx, it.req.GetDbName(), it.req.CollectionName) + if err != nil { + return err + } + channels, err := it.node.chMgr.getChannels(collID) + if err != nil { + return err + } + it.pchannels = channels + return nil +} + +func (it *importTask) getChannels() []pChan { + return it.pchannels +} + +func (it *importTask) Execute(ctx context.Context) error { + jobID, err := it.node.rowIDAllocator.AllocOne() + if err != nil { + log.Ctx(ctx).Warn("alloc job id failed", zap.Error(err)) + return err + } + resourceKey := message.NewImportJobIDResourceKey(jobID) + msg, err := message.NewImportMessageBuilderV1(). + WithHeader(&message.ImportMessageHeader{}).WithBody( + &msgpb.ImportMsg{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_Import, + Timestamp: it.BeginTs(), + }, + DbName: it.req.GetDbName(), + CollectionName: it.req.GetCollectionName(), + CollectionID: it.collectionID, + PartitionIDs: it.partitionIDs, + Options: funcutil.KeyValuePair2Map(it.req.GetOptions()), + Files: GetImportFiles(it.req.GetFiles()), + Schema: it.schema.CollectionSchema, + JobID: jobID, + }). + WithBroadcast(it.vchannels, resourceKey). + BuildBroadcast() + if err != nil { + log.Ctx(ctx).Warn("create import message failed", zap.Error(err)) + return err + } + resp, err := streaming.WAL().Broadcast().Append(ctx, msg) + if err != nil { + log.Ctx(ctx).Warn("broadcast import msg failed", zap.Error(err)) + return err + } + log.Ctx(ctx).Info( + "broadcast import msg success", + zap.Int64("jobID", jobID), + zap.Uint64("broadcastID", resp.BroadcastID), + zap.Any("appendResults", resp.AppendResults), + ) + it.resp.JobID = strconv.FormatInt(jobID, 10) + return nil +} + +func GetImportFiles(internals []*internalpb.ImportFile) []*msgpb.ImportFile { + return lo.Map(internals, func(internal *internalpb.ImportFile, _ int) *msgpb.ImportFile { + return &msgpb.ImportFile{ + Id: internal.GetId(), + Paths: internal.GetPaths(), + } + }) +} + +func (it *importTask) PostExecute(ctx context.Context) error { + return nil +} diff --git a/internal/streamingcoord/server/broadcaster/broadcast_manager.go b/internal/streamingcoord/server/broadcaster/broadcast_manager.go index 476117f131..edbc4b72f4 100644 --- a/internal/streamingcoord/server/broadcaster/broadcast_manager.go +++ b/internal/streamingcoord/server/broadcaster/broadcast_manager.go @@ -63,12 +63,10 @@ type broadcastTaskManager struct { // AddTask adds a new broadcast task into the manager. func (bm *broadcastTaskManager) AddTask(ctx context.Context, msg message.BroadcastMutableMessage) (*pendingBroadcastTask, error) { - id, err := resource.Resource().IDAllocator().Allocate(ctx) - if err != nil { - return nil, errors.Wrapf(err, "allocate new id failed") + var err error + if msg, err = bm.assignID(ctx, msg); err != nil { + return nil, err } - msg = msg.WithBroadcastID(id) - task, err := bm.addBroadcastTask(msg) if err != nil { return nil, err @@ -76,6 +74,30 @@ func (bm *broadcastTaskManager) AddTask(ctx context.Context, msg message.Broadca return newPendingBroadcastTask(task), nil } +func (bm *broadcastTaskManager) assignID(ctx context.Context, msg message.BroadcastMutableMessage) (message.BroadcastMutableMessage, error) { + // TODO: current implementation the header cannot be seen at flusher itself. + // only import message use it, so temporarily set the broadcast id here. + // need to refactor the message to make the broadcast header visible to flusher. + if msg.MessageType() == message.MessageTypeImport { + importMsg, err := message.AsMutableImportMessageV1(msg) + if err != nil { + return nil, err + } + body, err := importMsg.Body() + if err != nil { + return nil, err + } + return msg.WithBroadcastID(uint64(body.JobID)), nil + } + + id, err := resource.Resource().IDAllocator().Allocate(ctx) + if err != nil { + return nil, errors.Wrapf(err, "allocate new id failed") + } + msg = msg.WithBroadcastID(id) + return msg, nil +} + // Ack acknowledges the message at the specified vchannel. func (bm *broadcastTaskManager) Ack(ctx context.Context, broadcastID uint64, vchannel string) error { task, ok := bm.getBroadcastTaskByID(broadcastID) diff --git a/internal/streamingnode/server/flusher/flusherimpl/msg_handler_impl.go b/internal/streamingnode/server/flusher/flusherimpl/msg_handler_impl.go index 1be54ce30e..9d1b270897 100644 --- a/internal/streamingnode/server/flusher/flusherimpl/msg_handler_impl.go +++ b/internal/streamingnode/server/flusher/flusherimpl/msg_handler_impl.go @@ -20,11 +20,21 @@ import ( "context" "github.com/cockroachdb/errors" + "github.com/samber/lo" "go.uber.org/zap" + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" + "github.com/milvus-io/milvus/internal/distributed/streaming" + "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" + "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/streaming/util/message" + "github.com/milvus-io/milvus/pkg/streaming/util/types" + "github.com/milvus-io/milvus/pkg/util/funcutil" + "github.com/milvus-io/milvus/pkg/util/merr" + "github.com/milvus-io/milvus/pkg/util/retry" ) func newMsgHandler(wbMgr writebuffer.BufferManager) *msgHandlerImpl { @@ -75,3 +85,42 @@ func (impl *msgHandlerImpl) HandleManualFlush(vchannel string, flushMsg message. } return nil } + +func (impl *msgHandlerImpl) HandleImport(ctx context.Context, vchannel string, importMsg *msgpb.ImportMsg) error { + return retry.Do(ctx, func() (err error) { + defer func() { + if err == nil { + err = streaming.WAL().Broadcast().Ack(ctx, types.BroadcastAckRequest{ + BroadcastID: uint64(importMsg.GetJobID()), + VChannel: vchannel, + }) + } + }() + client, err := resource.Resource().DataCoordClient().GetWithContext(ctx) + if err != nil { + return err + } + importResp, err := client.ImportV2(ctx, &internalpb.ImportRequestInternal{ + CollectionID: importMsg.GetCollectionID(), + CollectionName: importMsg.GetCollectionName(), + PartitionIDs: importMsg.GetPartitionIDs(), + ChannelNames: []string{vchannel}, + Schema: importMsg.GetSchema(), + Files: lo.Map(importMsg.GetFiles(), util.ConvertInternalImportFile), + Options: funcutil.Map2KeyValuePair(importMsg.GetOptions()), + DataTimestamp: importMsg.GetBase().GetTimestamp(), + JobID: importMsg.GetJobID(), + }) + err = merr.CheckRPCCall(importResp, err) + if errors.Is(err, merr.ErrCollectionNotFound) { + log.Ctx(ctx).Warn("import message failed because of collection not found, skip it", zap.String("job_id", importResp.GetJobID()), zap.Error(err)) + return nil + } + if err != nil { + log.Ctx(ctx).Warn("import message failed", zap.String("job_id", importResp.GetJobID()), zap.Error(err)) + return err + } + log.Ctx(ctx).Info("import message handled", zap.String("job_id", importResp.GetJobID())) + return nil + }, retry.AttemptAlways()) +} diff --git a/pkg/go.mod b/pkg/go.mod index d4df256f57..56053d466a 100644 --- a/pkg/go.mod +++ b/pkg/go.mod @@ -15,7 +15,7 @@ require ( github.com/jolestar/go-commons-pool/v2 v2.1.2 github.com/json-iterator/go v1.1.12 github.com/klauspost/compress v1.17.7 - github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559 + github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b github.com/nats-io/nats-server/v2 v2.10.12 github.com/nats-io/nats.go v1.34.1 github.com/panjf2000/ants/v2 v2.7.2 diff --git a/pkg/go.sum b/pkg/go.sum index 47d8e4a323..1f4ce2d84e 100644 --- a/pkg/go.sum +++ b/pkg/go.sum @@ -492,8 +492,8 @@ github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119 h1:9VXijWu github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119/go.mod h1:DvXTE/K/RtHehxU8/GtDs4vFtfw64jJ3PaCnFri8CRg= github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b h1:TfeY0NxYxZzUfIfYe5qYDBzt4ZYRqzUjTR6CvUzjat8= github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b/go.mod h1:iwW+9cWfIzzDseEBCCeDSN5SD16Tidvy8cwQ7ZY8Qj4= -github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559 h1:c8n10eBkYU/HYaDUNAaKog4aIA3ZHO+GL7bHN2Ug/MA= -github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250208062437-5af22aa4b559/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs= +github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b h1:s3gdV+iYJMvo9mydBJVAEA2Uaz29eIuUnQK867U3G8I= +github.com/milvus-io/milvus-proto/go-api/v2 v2.5.0-beta.0.20250214033407-ad08272e542b/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs= github.com/milvus-io/pulsar-client-go v0.12.1 h1:O2JZp1tsYiO7C0MQ4hrUY/aJXnn2Gry6hpm7UodghmE= github.com/milvus-io/pulsar-client-go v0.12.1/go.mod h1:dkutuH4oS2pXiGm+Ti7fQZ4MRjrMPZ8IJeEGAWMeckk= github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA/g= diff --git a/pkg/metrics/metrics.go b/pkg/metrics/metrics.go index bf682f696d..b6025b8bd9 100644 --- a/pkg/metrics/metrics.go +++ b/pkg/metrics/metrics.go @@ -78,11 +78,12 @@ const ( Executing = "executing" Done = "done" - ImportStagePending = "pending" - ImportStagePreImport = "preimport" - ImportStageImport = "import" - ImportStageStats = "stats" - ImportStageBuildIndex = "build_index" + ImportStagePending = "pending" + ImportStagePreImport = "preimport" + ImportStageImport = "import" + ImportStageStats = "stats" + ImportStageBuildIndex = "build_index" + ImportStageWaitL0Import = "wait_l0_import" compactionTypeLabelName = "compaction_type" isVectorFieldLabelName = "is_vector_field" diff --git a/pkg/mq/msgstream/msg_for_import.go b/pkg/mq/msgstream/msg_for_import.go new file mode 100644 index 0000000000..0c6ba59d0f --- /dev/null +++ b/pkg/mq/msgstream/msg_for_import.go @@ -0,0 +1,78 @@ +/* + * 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 msgstream + +import ( + "google.golang.org/protobuf/proto" + + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" +) + +type ImportMsg struct { + BaseMsg + *msgpb.ImportMsg +} + +var _ TsMsg = (*ImportMsg)(nil) + +func (i *ImportMsg) ID() UniqueID { + return i.Base.MsgID +} + +func (i *ImportMsg) SetID(id UniqueID) { + i.Base.MsgID = id +} + +func (i *ImportMsg) Type() MsgType { + return i.Base.MsgType +} + +func (i *ImportMsg) SourceID() int64 { + return i.Base.SourceID +} + +func (i *ImportMsg) Marshal(input TsMsg) (MarshalType, error) { + importMsg := input.(*ImportMsg) + mb, err := proto.Marshal(importMsg.ImportMsg) + if err != nil { + return nil, err + } + return mb, nil +} + +func (i *ImportMsg) Unmarshal(input MarshalType) (TsMsg, error) { + importMsg := &msgpb.ImportMsg{} + in, err := convertToByteArray(input) + if err != nil { + return nil, err + } + err = proto.Unmarshal(in, importMsg) + if err != nil { + return nil, err + } + rr := &ImportMsg{ImportMsg: importMsg} + rr.BeginTimestamp = importMsg.GetBase().GetTimestamp() + rr.EndTimestamp = importMsg.GetBase().GetTimestamp() + + return rr, nil +} + +func (i *ImportMsg) Size() int { + return proto.Size(i.ImportMsg) +} diff --git a/pkg/mq/msgstream/msg_for_import_test.go b/pkg/mq/msgstream/msg_for_import_test.go new file mode 100644 index 0000000000..e987f9011c --- /dev/null +++ b/pkg/mq/msgstream/msg_for_import_test.go @@ -0,0 +1,64 @@ +/* + * 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 msgstream + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" +) + +func TestImportMsg(t *testing.T) { + var msg TsMsg = &ImportMsg{ + ImportMsg: &msgpb.ImportMsg{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_Import, + MsgID: 100, + Timestamp: 1000, + SourceID: 10000, + TargetID: 100000, + ReplicateInfo: nil, + }, + DbName: "unit_db", + }, + } + assert.EqualValues(t, 100, msg.ID()) + msg.SetID(200) + assert.EqualValues(t, 200, msg.ID()) + assert.Equal(t, commonpb.MsgType_Import, msg.Type()) + assert.EqualValues(t, 10000, msg.SourceID()) + + msgBytes, err := msg.Marshal(msg) + assert.NoError(t, err) + + var newMsg TsMsg = &ImportMsg{} + _, err = newMsg.Unmarshal("1") + assert.Error(t, err) + + newMsg, err = newMsg.Unmarshal(msgBytes) + assert.NoError(t, err) + assert.EqualValues(t, 200, newMsg.ID()) + assert.EqualValues(t, 1000, newMsg.BeginTs()) + assert.EqualValues(t, 1000, newMsg.EndTs()) + + assert.True(t, msg.Size() > 0) +} diff --git a/pkg/mq/msgstream/msg_for_index_test.go b/pkg/mq/msgstream/msg_for_index_test.go index bbbf64f650..068afa8608 100644 --- a/pkg/mq/msgstream/msg_for_index_test.go +++ b/pkg/mq/msgstream/msg_for_index_test.go @@ -50,7 +50,7 @@ func TestCreateIndex(t *testing.T) { msgBytes, err := msg.Marshal(msg) assert.NoError(t, err) - var newMsg TsMsg = &ReleaseCollectionMsg{} + var newMsg TsMsg = &CreateIndexMsg{} _, err = newMsg.Unmarshal("1") assert.Error(t, err) diff --git a/pkg/mq/msgstream/msg_for_replicate_test.go b/pkg/mq/msgstream/msg_for_replicate_test.go new file mode 100644 index 0000000000..2e73a391ad --- /dev/null +++ b/pkg/mq/msgstream/msg_for_replicate_test.go @@ -0,0 +1,64 @@ +/* + * 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 msgstream + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" +) + +func TestReplicateMsg(t *testing.T) { + var msg TsMsg = &ReplicateMsg{ + ReplicateMsg: &msgpb.ReplicateMsg{ + Base: &commonpb.MsgBase{ + MsgType: commonpb.MsgType_Replicate, + MsgID: 100, + Timestamp: 1000, + SourceID: 10000, + TargetID: 100000, + ReplicateInfo: nil, + }, + Database: "unit_db", + }, + } + assert.EqualValues(t, 100, msg.ID()) + msg.SetID(200) + assert.EqualValues(t, 200, msg.ID()) + assert.Equal(t, commonpb.MsgType_Replicate, msg.Type()) + assert.EqualValues(t, 10000, msg.SourceID()) + + msgBytes, err := msg.Marshal(msg) + assert.NoError(t, err) + + var newMsg TsMsg = &ReplicateMsg{} + _, err = newMsg.Unmarshal("1") + assert.Error(t, err) + + newMsg, err = newMsg.Unmarshal(msgBytes) + assert.NoError(t, err) + assert.EqualValues(t, 200, newMsg.ID()) + assert.EqualValues(t, 1000, newMsg.BeginTs()) + assert.EqualValues(t, 1000, newMsg.EndTs()) + + assert.True(t, msg.Size() > 0) +} diff --git a/pkg/mq/msgstream/unmarshal.go b/pkg/mq/msgstream/unmarshal.go index 8cd8de9eec..ff44cc8a1f 100644 --- a/pkg/mq/msgstream/unmarshal.go +++ b/pkg/mq/msgstream/unmarshal.go @@ -85,6 +85,7 @@ func (pudf *ProtoUDFactory) NewUnmarshalDispatcher() *ProtoUnmarshalDispatcher { operateUserRoleMsg := OperateUserRoleMsg{} operatePrivilegeMsg := OperatePrivilegeMsg{} replicateMsg := ReplicateMsg{} + importMsg := ImportMsg{} p := &ProtoUnmarshalDispatcher{} p.TempMap = make(map[commonpb.MsgType]UnmarshalFunc) @@ -115,6 +116,7 @@ func (pudf *ProtoUDFactory) NewUnmarshalDispatcher() *ProtoUnmarshalDispatcher { p.TempMap[commonpb.MsgType_OperateUserRole] = operateUserRoleMsg.Unmarshal p.TempMap[commonpb.MsgType_OperatePrivilege] = operatePrivilegeMsg.Unmarshal p.TempMap[commonpb.MsgType_Replicate] = replicateMsg.Unmarshal + p.TempMap[commonpb.MsgType_Import] = importMsg.Unmarshal return p } diff --git a/pkg/proto/data_coord.proto b/pkg/proto/data_coord.proto index fda4c10038..8f18ef510b 100644 --- a/pkg/proto/data_coord.proto +++ b/pkg/proto/data_coord.proto @@ -617,6 +617,8 @@ message CompactionPlan { IDRange pre_allocated_segmentIDs = 18; int64 slot_usage = 19; int64 max_size = 20; + // bf path for importing + // collection is importing } message CompactionSegment { @@ -885,6 +887,8 @@ message ImportJob { repeated internal.ImportFile files = 14; repeated common.KeyValuePair options = 15; string start_time = 16; + repeated string ready_vchannels = 17; + uint64 data_ts = 18; } enum ImportTaskStateV2 { @@ -895,6 +899,11 @@ enum ImportTaskStateV2 { Completed = 4; } +enum ImportTaskSourceV2 { + Request = 0; + L0Compaction = 1; +} + message PreImportTask { int64 jobID = 1; int64 taskID = 2; @@ -919,6 +928,7 @@ message ImportTaskV2 { repeated ImportFileStats file_stats = 9; repeated int64 stats_segmentIDs = 10; string created_time = 11; + ImportTaskSourceV2 source = 12; } enum GcCommand { diff --git a/pkg/proto/datapb/data_coord.pb.go b/pkg/proto/datapb/data_coord.pb.go index 7ed513a3cb..ad33bba7b3 100644 --- a/pkg/proto/datapb/data_coord.pb.go +++ b/pkg/proto/datapb/data_coord.pb.go @@ -314,6 +314,52 @@ func (ImportTaskStateV2) EnumDescriptor() ([]byte, []int) { return file_data_coord_proto_rawDescGZIP(), []int{4} } +type ImportTaskSourceV2 int32 + +const ( + ImportTaskSourceV2_Request ImportTaskSourceV2 = 0 + ImportTaskSourceV2_L0Compaction ImportTaskSourceV2 = 1 +) + +// Enum value maps for ImportTaskSourceV2. +var ( + ImportTaskSourceV2_name = map[int32]string{ + 0: "Request", + 1: "L0Compaction", + } + ImportTaskSourceV2_value = map[string]int32{ + "Request": 0, + "L0Compaction": 1, + } +) + +func (x ImportTaskSourceV2) Enum() *ImportTaskSourceV2 { + p := new(ImportTaskSourceV2) + *p = x + return p +} + +func (x ImportTaskSourceV2) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (ImportTaskSourceV2) Descriptor() protoreflect.EnumDescriptor { + return file_data_coord_proto_enumTypes[5].Descriptor() +} + +func (ImportTaskSourceV2) Type() protoreflect.EnumType { + return &file_data_coord_proto_enumTypes[5] +} + +func (x ImportTaskSourceV2) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use ImportTaskSourceV2.Descriptor instead. +func (ImportTaskSourceV2) EnumDescriptor() ([]byte, []int) { + return file_data_coord_proto_rawDescGZIP(), []int{5} +} + type GcCommand int32 const ( @@ -347,11 +393,11 @@ func (x GcCommand) String() string { } func (GcCommand) Descriptor() protoreflect.EnumDescriptor { - return file_data_coord_proto_enumTypes[5].Descriptor() + return file_data_coord_proto_enumTypes[6].Descriptor() } func (GcCommand) Type() protoreflect.EnumType { - return &file_data_coord_proto_enumTypes[5] + return &file_data_coord_proto_enumTypes[6] } func (x GcCommand) Number() protoreflect.EnumNumber { @@ -360,7 +406,7 @@ func (x GcCommand) Number() protoreflect.EnumNumber { // Deprecated: Use GcCommand.Descriptor instead. func (GcCommand) EnumDescriptor() ([]byte, []int) { - return file_data_coord_proto_rawDescGZIP(), []int{5} + return file_data_coord_proto_rawDescGZIP(), []int{6} } type CompactionTaskState int32 @@ -420,11 +466,11 @@ func (x CompactionTaskState) String() string { } func (CompactionTaskState) Descriptor() protoreflect.EnumDescriptor { - return file_data_coord_proto_enumTypes[6].Descriptor() + return file_data_coord_proto_enumTypes[7].Descriptor() } func (CompactionTaskState) Type() protoreflect.EnumType { - return &file_data_coord_proto_enumTypes[6] + return &file_data_coord_proto_enumTypes[7] } func (x CompactionTaskState) Number() protoreflect.EnumNumber { @@ -433,7 +479,7 @@ func (x CompactionTaskState) Number() protoreflect.EnumNumber { // Deprecated: Use CompactionTaskState.Descriptor instead. func (CompactionTaskState) EnumDescriptor() ([]byte, []int) { - return file_data_coord_proto_rawDescGZIP(), []int{6} + return file_data_coord_proto_rawDescGZIP(), []int{7} } // TODO: import google/protobuf/empty.proto @@ -7190,6 +7236,8 @@ type ImportJob struct { Files []*internalpb.ImportFile `protobuf:"bytes,14,rep,name=files,proto3" json:"files,omitempty"` Options []*commonpb.KeyValuePair `protobuf:"bytes,15,rep,name=options,proto3" json:"options,omitempty"` StartTime string `protobuf:"bytes,16,opt,name=start_time,json=startTime,proto3" json:"start_time,omitempty"` + ReadyVchannels []string `protobuf:"bytes,17,rep,name=ready_vchannels,json=readyVchannels,proto3" json:"ready_vchannels,omitempty"` + DataTs uint64 `protobuf:"varint,18,opt,name=data_ts,json=dataTs,proto3" json:"data_ts,omitempty"` } func (x *ImportJob) Reset() { @@ -7336,6 +7384,20 @@ func (x *ImportJob) GetStartTime() string { return "" } +func (x *ImportJob) GetReadyVchannels() []string { + if x != nil { + return x.ReadyVchannels + } + return nil +} + +func (x *ImportJob) GetDataTs() uint64 { + if x != nil { + return x.DataTs + } + return 0 +} + type PreImportTask struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -7463,6 +7525,7 @@ type ImportTaskV2 struct { FileStats []*ImportFileStats `protobuf:"bytes,9,rep,name=file_stats,json=fileStats,proto3" json:"file_stats,omitempty"` StatsSegmentIDs []int64 `protobuf:"varint,10,rep,packed,name=stats_segmentIDs,json=statsSegmentIDs,proto3" json:"stats_segmentIDs,omitempty"` CreatedTime string `protobuf:"bytes,11,opt,name=created_time,json=createdTime,proto3" json:"created_time,omitempty"` + Source ImportTaskSourceV2 `protobuf:"varint,12,opt,name=source,proto3,enum=milvus.proto.data.ImportTaskSourceV2" json:"source,omitempty"` } func (x *ImportTaskV2) Reset() { @@ -7574,6 +7637,13 @@ func (x *ImportTaskV2) GetCreatedTime() string { return "" } +func (x *ImportTaskV2) GetSource() ImportTaskSourceV2 { + if x != nil { + return x.Source + } + return ImportTaskSourceV2_Request +} + type GcControlRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -9460,7 +9530,7 @@ var file_data_coord_proto_rawDesc = []byte{ 0x73, 0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, - 0x73, 0x6b, 0x49, 0x44, 0x22, 0xfe, 0x04, 0x0a, 0x09, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, + 0x73, 0x6b, 0x49, 0x44, 0x22, 0xc0, 0x05, 0x0a, 0x09, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, @@ -9500,677 +9570,689 @@ var file_data_coord_proto_rawDesc = []byte{ 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x10, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, - 0x74, 0x54, 0x69, 0x6d, 0x65, 0x22, 0xd8, 0x02, 0x0a, 0x0d, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, - 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x12, 0x16, 0x0a, - 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, - 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, - 0x65, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, - 0x44, 0x12, 0x3a, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, - 0x74, 0x61, 0x74, 0x65, 0x56, 0x32, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, - 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, - 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x41, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x74, - 0x61, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, - 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x09, 0x66, - 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, - 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x63, - 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0c, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, - 0x22, 0xa2, 0x03, 0x0a, 0x0c, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x56, - 0x32, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, - 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, - 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, - 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x49, 0x44, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x3a, 0x0a, 0x05, 0x73, - 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, + 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x72, 0x65, 0x61, 0x64, 0x79, 0x5f, 0x76, + 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x11, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0e, + 0x72, 0x65, 0x61, 0x64, 0x79, 0x56, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x17, + 0x0a, 0x07, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x74, 0x73, 0x18, 0x12, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x06, 0x64, 0x61, 0x74, 0x61, 0x54, 0x73, 0x22, 0xd8, 0x02, 0x0a, 0x0d, 0x50, 0x72, 0x65, 0x49, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x12, + 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, + 0x65, 0x49, 0x44, 0x12, 0x3a, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, + 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x32, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x41, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, + 0x73, 0x74, 0x61, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, + 0x09, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x72, + 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x23, 0x0a, + 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0c, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x54, 0x69, + 0x6d, 0x65, 0x22, 0xe1, 0x03, 0x0a, 0x0c, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, + 0x6b, 0x56, 0x32, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, + 0x6b, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, + 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, + 0x44, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x49, 0x44, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x18, + 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, 0x3a, 0x0a, + 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x24, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, + 0x56, 0x32, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, + 0x73, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, + 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x41, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, + 0x74, 0x61, 0x74, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, - 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x32, - 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, - 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, - 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, - 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, - 0x54, 0x69, 0x6d, 0x65, 0x12, 0x41, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x74, 0x61, - 0x74, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, - 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x09, 0x66, 0x69, - 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x73, 0x74, 0x61, 0x74, 0x73, - 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, - 0x03, 0x52, 0x0f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, - 0x44, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, - 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x64, 0x54, 0x69, 0x6d, 0x65, 0x22, 0xb7, 0x01, 0x0a, 0x10, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x74, - 0x72, 0x6f, 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, 0x36, 0x0a, 0x07, - 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x07, 0x63, 0x6f, 0x6d, - 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x39, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, - 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x22, - 0x12, 0x0a, 0x10, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x22, 0x65, 0x0a, 0x11, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, - 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, 0x6e, 0x75, 0x6d, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, - 0x52, 0x08, 0x6e, 0x75, 0x6d, 0x53, 0x6c, 0x6f, 0x74, 0x73, 0x22, 0xa0, 0x09, 0x0a, 0x0e, 0x43, - 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x16, 0x0a, - 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, - 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, - 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, - 0x72, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x49, 0x44, 0x18, 0x03, 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, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, 0x6e, - 0x6e, 0x65, 0x6c, 0x12, 0x35, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x3c, 0x0a, 0x05, 0x73, 0x74, - 0x61, 0x74, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, - 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, 0x6c, - 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, - 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, 0x61, - 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x73, - 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x6e, 0x64, 0x5f, - 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x65, 0x6e, 0x64, 0x54, - 0x69, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, 0x69, - 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x05, 0x52, - 0x10, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x49, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, 0x64, - 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, - 0x18, 0x0c, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x54, 0x69, 0x6d, - 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x74, 0x6c, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, - 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, - 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x24, 0x0a, 0x0d, 0x69, 0x6e, 0x70, 0x75, - 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x03, 0x52, - 0x0d, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x26, - 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, - 0x18, 0x10, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x53, 0x65, - 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x03, 0x70, 0x6f, 0x73, 0x18, 0x11, 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, 0x03, 0x70, 0x6f, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, - 0x44, 0x18, 0x12, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, 0x12, - 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x13, 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, 0x52, - 0x0a, 0x14, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x6b, 0x65, 0x79, - 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, 0x65, - 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x12, - 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4b, 0x65, 0x79, 0x46, 0x69, 0x65, - 0x6c, 0x64, 0x12, 0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x15, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6d, 0x61, - 0x78, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x2e, 0x0a, 0x13, - 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x72, - 0x6f, 0x77, 0x73, 0x18, 0x16, 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x70, 0x72, 0x65, 0x66, 0x65, - 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x24, 0x0a, 0x0d, - 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x17, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x0d, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x54, 0x61, 0x73, 0x6b, - 0x49, 0x44, 0x12, 0x26, 0x0a, 0x0e, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x56, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x18, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x61, 0x6e, 0x61, 0x6c, - 0x79, 0x7a, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, 0x12, 0x6c, 0x61, - 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, - 0x18, 0x19, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6d, 0x61, - 0x78, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x1a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6d, 0x61, - 0x78, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x74, 0x6d, 0x70, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x73, 0x18, 0x1b, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0b, 0x74, 0x6d, 0x70, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x54, 0x0a, 0x18, 0x70, 0x72, 0x65, 0x5f, 0x61, - 0x6c, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x49, 0x44, 0x73, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x44, - 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x16, 0x70, 0x72, 0x65, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x61, - 0x74, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x22, 0xf6, 0x01, - 0x0a, 0x12, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, - 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, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x09, + 0x66, 0x69, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x73, 0x74, 0x61, + 0x74, 0x73, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x0a, 0x20, + 0x03, 0x28, 0x03, 0x52, 0x0f, 0x73, 0x74, 0x61, 0x74, 0x73, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x64, 0x5f, + 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x63, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x64, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, + 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x32, 0x52, 0x06, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x22, 0xb7, 0x01, 0x0a, 0x10, 0x47, 0x63, 0x43, 0x6f, 0x6e, + 0x74, 0x72, 0x6f, 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, 0x36, 0x0a, + 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x52, 0x07, 0x63, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x39, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, + 0x22, 0x12, 0x0a, 0x10, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x22, 0x65, 0x0a, 0x11, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, + 0x74, 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, 0x6e, 0x75, 0x6d, 0x5f, 0x73, 0x6c, 0x6f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x08, 0x6e, 0x75, 0x6d, 0x53, 0x6c, 0x6f, 0x74, 0x73, 0x22, 0xa0, 0x09, 0x0a, 0x0e, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x16, + 0x0a, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, + 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, + 0x72, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x72, 0x69, 0x67, 0x67, + 0x65, 0x72, 0x49, 0x44, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x03, 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, 0x1a, 0x0a, 0x08, 0x76, 0x43, - 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x76, 0x43, - 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, - 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, - 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, 0x05, - 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, - 0x12, 0x24, 0x0a, 0x0d, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x49, - 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, - 0x54, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, - 0x54, 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x6d, - 0x69, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x33, 0x0a, 0x19, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, - 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x2a, 0x3e, 0x0a, 0x0b, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x07, 0x0a, 0x03, 0x4e, 0x65, - 0x77, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x10, 0x01, 0x12, - 0x0b, 0x0a, 0x07, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, - 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x65, 0x64, 0x10, 0x03, 0x2a, 0x32, 0x0a, 0x0c, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x0a, 0x0a, 0x06, 0x4c, - 0x65, 0x67, 0x61, 0x63, 0x79, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, 0x4c, 0x30, 0x10, 0x01, 0x12, - 0x06, 0x0a, 0x02, 0x4c, 0x31, 0x10, 0x02, 0x12, 0x06, 0x0a, 0x02, 0x4c, 0x32, 0x10, 0x03, 0x2a, - 0x99, 0x01, 0x0a, 0x11, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, 0x61, 0x74, 0x63, 0x68, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x55, 0x6e, 0x63, 0x6f, 0x6d, 0x70, 0x6c, - 0x65, 0x74, 0x65, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, - 0x65, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x54, 0x6f, 0x57, 0x61, 0x74, 0x63, 0x68, 0x10, 0x02, - 0x12, 0x10, 0x0a, 0x0c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, - 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x46, 0x61, 0x69, 0x6c, 0x75, - 0x72, 0x65, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x6f, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, - 0x65, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x53, 0x75, - 0x63, 0x63, 0x65, 0x73, 0x73, 0x10, 0x06, 0x12, 0x12, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x65, 0x61, - 0x73, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x10, 0x07, 0x2a, 0xcd, 0x01, 0x0a, 0x0e, - 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x17, - 0x0a, 0x13, 0x55, 0x6e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x43, 0x6f, 0x6d, 0x70, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x65, 0x72, 0x67, 0x65, - 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x12, 0x11, 0x0a, 0x0d, - 0x4d, 0x69, 0x78, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x12, - 0x14, 0x0a, 0x10, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x69, 0x6e, 0x6f, 0x72, 0x43, 0x6f, - 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x61, - 0x6a, 0x6f, 0x72, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, - 0x1a, 0x0a, 0x16, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x30, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, - 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x18, 0x0a, 0x14, 0x43, - 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x22, 0x04, 0x08, 0x01, 0x10, 0x01, 0x2a, 0x55, 0x0a, 0x11, 0x49, - 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, 0x32, - 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x65, - 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x49, 0x6e, 0x50, 0x72, 0x6f, - 0x67, 0x72, 0x65, 0x73, 0x73, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, 0x65, - 0x64, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, - 0x10, 0x04, 0x2a, 0x29, 0x0a, 0x09, 0x47, 0x63, 0x43, 0x6f, 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, - 0x05, 0x0a, 0x01, 0x5f, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x50, 0x61, 0x75, 0x73, 0x65, 0x10, - 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x52, 0x65, 0x73, 0x75, 0x6d, 0x65, 0x10, 0x02, 0x2a, 0xb2, 0x01, - 0x0a, 0x13, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x73, 0x6b, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x75, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, - 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6e, 0x67, 0x10, - 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x10, - 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x10, 0x03, - 0x12, 0x0a, 0x0a, 0x06, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0b, 0x0a, 0x07, - 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x61, 0x6e, 0x61, - 0x6c, 0x79, 0x7a, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, 0x0c, 0x0a, 0x08, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x69, 0x6e, 0x67, 0x10, 0x07, 0x12, 0x0b, 0x0a, 0x07, 0x63, 0x6c, 0x65, 0x61, 0x6e, 0x65, - 0x64, 0x10, 0x08, 0x12, 0x0e, 0x0a, 0x0a, 0x6d, 0x65, 0x74, 0x61, 0x5f, 0x73, 0x61, 0x76, 0x65, - 0x64, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x73, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, - 0x10, 0x0a, 0x32, 0xcf, 0x29, 0x0a, 0x09, 0x44, 0x61, 0x74, 0x61, 0x43, 0x6f, 0x6f, 0x72, 0x64, - 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, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x35, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x3c, 0x0a, 0x05, 0x73, + 0x74, 0x61, 0x74, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x26, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x69, + 0x6c, 0x5f, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x66, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1d, 0x0a, 0x0a, 0x73, 0x74, + 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, + 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x65, 0x6e, 0x64, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x65, 0x6e, 0x64, + 0x54, 0x69, 0x6d, 0x65, 0x12, 0x2c, 0x0a, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x5f, + 0x69, 0x6e, 0x5f, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x05, + 0x52, 0x10, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x49, 0x6e, 0x53, 0x65, 0x63, 0x6f, 0x6e, + 0x64, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x74, 0x72, 0x79, 0x5f, 0x74, 0x69, 0x6d, 0x65, + 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x54, 0x69, + 0x6d, 0x65, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x5f, 0x74, 0x74, 0x6c, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x74, 0x6c, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, + 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, + 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x24, 0x0a, 0x0d, 0x69, 0x6e, 0x70, + 0x75, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x0f, 0x20, 0x03, 0x28, 0x03, + 0x52, 0x0d, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, + 0x26, 0x0a, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, + 0x73, 0x18, 0x10, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0e, 0x72, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2f, 0x0a, 0x03, 0x70, 0x6f, 0x73, 0x18, 0x11, + 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, 0x03, 0x70, 0x6f, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x6f, 0x64, 0x65, + 0x49, 0x44, 0x18, 0x12, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x6f, 0x64, 0x65, 0x49, 0x44, + 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x13, 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, + 0x52, 0x0a, 0x14, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x6b, 0x65, + 0x79, 0x5f, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x18, 0x14, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x73, 0x63, 0x68, + 0x65, 0x6d, 0x61, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, + 0x12, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4b, 0x65, 0x79, 0x46, 0x69, + 0x65, 0x6c, 0x64, 0x12, 0x28, 0x0a, 0x10, 0x6d, 0x61, 0x78, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x15, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x6d, + 0x61, 0x78, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x2e, 0x0a, + 0x13, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, + 0x72, 0x6f, 0x77, 0x73, 0x18, 0x16, 0x20, 0x01, 0x28, 0x03, 0x52, 0x11, 0x70, 0x72, 0x65, 0x66, + 0x65, 0x72, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x24, 0x0a, + 0x0d, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x17, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x54, 0x61, 0x73, + 0x6b, 0x49, 0x44, 0x12, 0x26, 0x0a, 0x0e, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x56, 0x65, + 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x18, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0e, 0x61, 0x6e, 0x61, + 0x6c, 0x79, 0x7a, 0x65, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x0a, 0x12, 0x6c, + 0x61, 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, + 0x65, 0x18, 0x19, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x6c, 0x61, 0x73, 0x74, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x53, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6d, + 0x61, 0x78, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x1a, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x6d, + 0x61, 0x78, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x74, 0x6d, 0x70, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x1b, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0b, 0x74, 0x6d, 0x70, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x54, 0x0a, 0x18, 0x70, 0x72, 0x65, 0x5f, + 0x61, 0x6c, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x49, 0x44, 0x73, 0x18, 0x1c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, + 0x44, 0x52, 0x61, 0x6e, 0x67, 0x65, 0x52, 0x16, 0x70, 0x72, 0x65, 0x41, 0x6c, 0x6c, 0x6f, 0x63, + 0x61, 0x74, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x22, 0xf6, + 0x01, 0x0a, 0x12, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, + 0x73, 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, 0x1a, 0x0a, 0x08, 0x76, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x76, + 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x73, 0x18, + 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, + 0x73, 0x12, 0x24, 0x0a, 0x0d, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x54, 0x61, 0x73, 0x6b, + 0x49, 0x44, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, + 0x65, 0x54, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x6d, 0x69, + 0x74, 0x54, 0x69, 0x6d, 0x65, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x63, 0x6f, 0x6d, + 0x6d, 0x69, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x33, 0x0a, 0x19, 0x44, 0x72, 0x6f, 0x70, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, 0x6c, 0x61, 0x6e, 0x49, 0x44, 0x2a, 0x3e, 0x0a, 0x0b, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x07, 0x0a, 0x03, 0x4e, + 0x65, 0x77, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x4e, 0x6f, 0x72, 0x6d, 0x61, 0x6c, 0x10, 0x01, + 0x12, 0x0b, 0x0a, 0x07, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x10, 0x02, 0x12, 0x0d, 0x0a, + 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x65, 0x64, 0x10, 0x03, 0x2a, 0x32, 0x0a, 0x0c, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x0a, 0x0a, 0x06, + 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x10, 0x00, 0x12, 0x06, 0x0a, 0x02, 0x4c, 0x30, 0x10, 0x01, + 0x12, 0x06, 0x0a, 0x02, 0x4c, 0x31, 0x10, 0x02, 0x12, 0x06, 0x0a, 0x02, 0x4c, 0x32, 0x10, 0x03, + 0x2a, 0x99, 0x01, 0x0a, 0x11, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, 0x61, 0x74, 0x63, + 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x55, 0x6e, 0x63, 0x6f, 0x6d, 0x70, + 0x6c, 0x65, 0x74, 0x65, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, + 0x74, 0x65, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x54, 0x6f, 0x57, 0x61, 0x74, 0x63, 0x68, 0x10, + 0x02, 0x12, 0x10, 0x0a, 0x0c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x53, 0x75, 0x63, 0x63, 0x65, 0x73, + 0x73, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x46, 0x61, 0x69, 0x6c, + 0x75, 0x72, 0x65, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x54, 0x6f, 0x52, 0x65, 0x6c, 0x65, 0x61, + 0x73, 0x65, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x53, + 0x75, 0x63, 0x63, 0x65, 0x73, 0x73, 0x10, 0x06, 0x12, 0x12, 0x0a, 0x0e, 0x52, 0x65, 0x6c, 0x65, + 0x61, 0x73, 0x65, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x10, 0x07, 0x2a, 0xcd, 0x01, 0x0a, + 0x0e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x17, 0x0a, 0x13, 0x55, 0x6e, 0x64, 0x65, 0x66, 0x69, 0x6e, 0x65, 0x64, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x65, 0x72, 0x67, + 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x12, 0x11, 0x0a, + 0x0d, 0x4d, 0x69, 0x78, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x03, + 0x12, 0x14, 0x0a, 0x10, 0x53, 0x69, 0x6e, 0x67, 0x6c, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x13, 0x0a, 0x0f, 0x4d, 0x69, 0x6e, 0x6f, 0x72, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x13, 0x0a, 0x0f, 0x4d, + 0x61, 0x6a, 0x6f, 0x72, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06, + 0x12, 0x1a, 0x0a, 0x16, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x30, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x18, 0x0a, 0x14, + 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x22, 0x04, 0x08, 0x01, 0x10, 0x01, 0x2a, 0x55, 0x0a, 0x11, + 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x56, + 0x32, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, + 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x49, 0x6e, 0x50, 0x72, + 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, + 0x65, 0x64, 0x10, 0x03, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x64, 0x10, 0x04, 0x2a, 0x33, 0x0a, 0x12, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, + 0x6b, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x56, 0x32, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x4c, 0x30, 0x43, 0x6f, 0x6d, 0x70, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x2a, 0x29, 0x0a, 0x09, 0x47, 0x63, 0x43, 0x6f, + 0x6d, 0x6d, 0x61, 0x6e, 0x64, 0x12, 0x05, 0x0a, 0x01, 0x5f, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, + 0x50, 0x61, 0x75, 0x73, 0x65, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x52, 0x65, 0x73, 0x75, 0x6d, + 0x65, 0x10, 0x02, 0x2a, 0xb2, 0x01, 0x0a, 0x13, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x54, 0x61, 0x73, 0x6b, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x75, + 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x65, 0x78, 0x65, 0x63, + 0x75, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x70, 0x69, 0x70, 0x65, 0x6c, + 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x70, 0x6c, + 0x65, 0x74, 0x65, 0x64, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x66, 0x61, 0x69, 0x6c, 0x65, 0x64, + 0x10, 0x04, 0x12, 0x0b, 0x0a, 0x07, 0x74, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x10, 0x05, 0x12, + 0x0d, 0x0a, 0x09, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, 0x0c, + 0x0a, 0x08, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x69, 0x6e, 0x67, 0x10, 0x07, 0x12, 0x0b, 0x0a, 0x07, + 0x63, 0x6c, 0x65, 0x61, 0x6e, 0x65, 0x64, 0x10, 0x08, 0x12, 0x0e, 0x0a, 0x0a, 0x6d, 0x65, 0x74, + 0x61, 0x5f, 0x73, 0x61, 0x76, 0x65, 0x64, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x73, 0x74, 0x61, + 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x10, 0x0a, 0x32, 0xcf, 0x29, 0x0a, 0x09, 0x44, 0x61, 0x74, + 0x61, 0x43, 0x6f, 0x6f, 0x72, 0x64, 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, 0x4c, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x12, + 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x61, 0x0a, 0x0c, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6d, 0x0a, 0x0f, 0x41, 0x73, 0x73, 0x69, 0x67, + 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, + 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x03, 0x88, 0x02, 0x01, 0x12, 0x67, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 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, + 0x6d, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, + 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x79, + 0x0a, 0x14, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, + 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, + 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, + 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x82, 0x01, 0x0a, 0x17, 0x47, 0x65, + 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, + 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, + 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7f, + 0x0a, 0x16, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, + 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, + 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, + 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x6f, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, + 0x6f, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 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, 0x4c, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x12, 0x1f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, - 0x75, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, - 0x6c, 0x75, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x61, - 0x0a, 0x0c, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x26, + 0x12, 0x5b, 0x0a, 0x0f, 0x53, 0x61, 0x76, 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, + 0x74, 0x68, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x61, 0x76, 0x65, 0x42, 0x69, 0x6e, 0x6c, + 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 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, 0x6a, 0x0a, + 0x0f, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, + 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, + 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, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x70, 0x0a, 0x11, 0x47, 0x65, 0x74, + 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x56, 0x32, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x6d, 0x0a, 0x0f, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x49, 0x44, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x41, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x03, 0x88, 0x02, 0x01, - 0x12, 0x67, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, - 0x66, 0x6f, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, - 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, 0x6d, 0x0a, 0x10, 0x47, 0x65, 0x74, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2a, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, - 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x79, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x49, - 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, - 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, - 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x42, 0x69, - 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x82, 0x01, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, - 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, - 0x63, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6f, 0x0a, 0x15, 0x47, 0x65, 0x74, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x43, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x49, 0x6e, 0x66, 0x6f, 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, 0x5b, 0x0a, 0x0f, 0x53, 0x61, - 0x76, 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, 0x73, 0x12, 0x29, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x53, 0x61, 0x76, 0x65, 0x42, 0x69, 0x6e, 0x6c, 0x6f, 0x67, 0x50, 0x61, 0x74, 0x68, - 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, 0x6a, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x52, 0x65, - 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, - 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 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, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, - 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x70, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, - 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x56, 0x32, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x56, 0x32, 0x1a, 0x2c, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x56, 0x32, 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x16, 0x47, + 0x65, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, + 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, + 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, + 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, + 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x00, 0x12, 0x76, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, - 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x56, 0x32, 0x1a, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x52, 0x65, 0x63, - 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x56, 0x32, 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x16, 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, 0x6e, - 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x12, - 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, - 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, - 0x52, 0x65, 0x63, 0x6f, 0x76, 0x65, 0x72, 0x79, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, - 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2c, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, - 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, - 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x65, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x76, 0x0a, 0x13, 0x47, - 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x73, 0x42, 0x79, 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, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, - 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x10, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, - 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, - 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x46, - 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 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, 0x71, 0x0a, 0x10, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, - 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4d, - 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4d, 0x61, 0x6e, - 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x77, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, - 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2e, 0x2e, + 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x79, 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, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x53, + 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x42, 0x79, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x10, 0x47, 0x65, 0x74, + 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2c, 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, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 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, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x80, 0x01, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x57, 0x69, 0x74, 0x68, 0x50, 0x6c, 0x61, 0x6e, 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, 0x61, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 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, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, - 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, - 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x66, 0x0a, 0x0d, 0x47, 0x65, 0x74, - 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, - 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, - 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x73, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, - 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, - 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, - 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6a, 0x0a, 0x0f, 0x53, 0x65, 0x74, 0x53, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, - 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x6b, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x31, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, - 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, - 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 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, - 0x6b, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, - 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x70, 0x6f, 0x69, 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, 0x63, 0x0a, 0x13, - 0x4d, 0x61, 0x72, 0x6b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x44, 0x72, 0x6f, 0x70, - 0x70, 0x65, 0x64, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x61, 0x72, 0x6b, 0x53, 0x65, 0x67, 0x6d, - 0x65, 0x6e, 0x74, 0x73, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 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, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x41, 0x6c, - 0x74, 0x65, 0x72, 0x65, 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, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 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, 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, 0x54, 0x0a, - 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x26, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 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, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, - 0x78, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, - 0x78, 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, 0x47, 0x65, 0x74, 0x49, 0x6e, - 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, - 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x7b, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, - 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, - 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, - 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, - 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x27, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, - 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, - 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x50, 0x0a, 0x09, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, - 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6e, 0x64, 0x65, 0x78, 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, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, - 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, - 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, - 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, - 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, - 0x63, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, - 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, - 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x30, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, - 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, - 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 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, - 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, - 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, - 0x65, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, - 0x78, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, - 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x58, 0x0a, 0x09, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x72, 0x6d, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x6d, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x72, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x65, 0x0a, 0x14, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, - 0x65, 0x54, 0x74, 0x4d, 0x73, 0x67, 0x73, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6f, - 0x72, 0x74, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x54, 0x74, 0x4d, 0x73, 0x67, 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, 0x4f, 0x0a, 0x09, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x74, - 0x72, 0x6f, 0x6c, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, - 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, 0x61, 0x0a, 0x08, 0x49, 0x6d, 0x70, 0x6f, 0x72, - 0x74, 0x56, 0x32, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, - 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, - 0x6c, 0x1a, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x78, 0x0a, 0x11, 0x47, 0x65, - 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, - 0x2f, 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, 0x49, 0x6d, 0x70, 0x6f, 0x72, - 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 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, 0x49, 0x6d, 0x70, 0x6f, - 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x12, 0x6e, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, - 0x72, 0x74, 0x73, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x4c, 0x69, 0x73, 0x74, - 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x6e, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x4c, - 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x22, 0x00, 0x32, 0xbf, 0x0f, 0x0a, 0x08, 0x44, 0x61, 0x74, 0x61, 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, - 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, + 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 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, 0x5b, 0x0a, 0x0f, 0x57, 0x61, 0x74, 0x63, 0x68, 0x44, 0x6d, 0x43, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 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, 0x12, - 0x57, 0x0a, 0x0d, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, + 0x73, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x41, 0x6c, 0x6c, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 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, 0x71, 0x0a, 0x10, 0x4d, 0x61, + 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x77, 0x0a, + 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x74, 0x65, 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, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 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, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, + 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x80, 0x01, 0x0a, 0x1b, 0x47, 0x65, 0x74, 0x43, 0x6f, + 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x57, 0x69, 0x74, + 0x68, 0x50, 0x6c, 0x61, 0x6e, 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, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 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, 0x47, 0x65, 0x74, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x57, 0x61, 0x74, + 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, + 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x66, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 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, 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, 0x50, 0x0a, 0x0c, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x12, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 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, 0x6d, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, - 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x29, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, - 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, - 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x0c, 0x53, 0x79, 0x6e, 0x63, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, 0x79, 0x6e, 0x63, - 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, 0x73, - 0x0a, 0x12, 0x52, 0x65, 0x73, 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, - 0x74, 0x61, 0x74, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x6e, 0x64, 0x53, - 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0d, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x43, 0x68, 0x61, 0x6e, - 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, 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, 0x12, 0x64, 0x0a, 0x16, - 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x79, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x47, 0x65, 0x74, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x56, + 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x2c, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x44, 0x72, 0x6f, 0x70, 0x56, 0x69, 0x72, 0x74, 0x75, 0x61, 0x6c, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6a, 0x0a, 0x0f, + 0x53, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x53, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x53, + 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6b, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, + 0x69, 0x63, 0x73, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 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, 0x6b, 0x0a, 0x17, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, + 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 0x6e, 0x74, + 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x64, 0x61, 0x74, 0x61, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, + 0x65, 0x6c, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x70, 0x6f, 0x69, 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, 0x7b, 0x0a, 0x1d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x43, 0x68, 0x61, 0x6e, 0x6e, - 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, - 0x65, 0x73, 0x73, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, - 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x33, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, - 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, - 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x4f, 0x0a, 0x09, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x23, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 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, 0x4b, 0x0a, 0x08, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x56, 0x32, 0x12, 0x20, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 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, 0x67, 0x0a, - 0x0e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, - 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, - 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, - 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, - 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x0b, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, - 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, - 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, - 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6d, - 0x70, 0x6f, 0x72, 0x74, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6d, 0x70, - 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, + 0x22, 0x00, 0x12, 0x63, 0x0a, 0x13, 0x4d, 0x61, 0x72, 0x6b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, 0x64, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x4d, 0x61, + 0x72, 0x6b, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x44, 0x72, 0x6f, 0x70, 0x70, 0x65, + 0x64, 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, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, + 0x63, 0x61, 0x73, 0x74, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x65, 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, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 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, + 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, 0x54, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, + 0x64, 0x65, 0x78, 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, 0x09, 0x51, 0x75, 0x65, - 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, - 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x52, 0x0a, 0x0a, 0x41, 0x6c, 0x74, + 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x41, 0x6c, 0x74, + 0x65, 0x72, 0x49, 0x6e, 0x64, 0x65, 0x78, 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, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x28, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, + 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x2f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, + 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, + 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x64, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, + 0x66, 0x6f, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, + 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x50, 0x0a, 0x09, 0x44, 0x72, 0x6f, 0x70, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x49, + 0x6e, 0x64, 0x65, 0x78, 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, 0x44, 0x65, + 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x28, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, + 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, + 0x69, 0x62, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x75, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, + 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, + 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x53, 0x74, 0x61, 0x74, 0x69, 0x73, 0x74, 0x69, 0x63, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7e, 0x0a, 0x15, 0x47, 0x65, 0x74, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, + 0x73, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 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, 0x64, 0x65, 0x78, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x60, 0x0a, 0x0b, 0x4c, 0x69, 0x73, + 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x65, + 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x58, 0x0a, 0x09, 0x47, + 0x63, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x6d, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x24, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x72, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x65, 0x0a, 0x14, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x44, + 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, 0x54, 0x74, 0x4d, 0x73, 0x67, 0x73, 0x12, 0x2e, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x52, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x44, 0x61, 0x74, 0x61, 0x4e, 0x6f, 0x64, 0x65, + 0x54, 0x74, 0x4d, 0x73, 0x67, 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, 0x4f, 0x0a, 0x09, + 0x47, 0x63, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x47, 0x63, + 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 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, 0x61, 0x0a, + 0x08, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x56, 0x32, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x1a, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, + 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x78, 0x0a, 0x11, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, + 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x2f, 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, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6e, 0x0a, 0x0b, 0x4c, 0x69, + 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x1a, 0x2a, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xbf, 0x0f, 0x0a, 0x08, 0x44, + 0x61, 0x74, 0x61, 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, 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, 0x5b, 0x0a, 0x0f, 0x57, 0x61, 0x74, 0x63, + 0x68, 0x44, 0x6d, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, - 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x00, 0x12, 0x61, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6d, 0x70, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, - 0x6f, 0x70, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, + 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, 0x12, 0x57, 0x0a, 0x0d, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, 0x6c, 0x75, 0x73, 0x68, + 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, 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, 0x50, 0x0a, 0x0c, + 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x56, 0x32, 0x12, 0x21, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 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, 0x6d, + 0x0a, 0x12, 0x47, 0x65, 0x74, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, + 0x0c, 0x53, 0x79, 0x6e, 0x63, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x26, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, + 0x61, 0x2e, 0x53, 0x79, 0x6e, 0x63, 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, 0x73, 0x0a, 0x12, 0x52, 0x65, 0x73, 0x65, 0x6e, 0x64, 0x53, 0x65, + 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, + 0x65, 0x73, 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x52, 0x65, 0x73, + 0x65, 0x6e, 0x64, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x57, 0x0a, 0x0d, 0x46, 0x6c, 0x75, + 0x73, 0x68, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x46, + 0x6c, 0x75, 0x73, 0x68, 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, 0x12, 0x64, 0x0a, 0x16, 0x4e, 0x6f, 0x74, 0x69, 0x66, 0x79, 0x43, 0x68, 0x61, 0x6e, + 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 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, 0x7b, 0x0a, 0x1d, 0x43, 0x68, 0x65, 0x63, + 0x6b, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x43, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x57, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x33, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, + 0x74, 0x61, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, + 0x69, 0x6f, 0x6e, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x09, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, + 0x72, 0x74, 0x12, 0x23, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 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, 0x42, 0x2e, 0x5a, 0x2c, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, - 0x64, 0x61, 0x74, 0x61, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x4b, 0x0a, 0x08, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x56, 0x32, 0x12, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 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, 0x67, 0x0a, 0x0e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, + 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, + 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5e, 0x0a, 0x0b, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x25, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x49, 0x6d, 0x70, 0x6f, + 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x51, 0x0a, 0x0a, + 0x44, 0x72, 0x6f, 0x70, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x44, + 0x72, 0x6f, 0x70, 0x49, 0x6d, 0x70, 0x6f, 0x72, 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, + 0x58, 0x0a, 0x09, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x12, 0x23, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, 0x61, 0x74, 0x61, + 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x64, 0x61, 0x74, 0x61, 0x2e, 0x51, 0x75, 0x65, 0x72, 0x79, 0x53, 0x6c, 0x6f, 0x74, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x61, 0x0a, 0x12, 0x44, 0x72, 0x6f, + 0x70, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x6c, 0x61, 0x6e, 0x12, + 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x64, + 0x61, 0x74, 0x61, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x50, 0x6c, 0x61, 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, 0x42, 0x2e, 0x5a, 0x2c, + 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, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x64, 0x61, 0x74, 0x61, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -10185,7 +10267,7 @@ func file_data_coord_proto_rawDescGZIP() []byte { return file_data_coord_proto_rawDescData } -var file_data_coord_proto_enumTypes = make([]protoimpl.EnumInfo, 7) +var file_data_coord_proto_enumTypes = make([]protoimpl.EnumInfo, 8) var file_data_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 108) var file_data_coord_proto_goTypes = []interface{}{ (SegmentType)(0), // 0: milvus.proto.data.SegmentType @@ -10193,510 +10275,512 @@ var file_data_coord_proto_goTypes = []interface{}{ (ChannelWatchState)(0), // 2: milvus.proto.data.ChannelWatchState (CompactionType)(0), // 3: milvus.proto.data.CompactionType (ImportTaskStateV2)(0), // 4: milvus.proto.data.ImportTaskStateV2 - (GcCommand)(0), // 5: milvus.proto.data.GcCommand - (CompactionTaskState)(0), // 6: milvus.proto.data.CompactionTaskState - (*Empty)(nil), // 7: milvus.proto.data.Empty - (*FlushRequest)(nil), // 8: milvus.proto.data.FlushRequest - (*FlushResponse)(nil), // 9: milvus.proto.data.FlushResponse - (*FlushChannelsRequest)(nil), // 10: milvus.proto.data.FlushChannelsRequest - (*SegmentIDRequest)(nil), // 11: milvus.proto.data.SegmentIDRequest - (*AllocSegmentRequest)(nil), // 12: milvus.proto.data.AllocSegmentRequest - (*AllocSegmentResponse)(nil), // 13: milvus.proto.data.AllocSegmentResponse - (*AssignSegmentIDRequest)(nil), // 14: milvus.proto.data.AssignSegmentIDRequest - (*SegmentIDAssignment)(nil), // 15: milvus.proto.data.SegmentIDAssignment - (*AssignSegmentIDResponse)(nil), // 16: milvus.proto.data.AssignSegmentIDResponse - (*GetSegmentStatesRequest)(nil), // 17: milvus.proto.data.GetSegmentStatesRequest - (*SegmentStateInfo)(nil), // 18: milvus.proto.data.SegmentStateInfo - (*GetSegmentStatesResponse)(nil), // 19: milvus.proto.data.GetSegmentStatesResponse - (*GetSegmentInfoRequest)(nil), // 20: milvus.proto.data.GetSegmentInfoRequest - (*GetSegmentInfoResponse)(nil), // 21: milvus.proto.data.GetSegmentInfoResponse - (*GetInsertBinlogPathsRequest)(nil), // 22: milvus.proto.data.GetInsertBinlogPathsRequest - (*GetInsertBinlogPathsResponse)(nil), // 23: milvus.proto.data.GetInsertBinlogPathsResponse - (*GetCollectionStatisticsRequest)(nil), // 24: milvus.proto.data.GetCollectionStatisticsRequest - (*GetCollectionStatisticsResponse)(nil), // 25: milvus.proto.data.GetCollectionStatisticsResponse - (*GetPartitionStatisticsRequest)(nil), // 26: milvus.proto.data.GetPartitionStatisticsRequest - (*GetPartitionStatisticsResponse)(nil), // 27: milvus.proto.data.GetPartitionStatisticsResponse - (*GetSegmentInfoChannelRequest)(nil), // 28: milvus.proto.data.GetSegmentInfoChannelRequest - (*VchannelInfo)(nil), // 29: milvus.proto.data.VchannelInfo - (*WatchDmChannelsRequest)(nil), // 30: milvus.proto.data.WatchDmChannelsRequest - (*FlushSegmentsRequest)(nil), // 31: milvus.proto.data.FlushSegmentsRequest - (*SegmentMsg)(nil), // 32: milvus.proto.data.SegmentMsg - (*SegmentInfo)(nil), // 33: milvus.proto.data.SegmentInfo - (*SegmentStartPosition)(nil), // 34: milvus.proto.data.SegmentStartPosition - (*SaveBinlogPathsRequest)(nil), // 35: milvus.proto.data.SaveBinlogPathsRequest - (*CheckPoint)(nil), // 36: milvus.proto.data.CheckPoint - (*DeltaLogInfo)(nil), // 37: milvus.proto.data.DeltaLogInfo - (*ChannelStatus)(nil), // 38: milvus.proto.data.ChannelStatus - (*DataNodeInfo)(nil), // 39: milvus.proto.data.DataNodeInfo - (*SegmentBinlogs)(nil), // 40: milvus.proto.data.SegmentBinlogs - (*FieldBinlog)(nil), // 41: milvus.proto.data.FieldBinlog - (*TextIndexStats)(nil), // 42: milvus.proto.data.TextIndexStats - (*Binlog)(nil), // 43: milvus.proto.data.Binlog - (*GetRecoveryInfoResponse)(nil), // 44: milvus.proto.data.GetRecoveryInfoResponse - (*GetRecoveryInfoRequest)(nil), // 45: milvus.proto.data.GetRecoveryInfoRequest - (*GetRecoveryInfoResponseV2)(nil), // 46: milvus.proto.data.GetRecoveryInfoResponseV2 - (*GetRecoveryInfoRequestV2)(nil), // 47: milvus.proto.data.GetRecoveryInfoRequestV2 - (*GetChannelRecoveryInfoRequest)(nil), // 48: milvus.proto.data.GetChannelRecoveryInfoRequest - (*GetChannelRecoveryInfoResponse)(nil), // 49: milvus.proto.data.GetChannelRecoveryInfoResponse - (*GetSegmentsByStatesRequest)(nil), // 50: milvus.proto.data.GetSegmentsByStatesRequest - (*GetSegmentsByStatesResponse)(nil), // 51: milvus.proto.data.GetSegmentsByStatesResponse - (*GetFlushedSegmentsRequest)(nil), // 52: milvus.proto.data.GetFlushedSegmentsRequest - (*GetFlushedSegmentsResponse)(nil), // 53: milvus.proto.data.GetFlushedSegmentsResponse - (*SegmentFlushCompletedMsg)(nil), // 54: milvus.proto.data.SegmentFlushCompletedMsg - (*ChannelWatchInfo)(nil), // 55: milvus.proto.data.ChannelWatchInfo - (*CompactionStateRequest)(nil), // 56: milvus.proto.data.CompactionStateRequest - (*SyncSegmentInfo)(nil), // 57: milvus.proto.data.SyncSegmentInfo - (*SyncSegmentsRequest)(nil), // 58: milvus.proto.data.SyncSegmentsRequest - (*CompactionSegmentBinlogs)(nil), // 59: milvus.proto.data.CompactionSegmentBinlogs - (*CompactionPlan)(nil), // 60: milvus.proto.data.CompactionPlan - (*CompactionSegment)(nil), // 61: milvus.proto.data.CompactionSegment - (*CompactionPlanResult)(nil), // 62: milvus.proto.data.CompactionPlanResult - (*CompactionStateResponse)(nil), // 63: milvus.proto.data.CompactionStateResponse - (*SegmentFieldBinlogMeta)(nil), // 64: milvus.proto.data.SegmentFieldBinlogMeta - (*WatchChannelsRequest)(nil), // 65: milvus.proto.data.WatchChannelsRequest - (*WatchChannelsResponse)(nil), // 66: milvus.proto.data.WatchChannelsResponse - (*SetSegmentStateRequest)(nil), // 67: milvus.proto.data.SetSegmentStateRequest - (*SetSegmentStateResponse)(nil), // 68: milvus.proto.data.SetSegmentStateResponse - (*DropVirtualChannelRequest)(nil), // 69: milvus.proto.data.DropVirtualChannelRequest - (*DropVirtualChannelSegment)(nil), // 70: milvus.proto.data.DropVirtualChannelSegment - (*DropVirtualChannelResponse)(nil), // 71: milvus.proto.data.DropVirtualChannelResponse - (*UpdateSegmentStatisticsRequest)(nil), // 72: milvus.proto.data.UpdateSegmentStatisticsRequest - (*UpdateChannelCheckpointRequest)(nil), // 73: milvus.proto.data.UpdateChannelCheckpointRequest - (*ResendSegmentStatsRequest)(nil), // 74: milvus.proto.data.ResendSegmentStatsRequest - (*ResendSegmentStatsResponse)(nil), // 75: milvus.proto.data.ResendSegmentStatsResponse - (*MarkSegmentsDroppedRequest)(nil), // 76: milvus.proto.data.MarkSegmentsDroppedRequest - (*SegmentReferenceLock)(nil), // 77: milvus.proto.data.SegmentReferenceLock - (*AlterCollectionRequest)(nil), // 78: milvus.proto.data.AlterCollectionRequest - (*GcConfirmRequest)(nil), // 79: milvus.proto.data.GcConfirmRequest - (*GcConfirmResponse)(nil), // 80: milvus.proto.data.GcConfirmResponse - (*ReportDataNodeTtMsgsRequest)(nil), // 81: milvus.proto.data.ReportDataNodeTtMsgsRequest - (*GetFlushStateRequest)(nil), // 82: milvus.proto.data.GetFlushStateRequest - (*ChannelOperationsRequest)(nil), // 83: milvus.proto.data.ChannelOperationsRequest - (*ChannelOperationProgressResponse)(nil), // 84: milvus.proto.data.ChannelOperationProgressResponse - (*PreImportRequest)(nil), // 85: milvus.proto.data.PreImportRequest - (*IDRange)(nil), // 86: milvus.proto.data.IDRange - (*ImportRequestSegment)(nil), // 87: milvus.proto.data.ImportRequestSegment - (*ImportRequest)(nil), // 88: milvus.proto.data.ImportRequest - (*QueryPreImportRequest)(nil), // 89: milvus.proto.data.QueryPreImportRequest - (*PartitionImportStats)(nil), // 90: milvus.proto.data.PartitionImportStats - (*ImportFileStats)(nil), // 91: milvus.proto.data.ImportFileStats - (*QueryPreImportResponse)(nil), // 92: milvus.proto.data.QueryPreImportResponse - (*QueryImportRequest)(nil), // 93: milvus.proto.data.QueryImportRequest - (*ImportSegmentInfo)(nil), // 94: milvus.proto.data.ImportSegmentInfo - (*QueryImportResponse)(nil), // 95: milvus.proto.data.QueryImportResponse - (*DropImportRequest)(nil), // 96: milvus.proto.data.DropImportRequest - (*ImportJob)(nil), // 97: milvus.proto.data.ImportJob - (*PreImportTask)(nil), // 98: milvus.proto.data.PreImportTask - (*ImportTaskV2)(nil), // 99: milvus.proto.data.ImportTaskV2 - (*GcControlRequest)(nil), // 100: milvus.proto.data.GcControlRequest - (*QuerySlotRequest)(nil), // 101: milvus.proto.data.QuerySlotRequest - (*QuerySlotResponse)(nil), // 102: milvus.proto.data.QuerySlotResponse - (*CompactionTask)(nil), // 103: milvus.proto.data.CompactionTask - (*PartitionStatsInfo)(nil), // 104: milvus.proto.data.PartitionStatsInfo - (*DropCompactionPlanRequest)(nil), // 105: milvus.proto.data.DropCompactionPlanRequest - nil, // 106: milvus.proto.data.FlushResponse.ChannelCpsEntry - nil, // 107: milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry - nil, // 108: milvus.proto.data.VchannelInfo.PartitionStatsVersionsEntry - nil, // 109: milvus.proto.data.SegmentInfo.TextStatsLogsEntry - nil, // 110: milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry - nil, // 111: milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry - nil, // 112: milvus.proto.data.PartitionImportStats.PartitionRowsEntry - nil, // 113: milvus.proto.data.PartitionImportStats.PartitionDataSizeEntry - nil, // 114: milvus.proto.data.ImportFileStats.HashedStatsEntry - (*commonpb.MsgBase)(nil), // 115: milvus.proto.common.MsgBase - (*commonpb.Status)(nil), // 116: milvus.proto.common.Status - (commonpb.SegmentState)(0), // 117: milvus.proto.common.SegmentState - (*msgpb.MsgPosition)(nil), // 118: milvus.proto.msg.MsgPosition - (*internalpb.StringList)(nil), // 119: milvus.proto.internal.StringList - (*commonpb.KeyValuePair)(nil), // 120: milvus.proto.common.KeyValuePair - (*schemapb.CollectionSchema)(nil), // 121: milvus.proto.schema.CollectionSchema - (*commonpb.KeyDataPair)(nil), // 122: milvus.proto.common.KeyDataPair - (*commonpb.SegmentStats)(nil), // 123: milvus.proto.common.SegmentStats - (*msgpb.DataNodeTtMsg)(nil), // 124: milvus.proto.msg.DataNodeTtMsg - (*internalpb.ImportFile)(nil), // 125: milvus.proto.internal.ImportFile - (internalpb.ImportJobState)(0), // 126: milvus.proto.internal.ImportJobState - (*schemapb.FieldSchema)(nil), // 127: milvus.proto.schema.FieldSchema - (*milvuspb.GetComponentStatesRequest)(nil), // 128: milvus.proto.milvus.GetComponentStatesRequest - (*internalpb.GetTimeTickChannelRequest)(nil), // 129: milvus.proto.internal.GetTimeTickChannelRequest - (*internalpb.GetStatisticsChannelRequest)(nil), // 130: milvus.proto.internal.GetStatisticsChannelRequest - (*milvuspb.GetFlushAllStateRequest)(nil), // 131: milvus.proto.milvus.GetFlushAllStateRequest - (*internalpb.ShowConfigurationsRequest)(nil), // 132: milvus.proto.internal.ShowConfigurationsRequest - (*milvuspb.GetMetricsRequest)(nil), // 133: milvus.proto.milvus.GetMetricsRequest - (*milvuspb.ManualCompactionRequest)(nil), // 134: milvus.proto.milvus.ManualCompactionRequest - (*milvuspb.GetCompactionStateRequest)(nil), // 135: milvus.proto.milvus.GetCompactionStateRequest - (*milvuspb.GetCompactionPlansRequest)(nil), // 136: milvus.proto.milvus.GetCompactionPlansRequest - (*milvuspb.CheckHealthRequest)(nil), // 137: milvus.proto.milvus.CheckHealthRequest - (*indexpb.CreateIndexRequest)(nil), // 138: milvus.proto.index.CreateIndexRequest - (*indexpb.AlterIndexRequest)(nil), // 139: milvus.proto.index.AlterIndexRequest - (*indexpb.GetIndexStateRequest)(nil), // 140: milvus.proto.index.GetIndexStateRequest - (*indexpb.GetSegmentIndexStateRequest)(nil), // 141: milvus.proto.index.GetSegmentIndexStateRequest - (*indexpb.GetIndexInfoRequest)(nil), // 142: milvus.proto.index.GetIndexInfoRequest - (*indexpb.DropIndexRequest)(nil), // 143: milvus.proto.index.DropIndexRequest - (*indexpb.DescribeIndexRequest)(nil), // 144: milvus.proto.index.DescribeIndexRequest - (*indexpb.GetIndexStatisticsRequest)(nil), // 145: milvus.proto.index.GetIndexStatisticsRequest - (*indexpb.GetIndexBuildProgressRequest)(nil), // 146: milvus.proto.index.GetIndexBuildProgressRequest - (*indexpb.ListIndexesRequest)(nil), // 147: milvus.proto.index.ListIndexesRequest - (*internalpb.ImportRequestInternal)(nil), // 148: milvus.proto.internal.ImportRequestInternal - (*internalpb.GetImportProgressRequest)(nil), // 149: milvus.proto.internal.GetImportProgressRequest - (*internalpb.ListImportsRequestInternal)(nil), // 150: milvus.proto.internal.ListImportsRequestInternal - (*milvuspb.ComponentStates)(nil), // 151: milvus.proto.milvus.ComponentStates - (*milvuspb.StringResponse)(nil), // 152: milvus.proto.milvus.StringResponse - (*milvuspb.GetFlushAllStateResponse)(nil), // 153: milvus.proto.milvus.GetFlushAllStateResponse - (*internalpb.ShowConfigurationsResponse)(nil), // 154: milvus.proto.internal.ShowConfigurationsResponse - (*milvuspb.GetMetricsResponse)(nil), // 155: milvus.proto.milvus.GetMetricsResponse - (*milvuspb.ManualCompactionResponse)(nil), // 156: milvus.proto.milvus.ManualCompactionResponse - (*milvuspb.GetCompactionStateResponse)(nil), // 157: milvus.proto.milvus.GetCompactionStateResponse - (*milvuspb.GetCompactionPlansResponse)(nil), // 158: milvus.proto.milvus.GetCompactionPlansResponse - (*milvuspb.GetFlushStateResponse)(nil), // 159: milvus.proto.milvus.GetFlushStateResponse - (*milvuspb.CheckHealthResponse)(nil), // 160: milvus.proto.milvus.CheckHealthResponse - (*indexpb.GetIndexStateResponse)(nil), // 161: milvus.proto.index.GetIndexStateResponse - (*indexpb.GetSegmentIndexStateResponse)(nil), // 162: milvus.proto.index.GetSegmentIndexStateResponse - (*indexpb.GetIndexInfoResponse)(nil), // 163: milvus.proto.index.GetIndexInfoResponse - (*indexpb.DescribeIndexResponse)(nil), // 164: milvus.proto.index.DescribeIndexResponse - (*indexpb.GetIndexStatisticsResponse)(nil), // 165: milvus.proto.index.GetIndexStatisticsResponse - (*indexpb.GetIndexBuildProgressResponse)(nil), // 166: milvus.proto.index.GetIndexBuildProgressResponse - (*indexpb.ListIndexesResponse)(nil), // 167: milvus.proto.index.ListIndexesResponse - (*internalpb.ImportResponse)(nil), // 168: milvus.proto.internal.ImportResponse - (*internalpb.GetImportProgressResponse)(nil), // 169: milvus.proto.internal.GetImportProgressResponse - (*internalpb.ListImportsResponse)(nil), // 170: milvus.proto.internal.ListImportsResponse + (ImportTaskSourceV2)(0), // 5: milvus.proto.data.ImportTaskSourceV2 + (GcCommand)(0), // 6: milvus.proto.data.GcCommand + (CompactionTaskState)(0), // 7: milvus.proto.data.CompactionTaskState + (*Empty)(nil), // 8: milvus.proto.data.Empty + (*FlushRequest)(nil), // 9: milvus.proto.data.FlushRequest + (*FlushResponse)(nil), // 10: milvus.proto.data.FlushResponse + (*FlushChannelsRequest)(nil), // 11: milvus.proto.data.FlushChannelsRequest + (*SegmentIDRequest)(nil), // 12: milvus.proto.data.SegmentIDRequest + (*AllocSegmentRequest)(nil), // 13: milvus.proto.data.AllocSegmentRequest + (*AllocSegmentResponse)(nil), // 14: milvus.proto.data.AllocSegmentResponse + (*AssignSegmentIDRequest)(nil), // 15: milvus.proto.data.AssignSegmentIDRequest + (*SegmentIDAssignment)(nil), // 16: milvus.proto.data.SegmentIDAssignment + (*AssignSegmentIDResponse)(nil), // 17: milvus.proto.data.AssignSegmentIDResponse + (*GetSegmentStatesRequest)(nil), // 18: milvus.proto.data.GetSegmentStatesRequest + (*SegmentStateInfo)(nil), // 19: milvus.proto.data.SegmentStateInfo + (*GetSegmentStatesResponse)(nil), // 20: milvus.proto.data.GetSegmentStatesResponse + (*GetSegmentInfoRequest)(nil), // 21: milvus.proto.data.GetSegmentInfoRequest + (*GetSegmentInfoResponse)(nil), // 22: milvus.proto.data.GetSegmentInfoResponse + (*GetInsertBinlogPathsRequest)(nil), // 23: milvus.proto.data.GetInsertBinlogPathsRequest + (*GetInsertBinlogPathsResponse)(nil), // 24: milvus.proto.data.GetInsertBinlogPathsResponse + (*GetCollectionStatisticsRequest)(nil), // 25: milvus.proto.data.GetCollectionStatisticsRequest + (*GetCollectionStatisticsResponse)(nil), // 26: milvus.proto.data.GetCollectionStatisticsResponse + (*GetPartitionStatisticsRequest)(nil), // 27: milvus.proto.data.GetPartitionStatisticsRequest + (*GetPartitionStatisticsResponse)(nil), // 28: milvus.proto.data.GetPartitionStatisticsResponse + (*GetSegmentInfoChannelRequest)(nil), // 29: milvus.proto.data.GetSegmentInfoChannelRequest + (*VchannelInfo)(nil), // 30: milvus.proto.data.VchannelInfo + (*WatchDmChannelsRequest)(nil), // 31: milvus.proto.data.WatchDmChannelsRequest + (*FlushSegmentsRequest)(nil), // 32: milvus.proto.data.FlushSegmentsRequest + (*SegmentMsg)(nil), // 33: milvus.proto.data.SegmentMsg + (*SegmentInfo)(nil), // 34: milvus.proto.data.SegmentInfo + (*SegmentStartPosition)(nil), // 35: milvus.proto.data.SegmentStartPosition + (*SaveBinlogPathsRequest)(nil), // 36: milvus.proto.data.SaveBinlogPathsRequest + (*CheckPoint)(nil), // 37: milvus.proto.data.CheckPoint + (*DeltaLogInfo)(nil), // 38: milvus.proto.data.DeltaLogInfo + (*ChannelStatus)(nil), // 39: milvus.proto.data.ChannelStatus + (*DataNodeInfo)(nil), // 40: milvus.proto.data.DataNodeInfo + (*SegmentBinlogs)(nil), // 41: milvus.proto.data.SegmentBinlogs + (*FieldBinlog)(nil), // 42: milvus.proto.data.FieldBinlog + (*TextIndexStats)(nil), // 43: milvus.proto.data.TextIndexStats + (*Binlog)(nil), // 44: milvus.proto.data.Binlog + (*GetRecoveryInfoResponse)(nil), // 45: milvus.proto.data.GetRecoveryInfoResponse + (*GetRecoveryInfoRequest)(nil), // 46: milvus.proto.data.GetRecoveryInfoRequest + (*GetRecoveryInfoResponseV2)(nil), // 47: milvus.proto.data.GetRecoveryInfoResponseV2 + (*GetRecoveryInfoRequestV2)(nil), // 48: milvus.proto.data.GetRecoveryInfoRequestV2 + (*GetChannelRecoveryInfoRequest)(nil), // 49: milvus.proto.data.GetChannelRecoveryInfoRequest + (*GetChannelRecoveryInfoResponse)(nil), // 50: milvus.proto.data.GetChannelRecoveryInfoResponse + (*GetSegmentsByStatesRequest)(nil), // 51: milvus.proto.data.GetSegmentsByStatesRequest + (*GetSegmentsByStatesResponse)(nil), // 52: milvus.proto.data.GetSegmentsByStatesResponse + (*GetFlushedSegmentsRequest)(nil), // 53: milvus.proto.data.GetFlushedSegmentsRequest + (*GetFlushedSegmentsResponse)(nil), // 54: milvus.proto.data.GetFlushedSegmentsResponse + (*SegmentFlushCompletedMsg)(nil), // 55: milvus.proto.data.SegmentFlushCompletedMsg + (*ChannelWatchInfo)(nil), // 56: milvus.proto.data.ChannelWatchInfo + (*CompactionStateRequest)(nil), // 57: milvus.proto.data.CompactionStateRequest + (*SyncSegmentInfo)(nil), // 58: milvus.proto.data.SyncSegmentInfo + (*SyncSegmentsRequest)(nil), // 59: milvus.proto.data.SyncSegmentsRequest + (*CompactionSegmentBinlogs)(nil), // 60: milvus.proto.data.CompactionSegmentBinlogs + (*CompactionPlan)(nil), // 61: milvus.proto.data.CompactionPlan + (*CompactionSegment)(nil), // 62: milvus.proto.data.CompactionSegment + (*CompactionPlanResult)(nil), // 63: milvus.proto.data.CompactionPlanResult + (*CompactionStateResponse)(nil), // 64: milvus.proto.data.CompactionStateResponse + (*SegmentFieldBinlogMeta)(nil), // 65: milvus.proto.data.SegmentFieldBinlogMeta + (*WatchChannelsRequest)(nil), // 66: milvus.proto.data.WatchChannelsRequest + (*WatchChannelsResponse)(nil), // 67: milvus.proto.data.WatchChannelsResponse + (*SetSegmentStateRequest)(nil), // 68: milvus.proto.data.SetSegmentStateRequest + (*SetSegmentStateResponse)(nil), // 69: milvus.proto.data.SetSegmentStateResponse + (*DropVirtualChannelRequest)(nil), // 70: milvus.proto.data.DropVirtualChannelRequest + (*DropVirtualChannelSegment)(nil), // 71: milvus.proto.data.DropVirtualChannelSegment + (*DropVirtualChannelResponse)(nil), // 72: milvus.proto.data.DropVirtualChannelResponse + (*UpdateSegmentStatisticsRequest)(nil), // 73: milvus.proto.data.UpdateSegmentStatisticsRequest + (*UpdateChannelCheckpointRequest)(nil), // 74: milvus.proto.data.UpdateChannelCheckpointRequest + (*ResendSegmentStatsRequest)(nil), // 75: milvus.proto.data.ResendSegmentStatsRequest + (*ResendSegmentStatsResponse)(nil), // 76: milvus.proto.data.ResendSegmentStatsResponse + (*MarkSegmentsDroppedRequest)(nil), // 77: milvus.proto.data.MarkSegmentsDroppedRequest + (*SegmentReferenceLock)(nil), // 78: milvus.proto.data.SegmentReferenceLock + (*AlterCollectionRequest)(nil), // 79: milvus.proto.data.AlterCollectionRequest + (*GcConfirmRequest)(nil), // 80: milvus.proto.data.GcConfirmRequest + (*GcConfirmResponse)(nil), // 81: milvus.proto.data.GcConfirmResponse + (*ReportDataNodeTtMsgsRequest)(nil), // 82: milvus.proto.data.ReportDataNodeTtMsgsRequest + (*GetFlushStateRequest)(nil), // 83: milvus.proto.data.GetFlushStateRequest + (*ChannelOperationsRequest)(nil), // 84: milvus.proto.data.ChannelOperationsRequest + (*ChannelOperationProgressResponse)(nil), // 85: milvus.proto.data.ChannelOperationProgressResponse + (*PreImportRequest)(nil), // 86: milvus.proto.data.PreImportRequest + (*IDRange)(nil), // 87: milvus.proto.data.IDRange + (*ImportRequestSegment)(nil), // 88: milvus.proto.data.ImportRequestSegment + (*ImportRequest)(nil), // 89: milvus.proto.data.ImportRequest + (*QueryPreImportRequest)(nil), // 90: milvus.proto.data.QueryPreImportRequest + (*PartitionImportStats)(nil), // 91: milvus.proto.data.PartitionImportStats + (*ImportFileStats)(nil), // 92: milvus.proto.data.ImportFileStats + (*QueryPreImportResponse)(nil), // 93: milvus.proto.data.QueryPreImportResponse + (*QueryImportRequest)(nil), // 94: milvus.proto.data.QueryImportRequest + (*ImportSegmentInfo)(nil), // 95: milvus.proto.data.ImportSegmentInfo + (*QueryImportResponse)(nil), // 96: milvus.proto.data.QueryImportResponse + (*DropImportRequest)(nil), // 97: milvus.proto.data.DropImportRequest + (*ImportJob)(nil), // 98: milvus.proto.data.ImportJob + (*PreImportTask)(nil), // 99: milvus.proto.data.PreImportTask + (*ImportTaskV2)(nil), // 100: milvus.proto.data.ImportTaskV2 + (*GcControlRequest)(nil), // 101: milvus.proto.data.GcControlRequest + (*QuerySlotRequest)(nil), // 102: milvus.proto.data.QuerySlotRequest + (*QuerySlotResponse)(nil), // 103: milvus.proto.data.QuerySlotResponse + (*CompactionTask)(nil), // 104: milvus.proto.data.CompactionTask + (*PartitionStatsInfo)(nil), // 105: milvus.proto.data.PartitionStatsInfo + (*DropCompactionPlanRequest)(nil), // 106: milvus.proto.data.DropCompactionPlanRequest + nil, // 107: milvus.proto.data.FlushResponse.ChannelCpsEntry + nil, // 108: milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry + nil, // 109: milvus.proto.data.VchannelInfo.PartitionStatsVersionsEntry + nil, // 110: milvus.proto.data.SegmentInfo.TextStatsLogsEntry + nil, // 111: milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry + nil, // 112: milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry + nil, // 113: milvus.proto.data.PartitionImportStats.PartitionRowsEntry + nil, // 114: milvus.proto.data.PartitionImportStats.PartitionDataSizeEntry + nil, // 115: milvus.proto.data.ImportFileStats.HashedStatsEntry + (*commonpb.MsgBase)(nil), // 116: milvus.proto.common.MsgBase + (*commonpb.Status)(nil), // 117: milvus.proto.common.Status + (commonpb.SegmentState)(0), // 118: milvus.proto.common.SegmentState + (*msgpb.MsgPosition)(nil), // 119: milvus.proto.msg.MsgPosition + (*internalpb.StringList)(nil), // 120: milvus.proto.internal.StringList + (*commonpb.KeyValuePair)(nil), // 121: milvus.proto.common.KeyValuePair + (*schemapb.CollectionSchema)(nil), // 122: milvus.proto.schema.CollectionSchema + (*commonpb.KeyDataPair)(nil), // 123: milvus.proto.common.KeyDataPair + (*commonpb.SegmentStats)(nil), // 124: milvus.proto.common.SegmentStats + (*msgpb.DataNodeTtMsg)(nil), // 125: milvus.proto.msg.DataNodeTtMsg + (*internalpb.ImportFile)(nil), // 126: milvus.proto.internal.ImportFile + (internalpb.ImportJobState)(0), // 127: milvus.proto.internal.ImportJobState + (*schemapb.FieldSchema)(nil), // 128: milvus.proto.schema.FieldSchema + (*milvuspb.GetComponentStatesRequest)(nil), // 129: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetTimeTickChannelRequest)(nil), // 130: milvus.proto.internal.GetTimeTickChannelRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 131: milvus.proto.internal.GetStatisticsChannelRequest + (*milvuspb.GetFlushAllStateRequest)(nil), // 132: milvus.proto.milvus.GetFlushAllStateRequest + (*internalpb.ShowConfigurationsRequest)(nil), // 133: milvus.proto.internal.ShowConfigurationsRequest + (*milvuspb.GetMetricsRequest)(nil), // 134: milvus.proto.milvus.GetMetricsRequest + (*milvuspb.ManualCompactionRequest)(nil), // 135: milvus.proto.milvus.ManualCompactionRequest + (*milvuspb.GetCompactionStateRequest)(nil), // 136: milvus.proto.milvus.GetCompactionStateRequest + (*milvuspb.GetCompactionPlansRequest)(nil), // 137: milvus.proto.milvus.GetCompactionPlansRequest + (*milvuspb.CheckHealthRequest)(nil), // 138: milvus.proto.milvus.CheckHealthRequest + (*indexpb.CreateIndexRequest)(nil), // 139: milvus.proto.index.CreateIndexRequest + (*indexpb.AlterIndexRequest)(nil), // 140: milvus.proto.index.AlterIndexRequest + (*indexpb.GetIndexStateRequest)(nil), // 141: milvus.proto.index.GetIndexStateRequest + (*indexpb.GetSegmentIndexStateRequest)(nil), // 142: milvus.proto.index.GetSegmentIndexStateRequest + (*indexpb.GetIndexInfoRequest)(nil), // 143: milvus.proto.index.GetIndexInfoRequest + (*indexpb.DropIndexRequest)(nil), // 144: milvus.proto.index.DropIndexRequest + (*indexpb.DescribeIndexRequest)(nil), // 145: milvus.proto.index.DescribeIndexRequest + (*indexpb.GetIndexStatisticsRequest)(nil), // 146: milvus.proto.index.GetIndexStatisticsRequest + (*indexpb.GetIndexBuildProgressRequest)(nil), // 147: milvus.proto.index.GetIndexBuildProgressRequest + (*indexpb.ListIndexesRequest)(nil), // 148: milvus.proto.index.ListIndexesRequest + (*internalpb.ImportRequestInternal)(nil), // 149: milvus.proto.internal.ImportRequestInternal + (*internalpb.GetImportProgressRequest)(nil), // 150: milvus.proto.internal.GetImportProgressRequest + (*internalpb.ListImportsRequestInternal)(nil), // 151: milvus.proto.internal.ListImportsRequestInternal + (*milvuspb.ComponentStates)(nil), // 152: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 153: milvus.proto.milvus.StringResponse + (*milvuspb.GetFlushAllStateResponse)(nil), // 154: milvus.proto.milvus.GetFlushAllStateResponse + (*internalpb.ShowConfigurationsResponse)(nil), // 155: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 156: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.ManualCompactionResponse)(nil), // 157: milvus.proto.milvus.ManualCompactionResponse + (*milvuspb.GetCompactionStateResponse)(nil), // 158: milvus.proto.milvus.GetCompactionStateResponse + (*milvuspb.GetCompactionPlansResponse)(nil), // 159: milvus.proto.milvus.GetCompactionPlansResponse + (*milvuspb.GetFlushStateResponse)(nil), // 160: milvus.proto.milvus.GetFlushStateResponse + (*milvuspb.CheckHealthResponse)(nil), // 161: milvus.proto.milvus.CheckHealthResponse + (*indexpb.GetIndexStateResponse)(nil), // 162: milvus.proto.index.GetIndexStateResponse + (*indexpb.GetSegmentIndexStateResponse)(nil), // 163: milvus.proto.index.GetSegmentIndexStateResponse + (*indexpb.GetIndexInfoResponse)(nil), // 164: milvus.proto.index.GetIndexInfoResponse + (*indexpb.DescribeIndexResponse)(nil), // 165: milvus.proto.index.DescribeIndexResponse + (*indexpb.GetIndexStatisticsResponse)(nil), // 166: milvus.proto.index.GetIndexStatisticsResponse + (*indexpb.GetIndexBuildProgressResponse)(nil), // 167: milvus.proto.index.GetIndexBuildProgressResponse + (*indexpb.ListIndexesResponse)(nil), // 168: milvus.proto.index.ListIndexesResponse + (*internalpb.ImportResponse)(nil), // 169: milvus.proto.internal.ImportResponse + (*internalpb.GetImportProgressResponse)(nil), // 170: milvus.proto.internal.GetImportProgressResponse + (*internalpb.ListImportsResponse)(nil), // 171: milvus.proto.internal.ListImportsResponse } var file_data_coord_proto_depIdxs = []int32{ - 115, // 0: milvus.proto.data.FlushRequest.base:type_name -> milvus.proto.common.MsgBase - 116, // 1: milvus.proto.data.FlushResponse.status:type_name -> milvus.proto.common.Status - 106, // 2: milvus.proto.data.FlushResponse.channel_cps:type_name -> milvus.proto.data.FlushResponse.ChannelCpsEntry - 115, // 3: milvus.proto.data.FlushChannelsRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 0: milvus.proto.data.FlushRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 1: milvus.proto.data.FlushResponse.status:type_name -> milvus.proto.common.Status + 107, // 2: milvus.proto.data.FlushResponse.channel_cps:type_name -> milvus.proto.data.FlushResponse.ChannelCpsEntry + 116, // 3: milvus.proto.data.FlushChannelsRequest.base:type_name -> milvus.proto.common.MsgBase 1, // 4: milvus.proto.data.SegmentIDRequest.level:type_name -> milvus.proto.data.SegmentLevel - 33, // 5: milvus.proto.data.AllocSegmentResponse.segment_info:type_name -> milvus.proto.data.SegmentInfo - 116, // 6: milvus.proto.data.AllocSegmentResponse.status:type_name -> milvus.proto.common.Status - 11, // 7: milvus.proto.data.AssignSegmentIDRequest.segmentIDRequests:type_name -> milvus.proto.data.SegmentIDRequest - 116, // 8: milvus.proto.data.SegmentIDAssignment.status:type_name -> milvus.proto.common.Status - 15, // 9: milvus.proto.data.AssignSegmentIDResponse.segIDAssignments:type_name -> milvus.proto.data.SegmentIDAssignment - 116, // 10: milvus.proto.data.AssignSegmentIDResponse.status:type_name -> milvus.proto.common.Status - 115, // 11: milvus.proto.data.GetSegmentStatesRequest.base:type_name -> milvus.proto.common.MsgBase - 117, // 12: milvus.proto.data.SegmentStateInfo.state:type_name -> milvus.proto.common.SegmentState - 118, // 13: milvus.proto.data.SegmentStateInfo.start_position:type_name -> milvus.proto.msg.MsgPosition - 118, // 14: milvus.proto.data.SegmentStateInfo.end_position:type_name -> milvus.proto.msg.MsgPosition - 116, // 15: milvus.proto.data.SegmentStateInfo.status:type_name -> milvus.proto.common.Status - 116, // 16: milvus.proto.data.GetSegmentStatesResponse.status:type_name -> milvus.proto.common.Status - 18, // 17: milvus.proto.data.GetSegmentStatesResponse.states:type_name -> milvus.proto.data.SegmentStateInfo - 115, // 18: milvus.proto.data.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 116, // 19: milvus.proto.data.GetSegmentInfoResponse.status:type_name -> milvus.proto.common.Status - 33, // 20: milvus.proto.data.GetSegmentInfoResponse.infos:type_name -> milvus.proto.data.SegmentInfo - 107, // 21: milvus.proto.data.GetSegmentInfoResponse.channel_checkpoint:type_name -> milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry - 115, // 22: milvus.proto.data.GetInsertBinlogPathsRequest.base:type_name -> milvus.proto.common.MsgBase - 119, // 23: milvus.proto.data.GetInsertBinlogPathsResponse.paths:type_name -> milvus.proto.internal.StringList - 116, // 24: milvus.proto.data.GetInsertBinlogPathsResponse.status:type_name -> milvus.proto.common.Status - 115, // 25: milvus.proto.data.GetCollectionStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase - 120, // 26: milvus.proto.data.GetCollectionStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair - 116, // 27: milvus.proto.data.GetCollectionStatisticsResponse.status:type_name -> milvus.proto.common.Status - 115, // 28: milvus.proto.data.GetPartitionStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase - 120, // 29: milvus.proto.data.GetPartitionStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair - 116, // 30: milvus.proto.data.GetPartitionStatisticsResponse.status:type_name -> milvus.proto.common.Status - 118, // 31: milvus.proto.data.VchannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition - 33, // 32: milvus.proto.data.VchannelInfo.unflushedSegments:type_name -> milvus.proto.data.SegmentInfo - 33, // 33: milvus.proto.data.VchannelInfo.flushedSegments:type_name -> milvus.proto.data.SegmentInfo - 33, // 34: milvus.proto.data.VchannelInfo.dropped_segments:type_name -> milvus.proto.data.SegmentInfo - 33, // 35: milvus.proto.data.VchannelInfo.indexed_segments:type_name -> milvus.proto.data.SegmentInfo - 108, // 36: milvus.proto.data.VchannelInfo.partition_stats_versions:type_name -> milvus.proto.data.VchannelInfo.PartitionStatsVersionsEntry - 115, // 37: milvus.proto.data.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase - 29, // 38: milvus.proto.data.WatchDmChannelsRequest.vchannels:type_name -> milvus.proto.data.VchannelInfo - 115, // 39: milvus.proto.data.FlushSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase - 115, // 40: milvus.proto.data.SegmentMsg.base:type_name -> milvus.proto.common.MsgBase - 33, // 41: milvus.proto.data.SegmentMsg.segment:type_name -> milvus.proto.data.SegmentInfo - 117, // 42: milvus.proto.data.SegmentInfo.state:type_name -> milvus.proto.common.SegmentState - 118, // 43: milvus.proto.data.SegmentInfo.start_position:type_name -> milvus.proto.msg.MsgPosition - 118, // 44: milvus.proto.data.SegmentInfo.dml_position:type_name -> milvus.proto.msg.MsgPosition - 41, // 45: milvus.proto.data.SegmentInfo.binlogs:type_name -> milvus.proto.data.FieldBinlog - 41, // 46: milvus.proto.data.SegmentInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog - 41, // 47: milvus.proto.data.SegmentInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 34, // 5: milvus.proto.data.AllocSegmentResponse.segment_info:type_name -> milvus.proto.data.SegmentInfo + 117, // 6: milvus.proto.data.AllocSegmentResponse.status:type_name -> milvus.proto.common.Status + 12, // 7: milvus.proto.data.AssignSegmentIDRequest.segmentIDRequests:type_name -> milvus.proto.data.SegmentIDRequest + 117, // 8: milvus.proto.data.SegmentIDAssignment.status:type_name -> milvus.proto.common.Status + 16, // 9: milvus.proto.data.AssignSegmentIDResponse.segIDAssignments:type_name -> milvus.proto.data.SegmentIDAssignment + 117, // 10: milvus.proto.data.AssignSegmentIDResponse.status:type_name -> milvus.proto.common.Status + 116, // 11: milvus.proto.data.GetSegmentStatesRequest.base:type_name -> milvus.proto.common.MsgBase + 118, // 12: milvus.proto.data.SegmentStateInfo.state:type_name -> milvus.proto.common.SegmentState + 119, // 13: milvus.proto.data.SegmentStateInfo.start_position:type_name -> milvus.proto.msg.MsgPosition + 119, // 14: milvus.proto.data.SegmentStateInfo.end_position:type_name -> milvus.proto.msg.MsgPosition + 117, // 15: milvus.proto.data.SegmentStateInfo.status:type_name -> milvus.proto.common.Status + 117, // 16: milvus.proto.data.GetSegmentStatesResponse.status:type_name -> milvus.proto.common.Status + 19, // 17: milvus.proto.data.GetSegmentStatesResponse.states:type_name -> milvus.proto.data.SegmentStateInfo + 116, // 18: milvus.proto.data.GetSegmentInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 19: milvus.proto.data.GetSegmentInfoResponse.status:type_name -> milvus.proto.common.Status + 34, // 20: milvus.proto.data.GetSegmentInfoResponse.infos:type_name -> milvus.proto.data.SegmentInfo + 108, // 21: milvus.proto.data.GetSegmentInfoResponse.channel_checkpoint:type_name -> milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry + 116, // 22: milvus.proto.data.GetInsertBinlogPathsRequest.base:type_name -> milvus.proto.common.MsgBase + 120, // 23: milvus.proto.data.GetInsertBinlogPathsResponse.paths:type_name -> milvus.proto.internal.StringList + 117, // 24: milvus.proto.data.GetInsertBinlogPathsResponse.status:type_name -> milvus.proto.common.Status + 116, // 25: milvus.proto.data.GetCollectionStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase + 121, // 26: milvus.proto.data.GetCollectionStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair + 117, // 27: milvus.proto.data.GetCollectionStatisticsResponse.status:type_name -> milvus.proto.common.Status + 116, // 28: milvus.proto.data.GetPartitionStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase + 121, // 29: milvus.proto.data.GetPartitionStatisticsResponse.stats:type_name -> milvus.proto.common.KeyValuePair + 117, // 30: milvus.proto.data.GetPartitionStatisticsResponse.status:type_name -> milvus.proto.common.Status + 119, // 31: milvus.proto.data.VchannelInfo.seek_position:type_name -> milvus.proto.msg.MsgPosition + 34, // 32: milvus.proto.data.VchannelInfo.unflushedSegments:type_name -> milvus.proto.data.SegmentInfo + 34, // 33: milvus.proto.data.VchannelInfo.flushedSegments:type_name -> milvus.proto.data.SegmentInfo + 34, // 34: milvus.proto.data.VchannelInfo.dropped_segments:type_name -> milvus.proto.data.SegmentInfo + 34, // 35: milvus.proto.data.VchannelInfo.indexed_segments:type_name -> milvus.proto.data.SegmentInfo + 109, // 36: milvus.proto.data.VchannelInfo.partition_stats_versions:type_name -> milvus.proto.data.VchannelInfo.PartitionStatsVersionsEntry + 116, // 37: milvus.proto.data.WatchDmChannelsRequest.base:type_name -> milvus.proto.common.MsgBase + 30, // 38: milvus.proto.data.WatchDmChannelsRequest.vchannels:type_name -> milvus.proto.data.VchannelInfo + 116, // 39: milvus.proto.data.FlushSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 116, // 40: milvus.proto.data.SegmentMsg.base:type_name -> milvus.proto.common.MsgBase + 34, // 41: milvus.proto.data.SegmentMsg.segment:type_name -> milvus.proto.data.SegmentInfo + 118, // 42: milvus.proto.data.SegmentInfo.state:type_name -> milvus.proto.common.SegmentState + 119, // 43: milvus.proto.data.SegmentInfo.start_position:type_name -> milvus.proto.msg.MsgPosition + 119, // 44: milvus.proto.data.SegmentInfo.dml_position:type_name -> milvus.proto.msg.MsgPosition + 42, // 45: milvus.proto.data.SegmentInfo.binlogs:type_name -> milvus.proto.data.FieldBinlog + 42, // 46: milvus.proto.data.SegmentInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog + 42, // 47: milvus.proto.data.SegmentInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog 1, // 48: milvus.proto.data.SegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel 1, // 49: milvus.proto.data.SegmentInfo.last_level:type_name -> milvus.proto.data.SegmentLevel - 109, // 50: milvus.proto.data.SegmentInfo.textStatsLogs:type_name -> milvus.proto.data.SegmentInfo.TextStatsLogsEntry - 41, // 51: milvus.proto.data.SegmentInfo.bm25statslogs:type_name -> milvus.proto.data.FieldBinlog - 118, // 52: milvus.proto.data.SegmentStartPosition.start_position:type_name -> milvus.proto.msg.MsgPosition - 115, // 53: milvus.proto.data.SaveBinlogPathsRequest.base:type_name -> milvus.proto.common.MsgBase - 41, // 54: milvus.proto.data.SaveBinlogPathsRequest.field2BinlogPaths:type_name -> milvus.proto.data.FieldBinlog - 36, // 55: milvus.proto.data.SaveBinlogPathsRequest.checkPoints:type_name -> milvus.proto.data.CheckPoint - 34, // 56: milvus.proto.data.SaveBinlogPathsRequest.start_positions:type_name -> milvus.proto.data.SegmentStartPosition - 41, // 57: milvus.proto.data.SaveBinlogPathsRequest.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog - 41, // 58: milvus.proto.data.SaveBinlogPathsRequest.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 110, // 50: milvus.proto.data.SegmentInfo.textStatsLogs:type_name -> milvus.proto.data.SegmentInfo.TextStatsLogsEntry + 42, // 51: milvus.proto.data.SegmentInfo.bm25statslogs:type_name -> milvus.proto.data.FieldBinlog + 119, // 52: milvus.proto.data.SegmentStartPosition.start_position:type_name -> milvus.proto.msg.MsgPosition + 116, // 53: milvus.proto.data.SaveBinlogPathsRequest.base:type_name -> milvus.proto.common.MsgBase + 42, // 54: milvus.proto.data.SaveBinlogPathsRequest.field2BinlogPaths:type_name -> milvus.proto.data.FieldBinlog + 37, // 55: milvus.proto.data.SaveBinlogPathsRequest.checkPoints:type_name -> milvus.proto.data.CheckPoint + 35, // 56: milvus.proto.data.SaveBinlogPathsRequest.start_positions:type_name -> milvus.proto.data.SegmentStartPosition + 42, // 57: milvus.proto.data.SaveBinlogPathsRequest.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog + 42, // 58: milvus.proto.data.SaveBinlogPathsRequest.deltalogs:type_name -> milvus.proto.data.FieldBinlog 1, // 59: milvus.proto.data.SaveBinlogPathsRequest.seg_level:type_name -> milvus.proto.data.SegmentLevel - 41, // 60: milvus.proto.data.SaveBinlogPathsRequest.field2Bm25logPaths:type_name -> milvus.proto.data.FieldBinlog - 118, // 61: milvus.proto.data.CheckPoint.position:type_name -> milvus.proto.msg.MsgPosition + 42, // 60: milvus.proto.data.SaveBinlogPathsRequest.field2Bm25logPaths:type_name -> milvus.proto.data.FieldBinlog + 119, // 61: milvus.proto.data.CheckPoint.position:type_name -> milvus.proto.msg.MsgPosition 2, // 62: milvus.proto.data.ChannelStatus.state:type_name -> milvus.proto.data.ChannelWatchState - 38, // 63: milvus.proto.data.DataNodeInfo.channels:type_name -> milvus.proto.data.ChannelStatus - 41, // 64: milvus.proto.data.SegmentBinlogs.fieldBinlogs:type_name -> milvus.proto.data.FieldBinlog - 41, // 65: milvus.proto.data.SegmentBinlogs.statslogs:type_name -> milvus.proto.data.FieldBinlog - 41, // 66: milvus.proto.data.SegmentBinlogs.deltalogs:type_name -> milvus.proto.data.FieldBinlog - 110, // 67: milvus.proto.data.SegmentBinlogs.textStatsLogs:type_name -> milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry - 43, // 68: milvus.proto.data.FieldBinlog.binlogs:type_name -> milvus.proto.data.Binlog - 116, // 69: milvus.proto.data.GetRecoveryInfoResponse.status:type_name -> milvus.proto.common.Status - 29, // 70: milvus.proto.data.GetRecoveryInfoResponse.channels:type_name -> milvus.proto.data.VchannelInfo - 40, // 71: milvus.proto.data.GetRecoveryInfoResponse.binlogs:type_name -> milvus.proto.data.SegmentBinlogs - 115, // 72: milvus.proto.data.GetRecoveryInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 116, // 73: milvus.proto.data.GetRecoveryInfoResponseV2.status:type_name -> milvus.proto.common.Status - 29, // 74: milvus.proto.data.GetRecoveryInfoResponseV2.channels:type_name -> milvus.proto.data.VchannelInfo - 33, // 75: milvus.proto.data.GetRecoveryInfoResponseV2.segments:type_name -> milvus.proto.data.SegmentInfo - 115, // 76: milvus.proto.data.GetRecoveryInfoRequestV2.base:type_name -> milvus.proto.common.MsgBase - 115, // 77: milvus.proto.data.GetChannelRecoveryInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 116, // 78: milvus.proto.data.GetChannelRecoveryInfoResponse.status:type_name -> milvus.proto.common.Status - 29, // 79: milvus.proto.data.GetChannelRecoveryInfoResponse.info:type_name -> milvus.proto.data.VchannelInfo - 121, // 80: milvus.proto.data.GetChannelRecoveryInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema - 115, // 81: milvus.proto.data.GetSegmentsByStatesRequest.base:type_name -> milvus.proto.common.MsgBase - 117, // 82: milvus.proto.data.GetSegmentsByStatesRequest.states:type_name -> milvus.proto.common.SegmentState - 116, // 83: milvus.proto.data.GetSegmentsByStatesResponse.status:type_name -> milvus.proto.common.Status - 115, // 84: milvus.proto.data.GetFlushedSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase - 116, // 85: milvus.proto.data.GetFlushedSegmentsResponse.status:type_name -> milvus.proto.common.Status - 115, // 86: milvus.proto.data.SegmentFlushCompletedMsg.base:type_name -> milvus.proto.common.MsgBase - 33, // 87: milvus.proto.data.SegmentFlushCompletedMsg.segment:type_name -> milvus.proto.data.SegmentInfo - 29, // 88: milvus.proto.data.ChannelWatchInfo.vchan:type_name -> milvus.proto.data.VchannelInfo + 39, // 63: milvus.proto.data.DataNodeInfo.channels:type_name -> milvus.proto.data.ChannelStatus + 42, // 64: milvus.proto.data.SegmentBinlogs.fieldBinlogs:type_name -> milvus.proto.data.FieldBinlog + 42, // 65: milvus.proto.data.SegmentBinlogs.statslogs:type_name -> milvus.proto.data.FieldBinlog + 42, // 66: milvus.proto.data.SegmentBinlogs.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 111, // 67: milvus.proto.data.SegmentBinlogs.textStatsLogs:type_name -> milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry + 44, // 68: milvus.proto.data.FieldBinlog.binlogs:type_name -> milvus.proto.data.Binlog + 117, // 69: milvus.proto.data.GetRecoveryInfoResponse.status:type_name -> milvus.proto.common.Status + 30, // 70: milvus.proto.data.GetRecoveryInfoResponse.channels:type_name -> milvus.proto.data.VchannelInfo + 41, // 71: milvus.proto.data.GetRecoveryInfoResponse.binlogs:type_name -> milvus.proto.data.SegmentBinlogs + 116, // 72: milvus.proto.data.GetRecoveryInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 73: milvus.proto.data.GetRecoveryInfoResponseV2.status:type_name -> milvus.proto.common.Status + 30, // 74: milvus.proto.data.GetRecoveryInfoResponseV2.channels:type_name -> milvus.proto.data.VchannelInfo + 34, // 75: milvus.proto.data.GetRecoveryInfoResponseV2.segments:type_name -> milvus.proto.data.SegmentInfo + 116, // 76: milvus.proto.data.GetRecoveryInfoRequestV2.base:type_name -> milvus.proto.common.MsgBase + 116, // 77: milvus.proto.data.GetChannelRecoveryInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 78: milvus.proto.data.GetChannelRecoveryInfoResponse.status:type_name -> milvus.proto.common.Status + 30, // 79: milvus.proto.data.GetChannelRecoveryInfoResponse.info:type_name -> milvus.proto.data.VchannelInfo + 122, // 80: milvus.proto.data.GetChannelRecoveryInfoResponse.schema:type_name -> milvus.proto.schema.CollectionSchema + 116, // 81: milvus.proto.data.GetSegmentsByStatesRequest.base:type_name -> milvus.proto.common.MsgBase + 118, // 82: milvus.proto.data.GetSegmentsByStatesRequest.states:type_name -> milvus.proto.common.SegmentState + 117, // 83: milvus.proto.data.GetSegmentsByStatesResponse.status:type_name -> milvus.proto.common.Status + 116, // 84: milvus.proto.data.GetFlushedSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 85: milvus.proto.data.GetFlushedSegmentsResponse.status:type_name -> milvus.proto.common.Status + 116, // 86: milvus.proto.data.SegmentFlushCompletedMsg.base:type_name -> milvus.proto.common.MsgBase + 34, // 87: milvus.proto.data.SegmentFlushCompletedMsg.segment:type_name -> milvus.proto.data.SegmentInfo + 30, // 88: milvus.proto.data.ChannelWatchInfo.vchan:type_name -> milvus.proto.data.VchannelInfo 2, // 89: milvus.proto.data.ChannelWatchInfo.state:type_name -> milvus.proto.data.ChannelWatchState - 121, // 90: milvus.proto.data.ChannelWatchInfo.schema:type_name -> milvus.proto.schema.CollectionSchema - 120, // 91: milvus.proto.data.ChannelWatchInfo.dbProperties:type_name -> milvus.proto.common.KeyValuePair - 115, // 92: milvus.proto.data.CompactionStateRequest.base:type_name -> milvus.proto.common.MsgBase - 41, // 93: milvus.proto.data.SyncSegmentInfo.pk_stats_log:type_name -> milvus.proto.data.FieldBinlog - 117, // 94: milvus.proto.data.SyncSegmentInfo.state:type_name -> milvus.proto.common.SegmentState + 122, // 90: milvus.proto.data.ChannelWatchInfo.schema:type_name -> milvus.proto.schema.CollectionSchema + 121, // 91: milvus.proto.data.ChannelWatchInfo.dbProperties:type_name -> milvus.proto.common.KeyValuePair + 116, // 92: milvus.proto.data.CompactionStateRequest.base:type_name -> milvus.proto.common.MsgBase + 42, // 93: milvus.proto.data.SyncSegmentInfo.pk_stats_log:type_name -> milvus.proto.data.FieldBinlog + 118, // 94: milvus.proto.data.SyncSegmentInfo.state:type_name -> milvus.proto.common.SegmentState 1, // 95: milvus.proto.data.SyncSegmentInfo.level:type_name -> milvus.proto.data.SegmentLevel - 41, // 96: milvus.proto.data.SyncSegmentsRequest.stats_logs:type_name -> milvus.proto.data.FieldBinlog - 111, // 97: milvus.proto.data.SyncSegmentsRequest.segment_infos:type_name -> milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry - 41, // 98: milvus.proto.data.CompactionSegmentBinlogs.fieldBinlogs:type_name -> milvus.proto.data.FieldBinlog - 41, // 99: milvus.proto.data.CompactionSegmentBinlogs.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog - 41, // 100: milvus.proto.data.CompactionSegmentBinlogs.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 42, // 96: milvus.proto.data.SyncSegmentsRequest.stats_logs:type_name -> milvus.proto.data.FieldBinlog + 112, // 97: milvus.proto.data.SyncSegmentsRequest.segment_infos:type_name -> milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry + 42, // 98: milvus.proto.data.CompactionSegmentBinlogs.fieldBinlogs:type_name -> milvus.proto.data.FieldBinlog + 42, // 99: milvus.proto.data.CompactionSegmentBinlogs.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog + 42, // 100: milvus.proto.data.CompactionSegmentBinlogs.deltalogs:type_name -> milvus.proto.data.FieldBinlog 1, // 101: milvus.proto.data.CompactionSegmentBinlogs.level:type_name -> milvus.proto.data.SegmentLevel - 59, // 102: milvus.proto.data.CompactionPlan.segmentBinlogs:type_name -> milvus.proto.data.CompactionSegmentBinlogs + 60, // 102: milvus.proto.data.CompactionPlan.segmentBinlogs:type_name -> milvus.proto.data.CompactionSegmentBinlogs 3, // 103: milvus.proto.data.CompactionPlan.type:type_name -> milvus.proto.data.CompactionType - 121, // 104: milvus.proto.data.CompactionPlan.schema:type_name -> milvus.proto.schema.CollectionSchema - 86, // 105: milvus.proto.data.CompactionPlan.pre_allocated_segmentIDs:type_name -> milvus.proto.data.IDRange - 41, // 106: milvus.proto.data.CompactionSegment.insert_logs:type_name -> milvus.proto.data.FieldBinlog - 41, // 107: milvus.proto.data.CompactionSegment.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog - 41, // 108: milvus.proto.data.CompactionSegment.deltalogs:type_name -> milvus.proto.data.FieldBinlog - 41, // 109: milvus.proto.data.CompactionSegment.bm25logs:type_name -> milvus.proto.data.FieldBinlog - 6, // 110: milvus.proto.data.CompactionPlanResult.state:type_name -> milvus.proto.data.CompactionTaskState - 61, // 111: milvus.proto.data.CompactionPlanResult.segments:type_name -> milvus.proto.data.CompactionSegment + 122, // 104: milvus.proto.data.CompactionPlan.schema:type_name -> milvus.proto.schema.CollectionSchema + 87, // 105: milvus.proto.data.CompactionPlan.pre_allocated_segmentIDs:type_name -> milvus.proto.data.IDRange + 42, // 106: milvus.proto.data.CompactionSegment.insert_logs:type_name -> milvus.proto.data.FieldBinlog + 42, // 107: milvus.proto.data.CompactionSegment.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog + 42, // 108: milvus.proto.data.CompactionSegment.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 42, // 109: milvus.proto.data.CompactionSegment.bm25logs:type_name -> milvus.proto.data.FieldBinlog + 7, // 110: milvus.proto.data.CompactionPlanResult.state:type_name -> milvus.proto.data.CompactionTaskState + 62, // 111: milvus.proto.data.CompactionPlanResult.segments:type_name -> milvus.proto.data.CompactionSegment 3, // 112: milvus.proto.data.CompactionPlanResult.type:type_name -> milvus.proto.data.CompactionType - 116, // 113: milvus.proto.data.CompactionStateResponse.status:type_name -> milvus.proto.common.Status - 62, // 114: milvus.proto.data.CompactionStateResponse.results:type_name -> milvus.proto.data.CompactionPlanResult - 122, // 115: milvus.proto.data.WatchChannelsRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair - 121, // 116: milvus.proto.data.WatchChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 120, // 117: milvus.proto.data.WatchChannelsRequest.db_properties:type_name -> milvus.proto.common.KeyValuePair - 116, // 118: milvus.proto.data.WatchChannelsResponse.status:type_name -> milvus.proto.common.Status - 115, // 119: milvus.proto.data.SetSegmentStateRequest.base:type_name -> milvus.proto.common.MsgBase - 117, // 120: milvus.proto.data.SetSegmentStateRequest.new_state:type_name -> milvus.proto.common.SegmentState - 116, // 121: milvus.proto.data.SetSegmentStateResponse.status:type_name -> milvus.proto.common.Status - 115, // 122: milvus.proto.data.DropVirtualChannelRequest.base:type_name -> milvus.proto.common.MsgBase - 70, // 123: milvus.proto.data.DropVirtualChannelRequest.segments:type_name -> milvus.proto.data.DropVirtualChannelSegment - 41, // 124: milvus.proto.data.DropVirtualChannelSegment.field2BinlogPaths:type_name -> milvus.proto.data.FieldBinlog - 41, // 125: milvus.proto.data.DropVirtualChannelSegment.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog - 41, // 126: milvus.proto.data.DropVirtualChannelSegment.deltalogs:type_name -> milvus.proto.data.FieldBinlog - 118, // 127: milvus.proto.data.DropVirtualChannelSegment.startPosition:type_name -> milvus.proto.msg.MsgPosition - 118, // 128: milvus.proto.data.DropVirtualChannelSegment.checkPoint:type_name -> milvus.proto.msg.MsgPosition - 116, // 129: milvus.proto.data.DropVirtualChannelResponse.status:type_name -> milvus.proto.common.Status - 115, // 130: milvus.proto.data.UpdateSegmentStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase - 123, // 131: milvus.proto.data.UpdateSegmentStatisticsRequest.stats:type_name -> milvus.proto.common.SegmentStats - 115, // 132: milvus.proto.data.UpdateChannelCheckpointRequest.base:type_name -> milvus.proto.common.MsgBase - 118, // 133: milvus.proto.data.UpdateChannelCheckpointRequest.position:type_name -> milvus.proto.msg.MsgPosition - 118, // 134: milvus.proto.data.UpdateChannelCheckpointRequest.channel_checkpoints:type_name -> milvus.proto.msg.MsgPosition - 115, // 135: milvus.proto.data.ResendSegmentStatsRequest.base:type_name -> milvus.proto.common.MsgBase - 116, // 136: milvus.proto.data.ResendSegmentStatsResponse.status:type_name -> milvus.proto.common.Status - 115, // 137: milvus.proto.data.MarkSegmentsDroppedRequest.base:type_name -> milvus.proto.common.MsgBase - 121, // 138: milvus.proto.data.AlterCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 122, // 139: milvus.proto.data.AlterCollectionRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair - 120, // 140: milvus.proto.data.AlterCollectionRequest.properties:type_name -> milvus.proto.common.KeyValuePair - 116, // 141: milvus.proto.data.GcConfirmResponse.status:type_name -> milvus.proto.common.Status - 115, // 142: milvus.proto.data.ReportDataNodeTtMsgsRequest.base:type_name -> milvus.proto.common.MsgBase - 124, // 143: milvus.proto.data.ReportDataNodeTtMsgsRequest.msgs:type_name -> milvus.proto.msg.DataNodeTtMsg - 55, // 144: milvus.proto.data.ChannelOperationsRequest.infos:type_name -> milvus.proto.data.ChannelWatchInfo - 116, // 145: milvus.proto.data.ChannelOperationProgressResponse.status:type_name -> milvus.proto.common.Status + 117, // 113: milvus.proto.data.CompactionStateResponse.status:type_name -> milvus.proto.common.Status + 63, // 114: milvus.proto.data.CompactionStateResponse.results:type_name -> milvus.proto.data.CompactionPlanResult + 123, // 115: milvus.proto.data.WatchChannelsRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair + 122, // 116: milvus.proto.data.WatchChannelsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 121, // 117: milvus.proto.data.WatchChannelsRequest.db_properties:type_name -> milvus.proto.common.KeyValuePair + 117, // 118: milvus.proto.data.WatchChannelsResponse.status:type_name -> milvus.proto.common.Status + 116, // 119: milvus.proto.data.SetSegmentStateRequest.base:type_name -> milvus.proto.common.MsgBase + 118, // 120: milvus.proto.data.SetSegmentStateRequest.new_state:type_name -> milvus.proto.common.SegmentState + 117, // 121: milvus.proto.data.SetSegmentStateResponse.status:type_name -> milvus.proto.common.Status + 116, // 122: milvus.proto.data.DropVirtualChannelRequest.base:type_name -> milvus.proto.common.MsgBase + 71, // 123: milvus.proto.data.DropVirtualChannelRequest.segments:type_name -> milvus.proto.data.DropVirtualChannelSegment + 42, // 124: milvus.proto.data.DropVirtualChannelSegment.field2BinlogPaths:type_name -> milvus.proto.data.FieldBinlog + 42, // 125: milvus.proto.data.DropVirtualChannelSegment.field2StatslogPaths:type_name -> milvus.proto.data.FieldBinlog + 42, // 126: milvus.proto.data.DropVirtualChannelSegment.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 119, // 127: milvus.proto.data.DropVirtualChannelSegment.startPosition:type_name -> milvus.proto.msg.MsgPosition + 119, // 128: milvus.proto.data.DropVirtualChannelSegment.checkPoint:type_name -> milvus.proto.msg.MsgPosition + 117, // 129: milvus.proto.data.DropVirtualChannelResponse.status:type_name -> milvus.proto.common.Status + 116, // 130: milvus.proto.data.UpdateSegmentStatisticsRequest.base:type_name -> milvus.proto.common.MsgBase + 124, // 131: milvus.proto.data.UpdateSegmentStatisticsRequest.stats:type_name -> milvus.proto.common.SegmentStats + 116, // 132: milvus.proto.data.UpdateChannelCheckpointRequest.base:type_name -> milvus.proto.common.MsgBase + 119, // 133: milvus.proto.data.UpdateChannelCheckpointRequest.position:type_name -> milvus.proto.msg.MsgPosition + 119, // 134: milvus.proto.data.UpdateChannelCheckpointRequest.channel_checkpoints:type_name -> milvus.proto.msg.MsgPosition + 116, // 135: milvus.proto.data.ResendSegmentStatsRequest.base:type_name -> milvus.proto.common.MsgBase + 117, // 136: milvus.proto.data.ResendSegmentStatsResponse.status:type_name -> milvus.proto.common.Status + 116, // 137: milvus.proto.data.MarkSegmentsDroppedRequest.base:type_name -> milvus.proto.common.MsgBase + 122, // 138: milvus.proto.data.AlterCollectionRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 123, // 139: milvus.proto.data.AlterCollectionRequest.start_positions:type_name -> milvus.proto.common.KeyDataPair + 121, // 140: milvus.proto.data.AlterCollectionRequest.properties:type_name -> milvus.proto.common.KeyValuePair + 117, // 141: milvus.proto.data.GcConfirmResponse.status:type_name -> milvus.proto.common.Status + 116, // 142: milvus.proto.data.ReportDataNodeTtMsgsRequest.base:type_name -> milvus.proto.common.MsgBase + 125, // 143: milvus.proto.data.ReportDataNodeTtMsgsRequest.msgs:type_name -> milvus.proto.msg.DataNodeTtMsg + 56, // 144: milvus.proto.data.ChannelOperationsRequest.infos:type_name -> milvus.proto.data.ChannelWatchInfo + 117, // 145: milvus.proto.data.ChannelOperationProgressResponse.status:type_name -> milvus.proto.common.Status 2, // 146: milvus.proto.data.ChannelOperationProgressResponse.state:type_name -> milvus.proto.data.ChannelWatchState - 121, // 147: milvus.proto.data.PreImportRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 125, // 148: milvus.proto.data.PreImportRequest.import_files:type_name -> milvus.proto.internal.ImportFile - 120, // 149: milvus.proto.data.PreImportRequest.options:type_name -> milvus.proto.common.KeyValuePair - 121, // 150: milvus.proto.data.ImportRequest.schema:type_name -> milvus.proto.schema.CollectionSchema - 125, // 151: milvus.proto.data.ImportRequest.files:type_name -> milvus.proto.internal.ImportFile - 120, // 152: milvus.proto.data.ImportRequest.options:type_name -> milvus.proto.common.KeyValuePair - 86, // 153: milvus.proto.data.ImportRequest.ID_range:type_name -> milvus.proto.data.IDRange - 87, // 154: milvus.proto.data.ImportRequest.request_segments:type_name -> milvus.proto.data.ImportRequestSegment - 112, // 155: milvus.proto.data.PartitionImportStats.partition_rows:type_name -> milvus.proto.data.PartitionImportStats.PartitionRowsEntry - 113, // 156: milvus.proto.data.PartitionImportStats.partition_data_size:type_name -> milvus.proto.data.PartitionImportStats.PartitionDataSizeEntry - 125, // 157: milvus.proto.data.ImportFileStats.import_file:type_name -> milvus.proto.internal.ImportFile - 114, // 158: milvus.proto.data.ImportFileStats.hashed_stats:type_name -> milvus.proto.data.ImportFileStats.HashedStatsEntry - 116, // 159: milvus.proto.data.QueryPreImportResponse.status:type_name -> milvus.proto.common.Status + 122, // 147: milvus.proto.data.PreImportRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 126, // 148: milvus.proto.data.PreImportRequest.import_files:type_name -> milvus.proto.internal.ImportFile + 121, // 149: milvus.proto.data.PreImportRequest.options:type_name -> milvus.proto.common.KeyValuePair + 122, // 150: milvus.proto.data.ImportRequest.schema:type_name -> milvus.proto.schema.CollectionSchema + 126, // 151: milvus.proto.data.ImportRequest.files:type_name -> milvus.proto.internal.ImportFile + 121, // 152: milvus.proto.data.ImportRequest.options:type_name -> milvus.proto.common.KeyValuePair + 87, // 153: milvus.proto.data.ImportRequest.ID_range:type_name -> milvus.proto.data.IDRange + 88, // 154: milvus.proto.data.ImportRequest.request_segments:type_name -> milvus.proto.data.ImportRequestSegment + 113, // 155: milvus.proto.data.PartitionImportStats.partition_rows:type_name -> milvus.proto.data.PartitionImportStats.PartitionRowsEntry + 114, // 156: milvus.proto.data.PartitionImportStats.partition_data_size:type_name -> milvus.proto.data.PartitionImportStats.PartitionDataSizeEntry + 126, // 157: milvus.proto.data.ImportFileStats.import_file:type_name -> milvus.proto.internal.ImportFile + 115, // 158: milvus.proto.data.ImportFileStats.hashed_stats:type_name -> milvus.proto.data.ImportFileStats.HashedStatsEntry + 117, // 159: milvus.proto.data.QueryPreImportResponse.status:type_name -> milvus.proto.common.Status 4, // 160: milvus.proto.data.QueryPreImportResponse.state:type_name -> milvus.proto.data.ImportTaskStateV2 - 91, // 161: milvus.proto.data.QueryPreImportResponse.file_stats:type_name -> milvus.proto.data.ImportFileStats - 41, // 162: milvus.proto.data.ImportSegmentInfo.binlogs:type_name -> milvus.proto.data.FieldBinlog - 41, // 163: milvus.proto.data.ImportSegmentInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog - 41, // 164: milvus.proto.data.ImportSegmentInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog - 41, // 165: milvus.proto.data.ImportSegmentInfo.bm25logs:type_name -> milvus.proto.data.FieldBinlog - 116, // 166: milvus.proto.data.QueryImportResponse.status:type_name -> milvus.proto.common.Status + 92, // 161: milvus.proto.data.QueryPreImportResponse.file_stats:type_name -> milvus.proto.data.ImportFileStats + 42, // 162: milvus.proto.data.ImportSegmentInfo.binlogs:type_name -> milvus.proto.data.FieldBinlog + 42, // 163: milvus.proto.data.ImportSegmentInfo.statslogs:type_name -> milvus.proto.data.FieldBinlog + 42, // 164: milvus.proto.data.ImportSegmentInfo.deltalogs:type_name -> milvus.proto.data.FieldBinlog + 42, // 165: milvus.proto.data.ImportSegmentInfo.bm25logs:type_name -> milvus.proto.data.FieldBinlog + 117, // 166: milvus.proto.data.QueryImportResponse.status:type_name -> milvus.proto.common.Status 4, // 167: milvus.proto.data.QueryImportResponse.state:type_name -> milvus.proto.data.ImportTaskStateV2 - 94, // 168: milvus.proto.data.QueryImportResponse.import_segments_info:type_name -> milvus.proto.data.ImportSegmentInfo - 121, // 169: milvus.proto.data.ImportJob.schema:type_name -> milvus.proto.schema.CollectionSchema - 126, // 170: milvus.proto.data.ImportJob.state:type_name -> milvus.proto.internal.ImportJobState - 125, // 171: milvus.proto.data.ImportJob.files:type_name -> milvus.proto.internal.ImportFile - 120, // 172: milvus.proto.data.ImportJob.options:type_name -> milvus.proto.common.KeyValuePair + 95, // 168: milvus.proto.data.QueryImportResponse.import_segments_info:type_name -> milvus.proto.data.ImportSegmentInfo + 122, // 169: milvus.proto.data.ImportJob.schema:type_name -> milvus.proto.schema.CollectionSchema + 127, // 170: milvus.proto.data.ImportJob.state:type_name -> milvus.proto.internal.ImportJobState + 126, // 171: milvus.proto.data.ImportJob.files:type_name -> milvus.proto.internal.ImportFile + 121, // 172: milvus.proto.data.ImportJob.options:type_name -> milvus.proto.common.KeyValuePair 4, // 173: milvus.proto.data.PreImportTask.state:type_name -> milvus.proto.data.ImportTaskStateV2 - 91, // 174: milvus.proto.data.PreImportTask.file_stats:type_name -> milvus.proto.data.ImportFileStats + 92, // 174: milvus.proto.data.PreImportTask.file_stats:type_name -> milvus.proto.data.ImportFileStats 4, // 175: milvus.proto.data.ImportTaskV2.state:type_name -> milvus.proto.data.ImportTaskStateV2 - 91, // 176: milvus.proto.data.ImportTaskV2.file_stats:type_name -> milvus.proto.data.ImportFileStats - 115, // 177: milvus.proto.data.GcControlRequest.base:type_name -> milvus.proto.common.MsgBase - 5, // 178: milvus.proto.data.GcControlRequest.command:type_name -> milvus.proto.data.GcCommand - 120, // 179: milvus.proto.data.GcControlRequest.params:type_name -> milvus.proto.common.KeyValuePair - 116, // 180: milvus.proto.data.QuerySlotResponse.status:type_name -> milvus.proto.common.Status - 3, // 181: milvus.proto.data.CompactionTask.type:type_name -> milvus.proto.data.CompactionType - 6, // 182: milvus.proto.data.CompactionTask.state:type_name -> milvus.proto.data.CompactionTaskState - 118, // 183: milvus.proto.data.CompactionTask.pos:type_name -> milvus.proto.msg.MsgPosition - 121, // 184: milvus.proto.data.CompactionTask.schema:type_name -> milvus.proto.schema.CollectionSchema - 127, // 185: milvus.proto.data.CompactionTask.clustering_key_field:type_name -> milvus.proto.schema.FieldSchema - 86, // 186: milvus.proto.data.CompactionTask.pre_allocated_segmentIDs:type_name -> milvus.proto.data.IDRange - 118, // 187: milvus.proto.data.FlushResponse.ChannelCpsEntry.value:type_name -> milvus.proto.msg.MsgPosition - 118, // 188: milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry.value:type_name -> milvus.proto.msg.MsgPosition - 42, // 189: milvus.proto.data.SegmentInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats - 42, // 190: milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats - 57, // 191: milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SyncSegmentInfo - 90, // 192: milvus.proto.data.ImportFileStats.HashedStatsEntry.value:type_name -> milvus.proto.data.PartitionImportStats - 128, // 193: milvus.proto.data.DataCoord.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest - 129, // 194: milvus.proto.data.DataCoord.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest - 130, // 195: milvus.proto.data.DataCoord.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest - 8, // 196: milvus.proto.data.DataCoord.Flush:input_type -> milvus.proto.data.FlushRequest - 12, // 197: milvus.proto.data.DataCoord.AllocSegment:input_type -> milvus.proto.data.AllocSegmentRequest - 14, // 198: milvus.proto.data.DataCoord.AssignSegmentID:input_type -> milvus.proto.data.AssignSegmentIDRequest - 20, // 199: milvus.proto.data.DataCoord.GetSegmentInfo:input_type -> milvus.proto.data.GetSegmentInfoRequest - 17, // 200: milvus.proto.data.DataCoord.GetSegmentStates:input_type -> milvus.proto.data.GetSegmentStatesRequest - 22, // 201: milvus.proto.data.DataCoord.GetInsertBinlogPaths:input_type -> milvus.proto.data.GetInsertBinlogPathsRequest - 24, // 202: milvus.proto.data.DataCoord.GetCollectionStatistics:input_type -> milvus.proto.data.GetCollectionStatisticsRequest - 26, // 203: milvus.proto.data.DataCoord.GetPartitionStatistics:input_type -> milvus.proto.data.GetPartitionStatisticsRequest - 28, // 204: milvus.proto.data.DataCoord.GetSegmentInfoChannel:input_type -> milvus.proto.data.GetSegmentInfoChannelRequest - 35, // 205: milvus.proto.data.DataCoord.SaveBinlogPaths:input_type -> milvus.proto.data.SaveBinlogPathsRequest - 45, // 206: milvus.proto.data.DataCoord.GetRecoveryInfo:input_type -> milvus.proto.data.GetRecoveryInfoRequest - 47, // 207: milvus.proto.data.DataCoord.GetRecoveryInfoV2:input_type -> milvus.proto.data.GetRecoveryInfoRequestV2 - 48, // 208: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:input_type -> milvus.proto.data.GetChannelRecoveryInfoRequest - 52, // 209: milvus.proto.data.DataCoord.GetFlushedSegments:input_type -> milvus.proto.data.GetFlushedSegmentsRequest - 50, // 210: milvus.proto.data.DataCoord.GetSegmentsByStates:input_type -> milvus.proto.data.GetSegmentsByStatesRequest - 131, // 211: milvus.proto.data.DataCoord.GetFlushAllState:input_type -> milvus.proto.milvus.GetFlushAllStateRequest - 132, // 212: milvus.proto.data.DataCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest - 133, // 213: milvus.proto.data.DataCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest - 134, // 214: milvus.proto.data.DataCoord.ManualCompaction:input_type -> milvus.proto.milvus.ManualCompactionRequest - 135, // 215: milvus.proto.data.DataCoord.GetCompactionState:input_type -> milvus.proto.milvus.GetCompactionStateRequest - 136, // 216: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:input_type -> milvus.proto.milvus.GetCompactionPlansRequest - 65, // 217: milvus.proto.data.DataCoord.WatchChannels:input_type -> milvus.proto.data.WatchChannelsRequest - 82, // 218: milvus.proto.data.DataCoord.GetFlushState:input_type -> milvus.proto.data.GetFlushStateRequest - 69, // 219: milvus.proto.data.DataCoord.DropVirtualChannel:input_type -> milvus.proto.data.DropVirtualChannelRequest - 67, // 220: milvus.proto.data.DataCoord.SetSegmentState:input_type -> milvus.proto.data.SetSegmentStateRequest - 72, // 221: milvus.proto.data.DataCoord.UpdateSegmentStatistics:input_type -> milvus.proto.data.UpdateSegmentStatisticsRequest - 73, // 222: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:input_type -> milvus.proto.data.UpdateChannelCheckpointRequest - 76, // 223: milvus.proto.data.DataCoord.MarkSegmentsDropped:input_type -> milvus.proto.data.MarkSegmentsDroppedRequest - 78, // 224: milvus.proto.data.DataCoord.BroadcastAlteredCollection:input_type -> milvus.proto.data.AlterCollectionRequest - 137, // 225: milvus.proto.data.DataCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest - 138, // 226: milvus.proto.data.DataCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest - 139, // 227: milvus.proto.data.DataCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest - 140, // 228: milvus.proto.data.DataCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest - 141, // 229: milvus.proto.data.DataCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest - 142, // 230: milvus.proto.data.DataCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest - 143, // 231: milvus.proto.data.DataCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest - 144, // 232: milvus.proto.data.DataCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest - 145, // 233: milvus.proto.data.DataCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest - 146, // 234: milvus.proto.data.DataCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest - 147, // 235: milvus.proto.data.DataCoord.ListIndexes:input_type -> milvus.proto.index.ListIndexesRequest - 79, // 236: milvus.proto.data.DataCoord.GcConfirm:input_type -> milvus.proto.data.GcConfirmRequest - 81, // 237: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:input_type -> milvus.proto.data.ReportDataNodeTtMsgsRequest - 100, // 238: milvus.proto.data.DataCoord.GcControl:input_type -> milvus.proto.data.GcControlRequest - 148, // 239: milvus.proto.data.DataCoord.ImportV2:input_type -> milvus.proto.internal.ImportRequestInternal - 149, // 240: milvus.proto.data.DataCoord.GetImportProgress:input_type -> milvus.proto.internal.GetImportProgressRequest - 150, // 241: milvus.proto.data.DataCoord.ListImports:input_type -> milvus.proto.internal.ListImportsRequestInternal - 128, // 242: milvus.proto.data.DataNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest - 130, // 243: milvus.proto.data.DataNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest - 30, // 244: milvus.proto.data.DataNode.WatchDmChannels:input_type -> milvus.proto.data.WatchDmChannelsRequest - 31, // 245: milvus.proto.data.DataNode.FlushSegments:input_type -> milvus.proto.data.FlushSegmentsRequest - 132, // 246: milvus.proto.data.DataNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest - 133, // 247: milvus.proto.data.DataNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest - 60, // 248: milvus.proto.data.DataNode.CompactionV2:input_type -> milvus.proto.data.CompactionPlan - 56, // 249: milvus.proto.data.DataNode.GetCompactionState:input_type -> milvus.proto.data.CompactionStateRequest - 58, // 250: milvus.proto.data.DataNode.SyncSegments:input_type -> milvus.proto.data.SyncSegmentsRequest - 74, // 251: milvus.proto.data.DataNode.ResendSegmentStats:input_type -> milvus.proto.data.ResendSegmentStatsRequest - 10, // 252: milvus.proto.data.DataNode.FlushChannels:input_type -> milvus.proto.data.FlushChannelsRequest - 83, // 253: milvus.proto.data.DataNode.NotifyChannelOperation:input_type -> milvus.proto.data.ChannelOperationsRequest - 55, // 254: milvus.proto.data.DataNode.CheckChannelOperationProgress:input_type -> milvus.proto.data.ChannelWatchInfo - 85, // 255: milvus.proto.data.DataNode.PreImport:input_type -> milvus.proto.data.PreImportRequest - 88, // 256: milvus.proto.data.DataNode.ImportV2:input_type -> milvus.proto.data.ImportRequest - 89, // 257: milvus.proto.data.DataNode.QueryPreImport:input_type -> milvus.proto.data.QueryPreImportRequest - 93, // 258: milvus.proto.data.DataNode.QueryImport:input_type -> milvus.proto.data.QueryImportRequest - 96, // 259: milvus.proto.data.DataNode.DropImport:input_type -> milvus.proto.data.DropImportRequest - 101, // 260: milvus.proto.data.DataNode.QuerySlot:input_type -> milvus.proto.data.QuerySlotRequest - 105, // 261: milvus.proto.data.DataNode.DropCompactionPlan:input_type -> milvus.proto.data.DropCompactionPlanRequest - 151, // 262: milvus.proto.data.DataCoord.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 152, // 263: milvus.proto.data.DataCoord.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse - 152, // 264: milvus.proto.data.DataCoord.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse - 9, // 265: milvus.proto.data.DataCoord.Flush:output_type -> milvus.proto.data.FlushResponse - 13, // 266: milvus.proto.data.DataCoord.AllocSegment:output_type -> milvus.proto.data.AllocSegmentResponse - 16, // 267: milvus.proto.data.DataCoord.AssignSegmentID:output_type -> milvus.proto.data.AssignSegmentIDResponse - 21, // 268: milvus.proto.data.DataCoord.GetSegmentInfo:output_type -> milvus.proto.data.GetSegmentInfoResponse - 19, // 269: milvus.proto.data.DataCoord.GetSegmentStates:output_type -> milvus.proto.data.GetSegmentStatesResponse - 23, // 270: milvus.proto.data.DataCoord.GetInsertBinlogPaths:output_type -> milvus.proto.data.GetInsertBinlogPathsResponse - 25, // 271: milvus.proto.data.DataCoord.GetCollectionStatistics:output_type -> milvus.proto.data.GetCollectionStatisticsResponse - 27, // 272: milvus.proto.data.DataCoord.GetPartitionStatistics:output_type -> milvus.proto.data.GetPartitionStatisticsResponse - 152, // 273: milvus.proto.data.DataCoord.GetSegmentInfoChannel:output_type -> milvus.proto.milvus.StringResponse - 116, // 274: milvus.proto.data.DataCoord.SaveBinlogPaths:output_type -> milvus.proto.common.Status - 44, // 275: milvus.proto.data.DataCoord.GetRecoveryInfo:output_type -> milvus.proto.data.GetRecoveryInfoResponse - 46, // 276: milvus.proto.data.DataCoord.GetRecoveryInfoV2:output_type -> milvus.proto.data.GetRecoveryInfoResponseV2 - 49, // 277: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:output_type -> milvus.proto.data.GetChannelRecoveryInfoResponse - 53, // 278: milvus.proto.data.DataCoord.GetFlushedSegments:output_type -> milvus.proto.data.GetFlushedSegmentsResponse - 51, // 279: milvus.proto.data.DataCoord.GetSegmentsByStates:output_type -> milvus.proto.data.GetSegmentsByStatesResponse - 153, // 280: milvus.proto.data.DataCoord.GetFlushAllState:output_type -> milvus.proto.milvus.GetFlushAllStateResponse - 154, // 281: milvus.proto.data.DataCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 155, // 282: milvus.proto.data.DataCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 156, // 283: milvus.proto.data.DataCoord.ManualCompaction:output_type -> milvus.proto.milvus.ManualCompactionResponse - 157, // 284: milvus.proto.data.DataCoord.GetCompactionState:output_type -> milvus.proto.milvus.GetCompactionStateResponse - 158, // 285: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:output_type -> milvus.proto.milvus.GetCompactionPlansResponse - 66, // 286: milvus.proto.data.DataCoord.WatchChannels:output_type -> milvus.proto.data.WatchChannelsResponse - 159, // 287: milvus.proto.data.DataCoord.GetFlushState:output_type -> milvus.proto.milvus.GetFlushStateResponse - 71, // 288: milvus.proto.data.DataCoord.DropVirtualChannel:output_type -> milvus.proto.data.DropVirtualChannelResponse - 68, // 289: milvus.proto.data.DataCoord.SetSegmentState:output_type -> milvus.proto.data.SetSegmentStateResponse - 116, // 290: milvus.proto.data.DataCoord.UpdateSegmentStatistics:output_type -> milvus.proto.common.Status - 116, // 291: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:output_type -> milvus.proto.common.Status - 116, // 292: milvus.proto.data.DataCoord.MarkSegmentsDropped:output_type -> milvus.proto.common.Status - 116, // 293: milvus.proto.data.DataCoord.BroadcastAlteredCollection:output_type -> milvus.proto.common.Status - 160, // 294: milvus.proto.data.DataCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse - 116, // 295: milvus.proto.data.DataCoord.CreateIndex:output_type -> milvus.proto.common.Status - 116, // 296: milvus.proto.data.DataCoord.AlterIndex:output_type -> milvus.proto.common.Status - 161, // 297: milvus.proto.data.DataCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse - 162, // 298: milvus.proto.data.DataCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse - 163, // 299: milvus.proto.data.DataCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse - 116, // 300: milvus.proto.data.DataCoord.DropIndex:output_type -> milvus.proto.common.Status - 164, // 301: milvus.proto.data.DataCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse - 165, // 302: milvus.proto.data.DataCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse - 166, // 303: milvus.proto.data.DataCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse - 167, // 304: milvus.proto.data.DataCoord.ListIndexes:output_type -> milvus.proto.index.ListIndexesResponse - 80, // 305: milvus.proto.data.DataCoord.GcConfirm:output_type -> milvus.proto.data.GcConfirmResponse - 116, // 306: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:output_type -> milvus.proto.common.Status - 116, // 307: milvus.proto.data.DataCoord.GcControl:output_type -> milvus.proto.common.Status - 168, // 308: milvus.proto.data.DataCoord.ImportV2:output_type -> milvus.proto.internal.ImportResponse - 169, // 309: milvus.proto.data.DataCoord.GetImportProgress:output_type -> milvus.proto.internal.GetImportProgressResponse - 170, // 310: milvus.proto.data.DataCoord.ListImports:output_type -> milvus.proto.internal.ListImportsResponse - 151, // 311: milvus.proto.data.DataNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 152, // 312: milvus.proto.data.DataNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse - 116, // 313: milvus.proto.data.DataNode.WatchDmChannels:output_type -> milvus.proto.common.Status - 116, // 314: milvus.proto.data.DataNode.FlushSegments:output_type -> milvus.proto.common.Status - 154, // 315: milvus.proto.data.DataNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 155, // 316: milvus.proto.data.DataNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 116, // 317: milvus.proto.data.DataNode.CompactionV2:output_type -> milvus.proto.common.Status - 63, // 318: milvus.proto.data.DataNode.GetCompactionState:output_type -> milvus.proto.data.CompactionStateResponse - 116, // 319: milvus.proto.data.DataNode.SyncSegments:output_type -> milvus.proto.common.Status - 75, // 320: milvus.proto.data.DataNode.ResendSegmentStats:output_type -> milvus.proto.data.ResendSegmentStatsResponse - 116, // 321: milvus.proto.data.DataNode.FlushChannels:output_type -> milvus.proto.common.Status - 116, // 322: milvus.proto.data.DataNode.NotifyChannelOperation:output_type -> milvus.proto.common.Status - 84, // 323: milvus.proto.data.DataNode.CheckChannelOperationProgress:output_type -> milvus.proto.data.ChannelOperationProgressResponse - 116, // 324: milvus.proto.data.DataNode.PreImport:output_type -> milvus.proto.common.Status - 116, // 325: milvus.proto.data.DataNode.ImportV2:output_type -> milvus.proto.common.Status - 92, // 326: milvus.proto.data.DataNode.QueryPreImport:output_type -> milvus.proto.data.QueryPreImportResponse - 95, // 327: milvus.proto.data.DataNode.QueryImport:output_type -> milvus.proto.data.QueryImportResponse - 116, // 328: milvus.proto.data.DataNode.DropImport:output_type -> milvus.proto.common.Status - 102, // 329: milvus.proto.data.DataNode.QuerySlot:output_type -> milvus.proto.data.QuerySlotResponse - 116, // 330: milvus.proto.data.DataNode.DropCompactionPlan:output_type -> milvus.proto.common.Status - 262, // [262:331] is the sub-list for method output_type - 193, // [193:262] is the sub-list for method input_type - 193, // [193:193] is the sub-list for extension type_name - 193, // [193:193] is the sub-list for extension extendee - 0, // [0:193] is the sub-list for field type_name + 92, // 176: milvus.proto.data.ImportTaskV2.file_stats:type_name -> milvus.proto.data.ImportFileStats + 5, // 177: milvus.proto.data.ImportTaskV2.source:type_name -> milvus.proto.data.ImportTaskSourceV2 + 116, // 178: milvus.proto.data.GcControlRequest.base:type_name -> milvus.proto.common.MsgBase + 6, // 179: milvus.proto.data.GcControlRequest.command:type_name -> milvus.proto.data.GcCommand + 121, // 180: milvus.proto.data.GcControlRequest.params:type_name -> milvus.proto.common.KeyValuePair + 117, // 181: milvus.proto.data.QuerySlotResponse.status:type_name -> milvus.proto.common.Status + 3, // 182: milvus.proto.data.CompactionTask.type:type_name -> milvus.proto.data.CompactionType + 7, // 183: milvus.proto.data.CompactionTask.state:type_name -> milvus.proto.data.CompactionTaskState + 119, // 184: milvus.proto.data.CompactionTask.pos:type_name -> milvus.proto.msg.MsgPosition + 122, // 185: milvus.proto.data.CompactionTask.schema:type_name -> milvus.proto.schema.CollectionSchema + 128, // 186: milvus.proto.data.CompactionTask.clustering_key_field:type_name -> milvus.proto.schema.FieldSchema + 87, // 187: milvus.proto.data.CompactionTask.pre_allocated_segmentIDs:type_name -> milvus.proto.data.IDRange + 119, // 188: milvus.proto.data.FlushResponse.ChannelCpsEntry.value:type_name -> milvus.proto.msg.MsgPosition + 119, // 189: milvus.proto.data.GetSegmentInfoResponse.ChannelCheckpointEntry.value:type_name -> milvus.proto.msg.MsgPosition + 43, // 190: milvus.proto.data.SegmentInfo.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 43, // 191: milvus.proto.data.SegmentBinlogs.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats + 58, // 192: milvus.proto.data.SyncSegmentsRequest.SegmentInfosEntry.value:type_name -> milvus.proto.data.SyncSegmentInfo + 91, // 193: milvus.proto.data.ImportFileStats.HashedStatsEntry.value:type_name -> milvus.proto.data.PartitionImportStats + 129, // 194: milvus.proto.data.DataCoord.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 130, // 195: milvus.proto.data.DataCoord.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest + 131, // 196: milvus.proto.data.DataCoord.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 9, // 197: milvus.proto.data.DataCoord.Flush:input_type -> milvus.proto.data.FlushRequest + 13, // 198: milvus.proto.data.DataCoord.AllocSegment:input_type -> milvus.proto.data.AllocSegmentRequest + 15, // 199: milvus.proto.data.DataCoord.AssignSegmentID:input_type -> milvus.proto.data.AssignSegmentIDRequest + 21, // 200: milvus.proto.data.DataCoord.GetSegmentInfo:input_type -> milvus.proto.data.GetSegmentInfoRequest + 18, // 201: milvus.proto.data.DataCoord.GetSegmentStates:input_type -> milvus.proto.data.GetSegmentStatesRequest + 23, // 202: milvus.proto.data.DataCoord.GetInsertBinlogPaths:input_type -> milvus.proto.data.GetInsertBinlogPathsRequest + 25, // 203: milvus.proto.data.DataCoord.GetCollectionStatistics:input_type -> milvus.proto.data.GetCollectionStatisticsRequest + 27, // 204: milvus.proto.data.DataCoord.GetPartitionStatistics:input_type -> milvus.proto.data.GetPartitionStatisticsRequest + 29, // 205: milvus.proto.data.DataCoord.GetSegmentInfoChannel:input_type -> milvus.proto.data.GetSegmentInfoChannelRequest + 36, // 206: milvus.proto.data.DataCoord.SaveBinlogPaths:input_type -> milvus.proto.data.SaveBinlogPathsRequest + 46, // 207: milvus.proto.data.DataCoord.GetRecoveryInfo:input_type -> milvus.proto.data.GetRecoveryInfoRequest + 48, // 208: milvus.proto.data.DataCoord.GetRecoveryInfoV2:input_type -> milvus.proto.data.GetRecoveryInfoRequestV2 + 49, // 209: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:input_type -> milvus.proto.data.GetChannelRecoveryInfoRequest + 53, // 210: milvus.proto.data.DataCoord.GetFlushedSegments:input_type -> milvus.proto.data.GetFlushedSegmentsRequest + 51, // 211: milvus.proto.data.DataCoord.GetSegmentsByStates:input_type -> milvus.proto.data.GetSegmentsByStatesRequest + 132, // 212: milvus.proto.data.DataCoord.GetFlushAllState:input_type -> milvus.proto.milvus.GetFlushAllStateRequest + 133, // 213: milvus.proto.data.DataCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 134, // 214: milvus.proto.data.DataCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 135, // 215: milvus.proto.data.DataCoord.ManualCompaction:input_type -> milvus.proto.milvus.ManualCompactionRequest + 136, // 216: milvus.proto.data.DataCoord.GetCompactionState:input_type -> milvus.proto.milvus.GetCompactionStateRequest + 137, // 217: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:input_type -> milvus.proto.milvus.GetCompactionPlansRequest + 66, // 218: milvus.proto.data.DataCoord.WatchChannels:input_type -> milvus.proto.data.WatchChannelsRequest + 83, // 219: milvus.proto.data.DataCoord.GetFlushState:input_type -> milvus.proto.data.GetFlushStateRequest + 70, // 220: milvus.proto.data.DataCoord.DropVirtualChannel:input_type -> milvus.proto.data.DropVirtualChannelRequest + 68, // 221: milvus.proto.data.DataCoord.SetSegmentState:input_type -> milvus.proto.data.SetSegmentStateRequest + 73, // 222: milvus.proto.data.DataCoord.UpdateSegmentStatistics:input_type -> milvus.proto.data.UpdateSegmentStatisticsRequest + 74, // 223: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:input_type -> milvus.proto.data.UpdateChannelCheckpointRequest + 77, // 224: milvus.proto.data.DataCoord.MarkSegmentsDropped:input_type -> milvus.proto.data.MarkSegmentsDroppedRequest + 79, // 225: milvus.proto.data.DataCoord.BroadcastAlteredCollection:input_type -> milvus.proto.data.AlterCollectionRequest + 138, // 226: milvus.proto.data.DataCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest + 139, // 227: milvus.proto.data.DataCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest + 140, // 228: milvus.proto.data.DataCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest + 141, // 229: milvus.proto.data.DataCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest + 142, // 230: milvus.proto.data.DataCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest + 143, // 231: milvus.proto.data.DataCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest + 144, // 232: milvus.proto.data.DataCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest + 145, // 233: milvus.proto.data.DataCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest + 146, // 234: milvus.proto.data.DataCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest + 147, // 235: milvus.proto.data.DataCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest + 148, // 236: milvus.proto.data.DataCoord.ListIndexes:input_type -> milvus.proto.index.ListIndexesRequest + 80, // 237: milvus.proto.data.DataCoord.GcConfirm:input_type -> milvus.proto.data.GcConfirmRequest + 82, // 238: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:input_type -> milvus.proto.data.ReportDataNodeTtMsgsRequest + 101, // 239: milvus.proto.data.DataCoord.GcControl:input_type -> milvus.proto.data.GcControlRequest + 149, // 240: milvus.proto.data.DataCoord.ImportV2:input_type -> milvus.proto.internal.ImportRequestInternal + 150, // 241: milvus.proto.data.DataCoord.GetImportProgress:input_type -> milvus.proto.internal.GetImportProgressRequest + 151, // 242: milvus.proto.data.DataCoord.ListImports:input_type -> milvus.proto.internal.ListImportsRequestInternal + 129, // 243: milvus.proto.data.DataNode.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 131, // 244: milvus.proto.data.DataNode.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 31, // 245: milvus.proto.data.DataNode.WatchDmChannels:input_type -> milvus.proto.data.WatchDmChannelsRequest + 32, // 246: milvus.proto.data.DataNode.FlushSegments:input_type -> milvus.proto.data.FlushSegmentsRequest + 133, // 247: milvus.proto.data.DataNode.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 134, // 248: milvus.proto.data.DataNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 61, // 249: milvus.proto.data.DataNode.CompactionV2:input_type -> milvus.proto.data.CompactionPlan + 57, // 250: milvus.proto.data.DataNode.GetCompactionState:input_type -> milvus.proto.data.CompactionStateRequest + 59, // 251: milvus.proto.data.DataNode.SyncSegments:input_type -> milvus.proto.data.SyncSegmentsRequest + 75, // 252: milvus.proto.data.DataNode.ResendSegmentStats:input_type -> milvus.proto.data.ResendSegmentStatsRequest + 11, // 253: milvus.proto.data.DataNode.FlushChannels:input_type -> milvus.proto.data.FlushChannelsRequest + 84, // 254: milvus.proto.data.DataNode.NotifyChannelOperation:input_type -> milvus.proto.data.ChannelOperationsRequest + 56, // 255: milvus.proto.data.DataNode.CheckChannelOperationProgress:input_type -> milvus.proto.data.ChannelWatchInfo + 86, // 256: milvus.proto.data.DataNode.PreImport:input_type -> milvus.proto.data.PreImportRequest + 89, // 257: milvus.proto.data.DataNode.ImportV2:input_type -> milvus.proto.data.ImportRequest + 90, // 258: milvus.proto.data.DataNode.QueryPreImport:input_type -> milvus.proto.data.QueryPreImportRequest + 94, // 259: milvus.proto.data.DataNode.QueryImport:input_type -> milvus.proto.data.QueryImportRequest + 97, // 260: milvus.proto.data.DataNode.DropImport:input_type -> milvus.proto.data.DropImportRequest + 102, // 261: milvus.proto.data.DataNode.QuerySlot:input_type -> milvus.proto.data.QuerySlotRequest + 106, // 262: milvus.proto.data.DataNode.DropCompactionPlan:input_type -> milvus.proto.data.DropCompactionPlanRequest + 152, // 263: milvus.proto.data.DataCoord.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 153, // 264: milvus.proto.data.DataCoord.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse + 153, // 265: milvus.proto.data.DataCoord.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 10, // 266: milvus.proto.data.DataCoord.Flush:output_type -> milvus.proto.data.FlushResponse + 14, // 267: milvus.proto.data.DataCoord.AllocSegment:output_type -> milvus.proto.data.AllocSegmentResponse + 17, // 268: milvus.proto.data.DataCoord.AssignSegmentID:output_type -> milvus.proto.data.AssignSegmentIDResponse + 22, // 269: milvus.proto.data.DataCoord.GetSegmentInfo:output_type -> milvus.proto.data.GetSegmentInfoResponse + 20, // 270: milvus.proto.data.DataCoord.GetSegmentStates:output_type -> milvus.proto.data.GetSegmentStatesResponse + 24, // 271: milvus.proto.data.DataCoord.GetInsertBinlogPaths:output_type -> milvus.proto.data.GetInsertBinlogPathsResponse + 26, // 272: milvus.proto.data.DataCoord.GetCollectionStatistics:output_type -> milvus.proto.data.GetCollectionStatisticsResponse + 28, // 273: milvus.proto.data.DataCoord.GetPartitionStatistics:output_type -> milvus.proto.data.GetPartitionStatisticsResponse + 153, // 274: milvus.proto.data.DataCoord.GetSegmentInfoChannel:output_type -> milvus.proto.milvus.StringResponse + 117, // 275: milvus.proto.data.DataCoord.SaveBinlogPaths:output_type -> milvus.proto.common.Status + 45, // 276: milvus.proto.data.DataCoord.GetRecoveryInfo:output_type -> milvus.proto.data.GetRecoveryInfoResponse + 47, // 277: milvus.proto.data.DataCoord.GetRecoveryInfoV2:output_type -> milvus.proto.data.GetRecoveryInfoResponseV2 + 50, // 278: milvus.proto.data.DataCoord.GetChannelRecoveryInfo:output_type -> milvus.proto.data.GetChannelRecoveryInfoResponse + 54, // 279: milvus.proto.data.DataCoord.GetFlushedSegments:output_type -> milvus.proto.data.GetFlushedSegmentsResponse + 52, // 280: milvus.proto.data.DataCoord.GetSegmentsByStates:output_type -> milvus.proto.data.GetSegmentsByStatesResponse + 154, // 281: milvus.proto.data.DataCoord.GetFlushAllState:output_type -> milvus.proto.milvus.GetFlushAllStateResponse + 155, // 282: milvus.proto.data.DataCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 156, // 283: milvus.proto.data.DataCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 157, // 284: milvus.proto.data.DataCoord.ManualCompaction:output_type -> milvus.proto.milvus.ManualCompactionResponse + 158, // 285: milvus.proto.data.DataCoord.GetCompactionState:output_type -> milvus.proto.milvus.GetCompactionStateResponse + 159, // 286: milvus.proto.data.DataCoord.GetCompactionStateWithPlans:output_type -> milvus.proto.milvus.GetCompactionPlansResponse + 67, // 287: milvus.proto.data.DataCoord.WatchChannels:output_type -> milvus.proto.data.WatchChannelsResponse + 160, // 288: milvus.proto.data.DataCoord.GetFlushState:output_type -> milvus.proto.milvus.GetFlushStateResponse + 72, // 289: milvus.proto.data.DataCoord.DropVirtualChannel:output_type -> milvus.proto.data.DropVirtualChannelResponse + 69, // 290: milvus.proto.data.DataCoord.SetSegmentState:output_type -> milvus.proto.data.SetSegmentStateResponse + 117, // 291: milvus.proto.data.DataCoord.UpdateSegmentStatistics:output_type -> milvus.proto.common.Status + 117, // 292: milvus.proto.data.DataCoord.UpdateChannelCheckpoint:output_type -> milvus.proto.common.Status + 117, // 293: milvus.proto.data.DataCoord.MarkSegmentsDropped:output_type -> milvus.proto.common.Status + 117, // 294: milvus.proto.data.DataCoord.BroadcastAlteredCollection:output_type -> milvus.proto.common.Status + 161, // 295: milvus.proto.data.DataCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 117, // 296: milvus.proto.data.DataCoord.CreateIndex:output_type -> milvus.proto.common.Status + 117, // 297: milvus.proto.data.DataCoord.AlterIndex:output_type -> milvus.proto.common.Status + 162, // 298: milvus.proto.data.DataCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse + 163, // 299: milvus.proto.data.DataCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse + 164, // 300: milvus.proto.data.DataCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse + 117, // 301: milvus.proto.data.DataCoord.DropIndex:output_type -> milvus.proto.common.Status + 165, // 302: milvus.proto.data.DataCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse + 166, // 303: milvus.proto.data.DataCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse + 167, // 304: milvus.proto.data.DataCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse + 168, // 305: milvus.proto.data.DataCoord.ListIndexes:output_type -> milvus.proto.index.ListIndexesResponse + 81, // 306: milvus.proto.data.DataCoord.GcConfirm:output_type -> milvus.proto.data.GcConfirmResponse + 117, // 307: milvus.proto.data.DataCoord.ReportDataNodeTtMsgs:output_type -> milvus.proto.common.Status + 117, // 308: milvus.proto.data.DataCoord.GcControl:output_type -> milvus.proto.common.Status + 169, // 309: milvus.proto.data.DataCoord.ImportV2:output_type -> milvus.proto.internal.ImportResponse + 170, // 310: milvus.proto.data.DataCoord.GetImportProgress:output_type -> milvus.proto.internal.GetImportProgressResponse + 171, // 311: milvus.proto.data.DataCoord.ListImports:output_type -> milvus.proto.internal.ListImportsResponse + 152, // 312: milvus.proto.data.DataNode.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 153, // 313: milvus.proto.data.DataNode.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 117, // 314: milvus.proto.data.DataNode.WatchDmChannels:output_type -> milvus.proto.common.Status + 117, // 315: milvus.proto.data.DataNode.FlushSegments:output_type -> milvus.proto.common.Status + 155, // 316: milvus.proto.data.DataNode.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 156, // 317: milvus.proto.data.DataNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 117, // 318: milvus.proto.data.DataNode.CompactionV2:output_type -> milvus.proto.common.Status + 64, // 319: milvus.proto.data.DataNode.GetCompactionState:output_type -> milvus.proto.data.CompactionStateResponse + 117, // 320: milvus.proto.data.DataNode.SyncSegments:output_type -> milvus.proto.common.Status + 76, // 321: milvus.proto.data.DataNode.ResendSegmentStats:output_type -> milvus.proto.data.ResendSegmentStatsResponse + 117, // 322: milvus.proto.data.DataNode.FlushChannels:output_type -> milvus.proto.common.Status + 117, // 323: milvus.proto.data.DataNode.NotifyChannelOperation:output_type -> milvus.proto.common.Status + 85, // 324: milvus.proto.data.DataNode.CheckChannelOperationProgress:output_type -> milvus.proto.data.ChannelOperationProgressResponse + 117, // 325: milvus.proto.data.DataNode.PreImport:output_type -> milvus.proto.common.Status + 117, // 326: milvus.proto.data.DataNode.ImportV2:output_type -> milvus.proto.common.Status + 93, // 327: milvus.proto.data.DataNode.QueryPreImport:output_type -> milvus.proto.data.QueryPreImportResponse + 96, // 328: milvus.proto.data.DataNode.QueryImport:output_type -> milvus.proto.data.QueryImportResponse + 117, // 329: milvus.proto.data.DataNode.DropImport:output_type -> milvus.proto.common.Status + 103, // 330: milvus.proto.data.DataNode.QuerySlot:output_type -> milvus.proto.data.QuerySlotResponse + 117, // 331: milvus.proto.data.DataNode.DropCompactionPlan:output_type -> milvus.proto.common.Status + 263, // [263:332] is the sub-list for method output_type + 194, // [194:263] is the sub-list for method input_type + 194, // [194:194] is the sub-list for extension type_name + 194, // [194:194] is the sub-list for extension extendee + 0, // [0:194] is the sub-list for field type_name } func init() { file_data_coord_proto_init() } @@ -11899,7 +11983,7 @@ func file_data_coord_proto_init() { File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_data_coord_proto_rawDesc, - NumEnums: 7, + NumEnums: 8, NumMessages: 108, NumExtensions: 0, NumServices: 2, diff --git a/pkg/proto/internal.proto b/pkg/proto/internal.proto index 8fac366937..24728f4b2c 100644 --- a/pkg/proto/internal.proto +++ b/pkg/proto/internal.proto @@ -323,7 +323,7 @@ message ImportFile { } message ImportRequestInternal { - int64 dbID = 1; + int64 dbID = 1 [deprecated=true]; int64 collectionID = 2; string collection_name = 3; repeated int64 partitionIDs = 4; @@ -331,6 +331,8 @@ message ImportRequestInternal { schema.CollectionSchema schema = 6; repeated ImportFile files = 7; repeated common.KeyValuePair options = 8; + uint64 data_timestamp = 9; + int64 jobID = 10; } message ImportRequest { diff --git a/pkg/proto/internalpb/internal.pb.go b/pkg/proto/internalpb/internal.pb.go index c53a40294b..68e3acedad 100644 --- a/pkg/proto/internalpb/internal.pb.go +++ b/pkg/proto/internalpb/internal.pb.go @@ -2793,6 +2793,7 @@ type ImportRequestInternal struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields + // Deprecated: Marked as deprecated in internal.proto. DbID int64 `protobuf:"varint,1,opt,name=dbID,proto3" json:"dbID,omitempty"` CollectionID int64 `protobuf:"varint,2,opt,name=collectionID,proto3" json:"collectionID,omitempty"` CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` @@ -2801,6 +2802,8 @@ type ImportRequestInternal struct { Schema *schemapb.CollectionSchema `protobuf:"bytes,6,opt,name=schema,proto3" json:"schema,omitempty"` Files []*ImportFile `protobuf:"bytes,7,rep,name=files,proto3" json:"files,omitempty"` Options []*commonpb.KeyValuePair `protobuf:"bytes,8,rep,name=options,proto3" json:"options,omitempty"` + DataTimestamp uint64 `protobuf:"varint,9,opt,name=data_timestamp,json=dataTimestamp,proto3" json:"data_timestamp,omitempty"` + JobID int64 `protobuf:"varint,10,opt,name=jobID,proto3" json:"jobID,omitempty"` } func (x *ImportRequestInternal) Reset() { @@ -2835,6 +2838,7 @@ func (*ImportRequestInternal) Descriptor() ([]byte, []int) { return file_internal_proto_rawDescGZIP(), []int{31} } +// Deprecated: Marked as deprecated in internal.proto. func (x *ImportRequestInternal) GetDbID() int64 { if x != nil { return x.DbID @@ -2891,6 +2895,20 @@ func (x *ImportRequestInternal) GetOptions() []*commonpb.KeyValuePair { return nil } +func (x *ImportRequestInternal) GetDataTimestamp() uint64 { + if x != nil { + return x.DataTimestamp + } + return 0 +} + +func (x *ImportRequestInternal) GetJobID() int64 { + if x != nil { + return x.JobID + } + return 0 +} + type ImportRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -3976,157 +3994,161 @@ var file_internal_proto_rawDesc = []byte{ 0x02, 0x20, 0x01, 0x28, 0x01, 0x52, 0x01, 0x72, 0x22, 0x32, 0x0a, 0x0a, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x22, 0xf6, 0x02, 0x0a, + 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x70, 0x61, 0x74, 0x68, 0x73, 0x22, 0xb7, 0x03, 0x0a, 0x15, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x6e, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 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, 0x27, - 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, - 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0c, 0x70, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x63, - 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x73, - 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x06, 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, - 0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, 0x6c, - 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x16, 0x0a, 0x04, 0x64, 0x62, 0x49, 0x44, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x42, 0x02, 0x18, 0x01, 0x52, 0x04, 0x64, 0x62, 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, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x70, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, + 0x03, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, + 0x23, 0x0a, 0x0d, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, + 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x4e, + 0x61, 0x6d, 0x65, 0x73, 0x12, 0x3d, 0x0a, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x06, + 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, 0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07, + 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, + 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x64, 0x61, 0x74, + 0x61, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x04, 0x52, 0x0d, 0x64, 0x61, 0x74, 0x61, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x22, 0xee, 0x01, 0x0a, 0x0d, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, + 0x69, 0x6c, 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x6f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, + 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x5b, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 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, - 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, 0x6f, 0x70, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xee, 0x01, 0x0a, 0x0d, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, - 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0d, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, - 0x12, 0x37, 0x0a, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, - 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x46, 0x69, - 0x6c, 0x65, 0x52, 0x05, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x12, 0x3b, 0x0a, 0x07, 0x6f, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, - 0x2e, 0x4b, 0x65, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x50, 0x61, 0x69, 0x72, 0x52, 0x07, 0x6f, - 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x5b, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, - 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, 0x14, 0x0a, - 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, - 0x62, 0x49, 0x44, 0x22, 0x49, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, - 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, - 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x22, 0x81, - 0x02, 0x0a, 0x12, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x50, 0x72, 0x6f, - 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, 0x12, - 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, - 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, 0x72, - 0x65, 0x73, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, - 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, - 0x6c, 0x65, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, - 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, 0x23, - 0x0a, 0x0d, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, - 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52, - 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, - 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, - 0x77, 0x73, 0x22, 0xc6, 0x03, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, - 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 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, 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, - 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, 0x61, - 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, - 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x70, 0x72, - 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, - 0x54, 0x69, 0x6d, 0x65, 0x12, 0x52, 0x0a, 0x0f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x70, 0x72, 0x6f, - 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, - 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, - 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x0e, 0x74, 0x61, 0x73, 0x6b, 0x50, 0x72, - 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6d, 0x70, 0x6f, - 0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, - 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, - 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, 0x0a, - 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x1a, 0x4c, - 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, 0x49, - 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 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, 0x22, 0x56, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, - 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, - 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x86, 0x02, 0x0a, 0x13, 0x4c, 0x69, - 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x14, + 0x0a, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, + 0x6f, 0x62, 0x49, 0x44, 0x22, 0x49, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x6a, 0x6f, 0x62, + 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x22, + 0x81, 0x02, 0x0a, 0x12, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, 0x6b, 0x50, 0x72, + 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x1b, 0x0a, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x66, 0x69, 0x6c, 0x65, 0x53, 0x69, 0x7a, 0x65, + 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x6f, 0x67, + 0x72, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x70, 0x72, 0x6f, 0x67, + 0x72, 0x65, 0x73, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, + 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, + 0x70, 0x6c, 0x65, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x73, 0x74, 0x61, + 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12, + 0x23, 0x0a, 0x0d, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, + 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, + 0x77, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, + 0x6f, 0x77, 0x73, 0x22, 0xc6, 0x03, 0x0a, 0x19, 0x47, 0x65, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 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, 0x16, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x73, 0x12, 0x3d, - 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32, 0x25, + 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3b, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, + 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x73, 0x74, + 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x70, + 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x08, 0x70, + 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x23, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x5f, 0x74, 0x69, 0x6d, + 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, + 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x52, 0x0a, 0x0f, 0x74, 0x61, 0x73, 0x6b, 0x5f, 0x70, 0x72, + 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x18, 0x0a, - 0x07, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, - 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x67, 0x72, - 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x70, 0x72, 0x6f, - 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, - 0x09, 0x52, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, - 0x65, 0x73, 0x2a, 0x45, 0x0a, 0x09, 0x52, 0x61, 0x74, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, 0x12, - 0x0b, 0x0a, 0x07, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, - 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x43, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x2a, 0xb9, 0x01, 0x0a, 0x08, 0x52, 0x61, - 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, 0x6c, - 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x44, 0x44, 0x4c, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, 0x44, - 0x44, 0x4c, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44, 0x4c, - 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, - 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, - 0x4c, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, - 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x44, 0x4d, 0x4c, 0x42, - 0x75, 0x6c, 0x6b, 0x4c, 0x6f, 0x61, 0x64, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x51, 0x4c, - 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x51, 0x4c, 0x51, - 0x75, 0x65, 0x72, 0x79, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x55, 0x70, 0x73, - 0x65, 0x72, 0x74, 0x10, 0x0a, 0x2a, 0x81, 0x01, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, - 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, 0x65, - 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, 0x12, - 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, - 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, 0x03, - 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, - 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x49, - 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, 0x09, - 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x73, 0x10, 0x07, 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, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x54, 0x61, 0x73, + 0x6b, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x0e, 0x74, 0x61, 0x73, 0x6b, 0x50, + 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x23, 0x0a, 0x0d, 0x69, 0x6d, 0x70, + 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x0c, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, + 0x0a, 0x0a, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x77, 0x73, 0x18, 0x09, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x09, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x52, 0x6f, 0x77, 0x73, 0x12, 0x1d, 0x0a, + 0x0a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x73, 0x74, 0x61, 0x72, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x54, 0x0a, 0x1a, + 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x62, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x64, 0x62, 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, 0x22, 0x56, 0x0a, 0x12, 0x4c, 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, + 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x17, 0x0a, 0x07, 0x64, 0x62, 0x5f, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x64, 0x62, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x86, 0x02, 0x0a, 0x13, 0x4c, + 0x69, 0x73, 0x74, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 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, 0x16, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x49, 0x44, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x6a, 0x6f, 0x62, 0x49, 0x44, 0x73, 0x12, + 0x3d, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0e, 0x32, + 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, + 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x18, + 0x0a, 0x07, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, + 0x07, 0x72, 0x65, 0x61, 0x73, 0x6f, 0x6e, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x67, + 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x70, 0x72, + 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, + 0x28, 0x09, 0x52, 0x0f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, + 0x6d, 0x65, 0x73, 0x2a, 0x45, 0x0a, 0x09, 0x52, 0x61, 0x74, 0x65, 0x53, 0x63, 0x6f, 0x70, 0x65, + 0x12, 0x0b, 0x0a, 0x07, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x10, 0x00, 0x12, 0x0c, 0x0a, + 0x08, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x10, 0x01, 0x12, 0x0e, 0x0a, 0x0a, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x2a, 0xb9, 0x01, 0x0a, 0x08, 0x52, + 0x61, 0x74, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x00, 0x12, 0x10, 0x0a, 0x0c, 0x44, 0x44, + 0x4c, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x01, 0x12, 0x0c, 0x0a, 0x08, + 0x44, 0x44, 0x4c, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x10, 0x02, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x44, + 0x4c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x44, 0x4c, 0x43, + 0x6f, 0x6d, 0x70, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x44, + 0x4d, 0x4c, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x05, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, + 0x4c, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x44, 0x4d, 0x4c, + 0x42, 0x75, 0x6c, 0x6b, 0x4c, 0x6f, 0x61, 0x64, 0x10, 0x07, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x51, + 0x4c, 0x53, 0x65, 0x61, 0x72, 0x63, 0x68, 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x51, 0x4c, + 0x51, 0x75, 0x65, 0x72, 0x79, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x44, 0x4d, 0x4c, 0x55, 0x70, + 0x73, 0x65, 0x72, 0x74, 0x10, 0x0a, 0x2a, 0x81, 0x01, 0x0a, 0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x08, 0x0a, 0x04, 0x4e, 0x6f, 0x6e, + 0x65, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x01, + 0x12, 0x10, 0x0a, 0x0c, 0x50, 0x72, 0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, + 0x10, 0x02, 0x12, 0x0d, 0x0a, 0x09, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x10, + 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x61, 0x69, 0x6c, 0x65, 0x64, 0x10, 0x04, 0x12, 0x0d, 0x0a, + 0x09, 0x43, 0x6f, 0x6d, 0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x69, 0x6e, 0x67, 0x10, 0x06, 0x12, + 0x09, 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x73, 0x10, 0x07, 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, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2f, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x70, 0x62, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/pkg/proto/messages.proto b/pkg/proto/messages.proto index 61b525deb9..604641c0e3 100644 --- a/pkg/proto/messages.proto +++ b/pkg/proto/messages.proto @@ -35,6 +35,7 @@ enum MessageType { DropPartition = 8; ManualFlush = 9; CreateSegment = 10; + Import = 11; // begin transaction message is only used for transaction, once a begin // transaction message is received, all messages combined with the // transaction message cannot be consumed until a CommitTxn message @@ -204,6 +205,8 @@ message RollbackTxnMessageHeader {} // Just do nothing now. message TxnMessageHeader {} +message ImportMessageHeader {} + /// /// Message Extra Response /// Used to add extra information when response to the client. diff --git a/pkg/proto/messagespb/messages.pb.go b/pkg/proto/messagespb/messages.pb.go index 0322d682e2..1a278a808a 100644 --- a/pkg/proto/messagespb/messages.pb.go +++ b/pkg/proto/messagespb/messages.pb.go @@ -35,6 +35,7 @@ const ( MessageType_DropPartition MessageType = 8 MessageType_ManualFlush MessageType = 9 MessageType_CreateSegment MessageType = 10 + MessageType_Import MessageType = 11 // begin transaction message is only used for transaction, once a begin // transaction message is received, all messages combined with the // transaction message cannot be consumed until a CommitTxn message @@ -72,6 +73,7 @@ var ( 8: "DropPartition", 9: "ManualFlush", 10: "CreateSegment", + 11: "Import", 900: "BeginTxn", 901: "CommitTxn", 902: "RollbackTxn", @@ -89,6 +91,7 @@ var ( "DropPartition": 8, "ManualFlush": 9, "CreateSegment": 10, + "Import": 11, "BeginTxn": 900, "CommitTxn": 901, "RollbackTxn": 902, @@ -1574,6 +1577,44 @@ func (*TxnMessageHeader) Descriptor() ([]byte, []int) { return file_messages_proto_rawDescGZIP(), []int{26} } +type ImportMessageHeader struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *ImportMessageHeader) Reset() { + *x = ImportMessageHeader{} + if protoimpl.UnsafeEnabled { + mi := &file_messages_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ImportMessageHeader) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ImportMessageHeader) ProtoMessage() {} + +func (x *ImportMessageHeader) ProtoReflect() protoreflect.Message { + mi := &file_messages_proto_msgTypes[27] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ImportMessageHeader.ProtoReflect.Descriptor instead. +func (*ImportMessageHeader) Descriptor() ([]byte, []int) { + return file_messages_proto_rawDescGZIP(), []int{27} +} + // ManualFlushExtraResponse is the extra response of manual flush message. type ManualFlushExtraResponse struct { state protoimpl.MessageState @@ -1586,7 +1627,7 @@ type ManualFlushExtraResponse struct { func (x *ManualFlushExtraResponse) Reset() { *x = ManualFlushExtraResponse{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[27] + mi := &file_messages_proto_msgTypes[28] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1599,7 +1640,7 @@ func (x *ManualFlushExtraResponse) String() string { func (*ManualFlushExtraResponse) ProtoMessage() {} func (x *ManualFlushExtraResponse) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[27] + mi := &file_messages_proto_msgTypes[28] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1612,7 +1653,7 @@ func (x *ManualFlushExtraResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ManualFlushExtraResponse.ProtoReflect.Descriptor instead. func (*ManualFlushExtraResponse) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{27} + return file_messages_proto_rawDescGZIP(), []int{28} } func (x *ManualFlushExtraResponse) GetSegmentIds() []int64 { @@ -1640,7 +1681,7 @@ type TxnContext struct { func (x *TxnContext) Reset() { *x = TxnContext{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[28] + mi := &file_messages_proto_msgTypes[29] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1653,7 +1694,7 @@ func (x *TxnContext) String() string { func (*TxnContext) ProtoMessage() {} func (x *TxnContext) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[28] + mi := &file_messages_proto_msgTypes[29] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1666,7 +1707,7 @@ func (x *TxnContext) ProtoReflect() protoreflect.Message { // Deprecated: Use TxnContext.ProtoReflect.Descriptor instead. func (*TxnContext) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{28} + return file_messages_proto_rawDescGZIP(), []int{29} } func (x *TxnContext) GetTxnId() int64 { @@ -1696,7 +1737,7 @@ type RMQMessageLayout struct { func (x *RMQMessageLayout) Reset() { *x = RMQMessageLayout{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[29] + mi := &file_messages_proto_msgTypes[30] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1709,7 +1750,7 @@ func (x *RMQMessageLayout) String() string { func (*RMQMessageLayout) ProtoMessage() {} func (x *RMQMessageLayout) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[29] + mi := &file_messages_proto_msgTypes[30] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1722,7 +1763,7 @@ func (x *RMQMessageLayout) ProtoReflect() protoreflect.Message { // Deprecated: Use RMQMessageLayout.ProtoReflect.Descriptor instead. func (*RMQMessageLayout) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{29} + return file_messages_proto_rawDescGZIP(), []int{30} } func (x *RMQMessageLayout) GetPayload() []byte { @@ -1753,7 +1794,7 @@ type BroadcastHeader struct { func (x *BroadcastHeader) Reset() { *x = BroadcastHeader{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[30] + mi := &file_messages_proto_msgTypes[31] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1766,7 +1807,7 @@ func (x *BroadcastHeader) String() string { func (*BroadcastHeader) ProtoMessage() {} func (x *BroadcastHeader) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[30] + mi := &file_messages_proto_msgTypes[31] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1779,7 +1820,7 @@ func (x *BroadcastHeader) ProtoReflect() protoreflect.Message { // Deprecated: Use BroadcastHeader.ProtoReflect.Descriptor instead. func (*BroadcastHeader) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{30} + return file_messages_proto_rawDescGZIP(), []int{31} } func (x *BroadcastHeader) GetBroadcastId() uint64 { @@ -1818,7 +1859,7 @@ type ResourceKey struct { func (x *ResourceKey) Reset() { *x = ResourceKey{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[31] + mi := &file_messages_proto_msgTypes[32] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1831,7 +1872,7 @@ func (x *ResourceKey) String() string { func (*ResourceKey) ProtoMessage() {} func (x *ResourceKey) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[31] + mi := &file_messages_proto_msgTypes[32] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1844,7 +1885,7 @@ func (x *ResourceKey) ProtoReflect() protoreflect.Message { // Deprecated: Use ResourceKey.ProtoReflect.Descriptor instead. func (*ResourceKey) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{31} + return file_messages_proto_rawDescGZIP(), []int{32} } func (x *ResourceKey) GetDomain() ResourceDomain { @@ -1877,7 +1918,7 @@ type BroadcastEvent struct { func (x *BroadcastEvent) Reset() { *x = BroadcastEvent{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[32] + mi := &file_messages_proto_msgTypes[33] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1890,7 +1931,7 @@ func (x *BroadcastEvent) String() string { func (*BroadcastEvent) ProtoMessage() {} func (x *BroadcastEvent) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[32] + mi := &file_messages_proto_msgTypes[33] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1903,7 +1944,7 @@ func (x *BroadcastEvent) ProtoReflect() protoreflect.Message { // Deprecated: Use BroadcastEvent.ProtoReflect.Descriptor instead. func (*BroadcastEvent) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{32} + return file_messages_proto_rawDescGZIP(), []int{33} } func (m *BroadcastEvent) GetEvent() isBroadcastEvent_Event { @@ -1955,7 +1996,7 @@ type BroadcastResourceKeyAckAll struct { func (x *BroadcastResourceKeyAckAll) Reset() { *x = BroadcastResourceKeyAckAll{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[33] + mi := &file_messages_proto_msgTypes[34] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1968,7 +2009,7 @@ func (x *BroadcastResourceKeyAckAll) String() string { func (*BroadcastResourceKeyAckAll) ProtoMessage() {} func (x *BroadcastResourceKeyAckAll) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[33] + mi := &file_messages_proto_msgTypes[34] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1981,7 +2022,7 @@ func (x *BroadcastResourceKeyAckAll) ProtoReflect() protoreflect.Message { // Deprecated: Use BroadcastResourceKeyAckAll.ProtoReflect.Descriptor instead. func (*BroadcastResourceKeyAckAll) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{33} + return file_messages_proto_rawDescGZIP(), []int{34} } func (x *BroadcastResourceKeyAckAll) GetResourceKey() *ResourceKey { @@ -2003,7 +2044,7 @@ type BroadcastResourceKeyAckOne struct { func (x *BroadcastResourceKeyAckOne) Reset() { *x = BroadcastResourceKeyAckOne{} if protoimpl.UnsafeEnabled { - mi := &file_messages_proto_msgTypes[34] + mi := &file_messages_proto_msgTypes[35] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -2016,7 +2057,7 @@ func (x *BroadcastResourceKeyAckOne) String() string { func (*BroadcastResourceKeyAckOne) ProtoMessage() {} func (x *BroadcastResourceKeyAckOne) ProtoReflect() protoreflect.Message { - mi := &file_messages_proto_msgTypes[34] + mi := &file_messages_proto_msgTypes[35] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -2029,7 +2070,7 @@ func (x *BroadcastResourceKeyAckOne) ProtoReflect() protoreflect.Message { // Deprecated: Use BroadcastResourceKeyAckOne.ProtoReflect.Descriptor instead. func (*BroadcastResourceKeyAckOne) Descriptor() ([]byte, []int) { - return file_messages_proto_rawDescGZIP(), []int{34} + return file_messages_proto_rawDescGZIP(), []int{35} } func (x *BroadcastResourceKeyAckOne) GetResourceKey() *ResourceKey { @@ -2176,107 +2217,109 @@ var file_messages_proto_rawDesc = []byte{ 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x1a, 0x0a, 0x18, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x12, 0x0a, 0x10, 0x54, 0x78, 0x6e, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x3b, 0x0a, 0x18, 0x4d, 0x61, - 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, - 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43, 0x6f, - 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78, 0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x16, - 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, - 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, - 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, - 0x6e, 0x64, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x10, 0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, - 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, - 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, - 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, - 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x2e, - 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, - 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 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, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f, 0x42, - 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21, - 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49, - 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, - 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, - 0x47, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, - 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, - 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0c, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, 0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d, 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, - 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, - 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06, - 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, 0xe5, 0x01, 0x0a, 0x0e, 0x42, 0x72, 0x6f, - 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x64, 0x0a, 0x14, 0x72, - 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, - 0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x11, - 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, - 0x6c, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, - 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x6f, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, - 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f, - 0x6e, 0x65, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, - 0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, - 0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x45, - 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x22, 0x15, 0x0a, 0x13, 0x49, 0x6d, + 0x70, 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x22, 0x3b, 0x0a, 0x18, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, + 0x45, 0x78, 0x74, 0x72, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1f, 0x0a, + 0x0b, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x03, 0x52, 0x0a, 0x73, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x73, 0x22, 0x5a, + 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x12, 0x15, 0x0a, 0x06, + 0x74, 0x78, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x74, 0x78, + 0x6e, 0x49, 0x64, 0x12, 0x35, 0x0a, 0x16, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, + 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x15, 0x6b, 0x65, 0x65, 0x70, 0x61, 0x6c, 0x69, 0x76, 0x65, 0x4d, 0x69, + 0x6c, 0x6c, 0x69, 0x73, 0x65, 0x63, 0x6f, 0x6e, 0x64, 0x73, 0x22, 0xc4, 0x01, 0x0a, 0x10, 0x52, + 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x12, + 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x57, 0x0a, 0x0a, 0x70, 0x72, 0x6f, + 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x37, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x4d, 0x51, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x4c, 0x61, 0x79, 0x6f, 0x75, 0x74, 0x2e, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x70, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, + 0x65, 0x73, 0x1a, 0x3d, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x70, 0x65, 0x72, 0x74, 0x69, 0x65, 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, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x9b, 0x01, 0x0a, 0x0f, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x48, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x21, 0x0a, 0x0c, 0x62, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, + 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x62, 0x72, 0x6f, + 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1c, 0x0a, 0x09, 0x76, 0x63, 0x68, 0x61, + 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x76, 0x63, 0x68, + 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x73, 0x12, 0x47, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x22, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, + 0x79, 0x52, 0x0c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x73, 0x22, + 0x5e, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x3d, + 0x0a, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x25, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, + 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x06, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x22, + 0xe5, 0x01, 0x0a, 0x0e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, + 0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, - 0x4f, 0x6e, 0x65, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, - 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0b, 0x72, - 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x2a, 0xfc, 0x01, 0x0a, 0x0b, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, - 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54, - 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, - 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a, - 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x04, 0x12, 0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12, - 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x11, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, - 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43, - 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0d, - 0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10, 0x84, 0x07, 0x12, 0x0e, 0x0a, - 0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10, 0x85, 0x07, 0x12, 0x10, 0x0a, - 0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x10, 0x86, 0x07, 0x12, - 0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x82, 0x01, 0x0a, 0x08, 0x54, 0x78, - 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78, 0x6e, 0x55, 0x6e, 0x6b, - 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e, 0x42, 0x65, 0x67, - 0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, 0x46, 0x6c, 0x69, - 0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, 0x6e, 0x43, 0x6f, - 0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, 0x43, 0x6f, 0x6d, - 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x4f, - 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11, 0x0a, 0x0d, 0x54, - 0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, 0x06, 0x2a, 0x6c, - 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, - 0x12, 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, - 0x69, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x1d, 0x0a, 0x19, 0x52, - 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x49, 0x6d, 0x70, - 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44, 0x10, 0x01, 0x12, 0x20, 0x0a, 0x1c, 0x52, 0x65, - 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x43, 0x6f, 0x6c, 0x6c, - 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x02, 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, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x70, 0x62, - 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x41, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, + 0x65, 0x79, 0x41, 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x64, 0x0a, 0x14, 0x72, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x61, 0x63, 0x6b, 0x5f, 0x6f, 0x6e, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x42, + 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x48, 0x00, 0x52, 0x11, 0x72, 0x65, 0x73, + 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x42, 0x07, + 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x63, 0x0a, 0x1a, 0x42, 0x72, 0x6f, 0x61, 0x64, + 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x41, + 0x63, 0x6b, 0x41, 0x6c, 0x6c, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x65, 0x73, 0x73, 0x61, + 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x52, + 0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, 0x79, 0x22, 0x63, 0x0a, 0x1a, + 0x42, 0x72, 0x6f, 0x61, 0x64, 0x63, 0x61, 0x73, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x4b, 0x65, 0x79, 0x41, 0x63, 0x6b, 0x4f, 0x6e, 0x65, 0x12, 0x45, 0x0a, 0x0c, 0x72, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x22, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x4b, 0x65, 0x79, 0x52, 0x0b, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x4b, 0x65, + 0x79, 0x2a, 0x88, 0x02, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x54, 0x79, 0x70, + 0x65, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, + 0x0a, 0x08, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, + 0x49, 0x6e, 0x73, 0x65, 0x72, 0x74, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x6c, 0x75, 0x73, 0x68, 0x10, 0x04, 0x12, + 0x14, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x10, 0x05, 0x12, 0x12, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, + 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x06, 0x12, 0x13, 0x0a, 0x0f, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, 0x07, 0x12, 0x11, + 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x10, + 0x08, 0x12, 0x0f, 0x0a, 0x0b, 0x4d, 0x61, 0x6e, 0x75, 0x61, 0x6c, 0x46, 0x6c, 0x75, 0x73, 0x68, + 0x10, 0x09, 0x12, 0x11, 0x0a, 0x0d, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x65, 0x67, 0x6d, + 0x65, 0x6e, 0x74, 0x10, 0x0a, 0x12, 0x0a, 0x0a, 0x06, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x10, + 0x0b, 0x12, 0x0d, 0x0a, 0x08, 0x42, 0x65, 0x67, 0x69, 0x6e, 0x54, 0x78, 0x6e, 0x10, 0x84, 0x07, + 0x12, 0x0e, 0x0a, 0x09, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x54, 0x78, 0x6e, 0x10, 0x85, 0x07, + 0x12, 0x10, 0x0a, 0x0b, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x54, 0x78, 0x6e, 0x10, + 0x86, 0x07, 0x12, 0x08, 0x0a, 0x03, 0x54, 0x78, 0x6e, 0x10, 0xe7, 0x07, 0x2a, 0x82, 0x01, 0x0a, + 0x08, 0x54, 0x78, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x78, 0x6e, + 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x54, 0x78, 0x6e, + 0x42, 0x65, 0x67, 0x69, 0x6e, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x49, 0x6e, + 0x46, 0x6c, 0x69, 0x67, 0x68, 0x74, 0x10, 0x02, 0x12, 0x0f, 0x0a, 0x0b, 0x54, 0x78, 0x6e, 0x4f, + 0x6e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x10, 0x03, 0x12, 0x10, 0x0a, 0x0c, 0x54, 0x78, 0x6e, + 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x10, 0x04, 0x12, 0x11, 0x0a, 0x0d, 0x54, + 0x78, 0x6e, 0x4f, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x10, 0x05, 0x12, 0x11, + 0x0a, 0x0d, 0x54, 0x78, 0x6e, 0x52, 0x6f, 0x6c, 0x6c, 0x62, 0x61, 0x63, 0x6b, 0x65, 0x64, 0x10, + 0x06, 0x2a, 0x6c, 0x0a, 0x0e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, + 0x61, 0x69, 0x6e, 0x12, 0x19, 0x0a, 0x15, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, + 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x1d, + 0x0a, 0x19, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, + 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x4a, 0x6f, 0x62, 0x49, 0x44, 0x10, 0x01, 0x12, 0x20, 0x0a, + 0x1c, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x43, + 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x10, 0x02, 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, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x73, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2292,7 +2335,7 @@ func file_messages_proto_rawDescGZIP() []byte { } var file_messages_proto_enumTypes = make([]protoimpl.EnumInfo, 3) -var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 38) +var file_messages_proto_msgTypes = make([]protoimpl.MessageInfo, 39) var file_messages_proto_goTypes = []interface{}{ (MessageType)(0), // 0: milvus.proto.messages.MessageType (TxnState)(0), // 1: milvus.proto.messages.TxnState @@ -2324,33 +2367,34 @@ var file_messages_proto_goTypes = []interface{}{ (*CommitTxnMessageHeader)(nil), // 27: milvus.proto.messages.CommitTxnMessageHeader (*RollbackTxnMessageHeader)(nil), // 28: milvus.proto.messages.RollbackTxnMessageHeader (*TxnMessageHeader)(nil), // 29: milvus.proto.messages.TxnMessageHeader - (*ManualFlushExtraResponse)(nil), // 30: milvus.proto.messages.ManualFlushExtraResponse - (*TxnContext)(nil), // 31: milvus.proto.messages.TxnContext - (*RMQMessageLayout)(nil), // 32: milvus.proto.messages.RMQMessageLayout - (*BroadcastHeader)(nil), // 33: milvus.proto.messages.BroadcastHeader - (*ResourceKey)(nil), // 34: milvus.proto.messages.ResourceKey - (*BroadcastEvent)(nil), // 35: milvus.proto.messages.BroadcastEvent - (*BroadcastResourceKeyAckAll)(nil), // 36: milvus.proto.messages.BroadcastResourceKeyAckAll - (*BroadcastResourceKeyAckOne)(nil), // 37: milvus.proto.messages.BroadcastResourceKeyAckOne - nil, // 38: milvus.proto.messages.Message.PropertiesEntry - nil, // 39: milvus.proto.messages.ImmutableMessage.PropertiesEntry - nil, // 40: milvus.proto.messages.RMQMessageLayout.PropertiesEntry + (*ImportMessageHeader)(nil), // 30: milvus.proto.messages.ImportMessageHeader + (*ManualFlushExtraResponse)(nil), // 31: milvus.proto.messages.ManualFlushExtraResponse + (*TxnContext)(nil), // 32: milvus.proto.messages.TxnContext + (*RMQMessageLayout)(nil), // 33: milvus.proto.messages.RMQMessageLayout + (*BroadcastHeader)(nil), // 34: milvus.proto.messages.BroadcastHeader + (*ResourceKey)(nil), // 35: milvus.proto.messages.ResourceKey + (*BroadcastEvent)(nil), // 36: milvus.proto.messages.BroadcastEvent + (*BroadcastResourceKeyAckAll)(nil), // 37: milvus.proto.messages.BroadcastResourceKeyAckAll + (*BroadcastResourceKeyAckOne)(nil), // 38: milvus.proto.messages.BroadcastResourceKeyAckOne + nil, // 39: milvus.proto.messages.Message.PropertiesEntry + nil, // 40: milvus.proto.messages.ImmutableMessage.PropertiesEntry + nil, // 41: milvus.proto.messages.RMQMessageLayout.PropertiesEntry } var file_messages_proto_depIdxs = []int32{ - 38, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry + 39, // 0: milvus.proto.messages.Message.properties:type_name -> milvus.proto.messages.Message.PropertiesEntry 3, // 1: milvus.proto.messages.ImmutableMessage.id:type_name -> milvus.proto.messages.MessageID - 39, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry + 40, // 2: milvus.proto.messages.ImmutableMessage.properties:type_name -> milvus.proto.messages.ImmutableMessage.PropertiesEntry 9, // 3: milvus.proto.messages.CreateSegmentMessageBody.segments:type_name -> milvus.proto.messages.CreateSegmentInfo 4, // 4: milvus.proto.messages.TxnMessageBody.messages:type_name -> milvus.proto.messages.Message 16, // 5: milvus.proto.messages.InsertMessageHeader.partitions:type_name -> milvus.proto.messages.PartitionSegmentAssignment 17, // 6: milvus.proto.messages.PartitionSegmentAssignment.segment_assignment:type_name -> milvus.proto.messages.SegmentAssignment - 40, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry - 34, // 8: milvus.proto.messages.BroadcastHeader.Resource_keys:type_name -> milvus.proto.messages.ResourceKey + 41, // 7: milvus.proto.messages.RMQMessageLayout.properties:type_name -> milvus.proto.messages.RMQMessageLayout.PropertiesEntry + 35, // 8: milvus.proto.messages.BroadcastHeader.Resource_keys:type_name -> milvus.proto.messages.ResourceKey 2, // 9: milvus.proto.messages.ResourceKey.domain:type_name -> milvus.proto.messages.ResourceDomain - 36, // 10: milvus.proto.messages.BroadcastEvent.resource_key_ack_all:type_name -> milvus.proto.messages.BroadcastResourceKeyAckAll - 37, // 11: milvus.proto.messages.BroadcastEvent.resource_key_ack_one:type_name -> milvus.proto.messages.BroadcastResourceKeyAckOne - 34, // 12: milvus.proto.messages.BroadcastResourceKeyAckAll.resource_key:type_name -> milvus.proto.messages.ResourceKey - 34, // 13: milvus.proto.messages.BroadcastResourceKeyAckOne.resource_key:type_name -> milvus.proto.messages.ResourceKey + 37, // 10: milvus.proto.messages.BroadcastEvent.resource_key_ack_all:type_name -> milvus.proto.messages.BroadcastResourceKeyAckAll + 38, // 11: milvus.proto.messages.BroadcastEvent.resource_key_ack_one:type_name -> milvus.proto.messages.BroadcastResourceKeyAckOne + 35, // 12: milvus.proto.messages.BroadcastResourceKeyAckAll.resource_key:type_name -> milvus.proto.messages.ResourceKey + 35, // 13: milvus.proto.messages.BroadcastResourceKeyAckOne.resource_key:type_name -> milvus.proto.messages.ResourceKey 14, // [14:14] is the sub-list for method output_type 14, // [14:14] is the sub-list for method input_type 14, // [14:14] is the sub-list for extension type_name @@ -2689,7 +2733,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ManualFlushExtraResponse); i { + switch v := v.(*ImportMessageHeader); i { case 0: return &v.state case 1: @@ -2701,7 +2745,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TxnContext); i { + switch v := v.(*ManualFlushExtraResponse); i { case 0: return &v.state case 1: @@ -2713,7 +2757,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*RMQMessageLayout); i { + switch v := v.(*TxnContext); i { case 0: return &v.state case 1: @@ -2725,7 +2769,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BroadcastHeader); i { + switch v := v.(*RMQMessageLayout); i { case 0: return &v.state case 1: @@ -2737,7 +2781,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ResourceKey); i { + switch v := v.(*BroadcastHeader); i { case 0: return &v.state case 1: @@ -2749,7 +2793,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BroadcastEvent); i { + switch v := v.(*ResourceKey); i { case 0: return &v.state case 1: @@ -2761,7 +2805,7 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*BroadcastResourceKeyAckAll); i { + switch v := v.(*BroadcastEvent); i { case 0: return &v.state case 1: @@ -2773,6 +2817,18 @@ func file_messages_proto_init() { } } file_messages_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*BroadcastResourceKeyAckAll); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_messages_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*BroadcastResourceKeyAckOne); i { case 0: return &v.state @@ -2785,7 +2841,7 @@ func file_messages_proto_init() { } } } - file_messages_proto_msgTypes[32].OneofWrappers = []interface{}{ + file_messages_proto_msgTypes[33].OneofWrappers = []interface{}{ (*BroadcastEvent_ResourceKeyAckAll)(nil), (*BroadcastEvent_ResourceKeyAckOne)(nil), } @@ -2795,7 +2851,7 @@ func file_messages_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_messages_proto_rawDesc, NumEnums: 3, - NumMessages: 38, + NumMessages: 39, NumExtensions: 0, NumServices: 0, }, diff --git a/pkg/streaming/util/message/adaptor/message.go b/pkg/streaming/util/message/adaptor/message.go index 9c3335d952..12a958bbe8 100644 --- a/pkg/streaming/util/message/adaptor/message.go +++ b/pkg/streaming/util/message/adaptor/message.go @@ -175,6 +175,12 @@ func recoverMessageFromHeader(tsMsg msgstream.TsMsg, msg message.ImmutableMessag return nil, errors.Wrap(err, "Failed to convert message to delete message") } return recoverDeleteMsgFromHeader(tsMsg.(*msgstream.DeleteMsg), deleteMessage.Header(), msg.TimeTick()) + case message.MessageTypeImport: + importMessage, err := message.AsImmutableImportMessageV1(msg) + if err != nil { + return nil, errors.Wrap(err, "Failed to convert message to import message") + } + return recoverImportMsgFromHeader(tsMsg.(*msgstream.ImportMsg), importMessage.Header(), msg.TimeTick()) default: return tsMsg, nil } @@ -220,3 +226,8 @@ func recoverDeleteMsgFromHeader(deleteMsg *msgstream.DeleteMsg, header *message. deleteMsg.Timestamps = timestamps return deleteMsg, nil } + +func recoverImportMsgFromHeader(importMsg *msgstream.ImportMsg, header *message.ImportMessageHeader, timetick uint64) (msgstream.TsMsg, error) { + importMsg.Base.Timestamp = timetick + return importMsg, nil +} diff --git a/pkg/streaming/util/message/adaptor/message_type.go b/pkg/streaming/util/message/adaptor/message_type.go index 8338ca78b2..2f83d4833e 100644 --- a/pkg/streaming/util/message/adaptor/message_type.go +++ b/pkg/streaming/util/message/adaptor/message_type.go @@ -16,6 +16,7 @@ var messageTypeToCommonpbMsgType = map[message.MessageType]commonpb.MsgType{ message.MessageTypeDropCollection: commonpb.MsgType_DropCollection, message.MessageTypeCreatePartition: commonpb.MsgType_CreatePartition, message.MessageTypeDropPartition: commonpb.MsgType_DropPartition, + message.MessageTypeImport: commonpb.MsgType_Import, } // MustGetCommonpbMsgTypeFromMessageType returns the commonpb.MsgType from message.MessageType. diff --git a/pkg/streaming/util/message/builder.go b/pkg/streaming/util/message/builder.go index 98688b1c41..8df84094a8 100644 --- a/pkg/streaming/util/message/builder.go +++ b/pkg/streaming/util/message/builder.go @@ -60,6 +60,7 @@ var ( NewDropCollectionMessageBuilderV1 = createNewMessageBuilderV1[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest]() NewCreatePartitionMessageBuilderV1 = createNewMessageBuilderV1[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest]() NewDropPartitionMessageBuilderV1 = createNewMessageBuilderV1[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest]() + NewImportMessageBuilderV1 = createNewMessageBuilderV1[*ImportMessageHeader, *msgpb.ImportMsg]() NewCreateSegmentMessageBuilderV2 = createNewMessageBuilderV2[*CreateSegmentMessageHeader, *CreateSegmentMessageBody]() NewFlushMessageBuilderV2 = createNewMessageBuilderV2[*FlushMessageHeader, *FlushMessageBody]() NewManualFlushMessageBuilderV2 = createNewMessageBuilderV2[*ManualFlushMessageHeader, *ManualFlushMessageBody]() diff --git a/pkg/streaming/util/message/message_type.go b/pkg/streaming/util/message/message_type.go index c09b49ba72..ca3f21bb1e 100644 --- a/pkg/streaming/util/message/message_type.go +++ b/pkg/streaming/util/message/message_type.go @@ -24,6 +24,7 @@ const ( MessageTypeBeginTxn MessageType = MessageType(messagespb.MessageType_BeginTxn) MessageTypeCommitTxn MessageType = MessageType(messagespb.MessageType_CommitTxn) MessageTypeRollbackTxn MessageType = MessageType(messagespb.MessageType_RollbackTxn) + MessageTypeImport MessageType = MessageType(messagespb.MessageType_Import) ) var messageTypeName = map[MessageType]string{ @@ -42,6 +43,7 @@ var messageTypeName = map[MessageType]string{ MessageTypeBeginTxn: "BEGIN_TXN", MessageTypeCommitTxn: "COMMIT_TXN", MessageTypeRollbackTxn: "ROLLBACK_TXN", + MessageTypeImport: "IMPORT", } // String implements fmt.Stringer interface. diff --git a/pkg/streaming/util/message/specialized_message.go b/pkg/streaming/util/message/specialized_message.go index 780d8b0f57..102edd1569 100644 --- a/pkg/streaming/util/message/specialized_message.go +++ b/pkg/streaming/util/message/specialized_message.go @@ -28,6 +28,7 @@ type ( CommitTxnMessageHeader = messagespb.CommitTxnMessageHeader RollbackTxnMessageHeader = messagespb.RollbackTxnMessageHeader TxnMessageHeader = messagespb.TxnMessageHeader + ImportMessageHeader = messagespb.ImportMessageHeader ) type ( @@ -60,6 +61,7 @@ var messageTypeMap = map[reflect.Type]MessageType{ reflect.TypeOf(&CommitTxnMessageHeader{}): MessageTypeCommitTxn, reflect.TypeOf(&RollbackTxnMessageHeader{}): MessageTypeRollbackTxn, reflect.TypeOf(&TxnMessageHeader{}): MessageTypeTxn, + reflect.TypeOf(&ImportMessageHeader{}): MessageTypeImport, } // A system preserved message, should not allowed to provide outside of the streaming system. @@ -80,6 +82,7 @@ type ( MutableDropCollectionMessageV1 = specializedMutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest] MutableCreatePartitionMessageV1 = specializedMutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest] MutableDropPartitionMessageV1 = specializedMutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest] + MutableImportMessageV1 = specializedMutableMessage[*ImportMessageHeader, *msgpb.ImportMsg] MutableCreateSegmentMessageV2 = specializedMutableMessage[*CreateSegmentMessageHeader, *CreateSegmentMessageBody] MutableFlushMessageV2 = specializedMutableMessage[*FlushMessageHeader, *FlushMessageBody] MutableBeginTxnMessageV2 = specializedMutableMessage[*BeginTxnMessageHeader, *BeginTxnMessageBody] @@ -93,6 +96,7 @@ type ( ImmutableDropCollectionMessageV1 = specializedImmutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest] ImmutableCreatePartitionMessageV1 = specializedImmutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest] ImmutableDropPartitionMessageV1 = specializedImmutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest] + ImmutableImportMessageV1 = specializedImmutableMessage[*ImportMessageHeader, *msgpb.ImportMsg] ImmutableCreateSegmentMessageV2 = specializedImmutableMessage[*CreateSegmentMessageHeader, *CreateSegmentMessageBody] ImmutableFlushMessageV2 = specializedImmutableMessage[*FlushMessageHeader, *FlushMessageBody] ImmutableManualFlushMessageV2 = specializedImmutableMessage[*ManualFlushMessageHeader, *ManualFlushMessageBody] @@ -110,6 +114,7 @@ var ( AsMutableDropCollectionMessageV1 = asSpecializedMutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest] AsMutableCreatePartitionMessageV1 = asSpecializedMutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest] AsMutableDropPartitionMessageV1 = asSpecializedMutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest] + AsMutableImportMessageV1 = asSpecializedMutableMessage[*ImportMessageHeader, *msgpb.ImportMsg] AsMutableCreateSegmentMessageV2 = asSpecializedMutableMessage[*CreateSegmentMessageHeader, *CreateSegmentMessageBody] AsMutableFlushMessageV2 = asSpecializedMutableMessage[*FlushMessageHeader, *FlushMessageBody] AsMutableManualFlushMessageV2 = asSpecializedMutableMessage[*ManualFlushMessageHeader, *ManualFlushMessageBody] @@ -124,6 +129,7 @@ var ( AsImmutableDropCollectionMessageV1 = asSpecializedImmutableMessage[*DropCollectionMessageHeader, *msgpb.DropCollectionRequest] AsImmutableCreatePartitionMessageV1 = asSpecializedImmutableMessage[*CreatePartitionMessageHeader, *msgpb.CreatePartitionRequest] AsImmutableDropPartitionMessageV1 = asSpecializedImmutableMessage[*DropPartitionMessageHeader, *msgpb.DropPartitionRequest] + AsImmutableImportMessageV1 = asSpecializedImmutableMessage[*ImportMessageHeader, *msgpb.ImportMsg] AsImmutableCreateSegmentMessageV2 = asSpecializedImmutableMessage[*CreateSegmentMessageHeader, *CreateSegmentMessageBody] AsImmutableFlushMessageV2 = asSpecializedImmutableMessage[*FlushMessageHeader, *FlushMessageBody] AsImmutableManualFlushMessageV2 = asSpecializedImmutableMessage[*ManualFlushMessageHeader, *ManualFlushMessageBody] @@ -143,7 +149,7 @@ var ( // Return nil, nil if the message is not the target specialized message. // Return nil, error if the message is the target specialized message but failed to decode the specialized header. // Return specializedMutableMessage, nil if the message is the target specialized message and successfully decoded the specialized header. -func asSpecializedMutableMessage[H proto.Message, B proto.Message](msg MutableMessage) (specializedMutableMessage[H, B], error) { +func asSpecializedMutableMessage[H proto.Message, B proto.Message](msg BasicMessage) (specializedMutableMessage[H, B], error) { underlying := msg.(*messageImpl) var header H diff --git a/scripts/run_intergration_test.sh b/scripts/run_intergration_test.sh index 999387e43c..2c9a1ecd48 100755 --- a/scripts/run_intergration_test.sh +++ b/scripts/run_intergration_test.sh @@ -40,6 +40,8 @@ for d in $(go list ./tests/integration/...); do echo "running coordrecovery" # simplified command to speed up coord init test since it is large. $TEST_CMD -tags dynamic,test -v -coverprofile=profile.out -covermode=atomic "$d" -caseTimeout=20m -timeout=30m + elif [[ $d == *"import"* ]]; then + go test -tags dynamic,test -v -coverprofile=profile.out -covermode=atomic "$d" -caseTimeout=20m -timeout=60m else $TEST_CMD -race -tags dynamic,test -v -coverpkg=./... -coverprofile=profile.out -covermode=atomic "$d" -caseTimeout=15m -timeout=30m fi diff --git a/tests/integration/compaction/l0_compaction_test.go b/tests/integration/compaction/l0_compaction_test.go index f495593539..26f864b193 100644 --- a/tests/integration/compaction/l0_compaction_test.go +++ b/tests/integration/compaction/l0_compaction_test.go @@ -230,7 +230,7 @@ func (s *CompactionSuite) TestL0Compaction() { // drop collection // status, err = c.Proxy.DropCollection(ctx, &milvuspb.DropCollectionRequest{ - // CollectionName: collectionName, + // CollectionName: collectionName, // }) // err = merr.CheckRPCCall(status, err) // s.NoError(err) diff --git a/tests/integration/import/dynamic_field_test.go b/tests/integration/import/dynamic_field_test.go index 14b1ab1c7d..f0d89dca48 100644 --- a/tests/integration/import/dynamic_field_test.go +++ b/tests/integration/import/dynamic_field_test.go @@ -47,7 +47,7 @@ func (s *BulkInsertSuite) testImportDynamicField() { ) c := s.Cluster - ctx, cancel := context.WithTimeout(c.GetContext(), 120*time.Second) + ctx, cancel := context.WithTimeout(c.GetContext(), 240*time.Second) defer cancel() collectionName := "TestBulkInsert_B_" + funcutil.GenRandomStr() diff --git a/tests/integration/import/import_test.go b/tests/integration/import/import_test.go index a979cd1186..8891428949 100644 --- a/tests/integration/import/import_test.go +++ b/tests/integration/import/import_test.go @@ -76,7 +76,7 @@ func (s *BulkInsertSuite) run() { ) c := s.Cluster - ctx, cancel := context.WithTimeout(c.GetContext(), 120*time.Second) + ctx, cancel := context.WithTimeout(c.GetContext(), 240*time.Second) defer cancel() collectionName := "TestBulkInsert" + funcutil.GenRandomStr() diff --git a/tests/integration/import/partition_key_test.go b/tests/integration/import/partition_key_test.go index 8ff58df999..73fc98e5ab 100644 --- a/tests/integration/import/partition_key_test.go +++ b/tests/integration/import/partition_key_test.go @@ -46,7 +46,7 @@ func (s *BulkInsertSuite) TestImportWithPartitionKey() { ) c := s.Cluster - ctx, cancel := context.WithTimeout(c.GetContext(), 120*time.Second) + ctx, cancel := context.WithTimeout(c.GetContext(), 240*time.Second) defer cancel() collectionName := "TestBulkInsert_WithPartitionKey_" + funcutil.GenRandomStr()