diff --git a/cmd/tools/migration/mmap/tool/main.go b/cmd/tools/migration/mmap/tool/main.go index 0c659e6185..db8e2a2fa8 100644 --- a/cmd/tools/migration/mmap/tool/main.go +++ b/cmd/tools/migration/mmap/tool/main.go @@ -134,7 +134,7 @@ func prepareRootCoordMeta(ctx context.Context, allocator tso.Allocator) rootcoor if ss, err = kvmetestore.NewSuffixSnapshot(metaKV, kvmetestore.SnapshotsSep, paramtable.Get().EtcdCfg.MetaRootPath.GetValue(), kvmetestore.SnapshotPrefix); err != nil { panic(err) } - catalog = &kvmetestore.Catalog{Txn: metaKV, Snapshot: ss} + catalog = kvmetestore.NewCatalog(metaKV, ss) case util.MetaStoreTypeTiKV: log.Ctx(ctx).Info("Using tikv as meta storage.") var metaKV kv.MetaKv @@ -148,7 +148,7 @@ func prepareRootCoordMeta(ctx context.Context, allocator tso.Allocator) rootcoor if ss, err = kvmetestore.NewSuffixSnapshot(metaKV, kvmetestore.SnapshotsSep, paramtable.Get().TiKVCfg.MetaRootPath.GetValue(), kvmetestore.SnapshotPrefix); err != nil { panic(err) } - catalog = &kvmetestore.Catalog{Txn: metaKV, Snapshot: ss} + catalog = kvmetestore.NewCatalog(metaKV, ss) default: panic(fmt.Sprintf("MetaStoreType %s not supported", paramtable.Get().MetaStoreCfg.MetaStoreType.GetValue())) } diff --git a/internal/datacoord/broker/coordinator_broker.go b/internal/datacoord/broker/coordinator_broker.go index 7f079be5f6..1ecf2699b9 100644 --- a/internal/datacoord/broker/coordinator_broker.go +++ b/internal/datacoord/broker/coordinator_broker.go @@ -27,6 +27,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -37,6 +38,7 @@ type Broker interface { DescribeCollectionInternal(ctx context.Context, collectionID int64) (*milvuspb.DescribeCollectionResponse, error) ShowPartitionsInternal(ctx context.Context, collectionID int64) ([]int64, error) ShowCollections(ctx context.Context, dbName string) (*milvuspb.ShowCollectionsResponse, error) + ShowCollectionIDs(ctx context.Context) (*rootcoordpb.ShowCollectionIDsResponse, error) ListDatabases(ctx context.Context) (*milvuspb.ListDatabasesResponse, error) HasCollection(ctx context.Context, collectionID int64) (bool, error) } @@ -116,6 +118,24 @@ func (b *coordinatorBroker) ShowCollections(ctx context.Context, dbName string) return resp, nil } +func (b *coordinatorBroker) ShowCollectionIDs(ctx context.Context) (*rootcoordpb.ShowCollectionIDsResponse, error) { + ctx, cancel := context.WithTimeout(ctx, paramtable.Get().QueryCoordCfg.BrokerTimeout.GetAsDuration(time.Millisecond)) + defer cancel() + resp, err := b.rootCoord.ShowCollectionIDs(ctx, &rootcoordpb.ShowCollectionIDsRequest{ + Base: commonpbutil.NewMsgBase( + commonpbutil.WithMsgType(commonpb.MsgType_ShowCollections), + ), + AllowUnavailable: true, + }) + + if err = merr.CheckRPCCall(resp, err); err != nil { + log.Ctx(ctx).Warn("ShowCollectionIDs failed", zap.Error(err)) + return nil, err + } + + return resp, nil +} + func (b *coordinatorBroker) ListDatabases(ctx context.Context) (*milvuspb.ListDatabasesResponse, error) { ctx, cancel := context.WithTimeout(ctx, paramtable.Get().QueryCoordCfg.BrokerTimeout.GetAsDuration(time.Millisecond)) defer cancel() diff --git a/internal/datacoord/broker/mock_coordinator_broker.go b/internal/datacoord/broker/mock_coordinator_broker.go index bdec77b28c..c820b946f5 100644 --- a/internal/datacoord/broker/mock_coordinator_broker.go +++ b/internal/datacoord/broker/mock_coordinator_broker.go @@ -7,6 +7,8 @@ import ( milvuspb "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" mock "github.com/stretchr/testify/mock" + + rootcoordpb "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" ) // MockBroker is an autogenerated mock type for the Broker type @@ -196,6 +198,64 @@ func (_c *MockBroker_ListDatabases_Call) RunAndReturn(run func(context.Context) return _c } +// ShowCollectionIDs provides a mock function with given fields: ctx +func (_m *MockBroker) ShowCollectionIDs(ctx context.Context) (*rootcoordpb.ShowCollectionIDsResponse, error) { + ret := _m.Called(ctx) + + if len(ret) == 0 { + panic("no return value specified for ShowCollectionIDs") + } + + var r0 *rootcoordpb.ShowCollectionIDsResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context) (*rootcoordpb.ShowCollectionIDsResponse, error)); ok { + return rf(ctx) + } + if rf, ok := ret.Get(0).(func(context.Context) *rootcoordpb.ShowCollectionIDsResponse); ok { + r0 = rf(ctx) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*rootcoordpb.ShowCollectionIDsResponse) + } + } + + if rf, ok := ret.Get(1).(func(context.Context) error); ok { + r1 = rf(ctx) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockBroker_ShowCollectionIDs_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ShowCollectionIDs' +type MockBroker_ShowCollectionIDs_Call struct { + *mock.Call +} + +// ShowCollectionIDs is a helper method to define mock.On call +// - ctx context.Context +func (_e *MockBroker_Expecter) ShowCollectionIDs(ctx interface{}) *MockBroker_ShowCollectionIDs_Call { + return &MockBroker_ShowCollectionIDs_Call{Call: _e.mock.On("ShowCollectionIDs", ctx)} +} + +func (_c *MockBroker_ShowCollectionIDs_Call) Run(run func(ctx context.Context)) *MockBroker_ShowCollectionIDs_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context)) + }) + return _c +} + +func (_c *MockBroker_ShowCollectionIDs_Call) Return(_a0 *rootcoordpb.ShowCollectionIDsResponse, _a1 error) *MockBroker_ShowCollectionIDs_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockBroker_ShowCollectionIDs_Call) RunAndReturn(run func(context.Context) (*rootcoordpb.ShowCollectionIDsResponse, error)) *MockBroker_ShowCollectionIDs_Call { + _c.Call.Return(run) + return _c +} + // ShowCollections provides a mock function with given fields: ctx, dbName func (_m *MockBroker) ShowCollections(ctx context.Context, dbName string) (*milvuspb.ShowCollectionsResponse, error) { ret := _m.Called(ctx, dbName) diff --git a/internal/datacoord/compaction_task_clustering_test.go b/internal/datacoord/compaction_task_clustering_test.go index c6f29c1c37..49c413a7cd 100644 --- a/internal/datacoord/compaction_task_clustering_test.go +++ b/internal/datacoord/compaction_task_clustering_test.go @@ -31,6 +31,7 @@ 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/datacoord/broker" "github.com/milvus-io/milvus/internal/datacoord/session" "github.com/milvus-io/milvus/internal/metastore/kv/datacoord" "github.com/milvus-io/milvus/internal/metastore/model" @@ -61,7 +62,9 @@ func (s *ClusteringCompactionTaskSuite) SetupTest() { ctx := context.Background() cm := storage.NewLocalChunkManager(storage.RootPath("")) catalog := datacoord.NewCatalog(NewMetaMemoryKV(), "", "") - meta, err := newMeta(ctx, catalog, cm) + broker := broker.NewMockBroker(s.T()) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + meta, err := newMeta(ctx, catalog, cm, broker) s.NoError(err) s.meta = meta diff --git a/internal/datacoord/garbage_collector_test.go b/internal/datacoord/garbage_collector_test.go index 2e30c13117..e27536405b 100644 --- a/internal/datacoord/garbage_collector_test.go +++ b/internal/datacoord/garbage_collector_test.go @@ -63,7 +63,7 @@ func Test_garbageCollector_basic(t *testing.T) { cli, _, _, _, _, err := initUtOSSEnv(bucketName, rootPath, 0) require.NoError(t, err) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) t.Run("normal gc", func(t *testing.T) { @@ -118,7 +118,7 @@ func Test_garbageCollector_scan(t *testing.T) { cli, inserts, stats, delta, others, err := initUtOSSEnv(bucketName, rootPath, 4) require.NoError(t, err) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) t.Run("key is reference", func(t *testing.T) { @@ -1602,7 +1602,7 @@ func (s *GarbageCollectorSuite) SetupTest() { s.cli, s.inserts, s.stats, s.delta, s.others, err = initUtOSSEnv(s.bucketName, s.rootPath, 4) s.Require().NoError(err) - s.meta, err = newMemoryMeta() + s.meta, err = newMemoryMeta(s.T()) s.Require().NoError(err) } diff --git a/internal/datacoord/import_checker_test.go b/internal/datacoord/import_checker_test.go index 32df31a2f2..f3d7428829 100644 --- a/internal/datacoord/import_checker_test.go +++ b/internal/datacoord/import_checker_test.go @@ -52,7 +52,6 @@ func (s *ImportCheckerSuite) SetupTest() { 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().ListSegments(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) @@ -68,10 +67,11 @@ func (s *ImportCheckerSuite) SetupTest() { s.NoError(err) s.imeta = imeta - meta, err := newMeta(context.TODO(), catalog, nil) - s.NoError(err) - broker := broker2.NewMockBroker(s.T()) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + + meta, err := newMeta(context.TODO(), catalog, nil, broker) + s.NoError(err) sjm := NewMockStatsJobManager(s.T()) diff --git a/internal/datacoord/import_scheduler_test.go b/internal/datacoord/import_scheduler_test.go index e52be9e63c..b5a928f305 100644 --- a/internal/datacoord/import_scheduler_test.go +++ b/internal/datacoord/import_scheduler_test.go @@ -28,6 +28,7 @@ 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/datacoord/broker" "github.com/milvus-io/milvus/internal/datacoord/session" "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/pkg/proto/datapb" @@ -56,7 +57,6 @@ func (s *ImportSchedulerSuite) SetupTest() { s.catalog.EXPECT().ListImportJobs(mock.Anything).Return(nil, nil) s.catalog.EXPECT().ListPreImportTasks(mock.Anything).Return(nil, nil) s.catalog.EXPECT().ListImportTasks(mock.Anything).Return(nil, nil) - s.catalog.EXPECT().ListSegments(mock.Anything).Return(nil, nil) s.catalog.EXPECT().ListChannelCheckpoint(mock.Anything).Return(nil, nil) s.catalog.EXPECT().ListIndexes(mock.Anything).Return(nil, nil) s.catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return(nil, nil) @@ -67,7 +67,9 @@ func (s *ImportSchedulerSuite) SetupTest() { s.cluster = NewMockCluster(s.T()) s.alloc = allocator.NewMockAllocator(s.T()) - s.meta, err = newMeta(context.TODO(), s.catalog, nil) + broker := broker.NewMockBroker(s.T()) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + s.meta, err = newMeta(context.TODO(), s.catalog, nil, broker) s.NoError(err) s.meta.AddCollection(&collectionInfo{ ID: s.collectionID, diff --git a/internal/datacoord/import_util_test.go b/internal/datacoord/import_util_test.go index 1d9729e0aa..6ee847adcc 100644 --- a/internal/datacoord/import_util_test.go +++ b/internal/datacoord/import_util_test.go @@ -32,6 +32,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" "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/metastore/mocks" mocks2 "github.com/milvus-io/milvus/internal/mocks" @@ -106,7 +107,6 @@ func TestImportUtil_NewImportTasks(t *testing.T) { alloc.EXPECT().AllocTimestamp(mock.Anything).Return(rand.Uint64(), nil) catalog := mocks.NewDataCoordCatalog(t) - catalog.EXPECT().ListSegments(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) @@ -116,7 +116,9 @@ func TestImportUtil_NewImportTasks(t *testing.T) { catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - meta, err := newMeta(context.TODO(), catalog, nil) + broker := broker.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) assert.NoError(t, err) tasks, err := NewImportTasks(fileGroups, job, alloc, meta) @@ -158,7 +160,6 @@ func TestImportUtil_AssembleRequest(t *testing.T) { } catalog := mocks.NewDataCoordCatalog(t) - catalog.EXPECT().ListSegments(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) @@ -175,7 +176,9 @@ func TestImportUtil_AssembleRequest(t *testing.T) { }) alloc.EXPECT().AllocTimestamp(mock.Anything).Return(800, nil) - meta, err := newMeta(context.TODO(), catalog, nil) + broker := broker.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) assert.NoError(t, err) segment := &SegmentInfo{ SegmentInfo: &datapb.SegmentInfo{ID: 5, IsImporting: true}, @@ -244,7 +247,6 @@ func TestImportUtil_CheckDiskQuota(t *testing.T) { catalog.EXPECT().SavePreImportTask(mock.Anything, mock.Anything).Return(nil) catalog.EXPECT().ListIndexes(mock.Anything).Return(nil, nil) catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return(nil, nil) - catalog.EXPECT().ListSegments(mock.Anything).Return(nil, nil) catalog.EXPECT().ListChannelCheckpoint(mock.Anything).Return(nil, nil) catalog.EXPECT().AddSegment(mock.Anything, mock.Anything).Return(nil) catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil) @@ -255,7 +257,9 @@ func TestImportUtil_CheckDiskQuota(t *testing.T) { imeta, err := NewImportMeta(context.TODO(), catalog) assert.NoError(t, err) - meta, err := newMeta(context.TODO(), catalog, nil) + broker := broker.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) assert.NoError(t, err) job := &importJob{ @@ -424,7 +428,6 @@ func TestImportUtil_GetImportProgress(t *testing.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().ListSegments(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) @@ -441,7 +444,9 @@ func TestImportUtil_GetImportProgress(t *testing.T) { imeta, err := NewImportMeta(context.TODO(), catalog) assert.NoError(t, err) - meta, err := newMeta(context.TODO(), catalog, nil) + broker := broker.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) assert.NoError(t, err) file1 := &internalpb.ImportFile{ diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index 560f3b793b..12df98eabf 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -42,13 +42,16 @@ import ( "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/proto/workerpb" + "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/lock" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/retry" "github.com/milvus-io/milvus/pkg/util/timerecord" "github.com/milvus-io/milvus/pkg/util/tsoutil" ) @@ -145,7 +148,7 @@ type dbInfo struct { } // NewMeta creates meta from provided `kv.TxnKV` -func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManager storage.ChunkManager) (*meta, error) { +func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManager storage.ChunkManager, broker broker.Broker) (*meta, error) { im, err := newIndexMeta(ctx, catalog) if err != nil { return nil, err @@ -183,7 +186,7 @@ func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManag compactionTaskMeta: ctm, statsTaskMeta: stm, } - err = mt.reloadFromKV() + err = mt.reloadFromKV(ctx, broker) if err != nil { return nil, err } @@ -191,39 +194,84 @@ func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManag } // reloadFromKV loads meta from KV storage -func (m *meta) reloadFromKV() error { +func (m *meta) reloadFromKV(ctx context.Context, broker broker.Broker) error { record := timerecord.NewTimeRecorder("datacoord") - segments, err := m.catalog.ListSegments(m.ctx) + + var ( + err error + resp *rootcoordpb.ShowCollectionIDsResponse + ) + // retry on un implemented for compatibility + retryErr := retry.Handle(ctx, func() (bool, error) { + resp, err = broker.ShowCollectionIDs(m.ctx) + if errors.Is(err, merr.ErrServiceUnimplemented) { + return true, err + } + return false, err + }) + if retryErr != nil { + return retryErr + } + log.Ctx(ctx).Info("datacoord show collections done", zap.Duration("dur", record.RecordSpan())) + + collectionIDs := make([]int64, 0, 4096) + for _, collections := range resp.GetDbCollections() { + collectionIDs = append(collectionIDs, collections.GetCollectionIDs()...) + } + + pool := conc.NewPool[any](paramtable.Get().MetaStoreCfg.ReadConcurrency.GetAsInt()) + futures := make([]*conc.Future[any], 0, len(collectionIDs)) + collectionSegments := make([][]*datapb.SegmentInfo, len(collectionIDs)) + for i, collectionID := range collectionIDs { + i := i + collectionID := collectionID + futures = append(futures, pool.Submit(func() (any, error) { + segments, err := m.catalog.ListSegments(m.ctx, collectionID) + if err != nil { + return nil, err + } + collectionSegments[i] = segments + return nil, nil + })) + } + err = conc.AwaitAll(futures...) if err != nil { return err } + + log.Ctx(ctx).Info("datacoord show segments done", zap.Duration("dur", record.RecordSpan())) + metrics.DataCoordNumCollections.WithLabelValues().Set(0) metrics.DataCoordNumSegments.Reset() numStoredRows := int64(0) - for _, segment := range segments { - // segments from catalog.ListSegments will not have logPath - m.segments.SetSegment(segment.ID, NewSegmentInfo(segment)) - metrics.DataCoordNumSegments.WithLabelValues(segment.GetState().String(), segment.GetLevel().String(), getSortStatus(segment.GetIsSorted())).Inc() - if segment.State == commonpb.SegmentState_Flushed { - numStoredRows += segment.NumOfRows + numSegments := 0 + for _, segments := range collectionSegments { + numSegments += len(segments) + for _, segment := range segments { + // segments from catalog.ListSegments will not have logPath + m.segments.SetSegment(segment.ID, NewSegmentInfo(segment)) + metrics.DataCoordNumSegments.WithLabelValues(segment.GetState().String(), segment.GetLevel().String(), getSortStatus(segment.GetIsSorted())).Inc() + if segment.State == commonpb.SegmentState_Flushed { + numStoredRows += segment.NumOfRows - insertFileNum := 0 - for _, fieldBinlog := range segment.GetBinlogs() { - insertFileNum += len(fieldBinlog.GetBinlogs()) - } - metrics.FlushedSegmentFileNum.WithLabelValues(metrics.InsertFileLabel).Observe(float64(insertFileNum)) + insertFileNum := 0 + for _, fieldBinlog := range segment.GetBinlogs() { + insertFileNum += len(fieldBinlog.GetBinlogs()) + } + metrics.FlushedSegmentFileNum.WithLabelValues(metrics.InsertFileLabel).Observe(float64(insertFileNum)) - statFileNum := 0 - for _, fieldBinlog := range segment.GetStatslogs() { - statFileNum += len(fieldBinlog.GetBinlogs()) - } - metrics.FlushedSegmentFileNum.WithLabelValues(metrics.StatFileLabel).Observe(float64(statFileNum)) + statFileNum := 0 + for _, fieldBinlog := range segment.GetStatslogs() { + statFileNum += len(fieldBinlog.GetBinlogs()) + } + metrics.FlushedSegmentFileNum.WithLabelValues(metrics.StatFileLabel).Observe(float64(statFileNum)) - deleteFileNum := 0 - for _, filedBinlog := range segment.GetDeltalogs() { - deleteFileNum += len(filedBinlog.GetBinlogs()) + deleteFileNum := 0 + for _, filedBinlog := range segment.GetDeltalogs() { + deleteFileNum += len(filedBinlog.GetBinlogs()) + } + metrics.FlushedSegmentFileNum.WithLabelValues(metrics.DeleteFileLabel).Observe(float64(deleteFileNum)) } - metrics.FlushedSegmentFileNum.WithLabelValues(metrics.DeleteFileLabel).Observe(float64(deleteFileNum)) } } @@ -240,7 +288,7 @@ func (m *meta) reloadFromKV() error { Set(float64(ts.Unix())) } - log.Info("DataCoord meta reloadFromKV done", zap.Duration("duration", record.ElapseSpan())) + log.Ctx(ctx).Info("DataCoord meta reloadFromKV done", zap.Int("numSegments", numSegments), zap.Duration("duration", record.ElapseSpan())) return nil } diff --git a/internal/datacoord/meta_test.go b/internal/datacoord/meta_test.go index 66413bd8c0..cb86048350 100644 --- a/internal/datacoord/meta_test.go +++ b/internal/datacoord/meta_test.go @@ -18,6 +18,7 @@ package datacoord import ( "context" + "math/rand" "sync/atomic" "testing" @@ -43,6 +44,7 @@ import ( "github.com/milvus-io/milvus/pkg/kv" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" @@ -70,9 +72,20 @@ func (suite *MetaReloadSuite) resetMock() { func (suite *MetaReloadSuite) TestReloadFromKV() { ctx, cancel := context.WithCancel(context.Background()) defer cancel() + suite.Run("ListSegments_fail", func() { defer suite.resetMock() - suite.catalog.EXPECT().ListSegments(mock.Anything).Return(nil, errors.New("mock")) + brk := broker.NewMockBroker(suite.T()) + brk.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{ + Status: merr.Success(), + DbCollections: []*rootcoordpb.DBCollections{ + { + DbName: "db_1", + CollectionIDs: []int64{100}, + }, + }, + }, nil) + suite.catalog.EXPECT().ListSegments(mock.Anything, mock.Anything).Return(nil, errors.New("mock")) suite.catalog.EXPECT().ListIndexes(mock.Anything).Return([]*model.Index{}, nil) suite.catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return([]*model.SegmentIndex{}, nil) suite.catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil) @@ -80,14 +93,15 @@ func (suite *MetaReloadSuite) TestReloadFromKV() { suite.catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) suite.catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - _, err := newMeta(ctx, suite.catalog, nil) + _, err := newMeta(ctx, suite.catalog, nil, brk) suite.Error(err) }) suite.Run("ListChannelCheckpoint_fail", func() { defer suite.resetMock() - - suite.catalog.EXPECT().ListSegments(mock.Anything).Return([]*datapb.SegmentInfo{}, nil) + brk := broker.NewMockBroker(suite.T()) + brk.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + suite.catalog.EXPECT().ListSegments(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil) suite.catalog.EXPECT().ListChannelCheckpoint(mock.Anything).Return(nil, errors.New("mock")) suite.catalog.EXPECT().ListIndexes(mock.Anything).Return([]*model.Index{}, nil) suite.catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return([]*model.SegmentIndex{}, nil) @@ -96,19 +110,29 @@ func (suite *MetaReloadSuite) TestReloadFromKV() { suite.catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) suite.catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - _, err := newMeta(ctx, suite.catalog, nil) + _, err := newMeta(ctx, suite.catalog, nil, brk) suite.Error(err) }) suite.Run("ok", func() { defer suite.resetMock() + brk := broker.NewMockBroker(suite.T()) + brk.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{ + Status: merr.Success(), + DbCollections: []*rootcoordpb.DBCollections{ + { + DbName: "db_1", + CollectionIDs: []int64{1}, + }, + }, + }, nil) suite.catalog.EXPECT().ListIndexes(mock.Anything).Return([]*model.Index{}, nil) suite.catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return([]*model.SegmentIndex{}, nil) suite.catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil) suite.catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) suite.catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) suite.catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) - suite.catalog.EXPECT().ListSegments(mock.Anything).Return([]*datapb.SegmentInfo{ + suite.catalog.EXPECT().ListSegments(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{ { ID: 1, CollectionID: 1, @@ -124,11 +148,56 @@ func (suite *MetaReloadSuite) TestReloadFromKV() { }, }, nil) - _, err := newMeta(ctx, suite.catalog, nil) + _, err := newMeta(ctx, suite.catalog, nil, brk) suite.NoError(err) suite.MetricsEqual(metrics.DataCoordNumSegments.WithLabelValues(metrics.FlushedSegmentLabel, datapb.SegmentLevel_Legacy.String(), "unsorted"), 1) }) + + suite.Run("test list segments", func() { + defer suite.resetMock() + brk := broker.NewMockBroker(suite.T()) + brk.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{ + Status: merr.Success(), + DbCollections: []*rootcoordpb.DBCollections{ + { + DbName: "db_1", + CollectionIDs: []int64{100, 101, 102}, + }, + { + DbName: "db_2", + CollectionIDs: []int64{200, 201, 202}, + }, + }, + }, nil) + + suite.catalog.EXPECT().ListIndexes(mock.Anything).Return([]*model.Index{}, nil) + suite.catalog.EXPECT().ListSegmentIndexes(mock.Anything).Return([]*model.SegmentIndex{}, nil) + suite.catalog.EXPECT().ListAnalyzeTasks(mock.Anything).Return(nil, nil) + suite.catalog.EXPECT().ListCompactionTask(mock.Anything).Return(nil, nil) + suite.catalog.EXPECT().ListPartitionStatsInfos(mock.Anything).Return(nil, nil) + suite.catalog.EXPECT().ListStatsTasks(mock.Anything).Return(nil, nil) + suite.catalog.EXPECT().ListChannelCheckpoint(mock.Anything).Return(nil, nil) + + suite.catalog.EXPECT().ListSegments(mock.Anything, mock.Anything).RunAndReturn( + func(ctx context.Context, collectionID int64) ([]*datapb.SegmentInfo, error) { + return []*datapb.SegmentInfo{ + { + ID: rand.Int63(), + CollectionID: collectionID, + State: commonpb.SegmentState_Flushed, + }, + }, nil + }) + + meta, err := newMeta(ctx, suite.catalog, nil, brk) + suite.NoError(err) + for _, collectionID := range []int64{100, 101, 102, 200, 201, 202} { + segments := meta.GetSegmentsOfCollection(ctx, collectionID) + suite.Len(segments, 1) + suite.Equal(collectionID, segments[0].GetCollectionID()) + } + }) } type MetaBasicSuite struct { @@ -150,7 +219,7 @@ func (suite *MetaBasicSuite) SetupTest() { suite.partIDs = []int64{100, 101} suite.channelName = "c1" - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(suite.T()) suite.Require().NoError(err) suite.meta = meta @@ -456,7 +525,7 @@ func TestMeta_Basic(t *testing.T) { const channelName = "c1" // mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) testSchema := newTestSchema() @@ -546,7 +615,9 @@ func TestMeta_Basic(t *testing.T) { metakv.EXPECT().WalkWithPrefix(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Maybe() metakv.EXPECT().LoadWithPrefix(mock.Anything, mock.Anything).Return(nil, nil, nil).Maybe() catalog := datacoord.NewCatalog(metakv, "", "") - meta, err := newMeta(context.TODO(), catalog, nil) + broker := broker.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) assert.NoError(t, err) err = meta.AddSegment(context.TODO(), NewSegmentInfo(&datapb.SegmentInfo{})) @@ -561,7 +632,7 @@ func TestMeta_Basic(t *testing.T) { metakv2.EXPECT().LoadWithPrefix(mock.Anything, mock.Anything).Return(nil, nil, nil).Maybe() metakv2.EXPECT().MultiSaveAndRemoveWithPrefix(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("failed")) catalog = datacoord.NewCatalog(metakv2, "", "") - meta, err = newMeta(context.TODO(), catalog, nil) + meta, err = newMeta(context.TODO(), catalog, nil, broker) assert.NoError(t, err) // nil, since no segment yet err = meta.DropSegment(context.TODO(), 0) @@ -574,7 +645,7 @@ func TestMeta_Basic(t *testing.T) { assert.Error(t, err) catalog = datacoord.NewCatalog(metakv, "", "") - meta, err = newMeta(context.TODO(), catalog, nil) + meta, err = newMeta(context.TODO(), catalog, nil, broker) assert.NoError(t, err) assert.NotNil(t, meta) }) @@ -694,7 +765,7 @@ func TestMeta_Basic(t *testing.T) { }) t.Run("Test AddAllocation", func(t *testing.T) { - meta, _ := newMemoryMeta() + meta, _ := newMemoryMeta(t) err := meta.AddAllocation(1, &Allocation{ SegmentID: 1, NumOfRows: 1, @@ -705,7 +776,7 @@ func TestMeta_Basic(t *testing.T) { } func TestGetUnFlushedSegments(t *testing.T) { - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) s1 := &datapb.SegmentInfo{ ID: 0, @@ -734,7 +805,7 @@ func TestGetUnFlushedSegments(t *testing.T) { func TestUpdateSegmentsInfo(t *testing.T) { t.Run("normal", func(t *testing.T) { - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) segment1 := NewSegmentInfo(&datapb.SegmentInfo{ @@ -785,7 +856,7 @@ func TestUpdateSegmentsInfo(t *testing.T) { }) t.Run("update compacted segment", func(t *testing.T) { - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) // segment not found @@ -811,7 +882,7 @@ func TestUpdateSegmentsInfo(t *testing.T) { assert.NoError(t, err) }) t.Run("update non-existed segment", func(t *testing.T) { - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) err = meta.UpdateSegmentsInfo( @@ -873,7 +944,7 @@ func TestUpdateSegmentsInfo(t *testing.T) { }) t.Run("update empty segment into flush", func(t *testing.T) { - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) meta.AddSegment(context.Background(), &SegmentInfo{SegmentInfo: &datapb.SegmentInfo{ID: 1, State: commonpb.SegmentState_Growing}}) err = meta.UpdateSegmentsInfo( @@ -885,7 +956,7 @@ func TestUpdateSegmentsInfo(t *testing.T) { }) t.Run("update checkpoints and start position of non existed segment", func(t *testing.T) { - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) segment1 := &SegmentInfo{SegmentInfo: &datapb.SegmentInfo{ID: 1, State: commonpb.SegmentState_Growing}} @@ -908,7 +979,9 @@ func TestUpdateSegmentsInfo(t *testing.T) { metakv.EXPECT().WalkWithPrefix(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil).Maybe() metakv.EXPECT().LoadWithPrefix(mock.Anything, mock.Anything).Return(nil, nil, nil).Maybe() catalog := datacoord.NewCatalog(metakv, "", "") - meta, err := newMeta(context.TODO(), catalog, nil) + broker := broker.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) assert.NoError(t, err) segmentInfo := &SegmentInfo{ @@ -1201,7 +1274,7 @@ func TestChannelCP(t *testing.T) { } t.Run("UpdateChannelCheckpoint", func(t *testing.T) { - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) // nil position @@ -1213,7 +1286,7 @@ func TestChannelCP(t *testing.T) { }) t.Run("UpdateChannelCheckpoints", func(t *testing.T) { - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) assert.Equal(t, 0, len(meta.channelCPs.checkpoints)) @@ -1229,7 +1302,7 @@ func TestChannelCP(t *testing.T) { }) t.Run("GetChannelCheckpoint", func(t *testing.T) { - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) position := meta.GetChannelCheckpoint(mockVChannel) @@ -1244,7 +1317,7 @@ func TestChannelCP(t *testing.T) { }) t.Run("DropChannelCheckpoint", func(t *testing.T) { - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) err = meta.DropChannelCheckpoint(mockVChannel) diff --git a/internal/datacoord/mock_test.go b/internal/datacoord/mock_test.go index 9630730b88..0ec9e78440 100644 --- a/internal/datacoord/mock_test.go +++ b/internal/datacoord/mock_test.go @@ -30,6 +30,7 @@ import ( "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/datacoord/allocator" + "github.com/milvus-io/milvus/internal/datacoord/broker" memkv "github.com/milvus-io/milvus/internal/kv/mem" "github.com/milvus-io/milvus/internal/metastore/kv/datacoord" "github.com/milvus-io/milvus/pkg/common" @@ -88,9 +89,11 @@ func (mm *metaMemoryKV) CompareVersionAndSwap(ctx context.Context, key string, v panic("implement me") } -func newMemoryMeta() (*meta, error) { +func newMemoryMeta(t *testing.T) (*meta, error) { catalog := datacoord.NewCatalog(NewMetaMemoryKV(), "", "") - return newMeta(context.TODO(), catalog, nil) + broker := broker.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + return newMeta(context.TODO(), catalog, nil, broker) } func newMockAllocator(t *testing.T) *allocator.MockAllocator { @@ -428,6 +431,12 @@ func (m *mockRootCoordClient) ShowCollections(ctx context.Context, req *milvuspb }, nil } +func (m *mockRootCoordClient) ShowCollectionIDs(ctx context.Context, req *rootcoordpb.ShowCollectionIDsRequest, opts ...grpc.CallOption) (*rootcoordpb.ShowCollectionIDsResponse, error) { + return &rootcoordpb.ShowCollectionIDsResponse{ + Status: merr.Success(), + }, nil +} + func (m *mockRootCoordClient) CreateDatabase(ctx context.Context, in *milvuspb.CreateDatabaseRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { panic("not implemented") // TODO: Implement } diff --git a/internal/datacoord/segment_manager_test.go b/internal/datacoord/segment_manager_test.go index 68374ab376..3c1f1eb32f 100644 --- a/internal/datacoord/segment_manager_test.go +++ b/internal/datacoord/segment_manager_test.go @@ -31,13 +31,16 @@ 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/datacoord/allocator" + "github.com/milvus-io/milvus/internal/datacoord/broker" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" mockkv "github.com/milvus-io/milvus/internal/kv/mocks" "github.com/milvus-io/milvus/internal/metastore/kv/datacoord" "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/pkg/proto/datapb" + "github.com/milvus-io/milvus/pkg/proto/rootcoordpb" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/lock" + "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -46,7 +49,7 @@ func TestManagerOptions(t *testing.T) { // ctx := context.Background() paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) segmentManager, _ := newSegmentManager(meta, mockAllocator) t.Run("test with alloc helper", func(t *testing.T) { @@ -107,7 +110,7 @@ func TestAllocSegment(t *testing.T) { paramtable.Init() Params.Save(Params.DataCoordCfg.AllocLatestExpireAttempt.Key, "1") mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) segmentManager, _ := newSegmentManager(meta, mockAllocator) @@ -195,17 +198,28 @@ func TestLastExpireReset(t *testing.T) { metaKV := etcdkv.NewEtcdKV(etcdCli, rootPath) metaKV.RemoveWithPrefix(ctx, "") catalog := datacoord.NewCatalog(metaKV, "", "") - meta, err := newMeta(context.TODO(), catalog, nil) + collID, err := mockAllocator.AllocID(ctx) + assert.Nil(t, err) + broker := broker.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(&rootcoordpb.ShowCollectionIDsResponse{ + Status: merr.Success(), + DbCollections: []*rootcoordpb.DBCollections{ + { + DbName: "db_1", + CollectionIDs: []int64{collID}, + }, + }, + }, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) assert.Nil(t, err) // add collection channelName := "c1" schema := newTestSchema() - collID, err := mockAllocator.AllocID(ctx) - assert.Nil(t, err) meta.AddCollection(&collectionInfo{ID: collID, Schema: schema}) initSegment := &SegmentInfo{ SegmentInfo: &datapb.SegmentInfo{ ID: 1, + CollectionID: collID, InsertChannel: "ch1", State: commonpb.SegmentState_Growing, }, @@ -247,7 +261,7 @@ func TestLastExpireReset(t *testing.T) { newMetaKV := etcdkv.NewEtcdKV(newEtcdCli, rootPath) defer newMetaKV.RemoveWithPrefix(ctx, "") newCatalog := datacoord.NewCatalog(newMetaKV, "", "") - restartedMeta, err := newMeta(context.TODO(), newCatalog, nil) + restartedMeta, err := newMeta(context.TODO(), newCatalog, nil, broker) restartedMeta.AddCollection(&collectionInfo{ID: collID, Schema: schema}) assert.Nil(t, err) newSegmentManager, _ := newSegmentManager(restartedMeta, mockAllocator) @@ -276,7 +290,7 @@ func TestLoadSegmentsFromMeta(t *testing.T) { ctx := context.Background() paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) schema := newTestSchema() @@ -334,7 +348,7 @@ func TestLoadSegmentsFromMeta(t *testing.T) { func TestSaveSegmentsToMeta(t *testing.T) { paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) schema := newTestSchema() @@ -356,7 +370,7 @@ func TestSaveSegmentsToMeta(t *testing.T) { func TestSaveSegmentsToMetaWithSpecificSegments(t *testing.T) { paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) schema := newTestSchema() @@ -378,7 +392,7 @@ func TestSaveSegmentsToMetaWithSpecificSegments(t *testing.T) { func TestDropSegment(t *testing.T) { paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) schema := newTestSchema() @@ -401,7 +415,7 @@ func TestDropSegment(t *testing.T) { func TestAllocRowsLargerThanOneSegment(t *testing.T) { paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) schema := newTestSchema() @@ -423,7 +437,7 @@ func TestAllocRowsLargerThanOneSegment(t *testing.T) { func TestExpireAllocation(t *testing.T) { paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) schema := newTestSchema() @@ -465,7 +479,7 @@ func TestGetFlushableSegments(t *testing.T) { t.Run("get flushable segments between small interval", func(t *testing.T) { paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) schema := newTestSchema() @@ -520,7 +534,7 @@ func TestTryToSealSegment(t *testing.T) { t.Run("normal seal with segment policies", func(t *testing.T) { paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) schema := newTestSchema() @@ -545,7 +559,7 @@ func TestTryToSealSegment(t *testing.T) { t.Run("normal seal with channel seal policies", func(t *testing.T) { paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) schema := newTestSchema() @@ -570,7 +584,7 @@ func TestTryToSealSegment(t *testing.T) { t.Run("normal seal with both segment & channel seal policy", func(t *testing.T) { paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) schema := newTestSchema() @@ -597,7 +611,7 @@ func TestTryToSealSegment(t *testing.T) { t.Run("test sealByMaxBinlogFileNumberPolicy", func(t *testing.T) { paramtable.Init() mockAllocator := newMockAllocator(t) - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) schema := newTestSchema() @@ -681,7 +695,9 @@ func TestTryToSealSegment(t *testing.T) { mockAllocator := newMockAllocator(t) memoryKV := NewMetaMemoryKV() catalog := datacoord.NewCatalog(memoryKV, "", "") - meta, err := newMeta(context.TODO(), catalog, nil) + broker := broker.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) assert.NoError(t, err) schema := newTestSchema() @@ -710,7 +726,9 @@ func TestTryToSealSegment(t *testing.T) { mockAllocator := newMockAllocator(t) memoryKV := NewMetaMemoryKV() catalog := datacoord.NewCatalog(memoryKV, "", "") - meta, err := newMeta(context.TODO(), catalog, nil) + broker := broker.NewMockBroker(t) + broker.EXPECT().ShowCollectionIDs(mock.Anything).Return(nil, nil) + meta, err := newMeta(context.TODO(), catalog, nil, broker) assert.NoError(t, err) schema := newTestSchema() diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index c2c523565c..24a1f4de64 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -48,6 +48,7 @@ import ( "github.com/milvus-io/milvus/internal/metastore/kv/datacoord" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/componentutil" "github.com/milvus-io/milvus/internal/util/dependency" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/internal/util/streamingutil" @@ -328,12 +329,20 @@ func (s *Server) Init() error { func (s *Server) initDataCoord() error { log := log.Ctx(s.ctx) - s.UpdateStateCode(commonpb.StateCode_Initializing) - var err error - if err = s.initRootCoordClient(); err != nil { + // wait for master init or healthy + log.Info("DataCoord try to wait for RootCoord ready") + if err := s.initRootCoordClient(); err != nil { return err } log.Info("init rootcoord client done") + err := componentutil.WaitForComponentHealthy(s.ctx, s.rootCoordClient, "RootCoord", 1000000, time.Millisecond*200) + if err != nil { + log.Error("DataCoord wait for RootCoord ready failed", zap.Error(err)) + return err + } + log.Info("DataCoord report RootCoord ready") + + s.UpdateStateCode(commonpb.StateCode_Initializing) s.broker = broker.NewCoordinatorBroker(s.rootCoordClient) s.allocator = allocator.NewRootCoordAllocator(s.rootCoordClient) @@ -657,7 +666,7 @@ func (s *Server) initMeta(chunkManager storage.ChunkManager) error { reloadEtcdFn := func() error { var err error catalog := datacoord.NewCatalog(s.kv, chunkManager.RootPath(), s.metaRootPath) - s.meta, err = newMeta(s.ctx, catalog, chunkManager) + s.meta, err = newMeta(s.ctx, catalog, chunkManager, s.broker) if err != nil { return err } diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index ec1fc9396b..4e4cde4967 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -2237,7 +2237,7 @@ func TestDataCoordServer_SetSegmentState(t *testing.T) { }) t.Run("dataCoord meta set state not exists", func(t *testing.T) { - meta, err := newMemoryMeta() + meta, err := newMemoryMeta(t) assert.NoError(t, err) svr := newTestServer(t, WithMeta(meta)) defer closeTestServer(t, svr) diff --git a/internal/distributed/rootcoord/client/client.go b/internal/distributed/rootcoord/client/client.go index 1d06f53ebc..97d839756d 100644 --- a/internal/distributed/rootcoord/client/client.go +++ b/internal/distributed/rootcoord/client/client.go @@ -232,6 +232,18 @@ func (c *Client) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectio }) } +// ShowCollectionIDs returns all collection IDs. +func (c *Client) ShowCollectionIDs(ctx context.Context, in *rootcoordpb.ShowCollectionIDsRequest, opts ...grpc.CallOption) (*rootcoordpb.ShowCollectionIDsResponse, error) { + in = typeutil.Clone(in) + commonpbutil.UpdateMsgBase( + in.GetBase(), + commonpbutil.FillMsgBaseFromClient(paramtable.GetNodeID(), commonpbutil.WithTargetID(c.grpcClient.GetNodeID())), + ) + return wrapGrpcCall(ctx, c, func(client rootcoordpb.RootCoordClient) (*rootcoordpb.ShowCollectionIDsResponse, error) { + return client.ShowCollectionIDs(ctx, in) + }) +} + func (c *Client) AlterCollection(ctx context.Context, request *milvuspb.AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { request = typeutil.Clone(request) commonpbutil.UpdateMsgBase( diff --git a/internal/distributed/rootcoord/client/client_test.go b/internal/distributed/rootcoord/client/client_test.go index b580b96761..299ddd03bd 100644 --- a/internal/distributed/rootcoord/client/client_test.go +++ b/internal/distributed/rootcoord/client/client_test.go @@ -104,6 +104,10 @@ func Test_NewClient(t *testing.T) { r, err := client.ShowCollections(ctx, nil) retCheck(retNotNil, r, err) } + { + r, err := client.ShowCollectionIDs(ctx, nil) + retCheck(retNotNil, r, err) + } { r, err := client.CreatePartition(ctx, nil) retCheck(retNotNil, r, err) @@ -350,6 +354,10 @@ func Test_NewClient(t *testing.T) { rTimeout, err := client.ShowCollections(shortCtx, nil) retCheck(rTimeout, err) } + { + rTimeout, err := client.ShowCollectionIDs(shortCtx, nil) + retCheck(rTimeout, err) + } { rTimeout, err := client.CreatePartition(shortCtx, nil) retCheck(rTimeout, err) diff --git a/internal/distributed/rootcoord/service.go b/internal/distributed/rootcoord/service.go index 948d52a9df..7fbf18e1cf 100644 --- a/internal/distributed/rootcoord/service.go +++ b/internal/distributed/rootcoord/service.go @@ -421,6 +421,11 @@ func (s *Server) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectio return s.rootCoord.ShowCollections(ctx, in) } +// ShowCollectionIDs returns all collection IDs. +func (s *Server) ShowCollectionIDs(ctx context.Context, in *rootcoordpb.ShowCollectionIDsRequest) (*rootcoordpb.ShowCollectionIDsResponse, error) { + return s.rootCoord.ShowCollectionIDs(ctx, in) +} + // CreatePartition creates a partition in a collection func (s *Server) CreatePartition(ctx context.Context, in *milvuspb.CreatePartitionRequest) (*commonpb.Status, error) { return s.rootCoord.CreatePartition(ctx, in) diff --git a/internal/metastore/catalog.go b/internal/metastore/catalog.go index 0238cf7455..a8ef2dcaab 100644 --- a/internal/metastore/catalog.go +++ b/internal/metastore/catalog.go @@ -120,7 +120,7 @@ type BinlogsIncrement struct { //go:generate mockery --name=DataCoordCatalog --with-expecter type DataCoordCatalog interface { - ListSegments(ctx context.Context) ([]*datapb.SegmentInfo, error) + ListSegments(ctx context.Context, collectionID int64) ([]*datapb.SegmentInfo, error) AddSegment(ctx context.Context, segment *datapb.SegmentInfo) error // TODO Remove this later, we should update flush segments info for each segment separately, so far we still need transaction AlterSegments(ctx context.Context, newSegments []*datapb.SegmentInfo, binlogs ...BinlogsIncrement) error @@ -186,7 +186,7 @@ type QueryCoordCatalog interface { SavePartition(ctx context.Context, info ...*querypb.PartitionLoadInfo) error SaveReplica(ctx context.Context, replicas ...*querypb.Replica) error GetCollections(ctx context.Context) ([]*querypb.CollectionLoadInfo, error) - GetPartitions(ctx context.Context) (map[int64][]*querypb.PartitionLoadInfo, error) + GetPartitions(ctx context.Context, collectionIDs []int64) (map[int64][]*querypb.PartitionLoadInfo, error) GetReplicas(ctx context.Context) ([]*querypb.Replica, error) ReleaseCollection(ctx context.Context, collection int64) error ReleasePartition(ctx context.Context, collection int64, partitions ...int64) error diff --git a/internal/metastore/kv/datacoord/kv_catalog.go b/internal/metastore/kv/datacoord/kv_catalog.go index 80c00ca134..5d4666d2ce 100644 --- a/internal/metastore/kv/datacoord/kv_catalog.go +++ b/internal/metastore/kv/datacoord/kv_catalog.go @@ -65,7 +65,7 @@ func NewCatalog(MetaKv kv.MetaKv, chunkManagerRootPath string, metaRootpath stri } } -func (kc *Catalog) ListSegments(ctx context.Context) ([]*datapb.SegmentInfo, error) { +func (kc *Catalog) ListSegments(ctx context.Context, collectionID int64) ([]*datapb.SegmentInfo, error) { group, _ := errgroup.WithContext(ctx) segments := make([]*datapb.SegmentInfo, 0) insertLogs := make(map[typeutil.UniqueID][]*datapb.FieldBinlog, 1) @@ -75,7 +75,7 @@ func (kc *Catalog) ListSegments(ctx context.Context) ([]*datapb.SegmentInfo, err executeFn := func(binlogType storage.BinlogType, result map[typeutil.UniqueID][]*datapb.FieldBinlog) { group.Go(func() error { - ret, err := kc.listBinlogs(ctx, binlogType) + ret, err := kc.listBinlogs(ctx, binlogType, collectionID) if err != nil { return err } @@ -91,7 +91,7 @@ func (kc *Catalog) ListSegments(ctx context.Context) ([]*datapb.SegmentInfo, err executeFn(storage.StatsBinlog, statsLogs) executeFn(storage.BM25Binlog, bm25Logs) group.Go(func() error { - ret, err := kc.listSegments(ctx) + ret, err := kc.listSegments(ctx, collectionID) if err != nil { return err } @@ -111,7 +111,7 @@ func (kc *Catalog) ListSegments(ctx context.Context) ([]*datapb.SegmentInfo, err return segments, nil } -func (kc *Catalog) listSegments(ctx context.Context) ([]*datapb.SegmentInfo, error) { +func (kc *Catalog) listSegments(ctx context.Context, collectionID int64) ([]*datapb.SegmentInfo, error) { segments := make([]*datapb.SegmentInfo, 0) applyFn := func(key []byte, value []byte) error { @@ -136,7 +136,7 @@ func (kc *Catalog) listSegments(ctx context.Context) ([]*datapb.SegmentInfo, err return nil } - err := kc.MetaKv.WalkWithPrefix(ctx, SegmentPrefix+"/", kc.paginationSize, applyFn) + err := kc.MetaKv.WalkWithPrefix(ctx, buildCollectionPrefix(collectionID), kc.paginationSize, applyFn) if err != nil { return nil, err } @@ -144,45 +144,34 @@ func (kc *Catalog) listSegments(ctx context.Context) ([]*datapb.SegmentInfo, err return segments, nil } -func (kc *Catalog) parseBinlogKey(key string, prefixIdx int) (int64, int64, int64, error) { - remainedKey := key[prefixIdx:] - keyWordGroup := strings.Split(remainedKey, "/") +func (kc *Catalog) parseBinlogKey(key string) (int64, error) { + // by-dev/meta/datacoord-meta/binlog/454086059555817418/454086059555817543/454329387504816753/1 + // ---------------------------------|collectionID |partitionID |segmentID |fieldID + keyWordGroup := strings.Split(key, "/") if len(keyWordGroup) < 3 { - return 0, 0, 0, fmt.Errorf("parse key: %s failed, trimmed key:%s", key, remainedKey) + return 0, fmt.Errorf("parse key: %s failed, key:%s", key, key) } - - collectionID, err := strconv.ParseInt(keyWordGroup[0], 10, 64) + segmentID, err := strconv.ParseInt(keyWordGroup[len(keyWordGroup)-2], 10, 64) if err != nil { - return 0, 0, 0, fmt.Errorf("parse key: %s failed, trimmed key:%s, %w", key, remainedKey, err) + return 0, fmt.Errorf("parse key failed, key:%s, %w", key, err) } - - partitionID, err := strconv.ParseInt(keyWordGroup[1], 10, 64) - if err != nil { - return 0, 0, 0, fmt.Errorf("parse key: %s failed, trimmed key:%s, %w", key, remainedKey, err) - } - - segmentID, err := strconv.ParseInt(keyWordGroup[2], 10, 64) - if err != nil { - return 0, 0, 0, fmt.Errorf("parse key: %s failed, trimmed key:%s, %w", key, remainedKey, err) - } - - return collectionID, partitionID, segmentID, nil + return segmentID, nil } -func (kc *Catalog) listBinlogs(ctx context.Context, binlogType storage.BinlogType) (map[typeutil.UniqueID][]*datapb.FieldBinlog, error) { +func (kc *Catalog) listBinlogs(ctx context.Context, binlogType storage.BinlogType, collectionID int64) (map[typeutil.UniqueID][]*datapb.FieldBinlog, error) { ret := make(map[typeutil.UniqueID][]*datapb.FieldBinlog) var err error var logPathPrefix string switch binlogType { case storage.InsertBinlog: - logPathPrefix = SegmentBinlogPathPrefix + logPathPrefix = fmt.Sprintf("%s/%d", SegmentBinlogPathPrefix, collectionID) case storage.DeleteBinlog: - logPathPrefix = SegmentDeltalogPathPrefix + logPathPrefix = fmt.Sprintf("%s/%d", SegmentDeltalogPathPrefix, collectionID) case storage.StatsBinlog: - logPathPrefix = SegmentStatslogPathPrefix + logPathPrefix = fmt.Sprintf("%s/%d", SegmentStatslogPathPrefix, collectionID) case storage.BM25Binlog: - logPathPrefix = SegmentBM25logPathPrefix + logPathPrefix = fmt.Sprintf("%s/%d", SegmentBM25logPathPrefix, collectionID) default: err = fmt.Errorf("invalid binlog type: %d", binlogType) } @@ -190,13 +179,6 @@ func (kc *Catalog) listBinlogs(ctx context.Context, binlogType storage.BinlogTyp return nil, err } - var prefixIdx int - if len(kc.metaRootpath) == 0 { - prefixIdx = len(logPathPrefix) + 1 - } else { - prefixIdx = len(kc.metaRootpath) + 1 + len(logPathPrefix) + 1 - } - applyFn := func(key []byte, value []byte) error { fieldBinlog := &datapb.FieldBinlog{} err := proto.Unmarshal(value, fieldBinlog) @@ -204,7 +186,7 @@ func (kc *Catalog) listBinlogs(ctx context.Context, binlogType storage.BinlogTyp return fmt.Errorf("failed to unmarshal datapb.FieldBinlog: %d, err:%w", fieldBinlog.FieldID, err) } - _, _, segmentID, err := kc.parseBinlogKey(string(key), prefixIdx) + segmentID, err := kc.parseBinlogKey(string(key)) if err != nil { return fmt.Errorf("prefix:%s, %w", path.Join(kc.metaRootpath, logPathPrefix), err) } diff --git a/internal/metastore/kv/datacoord/kv_catalog_test.go b/internal/metastore/kv/datacoord/kv_catalog_test.go index edf2349f48..b330a60ae0 100644 --- a/internal/metastore/kv/datacoord/kv_catalog_test.go +++ b/internal/metastore/kv/datacoord/kv_catalog_test.go @@ -44,7 +44,6 @@ import ( "github.com/milvus-io/milvus/pkg/proto/datapb" "github.com/milvus-io/milvus/pkg/proto/indexpb" "github.com/milvus-io/milvus/pkg/util/etcd" - "github.com/milvus-io/milvus/pkg/util/metautil" "github.com/milvus-io/milvus/pkg/util/paramtable" ) @@ -151,7 +150,7 @@ func Test_ListSegments(t *testing.T) { metakv.EXPECT().WalkWithPrefix(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(errors.New("error")) catalog := NewCatalog(metakv, rootPath, "") - ret, err := catalog.ListSegments(context.TODO()) + ret, err := catalog.ListSegments(context.TODO(), collectionID) assert.Nil(t, ret) assert.Error(t, err) }) @@ -198,7 +197,7 @@ func Test_ListSegments(t *testing.T) { }) catalog := NewCatalog(metakv, rootPath, "") - ret, err := catalog.ListSegments(context.TODO()) + ret, err := catalog.ListSegments(context.TODO(), collectionID) assert.NotNil(t, ret) assert.NoError(t, err) @@ -215,7 +214,7 @@ func Test_ListSegments(t *testing.T) { }) catalog := NewCatalog(metakv, rootPath, "") - ret, err := catalog.ListSegments(context.TODO()) + ret, err := catalog.ListSegments(context.TODO(), collectionID) assert.NotNil(t, ret) assert.NoError(t, err) assert.Zero(t, len(ret)) @@ -256,7 +255,7 @@ func Test_ListSegments(t *testing.T) { return errors.New("should not reach here") }) - ret, err := catalog.ListSegments(context.TODO()) + ret, err := catalog.ListSegments(context.TODO(), collectionID) assert.NotNil(t, ret) assert.NoError(t, err) @@ -745,44 +744,16 @@ func Test_ChannelExists_SaveError(t *testing.T) { func Test_parseBinlogKey(t *testing.T) { catalog := NewCatalog(nil, "", "") - t.Run("parse collection id fail", func(t *testing.T) { - ret1, ret2, ret3, err := catalog.parseBinlogKey("root/err/1/1/1", 5) - assert.Error(t, err) - assert.Equal(t, int64(0), ret1) - assert.Equal(t, int64(0), ret2) - assert.Equal(t, int64(0), ret3) - }) - - t.Run("parse partition id fail", func(t *testing.T) { - ret1, ret2, ret3, err := catalog.parseBinlogKey("root/1/err/1/1", 5) - assert.Error(t, err) - assert.Equal(t, int64(0), ret1) - assert.Equal(t, int64(0), ret2) - assert.Equal(t, int64(0), ret3) - }) - t.Run("parse segment id fail", func(t *testing.T) { - ret1, ret2, ret3, err := catalog.parseBinlogKey("root/1/1/err/1", 5) + segmentID, err := catalog.parseBinlogKey("root/1/1/err/1") assert.Error(t, err) - assert.Equal(t, int64(0), ret1) - assert.Equal(t, int64(0), ret2) - assert.Equal(t, int64(0), ret3) - }) - - t.Run("miss field", func(t *testing.T) { - ret1, ret2, ret3, err := catalog.parseBinlogKey("root/1/1/", 5) - assert.Error(t, err) - assert.Equal(t, int64(0), ret1) - assert.Equal(t, int64(0), ret2) - assert.Equal(t, int64(0), ret3) + assert.Equal(t, int64(0), segmentID) }) t.Run("test ok", func(t *testing.T) { - ret1, ret2, ret3, err := catalog.parseBinlogKey("root/1/1/1/1", 5) + segmentID, err := catalog.parseBinlogKey("root/1/1/1/1") assert.NoError(t, err) - assert.Equal(t, int64(1), ret1) - assert.Equal(t, int64(1), ret2) - assert.Equal(t, int64(1), ret3) + assert.Equal(t, int64(1), segmentID) }) } @@ -1193,7 +1164,7 @@ func BenchmarkCatalog_List1000Segments(b *testing.B) { b.ResetTimer() for i := 0; i < b.N; i++ { - segments, err := catalog.ListSegments(ctx) + segments, err := catalog.ListSegments(ctx, collectionID) assert.NoError(b, err) for _, s := range segments { assert.NotNil(b, s) @@ -1206,13 +1177,8 @@ func BenchmarkCatalog_List1000Segments(b *testing.B) { func generateSegments(ctx context.Context, catalog *Catalog, n int, rootPath string) { rand.Seed(time.Now().UnixNano()) - var collectionID int64 for i := 0; i < n; i++ { - if collectionID%25 == 0 { - collectionID = rand.Int63() - } - v := rand.Int63() segment := addSegment(rootPath, collectionID, v, v, v) err := catalog.AddSegment(ctx, segment) @@ -1229,7 +1195,7 @@ func addSegment(rootPath string, collectionID, partitionID, segmentID, fieldID i Binlogs: []*datapb.Binlog{ { EntriesNum: 10000, - LogPath: metautil.BuildInsertLogPath(rootPath, collectionID, partitionID, segmentID, fieldID, int64(rand.Int())), + LogID: int64(rand.Int()), }, }, }, @@ -1241,7 +1207,7 @@ func addSegment(rootPath string, collectionID, partitionID, segmentID, fieldID i Binlogs: []*datapb.Binlog{ { EntriesNum: 5, - LogPath: metautil.BuildDeltaLogPath(rootPath, collectionID, partitionID, segmentID, int64(rand.Int())), + LogID: int64(rand.Int()), }, }, }, @@ -1253,58 +1219,7 @@ func addSegment(rootPath string, collectionID, partitionID, segmentID, fieldID i Binlogs: []*datapb.Binlog{ { EntriesNum: 5, - LogPath: metautil.BuildStatsLogPath(rootPath, collectionID, partitionID, segmentID, fieldID, int64(rand.Int())), - }, - }, - }, - } - - return &datapb.SegmentInfo{ - ID: segmentID, - CollectionID: collectionID, - PartitionID: partitionID, - NumOfRows: 10000, - State: commonpb.SegmentState_Flushed, - Binlogs: binlogs, - Deltalogs: deltalogs, - Statslogs: statslogs, - } -} - -func getSegment(rootPath string, collectionID, partitionID, segmentID, fieldID int64, binlogNum int) *datapb.SegmentInfo { - binLogPaths := make([]*datapb.Binlog, binlogNum) - for i := 0; i < binlogNum; i++ { - binLogPaths[i] = &datapb.Binlog{ - EntriesNum: 10000, - LogPath: metautil.BuildInsertLogPath(rootPath, collectionID, partitionID, segmentID, fieldID, int64(i)), - } - } - binlogs = []*datapb.FieldBinlog{ - { - FieldID: fieldID, - Binlogs: binLogPaths, - }, - } - - deltalogs = []*datapb.FieldBinlog{ - { - FieldID: fieldID, - Binlogs: []*datapb.Binlog{ - { - EntriesNum: 5, - LogPath: metautil.BuildDeltaLogPath(rootPath, collectionID, partitionID, segmentID, int64(rand.Int())), - }, - }, - }, - } - - statslogs = []*datapb.FieldBinlog{ - { - FieldID: 1, - Binlogs: []*datapb.Binlog{ - { - EntriesNum: 5, - LogPath: metautil.BuildStatsLogPath(rootPath, collectionID, partitionID, segmentID, fieldID, int64(rand.Int())), + LogID: int64(rand.Int()), }, }, }, diff --git a/internal/metastore/kv/querycoord/kv_catalog.go b/internal/metastore/kv/querycoord/kv_catalog.go index ad963e38bc..82bab53e6d 100644 --- a/internal/metastore/kv/querycoord/kv_catalog.go +++ b/internal/metastore/kv/querycoord/kv_catalog.go @@ -17,6 +17,7 @@ import ( "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/proto/querypb" "github.com/milvus-io/milvus/pkg/util/compressor" + "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/paramtable" ) @@ -37,12 +38,16 @@ const ( type Catalog struct { cli kv.MetaKv paginationSize int + + pool *conc.Pool[any] } func NewCatalog(cli kv.MetaKv) Catalog { + ioPool := conc.NewPool[any](paramtable.Get().MetaStoreCfg.ReadConcurrency.GetAsInt()) return Catalog{ cli: cli, paginationSize: paramtable.Get().MetaStoreCfg.PaginationSize.GetAsInt(), + pool: ioPool, } } @@ -126,23 +131,40 @@ func (s Catalog) GetCollections(ctx context.Context) ([]*querypb.CollectionLoadI return ret, nil } -func (s Catalog) GetPartitions(ctx context.Context) (map[int64][]*querypb.PartitionLoadInfo, error) { - ret := make(map[int64][]*querypb.PartitionLoadInfo) - applyFn := func(key []byte, value []byte) error { - info := querypb.PartitionLoadInfo{} - if err := proto.Unmarshal(value, &info); err != nil { - return err - } - ret[info.GetCollectionID()] = append(ret[info.GetCollectionID()], &info) - return nil +func (s Catalog) GetPartitions(ctx context.Context, collectionIDs []int64) (map[int64][]*querypb.PartitionLoadInfo, error) { + collectionPartitions := make([][]*querypb.PartitionLoadInfo, len(collectionIDs)) + futures := make([]*conc.Future[any], 0, len(collectionIDs)) + for i, collectionID := range collectionIDs { + i := i + collectionID := collectionID + futures = append(futures, s.pool.Submit(func() (any, error) { + prefix := EncodePartitionLoadInfoPrefix(collectionID) + _, values, err := s.cli.LoadWithPrefix(ctx, prefix) + if err != nil { + return nil, err + } + ret := make([]*querypb.PartitionLoadInfo, 0, len(values)) + for _, v := range values { + info := querypb.PartitionLoadInfo{} + if err = proto.Unmarshal([]byte(v), &info); err != nil { + return nil, err + } + ret = append(ret, &info) + } + collectionPartitions[i] = ret + return nil, nil + })) } - - err := s.cli.WalkWithPrefix(ctx, PartitionLoadInfoPrefix, s.paginationSize, applyFn) + err := conc.AwaitAll(futures...) if err != nil { return nil, err } - return ret, nil + result := make(map[int64][]*querypb.PartitionLoadInfo, len(collectionIDs)) + for i, partitions := range collectionPartitions { + result[collectionIDs[i]] = partitions + } + return result, nil } func (s Catalog) GetReplicas(ctx context.Context) ([]*querypb.Replica, error) { @@ -335,6 +357,10 @@ func EncodePartitionLoadInfoKey(collection, partition int64) string { return fmt.Sprintf("%s/%d/%d", PartitionLoadInfoPrefix, collection, partition) } +func EncodePartitionLoadInfoPrefix(collection int64) string { + return fmt.Sprintf("%s/%d/", PartitionLoadInfoPrefix, collection) +} + func encodeReplicaKey(collection, replica int64) string { return fmt.Sprintf("%s/%d/%d", ReplicaPrefix, collection, replica) } diff --git a/internal/metastore/kv/querycoord/kv_catalog_test.go b/internal/metastore/kv/querycoord/kv_catalog_test.go index 5c2b86d791..e2a0f1e33e 100644 --- a/internal/metastore/kv/querycoord/kv_catalog_test.go +++ b/internal/metastore/kv/querycoord/kv_catalog_test.go @@ -6,6 +6,7 @@ import ( "testing" "github.com/cockroachdb/errors" + "github.com/samber/lo" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" @@ -99,7 +100,9 @@ func (suite *CatalogTestSuite) TestCollectionWithPartition() { suite.NoError(err) suite.Len(collections, 1) suite.Equal(int64(3), collections[0].GetCollectionID()) - partitions, err := suite.catalog.GetPartitions(ctx) + partitions, err := suite.catalog.GetPartitions(ctx, lo.Map(collections, func(collection *querypb.CollectionLoadInfo, _ int) int64 { + return collection.GetCollectionID() + })) suite.NoError(err) suite.Len(partitions, 1) suite.Len(partitions[int64(3)], 1) @@ -123,11 +126,56 @@ func (suite *CatalogTestSuite) TestPartition() { suite.catalog.ReleasePartition(ctx, 1) suite.catalog.ReleasePartition(ctx, 2) - partitions, err := suite.catalog.GetPartitions(ctx) + partitions, err := suite.catalog.GetPartitions(ctx, []int64{0}) suite.NoError(err) suite.Len(partitions, 1) } +func (suite *CatalogTestSuite) TestGetPartitions() { + ctx := context.Background() + suite.catalog.SaveCollection(ctx, &querypb.CollectionLoadInfo{ + CollectionID: 1, + }) + suite.catalog.SavePartition(ctx, &querypb.PartitionLoadInfo{ + CollectionID: 1, + PartitionID: 100, + }) + suite.catalog.SaveCollection(ctx, &querypb.CollectionLoadInfo{ + CollectionID: 2, + }) + suite.catalog.SavePartition(ctx, &querypb.PartitionLoadInfo{ + CollectionID: 2, + PartitionID: 200, + }) + suite.catalog.SaveCollection(ctx, &querypb.CollectionLoadInfo{ + CollectionID: 3, + }) + suite.catalog.SavePartition(ctx, &querypb.PartitionLoadInfo{ + CollectionID: 3, + PartitionID: 300, + }) + + partitions, err := suite.catalog.GetPartitions(ctx, []int64{1, 2, 3}) + suite.NoError(err) + suite.Len(partitions, 3) + suite.Len(partitions[int64(1)], 1) + suite.Len(partitions[int64(2)], 1) + suite.Len(partitions[int64(3)], 1) + partitions, err = suite.catalog.GetPartitions(ctx, []int64{2, 3}) + suite.NoError(err) + suite.Len(partitions, 2) + suite.Len(partitions[int64(2)], 1) + suite.Len(partitions[int64(3)], 1) + partitions, err = suite.catalog.GetPartitions(ctx, []int64{3}) + suite.NoError(err) + suite.Len(partitions, 1) + suite.Len(partitions[int64(3)], 1) + suite.Equal(int64(300), partitions[int64(3)][0].GetPartitionID()) + partitions, err = suite.catalog.GetPartitions(ctx, []int64{}) + suite.NoError(err) + suite.Len(partitions, 0) +} + func (suite *CatalogTestSuite) TestReleaseManyPartitions() { ctx := context.Background() partitionIDs := make([]int64, 0) @@ -141,9 +189,10 @@ func (suite *CatalogTestSuite) TestReleaseManyPartitions() { err := suite.catalog.ReleasePartition(ctx, 1, partitionIDs...) suite.NoError(err) - partitions, err := suite.catalog.GetPartitions(ctx) + partitions, err := suite.catalog.GetPartitions(ctx, []int64{1}) suite.NoError(err) - suite.Len(partitions, 0) + suite.Len(partitions, 1) + suite.Len(partitions[int64(1)], 0) } func (suite *CatalogTestSuite) TestReplica() { diff --git a/internal/metastore/kv/rootcoord/kv_catalog.go b/internal/metastore/kv/rootcoord/kv_catalog.go index 6063cdf9d2..81196da20f 100644 --- a/internal/metastore/kv/rootcoord/kv_catalog.go +++ b/internal/metastore/kv/rootcoord/kv_catalog.go @@ -23,10 +23,12 @@ import ( pb "github.com/milvus-io/milvus/pkg/proto/etcdpb" "github.com/milvus-io/milvus/pkg/proto/internalpb" "github.com/milvus-io/milvus/pkg/util" + "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/crypto" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" + "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -38,6 +40,13 @@ import ( type Catalog struct { Txn kv.TxnKV Snapshot kv.SnapShotKV + + pool *conc.Pool[any] +} + +func NewCatalog(metaKV kv.TxnKV, ss kv.SnapShotKV) metastore.RootCoordCatalog { + ioPool := conc.NewPool[any](paramtable.Get().MetaStoreCfg.ReadConcurrency.GetAsInt()) + return &Catalog{Txn: metaKV, Snapshot: ss, pool: ioPool} } func BuildCollectionKey(dbID typeutil.UniqueID, collectionID typeutil.UniqueID) string { @@ -498,7 +507,6 @@ func (kc *Catalog) appendPartitionAndFieldsInfo(ctx context.Context, collMeta *p return collection, nil } -// TODO: This function will be invoked many times if there are many databases, leading to significant overhead. func (kc *Catalog) batchAppendPartitionAndFieldsInfo(ctx context.Context, collMeta []*pb.CollectionInfo, ts typeutil.Timestamp, ) ([]*model.Collection, error) { @@ -798,27 +806,33 @@ func (kc *Catalog) ListCollections(ctx context.Context, dbID int64, ts typeutil. } start := time.Now() - colls := make([]*pb.CollectionInfo, 0, len(vals)) - for _, val := range vals { - collMeta := &pb.CollectionInfo{} - err := proto.Unmarshal([]byte(val), collMeta) - if err != nil { - log.Ctx(ctx).Warn("unmarshal collection info failed", zap.Error(err)) - continue - } - kc.fixDefaultDBIDConsistency(ctx, collMeta, ts) - colls = append(colls, collMeta) + colls := make([]*model.Collection, len(vals)) + futures := make([]*conc.Future[any], 0, len(vals)) + for i, val := range vals { + i := i + val := val + futures = append(futures, kc.pool.Submit(func() (any, error) { + collMeta := &pb.CollectionInfo{} + err := proto.Unmarshal([]byte(val), collMeta) + if err != nil { + log.Ctx(ctx).Warn("unmarshal collection info failed", zap.Error(err)) + return nil, err + } + kc.fixDefaultDBIDConsistency(ctx, collMeta, ts) + collection, err := kc.appendPartitionAndFieldsInfo(ctx, collMeta, ts) + if err != nil { + return nil, err + } + colls[i] = collection + return nil, nil + })) } - log.Ctx(ctx).Info("unmarshal all collection details cost", zap.Int64("db", dbID), zap.Duration("cost", time.Since(start))) - - start = time.Now() - ret, err := kc.batchAppendPartitionAndFieldsInfo(ctx, colls, ts) - log.Ctx(ctx).Info("append partition and fields info cost", zap.Int64("db", dbID), zap.Duration("cost", time.Since(start))) + err = conc.AwaitAll(futures...) if err != nil { return nil, err } - - return ret, nil + log.Ctx(ctx).Info("unmarshal all collection details cost", zap.Int64("db", dbID), zap.Duration("cost", time.Since(start))) + return colls, nil } // fixDefaultDBIDConsistency fix dbID consistency for collectionInfo. @@ -826,12 +840,12 @@ func (kc *Catalog) ListCollections(ctx context.Context, dbID int64, ts typeutil. // all collections in default database should be marked with dbID 1. // this method also update dbid in meta store when dbid is 0 // see also: https://github.com/milvus-io/milvus/issues/33608 -func (kv *Catalog) fixDefaultDBIDConsistency(ctx context.Context, collMeta *pb.CollectionInfo, ts typeutil.Timestamp) { +func (kc *Catalog) fixDefaultDBIDConsistency(ctx context.Context, collMeta *pb.CollectionInfo, ts typeutil.Timestamp) { if collMeta.DbId == util.NonDBID { coll := model.UnmarshalCollectionModel(collMeta) cloned := coll.Clone() cloned.DBID = util.DefaultDBID - kv.alterModifyCollection(ctx, coll, cloned, ts) + kc.alterModifyCollection(ctx, coll, cloned, ts) collMeta.DbId = util.DefaultDBID } diff --git a/internal/metastore/kv/rootcoord/kv_catalog_test.go b/internal/metastore/kv/rootcoord/kv_catalog_test.go index 20a0815c9c..bd1aaa2e2c 100644 --- a/internal/metastore/kv/rootcoord/kv_catalog_test.go +++ b/internal/metastore/kv/rootcoord/kv_catalog_test.go @@ -105,7 +105,7 @@ func TestCatalog_ListCollections(t *testing.T) { kv.On("LoadWithPrefix", mock.Anything, CollectionMetaPrefix, ts). Return(nil, nil, targetErr) - kc := Catalog{Snapshot: kv} + kc := NewCatalog(nil, kv) ret, err := kc.ListCollections(ctx, util.NonDBID, ts) assert.ErrorIs(t, err, targetErr) assert.Nil(t, ret) @@ -119,12 +119,8 @@ func TestCatalog_ListCollections(t *testing.T) { assert.NoError(t, err) kv.On("LoadWithPrefix", mock.Anything, CollectionMetaPrefix, ts). Return([]string{"key"}, []string{string(bColl)}, nil) - kv.On("LoadWithPrefix", mock.Anything, mock.MatchedBy( - func(prefix string) bool { - return strings.HasPrefix(prefix, PartitionMetaPrefix) - }), ts). - Return(nil, nil, targetErr) - kc := Catalog{Snapshot: kv} + kv.EXPECT().LoadWithPrefix(mock.Anything, mock.Anything, mock.Anything).Return(nil, nil, targetErr) + kc := NewCatalog(nil, kv) ret, err := kc.ListCollections(ctx, util.NonDBID, ts) assert.ErrorIs(t, err, targetErr) @@ -155,7 +151,7 @@ func TestCatalog_ListCollections(t *testing.T) { return strings.HasPrefix(prefix, FieldMetaPrefix) }), ts). Return(nil, nil, targetErr) - kc := Catalog{Snapshot: kv} + kc := NewCatalog(nil, kv) ret, err := kc.ListCollections(ctx, util.NonDBID, ts) assert.ErrorIs(t, err, targetErr) @@ -171,7 +167,7 @@ func TestCatalog_ListCollections(t *testing.T) { kv.On("LoadWithPrefix", mock.Anything, CollectionMetaPrefix, ts). Return([]string{"key"}, []string{string(bColl)}, nil) kv.On("MultiSaveAndRemove", mock.Anything, mock.Anything, mock.Anything, ts).Return(nil) - kc := Catalog{Snapshot: kv} + kc := NewCatalog(nil, kv) ret, err := kc.ListCollections(ctx, util.NonDBID, ts) assert.NoError(t, err) @@ -218,7 +214,7 @@ func TestCatalog_ListCollections(t *testing.T) { }), ts). Return([]string{"rootcoord/functions/1/1"}, []string{string(fcm)}, nil) - kc := Catalog{Snapshot: kv} + kc := NewCatalog(nil, kv) ret, err := kc.ListCollections(ctx, testDb, ts) assert.NoError(t, err) assert.NotNil(t, ret) @@ -269,7 +265,7 @@ func TestCatalog_ListCollections(t *testing.T) { Return([]string{"rootcoord/functions/1/1"}, []string{string(fcm)}, nil) kv.On("MultiSaveAndRemove", mock.Anything, mock.Anything, mock.Anything, ts).Return(nil) - kc := Catalog{Snapshot: kv} + kc := NewCatalog(nil, kv) ret, err := kc.ListCollections(ctx, util.NonDBID, ts) assert.NoError(t, err) @@ -285,7 +281,7 @@ func TestCatalog_loadCollection(t *testing.T) { ctx := context.Background() kv := mocks.NewSnapShotKV(t) kv.EXPECT().Load(mock.Anything, mock.Anything, mock.Anything).Return("", errors.New("mock")) - kc := Catalog{Snapshot: kv} + kc := NewCatalog(nil, kv).(*Catalog) _, err := kc.loadCollection(ctx, testDb, 1, 0) assert.Error(t, err) }) @@ -294,7 +290,7 @@ func TestCatalog_loadCollection(t *testing.T) { ctx := context.Background() kv := mocks.NewSnapShotKV(t) kv.EXPECT().Load(mock.Anything, mock.Anything, mock.Anything).Return("not in pb format", nil) - kc := Catalog{Snapshot: kv} + kc := NewCatalog(nil, kv).(*Catalog) _, err := kc.loadCollection(ctx, testDb, 1, 0) assert.Error(t, err) }) @@ -306,7 +302,7 @@ func TestCatalog_loadCollection(t *testing.T) { assert.NoError(t, err) kv := mocks.NewSnapShotKV(t) kv.EXPECT().Load(mock.Anything, mock.Anything, mock.Anything).Return(string(value), nil) - kc := Catalog{Snapshot: kv} + kc := NewCatalog(nil, kv).(*Catalog) got, err := kc.loadCollection(ctx, util.DefaultDBID, 1, 0) assert.NoError(t, err) assert.Equal(t, got.GetID(), coll.GetID()) @@ -324,7 +320,7 @@ func TestCatalog_loadCollection(t *testing.T) { kv := mocks.NewSnapShotKV(t) kv.EXPECT().Load(mock.Anything, mock.Anything, mock.Anything).Return(string(value), nil) kv.EXPECT().MultiSaveAndRemove(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) - kc := Catalog{Snapshot: kv} + kc := NewCatalog(nil, kv).(*Catalog) got, err := kc.loadCollection(ctx, util.NonDBID, 1, 0) assert.NoError(t, err) assert.Equal(t, got.GetID(), coll.GetID()) @@ -378,7 +374,7 @@ func Test_partitionExistByName(t *testing.T) { func TestCatalog_GetCollectionByID(t *testing.T) { ctx := context.TODO() ss := mocks.NewSnapShotKV(t) - c := Catalog{Snapshot: ss} + c := NewCatalog(nil, ss) ss.EXPECT().Load(mock.Anything, mock.Anything, mock.Anything).Return("", errors.New("load error")).Twice() coll, err := c.GetCollectionByID(ctx, 0, 1, 1) @@ -415,7 +411,7 @@ func TestCatalog_CreatePartitionV2(t *testing.T) { snapshot.LoadFunc = func(ctx context.Context, key string, ts typeutil.Timestamp) (string, error) { return "", errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err := kc.CreatePartition(ctx, 0, &model.Partition{}, 0) assert.Error(t, err) }) @@ -437,7 +433,7 @@ func TestCatalog_CreatePartitionV2(t *testing.T) { return errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err = kc.CreatePartition(ctx, 0, &model.Partition{}, 0) assert.Error(t, err) @@ -462,7 +458,7 @@ func TestCatalog_CreatePartitionV2(t *testing.T) { return string(value), nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err = kc.CreatePartition(ctx, 0, &model.Partition{PartitionID: partID}, 0) assert.Error(t, err) @@ -481,7 +477,7 @@ func TestCatalog_CreatePartitionV2(t *testing.T) { return string(value), nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err = kc.CreatePartition(ctx, 0, &model.Partition{PartitionName: partition}, 0) assert.Error(t, err) @@ -507,7 +503,7 @@ func TestCatalog_CreatePartitionV2(t *testing.T) { return errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err = kc.CreatePartition(ctx, 0, &model.Partition{}, 0) assert.Error(t, err) @@ -528,7 +524,7 @@ func TestCatalog_CreateAliasV2(t *testing.T) { return errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err := kc.CreateAlias(ctx, &model.Alias{}, 0) assert.Error(t, err) @@ -549,7 +545,7 @@ func TestCatalog_listPartitionsAfter210(t *testing.T) { return nil, nil, errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) _, err := kc.listPartitionsAfter210(ctx, 1, 0) assert.Error(t, err) @@ -563,7 +559,7 @@ func TestCatalog_listPartitionsAfter210(t *testing.T) { return []string{"key"}, []string{"not in pb format"}, nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) _, err := kc.listPartitionsAfter210(ctx, 1, 0) assert.Error(t, err) @@ -581,7 +577,7 @@ func TestCatalog_listPartitionsAfter210(t *testing.T) { return []string{"key"}, []string{string(value)}, nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) got, err := kc.listPartitionsAfter210(ctx, 1, 0) assert.NoError(t, err) @@ -607,7 +603,7 @@ func TestCatalog_listFieldsAfter210(t *testing.T) { return nil, nil, errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) _, err := kc.listFieldsAfter210(ctx, 1, 0) assert.Error(t, err) @@ -621,7 +617,7 @@ func TestCatalog_listFieldsAfter210(t *testing.T) { return []string{"key"}, []string{"not in pb format"}, nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) _, err := kc.listFieldsAfter210(ctx, 1, 0) assert.Error(t, err) @@ -639,7 +635,7 @@ func TestCatalog_listFieldsAfter210(t *testing.T) { return []string{"key"}, []string{string(value)}, nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) got, err := kc.listFieldsAfter210(ctx, 1, 0) assert.NoError(t, err) @@ -656,7 +652,7 @@ func TestCatalog_AlterAliasV2(t *testing.T) { return errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err := kc.AlterAlias(ctx, &model.Alias{}, 0) assert.Error(t, err) @@ -705,7 +701,7 @@ func TestCatalog_DropPartitionV2(t *testing.T) { snapshot.On("Load", mock.Anything, mock.Anything, mock.Anything).Return("not in codec format", nil) - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err := kc.DropPartition(ctx, 0, 100, 101, 0) assert.Error(t, err) @@ -718,7 +714,7 @@ func TestCatalog_DropPartitionV2(t *testing.T) { snapshot.On("Load", mock.Anything, mock.Anything, mock.Anything).Return("", merr.WrapErrIoKeyNotFound("partition")) - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err := kc.DropPartition(ctx, 0, 100, 101, 0) assert.NoError(t, err) @@ -739,7 +735,7 @@ func TestCatalog_DropPartitionV2(t *testing.T) { return errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err = kc.DropPartition(ctx, 0, 100, 101, 0) assert.Error(t, err) @@ -771,7 +767,7 @@ func TestCatalog_DropPartitionV2(t *testing.T) { return errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err = kc.DropPartition(ctx, 0, 100, 101, 0) assert.Error(t, err) @@ -792,7 +788,7 @@ func TestCatalog_DropAliasV2(t *testing.T) { return errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) err := kc.DropAlias(ctx, testDb, "alias", 0) assert.Error(t, err) @@ -813,7 +809,7 @@ func TestCatalog_listAliasesBefore210(t *testing.T) { return nil, nil, errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) _, err := kc.listAliasesBefore210(ctx, 0) assert.Error(t, err) @@ -827,7 +823,7 @@ func TestCatalog_listAliasesBefore210(t *testing.T) { return []string{"key"}, []string{"not in pb format"}, nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) _, err := kc.listAliasesBefore210(ctx, 0) assert.Error(t, err) @@ -845,7 +841,7 @@ func TestCatalog_listAliasesBefore210(t *testing.T) { return []string{"key"}, []string{string(value)}, nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) got, err := kc.listAliasesBefore210(ctx, 0) assert.NoError(t, err) @@ -863,7 +859,7 @@ func TestCatalog_listAliasesAfter210(t *testing.T) { return nil, nil, errors.New("mock") } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) _, err := kc.listAliasesAfter210WithDb(ctx, testDb, 0) assert.Error(t, err) @@ -877,7 +873,7 @@ func TestCatalog_listAliasesAfter210(t *testing.T) { return []string{"key"}, []string{"not in pb format"}, nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) _, err := kc.listAliasesAfter210WithDb(ctx, testDb, 0) assert.Error(t, err) @@ -895,7 +891,7 @@ func TestCatalog_listAliasesAfter210(t *testing.T) { return []string{"key"}, []string{string(value)}, nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) got, err := kc.listAliasesAfter210WithDb(ctx, testDb, 0) assert.NoError(t, err) @@ -913,7 +909,7 @@ func TestCatalog_ListAliasesV2(t *testing.T) { return []string{"key"}, []string{"not in pb format"}, nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) _, err := kc.ListAliases(ctx, testDb, 0) assert.Error(t, err) @@ -938,7 +934,7 @@ func TestCatalog_ListAliasesV2(t *testing.T) { return []string{"key"}, []string{string(value)}, nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) _, err = kc.ListAliases(ctx, util.NonDBID, 0) assert.Error(t, err) @@ -963,7 +959,7 @@ func TestCatalog_ListAliasesV2(t *testing.T) { return []string{}, []string{}, nil } - kc := Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) got, err := kc.ListAliases(ctx, testDb, 0) assert.NoError(t, err) @@ -1020,14 +1016,14 @@ func Test_batchMultiSaveAndRemove(t *testing.T) { func TestCatalog_AlterCollection(t *testing.T) { t.Run("add", func(t *testing.T) { - kc := &Catalog{} + kc := NewCatalog(nil, nil) ctx := context.Background() err := kc.AlterCollection(ctx, nil, nil, metastore.ADD, 0) assert.Error(t, err) }) t.Run("delete", func(t *testing.T) { - kc := &Catalog{} + kc := NewCatalog(nil, nil) ctx := context.Background() err := kc.AlterCollection(ctx, nil, nil, metastore.DELETE, 0) assert.Error(t, err) @@ -1040,7 +1036,7 @@ func TestCatalog_AlterCollection(t *testing.T) { kvs[key] = value return nil } - kc := &Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) ctx := context.Background() var collectionID int64 = 1 oldC := &model.Collection{CollectionID: collectionID, State: pb.CollectionState_CollectionCreating} @@ -1058,7 +1054,7 @@ func TestCatalog_AlterCollection(t *testing.T) { }) t.Run("modify, tenant id changed", func(t *testing.T) { - kc := &Catalog{} + kc := NewCatalog(nil, nil) ctx := context.Background() var collectionID int64 = 1 oldC := &model.Collection{TenantID: "1", CollectionID: collectionID, State: pb.CollectionState_CollectionCreating} @@ -1077,7 +1073,7 @@ func TestCatalog_AlterCollection(t *testing.T) { return nil } - kc := &Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) ctx := context.Background() oldC := &model.Collection{DBID: 0, CollectionID: collectionID, State: pb.CollectionState_CollectionCreated} newC := &model.Collection{DBID: 1, CollectionID: collectionID, State: pb.CollectionState_CollectionCreated} @@ -1088,14 +1084,14 @@ func TestCatalog_AlterCollection(t *testing.T) { func TestCatalog_AlterPartition(t *testing.T) { t.Run("add", func(t *testing.T) { - kc := &Catalog{} + kc := NewCatalog(nil, nil) ctx := context.Background() err := kc.AlterPartition(ctx, testDb, nil, nil, metastore.ADD, 0) assert.Error(t, err) }) t.Run("delete", func(t *testing.T) { - kc := &Catalog{} + kc := NewCatalog(nil, nil) ctx := context.Background() err := kc.AlterPartition(ctx, testDb, nil, nil, metastore.DELETE, 0) assert.Error(t, err) @@ -1108,7 +1104,7 @@ func TestCatalog_AlterPartition(t *testing.T) { kvs[key] = value return nil } - kc := &Catalog{Snapshot: snapshot} + kc := NewCatalog(nil, snapshot).(*Catalog) ctx := context.Background() var collectionID int64 = 1 var partitionID int64 = 2 @@ -1127,7 +1123,7 @@ func TestCatalog_AlterPartition(t *testing.T) { }) t.Run("modify, tenant id changed", func(t *testing.T) { - kc := &Catalog{} + kc := NewCatalog(nil, nil) ctx := context.Background() var collectionID int64 = 1 oldP := &model.Partition{PartitionID: 1, CollectionID: collectionID, State: pb.PartitionState_PartitionCreating} @@ -1196,7 +1192,7 @@ func withMockMultiSaveAndRemove(err error) mockSnapshotOpt { func TestCatalog_CreateCollection(t *testing.T) { t.Run("collection not creating", func(t *testing.T) { - kc := &Catalog{} + kc := NewCatalog(nil, nil) ctx := context.Background() coll := &model.Collection{State: pb.CollectionState_CollectionDropping} err := kc.CreateCollection(ctx, coll, 100) @@ -1205,7 +1201,7 @@ func TestCatalog_CreateCollection(t *testing.T) { t.Run("failed to save collection", func(t *testing.T) { mockSnapshot := newMockSnapshot(t, withMockSave(errors.New("error mock Save"))) - kc := &Catalog{Snapshot: mockSnapshot} + kc := NewCatalog(nil, mockSnapshot) ctx := context.Background() coll := &model.Collection{State: pb.CollectionState_CollectionCreating} err := kc.CreateCollection(ctx, coll, 100) @@ -1214,7 +1210,7 @@ func TestCatalog_CreateCollection(t *testing.T) { t.Run("succeed to save collection but failed to save other keys", func(t *testing.T) { mockSnapshot := newMockSnapshot(t, withMockSave(nil), withMockMultiSave(errors.New("error mock MultiSave"))) - kc := &Catalog{Snapshot: mockSnapshot} + kc := NewCatalog(nil, mockSnapshot) ctx := context.Background() coll := &model.Collection{ Partitions: []*model.Partition{ @@ -1228,7 +1224,7 @@ func TestCatalog_CreateCollection(t *testing.T) { t.Run("normal case", func(t *testing.T) { mockSnapshot := newMockSnapshot(t, withMockSave(nil), withMockMultiSave(nil)) - kc := &Catalog{Snapshot: mockSnapshot} + kc := NewCatalog(nil, mockSnapshot) ctx := context.Background() coll := &model.Collection{ Partitions: []*model.Partition{ @@ -1242,7 +1238,7 @@ func TestCatalog_CreateCollection(t *testing.T) { t.Run("create collection with function", func(t *testing.T) { mockSnapshot := newMockSnapshot(t, withMockSave(nil), withMockMultiSave(nil)) - kc := &Catalog{Snapshot: mockSnapshot} + kc := NewCatalog(nil, mockSnapshot) ctx := context.Background() coll := &model.Collection{ Partitions: []*model.Partition{ @@ -1282,7 +1278,7 @@ func TestCatalog_CreateCollection(t *testing.T) { func TestCatalog_DropCollection(t *testing.T) { t.Run("failed to remove", func(t *testing.T) { mockSnapshot := newMockSnapshot(t, withMockMultiSaveAndRemove(errors.New("error mock MultiSaveAndRemove"))) - kc := &Catalog{Snapshot: mockSnapshot} + kc := NewCatalog(nil, mockSnapshot) ctx := context.Background() coll := &model.Collection{ Partitions: []*model.Partition{ @@ -1318,7 +1314,7 @@ func TestCatalog_DropCollection(t *testing.T) { removeCollectionCalled = true return errors.New("error mock MultiSaveAndRemove") }).Once() - kc := &Catalog{Snapshot: mockSnapshot} + kc := NewCatalog(nil, mockSnapshot) ctx := context.Background() coll := &model.Collection{ Partitions: []*model.Partition{ @@ -1334,7 +1330,7 @@ func TestCatalog_DropCollection(t *testing.T) { t.Run("normal case", func(t *testing.T) { mockSnapshot := newMockSnapshot(t, withMockMultiSaveAndRemove(nil)) - kc := &Catalog{Snapshot: mockSnapshot} + kc := NewCatalog(nil, mockSnapshot) ctx := context.Background() coll := &model.Collection{ Partitions: []*model.Partition{ @@ -1348,7 +1344,7 @@ func TestCatalog_DropCollection(t *testing.T) { t.Run("drop collection with function", func(t *testing.T) { mockSnapshot := newMockSnapshot(t, withMockMultiSaveAndRemove(nil)) - kc := &Catalog{Snapshot: mockSnapshot} + kc := NewCatalog(nil, mockSnapshot) ctx := context.Background() coll := &model.Collection{ Partitions: []*model.Partition{ @@ -1397,7 +1393,7 @@ func TestRBAC_Credential(t *testing.T) { t.Run("test GetCredential", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) loadFailName = "invalid" loadFailKey = fmt.Sprintf("%s/%s", CredentialPrefix, loadFailName) @@ -1448,7 +1444,7 @@ func TestRBAC_Credential(t *testing.T) { t.Run("test CreateCredential", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) invalidName = "invalid" ) @@ -1498,7 +1494,7 @@ func TestRBAC_Credential(t *testing.T) { t.Run("test DropCredential", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) validName = "user1" validUserRoleKeyPrefix = funcutil.HandleTenantForEtcdKey(RoleMappingPrefix, util.DefaultTenant, validName) @@ -1556,7 +1552,7 @@ func TestRBAC_Credential(t *testing.T) { t.Run("test ListCredentials", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) cmu sync.RWMutex count = 0 @@ -1638,7 +1634,7 @@ func TestRBAC_Role(t *testing.T) { t.Run("test remove", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil).(*Catalog) notExistKey = "not-exist" errorKey = "error" @@ -1682,7 +1678,7 @@ func TestRBAC_Role(t *testing.T) { t.Run("test save", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil).(*Catalog) notExistKey = "not-exist" errorKey = "error" @@ -1727,7 +1723,7 @@ func TestRBAC_Role(t *testing.T) { t.Run("test CreateRole", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) notExistName = "not-exist" notExistPath = funcutil.HandleTenantForEtcdKey(RolePrefix, tenant, notExistName) @@ -1770,7 +1766,7 @@ func TestRBAC_Role(t *testing.T) { t.Run("test DropRole", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) validName = "role1" errorName = "error" @@ -1823,7 +1819,7 @@ func TestRBAC_Role(t *testing.T) { t.Run("test AlterUserRole", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) user = "default-user" @@ -1886,7 +1882,7 @@ func TestRBAC_Role(t *testing.T) { t.Run("test entity!=nil", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) errorLoad = "error" errorLoadPath = funcutil.HandleTenantForEtcdKey(RolePrefix, tenant, errorLoad) @@ -1962,7 +1958,7 @@ func TestRBAC_Role(t *testing.T) { t.Run("test entity is nil", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) ) // Return valid keys if loadWithPrefixReturn == True @@ -2019,7 +2015,7 @@ func TestRBAC_Role(t *testing.T) { t.Run("test ListUser", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil).(*Catalog) invalidUser = "invalid-user" invalidUserKey = funcutil.HandleTenantForEtcdKey(RoleMappingPrefix, tenant, invalidUser) @@ -2161,7 +2157,7 @@ func TestRBAC_Role(t *testing.T) { var ( loadWithPrefixReturn atomic.Bool kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) ) // Return valid keys if loadWithPrefixReturn == True @@ -2237,7 +2233,7 @@ func TestRBAC_Grant(t *testing.T) { t.Run("test AlterGrant", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) ) validRoleKey := funcutil.HandleTenantForEtcdKey(GranteePrefix, tenant, fmt.Sprintf("%s/%s/%s", validRole, object, objName)) @@ -2423,7 +2419,7 @@ func TestRBAC_Grant(t *testing.T) { t.Run("test DeleteGrant", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) errorRole = "error-role" errorRolePrefix = funcutil.HandleTenantForEtcdKey(GranteePrefix, tenant, errorRole+"/") @@ -2463,7 +2459,7 @@ func TestRBAC_Grant(t *testing.T) { t.Run("test ListGrant", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) ) // Mock Load in kv_catalog.go:L901 @@ -2583,7 +2579,7 @@ func TestRBAC_Grant(t *testing.T) { t.Run("test ListPolicy", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) firstLoadWithPrefixReturn atomic.Bool secondLoadWithPrefixReturn atomic.Bool @@ -2701,7 +2697,7 @@ func TestRBAC_Backup(t *testing.T) { metaKV := etcdkv.NewEtcdKV(etcdCli, rootPath) defer metaKV.RemoveWithPrefix(context.TODO(), "") defer metaKV.Close() - c := &Catalog{Txn: metaKV} + c := NewCatalog(metaKV, nil) ctx := context.Background() c.CreateRole(ctx, util.DefaultTenant, &milvuspb.RoleEntity{Name: "role1"}) @@ -2754,7 +2750,7 @@ func TestRBAC_Restore(t *testing.T) { metaKV := etcdkv.NewEtcdKV(etcdCli, rootPath) defer metaKV.RemoveWithPrefix(context.TODO(), "") defer metaKV.Close() - c := &Catalog{Txn: metaKV} + c := NewCatalog(metaKV, nil) ctx := context.Background() @@ -2918,7 +2914,7 @@ func TestRBAC_PrivilegeGroup(t *testing.T) { t.Run("test GetPrivilegeGroup", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) ) kvmock.EXPECT().Load(mock.Anything, key1).Return(string(v1), nil) kvmock.EXPECT().Load(mock.Anything, key2).Return("", merr.ErrIoKeyNotFound) @@ -2948,7 +2944,7 @@ func TestRBAC_PrivilegeGroup(t *testing.T) { t.Run("test DropPrivilegeGroup", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) ) kvmock.EXPECT().Remove(mock.Anything, key1).Return(nil) @@ -2978,7 +2974,7 @@ func TestRBAC_PrivilegeGroup(t *testing.T) { t.Run("test SavePrivilegeGroup", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) ) kvmock.EXPECT().Save(mock.Anything, key1, mock.Anything).Return(nil) @@ -3008,7 +3004,7 @@ func TestRBAC_PrivilegeGroup(t *testing.T) { t.Run("test ListPrivilegeGroups", func(t *testing.T) { var ( kvmock = mocks.NewTxnKV(t) - c = &Catalog{Txn: kvmock} + c = NewCatalog(kvmock, nil) ) kvmock.EXPECT().LoadWithPrefix(mock.Anything, PrivilegeGroupPrefix).Return( @@ -3038,7 +3034,7 @@ func getPrivilegeNames(privileges []*milvuspb.PrivilegeEntity) []string { func TestCatalog_AlterDatabase(t *testing.T) { kvmock := mocks.NewSnapShotKV(t) - c := &Catalog{Snapshot: kvmock} + c := NewCatalog(nil, kvmock) db := model.NewDatabase(1, "db", pb.DatabaseState_DatabaseCreated, nil) kvmock.EXPECT().Save(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) @@ -3065,7 +3061,7 @@ func TestCatalog_AlterDatabase(t *testing.T) { func TestCatalog_listFunctionError(t *testing.T) { mockSnapshot := newMockSnapshot(t) - kc := &Catalog{Snapshot: mockSnapshot} + kc := NewCatalog(nil, mockSnapshot).(*Catalog) mockSnapshot.EXPECT().LoadWithPrefix(mock.Anything, mock.Anything, mock.Anything).Return(nil, nil, fmt.Errorf("mock error")) _, err := kc.listFunctions(context.TODO(), 1, 1) assert.Error(t, err) diff --git a/internal/metastore/mocks/mock_datacoord_catalog.go b/internal/metastore/mocks/mock_datacoord_catalog.go index cb92b36529..698c4ffcaa 100644 --- a/internal/metastore/mocks/mock_datacoord_catalog.go +++ b/internal/metastore/mocks/mock_datacoord_catalog.go @@ -1620,9 +1620,9 @@ func (_c *DataCoordCatalog_ListSegmentIndexes_Call) RunAndReturn(run func(contex return _c } -// ListSegments provides a mock function with given fields: ctx -func (_m *DataCoordCatalog) ListSegments(ctx context.Context) ([]*datapb.SegmentInfo, error) { - ret := _m.Called(ctx) +// ListSegments provides a mock function with given fields: ctx, collectionID +func (_m *DataCoordCatalog) ListSegments(ctx context.Context, collectionID int64) ([]*datapb.SegmentInfo, error) { + ret := _m.Called(ctx, collectionID) if len(ret) == 0 { panic("no return value specified for ListSegments") @@ -1630,19 +1630,19 @@ func (_m *DataCoordCatalog) ListSegments(ctx context.Context) ([]*datapb.Segment var r0 []*datapb.SegmentInfo var r1 error - if rf, ok := ret.Get(0).(func(context.Context) ([]*datapb.SegmentInfo, error)); ok { - return rf(ctx) + if rf, ok := ret.Get(0).(func(context.Context, int64) ([]*datapb.SegmentInfo, error)); ok { + return rf(ctx, collectionID) } - if rf, ok := ret.Get(0).(func(context.Context) []*datapb.SegmentInfo); ok { - r0 = rf(ctx) + if rf, ok := ret.Get(0).(func(context.Context, int64) []*datapb.SegmentInfo); ok { + r0 = rf(ctx, collectionID) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]*datapb.SegmentInfo) } } - if rf, ok := ret.Get(1).(func(context.Context) error); ok { - r1 = rf(ctx) + if rf, ok := ret.Get(1).(func(context.Context, int64) error); ok { + r1 = rf(ctx, collectionID) } else { r1 = ret.Error(1) } @@ -1657,13 +1657,14 @@ type DataCoordCatalog_ListSegments_Call struct { // ListSegments is a helper method to define mock.On call // - ctx context.Context -func (_e *DataCoordCatalog_Expecter) ListSegments(ctx interface{}) *DataCoordCatalog_ListSegments_Call { - return &DataCoordCatalog_ListSegments_Call{Call: _e.mock.On("ListSegments", ctx)} +// - collectionID int64 +func (_e *DataCoordCatalog_Expecter) ListSegments(ctx interface{}, collectionID interface{}) *DataCoordCatalog_ListSegments_Call { + return &DataCoordCatalog_ListSegments_Call{Call: _e.mock.On("ListSegments", ctx, collectionID)} } -func (_c *DataCoordCatalog_ListSegments_Call) Run(run func(ctx context.Context)) *DataCoordCatalog_ListSegments_Call { +func (_c *DataCoordCatalog_ListSegments_Call) Run(run func(ctx context.Context, collectionID int64)) *DataCoordCatalog_ListSegments_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context)) + run(args[0].(context.Context), args[1].(int64)) }) return _c } @@ -1673,7 +1674,7 @@ func (_c *DataCoordCatalog_ListSegments_Call) Return(_a0 []*datapb.SegmentInfo, return _c } -func (_c *DataCoordCatalog_ListSegments_Call) RunAndReturn(run func(context.Context) ([]*datapb.SegmentInfo, error)) *DataCoordCatalog_ListSegments_Call { +func (_c *DataCoordCatalog_ListSegments_Call) RunAndReturn(run func(context.Context, int64) ([]*datapb.SegmentInfo, error)) *DataCoordCatalog_ListSegments_Call { _c.Call.Return(run) return _c } diff --git a/internal/metastore/mocks/mock_querycoord_catalog.go b/internal/metastore/mocks/mock_querycoord_catalog.go index 70fca13b38..b4def3b48f 100644 --- a/internal/metastore/mocks/mock_querycoord_catalog.go +++ b/internal/metastore/mocks/mock_querycoord_catalog.go @@ -139,9 +139,9 @@ func (_c *QueryCoordCatalog_GetCollections_Call) RunAndReturn(run func(context.C return _c } -// GetPartitions provides a mock function with given fields: ctx -func (_m *QueryCoordCatalog) GetPartitions(ctx context.Context) (map[int64][]*querypb.PartitionLoadInfo, error) { - ret := _m.Called(ctx) +// GetPartitions provides a mock function with given fields: ctx, collectionIDs +func (_m *QueryCoordCatalog) GetPartitions(ctx context.Context, collectionIDs []int64) (map[int64][]*querypb.PartitionLoadInfo, error) { + ret := _m.Called(ctx, collectionIDs) if len(ret) == 0 { panic("no return value specified for GetPartitions") @@ -149,19 +149,19 @@ func (_m *QueryCoordCatalog) GetPartitions(ctx context.Context) (map[int64][]*qu var r0 map[int64][]*querypb.PartitionLoadInfo var r1 error - if rf, ok := ret.Get(0).(func(context.Context) (map[int64][]*querypb.PartitionLoadInfo, error)); ok { - return rf(ctx) + if rf, ok := ret.Get(0).(func(context.Context, []int64) (map[int64][]*querypb.PartitionLoadInfo, error)); ok { + return rf(ctx, collectionIDs) } - if rf, ok := ret.Get(0).(func(context.Context) map[int64][]*querypb.PartitionLoadInfo); ok { - r0 = rf(ctx) + if rf, ok := ret.Get(0).(func(context.Context, []int64) map[int64][]*querypb.PartitionLoadInfo); ok { + r0 = rf(ctx, collectionIDs) } else { if ret.Get(0) != nil { r0 = ret.Get(0).(map[int64][]*querypb.PartitionLoadInfo) } } - if rf, ok := ret.Get(1).(func(context.Context) error); ok { - r1 = rf(ctx) + if rf, ok := ret.Get(1).(func(context.Context, []int64) error); ok { + r1 = rf(ctx, collectionIDs) } else { r1 = ret.Error(1) } @@ -176,13 +176,14 @@ type QueryCoordCatalog_GetPartitions_Call struct { // GetPartitions is a helper method to define mock.On call // - ctx context.Context -func (_e *QueryCoordCatalog_Expecter) GetPartitions(ctx interface{}) *QueryCoordCatalog_GetPartitions_Call { - return &QueryCoordCatalog_GetPartitions_Call{Call: _e.mock.On("GetPartitions", ctx)} +// - collectionIDs []int64 +func (_e *QueryCoordCatalog_Expecter) GetPartitions(ctx interface{}, collectionIDs interface{}) *QueryCoordCatalog_GetPartitions_Call { + return &QueryCoordCatalog_GetPartitions_Call{Call: _e.mock.On("GetPartitions", ctx, collectionIDs)} } -func (_c *QueryCoordCatalog_GetPartitions_Call) Run(run func(ctx context.Context)) *QueryCoordCatalog_GetPartitions_Call { +func (_c *QueryCoordCatalog_GetPartitions_Call) Run(run func(ctx context.Context, collectionIDs []int64)) *QueryCoordCatalog_GetPartitions_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context)) + run(args[0].(context.Context), args[1].([]int64)) }) return _c } @@ -192,7 +193,7 @@ func (_c *QueryCoordCatalog_GetPartitions_Call) Return(_a0 map[int64][]*querypb. return _c } -func (_c *QueryCoordCatalog_GetPartitions_Call) RunAndReturn(run func(context.Context) (map[int64][]*querypb.PartitionLoadInfo, error)) *QueryCoordCatalog_GetPartitions_Call { +func (_c *QueryCoordCatalog_GetPartitions_Call) RunAndReturn(run func(context.Context, []int64) (map[int64][]*querypb.PartitionLoadInfo, error)) *QueryCoordCatalog_GetPartitions_Call { _c.Call.Return(run) return _c } diff --git a/internal/metastore/mocks/mock_rootcoord_catalog.go b/internal/metastore/mocks/mock_rootcoord_catalog.go index 8bc119e07b..424e844a34 100644 --- a/internal/metastore/mocks/mock_rootcoord_catalog.go +++ b/internal/metastore/mocks/mock_rootcoord_catalog.go @@ -1,4 +1,4 @@ -// Code generated by mockery v2.32.4. DO NOT EDIT. +// Code generated by mockery v2.46.0. DO NOT EDIT. package mocks @@ -30,6 +30,10 @@ func (_m *RootCoordCatalog) EXPECT() *RootCoordCatalog_Expecter { func (_m *RootCoordCatalog) AlterAlias(ctx context.Context, alias *model.Alias, ts uint64) error { ret := _m.Called(ctx, alias, ts) + if len(ret) == 0 { + panic("no return value specified for AlterAlias") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, *model.Alias, uint64) error); ok { r0 = rf(ctx, alias, ts) @@ -74,6 +78,10 @@ func (_c *RootCoordCatalog_AlterAlias_Call) RunAndReturn(run func(context.Contex func (_m *RootCoordCatalog) AlterCollection(ctx context.Context, oldColl *model.Collection, newColl *model.Collection, alterType metastore.AlterType, ts uint64) error { ret := _m.Called(ctx, oldColl, newColl, alterType, ts) + if len(ret) == 0 { + panic("no return value specified for AlterCollection") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, *model.Collection, *model.Collection, metastore.AlterType, uint64) error); ok { r0 = rf(ctx, oldColl, newColl, alterType, ts) @@ -120,6 +128,10 @@ func (_c *RootCoordCatalog_AlterCollection_Call) RunAndReturn(run func(context.C func (_m *RootCoordCatalog) AlterCredential(ctx context.Context, credential *model.Credential) error { ret := _m.Called(ctx, credential) + if len(ret) == 0 { + panic("no return value specified for AlterCredential") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, *model.Credential) error); ok { r0 = rf(ctx, credential) @@ -163,6 +175,10 @@ func (_c *RootCoordCatalog_AlterCredential_Call) RunAndReturn(run func(context.C func (_m *RootCoordCatalog) AlterDatabase(ctx context.Context, newDB *model.Database, ts uint64) error { ret := _m.Called(ctx, newDB, ts) + if len(ret) == 0 { + panic("no return value specified for AlterDatabase") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, *model.Database, uint64) error); ok { r0 = rf(ctx, newDB, ts) @@ -207,6 +223,10 @@ func (_c *RootCoordCatalog_AlterDatabase_Call) RunAndReturn(run func(context.Con func (_m *RootCoordCatalog) AlterGrant(ctx context.Context, tenant string, entity *milvuspb.GrantEntity, operateType milvuspb.OperatePrivilegeType) error { ret := _m.Called(ctx, tenant, entity, operateType) + if len(ret) == 0 { + panic("no return value specified for AlterGrant") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, string, *milvuspb.GrantEntity, milvuspb.OperatePrivilegeType) error); ok { r0 = rf(ctx, tenant, entity, operateType) @@ -252,6 +272,10 @@ func (_c *RootCoordCatalog_AlterGrant_Call) RunAndReturn(run func(context.Contex func (_m *RootCoordCatalog) AlterPartition(ctx context.Context, dbID int64, oldPart *model.Partition, newPart *model.Partition, alterType metastore.AlterType, ts uint64) error { ret := _m.Called(ctx, dbID, oldPart, newPart, alterType, ts) + if len(ret) == 0 { + panic("no return value specified for AlterPartition") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, int64, *model.Partition, *model.Partition, metastore.AlterType, uint64) error); ok { r0 = rf(ctx, dbID, oldPart, newPart, alterType, ts) @@ -299,6 +323,10 @@ func (_c *RootCoordCatalog_AlterPartition_Call) RunAndReturn(run func(context.Co func (_m *RootCoordCatalog) AlterUserRole(ctx context.Context, tenant string, userEntity *milvuspb.UserEntity, roleEntity *milvuspb.RoleEntity, operateType milvuspb.OperateUserRoleType) error { ret := _m.Called(ctx, tenant, userEntity, roleEntity, operateType) + if len(ret) == 0 { + panic("no return value specified for AlterUserRole") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, string, *milvuspb.UserEntity, *milvuspb.RoleEntity, milvuspb.OperateUserRoleType) error); ok { r0 = rf(ctx, tenant, userEntity, roleEntity, operateType) @@ -345,6 +373,10 @@ func (_c *RootCoordCatalog_AlterUserRole_Call) RunAndReturn(run func(context.Con func (_m *RootCoordCatalog) BackupRBAC(ctx context.Context, tenant string) (*milvuspb.RBACMeta, error) { ret := _m.Called(ctx, tenant) + if len(ret) == 0 { + panic("no return value specified for BackupRBAC") + } + var r0 *milvuspb.RBACMeta var r1 error if rf, ok := ret.Get(0).(func(context.Context, string) (*milvuspb.RBACMeta, error)); ok { @@ -432,6 +464,10 @@ func (_c *RootCoordCatalog_Close_Call) RunAndReturn(run func()) *RootCoordCatalo func (_m *RootCoordCatalog) CollectionExists(ctx context.Context, dbID int64, collectionID int64, ts uint64) bool { ret := _m.Called(ctx, dbID, collectionID, ts) + if len(ret) == 0 { + panic("no return value specified for CollectionExists") + } + var r0 bool if rf, ok := ret.Get(0).(func(context.Context, int64, int64, uint64) bool); ok { r0 = rf(ctx, dbID, collectionID, ts) @@ -477,6 +513,10 @@ func (_c *RootCoordCatalog_CollectionExists_Call) RunAndReturn(run func(context. func (_m *RootCoordCatalog) CreateAlias(ctx context.Context, alias *model.Alias, ts uint64) error { ret := _m.Called(ctx, alias, ts) + if len(ret) == 0 { + panic("no return value specified for CreateAlias") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, *model.Alias, uint64) error); ok { r0 = rf(ctx, alias, ts) @@ -521,6 +561,10 @@ func (_c *RootCoordCatalog_CreateAlias_Call) RunAndReturn(run func(context.Conte func (_m *RootCoordCatalog) CreateCollection(ctx context.Context, collectionInfo *model.Collection, ts uint64) error { ret := _m.Called(ctx, collectionInfo, ts) + if len(ret) == 0 { + panic("no return value specified for CreateCollection") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, *model.Collection, uint64) error); ok { r0 = rf(ctx, collectionInfo, ts) @@ -565,6 +609,10 @@ func (_c *RootCoordCatalog_CreateCollection_Call) RunAndReturn(run func(context. func (_m *RootCoordCatalog) CreateCredential(ctx context.Context, credential *model.Credential) error { ret := _m.Called(ctx, credential) + if len(ret) == 0 { + panic("no return value specified for CreateCredential") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, *model.Credential) error); ok { r0 = rf(ctx, credential) @@ -608,6 +656,10 @@ func (_c *RootCoordCatalog_CreateCredential_Call) RunAndReturn(run func(context. func (_m *RootCoordCatalog) CreateDatabase(ctx context.Context, db *model.Database, ts uint64) error { ret := _m.Called(ctx, db, ts) + if len(ret) == 0 { + panic("no return value specified for CreateDatabase") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, *model.Database, uint64) error); ok { r0 = rf(ctx, db, ts) @@ -652,6 +704,10 @@ func (_c *RootCoordCatalog_CreateDatabase_Call) RunAndReturn(run func(context.Co func (_m *RootCoordCatalog) CreatePartition(ctx context.Context, dbID int64, partition *model.Partition, ts uint64) error { ret := _m.Called(ctx, dbID, partition, ts) + if len(ret) == 0 { + panic("no return value specified for CreatePartition") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, int64, *model.Partition, uint64) error); ok { r0 = rf(ctx, dbID, partition, ts) @@ -697,6 +753,10 @@ func (_c *RootCoordCatalog_CreatePartition_Call) RunAndReturn(run func(context.C func (_m *RootCoordCatalog) CreateRole(ctx context.Context, tenant string, entity *milvuspb.RoleEntity) error { ret := _m.Called(ctx, tenant, entity) + if len(ret) == 0 { + panic("no return value specified for CreateRole") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, string, *milvuspb.RoleEntity) error); ok { r0 = rf(ctx, tenant, entity) @@ -741,6 +801,10 @@ func (_c *RootCoordCatalog_CreateRole_Call) RunAndReturn(run func(context.Contex func (_m *RootCoordCatalog) DeleteGrant(ctx context.Context, tenant string, role *milvuspb.RoleEntity) error { ret := _m.Called(ctx, tenant, role) + if len(ret) == 0 { + panic("no return value specified for DeleteGrant") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, string, *milvuspb.RoleEntity) error); ok { r0 = rf(ctx, tenant, role) @@ -785,6 +849,10 @@ func (_c *RootCoordCatalog_DeleteGrant_Call) RunAndReturn(run func(context.Conte func (_m *RootCoordCatalog) DropAlias(ctx context.Context, dbID int64, alias string, ts uint64) error { ret := _m.Called(ctx, dbID, alias, ts) + if len(ret) == 0 { + panic("no return value specified for DropAlias") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, int64, string, uint64) error); ok { r0 = rf(ctx, dbID, alias, ts) @@ -830,6 +898,10 @@ func (_c *RootCoordCatalog_DropAlias_Call) RunAndReturn(run func(context.Context func (_m *RootCoordCatalog) DropCollection(ctx context.Context, collectionInfo *model.Collection, ts uint64) error { ret := _m.Called(ctx, collectionInfo, ts) + if len(ret) == 0 { + panic("no return value specified for DropCollection") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, *model.Collection, uint64) error); ok { r0 = rf(ctx, collectionInfo, ts) @@ -874,6 +946,10 @@ func (_c *RootCoordCatalog_DropCollection_Call) RunAndReturn(run func(context.Co func (_m *RootCoordCatalog) DropCredential(ctx context.Context, username string) error { ret := _m.Called(ctx, username) + if len(ret) == 0 { + panic("no return value specified for DropCredential") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, string) error); ok { r0 = rf(ctx, username) @@ -917,6 +993,10 @@ func (_c *RootCoordCatalog_DropCredential_Call) RunAndReturn(run func(context.Co func (_m *RootCoordCatalog) DropDatabase(ctx context.Context, dbID int64, ts uint64) error { ret := _m.Called(ctx, dbID, ts) + if len(ret) == 0 { + panic("no return value specified for DropDatabase") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, int64, uint64) error); ok { r0 = rf(ctx, dbID, ts) @@ -961,6 +1041,10 @@ func (_c *RootCoordCatalog_DropDatabase_Call) RunAndReturn(run func(context.Cont func (_m *RootCoordCatalog) DropPartition(ctx context.Context, dbID int64, collectionID int64, partitionID int64, ts uint64) error { ret := _m.Called(ctx, dbID, collectionID, partitionID, ts) + if len(ret) == 0 { + panic("no return value specified for DropPartition") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, int64, int64, int64, uint64) error); ok { r0 = rf(ctx, dbID, collectionID, partitionID, ts) @@ -1003,10 +1087,61 @@ func (_c *RootCoordCatalog_DropPartition_Call) RunAndReturn(run func(context.Con return _c } +// DropPrivilegeGroup provides a mock function with given fields: ctx, groupName +func (_m *RootCoordCatalog) DropPrivilegeGroup(ctx context.Context, groupName string) error { + ret := _m.Called(ctx, groupName) + + if len(ret) == 0 { + panic("no return value specified for DropPrivilegeGroup") + } + + var r0 error + if rf, ok := ret.Get(0).(func(context.Context, string) error); ok { + r0 = rf(ctx, groupName) + } else { + r0 = ret.Error(0) + } + + return r0 +} + +// RootCoordCatalog_DropPrivilegeGroup_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'DropPrivilegeGroup' +type RootCoordCatalog_DropPrivilegeGroup_Call struct { + *mock.Call +} + +// DropPrivilegeGroup is a helper method to define mock.On call +// - ctx context.Context +// - groupName string +func (_e *RootCoordCatalog_Expecter) DropPrivilegeGroup(ctx interface{}, groupName interface{}) *RootCoordCatalog_DropPrivilegeGroup_Call { + return &RootCoordCatalog_DropPrivilegeGroup_Call{Call: _e.mock.On("DropPrivilegeGroup", ctx, groupName)} +} + +func (_c *RootCoordCatalog_DropPrivilegeGroup_Call) Run(run func(ctx context.Context, groupName string)) *RootCoordCatalog_DropPrivilegeGroup_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string)) + }) + return _c +} + +func (_c *RootCoordCatalog_DropPrivilegeGroup_Call) Return(_a0 error) *RootCoordCatalog_DropPrivilegeGroup_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *RootCoordCatalog_DropPrivilegeGroup_Call) RunAndReturn(run func(context.Context, string) error) *RootCoordCatalog_DropPrivilegeGroup_Call { + _c.Call.Return(run) + return _c +} + // DropRole provides a mock function with given fields: ctx, tenant, roleName func (_m *RootCoordCatalog) DropRole(ctx context.Context, tenant string, roleName string) error { ret := _m.Called(ctx, tenant, roleName) + if len(ret) == 0 { + panic("no return value specified for DropRole") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, string, string) error); ok { r0 = rf(ctx, tenant, roleName) @@ -1051,6 +1186,10 @@ func (_c *RootCoordCatalog_DropRole_Call) RunAndReturn(run func(context.Context, func (_m *RootCoordCatalog) GetCollectionByID(ctx context.Context, dbID int64, ts uint64, collectionID int64) (*model.Collection, error) { ret := _m.Called(ctx, dbID, ts, collectionID) + if len(ret) == 0 { + panic("no return value specified for GetCollectionByID") + } + var r0 *model.Collection var r1 error if rf, ok := ret.Get(0).(func(context.Context, int64, uint64, int64) (*model.Collection, error)); ok { @@ -1108,6 +1247,10 @@ func (_c *RootCoordCatalog_GetCollectionByID_Call) RunAndReturn(run func(context func (_m *RootCoordCatalog) GetCollectionByName(ctx context.Context, dbID int64, collectionName string, ts uint64) (*model.Collection, error) { ret := _m.Called(ctx, dbID, collectionName, ts) + if len(ret) == 0 { + panic("no return value specified for GetCollectionByName") + } + var r0 *model.Collection var r1 error if rf, ok := ret.Get(0).(func(context.Context, int64, string, uint64) (*model.Collection, error)); ok { @@ -1165,6 +1308,10 @@ func (_c *RootCoordCatalog_GetCollectionByName_Call) RunAndReturn(run func(conte func (_m *RootCoordCatalog) GetCredential(ctx context.Context, username string) (*model.Credential, error) { ret := _m.Called(ctx, username) + if len(ret) == 0 { + panic("no return value specified for GetCredential") + } + var r0 *model.Credential var r1 error if rf, ok := ret.Get(0).(func(context.Context, string) (*model.Credential, error)); ok { @@ -1216,10 +1363,73 @@ func (_c *RootCoordCatalog_GetCredential_Call) RunAndReturn(run func(context.Con return _c } +// GetPrivilegeGroup provides a mock function with given fields: ctx, groupName +func (_m *RootCoordCatalog) GetPrivilegeGroup(ctx context.Context, groupName string) (*milvuspb.PrivilegeGroupInfo, error) { + ret := _m.Called(ctx, groupName) + + if len(ret) == 0 { + panic("no return value specified for GetPrivilegeGroup") + } + + var r0 *milvuspb.PrivilegeGroupInfo + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, string) (*milvuspb.PrivilegeGroupInfo, error)); ok { + return rf(ctx, groupName) + } + if rf, ok := ret.Get(0).(func(context.Context, string) *milvuspb.PrivilegeGroupInfo); ok { + r0 = rf(ctx, groupName) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*milvuspb.PrivilegeGroupInfo) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, string) error); ok { + r1 = rf(ctx, groupName) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// RootCoordCatalog_GetPrivilegeGroup_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetPrivilegeGroup' +type RootCoordCatalog_GetPrivilegeGroup_Call struct { + *mock.Call +} + +// GetPrivilegeGroup is a helper method to define mock.On call +// - ctx context.Context +// - groupName string +func (_e *RootCoordCatalog_Expecter) GetPrivilegeGroup(ctx interface{}, groupName interface{}) *RootCoordCatalog_GetPrivilegeGroup_Call { + return &RootCoordCatalog_GetPrivilegeGroup_Call{Call: _e.mock.On("GetPrivilegeGroup", ctx, groupName)} +} + +func (_c *RootCoordCatalog_GetPrivilegeGroup_Call) Run(run func(ctx context.Context, groupName string)) *RootCoordCatalog_GetPrivilegeGroup_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(string)) + }) + return _c +} + +func (_c *RootCoordCatalog_GetPrivilegeGroup_Call) Return(_a0 *milvuspb.PrivilegeGroupInfo, _a1 error) *RootCoordCatalog_GetPrivilegeGroup_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *RootCoordCatalog_GetPrivilegeGroup_Call) RunAndReturn(run func(context.Context, string) (*milvuspb.PrivilegeGroupInfo, error)) *RootCoordCatalog_GetPrivilegeGroup_Call { + _c.Call.Return(run) + return _c +} + // ListAliases provides a mock function with given fields: ctx, dbID, ts func (_m *RootCoordCatalog) ListAliases(ctx context.Context, dbID int64, ts uint64) ([]*model.Alias, error) { ret := _m.Called(ctx, dbID, ts) + if len(ret) == 0 { + panic("no return value specified for ListAliases") + } + var r0 []*model.Alias var r1 error if rf, ok := ret.Get(0).(func(context.Context, int64, uint64) ([]*model.Alias, error)); ok { @@ -1276,6 +1486,10 @@ func (_c *RootCoordCatalog_ListAliases_Call) RunAndReturn(run func(context.Conte func (_m *RootCoordCatalog) ListCollections(ctx context.Context, dbID int64, ts uint64) ([]*model.Collection, error) { ret := _m.Called(ctx, dbID, ts) + if len(ret) == 0 { + panic("no return value specified for ListCollections") + } + var r0 []*model.Collection var r1 error if rf, ok := ret.Get(0).(func(context.Context, int64, uint64) ([]*model.Collection, error)); ok { @@ -1332,6 +1546,10 @@ func (_c *RootCoordCatalog_ListCollections_Call) RunAndReturn(run func(context.C func (_m *RootCoordCatalog) ListCredentials(ctx context.Context) ([]string, error) { ret := _m.Called(ctx) + if len(ret) == 0 { + panic("no return value specified for ListCredentials") + } + var r0 []string var r1 error if rf, ok := ret.Get(0).(func(context.Context) ([]string, error)); ok { @@ -1386,6 +1604,10 @@ func (_c *RootCoordCatalog_ListCredentials_Call) RunAndReturn(run func(context.C func (_m *RootCoordCatalog) ListCredentialsWithPasswd(ctx context.Context) (map[string]string, error) { ret := _m.Called(ctx) + if len(ret) == 0 { + panic("no return value specified for ListCredentialsWithPasswd") + } + var r0 map[string]string var r1 error if rf, ok := ret.Get(0).(func(context.Context) (map[string]string, error)); ok { @@ -1440,6 +1662,10 @@ func (_c *RootCoordCatalog_ListCredentialsWithPasswd_Call) RunAndReturn(run func func (_m *RootCoordCatalog) ListDatabases(ctx context.Context, ts uint64) ([]*model.Database, error) { ret := _m.Called(ctx, ts) + if len(ret) == 0 { + panic("no return value specified for ListDatabases") + } + var r0 []*model.Database var r1 error if rf, ok := ret.Get(0).(func(context.Context, uint64) ([]*model.Database, error)); ok { @@ -1495,6 +1721,10 @@ func (_c *RootCoordCatalog_ListDatabases_Call) RunAndReturn(run func(context.Con func (_m *RootCoordCatalog) ListGrant(ctx context.Context, tenant string, entity *milvuspb.GrantEntity) ([]*milvuspb.GrantEntity, error) { ret := _m.Called(ctx, tenant, entity) + if len(ret) == 0 { + panic("no return value specified for ListGrant") + } + var r0 []*milvuspb.GrantEntity var r1 error if rf, ok := ret.Get(0).(func(context.Context, string, *milvuspb.GrantEntity) ([]*milvuspb.GrantEntity, error)); ok { @@ -1551,6 +1781,10 @@ func (_c *RootCoordCatalog_ListGrant_Call) RunAndReturn(run func(context.Context func (_m *RootCoordCatalog) ListPolicy(ctx context.Context, tenant string) ([]*milvuspb.GrantEntity, error) { ret := _m.Called(ctx, tenant) + if len(ret) == 0 { + panic("no return value specified for ListPolicy") + } + var r0 []*milvuspb.GrantEntity var r1 error if rf, ok := ret.Get(0).(func(context.Context, string) ([]*milvuspb.GrantEntity, error)); ok { @@ -1602,10 +1836,72 @@ func (_c *RootCoordCatalog_ListPolicy_Call) RunAndReturn(run func(context.Contex return _c } +// ListPrivilegeGroups provides a mock function with given fields: ctx +func (_m *RootCoordCatalog) ListPrivilegeGroups(ctx context.Context) ([]*milvuspb.PrivilegeGroupInfo, error) { + ret := _m.Called(ctx) + + if len(ret) == 0 { + panic("no return value specified for ListPrivilegeGroups") + } + + var r0 []*milvuspb.PrivilegeGroupInfo + var r1 error + if rf, ok := ret.Get(0).(func(context.Context) ([]*milvuspb.PrivilegeGroupInfo, error)); ok { + return rf(ctx) + } + if rf, ok := ret.Get(0).(func(context.Context) []*milvuspb.PrivilegeGroupInfo); ok { + r0 = rf(ctx) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*milvuspb.PrivilegeGroupInfo) + } + } + + if rf, ok := ret.Get(1).(func(context.Context) error); ok { + r1 = rf(ctx) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// RootCoordCatalog_ListPrivilegeGroups_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ListPrivilegeGroups' +type RootCoordCatalog_ListPrivilegeGroups_Call struct { + *mock.Call +} + +// ListPrivilegeGroups is a helper method to define mock.On call +// - ctx context.Context +func (_e *RootCoordCatalog_Expecter) ListPrivilegeGroups(ctx interface{}) *RootCoordCatalog_ListPrivilegeGroups_Call { + return &RootCoordCatalog_ListPrivilegeGroups_Call{Call: _e.mock.On("ListPrivilegeGroups", ctx)} +} + +func (_c *RootCoordCatalog_ListPrivilegeGroups_Call) Run(run func(ctx context.Context)) *RootCoordCatalog_ListPrivilegeGroups_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context)) + }) + return _c +} + +func (_c *RootCoordCatalog_ListPrivilegeGroups_Call) Return(_a0 []*milvuspb.PrivilegeGroupInfo, _a1 error) *RootCoordCatalog_ListPrivilegeGroups_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *RootCoordCatalog_ListPrivilegeGroups_Call) RunAndReturn(run func(context.Context) ([]*milvuspb.PrivilegeGroupInfo, error)) *RootCoordCatalog_ListPrivilegeGroups_Call { + _c.Call.Return(run) + return _c +} + // ListRole provides a mock function with given fields: ctx, tenant, entity, includeUserInfo func (_m *RootCoordCatalog) ListRole(ctx context.Context, tenant string, entity *milvuspb.RoleEntity, includeUserInfo bool) ([]*milvuspb.RoleResult, error) { ret := _m.Called(ctx, tenant, entity, includeUserInfo) + if len(ret) == 0 { + panic("no return value specified for ListRole") + } + var r0 []*milvuspb.RoleResult var r1 error if rf, ok := ret.Get(0).(func(context.Context, string, *milvuspb.RoleEntity, bool) ([]*milvuspb.RoleResult, error)); ok { @@ -1663,6 +1959,10 @@ func (_c *RootCoordCatalog_ListRole_Call) RunAndReturn(run func(context.Context, func (_m *RootCoordCatalog) ListUser(ctx context.Context, tenant string, entity *milvuspb.UserEntity, includeRoleInfo bool) ([]*milvuspb.UserResult, error) { ret := _m.Called(ctx, tenant, entity, includeRoleInfo) + if len(ret) == 0 { + panic("no return value specified for ListUser") + } + var r0 []*milvuspb.UserResult var r1 error if rf, ok := ret.Get(0).(func(context.Context, string, *milvuspb.UserEntity, bool) ([]*milvuspb.UserResult, error)); ok { @@ -1720,6 +2020,10 @@ func (_c *RootCoordCatalog_ListUser_Call) RunAndReturn(run func(context.Context, func (_m *RootCoordCatalog) ListUserRole(ctx context.Context, tenant string) ([]string, error) { ret := _m.Called(ctx, tenant) + if len(ret) == 0 { + panic("no return value specified for ListUserRole") + } + var r0 []string var r1 error if rf, ok := ret.Get(0).(func(context.Context, string) ([]string, error)); ok { @@ -1775,6 +2079,10 @@ func (_c *RootCoordCatalog_ListUserRole_Call) RunAndReturn(run func(context.Cont func (_m *RootCoordCatalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvuspb.RBACMeta) error { ret := _m.Called(ctx, tenant, meta) + if len(ret) == 0 { + panic("no return value specified for RestoreRBAC") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, string, *milvuspb.RBACMeta) error); ok { r0 = rf(ctx, tenant, meta) @@ -1815,122 +2123,14 @@ func (_c *RootCoordCatalog_RestoreRBAC_Call) RunAndReturn(run func(context.Conte return _c } -// NewRootCoordCatalog creates a new instance of RootCoordCatalog. 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 NewRootCoordCatalog(t interface { - mock.TestingT - Cleanup(func()) -}) *RootCoordCatalog { - mock := &RootCoordCatalog{} - mock.Mock.Test(t) - - t.Cleanup(func() { mock.AssertExpectations(t) }) - - return mock -} - -// GetPrivilegeGroup provides a mock function with given fields: ctx, groupName -func (_m *RootCoordCatalog) GetPrivilegeGroup(ctx context.Context, groupName string) (*milvuspb.PrivilegeGroupInfo, error) { - ret := _m.Called(ctx, groupName) - - var r0 *milvuspb.PrivilegeGroupInfo - var r1 error - if rf, ok := ret.Get(0).(func(context.Context, string) (*milvuspb.PrivilegeGroupInfo, error)); ok { - return rf(ctx, groupName) - } - if rf, ok := ret.Get(0).(func(context.Context, string) *milvuspb.PrivilegeGroupInfo); ok { - r0 = rf(ctx, groupName) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).(*milvuspb.PrivilegeGroupInfo) - } - } - - if rf, ok := ret.Get(1).(func(context.Context, string) error); ok { - r1 = rf(ctx, groupName) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// RootCoordCatalog_GetPrivilegeGroup_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetPrivilegeGroup' -type RootCoordCatalog_GetPrivilegeGroup_Call struct { - *mock.Call -} - -// GetPrivilegeGroup is a helper method to define mock.On call -// - ctx context.Context -// - groupName string -func (_e *RootCoordCatalog_Expecter) GetPrivilegeGroup(ctx interface{}, groupName interface{}) *RootCoordCatalog_GetPrivilegeGroup_Call { - return &RootCoordCatalog_GetPrivilegeGroup_Call{Call: _e.mock.On("GetPrivilegeGroup", ctx, groupName)} -} - -func (_c *RootCoordCatalog_GetPrivilegeGroup_Call) Run(run func(ctx context.Context, groupName string)) *RootCoordCatalog_GetPrivilegeGroup_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(string)) - }) - return _c -} - -func (_c *RootCoordCatalog_GetPrivilegeGroup_Call) Return(_a0 *milvuspb.PrivilegeGroupInfo, _a1 error) *RootCoordCatalog_GetPrivilegeGroup_Call { - _c.Call.Return(_a0, _a1) - return _c -} - -func (_c *RootCoordCatalog_GetPrivilegeGroup_Call) RunAndReturn(run func(context.Context, string) (*milvuspb.PrivilegeGroupInfo, error)) *RootCoordCatalog_GetPrivilegeGroup_Call { - _c.Call.Return(run) - return _c -} - -// DropPrivilegeGroup provides a mock function with given fields: ctx, groupName, privileges -func (_m *RootCoordCatalog) DropPrivilegeGroup(ctx context.Context, groupName string) error { - ret := _m.Called(ctx, groupName) - - var r0 error - if rf, ok := ret.Get(0).(func(context.Context, string) error); ok { - r0 = rf(ctx, groupName) - } else { - r0 = ret.Error(0) - } - - return r0 -} - -// RootCoordCatalog_DropPrivilegeGroup_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'DropPrivilegeGroup' -type RootCoordCatalog_DropPrivilegeGroup_Call struct { - *mock.Call -} - -// DropPrivilegeGroup is a helper method to define mock.On call -// - ctx context.Context -// - groupName string -func (_e *RootCoordCatalog_Expecter) DropPrivilegeGroup(ctx interface{}, groupName interface{}) *RootCoordCatalog_DropPrivilegeGroup_Call { - return &RootCoordCatalog_DropPrivilegeGroup_Call{Call: _e.mock.On("DropPrivilegeGroup", ctx, groupName)} -} - -func (_c *RootCoordCatalog_DropPrivilegeGroup_Call) Run(run func(ctx context.Context, groupName string)) *RootCoordCatalog_DropPrivilegeGroup_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context), args[1].(string)) - }) - return _c -} - -func (_c *RootCoordCatalog_DropPrivilegeGroup_Call) Return(_a0 error) *RootCoordCatalog_DropPrivilegeGroup_Call { - _c.Call.Return(_a0) - return _c -} - -func (_c *RootCoordCatalog_DropPrivilegeGroup_Call) RunAndReturn(run func(context.Context, string) error) *RootCoordCatalog_DropPrivilegeGroup_Call { - _c.Call.Return(run) - return _c -} - -// SavePrivilegeGroup provides a mock function with given fields: ctx, groupName, privileges +// SavePrivilegeGroup provides a mock function with given fields: ctx, data func (_m *RootCoordCatalog) SavePrivilegeGroup(ctx context.Context, data *milvuspb.PrivilegeGroupInfo) error { ret := _m.Called(ctx, data) + if len(ret) == 0 { + panic("no return value specified for SavePrivilegeGroup") + } + var r0 error if rf, ok := ret.Get(0).(func(context.Context, *milvuspb.PrivilegeGroupInfo) error); ok { r0 = rf(ctx, data) @@ -1948,7 +2148,7 @@ type RootCoordCatalog_SavePrivilegeGroup_Call struct { // SavePrivilegeGroup is a helper method to define mock.On call // - ctx context.Context -// - groupName string +// - data *milvuspb.PrivilegeGroupInfo func (_e *RootCoordCatalog_Expecter) SavePrivilegeGroup(ctx interface{}, data interface{}) *RootCoordCatalog_SavePrivilegeGroup_Call { return &RootCoordCatalog_SavePrivilegeGroup_Call{Call: _e.mock.On("SavePrivilegeGroup", ctx, data)} } @@ -1970,56 +2170,16 @@ func (_c *RootCoordCatalog_SavePrivilegeGroup_Call) RunAndReturn(run func(contex return _c } -// ListPrivilegeGroups provides a mock function with given fields: ctx -func (_m *RootCoordCatalog) ListPrivilegeGroups(ctx context.Context) ([]*milvuspb.PrivilegeGroupInfo, error) { - ret := _m.Called(ctx) +// NewRootCoordCatalog creates a new instance of RootCoordCatalog. 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 NewRootCoordCatalog(t interface { + mock.TestingT + Cleanup(func()) +}) *RootCoordCatalog { + mock := &RootCoordCatalog{} + mock.Mock.Test(t) - var r0 []*milvuspb.PrivilegeGroupInfo - var r1 error - if rf, ok := ret.Get(0).(func(context.Context) ([]*milvuspb.PrivilegeGroupInfo, error)); ok { - return rf(ctx) - } - if rf, ok := ret.Get(0).(func(context.Context) []*milvuspb.PrivilegeGroupInfo); ok { - r0 = rf(ctx) - } else { - if ret.Get(0) != nil { - r0 = ret.Get(0).([]*milvuspb.PrivilegeGroupInfo) - } - } + t.Cleanup(func() { mock.AssertExpectations(t) }) - if rf, ok := ret.Get(1).(func(context.Context) error); ok { - r1 = rf(ctx) - } else { - r1 = ret.Error(1) - } - - return r0, r1 -} - -// RootCoordCatalog_ListPrivilegeGroups_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ListPrivilegeGroups' -type RootCoordCatalog_ListPrivilegeGroups_Call struct { - *mock.Call -} - -// ListPrivilegeGroups is a helper method to define mock.On call -// - ctx context.Context -func (_e *RootCoordCatalog_Expecter) ListPrivilegeGroups(ctx interface{}) *RootCoordCatalog_ListPrivilegeGroups_Call { - return &RootCoordCatalog_ListPrivilegeGroups_Call{Call: _e.mock.On("ListPrivilegeGroups", ctx)} -} - -func (_c *RootCoordCatalog_ListPrivilegeGroups_Call) Run(run func(ctx context.Context)) *RootCoordCatalog_ListPrivilegeGroups_Call { - _c.Call.Run(func(args mock.Arguments) { - run(args[0].(context.Context)) - }) - return _c -} - -func (_c *RootCoordCatalog_ListPrivilegeGroups_Call) Return(_a0 []*milvuspb.PrivilegeGroupInfo, _a1 error) *RootCoordCatalog_ListPrivilegeGroups_Call { - _c.Call.Return(_a0, _a1) - return _c -} - -func (_c *RootCoordCatalog_ListPrivilegeGroups_Call) RunAndReturn(run func(context.Context) ([]*milvuspb.PrivilegeGroupInfo, error)) *RootCoordCatalog_ListPrivilegeGroups_Call { - _c.Call.Return(run) - return _c + return mock } diff --git a/internal/mocks/mock_rootcoord.go b/internal/mocks/mock_rootcoord.go index 2993e494d2..06ffeebe6d 100644 --- a/internal/mocks/mock_rootcoord.go +++ b/internal/mocks/mock_rootcoord.go @@ -3217,6 +3217,65 @@ func (_c *RootCoord_SetTiKVClient_Call) RunAndReturn(run func(*txnkv.Client)) *R return _c } +// ShowCollectionIDs provides a mock function with given fields: _a0, _a1 +func (_m *RootCoord) ShowCollectionIDs(_a0 context.Context, _a1 *rootcoordpb.ShowCollectionIDsRequest) (*rootcoordpb.ShowCollectionIDsResponse, error) { + ret := _m.Called(_a0, _a1) + + if len(ret) == 0 { + panic("no return value specified for ShowCollectionIDs") + } + + var r0 *rootcoordpb.ShowCollectionIDsResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *rootcoordpb.ShowCollectionIDsRequest) (*rootcoordpb.ShowCollectionIDsResponse, error)); ok { + return rf(_a0, _a1) + } + if rf, ok := ret.Get(0).(func(context.Context, *rootcoordpb.ShowCollectionIDsRequest) *rootcoordpb.ShowCollectionIDsResponse); ok { + r0 = rf(_a0, _a1) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*rootcoordpb.ShowCollectionIDsResponse) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *rootcoordpb.ShowCollectionIDsRequest) error); ok { + r1 = rf(_a0, _a1) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// RootCoord_ShowCollectionIDs_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ShowCollectionIDs' +type RootCoord_ShowCollectionIDs_Call struct { + *mock.Call +} + +// ShowCollectionIDs is a helper method to define mock.On call +// - _a0 context.Context +// - _a1 *rootcoordpb.ShowCollectionIDsRequest +func (_e *RootCoord_Expecter) ShowCollectionIDs(_a0 interface{}, _a1 interface{}) *RootCoord_ShowCollectionIDs_Call { + return &RootCoord_ShowCollectionIDs_Call{Call: _e.mock.On("ShowCollectionIDs", _a0, _a1)} +} + +func (_c *RootCoord_ShowCollectionIDs_Call) Run(run func(_a0 context.Context, _a1 *rootcoordpb.ShowCollectionIDsRequest)) *RootCoord_ShowCollectionIDs_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].(*rootcoordpb.ShowCollectionIDsRequest)) + }) + return _c +} + +func (_c *RootCoord_ShowCollectionIDs_Call) Return(_a0 *rootcoordpb.ShowCollectionIDsResponse, _a1 error) *RootCoord_ShowCollectionIDs_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *RootCoord_ShowCollectionIDs_Call) RunAndReturn(run func(context.Context, *rootcoordpb.ShowCollectionIDsRequest) (*rootcoordpb.ShowCollectionIDsResponse, error)) *RootCoord_ShowCollectionIDs_Call { + _c.Call.Return(run) + return _c +} + // ShowCollections provides a mock function with given fields: _a0, _a1 func (_m *RootCoord) ShowCollections(_a0 context.Context, _a1 *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) { ret := _m.Called(_a0, _a1) diff --git a/internal/mocks/mock_rootcoord_client.go b/internal/mocks/mock_rootcoord_client.go index 8d88706ce2..99dbc65cfd 100644 --- a/internal/mocks/mock_rootcoord_client.go +++ b/internal/mocks/mock_rootcoord_client.go @@ -3630,6 +3630,80 @@ func (_c *MockRootCoordClient_SelectUser_Call) RunAndReturn(run func(context.Con return _c } +// ShowCollectionIDs provides a mock function with given fields: ctx, in, opts +func (_m *MockRootCoordClient) ShowCollectionIDs(ctx context.Context, in *rootcoordpb.ShowCollectionIDsRequest, opts ...grpc.CallOption) (*rootcoordpb.ShowCollectionIDsResponse, error) { + _va := make([]interface{}, len(opts)) + for _i := range opts { + _va[_i] = opts[_i] + } + var _ca []interface{} + _ca = append(_ca, ctx, in) + _ca = append(_ca, _va...) + ret := _m.Called(_ca...) + + if len(ret) == 0 { + panic("no return value specified for ShowCollectionIDs") + } + + var r0 *rootcoordpb.ShowCollectionIDsResponse + var r1 error + if rf, ok := ret.Get(0).(func(context.Context, *rootcoordpb.ShowCollectionIDsRequest, ...grpc.CallOption) (*rootcoordpb.ShowCollectionIDsResponse, error)); ok { + return rf(ctx, in, opts...) + } + if rf, ok := ret.Get(0).(func(context.Context, *rootcoordpb.ShowCollectionIDsRequest, ...grpc.CallOption) *rootcoordpb.ShowCollectionIDsResponse); ok { + r0 = rf(ctx, in, opts...) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*rootcoordpb.ShowCollectionIDsResponse) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, *rootcoordpb.ShowCollectionIDsRequest, ...grpc.CallOption) error); ok { + r1 = rf(ctx, in, opts...) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockRootCoordClient_ShowCollectionIDs_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ShowCollectionIDs' +type MockRootCoordClient_ShowCollectionIDs_Call struct { + *mock.Call +} + +// ShowCollectionIDs is a helper method to define mock.On call +// - ctx context.Context +// - in *rootcoordpb.ShowCollectionIDsRequest +// - opts ...grpc.CallOption +func (_e *MockRootCoordClient_Expecter) ShowCollectionIDs(ctx interface{}, in interface{}, opts ...interface{}) *MockRootCoordClient_ShowCollectionIDs_Call { + return &MockRootCoordClient_ShowCollectionIDs_Call{Call: _e.mock.On("ShowCollectionIDs", + append([]interface{}{ctx, in}, opts...)...)} +} + +func (_c *MockRootCoordClient_ShowCollectionIDs_Call) Run(run func(ctx context.Context, in *rootcoordpb.ShowCollectionIDsRequest, opts ...grpc.CallOption)) *MockRootCoordClient_ShowCollectionIDs_Call { + _c.Call.Run(func(args mock.Arguments) { + variadicArgs := make([]grpc.CallOption, len(args)-2) + for i, a := range args[2:] { + if a != nil { + variadicArgs[i] = a.(grpc.CallOption) + } + } + run(args[0].(context.Context), args[1].(*rootcoordpb.ShowCollectionIDsRequest), variadicArgs...) + }) + return _c +} + +func (_c *MockRootCoordClient_ShowCollectionIDs_Call) Return(_a0 *rootcoordpb.ShowCollectionIDsResponse, _a1 error) *MockRootCoordClient_ShowCollectionIDs_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockRootCoordClient_ShowCollectionIDs_Call) RunAndReturn(run func(context.Context, *rootcoordpb.ShowCollectionIDsRequest, ...grpc.CallOption) (*rootcoordpb.ShowCollectionIDsResponse, error)) *MockRootCoordClient_ShowCollectionIDs_Call { + _c.Call.Return(run) + return _c +} + // ShowCollections provides a mock function with given fields: ctx, in, opts func (_m *MockRootCoordClient) ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowCollectionsResponse, error) { _va := make([]interface{}, len(opts)) diff --git a/internal/proxy/rootcoord_mock_test.go b/internal/proxy/rootcoord_mock_test.go index 99c5ad45a5..8cdf98c80b 100644 --- a/internal/proxy/rootcoord_mock_test.go +++ b/internal/proxy/rootcoord_mock_test.go @@ -576,6 +576,10 @@ func (coord *RootCoordMock) ShowCollections(ctx context.Context, req *milvuspb.S }, nil } +func (coord *RootCoordMock) ShowCollectionIDs(ctx context.Context, req *rootcoordpb.ShowCollectionIDsRequest, opts ...grpc.CallOption) (*rootcoordpb.ShowCollectionIDsResponse, error) { + panic("implements me") +} + func (coord *RootCoordMock) CreatePartition(ctx context.Context, req *milvuspb.CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { code := coord.state.Load().(commonpb.StateCode) if code != commonpb.StateCode_Healthy { diff --git a/internal/querycoordv2/meta/collection_manager.go b/internal/querycoordv2/meta/collection_manager.go index 41e27b22bf..3337c1b25c 100644 --- a/internal/querycoordv2/meta/collection_manager.go +++ b/internal/querycoordv2/meta/collection_manager.go @@ -123,18 +123,24 @@ func NewCollectionManager(catalog metastore.QueryCoordCatalog) *CollectionManage // Recover recovers collections from kv store, // panics if failed func (m *CollectionManager) Recover(ctx context.Context, broker Broker) error { + start := time.Now() collections, err := m.catalog.GetCollections(ctx) if err != nil { return err } - partitions, err := m.catalog.GetPartitions(ctx) + log.Ctx(ctx).Info("recover collections from kv store", zap.Duration("dur", time.Since(start))) + + start = time.Now() + partitions, err := m.catalog.GetPartitions(ctx, lo.Map(collections, func(collection *querypb.CollectionLoadInfo, _ int) int64 { + return collection.GetCollectionID() + })) if err != nil { return err } ctx = log.WithTraceID(ctx, strconv.FormatInt(time.Now().UnixNano(), 10)) ctxLog := log.Ctx(ctx) - ctxLog.Info("recover collections and partitions from kv store") + ctxLog.Info("recover partitions from kv store", zap.Duration("dur", time.Since(start))) for _, collection := range collections { if collection.GetReplicaNumber() <= 0 { @@ -207,11 +213,6 @@ func (m *CollectionManager) Recover(ctx context.Context, broker Broker) error { } } - err = m.upgradeRecover(ctx, broker) - if err != nil { - log.Warn("upgrade recover failed", zap.Error(err)) - return err - } return nil } @@ -245,73 +246,6 @@ func (m *CollectionManager) upgradeLoadFields(ctx context.Context, collection *q return nil } -// upgradeRecover recovers from old version <= 2.2.x for compatibility. -func (m *CollectionManager) upgradeRecover(ctx context.Context, broker Broker) error { - // for loaded collection from 2.2, it only save a old version CollectionLoadInfo without LoadType. - // we should update the CollectionLoadInfo and save all PartitionLoadInfo to meta store - for _, collection := range m.GetAllCollections(ctx) { - if collection.GetLoadType() == querypb.LoadType_UnKnownType { - partitionIDs, err := broker.GetPartitions(context.Background(), collection.GetCollectionID()) - if err != nil { - return err - } - partitions := lo.Map(partitionIDs, func(partitionID int64, _ int) *Partition { - return &Partition{ - PartitionLoadInfo: &querypb.PartitionLoadInfo{ - CollectionID: collection.GetCollectionID(), - PartitionID: partitionID, - ReplicaNumber: collection.GetReplicaNumber(), - Status: querypb.LoadStatus_Loaded, - FieldIndexID: collection.GetFieldIndexID(), - }, - LoadPercentage: 100, - } - }) - err = m.putPartition(ctx, partitions, true) - if err != nil { - return err - } - - newInfo := collection.Clone() - newInfo.LoadType = querypb.LoadType_LoadCollection - err = m.putCollection(ctx, true, newInfo) - if err != nil { - return err - } - } - } - - // for loaded partition from 2.2, it only save load PartitionLoadInfo. - // we should save it's CollectionLoadInfo to meta store - for _, partition := range m.GetAllPartitions(ctx) { - // In old version, collection would NOT be stored if the partition existed. - if !m.Exist(ctx, partition.GetCollectionID()) { - collectionInfo, err := broker.DescribeCollection(ctx, partition.GetCollectionID()) - if err != nil { - log.Warn("failed to describe collection from RootCoord", zap.Error(err)) - return err - } - - col := &Collection{ - CollectionLoadInfo: &querypb.CollectionLoadInfo{ - CollectionID: partition.GetCollectionID(), - ReplicaNumber: partition.GetReplicaNumber(), - Status: partition.GetStatus(), - FieldIndexID: partition.GetFieldIndexID(), - LoadType: querypb.LoadType_LoadPartition, - DbID: collectionInfo.GetDbId(), - }, - LoadPercentage: 100, - } - err = m.PutCollection(ctx, col) - if err != nil { - return err - } - } - } - return nil -} - func (m *CollectionManager) GetCollection(ctx context.Context, collectionID typeutil.UniqueID) *Collection { m.rwmutex.RLock() defer m.rwmutex.RUnlock() diff --git a/internal/querycoordv2/meta/collection_manager_test.go b/internal/querycoordv2/meta/collection_manager_test.go index 7e361019bd..be22916f11 100644 --- a/internal/querycoordv2/meta/collection_manager_test.go +++ b/internal/querycoordv2/meta/collection_manager_test.go @@ -479,71 +479,6 @@ func (suite *CollectionManagerSuite) TestUpdateLoadPercentage() { suite.Equal(querypb.LoadStatus_Loaded, mgr.CalculateLoadStatus(ctx, collection.CollectionID)) } -func (suite *CollectionManagerSuite) TestUpgradeRecover() { - suite.releaseAll() - mgr := suite.mgr - ctx := suite.ctx - - // put old version of collections and partitions - for i, collection := range suite.collections { - status := querypb.LoadStatus_Loaded - if suite.loadTypes[i] == querypb.LoadType_LoadCollection { - mgr.PutCollection(ctx, &Collection{ - CollectionLoadInfo: &querypb.CollectionLoadInfo{ - CollectionID: collection, - ReplicaNumber: suite.replicaNumber[i], - Status: status, - LoadType: querypb.LoadType_UnKnownType, // old version's collection didn't set loadType - }, - LoadPercentage: suite.colLoadPercent[i], - CreatedAt: time.Now(), - }) - } else { - for _, partition := range suite.partitions[collection] { - mgr.PutPartition(ctx, &Partition{ - PartitionLoadInfo: &querypb.PartitionLoadInfo{ - CollectionID: collection, - PartitionID: partition, - ReplicaNumber: suite.replicaNumber[i], - Status: status, - }, - LoadPercentage: suite.colLoadPercent[i], - CreatedAt: time.Now(), - }) - } - } - } - - // set expectations - for i, collection := range suite.collections { - if suite.loadTypes[i] == querypb.LoadType_LoadCollection { - suite.broker.EXPECT().GetPartitions(mock.Anything, collection).Return(suite.partitions[collection], nil) - } - suite.broker.EXPECT().DescribeCollection(mock.Anything, collection).Return(&milvuspb.DescribeCollectionResponse{ - Status: merr.Success(), - Schema: &schemapb.CollectionSchema{ - Fields: []*schemapb.FieldSchema{ - {FieldID: common.RowIDField}, - {FieldID: common.TimeStampField}, - {FieldID: 100, Name: "pk"}, - {FieldID: 101, Name: "vector"}, - }, - }, - }, nil).Maybe() - } - - // do recovery - suite.clearMemory() - err := mgr.Recover(ctx, suite.broker) - suite.NoError(err) - suite.checkLoadResult() - - for i, collection := range suite.collections { - newColl := mgr.GetCollection(ctx, collection) - suite.Equal(suite.loadTypes[i], newColl.GetLoadType()) - } -} - func (suite *CollectionManagerSuite) TestUpgradeLoadFields() { suite.releaseAll() mgr := suite.mgr diff --git a/internal/rootcoord/meta_table.go b/internal/rootcoord/meta_table.go index cf76a93547..64aa18e8a1 100644 --- a/internal/rootcoord/meta_table.go +++ b/internal/rootcoord/meta_table.go @@ -20,6 +20,7 @@ import ( "context" "fmt" "sync" + "time" "github.com/cockroachdb/errors" "github.com/samber/lo" @@ -190,6 +191,9 @@ func (mt *MetaTable) reload() error { collectionNum := int64(0) mt.names.createDbIfNotExist(dbName) + + start := time.Now() + // TODO: async list collections to accelerate cases with multiple databases. collections, err := mt.catalog.ListCollections(mt.ctx, db.ID, typeutil.MaxTimestamp) if err != nil { return err @@ -213,7 +217,8 @@ func (mt *MetaTable) reload() error { metrics.RootCoordNumOfPartitions.WithLabelValues().Add(float64(partitionNum)) log.Ctx(mt.ctx).Info("collections recovered from db", zap.String("db_name", dbName), zap.Int64("collection_num", collectionNum), - zap.Int64("partition_num", partitionNum)) + zap.Int64("partition_num", partitionNum), + zap.Duration("dur", time.Since(start))) } // recover aliases from db namespace diff --git a/internal/rootcoord/meta_table_test.go b/internal/rootcoord/meta_table_test.go index 1513dd9a3a..f5d4db657d 100644 --- a/internal/rootcoord/meta_table_test.go +++ b/internal/rootcoord/meta_table_test.go @@ -43,7 +43,7 @@ import ( ) func generateMetaTable(t *testing.T) *MetaTable { - return &MetaTable{catalog: &rootcoord.Catalog{Txn: memkv.NewMemoryKV()}} + return &MetaTable{catalog: rootcoord.NewCatalog(memkv.NewMemoryKV(), nil)} } func TestRbacAddCredential(t *testing.T) { diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index e00909b8a8..9499b90834 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -370,7 +370,7 @@ func (c *Core) initMetaTable(initCtx context.Context) error { if ss, err = kvmetestore.NewSuffixSnapshot(metaKV, kvmetestore.SnapshotsSep, Params.EtcdCfg.MetaRootPath.GetValue(), kvmetestore.SnapshotPrefix); err != nil { return err } - catalog = &kvmetestore.Catalog{Txn: metaKV, Snapshot: ss} + catalog = kvmetestore.NewCatalog(metaKV, ss) case util.MetaStoreTypeTiKV: log.Ctx(initCtx).Info("Using tikv as meta storage.") var ss *kvmetestore.SuffixSnapshot @@ -380,7 +380,7 @@ func (c *Core) initMetaTable(initCtx context.Context) error { if ss, err = kvmetestore.NewSuffixSnapshot(metaKV, kvmetestore.SnapshotsSep, Params.TiKVCfg.MetaRootPath.GetValue(), kvmetestore.SnapshotPrefix); err != nil { return err } - catalog = &kvmetestore.Catalog{Txn: metaKV, Snapshot: ss} + catalog = kvmetestore.NewCatalog(metaKV, ss) default: return retry.Unrecoverable(fmt.Errorf("not supported meta store: %s", Params.MetaStoreCfg.MetaStoreType.GetValue())) } @@ -1336,6 +1336,75 @@ func (c *Core) ShowCollections(ctx context.Context, in *milvuspb.ShowCollections return t.Rsp, nil } +// ShowCollectionIDs returns all collection IDs. +func (c *Core) ShowCollectionIDs(ctx context.Context, in *rootcoordpb.ShowCollectionIDsRequest) (*rootcoordpb.ShowCollectionIDsResponse, error) { + if err := merr.CheckHealthy(c.GetStateCode()); err != nil { + return &rootcoordpb.ShowCollectionIDsResponse{ + Status: merr.Status(err), + }, nil + } + + metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollectionIDs", metrics.TotalLabel).Inc() + tr := timerecord.NewTimeRecorder("ShowCollectionIDs") + + ts := typeutil.MaxTimestamp + log := log.Ctx(ctx).With(zap.Strings("dbNames", in.GetDbNames()), zap.Bool("allowUnavailable", in.GetAllowUnavailable())) + + // Currently, this interface is only called during startup, so there is no need to execute it within the scheduler. + var err error + var dbs []*model.Database + if len(in.GetDbNames()) == 0 { + // show all collections + dbs, err = c.meta.ListDatabases(ctx, ts) + if err != nil { + log.Info("failed to ListDatabases", zap.Error(err)) + metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollectionIDs", metrics.FailLabel).Inc() + return &rootcoordpb.ShowCollectionIDsResponse{ + Status: merr.Status(err), + }, nil + } + } else { + dbs = make([]*model.Database, 0, len(in.GetDbNames())) + for _, name := range in.GetDbNames() { + db, err := c.meta.GetDatabaseByName(ctx, name, ts) + if err != nil { + log.Info("failed to GetDatabaseByName", zap.Error(err)) + metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollectionIDs", metrics.FailLabel).Inc() + return &rootcoordpb.ShowCollectionIDsResponse{ + Status: merr.Status(err), + }, nil + } + dbs = append(dbs, db) + } + } + dbCollections := make([]*rootcoordpb.DBCollections, 0, len(dbs)) + for _, db := range dbs { + collections, err := c.meta.ListCollections(ctx, db.Name, ts, !in.GetAllowUnavailable()) + if err != nil { + log.Info("failed to ListCollections", zap.Error(err)) + metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollectionIDs", metrics.FailLabel).Inc() + return &rootcoordpb.ShowCollectionIDsResponse{ + Status: merr.Status(err), + }, nil + } + dbCollections = append(dbCollections, &rootcoordpb.DBCollections{ + DbName: db.Name, + CollectionIDs: lo.Map(collections, func(col *model.Collection, _ int) int64 { + return col.CollectionID + }), + }) + } + metrics.RootCoordDDLReqCounter.WithLabelValues("ShowCollectionIDs", metrics.SuccessLabel).Inc() + metrics.RootCoordDDLReqLatency.WithLabelValues("ShowCollectionIDs").Observe(float64(tr.ElapseSpan().Milliseconds())) + + log.Info("ShowCollectionIDs done", zap.Any("collectionIDs", dbCollections)) + + return &rootcoordpb.ShowCollectionIDsResponse{ + Status: merr.Success(), + DbCollections: dbCollections, + }, nil +} + func (c *Core) AlterCollection(ctx context.Context, in *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) { if err := merr.CheckHealthy(c.GetStateCode()); err != nil { return merr.Status(err), nil diff --git a/internal/rootcoord/root_coord_test.go b/internal/rootcoord/root_coord_test.go index 951b23d580..b505cf3ba2 100644 --- a/internal/rootcoord/root_coord_test.go +++ b/internal/rootcoord/root_coord_test.go @@ -731,6 +731,78 @@ func TestRootCoord_ShowCollections(t *testing.T) { }) } +func TestRootCoord_ShowCollectionIDs(t *testing.T) { + t.Run("not healthy", func(t *testing.T) { + c := newTestCore(withAbnormalCode()) + ctx := context.Background() + resp, err := c.ShowCollectionIDs(ctx, &rootcoordpb.ShowCollectionIDsRequest{}) + assert.NoError(t, err) + assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) + }) + + t.Run("test failed", func(t *testing.T) { + c := newTestCore(withHealthyCode()) + meta := mockrootcoord.NewIMetaTable(t) + c.meta = meta + + ctx := context.Background() + + // specify db names + meta.EXPECT().GetDatabaseByName(mock.Anything, mock.Anything, typeutil.MaxTimestamp).Return(nil, fmt.Errorf("mock err")) + resp, err := c.ShowCollectionIDs(ctx, &rootcoordpb.ShowCollectionIDsRequest{ + DbNames: []string{"db1"}, + AllowUnavailable: true, + }) + assert.NoError(t, err) + assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) + + // not specify db names + meta.EXPECT().ListDatabases(mock.Anything, mock.Anything).Return(nil, fmt.Errorf("mock err")) + resp, err = c.ShowCollectionIDs(ctx, &rootcoordpb.ShowCollectionIDsRequest{}) + assert.NoError(t, err) + assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) + + // list collections failed + meta.ExpectedCalls = nil + meta.EXPECT().ListDatabases(mock.Anything, mock.Anything).Return( + []*model.Database{model.NewDatabase(rand.Int63(), "db1", etcdpb.DatabaseState_DatabaseCreated, nil)}, nil) + meta.EXPECT().ListCollections(mock.Anything, mock.Anything, typeutil.MaxTimestamp, false).Return(nil, fmt.Errorf("mock err")) + resp, err = c.ShowCollectionIDs(ctx, &rootcoordpb.ShowCollectionIDsRequest{ + AllowUnavailable: true, + }) + assert.NoError(t, err) + assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) + }) + + t.Run("normal case, everything is ok", func(t *testing.T) { + c := newTestCore(withHealthyCode()) + meta := mockrootcoord.NewIMetaTable(t) + meta.EXPECT().ListCollections(mock.Anything, mock.Anything, typeutil.MaxTimestamp, false).Return([]*model.Collection{}, nil) + c.meta = meta + + ctx := context.Background() + + // specify db names + meta.EXPECT().GetDatabaseByName(mock.Anything, mock.Anything, typeutil.MaxTimestamp).Return( + model.NewDatabase(rand.Int63(), "db1", etcdpb.DatabaseState_DatabaseCreated, nil), nil) + resp, err := c.ShowCollectionIDs(ctx, &rootcoordpb.ShowCollectionIDsRequest{ + DbNames: []string{"db1"}, + AllowUnavailable: true, + }) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) + + // not specify db names + meta.EXPECT().ListDatabases(mock.Anything, mock.Anything).Return( + []*model.Database{model.NewDatabase(rand.Int63(), "db1", etcdpb.DatabaseState_DatabaseCreated, nil)}, nil) + resp, err = c.ShowCollectionIDs(ctx, &rootcoordpb.ShowCollectionIDsRequest{ + AllowUnavailable: true, + }) + assert.NoError(t, err) + assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) + }) +} + func TestRootCoord_HasPartition(t *testing.T) { t.Run("not healthy", func(t *testing.T) { c := newTestCore(withAbnormalCode()) diff --git a/internal/util/mock/grpc_rootcoord_client.go b/internal/util/mock/grpc_rootcoord_client.go index b19e9ebc94..2a415f8576 100644 --- a/internal/util/mock/grpc_rootcoord_client.go +++ b/internal/util/mock/grpc_rootcoord_client.go @@ -158,6 +158,10 @@ func (m *GrpcRootCoordClient) ShowCollections(ctx context.Context, in *milvuspb. return &milvuspb.ShowCollectionsResponse{}, m.Err } +func (m *GrpcRootCoordClient) ShowCollectionIDs(ctx context.Context, in *rootcoordpb.ShowCollectionIDsRequest, opts ...grpc.CallOption) (*rootcoordpb.ShowCollectionIDsResponse, error) { + return &rootcoordpb.ShowCollectionIDsResponse{}, m.Err +} + func (m *GrpcRootCoordClient) CreatePartition(ctx context.Context, in *milvuspb.CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { return &commonpb.Status{}, m.Err } diff --git a/pkg/proto/root_coord.proto b/pkg/proto/root_coord.proto index 7aa99db7c9..d391f457e3 100644 --- a/pkg/proto/root_coord.proto +++ b/pkg/proto/root_coord.proto @@ -63,6 +63,8 @@ service RootCoord { */ rpc ShowCollections(milvus.ShowCollectionsRequest) returns (milvus.ShowCollectionsResponse) {} + rpc ShowCollectionIDs(ShowCollectionIDsRequest) returns (ShowCollectionIDsResponse) {} + rpc AlterCollection(milvus.AlterCollectionRequest) returns (common.Status) {} rpc AlterCollectionField(milvus.AlterCollectionFieldRequest) returns (common.Status) {} @@ -247,4 +249,20 @@ message CollectionInfoOnPChannel { message PartitionInfoOnPChannel { int64 partition_id = 1; -} \ No newline at end of file +} + +message ShowCollectionIDsRequest { + common.MsgBase base = 1; + repeated string db_names = 2; + bool allowUnavailable = 3; +} + +message DBCollections { + string db_name = 1; + repeated int64 collectionIDs = 2; +} + +message ShowCollectionIDsResponse { + common.Status status = 1; + repeated DBCollections db_collections = 2; +} diff --git a/pkg/proto/rootcoordpb/root_coord.pb.go b/pkg/proto/rootcoordpb/root_coord.pb.go index e6ddcbb94f..fc93a2263c 100644 --- a/pkg/proto/rootcoordpb/root_coord.pb.go +++ b/pkg/proto/rootcoordpb/root_coord.pb.go @@ -1077,6 +1077,179 @@ func (x *PartitionInfoOnPChannel) GetPartitionId() int64 { return 0 } +type ShowCollectionIDsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` + DbNames []string `protobuf:"bytes,2,rep,name=db_names,json=dbNames,proto3" json:"db_names,omitempty"` + AllowUnavailable bool `protobuf:"varint,3,opt,name=allowUnavailable,proto3" json:"allowUnavailable,omitempty"` +} + +func (x *ShowCollectionIDsRequest) Reset() { + *x = ShowCollectionIDsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ShowCollectionIDsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ShowCollectionIDsRequest) ProtoMessage() {} + +func (x *ShowCollectionIDsRequest) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[17] + 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 ShowCollectionIDsRequest.ProtoReflect.Descriptor instead. +func (*ShowCollectionIDsRequest) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{17} +} + +func (x *ShowCollectionIDsRequest) GetBase() *commonpb.MsgBase { + if x != nil { + return x.Base + } + return nil +} + +func (x *ShowCollectionIDsRequest) GetDbNames() []string { + if x != nil { + return x.DbNames + } + return nil +} + +func (x *ShowCollectionIDsRequest) GetAllowUnavailable() bool { + if x != nil { + return x.AllowUnavailable + } + return false +} + +type DBCollections struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + DbName string `protobuf:"bytes,1,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` + CollectionIDs []int64 `protobuf:"varint,2,rep,packed,name=collectionIDs,proto3" json:"collectionIDs,omitempty"` +} + +func (x *DBCollections) Reset() { + *x = DBCollections{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *DBCollections) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DBCollections) ProtoMessage() {} + +func (x *DBCollections) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[18] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DBCollections.ProtoReflect.Descriptor instead. +func (*DBCollections) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{18} +} + +func (x *DBCollections) GetDbName() string { + if x != nil { + return x.DbName + } + return "" +} + +func (x *DBCollections) GetCollectionIDs() []int64 { + if x != nil { + return x.CollectionIDs + } + return nil +} + +type ShowCollectionIDsResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + DbCollections []*DBCollections `protobuf:"bytes,2,rep,name=db_collections,json=dbCollections,proto3" json:"db_collections,omitempty"` +} + +func (x *ShowCollectionIDsResponse) Reset() { + *x = ShowCollectionIDsResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_root_coord_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ShowCollectionIDsResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ShowCollectionIDsResponse) ProtoMessage() {} + +func (x *ShowCollectionIDsResponse) ProtoReflect() protoreflect.Message { + mi := &file_root_coord_proto_msgTypes[19] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ShowCollectionIDsResponse.ProtoReflect.Descriptor instead. +func (*ShowCollectionIDsResponse) Descriptor() ([]byte, []int) { + return file_root_coord_proto_rawDescGZIP(), []int{19} +} + +func (x *ShowCollectionIDsResponse) GetStatus() *commonpb.Status { + if x != nil { + return x.Status + } + return nil +} + +func (x *ShowCollectionIDsResponse) GetDbCollections() []*DBCollections { + if x != nil { + return x.DbCollections + } + return nil +} + var File_root_coord_proto protoreflect.FileDescriptor var file_root_coord_proto_rawDesc = []byte{ @@ -1257,362 +1430,394 @@ var file_root_coord_proto_rawDesc = []byte{ 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x4f, 0x6e, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x21, 0x0a, 0x0c, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x70, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x32, 0xec, 0x2b, 0x0a, 0x09, 0x52, 0x6f, 0x6f, - 0x74, 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, 0x5f, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x44, 0x72, 0x6f, 0x70, 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, 0x5f, 0x0a, 0x0d, 0x48, 0x61, 0x73, 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, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x48, 0x61, 0x73, 0x43, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x42, 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x77, 0x0a, 0x12, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, - 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2e, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7f, - 0x0a, 0x1a, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x2e, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x55, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x27, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x69, 0x61, 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, 0x51, 0x0a, 0x09, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, - 0x69, 0x61, 0x73, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, - 0x69, 0x61, 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, 0x53, 0x0a, 0x0a, 0x41, 0x6c, 0x74, - 0x65, 0x72, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x41, 0x6c, - 0x74, 0x65, 0x72, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x93, 0x01, 0x0a, 0x18, 0x53, 0x68, 0x6f, + 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x30, 0x0a, 0x04, 0x62, 0x61, 0x73, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x4d, 0x73, 0x67, 0x42, 0x61, 0x73, + 0x65, 0x52, 0x04, 0x62, 0x61, 0x73, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x64, 0x62, 0x5f, 0x6e, 0x61, + 0x6d, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x64, 0x62, 0x4e, 0x61, 0x6d, + 0x65, 0x73, 0x12, 0x2a, 0x0a, 0x10, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x55, 0x6e, 0x61, 0x76, 0x61, + 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x10, 0x61, 0x6c, + 0x6c, 0x6f, 0x77, 0x55, 0x6e, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x22, 0x4e, + 0x0a, 0x0d, 0x44, 0x42, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 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, 0x24, 0x0a, 0x0d, 0x63, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x03, 0x52, + 0x0d, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x22, 0x9e, + 0x01, 0x0a, 0x19, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x49, 0x44, 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, 0x4c, 0x0a, 0x0e, 0x64, 0x62, 0x5f, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, + 0x72, 0x64, 0x2e, 0x44, 0x42, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x52, 0x0d, 0x64, 0x62, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x32, + 0xe8, 0x2c, 0x0a, 0x09, 0x52, 0x6f, 0x6f, 0x74, 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, 0x5f, 0x0a, + 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x68, - 0x0a, 0x0d, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, - 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x41, 0x6c, - 0x69, 0x61, 0x73, 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, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, - 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, - 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6e, 0x0a, 0x0f, - 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, - 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0f, - 0x41, 0x6c, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5b, + 0x0a, 0x0e, 0x44, 0x72, 0x6f, 0x70, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, 0x6f, 0x70, 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, 0x67, 0x0a, 0x14, 0x41, - 0x6c, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, - 0x65, 0x6c, 0x64, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x43, - 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x65, 0x6c, 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, 0x5d, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x22, 0x00, 0x12, 0x59, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, - 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5d, - 0x0a, 0x0c, 0x48, 0x61, 0x73, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x28, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x48, 0x61, 0x73, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x42, - 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x6b, 0x0a, - 0x0e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, - 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, 0x16, 0x53, 0x68, - 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x49, 0x6e, 0x74, 0x65, - 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x65, 0x0a, 0x0c, 0x53, 0x68, 0x6f, 0x77, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, - 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, - 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x53, 0x68, 0x6f, 0x77, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x74, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x50, 0x43, 0x68, - 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, - 0x72, 0x64, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, - 0x72, 0x64, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, - 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x71, 0x0a, 0x0e, - 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x2d, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, - 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x54, 0x69, 0x6d, - 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2e, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, - 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x54, 0x69, 0x6d, 0x65, - 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, - 0x5c, 0x0a, 0x07, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x49, 0x44, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, - 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, - 0x63, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x61, 0x0a, - 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x69, - 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x43, - 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x4d, 0x73, - 0x67, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, - 0x12, 0x72, 0x0a, 0x1d, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, - 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x43, 0x61, 0x63, 0x68, - 0x65, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, - 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x43, 0x61, 0x63, 0x68, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x22, 0x00, 0x12, 0x7b, 0x0a, 0x12, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, - 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, - 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, - 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, - 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x47, 0x65, - 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x58, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, - 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x43, - 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 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, 0x10, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, - 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, - 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 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, 0x5f, 0x0a, 0x10, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, - 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, 0x68, 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x43, - 0x72, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, - 0x69, 0x73, 0x74, 0x43, 0x72, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 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, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x72, - 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x6e, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, - 0x61, 0x6c, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x47, 0x65, 0x74, 0x43, - 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x47, 0x65, 0x74, 0x43, 0x72, 0x65, - 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, - 0x00, 0x12, 0x53, 0x0a, 0x0a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6c, 0x65, 0x12, - 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x6f, 0x6c, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x08, 0x44, 0x72, 0x6f, 0x70, 0x52, 0x6f, - 0x6c, 0x65, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x52, 0x6f, 0x6c, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0f, 0x4f, 0x70, 0x65, 0x72, 0x61, - 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x6f, 0x6c, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, - 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, - 0x74, 0x52, 0x6f, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, + 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0d, 0x48, + 0x61, 0x73, 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, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x52, 0x6f, 0x6c, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x53, 0x65, 0x6c, 0x65, 0x63, - 0x74, 0x55, 0x73, 0x65, 0x72, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x65, 0x6c, 0x65, - 0x63, 0x74, 0x55, 0x73, 0x65, 0x72, 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, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x55, 0x73, 0x65, 0x72, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x10, 0x4f, 0x70, 0x65, 0x72, - 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x12, 0x2c, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, - 0x65, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, - 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x0b, 0x53, 0x65, 0x6c, - 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x6e, 0x74, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, - 0x65, 0x6c, 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x6e, 0x74, 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, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x47, 0x72, - 0x61, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x63, 0x0a, - 0x0a, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x28, 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, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 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, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x22, 0x00, 0x12, 0x67, 0x0a, 0x0a, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x42, 0x41, 0x43, - 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x42, 0x41, - 0x43, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x42, 0x41, 0x43, 0x4d, 0x65, 0x74, - 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x59, 0x0a, 0x0b, 0x52, - 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x42, 0x41, 0x43, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, + 0x75, 0x73, 0x2e, 0x48, 0x61, 0x73, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x42, 0x6f, + 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x77, 0x0a, 0x12, + 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x7f, 0x0a, 0x1a, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, + 0x62, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x55, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x41, 0x6c, 0x69, 0x61, 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, 0x51, 0x0a, + 0x09, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x42, 0x41, 0x43, 0x4d, 0x65, 0x74, 0x61, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x67, 0x0a, 0x14, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x30, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, - 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, - 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, - 0x63, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, - 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, 0x6f, 0x70, - 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, - 0x75, 0x73, 0x22, 0x00, 0x12, 0x7a, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x69, 0x76, - 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 0x2f, 0x2e, 0x6d, 0x69, - 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, - 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, - 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, - 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x69, 0x0a, 0x15, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, - 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x31, 0x2e, 0x6d, 0x69, 0x6c, 0x76, - 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, - 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, - 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, - 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, - 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 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, - 0x5f, 0x0a, 0x10, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, - 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x41, 0x6c, 0x69, 0x61, 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, 0x5b, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, - 0x73, 0x65, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, - 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, - 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, - 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x57, 0x0a, - 0x0c, 0x44, 0x72, 0x6f, 0x70, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x28, 0x2e, - 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, + 0x12, 0x53, 0x0a, 0x0a, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x26, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x41, 0x6c, 0x69, 0x61, 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, 0x68, 0x0a, 0x0d, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, + 0x65, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x65, 0x73, + 0x63, 0x72, 0x69, 0x62, 0x65, 0x41, 0x6c, 0x69, 0x61, 0x73, 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, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, + 0x41, 0x6c, 0x69, 0x61, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x62, 0x0a, 0x0b, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x12, 0x27, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, + 0x73, 0x74, 0x41, 0x6c, 0x69, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x6e, 0x0a, 0x0f, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, + 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, + 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x7a, 0x0a, 0x11, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, + 0x64, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x49, 0x44, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x31, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, + 0x6f, 0x72, 0x64, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x49, 0x44, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x5d, 0x0a, 0x0f, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 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, 0x67, + 0x0a, 0x14, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x41, 0x6c, 0x74, + 0x65, 0x72, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x46, 0x69, 0x65, 0x6c, + 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, 0x5d, 0x0a, 0x0f, 0x43, 0x72, 0x65, 0x61, 0x74, + 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x68, 0x0a, 0x0d, 0x4c, 0x69, 0x73, 0x74, 0x44, 0x61, - 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, - 0x73, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 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, 0x4c, 0x69, 0x73, 0x74, 0x44, 0x61, 0x74, - 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, - 0x12, 0x77, 0x0a, 0x10, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x44, 0x61, 0x74, 0x61, - 0x62, 0x61, 0x73, 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x44, 0x65, - 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x44, - 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x0d, 0x41, 0x6c, 0x74, - 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, - 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, - 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, + 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x59, 0x0a, 0x0d, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, + 0x6f, 0x70, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, + 0x00, 0x12, 0x5d, 0x0a, 0x0c, 0x48, 0x61, 0x73, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x48, 0x61, 0x73, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x42, 0x6f, 0x6f, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x6b, 0x0a, 0x0e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2b, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x73, 0x0a, + 0x16, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x49, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, + 0x6f, 0x77, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x65, 0x0a, 0x0c, 0x53, 0x68, 0x6f, 0x77, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, + 0x74, 0x73, 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x53, 0x65, 0x67, + 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x53, 0x68, 0x6f, 0x77, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x74, 0x0a, 0x0f, 0x47, 0x65, 0x74, + 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x2e, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, + 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x2f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, + 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, + 0x71, 0x0a, 0x0e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x12, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, + 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x07, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x49, 0x44, 0x12, 0x26, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, + 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x6c, 0x6f, 0x63, 0x49, 0x44, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x41, + 0x6c, 0x6c, 0x6f, 0x63, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x61, 0x0a, 0x15, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, + 0x6c, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, 0x6b, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2e, 0x43, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x54, 0x69, 0x6d, 0x65, 0x54, 0x69, 0x63, + 0x6b, 0x4d, 0x73, 0x67, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x22, 0x00, 0x12, 0x72, 0x0a, 0x1d, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, + 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x74, 0x61, 0x43, + 0x61, 0x63, 0x68, 0x65, 0x12, 0x32, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x70, 0x72, 0x6f, 0x78, 0x79, 0x2e, 0x49, 0x6e, 0x76, 0x61, 0x6c, 0x69, + 0x64, 0x61, 0x74, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x4d, 0x65, 0x74, 0x61, 0x43, 0x61, 0x63, 0x68, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x42, 0x33, 0x5a, 0x31, 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, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x33, + 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, 0x58, 0x0a, 0x10, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x43, + 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2e, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x66, 0x6f, + 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, 0x10, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, + 0x69, 0x61, 0x6c, 0x12, 0x25, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x43, 0x72, 0x65, 0x64, + 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x49, 0x6e, 0x66, 0x6f, 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, 0x5f, 0x0a, 0x10, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x12, 0x2c, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, + 0x74, 0x69, 0x61, 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, 0x68, 0x0a, 0x0d, 0x4c, 0x69, + 0x73, 0x74, 0x43, 0x72, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x43, 0x72, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 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, 0x4c, 0x69, 0x73, + 0x74, 0x43, 0x72, 0x65, 0x64, 0x55, 0x73, 0x65, 0x72, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x6e, 0x0a, 0x0d, 0x47, 0x65, 0x74, 0x43, 0x72, 0x65, 0x64, 0x65, + 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x47, + 0x65, 0x74, 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x2d, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x47, 0x65, 0x74, + 0x43, 0x72, 0x65, 0x64, 0x65, 0x6e, 0x74, 0x69, 0x61, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0x00, 0x12, 0x53, 0x0a, 0x0a, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x6f, + 0x6c, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, + 0x6f, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x4f, 0x0a, 0x08, 0x44, 0x72, 0x6f, + 0x70, 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x24, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, 0x6f, 0x70, + 0x52, 0x6f, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5d, 0x0a, 0x0f, 0x4f, 0x70, + 0x65, 0x72, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x2b, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x55, 0x73, 0x65, 0x72, 0x52, + 0x6f, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x53, 0x65, 0x6c, + 0x65, 0x63, 0x74, 0x52, 0x6f, 0x6c, 0x65, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, 0x65, + 0x6c, 0x65, 0x63, 0x74, 0x52, 0x6f, 0x6c, 0x65, 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, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x52, 0x6f, 0x6c, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x0a, 0x53, 0x65, + 0x6c, 0x65, 0x63, 0x74, 0x55, 0x73, 0x65, 0x72, 0x12, 0x26, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x53, + 0x65, 0x6c, 0x65, 0x63, 0x74, 0x55, 0x73, 0x65, 0x72, 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, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x55, 0x73, 0x65, + 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5f, 0x0a, 0x10, 0x4f, + 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x12, + 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, + 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, + 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x62, 0x0a, 0x0b, + 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x6e, 0x74, 0x12, 0x27, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x47, 0x72, 0x61, 0x6e, 0x74, 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, 0x53, 0x65, 0x6c, 0x65, 0x63, + 0x74, 0x47, 0x72, 0x61, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, + 0x12, 0x63, 0x0a, 0x0a, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x28, + 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, 0x50, 0x6f, 0x6c, 0x69, 0x63, + 0x79, 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, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, + 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x67, 0x0a, 0x0a, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, + 0x42, 0x41, 0x43, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, + 0x52, 0x42, 0x41, 0x43, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x2b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x42, 0x61, 0x63, 0x6b, 0x75, 0x70, 0x52, 0x42, 0x41, 0x43, + 0x4d, 0x65, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x59, + 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x42, 0x41, 0x43, 0x12, 0x2b, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x52, 0x65, 0x73, 0x74, 0x6f, 0x72, 0x65, 0x52, 0x42, 0x41, 0x43, 0x4d, + 0x65, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x67, 0x0a, 0x14, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x12, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x50, 0x72, + 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, + 0x22, 0x00, 0x12, 0x63, 0x0a, 0x12, 0x44, 0x72, 0x6f, 0x70, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, + 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x2e, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, + 0x72, 0x6f, 0x70, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x7a, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x50, + 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x12, 0x2f, + 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, 0x65, + 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, + 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x69, 0x0a, 0x15, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x50, 0x72, + 0x69, 0x76, 0x69, 0x6c, 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x12, 0x31, 0x2e, 0x6d, + 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, + 0x75, 0x73, 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x65, 0x50, 0x72, 0x69, 0x76, 0x69, 0x6c, + 0x65, 0x67, 0x65, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, + 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, + 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 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, 0x5f, 0x0a, 0x10, 0x52, 0x65, 0x6e, 0x61, 0x6d, 0x65, 0x43, 0x6f, 0x6c, 0x6c, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2c, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x52, 0x65, 0x6e, + 0x61, 0x6d, 0x65, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x22, 0x00, 0x12, 0x5b, 0x0a, 0x0e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x44, 0x61, 0x74, + 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x2a, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, + 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, + 0x12, 0x57, 0x0a, 0x0c, 0x44, 0x72, 0x6f, 0x70, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, + 0x12, 0x28, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x44, 0x72, 0x6f, 0x70, 0x44, 0x61, 0x74, 0x61, 0x62, + 0x61, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, + 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x12, 0x68, 0x0a, 0x0d, 0x4c, 0x69, 0x73, + 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 0x12, 0x29, 0x2e, 0x6d, 0x69, 0x6c, + 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x4c, 0x69, 0x73, 0x74, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 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, 0x4c, 0x69, 0x73, 0x74, + 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x22, 0x00, 0x12, 0x77, 0x0a, 0x10, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x44, + 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x2f, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, + 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, + 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x30, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, + 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, + 0x64, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, + 0x73, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x5c, 0x0a, 0x0d, + 0x41, 0x6c, 0x74, 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, 0x62, 0x61, 0x73, 0x65, 0x12, 0x2c, 0x2e, + 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x72, 0x6f, 0x6f, + 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x41, 0x6c, 0x74, 0x65, 0x72, 0x44, 0x61, 0x74, 0x61, + 0x62, 0x61, 0x73, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1b, 0x2e, 0x6d, 0x69, + 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x22, 0x00, 0x42, 0x33, 0x5a, 0x31, 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, 0x72, 0x6f, 0x6f, 0x74, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x70, 0x62, 0x62, + 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1627,7 +1832,7 @@ func file_root_coord_proto_rawDescGZIP() []byte { return file_root_coord_proto_rawDescData } -var file_root_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 19) +var file_root_coord_proto_msgTypes = make([]protoimpl.MessageInfo, 22) var file_root_coord_proto_goTypes = []interface{}{ (*AllocTimestampRequest)(nil), // 0: milvus.proto.rootcoord.AllocTimestampRequest (*AllocTimestampResponse)(nil), // 1: milvus.proto.rootcoord.AllocTimestampResponse @@ -1646,219 +1851,227 @@ var file_root_coord_proto_goTypes = []interface{}{ (*GetPChannelInfoResponse)(nil), // 14: milvus.proto.rootcoord.GetPChannelInfoResponse (*CollectionInfoOnPChannel)(nil), // 15: milvus.proto.rootcoord.CollectionInfoOnPChannel (*PartitionInfoOnPChannel)(nil), // 16: milvus.proto.rootcoord.PartitionInfoOnPChannel - nil, // 17: milvus.proto.rootcoord.SegmentInfos.ExtraIndexInfosEntry - nil, // 18: milvus.proto.rootcoord.DescribeSegmentsResponse.SegmentInfosEntry - (*commonpb.MsgBase)(nil), // 19: milvus.proto.common.MsgBase - (*commonpb.Status)(nil), // 20: milvus.proto.common.Status - (*etcdpb.SegmentIndexInfo)(nil), // 21: milvus.proto.etcd.SegmentIndexInfo - (*commonpb.KeyValuePair)(nil), // 22: milvus.proto.common.KeyValuePair - (*etcdpb.IndexInfo)(nil), // 23: milvus.proto.etcd.IndexInfo - (*milvuspb.GetComponentStatesRequest)(nil), // 24: milvus.proto.milvus.GetComponentStatesRequest - (*internalpb.GetTimeTickChannelRequest)(nil), // 25: milvus.proto.internal.GetTimeTickChannelRequest - (*internalpb.GetStatisticsChannelRequest)(nil), // 26: milvus.proto.internal.GetStatisticsChannelRequest - (*milvuspb.CreateCollectionRequest)(nil), // 27: milvus.proto.milvus.CreateCollectionRequest - (*milvuspb.DropCollectionRequest)(nil), // 28: milvus.proto.milvus.DropCollectionRequest - (*milvuspb.HasCollectionRequest)(nil), // 29: milvus.proto.milvus.HasCollectionRequest - (*milvuspb.DescribeCollectionRequest)(nil), // 30: milvus.proto.milvus.DescribeCollectionRequest - (*milvuspb.CreateAliasRequest)(nil), // 31: milvus.proto.milvus.CreateAliasRequest - (*milvuspb.DropAliasRequest)(nil), // 32: milvus.proto.milvus.DropAliasRequest - (*milvuspb.AlterAliasRequest)(nil), // 33: milvus.proto.milvus.AlterAliasRequest - (*milvuspb.DescribeAliasRequest)(nil), // 34: milvus.proto.milvus.DescribeAliasRequest - (*milvuspb.ListAliasesRequest)(nil), // 35: milvus.proto.milvus.ListAliasesRequest - (*milvuspb.ShowCollectionsRequest)(nil), // 36: milvus.proto.milvus.ShowCollectionsRequest - (*milvuspb.AlterCollectionRequest)(nil), // 37: milvus.proto.milvus.AlterCollectionRequest - (*milvuspb.AlterCollectionFieldRequest)(nil), // 38: milvus.proto.milvus.AlterCollectionFieldRequest - (*milvuspb.CreatePartitionRequest)(nil), // 39: milvus.proto.milvus.CreatePartitionRequest - (*milvuspb.DropPartitionRequest)(nil), // 40: milvus.proto.milvus.DropPartitionRequest - (*milvuspb.HasPartitionRequest)(nil), // 41: milvus.proto.milvus.HasPartitionRequest - (*milvuspb.ShowPartitionsRequest)(nil), // 42: milvus.proto.milvus.ShowPartitionsRequest - (*milvuspb.ShowSegmentsRequest)(nil), // 43: milvus.proto.milvus.ShowSegmentsRequest - (*internalpb.ChannelTimeTickMsg)(nil), // 44: milvus.proto.internal.ChannelTimeTickMsg - (*proxypb.InvalidateCollMetaCacheRequest)(nil), // 45: milvus.proto.proxy.InvalidateCollMetaCacheRequest - (*internalpb.ShowConfigurationsRequest)(nil), // 46: milvus.proto.internal.ShowConfigurationsRequest - (*milvuspb.GetMetricsRequest)(nil), // 47: milvus.proto.milvus.GetMetricsRequest - (*internalpb.CredentialInfo)(nil), // 48: milvus.proto.internal.CredentialInfo - (*milvuspb.DeleteCredentialRequest)(nil), // 49: milvus.proto.milvus.DeleteCredentialRequest - (*milvuspb.ListCredUsersRequest)(nil), // 50: milvus.proto.milvus.ListCredUsersRequest - (*milvuspb.CreateRoleRequest)(nil), // 51: milvus.proto.milvus.CreateRoleRequest - (*milvuspb.DropRoleRequest)(nil), // 52: milvus.proto.milvus.DropRoleRequest - (*milvuspb.OperateUserRoleRequest)(nil), // 53: milvus.proto.milvus.OperateUserRoleRequest - (*milvuspb.SelectRoleRequest)(nil), // 54: milvus.proto.milvus.SelectRoleRequest - (*milvuspb.SelectUserRequest)(nil), // 55: milvus.proto.milvus.SelectUserRequest - (*milvuspb.OperatePrivilegeRequest)(nil), // 56: milvus.proto.milvus.OperatePrivilegeRequest - (*milvuspb.SelectGrantRequest)(nil), // 57: milvus.proto.milvus.SelectGrantRequest - (*internalpb.ListPolicyRequest)(nil), // 58: milvus.proto.internal.ListPolicyRequest - (*milvuspb.BackupRBACMetaRequest)(nil), // 59: milvus.proto.milvus.BackupRBACMetaRequest - (*milvuspb.RestoreRBACMetaRequest)(nil), // 60: milvus.proto.milvus.RestoreRBACMetaRequest - (*milvuspb.CreatePrivilegeGroupRequest)(nil), // 61: milvus.proto.milvus.CreatePrivilegeGroupRequest - (*milvuspb.DropPrivilegeGroupRequest)(nil), // 62: milvus.proto.milvus.DropPrivilegeGroupRequest - (*milvuspb.ListPrivilegeGroupsRequest)(nil), // 63: milvus.proto.milvus.ListPrivilegeGroupsRequest - (*milvuspb.OperatePrivilegeGroupRequest)(nil), // 64: milvus.proto.milvus.OperatePrivilegeGroupRequest - (*milvuspb.CheckHealthRequest)(nil), // 65: milvus.proto.milvus.CheckHealthRequest - (*milvuspb.RenameCollectionRequest)(nil), // 66: milvus.proto.milvus.RenameCollectionRequest - (*milvuspb.CreateDatabaseRequest)(nil), // 67: milvus.proto.milvus.CreateDatabaseRequest - (*milvuspb.DropDatabaseRequest)(nil), // 68: milvus.proto.milvus.DropDatabaseRequest - (*milvuspb.ListDatabasesRequest)(nil), // 69: milvus.proto.milvus.ListDatabasesRequest - (*milvuspb.ComponentStates)(nil), // 70: milvus.proto.milvus.ComponentStates - (*milvuspb.StringResponse)(nil), // 71: milvus.proto.milvus.StringResponse - (*milvuspb.BoolResponse)(nil), // 72: milvus.proto.milvus.BoolResponse - (*milvuspb.DescribeCollectionResponse)(nil), // 73: milvus.proto.milvus.DescribeCollectionResponse - (*milvuspb.DescribeAliasResponse)(nil), // 74: milvus.proto.milvus.DescribeAliasResponse - (*milvuspb.ListAliasesResponse)(nil), // 75: milvus.proto.milvus.ListAliasesResponse - (*milvuspb.ShowCollectionsResponse)(nil), // 76: milvus.proto.milvus.ShowCollectionsResponse - (*milvuspb.ShowPartitionsResponse)(nil), // 77: milvus.proto.milvus.ShowPartitionsResponse - (*milvuspb.ShowSegmentsResponse)(nil), // 78: milvus.proto.milvus.ShowSegmentsResponse - (*internalpb.ShowConfigurationsResponse)(nil), // 79: milvus.proto.internal.ShowConfigurationsResponse - (*milvuspb.GetMetricsResponse)(nil), // 80: milvus.proto.milvus.GetMetricsResponse - (*milvuspb.ListCredUsersResponse)(nil), // 81: milvus.proto.milvus.ListCredUsersResponse - (*milvuspb.SelectRoleResponse)(nil), // 82: milvus.proto.milvus.SelectRoleResponse - (*milvuspb.SelectUserResponse)(nil), // 83: milvus.proto.milvus.SelectUserResponse - (*milvuspb.SelectGrantResponse)(nil), // 84: milvus.proto.milvus.SelectGrantResponse - (*internalpb.ListPolicyResponse)(nil), // 85: milvus.proto.internal.ListPolicyResponse - (*milvuspb.BackupRBACMetaResponse)(nil), // 86: milvus.proto.milvus.BackupRBACMetaResponse - (*milvuspb.ListPrivilegeGroupsResponse)(nil), // 87: milvus.proto.milvus.ListPrivilegeGroupsResponse - (*milvuspb.CheckHealthResponse)(nil), // 88: milvus.proto.milvus.CheckHealthResponse - (*milvuspb.ListDatabasesResponse)(nil), // 89: milvus.proto.milvus.ListDatabasesResponse + (*ShowCollectionIDsRequest)(nil), // 17: milvus.proto.rootcoord.ShowCollectionIDsRequest + (*DBCollections)(nil), // 18: milvus.proto.rootcoord.DBCollections + (*ShowCollectionIDsResponse)(nil), // 19: milvus.proto.rootcoord.ShowCollectionIDsResponse + nil, // 20: milvus.proto.rootcoord.SegmentInfos.ExtraIndexInfosEntry + nil, // 21: milvus.proto.rootcoord.DescribeSegmentsResponse.SegmentInfosEntry + (*commonpb.MsgBase)(nil), // 22: milvus.proto.common.MsgBase + (*commonpb.Status)(nil), // 23: milvus.proto.common.Status + (*etcdpb.SegmentIndexInfo)(nil), // 24: milvus.proto.etcd.SegmentIndexInfo + (*commonpb.KeyValuePair)(nil), // 25: milvus.proto.common.KeyValuePair + (*etcdpb.IndexInfo)(nil), // 26: milvus.proto.etcd.IndexInfo + (*milvuspb.GetComponentStatesRequest)(nil), // 27: milvus.proto.milvus.GetComponentStatesRequest + (*internalpb.GetTimeTickChannelRequest)(nil), // 28: milvus.proto.internal.GetTimeTickChannelRequest + (*internalpb.GetStatisticsChannelRequest)(nil), // 29: milvus.proto.internal.GetStatisticsChannelRequest + (*milvuspb.CreateCollectionRequest)(nil), // 30: milvus.proto.milvus.CreateCollectionRequest + (*milvuspb.DropCollectionRequest)(nil), // 31: milvus.proto.milvus.DropCollectionRequest + (*milvuspb.HasCollectionRequest)(nil), // 32: milvus.proto.milvus.HasCollectionRequest + (*milvuspb.DescribeCollectionRequest)(nil), // 33: milvus.proto.milvus.DescribeCollectionRequest + (*milvuspb.CreateAliasRequest)(nil), // 34: milvus.proto.milvus.CreateAliasRequest + (*milvuspb.DropAliasRequest)(nil), // 35: milvus.proto.milvus.DropAliasRequest + (*milvuspb.AlterAliasRequest)(nil), // 36: milvus.proto.milvus.AlterAliasRequest + (*milvuspb.DescribeAliasRequest)(nil), // 37: milvus.proto.milvus.DescribeAliasRequest + (*milvuspb.ListAliasesRequest)(nil), // 38: milvus.proto.milvus.ListAliasesRequest + (*milvuspb.ShowCollectionsRequest)(nil), // 39: milvus.proto.milvus.ShowCollectionsRequest + (*milvuspb.AlterCollectionRequest)(nil), // 40: milvus.proto.milvus.AlterCollectionRequest + (*milvuspb.AlterCollectionFieldRequest)(nil), // 41: milvus.proto.milvus.AlterCollectionFieldRequest + (*milvuspb.CreatePartitionRequest)(nil), // 42: milvus.proto.milvus.CreatePartitionRequest + (*milvuspb.DropPartitionRequest)(nil), // 43: milvus.proto.milvus.DropPartitionRequest + (*milvuspb.HasPartitionRequest)(nil), // 44: milvus.proto.milvus.HasPartitionRequest + (*milvuspb.ShowPartitionsRequest)(nil), // 45: milvus.proto.milvus.ShowPartitionsRequest + (*milvuspb.ShowSegmentsRequest)(nil), // 46: milvus.proto.milvus.ShowSegmentsRequest + (*internalpb.ChannelTimeTickMsg)(nil), // 47: milvus.proto.internal.ChannelTimeTickMsg + (*proxypb.InvalidateCollMetaCacheRequest)(nil), // 48: milvus.proto.proxy.InvalidateCollMetaCacheRequest + (*internalpb.ShowConfigurationsRequest)(nil), // 49: milvus.proto.internal.ShowConfigurationsRequest + (*milvuspb.GetMetricsRequest)(nil), // 50: milvus.proto.milvus.GetMetricsRequest + (*internalpb.CredentialInfo)(nil), // 51: milvus.proto.internal.CredentialInfo + (*milvuspb.DeleteCredentialRequest)(nil), // 52: milvus.proto.milvus.DeleteCredentialRequest + (*milvuspb.ListCredUsersRequest)(nil), // 53: milvus.proto.milvus.ListCredUsersRequest + (*milvuspb.CreateRoleRequest)(nil), // 54: milvus.proto.milvus.CreateRoleRequest + (*milvuspb.DropRoleRequest)(nil), // 55: milvus.proto.milvus.DropRoleRequest + (*milvuspb.OperateUserRoleRequest)(nil), // 56: milvus.proto.milvus.OperateUserRoleRequest + (*milvuspb.SelectRoleRequest)(nil), // 57: milvus.proto.milvus.SelectRoleRequest + (*milvuspb.SelectUserRequest)(nil), // 58: milvus.proto.milvus.SelectUserRequest + (*milvuspb.OperatePrivilegeRequest)(nil), // 59: milvus.proto.milvus.OperatePrivilegeRequest + (*milvuspb.SelectGrantRequest)(nil), // 60: milvus.proto.milvus.SelectGrantRequest + (*internalpb.ListPolicyRequest)(nil), // 61: milvus.proto.internal.ListPolicyRequest + (*milvuspb.BackupRBACMetaRequest)(nil), // 62: milvus.proto.milvus.BackupRBACMetaRequest + (*milvuspb.RestoreRBACMetaRequest)(nil), // 63: milvus.proto.milvus.RestoreRBACMetaRequest + (*milvuspb.CreatePrivilegeGroupRequest)(nil), // 64: milvus.proto.milvus.CreatePrivilegeGroupRequest + (*milvuspb.DropPrivilegeGroupRequest)(nil), // 65: milvus.proto.milvus.DropPrivilegeGroupRequest + (*milvuspb.ListPrivilegeGroupsRequest)(nil), // 66: milvus.proto.milvus.ListPrivilegeGroupsRequest + (*milvuspb.OperatePrivilegeGroupRequest)(nil), // 67: milvus.proto.milvus.OperatePrivilegeGroupRequest + (*milvuspb.CheckHealthRequest)(nil), // 68: milvus.proto.milvus.CheckHealthRequest + (*milvuspb.RenameCollectionRequest)(nil), // 69: milvus.proto.milvus.RenameCollectionRequest + (*milvuspb.CreateDatabaseRequest)(nil), // 70: milvus.proto.milvus.CreateDatabaseRequest + (*milvuspb.DropDatabaseRequest)(nil), // 71: milvus.proto.milvus.DropDatabaseRequest + (*milvuspb.ListDatabasesRequest)(nil), // 72: milvus.proto.milvus.ListDatabasesRequest + (*milvuspb.ComponentStates)(nil), // 73: milvus.proto.milvus.ComponentStates + (*milvuspb.StringResponse)(nil), // 74: milvus.proto.milvus.StringResponse + (*milvuspb.BoolResponse)(nil), // 75: milvus.proto.milvus.BoolResponse + (*milvuspb.DescribeCollectionResponse)(nil), // 76: milvus.proto.milvus.DescribeCollectionResponse + (*milvuspb.DescribeAliasResponse)(nil), // 77: milvus.proto.milvus.DescribeAliasResponse + (*milvuspb.ListAliasesResponse)(nil), // 78: milvus.proto.milvus.ListAliasesResponse + (*milvuspb.ShowCollectionsResponse)(nil), // 79: milvus.proto.milvus.ShowCollectionsResponse + (*milvuspb.ShowPartitionsResponse)(nil), // 80: milvus.proto.milvus.ShowPartitionsResponse + (*milvuspb.ShowSegmentsResponse)(nil), // 81: milvus.proto.milvus.ShowSegmentsResponse + (*internalpb.ShowConfigurationsResponse)(nil), // 82: milvus.proto.internal.ShowConfigurationsResponse + (*milvuspb.GetMetricsResponse)(nil), // 83: milvus.proto.milvus.GetMetricsResponse + (*milvuspb.ListCredUsersResponse)(nil), // 84: milvus.proto.milvus.ListCredUsersResponse + (*milvuspb.SelectRoleResponse)(nil), // 85: milvus.proto.milvus.SelectRoleResponse + (*milvuspb.SelectUserResponse)(nil), // 86: milvus.proto.milvus.SelectUserResponse + (*milvuspb.SelectGrantResponse)(nil), // 87: milvus.proto.milvus.SelectGrantResponse + (*internalpb.ListPolicyResponse)(nil), // 88: milvus.proto.internal.ListPolicyResponse + (*milvuspb.BackupRBACMetaResponse)(nil), // 89: milvus.proto.milvus.BackupRBACMetaResponse + (*milvuspb.ListPrivilegeGroupsResponse)(nil), // 90: milvus.proto.milvus.ListPrivilegeGroupsResponse + (*milvuspb.CheckHealthResponse)(nil), // 91: milvus.proto.milvus.CheckHealthResponse + (*milvuspb.ListDatabasesResponse)(nil), // 92: milvus.proto.milvus.ListDatabasesResponse } var file_root_coord_proto_depIdxs = []int32{ - 19, // 0: milvus.proto.rootcoord.AllocTimestampRequest.base:type_name -> milvus.proto.common.MsgBase - 20, // 1: milvus.proto.rootcoord.AllocTimestampResponse.status:type_name -> milvus.proto.common.Status - 19, // 2: milvus.proto.rootcoord.AllocIDRequest.base:type_name -> milvus.proto.common.MsgBase - 20, // 3: milvus.proto.rootcoord.AllocIDResponse.status:type_name -> milvus.proto.common.Status - 19, // 4: milvus.proto.rootcoord.DescribeSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase + 22, // 0: milvus.proto.rootcoord.AllocTimestampRequest.base:type_name -> milvus.proto.common.MsgBase + 23, // 1: milvus.proto.rootcoord.AllocTimestampResponse.status:type_name -> milvus.proto.common.Status + 22, // 2: milvus.proto.rootcoord.AllocIDRequest.base:type_name -> milvus.proto.common.MsgBase + 23, // 3: milvus.proto.rootcoord.AllocIDResponse.status:type_name -> milvus.proto.common.Status + 22, // 4: milvus.proto.rootcoord.DescribeSegmentsRequest.base:type_name -> milvus.proto.common.MsgBase 5, // 5: milvus.proto.rootcoord.SegmentInfos.base_info:type_name -> milvus.proto.rootcoord.SegmentBaseInfo - 21, // 6: milvus.proto.rootcoord.SegmentInfos.index_infos:type_name -> milvus.proto.etcd.SegmentIndexInfo - 17, // 7: milvus.proto.rootcoord.SegmentInfos.extra_index_infos:type_name -> milvus.proto.rootcoord.SegmentInfos.ExtraIndexInfosEntry - 20, // 8: milvus.proto.rootcoord.DescribeSegmentsResponse.status:type_name -> milvus.proto.common.Status - 18, // 9: milvus.proto.rootcoord.DescribeSegmentsResponse.segment_infos:type_name -> milvus.proto.rootcoord.DescribeSegmentsResponse.SegmentInfosEntry - 19, // 10: milvus.proto.rootcoord.GetCredentialRequest.base:type_name -> milvus.proto.common.MsgBase - 20, // 11: milvus.proto.rootcoord.GetCredentialResponse.status:type_name -> milvus.proto.common.Status - 19, // 12: milvus.proto.rootcoord.DescribeDatabaseRequest.base:type_name -> milvus.proto.common.MsgBase - 20, // 13: milvus.proto.rootcoord.DescribeDatabaseResponse.status:type_name -> milvus.proto.common.Status - 22, // 14: milvus.proto.rootcoord.DescribeDatabaseResponse.properties:type_name -> milvus.proto.common.KeyValuePair - 19, // 15: milvus.proto.rootcoord.AlterDatabaseRequest.base:type_name -> milvus.proto.common.MsgBase - 22, // 16: milvus.proto.rootcoord.AlterDatabaseRequest.properties:type_name -> milvus.proto.common.KeyValuePair - 19, // 17: milvus.proto.rootcoord.GetPChannelInfoRequest.base:type_name -> milvus.proto.common.MsgBase - 20, // 18: milvus.proto.rootcoord.GetPChannelInfoResponse.status:type_name -> milvus.proto.common.Status + 24, // 6: milvus.proto.rootcoord.SegmentInfos.index_infos:type_name -> milvus.proto.etcd.SegmentIndexInfo + 20, // 7: milvus.proto.rootcoord.SegmentInfos.extra_index_infos:type_name -> milvus.proto.rootcoord.SegmentInfos.ExtraIndexInfosEntry + 23, // 8: milvus.proto.rootcoord.DescribeSegmentsResponse.status:type_name -> milvus.proto.common.Status + 21, // 9: milvus.proto.rootcoord.DescribeSegmentsResponse.segment_infos:type_name -> milvus.proto.rootcoord.DescribeSegmentsResponse.SegmentInfosEntry + 22, // 10: milvus.proto.rootcoord.GetCredentialRequest.base:type_name -> milvus.proto.common.MsgBase + 23, // 11: milvus.proto.rootcoord.GetCredentialResponse.status:type_name -> milvus.proto.common.Status + 22, // 12: milvus.proto.rootcoord.DescribeDatabaseRequest.base:type_name -> milvus.proto.common.MsgBase + 23, // 13: milvus.proto.rootcoord.DescribeDatabaseResponse.status:type_name -> milvus.proto.common.Status + 25, // 14: milvus.proto.rootcoord.DescribeDatabaseResponse.properties:type_name -> milvus.proto.common.KeyValuePair + 22, // 15: milvus.proto.rootcoord.AlterDatabaseRequest.base:type_name -> milvus.proto.common.MsgBase + 25, // 16: milvus.proto.rootcoord.AlterDatabaseRequest.properties:type_name -> milvus.proto.common.KeyValuePair + 22, // 17: milvus.proto.rootcoord.GetPChannelInfoRequest.base:type_name -> milvus.proto.common.MsgBase + 23, // 18: milvus.proto.rootcoord.GetPChannelInfoResponse.status:type_name -> milvus.proto.common.Status 15, // 19: milvus.proto.rootcoord.GetPChannelInfoResponse.collections:type_name -> milvus.proto.rootcoord.CollectionInfoOnPChannel 16, // 20: milvus.proto.rootcoord.CollectionInfoOnPChannel.partitions:type_name -> milvus.proto.rootcoord.PartitionInfoOnPChannel - 23, // 21: milvus.proto.rootcoord.SegmentInfos.ExtraIndexInfosEntry.value:type_name -> milvus.proto.etcd.IndexInfo - 6, // 22: milvus.proto.rootcoord.DescribeSegmentsResponse.SegmentInfosEntry.value:type_name -> milvus.proto.rootcoord.SegmentInfos - 24, // 23: milvus.proto.rootcoord.RootCoord.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest - 25, // 24: milvus.proto.rootcoord.RootCoord.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest - 26, // 25: milvus.proto.rootcoord.RootCoord.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest - 27, // 26: milvus.proto.rootcoord.RootCoord.CreateCollection:input_type -> milvus.proto.milvus.CreateCollectionRequest - 28, // 27: milvus.proto.rootcoord.RootCoord.DropCollection:input_type -> milvus.proto.milvus.DropCollectionRequest - 29, // 28: milvus.proto.rootcoord.RootCoord.HasCollection:input_type -> milvus.proto.milvus.HasCollectionRequest - 30, // 29: milvus.proto.rootcoord.RootCoord.DescribeCollection:input_type -> milvus.proto.milvus.DescribeCollectionRequest - 30, // 30: milvus.proto.rootcoord.RootCoord.DescribeCollectionInternal:input_type -> milvus.proto.milvus.DescribeCollectionRequest - 31, // 31: milvus.proto.rootcoord.RootCoord.CreateAlias:input_type -> milvus.proto.milvus.CreateAliasRequest - 32, // 32: milvus.proto.rootcoord.RootCoord.DropAlias:input_type -> milvus.proto.milvus.DropAliasRequest - 33, // 33: milvus.proto.rootcoord.RootCoord.AlterAlias:input_type -> milvus.proto.milvus.AlterAliasRequest - 34, // 34: milvus.proto.rootcoord.RootCoord.DescribeAlias:input_type -> milvus.proto.milvus.DescribeAliasRequest - 35, // 35: milvus.proto.rootcoord.RootCoord.ListAliases:input_type -> milvus.proto.milvus.ListAliasesRequest - 36, // 36: milvus.proto.rootcoord.RootCoord.ShowCollections:input_type -> milvus.proto.milvus.ShowCollectionsRequest - 37, // 37: milvus.proto.rootcoord.RootCoord.AlterCollection:input_type -> milvus.proto.milvus.AlterCollectionRequest - 38, // 38: milvus.proto.rootcoord.RootCoord.AlterCollectionField:input_type -> milvus.proto.milvus.AlterCollectionFieldRequest - 39, // 39: milvus.proto.rootcoord.RootCoord.CreatePartition:input_type -> milvus.proto.milvus.CreatePartitionRequest - 40, // 40: milvus.proto.rootcoord.RootCoord.DropPartition:input_type -> milvus.proto.milvus.DropPartitionRequest - 41, // 41: milvus.proto.rootcoord.RootCoord.HasPartition:input_type -> milvus.proto.milvus.HasPartitionRequest - 42, // 42: milvus.proto.rootcoord.RootCoord.ShowPartitions:input_type -> milvus.proto.milvus.ShowPartitionsRequest - 42, // 43: milvus.proto.rootcoord.RootCoord.ShowPartitionsInternal:input_type -> milvus.proto.milvus.ShowPartitionsRequest - 43, // 44: milvus.proto.rootcoord.RootCoord.ShowSegments:input_type -> milvus.proto.milvus.ShowSegmentsRequest - 13, // 45: milvus.proto.rootcoord.RootCoord.GetPChannelInfo:input_type -> milvus.proto.rootcoord.GetPChannelInfoRequest - 0, // 46: milvus.proto.rootcoord.RootCoord.AllocTimestamp:input_type -> milvus.proto.rootcoord.AllocTimestampRequest - 2, // 47: milvus.proto.rootcoord.RootCoord.AllocID:input_type -> milvus.proto.rootcoord.AllocIDRequest - 44, // 48: milvus.proto.rootcoord.RootCoord.UpdateChannelTimeTick:input_type -> milvus.proto.internal.ChannelTimeTickMsg - 45, // 49: milvus.proto.rootcoord.RootCoord.InvalidateCollectionMetaCache:input_type -> milvus.proto.proxy.InvalidateCollMetaCacheRequest - 46, // 50: milvus.proto.rootcoord.RootCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest - 47, // 51: milvus.proto.rootcoord.RootCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest - 48, // 52: milvus.proto.rootcoord.RootCoord.CreateCredential:input_type -> milvus.proto.internal.CredentialInfo - 48, // 53: milvus.proto.rootcoord.RootCoord.UpdateCredential:input_type -> milvus.proto.internal.CredentialInfo - 49, // 54: milvus.proto.rootcoord.RootCoord.DeleteCredential:input_type -> milvus.proto.milvus.DeleteCredentialRequest - 50, // 55: milvus.proto.rootcoord.RootCoord.ListCredUsers:input_type -> milvus.proto.milvus.ListCredUsersRequest - 8, // 56: milvus.proto.rootcoord.RootCoord.GetCredential:input_type -> milvus.proto.rootcoord.GetCredentialRequest - 51, // 57: milvus.proto.rootcoord.RootCoord.CreateRole:input_type -> milvus.proto.milvus.CreateRoleRequest - 52, // 58: milvus.proto.rootcoord.RootCoord.DropRole:input_type -> milvus.proto.milvus.DropRoleRequest - 53, // 59: milvus.proto.rootcoord.RootCoord.OperateUserRole:input_type -> milvus.proto.milvus.OperateUserRoleRequest - 54, // 60: milvus.proto.rootcoord.RootCoord.SelectRole:input_type -> milvus.proto.milvus.SelectRoleRequest - 55, // 61: milvus.proto.rootcoord.RootCoord.SelectUser:input_type -> milvus.proto.milvus.SelectUserRequest - 56, // 62: milvus.proto.rootcoord.RootCoord.OperatePrivilege:input_type -> milvus.proto.milvus.OperatePrivilegeRequest - 57, // 63: milvus.proto.rootcoord.RootCoord.SelectGrant:input_type -> milvus.proto.milvus.SelectGrantRequest - 58, // 64: milvus.proto.rootcoord.RootCoord.ListPolicy:input_type -> milvus.proto.internal.ListPolicyRequest - 59, // 65: milvus.proto.rootcoord.RootCoord.BackupRBAC:input_type -> milvus.proto.milvus.BackupRBACMetaRequest - 60, // 66: milvus.proto.rootcoord.RootCoord.RestoreRBAC:input_type -> milvus.proto.milvus.RestoreRBACMetaRequest - 61, // 67: milvus.proto.rootcoord.RootCoord.CreatePrivilegeGroup:input_type -> milvus.proto.milvus.CreatePrivilegeGroupRequest - 62, // 68: milvus.proto.rootcoord.RootCoord.DropPrivilegeGroup:input_type -> milvus.proto.milvus.DropPrivilegeGroupRequest - 63, // 69: milvus.proto.rootcoord.RootCoord.ListPrivilegeGroups:input_type -> milvus.proto.milvus.ListPrivilegeGroupsRequest - 64, // 70: milvus.proto.rootcoord.RootCoord.OperatePrivilegeGroup:input_type -> milvus.proto.milvus.OperatePrivilegeGroupRequest - 65, // 71: milvus.proto.rootcoord.RootCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest - 66, // 72: milvus.proto.rootcoord.RootCoord.RenameCollection:input_type -> milvus.proto.milvus.RenameCollectionRequest - 67, // 73: milvus.proto.rootcoord.RootCoord.CreateDatabase:input_type -> milvus.proto.milvus.CreateDatabaseRequest - 68, // 74: milvus.proto.rootcoord.RootCoord.DropDatabase:input_type -> milvus.proto.milvus.DropDatabaseRequest - 69, // 75: milvus.proto.rootcoord.RootCoord.ListDatabases:input_type -> milvus.proto.milvus.ListDatabasesRequest - 10, // 76: milvus.proto.rootcoord.RootCoord.DescribeDatabase:input_type -> milvus.proto.rootcoord.DescribeDatabaseRequest - 12, // 77: milvus.proto.rootcoord.RootCoord.AlterDatabase:input_type -> milvus.proto.rootcoord.AlterDatabaseRequest - 70, // 78: milvus.proto.rootcoord.RootCoord.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates - 71, // 79: milvus.proto.rootcoord.RootCoord.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse - 71, // 80: milvus.proto.rootcoord.RootCoord.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse - 20, // 81: milvus.proto.rootcoord.RootCoord.CreateCollection:output_type -> milvus.proto.common.Status - 20, // 82: milvus.proto.rootcoord.RootCoord.DropCollection:output_type -> milvus.proto.common.Status - 72, // 83: milvus.proto.rootcoord.RootCoord.HasCollection:output_type -> milvus.proto.milvus.BoolResponse - 73, // 84: milvus.proto.rootcoord.RootCoord.DescribeCollection:output_type -> milvus.proto.milvus.DescribeCollectionResponse - 73, // 85: milvus.proto.rootcoord.RootCoord.DescribeCollectionInternal:output_type -> milvus.proto.milvus.DescribeCollectionResponse - 20, // 86: milvus.proto.rootcoord.RootCoord.CreateAlias:output_type -> milvus.proto.common.Status - 20, // 87: milvus.proto.rootcoord.RootCoord.DropAlias:output_type -> milvus.proto.common.Status - 20, // 88: milvus.proto.rootcoord.RootCoord.AlterAlias:output_type -> milvus.proto.common.Status - 74, // 89: milvus.proto.rootcoord.RootCoord.DescribeAlias:output_type -> milvus.proto.milvus.DescribeAliasResponse - 75, // 90: milvus.proto.rootcoord.RootCoord.ListAliases:output_type -> milvus.proto.milvus.ListAliasesResponse - 76, // 91: milvus.proto.rootcoord.RootCoord.ShowCollections:output_type -> milvus.proto.milvus.ShowCollectionsResponse - 20, // 92: milvus.proto.rootcoord.RootCoord.AlterCollection:output_type -> milvus.proto.common.Status - 20, // 93: milvus.proto.rootcoord.RootCoord.AlterCollectionField:output_type -> milvus.proto.common.Status - 20, // 94: milvus.proto.rootcoord.RootCoord.CreatePartition:output_type -> milvus.proto.common.Status - 20, // 95: milvus.proto.rootcoord.RootCoord.DropPartition:output_type -> milvus.proto.common.Status - 72, // 96: milvus.proto.rootcoord.RootCoord.HasPartition:output_type -> milvus.proto.milvus.BoolResponse - 77, // 97: milvus.proto.rootcoord.RootCoord.ShowPartitions:output_type -> milvus.proto.milvus.ShowPartitionsResponse - 77, // 98: milvus.proto.rootcoord.RootCoord.ShowPartitionsInternal:output_type -> milvus.proto.milvus.ShowPartitionsResponse - 78, // 99: milvus.proto.rootcoord.RootCoord.ShowSegments:output_type -> milvus.proto.milvus.ShowSegmentsResponse - 14, // 100: milvus.proto.rootcoord.RootCoord.GetPChannelInfo:output_type -> milvus.proto.rootcoord.GetPChannelInfoResponse - 1, // 101: milvus.proto.rootcoord.RootCoord.AllocTimestamp:output_type -> milvus.proto.rootcoord.AllocTimestampResponse - 3, // 102: milvus.proto.rootcoord.RootCoord.AllocID:output_type -> milvus.proto.rootcoord.AllocIDResponse - 20, // 103: milvus.proto.rootcoord.RootCoord.UpdateChannelTimeTick:output_type -> milvus.proto.common.Status - 20, // 104: milvus.proto.rootcoord.RootCoord.InvalidateCollectionMetaCache:output_type -> milvus.proto.common.Status - 79, // 105: milvus.proto.rootcoord.RootCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse - 80, // 106: milvus.proto.rootcoord.RootCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse - 20, // 107: milvus.proto.rootcoord.RootCoord.CreateCredential:output_type -> milvus.proto.common.Status - 20, // 108: milvus.proto.rootcoord.RootCoord.UpdateCredential:output_type -> milvus.proto.common.Status - 20, // 109: milvus.proto.rootcoord.RootCoord.DeleteCredential:output_type -> milvus.proto.common.Status - 81, // 110: milvus.proto.rootcoord.RootCoord.ListCredUsers:output_type -> milvus.proto.milvus.ListCredUsersResponse - 9, // 111: milvus.proto.rootcoord.RootCoord.GetCredential:output_type -> milvus.proto.rootcoord.GetCredentialResponse - 20, // 112: milvus.proto.rootcoord.RootCoord.CreateRole:output_type -> milvus.proto.common.Status - 20, // 113: milvus.proto.rootcoord.RootCoord.DropRole:output_type -> milvus.proto.common.Status - 20, // 114: milvus.proto.rootcoord.RootCoord.OperateUserRole:output_type -> milvus.proto.common.Status - 82, // 115: milvus.proto.rootcoord.RootCoord.SelectRole:output_type -> milvus.proto.milvus.SelectRoleResponse - 83, // 116: milvus.proto.rootcoord.RootCoord.SelectUser:output_type -> milvus.proto.milvus.SelectUserResponse - 20, // 117: milvus.proto.rootcoord.RootCoord.OperatePrivilege:output_type -> milvus.proto.common.Status - 84, // 118: milvus.proto.rootcoord.RootCoord.SelectGrant:output_type -> milvus.proto.milvus.SelectGrantResponse - 85, // 119: milvus.proto.rootcoord.RootCoord.ListPolicy:output_type -> milvus.proto.internal.ListPolicyResponse - 86, // 120: milvus.proto.rootcoord.RootCoord.BackupRBAC:output_type -> milvus.proto.milvus.BackupRBACMetaResponse - 20, // 121: milvus.proto.rootcoord.RootCoord.RestoreRBAC:output_type -> milvus.proto.common.Status - 20, // 122: milvus.proto.rootcoord.RootCoord.CreatePrivilegeGroup:output_type -> milvus.proto.common.Status - 20, // 123: milvus.proto.rootcoord.RootCoord.DropPrivilegeGroup:output_type -> milvus.proto.common.Status - 87, // 124: milvus.proto.rootcoord.RootCoord.ListPrivilegeGroups:output_type -> milvus.proto.milvus.ListPrivilegeGroupsResponse - 20, // 125: milvus.proto.rootcoord.RootCoord.OperatePrivilegeGroup:output_type -> milvus.proto.common.Status - 88, // 126: milvus.proto.rootcoord.RootCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse - 20, // 127: milvus.proto.rootcoord.RootCoord.RenameCollection:output_type -> milvus.proto.common.Status - 20, // 128: milvus.proto.rootcoord.RootCoord.CreateDatabase:output_type -> milvus.proto.common.Status - 20, // 129: milvus.proto.rootcoord.RootCoord.DropDatabase:output_type -> milvus.proto.common.Status - 89, // 130: milvus.proto.rootcoord.RootCoord.ListDatabases:output_type -> milvus.proto.milvus.ListDatabasesResponse - 11, // 131: milvus.proto.rootcoord.RootCoord.DescribeDatabase:output_type -> milvus.proto.rootcoord.DescribeDatabaseResponse - 20, // 132: milvus.proto.rootcoord.RootCoord.AlterDatabase:output_type -> milvus.proto.common.Status - 78, // [78:133] is the sub-list for method output_type - 23, // [23:78] is the sub-list for method input_type - 23, // [23:23] is the sub-list for extension type_name - 23, // [23:23] is the sub-list for extension extendee - 0, // [0:23] is the sub-list for field type_name + 22, // 21: milvus.proto.rootcoord.ShowCollectionIDsRequest.base:type_name -> milvus.proto.common.MsgBase + 23, // 22: milvus.proto.rootcoord.ShowCollectionIDsResponse.status:type_name -> milvus.proto.common.Status + 18, // 23: milvus.proto.rootcoord.ShowCollectionIDsResponse.db_collections:type_name -> milvus.proto.rootcoord.DBCollections + 26, // 24: milvus.proto.rootcoord.SegmentInfos.ExtraIndexInfosEntry.value:type_name -> milvus.proto.etcd.IndexInfo + 6, // 25: milvus.proto.rootcoord.DescribeSegmentsResponse.SegmentInfosEntry.value:type_name -> milvus.proto.rootcoord.SegmentInfos + 27, // 26: milvus.proto.rootcoord.RootCoord.GetComponentStates:input_type -> milvus.proto.milvus.GetComponentStatesRequest + 28, // 27: milvus.proto.rootcoord.RootCoord.GetTimeTickChannel:input_type -> milvus.proto.internal.GetTimeTickChannelRequest + 29, // 28: milvus.proto.rootcoord.RootCoord.GetStatisticsChannel:input_type -> milvus.proto.internal.GetStatisticsChannelRequest + 30, // 29: milvus.proto.rootcoord.RootCoord.CreateCollection:input_type -> milvus.proto.milvus.CreateCollectionRequest + 31, // 30: milvus.proto.rootcoord.RootCoord.DropCollection:input_type -> milvus.proto.milvus.DropCollectionRequest + 32, // 31: milvus.proto.rootcoord.RootCoord.HasCollection:input_type -> milvus.proto.milvus.HasCollectionRequest + 33, // 32: milvus.proto.rootcoord.RootCoord.DescribeCollection:input_type -> milvus.proto.milvus.DescribeCollectionRequest + 33, // 33: milvus.proto.rootcoord.RootCoord.DescribeCollectionInternal:input_type -> milvus.proto.milvus.DescribeCollectionRequest + 34, // 34: milvus.proto.rootcoord.RootCoord.CreateAlias:input_type -> milvus.proto.milvus.CreateAliasRequest + 35, // 35: milvus.proto.rootcoord.RootCoord.DropAlias:input_type -> milvus.proto.milvus.DropAliasRequest + 36, // 36: milvus.proto.rootcoord.RootCoord.AlterAlias:input_type -> milvus.proto.milvus.AlterAliasRequest + 37, // 37: milvus.proto.rootcoord.RootCoord.DescribeAlias:input_type -> milvus.proto.milvus.DescribeAliasRequest + 38, // 38: milvus.proto.rootcoord.RootCoord.ListAliases:input_type -> milvus.proto.milvus.ListAliasesRequest + 39, // 39: milvus.proto.rootcoord.RootCoord.ShowCollections:input_type -> milvus.proto.milvus.ShowCollectionsRequest + 17, // 40: milvus.proto.rootcoord.RootCoord.ShowCollectionIDs:input_type -> milvus.proto.rootcoord.ShowCollectionIDsRequest + 40, // 41: milvus.proto.rootcoord.RootCoord.AlterCollection:input_type -> milvus.proto.milvus.AlterCollectionRequest + 41, // 42: milvus.proto.rootcoord.RootCoord.AlterCollectionField:input_type -> milvus.proto.milvus.AlterCollectionFieldRequest + 42, // 43: milvus.proto.rootcoord.RootCoord.CreatePartition:input_type -> milvus.proto.milvus.CreatePartitionRequest + 43, // 44: milvus.proto.rootcoord.RootCoord.DropPartition:input_type -> milvus.proto.milvus.DropPartitionRequest + 44, // 45: milvus.proto.rootcoord.RootCoord.HasPartition:input_type -> milvus.proto.milvus.HasPartitionRequest + 45, // 46: milvus.proto.rootcoord.RootCoord.ShowPartitions:input_type -> milvus.proto.milvus.ShowPartitionsRequest + 45, // 47: milvus.proto.rootcoord.RootCoord.ShowPartitionsInternal:input_type -> milvus.proto.milvus.ShowPartitionsRequest + 46, // 48: milvus.proto.rootcoord.RootCoord.ShowSegments:input_type -> milvus.proto.milvus.ShowSegmentsRequest + 13, // 49: milvus.proto.rootcoord.RootCoord.GetPChannelInfo:input_type -> milvus.proto.rootcoord.GetPChannelInfoRequest + 0, // 50: milvus.proto.rootcoord.RootCoord.AllocTimestamp:input_type -> milvus.proto.rootcoord.AllocTimestampRequest + 2, // 51: milvus.proto.rootcoord.RootCoord.AllocID:input_type -> milvus.proto.rootcoord.AllocIDRequest + 47, // 52: milvus.proto.rootcoord.RootCoord.UpdateChannelTimeTick:input_type -> milvus.proto.internal.ChannelTimeTickMsg + 48, // 53: milvus.proto.rootcoord.RootCoord.InvalidateCollectionMetaCache:input_type -> milvus.proto.proxy.InvalidateCollMetaCacheRequest + 49, // 54: milvus.proto.rootcoord.RootCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest + 50, // 55: milvus.proto.rootcoord.RootCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest + 51, // 56: milvus.proto.rootcoord.RootCoord.CreateCredential:input_type -> milvus.proto.internal.CredentialInfo + 51, // 57: milvus.proto.rootcoord.RootCoord.UpdateCredential:input_type -> milvus.proto.internal.CredentialInfo + 52, // 58: milvus.proto.rootcoord.RootCoord.DeleteCredential:input_type -> milvus.proto.milvus.DeleteCredentialRequest + 53, // 59: milvus.proto.rootcoord.RootCoord.ListCredUsers:input_type -> milvus.proto.milvus.ListCredUsersRequest + 8, // 60: milvus.proto.rootcoord.RootCoord.GetCredential:input_type -> milvus.proto.rootcoord.GetCredentialRequest + 54, // 61: milvus.proto.rootcoord.RootCoord.CreateRole:input_type -> milvus.proto.milvus.CreateRoleRequest + 55, // 62: milvus.proto.rootcoord.RootCoord.DropRole:input_type -> milvus.proto.milvus.DropRoleRequest + 56, // 63: milvus.proto.rootcoord.RootCoord.OperateUserRole:input_type -> milvus.proto.milvus.OperateUserRoleRequest + 57, // 64: milvus.proto.rootcoord.RootCoord.SelectRole:input_type -> milvus.proto.milvus.SelectRoleRequest + 58, // 65: milvus.proto.rootcoord.RootCoord.SelectUser:input_type -> milvus.proto.milvus.SelectUserRequest + 59, // 66: milvus.proto.rootcoord.RootCoord.OperatePrivilege:input_type -> milvus.proto.milvus.OperatePrivilegeRequest + 60, // 67: milvus.proto.rootcoord.RootCoord.SelectGrant:input_type -> milvus.proto.milvus.SelectGrantRequest + 61, // 68: milvus.proto.rootcoord.RootCoord.ListPolicy:input_type -> milvus.proto.internal.ListPolicyRequest + 62, // 69: milvus.proto.rootcoord.RootCoord.BackupRBAC:input_type -> milvus.proto.milvus.BackupRBACMetaRequest + 63, // 70: milvus.proto.rootcoord.RootCoord.RestoreRBAC:input_type -> milvus.proto.milvus.RestoreRBACMetaRequest + 64, // 71: milvus.proto.rootcoord.RootCoord.CreatePrivilegeGroup:input_type -> milvus.proto.milvus.CreatePrivilegeGroupRequest + 65, // 72: milvus.proto.rootcoord.RootCoord.DropPrivilegeGroup:input_type -> milvus.proto.milvus.DropPrivilegeGroupRequest + 66, // 73: milvus.proto.rootcoord.RootCoord.ListPrivilegeGroups:input_type -> milvus.proto.milvus.ListPrivilegeGroupsRequest + 67, // 74: milvus.proto.rootcoord.RootCoord.OperatePrivilegeGroup:input_type -> milvus.proto.milvus.OperatePrivilegeGroupRequest + 68, // 75: milvus.proto.rootcoord.RootCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest + 69, // 76: milvus.proto.rootcoord.RootCoord.RenameCollection:input_type -> milvus.proto.milvus.RenameCollectionRequest + 70, // 77: milvus.proto.rootcoord.RootCoord.CreateDatabase:input_type -> milvus.proto.milvus.CreateDatabaseRequest + 71, // 78: milvus.proto.rootcoord.RootCoord.DropDatabase:input_type -> milvus.proto.milvus.DropDatabaseRequest + 72, // 79: milvus.proto.rootcoord.RootCoord.ListDatabases:input_type -> milvus.proto.milvus.ListDatabasesRequest + 10, // 80: milvus.proto.rootcoord.RootCoord.DescribeDatabase:input_type -> milvus.proto.rootcoord.DescribeDatabaseRequest + 12, // 81: milvus.proto.rootcoord.RootCoord.AlterDatabase:input_type -> milvus.proto.rootcoord.AlterDatabaseRequest + 73, // 82: milvus.proto.rootcoord.RootCoord.GetComponentStates:output_type -> milvus.proto.milvus.ComponentStates + 74, // 83: milvus.proto.rootcoord.RootCoord.GetTimeTickChannel:output_type -> milvus.proto.milvus.StringResponse + 74, // 84: milvus.proto.rootcoord.RootCoord.GetStatisticsChannel:output_type -> milvus.proto.milvus.StringResponse + 23, // 85: milvus.proto.rootcoord.RootCoord.CreateCollection:output_type -> milvus.proto.common.Status + 23, // 86: milvus.proto.rootcoord.RootCoord.DropCollection:output_type -> milvus.proto.common.Status + 75, // 87: milvus.proto.rootcoord.RootCoord.HasCollection:output_type -> milvus.proto.milvus.BoolResponse + 76, // 88: milvus.proto.rootcoord.RootCoord.DescribeCollection:output_type -> milvus.proto.milvus.DescribeCollectionResponse + 76, // 89: milvus.proto.rootcoord.RootCoord.DescribeCollectionInternal:output_type -> milvus.proto.milvus.DescribeCollectionResponse + 23, // 90: milvus.proto.rootcoord.RootCoord.CreateAlias:output_type -> milvus.proto.common.Status + 23, // 91: milvus.proto.rootcoord.RootCoord.DropAlias:output_type -> milvus.proto.common.Status + 23, // 92: milvus.proto.rootcoord.RootCoord.AlterAlias:output_type -> milvus.proto.common.Status + 77, // 93: milvus.proto.rootcoord.RootCoord.DescribeAlias:output_type -> milvus.proto.milvus.DescribeAliasResponse + 78, // 94: milvus.proto.rootcoord.RootCoord.ListAliases:output_type -> milvus.proto.milvus.ListAliasesResponse + 79, // 95: milvus.proto.rootcoord.RootCoord.ShowCollections:output_type -> milvus.proto.milvus.ShowCollectionsResponse + 19, // 96: milvus.proto.rootcoord.RootCoord.ShowCollectionIDs:output_type -> milvus.proto.rootcoord.ShowCollectionIDsResponse + 23, // 97: milvus.proto.rootcoord.RootCoord.AlterCollection:output_type -> milvus.proto.common.Status + 23, // 98: milvus.proto.rootcoord.RootCoord.AlterCollectionField:output_type -> milvus.proto.common.Status + 23, // 99: milvus.proto.rootcoord.RootCoord.CreatePartition:output_type -> milvus.proto.common.Status + 23, // 100: milvus.proto.rootcoord.RootCoord.DropPartition:output_type -> milvus.proto.common.Status + 75, // 101: milvus.proto.rootcoord.RootCoord.HasPartition:output_type -> milvus.proto.milvus.BoolResponse + 80, // 102: milvus.proto.rootcoord.RootCoord.ShowPartitions:output_type -> milvus.proto.milvus.ShowPartitionsResponse + 80, // 103: milvus.proto.rootcoord.RootCoord.ShowPartitionsInternal:output_type -> milvus.proto.milvus.ShowPartitionsResponse + 81, // 104: milvus.proto.rootcoord.RootCoord.ShowSegments:output_type -> milvus.proto.milvus.ShowSegmentsResponse + 14, // 105: milvus.proto.rootcoord.RootCoord.GetPChannelInfo:output_type -> milvus.proto.rootcoord.GetPChannelInfoResponse + 1, // 106: milvus.proto.rootcoord.RootCoord.AllocTimestamp:output_type -> milvus.proto.rootcoord.AllocTimestampResponse + 3, // 107: milvus.proto.rootcoord.RootCoord.AllocID:output_type -> milvus.proto.rootcoord.AllocIDResponse + 23, // 108: milvus.proto.rootcoord.RootCoord.UpdateChannelTimeTick:output_type -> milvus.proto.common.Status + 23, // 109: milvus.proto.rootcoord.RootCoord.InvalidateCollectionMetaCache:output_type -> milvus.proto.common.Status + 82, // 110: milvus.proto.rootcoord.RootCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse + 83, // 111: milvus.proto.rootcoord.RootCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse + 23, // 112: milvus.proto.rootcoord.RootCoord.CreateCredential:output_type -> milvus.proto.common.Status + 23, // 113: milvus.proto.rootcoord.RootCoord.UpdateCredential:output_type -> milvus.proto.common.Status + 23, // 114: milvus.proto.rootcoord.RootCoord.DeleteCredential:output_type -> milvus.proto.common.Status + 84, // 115: milvus.proto.rootcoord.RootCoord.ListCredUsers:output_type -> milvus.proto.milvus.ListCredUsersResponse + 9, // 116: milvus.proto.rootcoord.RootCoord.GetCredential:output_type -> milvus.proto.rootcoord.GetCredentialResponse + 23, // 117: milvus.proto.rootcoord.RootCoord.CreateRole:output_type -> milvus.proto.common.Status + 23, // 118: milvus.proto.rootcoord.RootCoord.DropRole:output_type -> milvus.proto.common.Status + 23, // 119: milvus.proto.rootcoord.RootCoord.OperateUserRole:output_type -> milvus.proto.common.Status + 85, // 120: milvus.proto.rootcoord.RootCoord.SelectRole:output_type -> milvus.proto.milvus.SelectRoleResponse + 86, // 121: milvus.proto.rootcoord.RootCoord.SelectUser:output_type -> milvus.proto.milvus.SelectUserResponse + 23, // 122: milvus.proto.rootcoord.RootCoord.OperatePrivilege:output_type -> milvus.proto.common.Status + 87, // 123: milvus.proto.rootcoord.RootCoord.SelectGrant:output_type -> milvus.proto.milvus.SelectGrantResponse + 88, // 124: milvus.proto.rootcoord.RootCoord.ListPolicy:output_type -> milvus.proto.internal.ListPolicyResponse + 89, // 125: milvus.proto.rootcoord.RootCoord.BackupRBAC:output_type -> milvus.proto.milvus.BackupRBACMetaResponse + 23, // 126: milvus.proto.rootcoord.RootCoord.RestoreRBAC:output_type -> milvus.proto.common.Status + 23, // 127: milvus.proto.rootcoord.RootCoord.CreatePrivilegeGroup:output_type -> milvus.proto.common.Status + 23, // 128: milvus.proto.rootcoord.RootCoord.DropPrivilegeGroup:output_type -> milvus.proto.common.Status + 90, // 129: milvus.proto.rootcoord.RootCoord.ListPrivilegeGroups:output_type -> milvus.proto.milvus.ListPrivilegeGroupsResponse + 23, // 130: milvus.proto.rootcoord.RootCoord.OperatePrivilegeGroup:output_type -> milvus.proto.common.Status + 91, // 131: milvus.proto.rootcoord.RootCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse + 23, // 132: milvus.proto.rootcoord.RootCoord.RenameCollection:output_type -> milvus.proto.common.Status + 23, // 133: milvus.proto.rootcoord.RootCoord.CreateDatabase:output_type -> milvus.proto.common.Status + 23, // 134: milvus.proto.rootcoord.RootCoord.DropDatabase:output_type -> milvus.proto.common.Status + 92, // 135: milvus.proto.rootcoord.RootCoord.ListDatabases:output_type -> milvus.proto.milvus.ListDatabasesResponse + 11, // 136: milvus.proto.rootcoord.RootCoord.DescribeDatabase:output_type -> milvus.proto.rootcoord.DescribeDatabaseResponse + 23, // 137: milvus.proto.rootcoord.RootCoord.AlterDatabase:output_type -> milvus.proto.common.Status + 82, // [82:138] is the sub-list for method output_type + 26, // [26:82] is the sub-list for method input_type + 26, // [26:26] is the sub-list for extension type_name + 26, // [26:26] is the sub-list for extension extendee + 0, // [0:26] is the sub-list for field type_name } func init() { file_root_coord_proto_init() } @@ -2071,6 +2284,42 @@ func file_root_coord_proto_init() { return nil } } + file_root_coord_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ShowCollectionIDsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*DBCollections); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_root_coord_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ShowCollectionIDsResponse); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ @@ -2078,7 +2327,7 @@ func file_root_coord_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_root_coord_proto_rawDesc, NumEnums: 0, - NumMessages: 19, + NumMessages: 22, NumExtensions: 0, NumServices: 1, }, diff --git a/pkg/proto/rootcoordpb/root_coord_grpc.pb.go b/pkg/proto/rootcoordpb/root_coord_grpc.pb.go index 57260ee8ff..252a20af9e 100644 --- a/pkg/proto/rootcoordpb/root_coord_grpc.pb.go +++ b/pkg/proto/rootcoordpb/root_coord_grpc.pb.go @@ -37,6 +37,7 @@ const ( RootCoord_DescribeAlias_FullMethodName = "/milvus.proto.rootcoord.RootCoord/DescribeAlias" RootCoord_ListAliases_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ListAliases" RootCoord_ShowCollections_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ShowCollections" + RootCoord_ShowCollectionIDs_FullMethodName = "/milvus.proto.rootcoord.RootCoord/ShowCollectionIDs" RootCoord_AlterCollection_FullMethodName = "/milvus.proto.rootcoord.RootCoord/AlterCollection" RootCoord_AlterCollectionField_FullMethodName = "/milvus.proto.rootcoord.RootCoord/AlterCollectionField" RootCoord_CreatePartition_FullMethodName = "/milvus.proto.rootcoord.RootCoord/CreatePartition" @@ -126,6 +127,7 @@ type RootCoordClient interface { // // @return StringListResponse, collection name list ShowCollections(ctx context.Context, in *milvuspb.ShowCollectionsRequest, opts ...grpc.CallOption) (*milvuspb.ShowCollectionsResponse, error) + ShowCollectionIDs(ctx context.Context, in *ShowCollectionIDsRequest, opts ...grpc.CallOption) (*ShowCollectionIDsResponse, error) AlterCollection(ctx context.Context, in *milvuspb.AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) AlterCollectionField(ctx context.Context, in *milvuspb.AlterCollectionFieldRequest, opts ...grpc.CallOption) (*commonpb.Status, error) // * @@ -325,6 +327,15 @@ func (c *rootCoordClient) ShowCollections(ctx context.Context, in *milvuspb.Show return out, nil } +func (c *rootCoordClient) ShowCollectionIDs(ctx context.Context, in *ShowCollectionIDsRequest, opts ...grpc.CallOption) (*ShowCollectionIDsResponse, error) { + out := new(ShowCollectionIDsResponse) + err := c.cc.Invoke(ctx, RootCoord_ShowCollectionIDs_FullMethodName, in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + func (c *rootCoordClient) AlterCollection(ctx context.Context, in *milvuspb.AlterCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) { out := new(commonpb.Status) err := c.cc.Invoke(ctx, RootCoord_AlterCollection_FullMethodName, in, out, opts...) @@ -740,6 +751,7 @@ type RootCoordServer interface { // // @return StringListResponse, collection name list ShowCollections(context.Context, *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) + ShowCollectionIDs(context.Context, *ShowCollectionIDsRequest) (*ShowCollectionIDsResponse, error) AlterCollection(context.Context, *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) AlterCollectionField(context.Context, *milvuspb.AlterCollectionFieldRequest) (*commonpb.Status, error) // * @@ -851,6 +863,9 @@ func (UnimplementedRootCoordServer) ListAliases(context.Context, *milvuspb.ListA func (UnimplementedRootCoordServer) ShowCollections(context.Context, *milvuspb.ShowCollectionsRequest) (*milvuspb.ShowCollectionsResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method ShowCollections not implemented") } +func (UnimplementedRootCoordServer) ShowCollectionIDs(context.Context, *ShowCollectionIDsRequest) (*ShowCollectionIDsResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method ShowCollectionIDs not implemented") +} func (UnimplementedRootCoordServer) AlterCollection(context.Context, *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) { return nil, status.Errorf(codes.Unimplemented, "method AlterCollection not implemented") } @@ -1238,6 +1253,24 @@ func _RootCoord_ShowCollections_Handler(srv interface{}, ctx context.Context, de return interceptor(ctx, in, info, handler) } +func _RootCoord_ShowCollectionIDs_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(ShowCollectionIDsRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(RootCoordServer).ShowCollectionIDs(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: RootCoord_ShowCollectionIDs_FullMethodName, + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(RootCoordServer).ShowCollectionIDs(ctx, req.(*ShowCollectionIDsRequest)) + } + return interceptor(ctx, in, info, handler) +} + func _RootCoord_AlterCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(milvuspb.AlterCollectionRequest) if err := dec(in); err != nil { @@ -2039,6 +2072,10 @@ var RootCoord_ServiceDesc = grpc.ServiceDesc{ MethodName: "ShowCollections", Handler: _RootCoord_ShowCollections_Handler, }, + { + MethodName: "ShowCollectionIDs", + Handler: _RootCoord_ShowCollectionIDs_Handler, + }, { MethodName: "AlterCollection", Handler: _RootCoord_AlterCollection_Handler, diff --git a/pkg/util/paramtable/service_param.go b/pkg/util/paramtable/service_param.go index d648611bb0..65bb356e1b 100644 --- a/pkg/util/paramtable/service_param.go +++ b/pkg/util/paramtable/service_param.go @@ -459,6 +459,7 @@ type MetaStoreConfig struct { SnapshotTTLSeconds ParamItem `refreshable:"true"` SnapshotReserveTimeSeconds ParamItem `refreshable:"true"` PaginationSize ParamItem `refreshable:"true"` + ReadConcurrency ParamItem `refreshable:"true"` } func (p *MetaStoreConfig) Init(base *BaseTable) { @@ -492,11 +493,19 @@ func (p *MetaStoreConfig) Init(base *BaseTable) { p.PaginationSize = ParamItem{ Key: "metastore.paginationSize", Version: "2.5.1", - DefaultValue: "10000", + DefaultValue: "100000", Doc: `limits the number of results to return from metastore.`, } p.PaginationSize.Init(base.mgr) + p.ReadConcurrency = ParamItem{ + Key: "metastore.readConcurrency", + Version: "2.5.1", + DefaultValue: "32", + Doc: `read concurrency for fetching metadata from the metastore.`, + } + p.ReadConcurrency.Init(base.mgr) + // TODO: The initialization operation of metadata storage is called in the initialization phase of every node. // There should be a single initialization operation for meta store, then move the metrics registration to there. metrics.RegisterMetaType(p.MetaStoreType.GetValue()) diff --git a/pkg/util/paramtable/service_param_test.go b/pkg/util/paramtable/service_param_test.go index 863b36a89f..8a1aed83ca 100644 --- a/pkg/util/paramtable/service_param_test.go +++ b/pkg/util/paramtable/service_param_test.go @@ -230,7 +230,8 @@ func TestServiceParam(t *testing.T) { assert.Equal(t, util.MetaStoreTypeEtcd, Params.MetaStoreType.GetValue()) assert.Equal(t, 86400*time.Second, Params.SnapshotTTLSeconds.GetAsDuration(time.Second)) assert.Equal(t, 3600*time.Second, Params.SnapshotReserveTimeSeconds.GetAsDuration(time.Second)) - assert.Equal(t, 10000, Params.PaginationSize.GetAsInt()) + assert.Equal(t, 100000, Params.PaginationSize.GetAsInt()) + assert.Equal(t, 32, Params.ReadConcurrency.GetAsInt()) }) }