From b1a1cca10b5af6091354dcbdd2e52fd3384950de Mon Sep 17 00:00:00 2001 From: SimFG Date: Thu, 28 Mar 2024 06:33:11 +0800 Subject: [PATCH] feat: add more operation detail info for better allocation (#30438) issue: #30436 --------- Signed-off-by: SimFG --- .golangci.yml | 5 +- go.mod | 17 +- internal/core/src/common/QueryResult.h | 2 + .../query/visitors/ExecPlanNodeVisitor.cpp | 6 + internal/core/src/segcore/Reduce.cpp | 4 + .../core/src/segcore/SegmentInterface.cpp | 1 + internal/datacoord/index_builder.go | 6 +- internal/datacoord/metrics_info.go | 25 + internal/datacoord/segment_manager_test.go | 4 +- internal/datacoord/server.go | 7 +- internal/datacoord/server_test.go | 65 + internal/datacoord/services.go | 17 + internal/datacoord/session_manager.go | 1 - internal/datanode/data_sync_service_test.go | 24 +- .../proxy/httpserver/handler_v1_test.go | 30 +- .../proxy/httpserver/utils_test.go | 20 +- .../kv/querycoord/kv_catalog_test.go | 6 +- internal/metastore/model/segment.go | 4 +- internal/proto/internal.proto | 2 + ...emove time travel ralted testcase (#26119) | 2132 ----------------- internal/proto/segcore.proto | 1 + .../proxy/authentication_interceptor_test.go | 3 +- internal/proxy/channels_mgr.go | 1 - internal/proxy/count_reducer.go | 2 + internal/proxy/hook_interceptor.go | 94 +- internal/proxy/hook_interceptor_test.go | 33 +- internal/proxy/impl.go | 61 +- internal/proxy/meta_cache.go | 98 +- internal/proxy/meta_cache_test.go | 146 ++ internal/proxy/mock_cache.go | 64 + internal/proxy/proxy.go | 105 +- internal/proxy/proxy_test.go | 231 +- internal/proxy/search_reduce_util.go | 2 + internal/proxy/task_delete.go | 9 +- internal/proxy/task_hybrid_search.go | 7 + internal/proxy/task_query.go | 17 +- internal/proxy/task_scheduler.go | 1 - internal/proxy/task_search.go | 6 + internal/proxy/util.go | 24 +- .../querynodev2/segments/count_reducer.go | 12 +- internal/querynodev2/segments/result.go | 3 + internal/querynodev2/segments/retrieve.go | 7 +- internal/querynodev2/segments/segment.go | 4 +- internal/querynodev2/tasks/query_task.go | 1 + internal/rootcoord/constrant.go | 4 +- internal/storage/utils.go | 8 +- internal/storage/utils_test.go | 8 +- internal/util/hookutil/constant.go | 43 + internal/util/hookutil/default.go | 72 + internal/util/hookutil/hook.go | 102 + internal/util/hookutil/hook_test.go | 52 + internal/util/typeutil/result_helper.go | 4 +- pkg/go.mod | 2 +- pkg/go.sum | 4 +- pkg/util/metricsinfo/metric_type.go | 3 + pkg/util/typeutil/get_dim.go | 11 + .../hellomilvus/hello_milvus_test.go | 113 +- tests/integration/minicluster_v2.go | 34 + .../partitionkey/partition_key_test.go | 411 ++++ tests/integration/suite.go | 19 + tests/integration/util_insert.go | 46 +- 61 files changed, 1891 insertions(+), 2355 deletions(-) delete mode 100644 internal/proto/internalpb/internal.pb.go~parent of ca1349708... Remove time travel ralted testcase (#26119) create mode 100644 internal/util/hookutil/constant.go create mode 100644 internal/util/hookutil/default.go create mode 100644 internal/util/hookutil/hook.go create mode 100644 internal/util/hookutil/hook_test.go create mode 100644 tests/integration/partitionkey/partition_key_test.go diff --git a/.golangci.yml b/.golangci.yml index c584ff7e2e..9ef9fc5260 100644 --- a/.golangci.yml +++ b/.golangci.yml @@ -140,6 +140,8 @@ issues: - which can be annoying to use # Binds to all network interfaces - G102 + # Use of unsafe calls should be audited + - G103 # Errors unhandled - G104 # file/folder Permission @@ -164,4 +166,5 @@ issues: max-same-issues: 0 service: - golangci-lint-version: 1.55.2 # use the fixed version to not introduce new linters unexpectedly + # use the fixed version to not introduce new linters unexpectedly + golangci-lint-version: 1.55.2 diff --git a/go.mod b/go.mod index 96a1b79059..f1856164d6 100644 --- a/go.mod +++ b/go.mod @@ -9,6 +9,7 @@ require ( github.com/Azure/azure-sdk-for-go/sdk/storage/azblob v1.1.0 github.com/aliyun/credentials-go v1.2.7 github.com/antlr/antlr4/runtime/Go/antlr v0.0.0-20210826220005-b48c857c3a0e + github.com/apache/arrow/go/v12 v12.0.1 github.com/apache/pulsar-client-go v0.6.1-0.20210728062540-29414db801a7 github.com/bits-and-blooms/bloom/v3 v3.0.1 github.com/blang/semver/v4 v4.0.0 @@ -17,6 +18,7 @@ require ( github.com/cockroachdb/errors v1.9.1 github.com/containerd/cgroups/v3 v3.0.3 // indirect github.com/gin-gonic/gin v1.9.1 + github.com/go-playground/validator/v10 v10.14.0 github.com/gofrs/flock v0.8.1 github.com/gogo/protobuf v1.3.2 github.com/golang/protobuf v1.5.3 @@ -26,9 +28,11 @@ require ( github.com/mgutz/ansi v0.0.0-20200706080929-d51e80ef957d github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240317125658-67a0f065c1de github.com/minio/minio-go/v7 v7.0.61 + github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81 github.com/prometheus/client_golang v1.14.0 github.com/prometheus/client_model v0.3.0 github.com/prometheus/common v0.42.0 + github.com/quasilyte/go-ruleguard/dsl v0.3.22 github.com/samber/lo v1.27.0 github.com/sbinet/npyio v0.6.0 github.com/soheilhy/cmux v0.1.5 @@ -36,6 +40,7 @@ require ( github.com/spf13/viper v1.8.1 github.com/stretchr/testify v1.8.4 github.com/tecbot/gorocksdb v0.0.0-20191217155057-f0fad39f321c + github.com/tencentcloud/tencentcloud-sdk-go/tencentcloud/common v1.0.865 github.com/tidwall/gjson v1.14.4 github.com/tikv/client-go/v2 v2.0.4 go.etcd.io/etcd/api/v3 v3.5.5 @@ -49,6 +54,7 @@ require ( go.uber.org/zap v1.24.0 golang.org/x/crypto v0.16.0 golang.org/x/exp v0.0.0-20230728194245-b0cb94b80691 + golang.org/x/net v0.19.0 golang.org/x/oauth2 v0.8.0 golang.org/x/sync v0.5.0 golang.org/x/text v0.14.0 @@ -56,18 +62,9 @@ require ( google.golang.org/grpc/examples v0.0.0-20220617181431-3e7b97febc7f ) -require github.com/apache/arrow/go/v12 v12.0.1 - require github.com/milvus-io/milvus-storage/go v0.0.0-20231227072638-ebd0b8e56d70 -require ( - github.com/go-playground/validator/v10 v10.14.0 - github.com/milvus-io/milvus/pkg v0.0.0-00010101000000-000000000000 - github.com/pingcap/log v1.1.1-0.20221015072633-39906604fb81 - github.com/quasilyte/go-ruleguard/dsl v0.3.22 - github.com/tencentcloud/tencentcloud-sdk-go/tencentcloud/common v1.0.865 - golang.org/x/net v0.19.0 -) +require github.com/milvus-io/milvus/pkg v0.0.0-00010101000000-000000000000 require ( cloud.google.com/go/compute v1.20.1 // indirect diff --git a/internal/core/src/common/QueryResult.h b/internal/core/src/common/QueryResult.h index 657df423d3..e516a9f115 100644 --- a/internal/core/src/common/QueryResult.h +++ b/internal/core/src/common/QueryResult.h @@ -189,6 +189,7 @@ struct SearchResult { public: int64_t total_nq_; int64_t unity_topK_; + int64_t total_data_cnt_; void* segment_; // first fill data during search, and then update data after reducing search results @@ -223,6 +224,7 @@ struct RetrieveResult { RetrieveResult() = default; public: + int64_t total_data_cnt_; void* segment_; std::vector result_offsets_; std::vector field_data_; diff --git a/internal/core/src/query/visitors/ExecPlanNodeVisitor.cpp b/internal/core/src/query/visitors/ExecPlanNodeVisitor.cpp index c98ba32960..ccab290ac9 100644 --- a/internal/core/src/query/visitors/ExecPlanNodeVisitor.cpp +++ b/internal/core/src/query/visitors/ExecPlanNodeVisitor.cpp @@ -71,6 +71,7 @@ empty_search_result(int64_t num_queries, SearchInfo& search_info) { SearchResult final_result; final_result.total_nq_ = num_queries; final_result.unity_topK_ = 0; // no result + final_result.total_data_cnt_ = 0; return final_result; } @@ -212,6 +213,7 @@ ExecPlanNodeVisitor::VectorVisitorImpl(VectorPlanNode& node) { timestamp_, final_view, search_result); + search_result.total_data_cnt_ = final_view.size(); if (search_result.vector_iterators_.has_value()) { std::vector group_by_values; GroupBy(search_result.vector_iterators_.value(), @@ -239,6 +241,7 @@ wrap_num_entities(int64_t cnt) { auto scalar = arr.mutable_scalars(); scalar->mutable_long_data()->mutable_data()->Add(cnt); retrieve_result->field_data_ = {arr}; + retrieve_result->total_data_cnt_ = 0; return retrieve_result; } @@ -249,6 +252,7 @@ ExecPlanNodeVisitor::visit(RetrievePlanNode& node) { dynamic_cast(&segment_); AssertInfo(segment, "Support SegmentSmallIndex Only"); RetrieveResult retrieve_result; + retrieve_result.total_data_cnt_ = 0; auto active_count = segment->get_active_count(timestamp_); @@ -295,10 +299,12 @@ ExecPlanNodeVisitor::visit(RetrievePlanNode& node) { if (node.is_count_) { auto cnt = bitset_holder.size() - bitset_holder.count(); retrieve_result = *(wrap_num_entities(cnt)); + retrieve_result.total_data_cnt_ = bitset_holder.size(); retrieve_result_opt_ = std::move(retrieve_result); return; } + retrieve_result.total_data_cnt_ = bitset_holder.size(); bool false_filtered_out = false; if (get_cache_offset) { segment->timestamp_filter(bitset_holder, cache_offsets, timestamp_); diff --git a/internal/core/src/segcore/Reduce.cpp b/internal/core/src/segcore/Reduce.cpp index 7215e8ba21..5bf4830fa6 100644 --- a/internal/core/src/segcore/Reduce.cpp +++ b/internal/core/src/segcore/Reduce.cpp @@ -348,12 +348,14 @@ ReduceHelper::GetSearchResultDataSlice(int slice_index) { auto nq_end = slice_nqs_prefix_sum_[slice_index + 1]; int64_t result_count = 0; + int64_t all_search_count = 0; for (auto search_result : search_results_) { AssertInfo(search_result->topk_per_nq_prefix_sum_.size() == search_result->total_nq_ + 1, "incorrect topk_per_nq_prefix_sum_ size in search result"); result_count += search_result->topk_per_nq_prefix_sum_[nq_end] - search_result->topk_per_nq_prefix_sum_[nq_begin]; + all_search_count += search_result->total_data_cnt_; } auto search_result_data = @@ -363,6 +365,8 @@ ReduceHelper::GetSearchResultDataSlice(int slice_index) { search_result_data->set_num_queries(nq_end - nq_begin); search_result_data->mutable_topks()->Resize(nq_end - nq_begin, 0); + search_result_data->set_all_search_count(all_search_count); + // `result_pairs` contains the SearchResult and result_offset info, used for filling output fields std::vector> result_pairs(result_count); diff --git a/internal/core/src/segcore/SegmentInterface.cpp b/internal/core/src/segcore/SegmentInterface.cpp index 962e86eedf..8f44515b1c 100644 --- a/internal/core/src/segcore/SegmentInterface.cpp +++ b/internal/core/src/segcore/SegmentInterface.cpp @@ -100,6 +100,7 @@ SegmentInternalInterface::Retrieve(const query::RetrievePlan* plan, fmt::format("query results exceed the limit size ", limit_size)); } + results->set_all_retrieve_count(retrieve_results.total_data_cnt_); if (plan->plan_node_->is_count_) { AssertInfo(retrieve_results.field_data_.size() == 1, "count result should only have one column"); diff --git a/internal/datacoord/index_builder.go b/internal/datacoord/index_builder.go index 70fd1a98c8..4a3f258489 100644 --- a/internal/datacoord/index_builder.go +++ b/internal/datacoord/index_builder.go @@ -205,7 +205,7 @@ func (ib *indexBuilder) run() { } } -func getBinLogIds(segment *SegmentInfo, fieldID int64) []int64 { +func getBinLogIDs(segment *SegmentInfo, fieldID int64) []int64 { binlogIDs := make([]int64, 0) for _, fieldBinLog := range segment.GetBinlogs() { if fieldBinLog.GetFieldID() == fieldID { @@ -299,7 +299,7 @@ func (ib *indexBuilder) process(buildID UniqueID) bool { FieldID: partitionKeyField.FieldID, FieldName: partitionKeyField.Name, FieldType: int32(partitionKeyField.DataType), - DataIds: getBinLogIds(segment, partitionKeyField.FieldID), + DataIds: getBinLogIDs(segment, partitionKeyField.FieldID), }) } } @@ -333,7 +333,7 @@ func (ib *indexBuilder) process(buildID UniqueID) bool { } fieldID := ib.meta.indexMeta.GetFieldIDByIndexID(meta.CollectionID, meta.IndexID) - binlogIDs := getBinLogIds(segment, fieldID) + binlogIDs := getBinLogIDs(segment, fieldID) if isDiskANNIndex(GetIndexType(indexParams)) { var err error indexParams, err = indexparams.UpdateDiskIndexBuildParams(Params, indexParams) diff --git a/internal/datacoord/metrics_info.go b/internal/datacoord/metrics_info.go index 78b4890af8..360004927a 100644 --- a/internal/datacoord/metrics_info.go +++ b/internal/datacoord/metrics_info.go @@ -139,6 +139,31 @@ func (s *Server) getSystemInfoMetrics( return resp, nil } +func (s *Server) getCollectionStorageMetrics(ctx context.Context) (*milvuspb.GetMetricsResponse, error) { + coordTopology := metricsinfo.DataCoordTopology{ + Cluster: metricsinfo.DataClusterTopology{ + Self: s.getDataCoordMetrics(ctx), + }, + Connections: metricsinfo.ConnTopology{ + Name: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, paramtable.GetNodeID()), + ConnectedComponents: []metricsinfo.ConnectionInfo{}, + }, + } + + resp := &milvuspb.GetMetricsResponse{ + Status: merr.Success(), + ComponentName: metricsinfo.ConstructComponentName(typeutil.DataCoordRole, paramtable.GetNodeID()), + } + var err error + resp.Response, err = metricsinfo.MarshalTopology(coordTopology) + if err != nil { + resp.Status = merr.Status(err) + return resp, nil + } + + return resp, nil +} + // getDataCoordMetrics composes datacoord infos func (s *Server) getDataCoordMetrics(ctx context.Context) metricsinfo.DataCoordInfos { ret := metricsinfo.DataCoordInfos{ diff --git a/internal/datacoord/segment_manager_test.go b/internal/datacoord/segment_manager_test.go index 9337561704..69cb83d12f 100644 --- a/internal/datacoord/segment_manager_test.go +++ b/internal/datacoord/segment_manager_test.go @@ -236,8 +236,8 @@ func TestLastExpireReset(t *testing.T) { assert.Equal(t, expire1, segment1.GetLastExpireTime()) assert.Equal(t, expire2, segment2.GetLastExpireTime()) assert.True(t, segment3.GetLastExpireTime() > expire3) - flushableSegIds, _ := newSegmentManager.GetFlushableSegments(context.Background(), channelName, expire3) - assert.ElementsMatch(t, []UniqueID{segmentID1, segmentID2}, flushableSegIds) // segment1 and segment2 can be flushed + flushableSegIDs, _ := newSegmentManager.GetFlushableSegments(context.Background(), channelName, expire3) + assert.ElementsMatch(t, []UniqueID{segmentID1, segmentID2}, flushableSegIDs) // segment1 and segment2 can be flushed newAlloc, err := newSegmentManager.AllocSegment(context.Background(), collID, 0, channelName, 2000) assert.Nil(t, err) assert.Equal(t, segmentID3, newAlloc[0].SegmentID) // segment3 still can be used to allocate diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 9b5fb7993c..4605f10ca1 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -94,7 +94,7 @@ type rootCoordCreatorFunc func(ctx context.Context) (types.RootCoordClient, erro // makes sure Server implements `DataCoord` var _ types.DataCoord = (*Server)(nil) -var Params *paramtable.ComponentParam = paramtable.Get() +var Params = paramtable.Get() // Server implements `types.DataCoord` // handles Data Coordinator related jobs @@ -161,6 +161,11 @@ type Server struct { broker broker.Broker } +type CollectionNameInfo struct { + CollectionName string + DBName string +} + // ServerHelper datacoord server injection helper type ServerHelper struct { eventAfterHandleDataNodeTt func() diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index 6f3742017e..1d221b1507 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -34,6 +34,7 @@ import ( "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" clientv3 "go.etcd.io/etcd/client/v3" + "go.uber.org/atomic" "go.uber.org/zap" "google.golang.org/grpc" @@ -3836,3 +3837,67 @@ func TestUpdateAutoBalanceConfigLoop(t *testing.T) { wg.Wait() }) } + +func TestGetCollectionStorage(t *testing.T) { + paramtable.Init() + mockSession := sessionutil.NewMockSession(t) + mockSession.EXPECT().GetAddress().Return("localhost:8888") + size := atomic.NewInt64(100) + + s := &Server{ + session: mockSession, + meta: &meta{ + segments: &SegmentsInfo{ + segments: map[UniqueID]*SegmentInfo{ + 1: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + State: commonpb.SegmentState_Growing, + CollectionID: 10001, + PartitionID: 10000, + NumOfRows: 10, + }, + size: *size, + }, + 2: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 2, + State: commonpb.SegmentState_Dropped, + CollectionID: 10001, + PartitionID: 10000, + NumOfRows: 10, + }, + size: *size, + }, + 3: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 3, + State: commonpb.SegmentState_Flushed, + CollectionID: 10002, + PartitionID: 9999, + NumOfRows: 10, + }, + size: *size, + }, + }, + }, + }, + } + s.stateCode.Store(commonpb.StateCode_Healthy) + + req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.CollectionStorageMetrics) + assert.NoError(t, err) + resp, err := s.GetMetrics(context.TODO(), req) + assert.NoError(t, err) + + var coordTopology metricsinfo.DataCoordTopology + err = metricsinfo.UnmarshalTopology(resp.Response, &coordTopology) + assert.NoError(t, err) + + m := coordTopology.Cluster.Self.QuotaMetrics + assert.NotNil(t, m) + assert.Equal(t, int64(200), m.TotalBinlogSize) + assert.Len(t, m.CollectionBinlogSize, 2) + assert.Equal(t, int64(100), m.CollectionBinlogSize[10001]) + assert.Equal(t, int64(100), m.CollectionBinlogSize[10002]) +} diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index 03e2982b09..dce783af52 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -1029,6 +1029,23 @@ func (s *Server) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest zap.Any("metrics", metrics), // TODO(dragondriver): necessary? may be very large zap.Error(err)) + return metrics, nil + } else if metricType == metricsinfo.CollectionStorageMetrics { + metrics, err := s.getCollectionStorageMetrics(ctx) + if err != nil { + log.Warn("DataCoord GetMetrics CollectionStorage failed", zap.Int64("nodeID", paramtable.GetNodeID()), zap.Error(err)) + return &milvuspb.GetMetricsResponse{ + Status: merr.Status(err), + }, nil + } + + log.RatedDebug(60, "DataCoord.GetMetrics CollectionStorage", + zap.Int64("nodeID", paramtable.GetNodeID()), + zap.String("req", req.Request), + zap.String("metricType", metricType), + zap.Any("metrics", metrics), + zap.Error(err)) + return metrics, nil } diff --git a/internal/datacoord/session_manager.go b/internal/datacoord/session_manager.go index 45049ba941..934d3d5399 100644 --- a/internal/datacoord/session_manager.go +++ b/internal/datacoord/session_manager.go @@ -232,7 +232,6 @@ func (c *SessionManagerImpl) SyncSegments(nodeID int64, req *datapb.SyncSegments } return nil }) - if err != nil { log.Warn("failed to sync segments after retry", zap.Error(err)) return err diff --git a/internal/datanode/data_sync_service_test.go b/internal/datanode/data_sync_service_test.go index aaba16a3be..55c9f58fac 100644 --- a/internal/datanode/data_sync_service_test.go +++ b/internal/datanode/data_sync_service_test.go @@ -108,20 +108,20 @@ func getVchanInfo(info *testInfo) *datapb.VchannelInfo { ufs = []*datapb.SegmentInfo{} } - var ufsIds []int64 - var fsIds []int64 + var ufsIDs []int64 + var fsIDs []int64 for _, segmentInfo := range ufs { - ufsIds = append(ufsIds, segmentInfo.ID) + ufsIDs = append(ufsIDs, segmentInfo.ID) } for _, segmentInfo := range fs { - fsIds = append(fsIds, segmentInfo.ID) + fsIDs = append(fsIDs, segmentInfo.ID) } vi := &datapb.VchannelInfo{ CollectionID: info.collID, ChannelName: info.chanName, SeekPosition: &msgpb.MsgPosition{}, - UnflushedSegmentIds: ufsIds, - FlushedSegmentIds: fsIds, + UnflushedSegmentIds: ufsIDs, + FlushedSegmentIds: fsIDs, } return vi } @@ -465,13 +465,13 @@ func (s *DataSyncServiceSuite) TestStartStop() { NumOfRows: 0, DmlPosition: &msgpb.MsgPosition{}, }} - var ufsIds []int64 - var fsIds []int64 + var ufsIDs []int64 + var fsIDs []int64 for _, segmentInfo := range ufs { - ufsIds = append(ufsIds, segmentInfo.ID) + ufsIDs = append(ufsIDs, segmentInfo.ID) } for _, segmentInfo := range fs { - fsIds = append(fsIds, segmentInfo.ID) + fsIDs = append(fsIDs, segmentInfo.ID) } watchInfo := &datapb.ChannelWatchInfo{ @@ -479,8 +479,8 @@ func (s *DataSyncServiceSuite) TestStartStop() { Vchan: &datapb.VchannelInfo{ CollectionID: collMeta.ID, ChannelName: insertChannelName, - UnflushedSegmentIds: ufsIds, - FlushedSegmentIds: fsIds, + UnflushedSegmentIds: ufsIDs, + FlushedSegmentIds: fsIDs, }, } diff --git a/internal/distributed/proxy/httpserver/handler_v1_test.go b/internal/distributed/proxy/httpserver/handler_v1_test.go index 8c071fdb1e..521a817cfe 100644 --- a/internal/distributed/proxy/httpserver/handler_v1_test.go +++ b/internal/distributed/proxy/httpserver/handler_v1_test.go @@ -691,7 +691,7 @@ func TestInsert(t *testing.T) { mp5, _ = wrapWithDescribeColl(t, mp5, ReturnSuccess, 1, nil) mp5.EXPECT().Insert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(schemapb.DataType_Int64), + IDs: genIDs(schemapb.DataType_Int64), InsertCnt: 3, }, nil).Once() testCases = append(testCases, testCase{ @@ -705,7 +705,7 @@ func TestInsert(t *testing.T) { mp6, _ = wrapWithDescribeColl(t, mp6, ReturnSuccess, 1, nil) mp6.EXPECT().Insert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(schemapb.DataType_VarChar), + IDs: genIDs(schemapb.DataType_VarChar), InsertCnt: 3, }, nil).Once() testCases = append(testCases, testCase{ @@ -776,7 +776,7 @@ func TestInsertForDataType(t *testing.T) { }, nil).Once() mp.EXPECT().Insert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(schemapb.DataType_Int64), + IDs: genIDs(schemapb.DataType_Int64), InsertCnt: 3, }, nil).Once() testEngine := initHTTPServer(mp, true) @@ -844,7 +844,7 @@ func TestReturnInt64(t *testing.T) { }, nil).Once() mp.EXPECT().Insert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(dataType), + IDs: genIDs(dataType), InsertCnt: 3, }, nil).Once() testEngine := initHTTPServer(mp, true) @@ -875,7 +875,7 @@ func TestReturnInt64(t *testing.T) { }, nil).Once() mp.EXPECT().Upsert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(dataType), + IDs: genIDs(dataType), UpsertCnt: 3, }, nil).Once() testEngine := initHTTPServer(mp, true) @@ -906,7 +906,7 @@ func TestReturnInt64(t *testing.T) { }, nil).Once() mp.EXPECT().Insert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(dataType), + IDs: genIDs(dataType), InsertCnt: 3, }, nil).Once() testEngine := initHTTPServer(mp, true) @@ -938,7 +938,7 @@ func TestReturnInt64(t *testing.T) { }, nil).Once() mp.EXPECT().Upsert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(dataType), + IDs: genIDs(dataType), UpsertCnt: 3, }, nil).Once() testEngine := initHTTPServer(mp, true) @@ -971,7 +971,7 @@ func TestReturnInt64(t *testing.T) { }, nil).Once() mp.EXPECT().Insert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(dataType), + IDs: genIDs(dataType), InsertCnt: 3, }, nil).Once() testEngine := initHTTPServer(mp, true) @@ -1002,7 +1002,7 @@ func TestReturnInt64(t *testing.T) { }, nil).Once() mp.EXPECT().Upsert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(dataType), + IDs: genIDs(dataType), UpsertCnt: 3, }, nil).Once() testEngine := initHTTPServer(mp, true) @@ -1033,7 +1033,7 @@ func TestReturnInt64(t *testing.T) { }, nil).Once() mp.EXPECT().Insert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(dataType), + IDs: genIDs(dataType), InsertCnt: 3, }, nil).Once() testEngine := initHTTPServer(mp, true) @@ -1065,7 +1065,7 @@ func TestReturnInt64(t *testing.T) { }, nil).Once() mp.EXPECT().Upsert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(dataType), + IDs: genIDs(dataType), UpsertCnt: 3, }, nil).Once() testEngine := initHTTPServer(mp, true) @@ -1132,7 +1132,7 @@ func TestUpsert(t *testing.T) { mp5, _ = wrapWithDescribeColl(t, mp5, ReturnSuccess, 1, nil) mp5.EXPECT().Upsert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(schemapb.DataType_Int64), + IDs: genIDs(schemapb.DataType_Int64), UpsertCnt: 3, }, nil).Once() testCases = append(testCases, testCase{ @@ -1146,7 +1146,7 @@ func TestUpsert(t *testing.T) { mp6, _ = wrapWithDescribeColl(t, mp6, ReturnSuccess, 1, nil) mp6.EXPECT().Upsert(mock.Anything, mock.Anything).Return(&milvuspb.MutationResult{ Status: &StatusSuccess, - IDs: genIds(schemapb.DataType_VarChar), + IDs: genIDs(schemapb.DataType_VarChar), UpsertCnt: 3, }, nil).Once() testCases = append(testCases, testCase{ @@ -1198,8 +1198,8 @@ func TestUpsert(t *testing.T) { }) } -func genIds(dataType schemapb.DataType) *schemapb.IDs { - return generateIds(dataType, 3) +func genIDs(dataType schemapb.DataType) *schemapb.IDs { + return generateIDs(dataType, 3) } func TestSearch(t *testing.T) { diff --git a/internal/distributed/proxy/httpserver/utils_test.go b/internal/distributed/proxy/httpserver/utils_test.go index bda148c3e2..945783c335 100644 --- a/internal/distributed/proxy/httpserver/utils_test.go +++ b/internal/distributed/proxy/httpserver/utils_test.go @@ -37,7 +37,7 @@ func generatePrimaryField(datatype schemapb.DataType) schemapb.FieldSchema { } } -func generateIds(dataType schemapb.DataType, num int) *schemapb.IDs { +func generateIDs(dataType schemapb.DataType, num int) *schemapb.IDs { var intArray []int64 if num == 0 { intArray = []int64{} @@ -684,7 +684,7 @@ func compareRows(row1 []map[string]interface{}, row2 []map[string]interface{}, c func TestBuildQueryResp(t *testing.T) { outputFields := []string{FieldBookID, FieldWordCount, "author", "date"} - rows, err := buildQueryResp(int64(0), outputFields, generateFieldData(), generateIds(schemapb.DataType_Int64, 3), DefaultScores, true) // []*schemapb.FieldData{&fieldData1, &fieldData2, &fieldData3} + rows, err := buildQueryResp(int64(0), outputFields, generateFieldData(), generateIDs(schemapb.DataType_Int64, 3), DefaultScores, true) // []*schemapb.FieldData{&fieldData1, &fieldData2, &fieldData3} assert.Equal(t, nil, err) exceptRows := generateSearchResult(schemapb.DataType_Int64) assert.Equal(t, true, compareRows(rows, exceptRows, compareRow)) @@ -1298,7 +1298,7 @@ func TestBuildQueryResps(t *testing.T) { outputFields := []string{"XXX", "YYY"} outputFieldsList := [][]string{outputFields, {"$meta"}, {"$meta", FieldBookID, FieldBookIntro, "YYY"}} for _, theOutputFields := range outputFieldsList { - rows, err := buildQueryResp(int64(0), theOutputFields, newFieldData(generateFieldData(), schemapb.DataType_None), generateIds(schemapb.DataType_Int64, 3), DefaultScores, true) + rows, err := buildQueryResp(int64(0), theOutputFields, newFieldData(generateFieldData(), schemapb.DataType_None), generateIDs(schemapb.DataType_Int64, 3), DefaultScores, true) assert.Equal(t, nil, err) exceptRows := newSearchResult(generateSearchResult(schemapb.DataType_Int64)) assert.Equal(t, true, compareRows(rows, exceptRows, compareRow)) @@ -1312,29 +1312,29 @@ func TestBuildQueryResps(t *testing.T) { schemapb.DataType_JSON, schemapb.DataType_Array, } for _, dateType := range dataTypes { - _, err := buildQueryResp(int64(0), outputFields, newFieldData([]*schemapb.FieldData{}, dateType), generateIds(schemapb.DataType_Int64, 3), DefaultScores, true) + _, err := buildQueryResp(int64(0), outputFields, newFieldData([]*schemapb.FieldData{}, dateType), generateIDs(schemapb.DataType_Int64, 3), DefaultScores, true) assert.Equal(t, nil, err) } - _, err := buildQueryResp(int64(0), outputFields, newFieldData([]*schemapb.FieldData{}, 1000), generateIds(schemapb.DataType_Int64, 3), DefaultScores, true) + _, err := buildQueryResp(int64(0), outputFields, newFieldData([]*schemapb.FieldData{}, 1000), generateIDs(schemapb.DataType_Int64, 3), DefaultScores, true) assert.Equal(t, "the type(1000) of field(wrong-field-type) is not supported, use other sdk please", err.Error()) - res, err := buildQueryResp(int64(0), outputFields, []*schemapb.FieldData{}, generateIds(schemapb.DataType_Int64, 3), DefaultScores, true) + res, err := buildQueryResp(int64(0), outputFields, []*schemapb.FieldData{}, generateIDs(schemapb.DataType_Int64, 3), DefaultScores, true) assert.Equal(t, 3, len(res)) assert.Equal(t, nil, err) - res, err = buildQueryResp(int64(0), outputFields, []*schemapb.FieldData{}, generateIds(schemapb.DataType_Int64, 3), DefaultScores, false) + res, err = buildQueryResp(int64(0), outputFields, []*schemapb.FieldData{}, generateIDs(schemapb.DataType_Int64, 3), DefaultScores, false) assert.Equal(t, 3, len(res)) assert.Equal(t, nil, err) - res, err = buildQueryResp(int64(0), outputFields, []*schemapb.FieldData{}, generateIds(schemapb.DataType_VarChar, 3), DefaultScores, true) + res, err = buildQueryResp(int64(0), outputFields, []*schemapb.FieldData{}, generateIDs(schemapb.DataType_VarChar, 3), DefaultScores, true) assert.Equal(t, 3, len(res)) assert.Equal(t, nil, err) - _, err = buildQueryResp(int64(0), outputFields, generateFieldData(), generateIds(schemapb.DataType_Int64, 3), DefaultScores, false) + _, err = buildQueryResp(int64(0), outputFields, generateFieldData(), generateIDs(schemapb.DataType_Int64, 3), DefaultScores, false) assert.Equal(t, nil, err) // len(rows) != len(scores), didn't show distance - _, err = buildQueryResp(int64(0), outputFields, newFieldData(generateFieldData(), schemapb.DataType_None), generateIds(schemapb.DataType_Int64, 3), []float32{0.01, 0.04}, true) + _, err = buildQueryResp(int64(0), outputFields, newFieldData(generateFieldData(), schemapb.DataType_None), generateIDs(schemapb.DataType_Int64, 3), []float32{0.01, 0.04}, true) assert.Equal(t, nil, err) } diff --git a/internal/metastore/kv/querycoord/kv_catalog_test.go b/internal/metastore/kv/querycoord/kv_catalog_test.go index 8b4bba050f..001ac310fe 100644 --- a/internal/metastore/kv/querycoord/kv_catalog_test.go +++ b/internal/metastore/kv/querycoord/kv_catalog_test.go @@ -125,16 +125,16 @@ func (suite *CatalogTestSuite) TestPartition() { } func (suite *CatalogTestSuite) TestReleaseManyPartitions() { - partitionIds := make([]int64, 0) + partitionIDs := make([]int64, 0) for i := 1; i <= 150; i++ { suite.catalog.SavePartition(&querypb.PartitionLoadInfo{ CollectionID: 1, PartitionID: int64(i), }) - partitionIds = append(partitionIds, int64(i)) + partitionIDs = append(partitionIDs, int64(i)) } - err := suite.catalog.ReleasePartition(1, partitionIds...) + err := suite.catalog.ReleasePartition(1, partitionIDs...) suite.NoError(err) partitions, err := suite.catalog.GetPartitions() suite.NoError(err) diff --git a/internal/metastore/model/segment.go b/internal/metastore/model/segment.go index 5c119ec2ad..3479299cd7 100644 --- a/internal/metastore/model/segment.go +++ b/internal/metastore/model/segment.go @@ -16,6 +16,6 @@ type Segment struct { CreatedByCompaction bool SegmentState commonpb.SegmentState // IndexInfos []*SegmentIndex - ReplicaIds []int64 - NodeIds []int64 + ReplicaIDs []int64 + NodeIDs []int64 } diff --git a/internal/proto/internal.proto b/internal/proto/internal.proto index 0daf4c207d..002b86c390 100644 --- a/internal/proto/internal.proto +++ b/internal/proto/internal.proto @@ -134,6 +134,7 @@ message SearchResults { // search request cost CostAggregation costAggregation = 13; map channels_mvcc = 14; + int64 all_search_count = 15; } message CostAggregation { @@ -175,6 +176,7 @@ message RetrieveResults { // query request cost CostAggregation costAggregation = 13; + int64 all_retrieve_count = 14; } message LoadIndex { diff --git a/internal/proto/internalpb/internal.pb.go~parent of ca1349708... Remove time travel ralted testcase (#26119) b/internal/proto/internalpb/internal.pb.go~parent of ca1349708... Remove time travel ralted testcase (#26119) deleted file mode 100644 index 65268d2d6e..0000000000 --- a/internal/proto/internalpb/internal.pb.go~parent of ca1349708... Remove time travel ralted testcase (#26119) +++ /dev/null @@ -1,2132 +0,0 @@ -// Code generated by protoc-gen-go. DO NOT EDIT. -// source: internal.proto - -package internalpb - -import ( - fmt "fmt" - proto "github.com/golang/protobuf/proto" - commonpb "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" - schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - math "math" -) - -// Reference imports to suppress errors if they are not otherwise used. -var _ = proto.Marshal -var _ = fmt.Errorf -var _ = math.Inf - -// This is a compile-time assertion to ensure that this generated file -// is compatible with the proto package it is being compiled against. -// A compilation error at this line likely means your copy of the -// proto package needs to be updated. -const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package - -type RateType int32 - -const ( - RateType_DDLCollection RateType = 0 - RateType_DDLPartition RateType = 1 - RateType_DDLIndex RateType = 2 - RateType_DDLFlush RateType = 3 - RateType_DDLCompaction RateType = 4 - RateType_DMLInsert RateType = 5 - RateType_DMLDelete RateType = 6 - RateType_DMLBulkLoad RateType = 7 - RateType_DQLSearch RateType = 8 - RateType_DQLQuery RateType = 9 - RateType_DMLUpsert RateType = 10 -) - -var RateType_name = map[int32]string{ - 0: "DDLCollection", - 1: "DDLPartition", - 2: "DDLIndex", - 3: "DDLFlush", - 4: "DDLCompaction", - 5: "DMLInsert", - 6: "DMLDelete", - 7: "DMLBulkLoad", - 8: "DQLSearch", - 9: "DQLQuery", - 10: "DMLUpsert", -} - -var RateType_value = map[string]int32{ - "DDLCollection": 0, - "DDLPartition": 1, - "DDLIndex": 2, - "DDLFlush": 3, - "DDLCompaction": 4, - "DMLInsert": 5, - "DMLDelete": 6, - "DMLBulkLoad": 7, - "DQLSearch": 8, - "DQLQuery": 9, - "DMLUpsert": 10, -} - -func (x RateType) String() string { - return proto.EnumName(RateType_name, int32(x)) -} - -func (RateType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{0} -} - -type GetTimeTickChannelRequest struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *GetTimeTickChannelRequest) Reset() { *m = GetTimeTickChannelRequest{} } -func (m *GetTimeTickChannelRequest) String() string { return proto.CompactTextString(m) } -func (*GetTimeTickChannelRequest) ProtoMessage() {} -func (*GetTimeTickChannelRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{0} -} - -func (m *GetTimeTickChannelRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_GetTimeTickChannelRequest.Unmarshal(m, b) -} -func (m *GetTimeTickChannelRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_GetTimeTickChannelRequest.Marshal(b, m, deterministic) -} -func (m *GetTimeTickChannelRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetTimeTickChannelRequest.Merge(m, src) -} -func (m *GetTimeTickChannelRequest) XXX_Size() int { - return xxx_messageInfo_GetTimeTickChannelRequest.Size(m) -} -func (m *GetTimeTickChannelRequest) XXX_DiscardUnknown() { - xxx_messageInfo_GetTimeTickChannelRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_GetTimeTickChannelRequest proto.InternalMessageInfo - -type GetStatisticsChannelRequest struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *GetStatisticsChannelRequest) Reset() { *m = GetStatisticsChannelRequest{} } -func (m *GetStatisticsChannelRequest) String() string { return proto.CompactTextString(m) } -func (*GetStatisticsChannelRequest) ProtoMessage() {} -func (*GetStatisticsChannelRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{1} -} - -func (m *GetStatisticsChannelRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_GetStatisticsChannelRequest.Unmarshal(m, b) -} -func (m *GetStatisticsChannelRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_GetStatisticsChannelRequest.Marshal(b, m, deterministic) -} -func (m *GetStatisticsChannelRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetStatisticsChannelRequest.Merge(m, src) -} -func (m *GetStatisticsChannelRequest) XXX_Size() int { - return xxx_messageInfo_GetStatisticsChannelRequest.Size(m) -} -func (m *GetStatisticsChannelRequest) XXX_DiscardUnknown() { - xxx_messageInfo_GetStatisticsChannelRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_GetStatisticsChannelRequest proto.InternalMessageInfo - -type GetDdChannelRequest struct { - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *GetDdChannelRequest) Reset() { *m = GetDdChannelRequest{} } -func (m *GetDdChannelRequest) String() string { return proto.CompactTextString(m) } -func (*GetDdChannelRequest) ProtoMessage() {} -func (*GetDdChannelRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{2} -} - -func (m *GetDdChannelRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_GetDdChannelRequest.Unmarshal(m, b) -} -func (m *GetDdChannelRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_GetDdChannelRequest.Marshal(b, m, deterministic) -} -func (m *GetDdChannelRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetDdChannelRequest.Merge(m, src) -} -func (m *GetDdChannelRequest) XXX_Size() int { - return xxx_messageInfo_GetDdChannelRequest.Size(m) -} -func (m *GetDdChannelRequest) XXX_DiscardUnknown() { - xxx_messageInfo_GetDdChannelRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_GetDdChannelRequest proto.InternalMessageInfo - -type NodeInfo struct { - Address *commonpb.Address `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"` - Role string `protobuf:"bytes,2,opt,name=role,proto3" json:"role,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *NodeInfo) Reset() { *m = NodeInfo{} } -func (m *NodeInfo) String() string { return proto.CompactTextString(m) } -func (*NodeInfo) ProtoMessage() {} -func (*NodeInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{3} -} - -func (m *NodeInfo) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_NodeInfo.Unmarshal(m, b) -} -func (m *NodeInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_NodeInfo.Marshal(b, m, deterministic) -} -func (m *NodeInfo) XXX_Merge(src proto.Message) { - xxx_messageInfo_NodeInfo.Merge(m, src) -} -func (m *NodeInfo) XXX_Size() int { - return xxx_messageInfo_NodeInfo.Size(m) -} -func (m *NodeInfo) XXX_DiscardUnknown() { - xxx_messageInfo_NodeInfo.DiscardUnknown(m) -} - -var xxx_messageInfo_NodeInfo proto.InternalMessageInfo - -func (m *NodeInfo) GetAddress() *commonpb.Address { - if m != nil { - return m.Address - } - return nil -} - -func (m *NodeInfo) GetRole() string { - if m != nil { - return m.Role - } - return "" -} - -type InitParams struct { - NodeID int64 `protobuf:"varint,1,opt,name=nodeID,proto3" json:"nodeID,omitempty"` - StartParams []*commonpb.KeyValuePair `protobuf:"bytes,2,rep,name=start_params,json=startParams,proto3" json:"start_params,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *InitParams) Reset() { *m = InitParams{} } -func (m *InitParams) String() string { return proto.CompactTextString(m) } -func (*InitParams) ProtoMessage() {} -func (*InitParams) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{4} -} - -func (m *InitParams) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_InitParams.Unmarshal(m, b) -} -func (m *InitParams) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_InitParams.Marshal(b, m, deterministic) -} -func (m *InitParams) XXX_Merge(src proto.Message) { - xxx_messageInfo_InitParams.Merge(m, src) -} -func (m *InitParams) XXX_Size() int { - return xxx_messageInfo_InitParams.Size(m) -} -func (m *InitParams) XXX_DiscardUnknown() { - xxx_messageInfo_InitParams.DiscardUnknown(m) -} - -var xxx_messageInfo_InitParams proto.InternalMessageInfo - -func (m *InitParams) GetNodeID() int64 { - if m != nil { - return m.NodeID - } - return 0 -} - -func (m *InitParams) GetStartParams() []*commonpb.KeyValuePair { - if m != nil { - return m.StartParams - } - return nil -} - -type StringList struct { - Values []string `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"` - Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *StringList) Reset() { *m = StringList{} } -func (m *StringList) String() string { return proto.CompactTextString(m) } -func (*StringList) ProtoMessage() {} -func (*StringList) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{5} -} - -func (m *StringList) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_StringList.Unmarshal(m, b) -} -func (m *StringList) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_StringList.Marshal(b, m, deterministic) -} -func (m *StringList) XXX_Merge(src proto.Message) { - xxx_messageInfo_StringList.Merge(m, src) -} -func (m *StringList) XXX_Size() int { - return xxx_messageInfo_StringList.Size(m) -} -func (m *StringList) XXX_DiscardUnknown() { - xxx_messageInfo_StringList.DiscardUnknown(m) -} - -var xxx_messageInfo_StringList proto.InternalMessageInfo - -func (m *StringList) GetValues() []string { - if m != nil { - return m.Values - } - return nil -} - -func (m *StringList) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - -type GetStatisticsRequest struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - // Not useful for now - DbID int64 `protobuf:"varint,2,opt,name=dbID,proto3" json:"dbID,omitempty"` - // The collection you want get statistics - CollectionID int64 `protobuf:"varint,3,opt,name=collectionID,proto3" json:"collectionID,omitempty"` - // The partitions you want get statistics - PartitionIDs []int64 `protobuf:"varint,4,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` - // timestamp of the statistics - TravelTimestamp uint64 `protobuf:"varint,5,opt,name=travel_timestamp,json=travelTimestamp,proto3" json:"travel_timestamp,omitempty"` - GuaranteeTimestamp uint64 `protobuf:"varint,6,opt,name=guarantee_timestamp,json=guaranteeTimestamp,proto3" json:"guarantee_timestamp,omitempty"` - TimeoutTimestamp uint64 `protobuf:"varint,7,opt,name=timeout_timestamp,json=timeoutTimestamp,proto3" json:"timeout_timestamp,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *GetStatisticsRequest) Reset() { *m = GetStatisticsRequest{} } -func (m *GetStatisticsRequest) String() string { return proto.CompactTextString(m) } -func (*GetStatisticsRequest) ProtoMessage() {} -func (*GetStatisticsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{6} -} - -func (m *GetStatisticsRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_GetStatisticsRequest.Unmarshal(m, b) -} -func (m *GetStatisticsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_GetStatisticsRequest.Marshal(b, m, deterministic) -} -func (m *GetStatisticsRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetStatisticsRequest.Merge(m, src) -} -func (m *GetStatisticsRequest) XXX_Size() int { - return xxx_messageInfo_GetStatisticsRequest.Size(m) -} -func (m *GetStatisticsRequest) XXX_DiscardUnknown() { - xxx_messageInfo_GetStatisticsRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_GetStatisticsRequest proto.InternalMessageInfo - -func (m *GetStatisticsRequest) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *GetStatisticsRequest) GetDbID() int64 { - if m != nil { - return m.DbID - } - return 0 -} - -func (m *GetStatisticsRequest) GetCollectionID() int64 { - if m != nil { - return m.CollectionID - } - return 0 -} - -func (m *GetStatisticsRequest) GetPartitionIDs() []int64 { - if m != nil { - return m.PartitionIDs - } - return nil -} - -func (m *GetStatisticsRequest) GetTravelTimestamp() uint64 { - if m != nil { - return m.TravelTimestamp - } - return 0 -} - -func (m *GetStatisticsRequest) GetGuaranteeTimestamp() uint64 { - if m != nil { - return m.GuaranteeTimestamp - } - return 0 -} - -func (m *GetStatisticsRequest) GetTimeoutTimestamp() uint64 { - if m != nil { - return m.TimeoutTimestamp - } - return 0 -} - -type GetStatisticsResponse struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - // Contain error_code and reason - Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` - // Collection statistics data. Contain pairs like {"row_count": "1"} - Stats []*commonpb.KeyValuePair `protobuf:"bytes,3,rep,name=stats,proto3" json:"stats,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *GetStatisticsResponse) Reset() { *m = GetStatisticsResponse{} } -func (m *GetStatisticsResponse) String() string { return proto.CompactTextString(m) } -func (*GetStatisticsResponse) ProtoMessage() {} -func (*GetStatisticsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{7} -} - -func (m *GetStatisticsResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_GetStatisticsResponse.Unmarshal(m, b) -} -func (m *GetStatisticsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_GetStatisticsResponse.Marshal(b, m, deterministic) -} -func (m *GetStatisticsResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_GetStatisticsResponse.Merge(m, src) -} -func (m *GetStatisticsResponse) XXX_Size() int { - return xxx_messageInfo_GetStatisticsResponse.Size(m) -} -func (m *GetStatisticsResponse) XXX_DiscardUnknown() { - xxx_messageInfo_GetStatisticsResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_GetStatisticsResponse proto.InternalMessageInfo - -func (m *GetStatisticsResponse) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *GetStatisticsResponse) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - -func (m *GetStatisticsResponse) GetStats() []*commonpb.KeyValuePair { - if m != nil { - return m.Stats - } - return nil -} - -type CreateAliasRequest struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` - CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` - Alias string `protobuf:"bytes,4,opt,name=alias,proto3" json:"alias,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CreateAliasRequest) Reset() { *m = CreateAliasRequest{} } -func (m *CreateAliasRequest) String() string { return proto.CompactTextString(m) } -func (*CreateAliasRequest) ProtoMessage() {} -func (*CreateAliasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{8} -} - -func (m *CreateAliasRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CreateAliasRequest.Unmarshal(m, b) -} -func (m *CreateAliasRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CreateAliasRequest.Marshal(b, m, deterministic) -} -func (m *CreateAliasRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_CreateAliasRequest.Merge(m, src) -} -func (m *CreateAliasRequest) XXX_Size() int { - return xxx_messageInfo_CreateAliasRequest.Size(m) -} -func (m *CreateAliasRequest) XXX_DiscardUnknown() { - xxx_messageInfo_CreateAliasRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_CreateAliasRequest proto.InternalMessageInfo - -func (m *CreateAliasRequest) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *CreateAliasRequest) GetDbName() string { - if m != nil { - return m.DbName - } - return "" -} - -func (m *CreateAliasRequest) GetCollectionName() string { - if m != nil { - return m.CollectionName - } - return "" -} - -func (m *CreateAliasRequest) GetAlias() string { - if m != nil { - return m.Alias - } - return "" -} - -type DropAliasRequest struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` - Alias string `protobuf:"bytes,3,opt,name=alias,proto3" json:"alias,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *DropAliasRequest) Reset() { *m = DropAliasRequest{} } -func (m *DropAliasRequest) String() string { return proto.CompactTextString(m) } -func (*DropAliasRequest) ProtoMessage() {} -func (*DropAliasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{9} -} - -func (m *DropAliasRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_DropAliasRequest.Unmarshal(m, b) -} -func (m *DropAliasRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_DropAliasRequest.Marshal(b, m, deterministic) -} -func (m *DropAliasRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_DropAliasRequest.Merge(m, src) -} -func (m *DropAliasRequest) XXX_Size() int { - return xxx_messageInfo_DropAliasRequest.Size(m) -} -func (m *DropAliasRequest) XXX_DiscardUnknown() { - xxx_messageInfo_DropAliasRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_DropAliasRequest proto.InternalMessageInfo - -func (m *DropAliasRequest) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *DropAliasRequest) GetDbName() string { - if m != nil { - return m.DbName - } - return "" -} - -func (m *DropAliasRequest) GetAlias() string { - if m != nil { - return m.Alias - } - return "" -} - -type AlterAliasRequest struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` - CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` - Alias string `protobuf:"bytes,4,opt,name=alias,proto3" json:"alias,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *AlterAliasRequest) Reset() { *m = AlterAliasRequest{} } -func (m *AlterAliasRequest) String() string { return proto.CompactTextString(m) } -func (*AlterAliasRequest) ProtoMessage() {} -func (*AlterAliasRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{10} -} - -func (m *AlterAliasRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_AlterAliasRequest.Unmarshal(m, b) -} -func (m *AlterAliasRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_AlterAliasRequest.Marshal(b, m, deterministic) -} -func (m *AlterAliasRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_AlterAliasRequest.Merge(m, src) -} -func (m *AlterAliasRequest) XXX_Size() int { - return xxx_messageInfo_AlterAliasRequest.Size(m) -} -func (m *AlterAliasRequest) XXX_DiscardUnknown() { - xxx_messageInfo_AlterAliasRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_AlterAliasRequest proto.InternalMessageInfo - -func (m *AlterAliasRequest) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *AlterAliasRequest) GetDbName() string { - if m != nil { - return m.DbName - } - return "" -} - -func (m *AlterAliasRequest) GetCollectionName() string { - if m != nil { - return m.CollectionName - } - return "" -} - -func (m *AlterAliasRequest) GetAlias() string { - if m != nil { - return m.Alias - } - return "" -} - -type CreateIndexRequest struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - DbName string `protobuf:"bytes,2,opt,name=db_name,json=dbName,proto3" json:"db_name,omitempty"` - CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` - FieldName string `protobuf:"bytes,4,opt,name=field_name,json=fieldName,proto3" json:"field_name,omitempty"` - DbID int64 `protobuf:"varint,5,opt,name=dbID,proto3" json:"dbID,omitempty"` - CollectionID int64 `protobuf:"varint,6,opt,name=collectionID,proto3" json:"collectionID,omitempty"` - FieldID int64 `protobuf:"varint,7,opt,name=fieldID,proto3" json:"fieldID,omitempty"` - ExtraParams []*commonpb.KeyValuePair `protobuf:"bytes,8,rep,name=extra_params,json=extraParams,proto3" json:"extra_params,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CreateIndexRequest) Reset() { *m = CreateIndexRequest{} } -func (m *CreateIndexRequest) String() string { return proto.CompactTextString(m) } -func (*CreateIndexRequest) ProtoMessage() {} -func (*CreateIndexRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{11} -} - -func (m *CreateIndexRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CreateIndexRequest.Unmarshal(m, b) -} -func (m *CreateIndexRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CreateIndexRequest.Marshal(b, m, deterministic) -} -func (m *CreateIndexRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_CreateIndexRequest.Merge(m, src) -} -func (m *CreateIndexRequest) XXX_Size() int { - return xxx_messageInfo_CreateIndexRequest.Size(m) -} -func (m *CreateIndexRequest) XXX_DiscardUnknown() { - xxx_messageInfo_CreateIndexRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_CreateIndexRequest proto.InternalMessageInfo - -func (m *CreateIndexRequest) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *CreateIndexRequest) GetDbName() string { - if m != nil { - return m.DbName - } - return "" -} - -func (m *CreateIndexRequest) GetCollectionName() string { - if m != nil { - return m.CollectionName - } - return "" -} - -func (m *CreateIndexRequest) GetFieldName() string { - if m != nil { - return m.FieldName - } - return "" -} - -func (m *CreateIndexRequest) GetDbID() int64 { - if m != nil { - return m.DbID - } - return 0 -} - -func (m *CreateIndexRequest) GetCollectionID() int64 { - if m != nil { - return m.CollectionID - } - return 0 -} - -func (m *CreateIndexRequest) GetFieldID() int64 { - if m != nil { - return m.FieldID - } - return 0 -} - -func (m *CreateIndexRequest) GetExtraParams() []*commonpb.KeyValuePair { - if m != nil { - return m.ExtraParams - } - return nil -} - -type SearchRequest struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - ReqID int64 `protobuf:"varint,2,opt,name=reqID,proto3" json:"reqID,omitempty"` - DbID int64 `protobuf:"varint,3,opt,name=dbID,proto3" json:"dbID,omitempty"` - CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` - PartitionIDs []int64 `protobuf:"varint,5,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` - Dsl string `protobuf:"bytes,6,opt,name=dsl,proto3" json:"dsl,omitempty"` - // serialized `PlaceholderGroup` - PlaceholderGroup []byte `protobuf:"bytes,7,opt,name=placeholder_group,json=placeholderGroup,proto3" json:"placeholder_group,omitempty"` - DslType commonpb.DslType `protobuf:"varint,8,opt,name=dsl_type,json=dslType,proto3,enum=milvus.proto.common.DslType" json:"dsl_type,omitempty"` - SerializedExprPlan []byte `protobuf:"bytes,9,opt,name=serialized_expr_plan,json=serializedExprPlan,proto3" json:"serialized_expr_plan,omitempty"` - OutputFieldsId []int64 `protobuf:"varint,10,rep,packed,name=output_fields_id,json=outputFieldsId,proto3" json:"output_fields_id,omitempty"` - TravelTimestamp uint64 `protobuf:"varint,11,opt,name=travel_timestamp,json=travelTimestamp,proto3" json:"travel_timestamp,omitempty"` - GuaranteeTimestamp uint64 `protobuf:"varint,12,opt,name=guarantee_timestamp,json=guaranteeTimestamp,proto3" json:"guarantee_timestamp,omitempty"` - TimeoutTimestamp uint64 `protobuf:"varint,13,opt,name=timeout_timestamp,json=timeoutTimestamp,proto3" json:"timeout_timestamp,omitempty"` - Nq int64 `protobuf:"varint,14,opt,name=nq,proto3" json:"nq,omitempty"` - Topk int64 `protobuf:"varint,15,opt,name=topk,proto3" json:"topk,omitempty"` - MetricType string `protobuf:"bytes,16,opt,name=metricType,proto3" json:"metricType,omitempty"` - IgnoreGrowing bool `protobuf:"varint,17,opt,name=ignoreGrowing,proto3" json:"ignoreGrowing,omitempty"` - Username string `protobuf:"bytes,18,opt,name=username,proto3" json:"username,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *SearchRequest) Reset() { *m = SearchRequest{} } -func (m *SearchRequest) String() string { return proto.CompactTextString(m) } -func (*SearchRequest) ProtoMessage() {} -func (*SearchRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{12} -} - -func (m *SearchRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SearchRequest.Unmarshal(m, b) -} -func (m *SearchRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SearchRequest.Marshal(b, m, deterministic) -} -func (m *SearchRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_SearchRequest.Merge(m, src) -} -func (m *SearchRequest) XXX_Size() int { - return xxx_messageInfo_SearchRequest.Size(m) -} -func (m *SearchRequest) XXX_DiscardUnknown() { - xxx_messageInfo_SearchRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_SearchRequest proto.InternalMessageInfo - -func (m *SearchRequest) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *SearchRequest) GetReqID() int64 { - if m != nil { - return m.ReqID - } - return 0 -} - -func (m *SearchRequest) GetDbID() int64 { - if m != nil { - return m.DbID - } - return 0 -} - -func (m *SearchRequest) GetCollectionID() int64 { - if m != nil { - return m.CollectionID - } - return 0 -} - -func (m *SearchRequest) GetPartitionIDs() []int64 { - if m != nil { - return m.PartitionIDs - } - return nil -} - -func (m *SearchRequest) GetDsl() string { - if m != nil { - return m.Dsl - } - return "" -} - -func (m *SearchRequest) GetPlaceholderGroup() []byte { - if m != nil { - return m.PlaceholderGroup - } - return nil -} - -func (m *SearchRequest) GetDslType() commonpb.DslType { - if m != nil { - return m.DslType - } - return commonpb.DslType_Dsl -} - -func (m *SearchRequest) GetSerializedExprPlan() []byte { - if m != nil { - return m.SerializedExprPlan - } - return nil -} - -func (m *SearchRequest) GetOutputFieldsId() []int64 { - if m != nil { - return m.OutputFieldsId - } - return nil -} - -func (m *SearchRequest) GetTravelTimestamp() uint64 { - if m != nil { - return m.TravelTimestamp - } - return 0 -} - -func (m *SearchRequest) GetGuaranteeTimestamp() uint64 { - if m != nil { - return m.GuaranteeTimestamp - } - return 0 -} - -func (m *SearchRequest) GetTimeoutTimestamp() uint64 { - if m != nil { - return m.TimeoutTimestamp - } - return 0 -} - -func (m *SearchRequest) GetNq() int64 { - if m != nil { - return m.Nq - } - return 0 -} - -func (m *SearchRequest) GetTopk() int64 { - if m != nil { - return m.Topk - } - return 0 -} - -func (m *SearchRequest) GetMetricType() string { - if m != nil { - return m.MetricType - } - return "" -} - -func (m *SearchRequest) GetIgnoreGrowing() bool { - if m != nil { - return m.IgnoreGrowing - } - return false -} - -func (m *SearchRequest) GetUsername() string { - if m != nil { - return m.Username - } - return "" -} - -type SearchResults struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` - ReqID int64 `protobuf:"varint,3,opt,name=reqID,proto3" json:"reqID,omitempty"` - MetricType string `protobuf:"bytes,4,opt,name=metric_type,json=metricType,proto3" json:"metric_type,omitempty"` - NumQueries int64 `protobuf:"varint,5,opt,name=num_queries,json=numQueries,proto3" json:"num_queries,omitempty"` - TopK int64 `protobuf:"varint,6,opt,name=top_k,json=topK,proto3" json:"top_k,omitempty"` - SealedSegmentIDsSearched []int64 `protobuf:"varint,7,rep,packed,name=sealed_segmentIDs_searched,json=sealedSegmentIDsSearched,proto3" json:"sealed_segmentIDs_searched,omitempty"` - ChannelIDsSearched []string `protobuf:"bytes,8,rep,name=channelIDs_searched,json=channelIDsSearched,proto3" json:"channelIDs_searched,omitempty"` - GlobalSealedSegmentIDs []int64 `protobuf:"varint,9,rep,packed,name=global_sealed_segmentIDs,json=globalSealedSegmentIDs,proto3" json:"global_sealed_segmentIDs,omitempty"` - // schema.SearchResultsData inside - SlicedBlob []byte `protobuf:"bytes,10,opt,name=sliced_blob,json=slicedBlob,proto3" json:"sliced_blob,omitempty"` - SlicedNumCount int64 `protobuf:"varint,11,opt,name=sliced_num_count,json=slicedNumCount,proto3" json:"sliced_num_count,omitempty"` - SlicedOffset int64 `protobuf:"varint,12,opt,name=sliced_offset,json=slicedOffset,proto3" json:"sliced_offset,omitempty"` - // search request cost - CostAggregation *CostAggregation `protobuf:"bytes,13,opt,name=costAggregation,proto3" json:"costAggregation,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *SearchResults) Reset() { *m = SearchResults{} } -func (m *SearchResults) String() string { return proto.CompactTextString(m) } -func (*SearchResults) ProtoMessage() {} -func (*SearchResults) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{13} -} - -func (m *SearchResults) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SearchResults.Unmarshal(m, b) -} -func (m *SearchResults) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SearchResults.Marshal(b, m, deterministic) -} -func (m *SearchResults) XXX_Merge(src proto.Message) { - xxx_messageInfo_SearchResults.Merge(m, src) -} -func (m *SearchResults) XXX_Size() int { - return xxx_messageInfo_SearchResults.Size(m) -} -func (m *SearchResults) XXX_DiscardUnknown() { - xxx_messageInfo_SearchResults.DiscardUnknown(m) -} - -var xxx_messageInfo_SearchResults proto.InternalMessageInfo - -func (m *SearchResults) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *SearchResults) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - -func (m *SearchResults) GetReqID() int64 { - if m != nil { - return m.ReqID - } - return 0 -} - -func (m *SearchResults) GetMetricType() string { - if m != nil { - return m.MetricType - } - return "" -} - -func (m *SearchResults) GetNumQueries() int64 { - if m != nil { - return m.NumQueries - } - return 0 -} - -func (m *SearchResults) GetTopK() int64 { - if m != nil { - return m.TopK - } - return 0 -} - -func (m *SearchResults) GetSealedSegmentIDsSearched() []int64 { - if m != nil { - return m.SealedSegmentIDsSearched - } - return nil -} - -func (m *SearchResults) GetChannelIDsSearched() []string { - if m != nil { - return m.ChannelIDsSearched - } - return nil -} - -func (m *SearchResults) GetGlobalSealedSegmentIDs() []int64 { - if m != nil { - return m.GlobalSealedSegmentIDs - } - return nil -} - -func (m *SearchResults) GetSlicedBlob() []byte { - if m != nil { - return m.SlicedBlob - } - return nil -} - -func (m *SearchResults) GetSlicedNumCount() int64 { - if m != nil { - return m.SlicedNumCount - } - return 0 -} - -func (m *SearchResults) GetSlicedOffset() int64 { - if m != nil { - return m.SlicedOffset - } - return 0 -} - -func (m *SearchResults) GetCostAggregation() *CostAggregation { - if m != nil { - return m.CostAggregation - } - return nil -} - -type CostAggregation struct { - ResponseTime int64 `protobuf:"varint,1,opt,name=responseTime,proto3" json:"responseTime,omitempty"` - ServiceTime int64 `protobuf:"varint,2,opt,name=serviceTime,proto3" json:"serviceTime,omitempty"` - TotalNQ int64 `protobuf:"varint,3,opt,name=totalNQ,proto3" json:"totalNQ,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CostAggregation) Reset() { *m = CostAggregation{} } -func (m *CostAggregation) String() string { return proto.CompactTextString(m) } -func (*CostAggregation) ProtoMessage() {} -func (*CostAggregation) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{14} -} - -func (m *CostAggregation) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CostAggregation.Unmarshal(m, b) -} -func (m *CostAggregation) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CostAggregation.Marshal(b, m, deterministic) -} -func (m *CostAggregation) XXX_Merge(src proto.Message) { - xxx_messageInfo_CostAggregation.Merge(m, src) -} -func (m *CostAggregation) XXX_Size() int { - return xxx_messageInfo_CostAggregation.Size(m) -} -func (m *CostAggregation) XXX_DiscardUnknown() { - xxx_messageInfo_CostAggregation.DiscardUnknown(m) -} - -var xxx_messageInfo_CostAggregation proto.InternalMessageInfo - -func (m *CostAggregation) GetResponseTime() int64 { - if m != nil { - return m.ResponseTime - } - return 0 -} - -func (m *CostAggregation) GetServiceTime() int64 { - if m != nil { - return m.ServiceTime - } - return 0 -} - -func (m *CostAggregation) GetTotalNQ() int64 { - if m != nil { - return m.TotalNQ - } - return 0 -} - -type RetrieveRequest struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - ReqID int64 `protobuf:"varint,2,opt,name=reqID,proto3" json:"reqID,omitempty"` - DbID int64 `protobuf:"varint,3,opt,name=dbID,proto3" json:"dbID,omitempty"` - CollectionID int64 `protobuf:"varint,4,opt,name=collectionID,proto3" json:"collectionID,omitempty"` - PartitionIDs []int64 `protobuf:"varint,5,rep,packed,name=partitionIDs,proto3" json:"partitionIDs,omitempty"` - SerializedExprPlan []byte `protobuf:"bytes,6,opt,name=serialized_expr_plan,json=serializedExprPlan,proto3" json:"serialized_expr_plan,omitempty"` - OutputFieldsId []int64 `protobuf:"varint,7,rep,packed,name=output_fields_id,json=outputFieldsId,proto3" json:"output_fields_id,omitempty"` - TravelTimestamp uint64 `protobuf:"varint,8,opt,name=travel_timestamp,json=travelTimestamp,proto3" json:"travel_timestamp,omitempty"` - GuaranteeTimestamp uint64 `protobuf:"varint,9,opt,name=guarantee_timestamp,json=guaranteeTimestamp,proto3" json:"guarantee_timestamp,omitempty"` - TimeoutTimestamp uint64 `protobuf:"varint,10,opt,name=timeout_timestamp,json=timeoutTimestamp,proto3" json:"timeout_timestamp,omitempty"` - Limit int64 `protobuf:"varint,11,opt,name=limit,proto3" json:"limit,omitempty"` - IgnoreGrowing bool `protobuf:"varint,12,opt,name=ignoreGrowing,proto3" json:"ignoreGrowing,omitempty"` - IsCount bool `protobuf:"varint,13,opt,name=is_count,json=isCount,proto3" json:"is_count,omitempty"` - IterationExtensionReduceRate int64 `protobuf:"varint,14,opt,name=iteration_extension_reduce_rate,json=iterationExtensionReduceRate,proto3" json:"iteration_extension_reduce_rate,omitempty"` - Username string `protobuf:"bytes,15,opt,name=username,proto3" json:"username,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *RetrieveRequest) Reset() { *m = RetrieveRequest{} } -func (m *RetrieveRequest) String() string { return proto.CompactTextString(m) } -func (*RetrieveRequest) ProtoMessage() {} -func (*RetrieveRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{15} -} - -func (m *RetrieveRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RetrieveRequest.Unmarshal(m, b) -} -func (m *RetrieveRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RetrieveRequest.Marshal(b, m, deterministic) -} -func (m *RetrieveRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_RetrieveRequest.Merge(m, src) -} -func (m *RetrieveRequest) XXX_Size() int { - return xxx_messageInfo_RetrieveRequest.Size(m) -} -func (m *RetrieveRequest) XXX_DiscardUnknown() { - xxx_messageInfo_RetrieveRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_RetrieveRequest proto.InternalMessageInfo - -func (m *RetrieveRequest) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *RetrieveRequest) GetReqID() int64 { - if m != nil { - return m.ReqID - } - return 0 -} - -func (m *RetrieveRequest) GetDbID() int64 { - if m != nil { - return m.DbID - } - return 0 -} - -func (m *RetrieveRequest) GetCollectionID() int64 { - if m != nil { - return m.CollectionID - } - return 0 -} - -func (m *RetrieveRequest) GetPartitionIDs() []int64 { - if m != nil { - return m.PartitionIDs - } - return nil -} - -func (m *RetrieveRequest) GetSerializedExprPlan() []byte { - if m != nil { - return m.SerializedExprPlan - } - return nil -} - -func (m *RetrieveRequest) GetOutputFieldsId() []int64 { - if m != nil { - return m.OutputFieldsId - } - return nil -} - -func (m *RetrieveRequest) GetTravelTimestamp() uint64 { - if m != nil { - return m.TravelTimestamp - } - return 0 -} - -func (m *RetrieveRequest) GetGuaranteeTimestamp() uint64 { - if m != nil { - return m.GuaranteeTimestamp - } - return 0 -} - -func (m *RetrieveRequest) GetTimeoutTimestamp() uint64 { - if m != nil { - return m.TimeoutTimestamp - } - return 0 -} - -func (m *RetrieveRequest) GetLimit() int64 { - if m != nil { - return m.Limit - } - return 0 -} - -func (m *RetrieveRequest) GetIgnoreGrowing() bool { - if m != nil { - return m.IgnoreGrowing - } - return false -} - -func (m *RetrieveRequest) GetIsCount() bool { - if m != nil { - return m.IsCount - } - return false -} - -func (m *RetrieveRequest) GetIterationExtensionReduceRate() int64 { - if m != nil { - return m.IterationExtensionReduceRate - } - return 0 -} - -func (m *RetrieveRequest) GetUsername() string { - if m != nil { - return m.Username - } - return "" -} - -type RetrieveResults struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - Status *commonpb.Status `protobuf:"bytes,2,opt,name=status,proto3" json:"status,omitempty"` - ReqID int64 `protobuf:"varint,3,opt,name=reqID,proto3" json:"reqID,omitempty"` - Ids *schemapb.IDs `protobuf:"bytes,4,opt,name=ids,proto3" json:"ids,omitempty"` - FieldsData []*schemapb.FieldData `protobuf:"bytes,5,rep,name=fields_data,json=fieldsData,proto3" json:"fields_data,omitempty"` - SealedSegmentIDsRetrieved []int64 `protobuf:"varint,6,rep,packed,name=sealed_segmentIDs_retrieved,json=sealedSegmentIDsRetrieved,proto3" json:"sealed_segmentIDs_retrieved,omitempty"` - ChannelIDsRetrieved []string `protobuf:"bytes,7,rep,name=channelIDs_retrieved,json=channelIDsRetrieved,proto3" json:"channelIDs_retrieved,omitempty"` - GlobalSealedSegmentIDs []int64 `protobuf:"varint,8,rep,packed,name=global_sealed_segmentIDs,json=globalSealedSegmentIDs,proto3" json:"global_sealed_segmentIDs,omitempty"` - // query request cost - CostAggregation *CostAggregation `protobuf:"bytes,13,opt,name=costAggregation,proto3" json:"costAggregation,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *RetrieveResults) Reset() { *m = RetrieveResults{} } -func (m *RetrieveResults) String() string { return proto.CompactTextString(m) } -func (*RetrieveResults) ProtoMessage() {} -func (*RetrieveResults) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{16} -} - -func (m *RetrieveResults) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_RetrieveResults.Unmarshal(m, b) -} -func (m *RetrieveResults) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_RetrieveResults.Marshal(b, m, deterministic) -} -func (m *RetrieveResults) XXX_Merge(src proto.Message) { - xxx_messageInfo_RetrieveResults.Merge(m, src) -} -func (m *RetrieveResults) XXX_Size() int { - return xxx_messageInfo_RetrieveResults.Size(m) -} -func (m *RetrieveResults) XXX_DiscardUnknown() { - xxx_messageInfo_RetrieveResults.DiscardUnknown(m) -} - -var xxx_messageInfo_RetrieveResults proto.InternalMessageInfo - -func (m *RetrieveResults) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *RetrieveResults) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - -func (m *RetrieveResults) GetReqID() int64 { - if m != nil { - return m.ReqID - } - return 0 -} - -func (m *RetrieveResults) GetIds() *schemapb.IDs { - if m != nil { - return m.Ids - } - return nil -} - -func (m *RetrieveResults) GetFieldsData() []*schemapb.FieldData { - if m != nil { - return m.FieldsData - } - return nil -} - -func (m *RetrieveResults) GetSealedSegmentIDsRetrieved() []int64 { - if m != nil { - return m.SealedSegmentIDsRetrieved - } - return nil -} - -func (m *RetrieveResults) GetChannelIDsRetrieved() []string { - if m != nil { - return m.ChannelIDsRetrieved - } - return nil -} - -func (m *RetrieveResults) GetGlobalSealedSegmentIDs() []int64 { - if m != nil { - return m.GlobalSealedSegmentIDs - } - return nil -} - -func (m *RetrieveResults) GetCostAggregation() *CostAggregation { - if m != nil { - return m.CostAggregation - } - return nil -} - -type LoadIndex struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - SegmentID int64 `protobuf:"varint,2,opt,name=segmentID,proto3" json:"segmentID,omitempty"` - FieldName string `protobuf:"bytes,3,opt,name=fieldName,proto3" json:"fieldName,omitempty"` - FieldID int64 `protobuf:"varint,4,opt,name=fieldID,proto3" json:"fieldID,omitempty"` - IndexPaths []string `protobuf:"bytes,5,rep,name=index_paths,json=indexPaths,proto3" json:"index_paths,omitempty"` - IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,6,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *LoadIndex) Reset() { *m = LoadIndex{} } -func (m *LoadIndex) String() string { return proto.CompactTextString(m) } -func (*LoadIndex) ProtoMessage() {} -func (*LoadIndex) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{17} -} - -func (m *LoadIndex) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_LoadIndex.Unmarshal(m, b) -} -func (m *LoadIndex) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_LoadIndex.Marshal(b, m, deterministic) -} -func (m *LoadIndex) XXX_Merge(src proto.Message) { - xxx_messageInfo_LoadIndex.Merge(m, src) -} -func (m *LoadIndex) XXX_Size() int { - return xxx_messageInfo_LoadIndex.Size(m) -} -func (m *LoadIndex) XXX_DiscardUnknown() { - xxx_messageInfo_LoadIndex.DiscardUnknown(m) -} - -var xxx_messageInfo_LoadIndex proto.InternalMessageInfo - -func (m *LoadIndex) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *LoadIndex) GetSegmentID() int64 { - if m != nil { - return m.SegmentID - } - return 0 -} - -func (m *LoadIndex) GetFieldName() string { - if m != nil { - return m.FieldName - } - return "" -} - -func (m *LoadIndex) GetFieldID() int64 { - if m != nil { - return m.FieldID - } - return 0 -} - -func (m *LoadIndex) GetIndexPaths() []string { - if m != nil { - return m.IndexPaths - } - return nil -} - -func (m *LoadIndex) GetIndexParams() []*commonpb.KeyValuePair { - if m != nil { - return m.IndexParams - } - return nil -} - -type IndexStats struct { - IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,1,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"` - NumRelatedSegments int64 `protobuf:"varint,2,opt,name=num_related_segments,json=numRelatedSegments,proto3" json:"num_related_segments,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *IndexStats) Reset() { *m = IndexStats{} } -func (m *IndexStats) String() string { return proto.CompactTextString(m) } -func (*IndexStats) ProtoMessage() {} -func (*IndexStats) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{18} -} - -func (m *IndexStats) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_IndexStats.Unmarshal(m, b) -} -func (m *IndexStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_IndexStats.Marshal(b, m, deterministic) -} -func (m *IndexStats) XXX_Merge(src proto.Message) { - xxx_messageInfo_IndexStats.Merge(m, src) -} -func (m *IndexStats) XXX_Size() int { - return xxx_messageInfo_IndexStats.Size(m) -} -func (m *IndexStats) XXX_DiscardUnknown() { - xxx_messageInfo_IndexStats.DiscardUnknown(m) -} - -var xxx_messageInfo_IndexStats proto.InternalMessageInfo - -func (m *IndexStats) GetIndexParams() []*commonpb.KeyValuePair { - if m != nil { - return m.IndexParams - } - return nil -} - -func (m *IndexStats) GetNumRelatedSegments() int64 { - if m != nil { - return m.NumRelatedSegments - } - return 0 -} - -type FieldStats struct { - CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"` - FieldID int64 `protobuf:"varint,2,opt,name=fieldID,proto3" json:"fieldID,omitempty"` - IndexStats []*IndexStats `protobuf:"bytes,3,rep,name=index_stats,json=indexStats,proto3" json:"index_stats,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *FieldStats) Reset() { *m = FieldStats{} } -func (m *FieldStats) String() string { return proto.CompactTextString(m) } -func (*FieldStats) ProtoMessage() {} -func (*FieldStats) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{19} -} - -func (m *FieldStats) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_FieldStats.Unmarshal(m, b) -} -func (m *FieldStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_FieldStats.Marshal(b, m, deterministic) -} -func (m *FieldStats) XXX_Merge(src proto.Message) { - xxx_messageInfo_FieldStats.Merge(m, src) -} -func (m *FieldStats) XXX_Size() int { - return xxx_messageInfo_FieldStats.Size(m) -} -func (m *FieldStats) XXX_DiscardUnknown() { - xxx_messageInfo_FieldStats.DiscardUnknown(m) -} - -var xxx_messageInfo_FieldStats proto.InternalMessageInfo - -func (m *FieldStats) GetCollectionID() int64 { - if m != nil { - return m.CollectionID - } - return 0 -} - -func (m *FieldStats) GetFieldID() int64 { - if m != nil { - return m.FieldID - } - return 0 -} - -func (m *FieldStats) GetIndexStats() []*IndexStats { - if m != nil { - return m.IndexStats - } - return nil -} - -type SegmentStats struct { - SegmentID int64 `protobuf:"varint,1,opt,name=segmentID,proto3" json:"segmentID,omitempty"` - MemorySize int64 `protobuf:"varint,2,opt,name=memory_size,json=memorySize,proto3" json:"memory_size,omitempty"` - NumRows int64 `protobuf:"varint,3,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"` - RecentlyModified bool `protobuf:"varint,4,opt,name=recently_modified,json=recentlyModified,proto3" json:"recently_modified,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *SegmentStats) Reset() { *m = SegmentStats{} } -func (m *SegmentStats) String() string { return proto.CompactTextString(m) } -func (*SegmentStats) ProtoMessage() {} -func (*SegmentStats) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{20} -} - -func (m *SegmentStats) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_SegmentStats.Unmarshal(m, b) -} -func (m *SegmentStats) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_SegmentStats.Marshal(b, m, deterministic) -} -func (m *SegmentStats) XXX_Merge(src proto.Message) { - xxx_messageInfo_SegmentStats.Merge(m, src) -} -func (m *SegmentStats) XXX_Size() int { - return xxx_messageInfo_SegmentStats.Size(m) -} -func (m *SegmentStats) XXX_DiscardUnknown() { - xxx_messageInfo_SegmentStats.DiscardUnknown(m) -} - -var xxx_messageInfo_SegmentStats proto.InternalMessageInfo - -func (m *SegmentStats) GetSegmentID() int64 { - if m != nil { - return m.SegmentID - } - return 0 -} - -func (m *SegmentStats) GetMemorySize() int64 { - if m != nil { - return m.MemorySize - } - return 0 -} - -func (m *SegmentStats) GetNumRows() int64 { - if m != nil { - return m.NumRows - } - return 0 -} - -func (m *SegmentStats) GetRecentlyModified() bool { - if m != nil { - return m.RecentlyModified - } - return false -} - -type ChannelTimeTickMsg struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - ChannelNames []string `protobuf:"bytes,2,rep,name=channelNames,proto3" json:"channelNames,omitempty"` - Timestamps []uint64 `protobuf:"varint,3,rep,packed,name=timestamps,proto3" json:"timestamps,omitempty"` - DefaultTimestamp uint64 `protobuf:"varint,4,opt,name=default_timestamp,json=defaultTimestamp,proto3" json:"default_timestamp,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ChannelTimeTickMsg) Reset() { *m = ChannelTimeTickMsg{} } -func (m *ChannelTimeTickMsg) String() string { return proto.CompactTextString(m) } -func (*ChannelTimeTickMsg) ProtoMessage() {} -func (*ChannelTimeTickMsg) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{21} -} - -func (m *ChannelTimeTickMsg) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ChannelTimeTickMsg.Unmarshal(m, b) -} -func (m *ChannelTimeTickMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ChannelTimeTickMsg.Marshal(b, m, deterministic) -} -func (m *ChannelTimeTickMsg) XXX_Merge(src proto.Message) { - xxx_messageInfo_ChannelTimeTickMsg.Merge(m, src) -} -func (m *ChannelTimeTickMsg) XXX_Size() int { - return xxx_messageInfo_ChannelTimeTickMsg.Size(m) -} -func (m *ChannelTimeTickMsg) XXX_DiscardUnknown() { - xxx_messageInfo_ChannelTimeTickMsg.DiscardUnknown(m) -} - -var xxx_messageInfo_ChannelTimeTickMsg proto.InternalMessageInfo - -func (m *ChannelTimeTickMsg) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *ChannelTimeTickMsg) GetChannelNames() []string { - if m != nil { - return m.ChannelNames - } - return nil -} - -func (m *ChannelTimeTickMsg) GetTimestamps() []uint64 { - if m != nil { - return m.Timestamps - } - return nil -} - -func (m *ChannelTimeTickMsg) GetDefaultTimestamp() uint64 { - if m != nil { - return m.DefaultTimestamp - } - return 0 -} - -type CredentialInfo struct { - Username string `protobuf:"bytes,1,opt,name=username,proto3" json:"username,omitempty"` - // encrypted by bcrypt (for higher security level) - EncryptedPassword string `protobuf:"bytes,2,opt,name=encrypted_password,json=encryptedPassword,proto3" json:"encrypted_password,omitempty"` - Tenant string `protobuf:"bytes,3,opt,name=tenant,proto3" json:"tenant,omitempty"` - IsSuper bool `protobuf:"varint,4,opt,name=is_super,json=isSuper,proto3" json:"is_super,omitempty"` - // encrypted by sha256 (for good performance in cache mapping) - Sha256Password string `protobuf:"bytes,5,opt,name=sha256_password,json=sha256Password,proto3" json:"sha256_password,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *CredentialInfo) Reset() { *m = CredentialInfo{} } -func (m *CredentialInfo) String() string { return proto.CompactTextString(m) } -func (*CredentialInfo) ProtoMessage() {} -func (*CredentialInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{22} -} - -func (m *CredentialInfo) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_CredentialInfo.Unmarshal(m, b) -} -func (m *CredentialInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_CredentialInfo.Marshal(b, m, deterministic) -} -func (m *CredentialInfo) XXX_Merge(src proto.Message) { - xxx_messageInfo_CredentialInfo.Merge(m, src) -} -func (m *CredentialInfo) XXX_Size() int { - return xxx_messageInfo_CredentialInfo.Size(m) -} -func (m *CredentialInfo) XXX_DiscardUnknown() { - xxx_messageInfo_CredentialInfo.DiscardUnknown(m) -} - -var xxx_messageInfo_CredentialInfo proto.InternalMessageInfo - -func (m *CredentialInfo) GetUsername() string { - if m != nil { - return m.Username - } - return "" -} - -func (m *CredentialInfo) GetEncryptedPassword() string { - if m != nil { - return m.EncryptedPassword - } - return "" -} - -func (m *CredentialInfo) GetTenant() string { - if m != nil { - return m.Tenant - } - return "" -} - -func (m *CredentialInfo) GetIsSuper() bool { - if m != nil { - return m.IsSuper - } - return false -} - -func (m *CredentialInfo) GetSha256Password() string { - if m != nil { - return m.Sha256Password - } - return "" -} - -type ListPolicyRequest struct { - // Not useful for now - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ListPolicyRequest) Reset() { *m = ListPolicyRequest{} } -func (m *ListPolicyRequest) String() string { return proto.CompactTextString(m) } -func (*ListPolicyRequest) ProtoMessage() {} -func (*ListPolicyRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{23} -} - -func (m *ListPolicyRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ListPolicyRequest.Unmarshal(m, b) -} -func (m *ListPolicyRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ListPolicyRequest.Marshal(b, m, deterministic) -} -func (m *ListPolicyRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_ListPolicyRequest.Merge(m, src) -} -func (m *ListPolicyRequest) XXX_Size() int { - return xxx_messageInfo_ListPolicyRequest.Size(m) -} -func (m *ListPolicyRequest) XXX_DiscardUnknown() { - xxx_messageInfo_ListPolicyRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_ListPolicyRequest proto.InternalMessageInfo - -func (m *ListPolicyRequest) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -type ListPolicyResponse struct { - // Contain error_code and reason - Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` - PolicyInfos []string `protobuf:"bytes,2,rep,name=policy_infos,json=policyInfos,proto3" json:"policy_infos,omitempty"` - UserRoles []string `protobuf:"bytes,3,rep,name=user_roles,json=userRoles,proto3" json:"user_roles,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ListPolicyResponse) Reset() { *m = ListPolicyResponse{} } -func (m *ListPolicyResponse) String() string { return proto.CompactTextString(m) } -func (*ListPolicyResponse) ProtoMessage() {} -func (*ListPolicyResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{24} -} - -func (m *ListPolicyResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ListPolicyResponse.Unmarshal(m, b) -} -func (m *ListPolicyResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ListPolicyResponse.Marshal(b, m, deterministic) -} -func (m *ListPolicyResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_ListPolicyResponse.Merge(m, src) -} -func (m *ListPolicyResponse) XXX_Size() int { - return xxx_messageInfo_ListPolicyResponse.Size(m) -} -func (m *ListPolicyResponse) XXX_DiscardUnknown() { - xxx_messageInfo_ListPolicyResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_ListPolicyResponse proto.InternalMessageInfo - -func (m *ListPolicyResponse) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - -func (m *ListPolicyResponse) GetPolicyInfos() []string { - if m != nil { - return m.PolicyInfos - } - return nil -} - -func (m *ListPolicyResponse) GetUserRoles() []string { - if m != nil { - return m.UserRoles - } - return nil -} - -type ShowConfigurationsRequest struct { - Base *commonpb.MsgBase `protobuf:"bytes,1,opt,name=base,proto3" json:"base,omitempty"` - Pattern string `protobuf:"bytes,2,opt,name=pattern,proto3" json:"pattern,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ShowConfigurationsRequest) Reset() { *m = ShowConfigurationsRequest{} } -func (m *ShowConfigurationsRequest) String() string { return proto.CompactTextString(m) } -func (*ShowConfigurationsRequest) ProtoMessage() {} -func (*ShowConfigurationsRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{25} -} - -func (m *ShowConfigurationsRequest) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ShowConfigurationsRequest.Unmarshal(m, b) -} -func (m *ShowConfigurationsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ShowConfigurationsRequest.Marshal(b, m, deterministic) -} -func (m *ShowConfigurationsRequest) XXX_Merge(src proto.Message) { - xxx_messageInfo_ShowConfigurationsRequest.Merge(m, src) -} -func (m *ShowConfigurationsRequest) XXX_Size() int { - return xxx_messageInfo_ShowConfigurationsRequest.Size(m) -} -func (m *ShowConfigurationsRequest) XXX_DiscardUnknown() { - xxx_messageInfo_ShowConfigurationsRequest.DiscardUnknown(m) -} - -var xxx_messageInfo_ShowConfigurationsRequest proto.InternalMessageInfo - -func (m *ShowConfigurationsRequest) GetBase() *commonpb.MsgBase { - if m != nil { - return m.Base - } - return nil -} - -func (m *ShowConfigurationsRequest) GetPattern() string { - if m != nil { - return m.Pattern - } - return "" -} - -type ShowConfigurationsResponse struct { - Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` - Configuations []*commonpb.KeyValuePair `protobuf:"bytes,2,rep,name=configuations,proto3" json:"configuations,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *ShowConfigurationsResponse) Reset() { *m = ShowConfigurationsResponse{} } -func (m *ShowConfigurationsResponse) String() string { return proto.CompactTextString(m) } -func (*ShowConfigurationsResponse) ProtoMessage() {} -func (*ShowConfigurationsResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{26} -} - -func (m *ShowConfigurationsResponse) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_ShowConfigurationsResponse.Unmarshal(m, b) -} -func (m *ShowConfigurationsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_ShowConfigurationsResponse.Marshal(b, m, deterministic) -} -func (m *ShowConfigurationsResponse) XXX_Merge(src proto.Message) { - xxx_messageInfo_ShowConfigurationsResponse.Merge(m, src) -} -func (m *ShowConfigurationsResponse) XXX_Size() int { - return xxx_messageInfo_ShowConfigurationsResponse.Size(m) -} -func (m *ShowConfigurationsResponse) XXX_DiscardUnknown() { - xxx_messageInfo_ShowConfigurationsResponse.DiscardUnknown(m) -} - -var xxx_messageInfo_ShowConfigurationsResponse proto.InternalMessageInfo - -func (m *ShowConfigurationsResponse) GetStatus() *commonpb.Status { - if m != nil { - return m.Status - } - return nil -} - -func (m *ShowConfigurationsResponse) GetConfiguations() []*commonpb.KeyValuePair { - if m != nil { - return m.Configuations - } - return nil -} - -type Rate struct { - Rt RateType `protobuf:"varint,1,opt,name=rt,proto3,enum=milvus.proto.internal.RateType" json:"rt,omitempty"` - R float64 `protobuf:"fixed64,2,opt,name=r,proto3" json:"r,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` -} - -func (m *Rate) Reset() { *m = Rate{} } -func (m *Rate) String() string { return proto.CompactTextString(m) } -func (*Rate) ProtoMessage() {} -func (*Rate) Descriptor() ([]byte, []int) { - return fileDescriptor_41f4a519b878ee3b, []int{27} -} - -func (m *Rate) XXX_Unmarshal(b []byte) error { - return xxx_messageInfo_Rate.Unmarshal(m, b) -} -func (m *Rate) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { - return xxx_messageInfo_Rate.Marshal(b, m, deterministic) -} -func (m *Rate) XXX_Merge(src proto.Message) { - xxx_messageInfo_Rate.Merge(m, src) -} -func (m *Rate) XXX_Size() int { - return xxx_messageInfo_Rate.Size(m) -} -func (m *Rate) XXX_DiscardUnknown() { - xxx_messageInfo_Rate.DiscardUnknown(m) -} - -var xxx_messageInfo_Rate proto.InternalMessageInfo - -func (m *Rate) GetRt() RateType { - if m != nil { - return m.Rt - } - return RateType_DDLCollection -} - -func (m *Rate) GetR() float64 { - if m != nil { - return m.R - } - return 0 -} - -func init() { - proto.RegisterEnum("milvus.proto.internal.RateType", RateType_name, RateType_value) - proto.RegisterType((*GetTimeTickChannelRequest)(nil), "milvus.proto.internal.GetTimeTickChannelRequest") - proto.RegisterType((*GetStatisticsChannelRequest)(nil), "milvus.proto.internal.GetStatisticsChannelRequest") - proto.RegisterType((*GetDdChannelRequest)(nil), "milvus.proto.internal.GetDdChannelRequest") - proto.RegisterType((*NodeInfo)(nil), "milvus.proto.internal.NodeInfo") - proto.RegisterType((*InitParams)(nil), "milvus.proto.internal.InitParams") - proto.RegisterType((*StringList)(nil), "milvus.proto.internal.StringList") - proto.RegisterType((*GetStatisticsRequest)(nil), "milvus.proto.internal.GetStatisticsRequest") - proto.RegisterType((*GetStatisticsResponse)(nil), "milvus.proto.internal.GetStatisticsResponse") - proto.RegisterType((*CreateAliasRequest)(nil), "milvus.proto.internal.CreateAliasRequest") - proto.RegisterType((*DropAliasRequest)(nil), "milvus.proto.internal.DropAliasRequest") - proto.RegisterType((*AlterAliasRequest)(nil), "milvus.proto.internal.AlterAliasRequest") - proto.RegisterType((*CreateIndexRequest)(nil), "milvus.proto.internal.CreateIndexRequest") - proto.RegisterType((*SearchRequest)(nil), "milvus.proto.internal.SearchRequest") - proto.RegisterType((*SearchResults)(nil), "milvus.proto.internal.SearchResults") - proto.RegisterType((*CostAggregation)(nil), "milvus.proto.internal.CostAggregation") - proto.RegisterType((*RetrieveRequest)(nil), "milvus.proto.internal.RetrieveRequest") - proto.RegisterType((*RetrieveResults)(nil), "milvus.proto.internal.RetrieveResults") - proto.RegisterType((*LoadIndex)(nil), "milvus.proto.internal.LoadIndex") - proto.RegisterType((*IndexStats)(nil), "milvus.proto.internal.IndexStats") - proto.RegisterType((*FieldStats)(nil), "milvus.proto.internal.FieldStats") - proto.RegisterType((*SegmentStats)(nil), "milvus.proto.internal.SegmentStats") - proto.RegisterType((*ChannelTimeTickMsg)(nil), "milvus.proto.internal.ChannelTimeTickMsg") - proto.RegisterType((*CredentialInfo)(nil), "milvus.proto.internal.CredentialInfo") - proto.RegisterType((*ListPolicyRequest)(nil), "milvus.proto.internal.ListPolicyRequest") - proto.RegisterType((*ListPolicyResponse)(nil), "milvus.proto.internal.ListPolicyResponse") - proto.RegisterType((*ShowConfigurationsRequest)(nil), "milvus.proto.internal.ShowConfigurationsRequest") - proto.RegisterType((*ShowConfigurationsResponse)(nil), "milvus.proto.internal.ShowConfigurationsResponse") - proto.RegisterType((*Rate)(nil), "milvus.proto.internal.Rate") -} - -func init() { proto.RegisterFile("internal.proto", fileDescriptor_41f4a519b878ee3b) } - -var fileDescriptor_41f4a519b878ee3b = []byte{ - // 1927 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xd4, 0x58, 0x4b, 0x73, 0x1c, 0x49, - 0x11, 0xa6, 0xe7, 0x3d, 0x39, 0x23, 0x69, 0x54, 0x96, 0x4d, 0xfb, 0xb1, 0x6b, 0x6d, 0x43, 0x80, - 0x58, 0x62, 0xed, 0x45, 0x1b, 0xbb, 0xe6, 0x40, 0x40, 0xd8, 0x6a, 0xaf, 0x62, 0x62, 0xc7, 0x46, - 0xee, 0x31, 0x1b, 0x01, 0x97, 0x8e, 0x9a, 0xe9, 0xd4, 0xa8, 0x70, 0xbf, 0x54, 0x55, 0x6d, 0x49, - 0x3e, 0x73, 0x23, 0x82, 0x1b, 0x1c, 0x88, 0x80, 0x7f, 0xc0, 0x79, 0x83, 0x13, 0xff, 0x80, 0x13, - 0xbf, 0x66, 0x4f, 0x44, 0x3d, 0x7a, 0x5e, 0x1a, 0x2b, 0x24, 0x99, 0xc7, 0xee, 0xad, 0x33, 0xf3, - 0xab, 0xac, 0xaa, 0xcc, 0xac, 0xaf, 0xb2, 0x1a, 0xd6, 0x59, 0x2a, 0x91, 0xa7, 0x34, 0x7e, 0x90, - 0xf3, 0x4c, 0x66, 0xe4, 0x66, 0xc2, 0xe2, 0xd7, 0x85, 0x30, 0xd2, 0x83, 0xd2, 0x78, 0xa7, 0x3b, - 0xce, 0x92, 0x24, 0x4b, 0x8d, 0xfa, 0x4e, 0x57, 0x8c, 0x8f, 0x30, 0xa1, 0x46, 0xf2, 0xee, 0xc2, - 0xed, 0x7d, 0x94, 0x2f, 0x59, 0x82, 0x2f, 0xd9, 0xf8, 0xd5, 0xde, 0x11, 0x4d, 0x53, 0x8c, 0x03, - 0x3c, 0x2e, 0x50, 0x48, 0xef, 0x3d, 0xb8, 0xbb, 0x8f, 0x72, 0x28, 0xa9, 0x64, 0x42, 0xb2, 0xb1, - 0x58, 0x32, 0xdf, 0x84, 0x1b, 0xfb, 0x28, 0xfd, 0x68, 0x49, 0xfd, 0x25, 0xb4, 0x9e, 0x67, 0x11, - 0xf6, 0xd3, 0xc3, 0x8c, 0x7c, 0x06, 0x4d, 0x1a, 0x45, 0x1c, 0x85, 0x70, 0x9d, 0x6d, 0x67, 0xa7, - 0xb3, 0x7b, 0xef, 0xc1, 0xc2, 0x1a, 0xed, 0xca, 0x1e, 0x1b, 0x4c, 0x50, 0x82, 0x09, 0x81, 0x1a, - 0xcf, 0x62, 0x74, 0x2b, 0xdb, 0xce, 0x4e, 0x3b, 0xd0, 0xdf, 0xde, 0x6f, 0x01, 0xfa, 0x29, 0x93, - 0x07, 0x94, 0xd3, 0x44, 0x90, 0x5b, 0xd0, 0x48, 0xd5, 0x2c, 0xbe, 0x76, 0x5c, 0x0d, 0xac, 0x44, - 0x7c, 0xe8, 0x0a, 0x49, 0xb9, 0x0c, 0x73, 0x8d, 0x73, 0x2b, 0xdb, 0xd5, 0x9d, 0xce, 0xee, 0x07, - 0x2b, 0xa7, 0xfd, 0x02, 0xcf, 0xbe, 0xa4, 0x71, 0x81, 0x07, 0x94, 0xf1, 0xa0, 0xa3, 0x87, 0x19, - 0xef, 0xde, 0xaf, 0x01, 0x86, 0x92, 0xb3, 0x74, 0x32, 0x60, 0x42, 0xaa, 0xb9, 0x5e, 0x2b, 0x9c, - 0xda, 0x44, 0x75, 0xa7, 0x1d, 0x58, 0x89, 0x7c, 0x02, 0x0d, 0x21, 0xa9, 0x2c, 0x84, 0x5e, 0x67, - 0x67, 0xf7, 0xee, 0xca, 0x59, 0x86, 0x1a, 0x12, 0x58, 0xa8, 0xf7, 0xb7, 0x0a, 0x6c, 0x2d, 0x44, - 0xd5, 0xc6, 0x8d, 0x7c, 0x0c, 0xb5, 0x11, 0x15, 0x78, 0x61, 0xa0, 0x9e, 0x89, 0xc9, 0x13, 0x2a, - 0x30, 0xd0, 0x48, 0x15, 0xa5, 0x68, 0xd4, 0xf7, 0xf5, 0xec, 0xd5, 0x40, 0x7f, 0x13, 0x0f, 0xba, - 0xe3, 0x2c, 0x8e, 0x71, 0x2c, 0x59, 0x96, 0xf6, 0x7d, 0xb7, 0xaa, 0x6d, 0x0b, 0x3a, 0x85, 0xc9, - 0x29, 0x97, 0xcc, 0x88, 0xc2, 0xad, 0x6d, 0x57, 0x15, 0x66, 0x5e, 0x47, 0x7e, 0x04, 0x3d, 0xc9, - 0xe9, 0x6b, 0x8c, 0x43, 0xc9, 0x12, 0x14, 0x92, 0x26, 0xb9, 0x5b, 0xdf, 0x76, 0x76, 0x6a, 0xc1, - 0x86, 0xd1, 0xbf, 0x2c, 0xd5, 0xe4, 0x21, 0xdc, 0x98, 0x14, 0x94, 0xd3, 0x54, 0x22, 0xce, 0xa1, - 0x1b, 0x1a, 0x4d, 0xa6, 0xa6, 0xd9, 0x80, 0x1f, 0xc3, 0xa6, 0x82, 0x65, 0x85, 0x9c, 0x83, 0x37, - 0x35, 0xbc, 0x67, 0x0d, 0x53, 0xb0, 0xf7, 0x95, 0x03, 0x37, 0x97, 0xe2, 0x25, 0xf2, 0x2c, 0x15, - 0x78, 0x8d, 0x80, 0x5d, 0x27, 0x61, 0xe4, 0x11, 0xd4, 0xd5, 0x97, 0x70, 0xab, 0x97, 0x2d, 0x25, - 0x83, 0xf7, 0xfe, 0xea, 0x00, 0xd9, 0xe3, 0x48, 0x25, 0x3e, 0x8e, 0x19, 0x7d, 0x87, 0x3c, 0x7f, - 0x17, 0x9a, 0xd1, 0x28, 0x4c, 0x69, 0x52, 0x1e, 0x88, 0x46, 0x34, 0x7a, 0x4e, 0x13, 0x24, 0x3f, - 0x84, 0x8d, 0x59, 0x62, 0x0d, 0xa0, 0xaa, 0x01, 0xeb, 0x33, 0xb5, 0x06, 0x6e, 0x41, 0x9d, 0xaa, - 0x35, 0xb8, 0x35, 0x6d, 0x36, 0x82, 0x27, 0xa0, 0xe7, 0xf3, 0x2c, 0xff, 0x6f, 0xad, 0x6e, 0x3a, - 0x69, 0x75, 0x7e, 0xd2, 0xbf, 0x38, 0xb0, 0xf9, 0x38, 0x96, 0xc8, 0xbf, 0xa1, 0x41, 0xf9, 0x47, - 0xa5, 0xcc, 0x5a, 0x3f, 0x8d, 0xf0, 0xf4, 0xff, 0xb9, 0xc0, 0xf7, 0x00, 0x0e, 0x19, 0xc6, 0x91, - 0xc1, 0x98, 0x55, 0xb6, 0xb5, 0x46, 0x9b, 0xcb, 0xe3, 0x5f, 0xbf, 0xe0, 0xf8, 0x37, 0x56, 0x1c, - 0x7f, 0x17, 0x9a, 0xda, 0x49, 0xdf, 0xd7, 0x87, 0xae, 0x1a, 0x94, 0xa2, 0x22, 0x4f, 0x3c, 0x95, - 0x9c, 0x96, 0xe4, 0xd9, 0xba, 0x34, 0x79, 0xea, 0x61, 0x96, 0x3c, 0xff, 0x54, 0x87, 0xb5, 0x21, - 0x52, 0x3e, 0x3e, 0xba, 0x7e, 0xf0, 0xb6, 0xa0, 0xce, 0xf1, 0x78, 0xca, 0x6d, 0x46, 0x98, 0xee, - 0xb8, 0x7a, 0xc1, 0x8e, 0x6b, 0x97, 0x20, 0xbc, 0xfa, 0x0a, 0xc2, 0xeb, 0x41, 0x35, 0x12, 0xb1, - 0x0e, 0x58, 0x3b, 0x50, 0x9f, 0x8a, 0xa6, 0xf2, 0x98, 0x8e, 0xf1, 0x28, 0x8b, 0x23, 0xe4, 0xe1, - 0x84, 0x67, 0x85, 0xa1, 0xa9, 0x6e, 0xd0, 0x9b, 0x33, 0xec, 0x2b, 0x3d, 0x79, 0x04, 0xad, 0x48, - 0xc4, 0xa1, 0x3c, 0xcb, 0xd1, 0x6d, 0x6d, 0x3b, 0x3b, 0xeb, 0x6f, 0xd9, 0xa6, 0x2f, 0xe2, 0x97, - 0x67, 0x39, 0x06, 0xcd, 0xc8, 0x7c, 0x90, 0x8f, 0x61, 0x4b, 0x20, 0x67, 0x34, 0x66, 0x6f, 0x30, - 0x0a, 0xf1, 0x34, 0xe7, 0x61, 0x1e, 0xd3, 0xd4, 0x6d, 0xeb, 0x89, 0xc8, 0xcc, 0xf6, 0xf4, 0x34, - 0xe7, 0x07, 0x31, 0x4d, 0xc9, 0x0e, 0xf4, 0xb2, 0x42, 0xe6, 0x85, 0x0c, 0x75, 0xde, 0x44, 0xc8, - 0x22, 0x17, 0xf4, 0x8e, 0xd6, 0x8d, 0xfe, 0x73, 0xad, 0xee, 0x47, 0x2b, 0x49, 0xbc, 0x73, 0x25, - 0x12, 0xef, 0x5e, 0x8d, 0xc4, 0xd7, 0x56, 0x93, 0x38, 0x59, 0x87, 0x4a, 0x7a, 0xec, 0xae, 0xeb, - 0xd4, 0x54, 0xd2, 0x63, 0x95, 0x48, 0x99, 0xe5, 0xaf, 0xdc, 0x0d, 0x93, 0x48, 0xf5, 0x4d, 0xde, - 0x07, 0x48, 0x50, 0x72, 0x36, 0x56, 0x61, 0x71, 0x7b, 0x3a, 0x0f, 0x73, 0x1a, 0xf2, 0x7d, 0x58, - 0x63, 0x93, 0x34, 0xe3, 0xb8, 0xcf, 0xb3, 0x13, 0x96, 0x4e, 0xdc, 0xcd, 0x6d, 0x67, 0xa7, 0x15, - 0x2c, 0x2a, 0xc9, 0x1d, 0x68, 0x15, 0x42, 0xf5, 0x3d, 0x09, 0xba, 0x44, 0xfb, 0x98, 0xca, 0xde, - 0x3f, 0x6b, 0xb3, 0xc2, 0x14, 0x45, 0x2c, 0xc5, 0xff, 0xea, 0x0a, 0x99, 0x56, 0x73, 0x75, 0xbe, - 0x9a, 0xef, 0x43, 0xc7, 0x6c, 0xcf, 0x54, 0x4d, 0xed, 0xdc, 0x8e, 0xef, 0x43, 0x27, 0x2d, 0x92, - 0xf0, 0xb8, 0x40, 0xce, 0x50, 0xd8, 0x73, 0x0e, 0x69, 0x91, 0xbc, 0x30, 0x1a, 0x72, 0x03, 0xea, - 0x32, 0xcb, 0xc3, 0x57, 0xf6, 0x98, 0xab, 0x38, 0x7e, 0x41, 0x7e, 0x06, 0x77, 0x04, 0xd2, 0x18, - 0xa3, 0x50, 0xe0, 0x24, 0xc1, 0x54, 0xf6, 0x7d, 0x11, 0x0a, 0xbd, 0x6d, 0x8c, 0xdc, 0xa6, 0x2e, - 0x14, 0xd7, 0x20, 0x86, 0x53, 0xc0, 0xd0, 0xda, 0x55, 0x1d, 0x8c, 0x4d, 0x3f, 0xb7, 0x30, 0xac, - 0xa5, 0x1b, 0x1f, 0x32, 0x33, 0x4d, 0x07, 0xfc, 0x14, 0xdc, 0x49, 0x9c, 0x8d, 0x68, 0x1c, 0x9e, - 0x9b, 0xd5, 0x6d, 0xeb, 0xc9, 0x6e, 0x19, 0xfb, 0x70, 0x69, 0x4a, 0xb5, 0x3d, 0x11, 0xb3, 0x31, - 0x46, 0xe1, 0x28, 0xce, 0x46, 0x2e, 0xe8, 0x82, 0x07, 0xa3, 0x7a, 0x12, 0x67, 0x23, 0x55, 0xe8, - 0x16, 0xa0, 0xc2, 0x30, 0xce, 0x8a, 0x54, 0xea, 0xf2, 0xad, 0x06, 0xeb, 0x46, 0xff, 0xbc, 0x48, - 0xf6, 0x94, 0x96, 0x7c, 0x0f, 0xd6, 0x2c, 0x32, 0x3b, 0x3c, 0x14, 0x28, 0x75, 0xdd, 0x56, 0x83, - 0xae, 0x51, 0xfe, 0x52, 0xeb, 0xc8, 0x81, 0xe2, 0x5d, 0x21, 0x1f, 0x4f, 0x26, 0x1c, 0x27, 0x54, - 0x9d, 0x7b, 0x5d, 0xaf, 0x9d, 0xdd, 0x1f, 0x3c, 0x58, 0xd9, 0x38, 0x3f, 0xd8, 0x5b, 0x44, 0x07, - 0xcb, 0xc3, 0xbd, 0x63, 0xd8, 0x58, 0xc2, 0x28, 0xaa, 0xe1, 0xb6, 0x41, 0x51, 0xe5, 0x6f, 0xbb, - 0xd3, 0x05, 0x1d, 0xd9, 0x86, 0x8e, 0x40, 0xfe, 0x9a, 0x8d, 0x0d, 0xc4, 0x50, 0xdc, 0xbc, 0x4a, - 0x51, 0xb4, 0xcc, 0x24, 0x8d, 0x9f, 0xbf, 0xb0, 0x25, 0x53, 0x8a, 0xde, 0xbf, 0x6a, 0xb0, 0x11, - 0xa8, 0x12, 0xc1, 0xd7, 0xf8, 0x6d, 0xa2, 0xd7, 0xb7, 0xd1, 0x5c, 0xe3, 0x4a, 0x34, 0xd7, 0xbc, - 0x34, 0xcd, 0xb5, 0xae, 0x44, 0x73, 0xed, 0xab, 0xd1, 0x1c, 0xbc, 0x85, 0xe6, 0xb6, 0xa0, 0x1e, - 0xb3, 0x84, 0x95, 0x55, 0x6a, 0x84, 0xf3, 0xc4, 0xd5, 0x5d, 0x45, 0x5c, 0xb7, 0xa1, 0xc5, 0x84, - 0x2d, 0xf2, 0x35, 0x0d, 0x68, 0x32, 0x61, 0xaa, 0xfb, 0x29, 0xdc, 0x67, 0x12, 0xb9, 0x2e, 0xb0, - 0x10, 0x4f, 0x25, 0xa6, 0x42, 0x7d, 0x71, 0x8c, 0x8a, 0x31, 0x86, 0x9c, 0x4a, 0xb4, 0xd4, 0x7a, - 0x6f, 0x0a, 0x7b, 0x5a, 0xa2, 0x02, 0x0d, 0x0a, 0xa8, 0xc4, 0x05, 0x6a, 0xdc, 0x58, 0xa2, 0xc6, - 0xaf, 0xab, 0xf3, 0x65, 0xf5, 0x0d, 0x20, 0xc7, 0x0f, 0xa1, 0xca, 0x22, 0xd3, 0x9a, 0x75, 0x76, - 0xdd, 0x45, 0x3f, 0xf6, 0x05, 0xdb, 0xf7, 0x45, 0xa0, 0x40, 0xe4, 0x17, 0xd0, 0xb1, 0x25, 0x12, - 0x51, 0x49, 0x75, 0xf9, 0x75, 0x76, 0xdf, 0x5f, 0x39, 0x46, 0xd7, 0x8c, 0x4f, 0x25, 0x0d, 0x4c, - 0x6b, 0x25, 0xd4, 0x37, 0xf9, 0x39, 0xdc, 0x3d, 0x4f, 0x99, 0xdc, 0x86, 0x23, 0x72, 0x1b, 0xba, - 0xea, 0x6e, 0x2f, 0x73, 0x66, 0x19, 0xaf, 0x88, 0xfc, 0x04, 0xb6, 0xe6, 0x48, 0x73, 0x36, 0xb0, - 0xa9, 0x59, 0x73, 0x8e, 0x50, 0x67, 0x43, 0x2e, 0xa2, 0xcd, 0xd6, 0x85, 0xb4, 0xf9, 0x9f, 0xa7, - 0xb1, 0xaf, 0x1d, 0x68, 0x0f, 0x32, 0x1a, 0xe9, 0x86, 0xf7, 0x1a, 0x69, 0xbf, 0x07, 0xed, 0xe9, - 0xea, 0x2d, 0xa3, 0xcc, 0x14, 0xca, 0x3a, 0xed, 0x59, 0x6d, 0xa3, 0x3b, 0xd7, 0xc4, 0xce, 0x35, - 0xa3, 0xb5, 0xc5, 0x66, 0xf4, 0x3e, 0x74, 0x98, 0x5a, 0x50, 0x98, 0x53, 0x79, 0x64, 0x48, 0xa5, - 0x1d, 0x80, 0x56, 0x1d, 0x28, 0x8d, 0xea, 0x56, 0x4b, 0x80, 0xee, 0x56, 0x1b, 0x97, 0xee, 0x56, - 0xad, 0x13, 0xdd, 0xad, 0xfe, 0xce, 0x01, 0xd0, 0x1b, 0x57, 0x65, 0x79, 0xde, 0xa9, 0x73, 0x1d, - 0xa7, 0x8a, 0xed, 0xd4, 0x95, 0xc5, 0x31, 0xa6, 0x72, 0x96, 0x5b, 0x61, 0x83, 0x43, 0xd2, 0x22, - 0x09, 0x8c, 0xc9, 0xe6, 0x55, 0x78, 0x7f, 0x70, 0x00, 0x74, 0x71, 0x9a, 0x65, 0x2c, 0xd3, 0xae, - 0x73, 0x71, 0x1f, 0x5f, 0x59, 0x0c, 0xdd, 0x93, 0x32, 0x74, 0x17, 0x3c, 0x5c, 0xa7, 0xe5, 0x31, - 0xdb, 0xbc, 0x8d, 0xae, 0xfe, 0xf6, 0xfe, 0xe8, 0x40, 0xd7, 0xae, 0xce, 0x2c, 0x69, 0x21, 0xcb, - 0xce, 0x72, 0x96, 0x75, 0x33, 0x93, 0x64, 0xfc, 0x2c, 0x14, 0xec, 0x4d, 0x79, 0xa7, 0x81, 0x51, - 0x0d, 0xd9, 0x1b, 0x54, 0xfc, 0xa6, 0x43, 0x92, 0x9d, 0x88, 0xf2, 0x4e, 0x53, 0x61, 0xc8, 0x4e, - 0x84, 0xe2, 0x58, 0x8e, 0x63, 0x4c, 0x65, 0x7c, 0x16, 0x26, 0x59, 0xc4, 0x0e, 0x19, 0x46, 0xba, - 0x1a, 0x5a, 0x41, 0xaf, 0x34, 0x3c, 0xb3, 0x7a, 0xef, 0x2b, 0xf5, 0xaa, 0x36, 0x07, 0xaa, 0xfc, - 0x6d, 0xf5, 0x4c, 0x4c, 0xae, 0x51, 0xb5, 0x2a, 0xc4, 0xc6, 0x8f, 0x2a, 0x44, 0xf3, 0xa7, 0xa8, - 0x1d, 0x2c, 0xe8, 0x54, 0x4f, 0x3a, 0x65, 0x7d, 0x13, 0xc7, 0x5a, 0x30, 0xa7, 0x51, 0x2b, 0x8f, - 0xf0, 0x90, 0x16, 0xf1, 0xfc, 0xed, 0x50, 0x33, 0xb7, 0x83, 0x35, 0x2c, 0xfc, 0xc9, 0x58, 0xdf, - 0xe3, 0x18, 0x61, 0x2a, 0x19, 0x8d, 0xf5, 0xff, 0xb1, 0x79, 0x4a, 0x76, 0x16, 0x29, 0x99, 0x7c, - 0x04, 0x04, 0xd3, 0x31, 0x3f, 0xcb, 0x55, 0x05, 0xe5, 0x54, 0x88, 0x93, 0x8c, 0x47, 0xf6, 0x29, - 0xb9, 0x39, 0xb5, 0x1c, 0x58, 0x03, 0xb9, 0x05, 0x0d, 0x89, 0x29, 0x4d, 0xa5, 0x3d, 0x63, 0x56, - 0xb2, 0xf7, 0x8a, 0x28, 0x72, 0xe4, 0x36, 0xa6, 0x4d, 0x26, 0x86, 0x4a, 0x54, 0x0f, 0x51, 0x71, - 0x44, 0x77, 0x3f, 0xfd, 0x6c, 0xe6, 0xbe, 0x6e, 0x1e, 0xa2, 0x46, 0x5d, 0xfa, 0xf6, 0x9e, 0xc2, - 0xe6, 0x80, 0x09, 0x79, 0x90, 0xc5, 0x6c, 0x7c, 0x76, 0xed, 0xae, 0xc3, 0xfb, 0xbd, 0x03, 0x64, - 0xde, 0x8f, 0xfd, 0x8f, 0x33, 0xbb, 0x35, 0x9c, 0xcb, 0xdf, 0x1a, 0x1f, 0x40, 0x37, 0xd7, 0x6e, - 0x42, 0x96, 0x1e, 0x66, 0x65, 0xf6, 0x3a, 0x46, 0xa7, 0x62, 0x2b, 0xd4, 0xf3, 0x59, 0x05, 0x33, - 0xe4, 0x59, 0x8c, 0x26, 0x79, 0xed, 0xa0, 0xad, 0x34, 0x81, 0x52, 0x78, 0x13, 0xb8, 0x3d, 0x3c, - 0xca, 0x4e, 0xf6, 0xb2, 0xf4, 0x90, 0x4d, 0x0a, 0x73, 0x6d, 0xbe, 0xc3, 0xff, 0x08, 0x17, 0x9a, - 0x39, 0x95, 0xea, 0x4c, 0xd9, 0x1c, 0x95, 0xa2, 0xf7, 0x67, 0x07, 0xee, 0xac, 0x9a, 0xe9, 0x5d, - 0xb6, 0xbf, 0x0f, 0x6b, 0x63, 0xe3, 0xce, 0x78, 0xbb, 0xfc, 0x7f, 0xce, 0xc5, 0x71, 0xde, 0x53, - 0xa8, 0xe9, 0xe6, 0xe0, 0x21, 0x54, 0xb8, 0xd4, 0x2b, 0x58, 0xdf, 0xbd, 0xff, 0x16, 0xa6, 0x50, - 0x40, 0xfd, 0x78, 0xad, 0x70, 0x49, 0xba, 0xe0, 0x70, 0xbd, 0x53, 0x27, 0x70, 0xf8, 0x87, 0x7f, - 0x77, 0xa0, 0x55, 0x9a, 0xc9, 0x26, 0xac, 0xf9, 0xfe, 0x60, 0x6f, 0xca, 0x55, 0xbd, 0xef, 0x90, - 0x1e, 0x74, 0x7d, 0x7f, 0x70, 0x50, 0x76, 0x84, 0x3d, 0x87, 0x74, 0xa1, 0xe5, 0xfb, 0x03, 0x4d, - 0x3e, 0xbd, 0x8a, 0x95, 0x3e, 0x8f, 0x0b, 0x71, 0xd4, 0xab, 0x4e, 0x1d, 0x24, 0x39, 0x35, 0x0e, - 0x6a, 0x64, 0x0d, 0xda, 0xfe, 0xb3, 0x41, 0x3f, 0x15, 0xc8, 0x65, 0xaf, 0x6e, 0x45, 0x1f, 0x63, - 0x94, 0xd8, 0x6b, 0x90, 0x0d, 0xe8, 0xf8, 0xcf, 0x06, 0x4f, 0x8a, 0xf8, 0x95, 0xba, 0xc7, 0x7a, - 0x4d, 0x6d, 0x7f, 0x31, 0x30, 0x8f, 0x94, 0x5e, 0x4b, 0xbb, 0x7f, 0x31, 0x50, 0xcf, 0xa6, 0xb3, - 0x5e, 0xdb, 0x0e, 0xfe, 0x55, 0xae, 0x7d, 0xc1, 0x93, 0x47, 0xbf, 0xf9, 0x74, 0xc2, 0xe4, 0x51, - 0x31, 0x52, 0xf1, 0x7a, 0x68, 0xb6, 0xfe, 0x11, 0xcb, 0xec, 0xd7, 0xc3, 0x72, 0xfb, 0x0f, 0x75, - 0x34, 0xa6, 0x62, 0x3e, 0x1a, 0x35, 0xb4, 0xe6, 0x93, 0x7f, 0x07, 0x00, 0x00, 0xff, 0xff, 0xc1, - 0x18, 0x48, 0x84, 0x88, 0x17, 0x00, 0x00, -} diff --git a/internal/proto/segcore.proto b/internal/proto/segcore.proto index 92b056bda3..ea7697f48c 100644 --- a/internal/proto/segcore.proto +++ b/internal/proto/segcore.proto @@ -9,6 +9,7 @@ message RetrieveResults { schema.IDs ids = 1; repeated int64 offset = 2; repeated schema.FieldData fields_data = 3; + int64 all_retrieve_count = 4; } message LoadFieldMeta { diff --git a/internal/proxy/authentication_interceptor_test.go b/internal/proxy/authentication_interceptor_test.go index 7eda478be7..be2863cd31 100644 --- a/internal/proxy/authentication_interceptor_test.go +++ b/internal/proxy/authentication_interceptor_test.go @@ -10,6 +10,7 @@ import ( "google.golang.org/grpc/metadata" "github.com/milvus-io/milvus/internal/mocks" + "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/crypto" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -140,5 +141,5 @@ func TestAuthenticationInterceptor(t *testing.T) { user, _ := parseMD(rawToken) assert.Equal(t, "mockUser", user) } - hoo = defaultHook{} + hoo = hookutil.DefaultHook{} } diff --git a/internal/proxy/channels_mgr.go b/internal/proxy/channels_mgr.go index 69d3fbc9ef..641a23b726 100644 --- a/internal/proxy/channels_mgr.go +++ b/internal/proxy/channels_mgr.go @@ -177,7 +177,6 @@ func createStream(factory msgstream.Factory, pchans []pChan, repack repackFuncTy var err error stream, err = factory.NewMsgStream(context.Background()) - if err != nil { return nil, err } diff --git a/internal/proxy/count_reducer.go b/internal/proxy/count_reducer.go index 90d1cb9137..7c8cdd7e69 100644 --- a/internal/proxy/count_reducer.go +++ b/internal/proxy/count_reducer.go @@ -4,6 +4,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/proto/internalpb" "github.com/milvus-io/milvus/internal/util/funcutil" + "github.com/milvus-io/milvus/pkg/util/merr" ) type cntReducer struct { @@ -20,6 +21,7 @@ func (r *cntReducer) Reduce(results []*internalpb.RetrieveResults) (*milvuspb.Qu cnt += c } res := funcutil.WrapCntToQueryResults(cnt) + res.Status = merr.Success() res.CollectionName = r.collectionName return res, nil } diff --git a/internal/proxy/hook_interceptor.go b/internal/proxy/hook_interceptor.go index 008c6c1ea9..79e833b583 100644 --- a/internal/proxy/hook_interceptor.go +++ b/internal/proxy/hook_interceptor.go @@ -2,93 +2,24 @@ package proxy import ( "context" - "fmt" - "plugin" "strconv" "strings" - "github.com/cockroachdb/errors" "go.uber.org/zap" "google.golang.org/grpc" "github.com/milvus-io/milvus-proto/go-api/v2/hook" - "github.com/milvus-io/milvus/pkg/config" + "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/util/paramtable" ) -type defaultHook struct{} - -func (d defaultHook) VerifyAPIKey(key string) (string, error) { - return "", errors.New("default hook, can't verify api key") -} - -func (d defaultHook) Init(params map[string]string) error { - return nil -} - -func (d defaultHook) Mock(ctx context.Context, req interface{}, fullMethod string) (bool, interface{}, error) { - return false, nil, nil -} - -func (d defaultHook) Before(ctx context.Context, req interface{}, fullMethod string) (context.Context, error) { - return ctx, nil -} - -func (d defaultHook) After(ctx context.Context, result interface{}, err error, fullMethod string) error { - return nil -} - -func (d defaultHook) Release() {} - var hoo hook.Hook -func initHook() error { - path := Params.ProxyCfg.SoPath.GetValue() - if path == "" { - hoo = defaultHook{} - return nil - } - - logger.Debug("start to load plugin", zap.String("path", path)) - p, err := plugin.Open(path) - if err != nil { - return fmt.Errorf("fail to open the plugin, error: %s", err.Error()) - } - logger.Debug("plugin open") - - h, err := p.Lookup("MilvusHook") - if err != nil { - return fmt.Errorf("fail to the 'MilvusHook' object in the plugin, error: %s", err.Error()) - } - - var ok bool - hoo, ok = h.(hook.Hook) - if !ok { - return fmt.Errorf("fail to convert the `Hook` interface") - } - if err = hoo.Init(paramtable.GetHookParams().SoConfig.GetValue()); err != nil { - return fmt.Errorf("fail to init configs for the hook, error: %s", err.Error()) - } - paramtable.GetHookParams().WatchHookWithPrefix("watch_hook", "", func(event *config.Event) { - log.Info("receive the hook refresh event", zap.Any("event", event)) - go func() { - soConfig := paramtable.GetHookParams().SoConfig.GetValue() - log.Info("refresh hook configs", zap.Any("config", soConfig)) - if err = hoo.Init(soConfig); err != nil { - log.Panic("fail to init configs for the hook when refreshing", zap.Error(err)) - } - }() - }) - return nil -} - func UnaryServerHookInterceptor() grpc.UnaryServerInterceptor { - if hookError := initHook(); hookError != nil { - logger.Error("hook error", zap.String("path", Params.ProxyCfg.SoPath.GetValue()), zap.Error(hookError)) - hoo = defaultHook{} - } + hookutil.InitOnceHook() + hoo = hookutil.Hoo return func(ctx context.Context, req any, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) { var ( fullMethod = info.FullMethod @@ -145,24 +76,13 @@ func getCurrentUser(ctx context.Context) string { return username } -// MockAPIHook is a mock hook for api key verification, ONLY FOR TEST -type MockAPIHook struct { - defaultHook - mockErr error - apiUser string -} - -func (m MockAPIHook) VerifyAPIKey(apiKey string) (string, error) { - return m.apiUser, m.mockErr -} - func SetMockAPIHook(apiUser string, mockErr error) { if apiUser == "" && mockErr == nil { - hoo = defaultHook{} + hoo = &hookutil.DefaultHook{} return } - hoo = MockAPIHook{ - mockErr: mockErr, - apiUser: apiUser, + hoo = &hookutil.MockAPIHook{ + MockErr: mockErr, + User: apiUser, } } diff --git a/internal/proxy/hook_interceptor_test.go b/internal/proxy/hook_interceptor_test.go index a387053b8e..3641f86d25 100644 --- a/internal/proxy/hook_interceptor_test.go +++ b/internal/proxy/hook_interceptor_test.go @@ -8,22 +8,11 @@ import ( "github.com/stretchr/testify/assert" "google.golang.org/grpc" - "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/internal/util/hookutil" ) -func TestInitHook(t *testing.T) { - paramtable.Get().Save(Params.ProxyCfg.SoPath.Key, "") - initHook() - assert.IsType(t, defaultHook{}, hoo) - - paramtable.Get().Save(Params.ProxyCfg.SoPath.Key, "/a/b/hook.so") - err := initHook() - assert.Error(t, err) - paramtable.Get().Save(Params.ProxyCfg.SoPath.Key, "") -} - type mockHook struct { - defaultHook + hookutil.DefaultHook mockRes interface{} mockErr error } @@ -39,7 +28,7 @@ type req struct { type BeforeMockCtxKey int type beforeMock struct { - defaultHook + hookutil.DefaultHook method string ctxKey BeforeMockCtxKey ctxValue string @@ -60,7 +49,7 @@ type resp struct { } type afterMock struct { - defaultHook + hookutil.DefaultHook method string err error } @@ -129,7 +118,7 @@ func TestHookInterceptor(t *testing.T) { assert.Equal(t, re.method, afterHoo.method) assert.Equal(t, err, afterHoo.err) - hoo = defaultHook{} + hoo = &hookutil.DefaultHook{} res, err = interceptor(ctx, r, info, func(ctx context.Context, r interface{}) (interface{}, error) { return &resp{ method: r.(*req).method, @@ -139,18 +128,6 @@ func TestHookInterceptor(t *testing.T) { assert.NoError(t, err) } -func TestDefaultHook(t *testing.T) { - d := defaultHook{} - assert.NoError(t, d.Init(nil)) - { - _, err := d.VerifyAPIKey("key") - assert.Error(t, err) - } - assert.NotPanics(t, func() { - d.Release() - }) -} - func TestUpdateProxyFunctionCallMetric(t *testing.T) { assert.NotPanics(t, func() { updateProxyFunctionCallMetric("/milvus.proto.milvus.MilvusService/Flush") diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index da927aa76b..325f232827 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -42,6 +42,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/proxypb" "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proxy/connection" + "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/internal/util/importutilv2" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" @@ -2394,6 +2395,15 @@ func (node *Proxy) Insert(ctx context.Context, request *milvuspb.InsertRequest) metrics.ProxyFunctionCall.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method, metrics.SuccessLabel).Inc() successCnt := it.result.InsertCnt - int64(len(it.result.ErrIndex)) + v := Extension.Report(map[string]any{ + hookutil.OpTypeKey: hookutil.OpTypeInsert, + hookutil.DatabaseKey: request.DbName, + hookutil.UsernameKey: GetCurUserFromContextOrDefault(ctx), + hookutil.DataSizeKey: proto.Size(request), + hookutil.SuccessCntKey: successCnt, + hookutil.FailCntKey: len(it.result.ErrIndex), + }) + SetReportValue(it.result.GetStatus(), v) metrics.ProxyInsertVectors.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Add(float64(successCnt)) metrics.ProxyMutationLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), metrics.InsertLabel).Observe(float64(tr.ElapseSpan().Milliseconds())) metrics.ProxyCollectionMutationLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), metrics.InsertLabel, request.CollectionName).Observe(float64(tr.ElapseSpan().Milliseconds())) @@ -2469,6 +2479,15 @@ func (node *Proxy) Delete(ctx context.Context, request *milvuspb.DeleteRequest) successCnt := dr.result.GetDeleteCnt() metrics.ProxyDeleteVectors.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Add(float64(successCnt)) + v := Extension.Report(map[string]any{ + hookutil.OpTypeKey: hookutil.OpTypeDelete, + hookutil.DatabaseKey: request.DbName, + hookutil.UsernameKey: GetCurUserFromContextOrDefault(ctx), + hookutil.SuccessCntKey: successCnt, + hookutil.RelatedCntKey: dr.allQueryCnt.Load(), + }) + SetReportValue(dr.result.GetStatus(), v) + metrics.ProxyFunctionCall.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method, metrics.SuccessLabel).Inc() metrics.ProxyMutationLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), metrics.DeleteLabel).Observe(float64(tr.ElapseSpan().Milliseconds())) @@ -2584,6 +2603,16 @@ func (node *Proxy) Upsert(ctx context.Context, request *milvuspb.UpsertRequest) // UpsertCnt always equals to the number of entities in the request it.result.UpsertCnt = int64(request.NumRows) + v := Extension.Report(map[string]any{ + hookutil.OpTypeKey: hookutil.OpTypeUpsert, + hookutil.DatabaseKey: request.DbName, + hookutil.UsernameKey: GetCurUserFromContextOrDefault(ctx), + hookutil.DataSizeKey: proto.Size(it.req), + hookutil.SuccessCntKey: it.result.UpsertCnt, + hookutil.FailCntKey: len(it.result.ErrIndex), + }) + SetReportValue(it.result.GetStatus(), v) + rateCol.Add(internalpb.RateType_DMLUpsert.String(), float64(it.upsertMsg.DeleteMsg.Size()+it.upsertMsg.DeleteMsg.Size())) metrics.ProxyFunctionCall.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), method, @@ -2759,6 +2788,15 @@ func (node *Proxy) Search(ctx context.Context, request *milvuspb.SearchRequest) if qt.result != nil { sentSize := proto.Size(qt.result) + v := Extension.Report(map[string]any{ + hookutil.OpTypeKey: hookutil.OpTypeSearch, + hookutil.DatabaseKey: request.DbName, + hookutil.UsernameKey: GetCurUserFromContextOrDefault(ctx), + hookutil.DataSizeKey: sentSize, + hookutil.RelatedCntKey: qt.result.GetResults().GetAllSearchCount(), + hookutil.DimensionKey: qt.dimension, + }) + SetReportValue(qt.result.GetStatus(), v) metrics.ProxyReadReqSendBytes.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Add(float64(sentSize)) rateCol.Add(metricsinfo.ReadResultThroughput, float64(sentSize)) } @@ -2902,6 +2940,15 @@ func (node *Proxy) HybridSearch(ctx context.Context, request *milvuspb.HybridSea if qt.result != nil { sentSize := proto.Size(qt.result) + v := Extension.Report(map[string]any{ + hookutil.OpTypeKey: hookutil.OpTypeHybridSearch, + hookutil.DatabaseKey: request.DbName, + hookutil.UsernameKey: GetCurUserFromContextOrDefault(ctx), + hookutil.DataSizeKey: sentSize, + hookutil.RelatedCntKey: qt.result.GetResults().GetAllSearchCount(), + hookutil.DimensionKey: qt.dimension, + }) + SetReportValue(qt.result.GetStatus(), v) metrics.ProxyReadReqSendBytes.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Add(float64(sentSize)) rateCol.Add(metricsinfo.ReadResultThroughput, float64(sentSize)) } @@ -3182,7 +3229,19 @@ func (node *Proxy) Query(ctx context.Context, request *milvuspb.QueryRequest) (* qc: node.queryCoord, lb: node.lbPolicy, } - return node.query(ctx, qt) + res, err := node.query(ctx, qt) + if merr.Ok(res.Status) && err == nil { + v := Extension.Report(map[string]any{ + hookutil.OpTypeKey: hookutil.OpTypeQuery, + hookutil.DatabaseKey: request.DbName, + hookutil.UsernameKey: GetCurUserFromContextOrDefault(ctx), + hookutil.DataSizeKey: proto.Size(res), + hookutil.RelatedCntKey: qt.allQueryCnt, + hookutil.DimensionKey: qt.dimension, + }) + SetReportValue(res.Status, v) + } + return res, err } // CreateAlias create alias for collection, then you can search the collection with alias. diff --git a/internal/proxy/meta_cache.go b/internal/proxy/meta_cache.go index f4bae21eb8..1a5326746e 100644 --- a/internal/proxy/meta_cache.go +++ b/internal/proxy/meta_cache.go @@ -59,6 +59,8 @@ type Cache interface { GetCollectionName(ctx context.Context, database string, collectionID int64) (string, error) // GetCollectionInfo get collection's information by name or collection id, such as schema, and etc. GetCollectionInfo(ctx context.Context, database, collectionName string, collectionID int64) (*collectionBasicInfo, error) + // GetCollectionNamesByID get collection name and database name by collection id + GetCollectionNamesByID(ctx context.Context, collectionID []UniqueID) ([]string, []string, error) // GetPartitionID get partition's identifier of specific collection. GetPartitionID(ctx context.Context, database, collectionName string, partitionName string) (typeutil.UniqueID, error) // GetPartitions get all partitions' id of specific collection. @@ -242,11 +244,12 @@ type MetaCache struct { rootCoord types.RootCoordClient queryCoord types.QueryCoordClient - collInfo map[string]map[string]*collectionInfo // database -> collectionName -> collection_info - collLeader map[string]map[string]*shardLeaders // database -> collectionName -> collection_leaders - credMap map[string]*internalpb.CredentialInfo // cache for credential, lazy load - privilegeInfos map[string]struct{} // privileges cache - userToRoles map[string]map[string]struct{} // user to role cache + collInfo map[string]map[string]*collectionInfo // database -> collectionName -> collection_info + collLeader map[string]map[string]*shardLeaders // database -> collectionName -> collection_leaders + dbInfo map[string]map[typeutil.UniqueID]string // database -> collectionID -> collectionName + credMap map[string]*internalpb.CredentialInfo // cache for credential, lazy load + privilegeInfos map[string]struct{} // privileges cache + userToRoles map[string]map[string]struct{} // user to role cache mu sync.RWMutex credMut sync.RWMutex leaderMut sync.RWMutex @@ -288,6 +291,7 @@ func NewMetaCache(rootCoord types.RootCoordClient, queryCoord types.QueryCoordCl queryCoord: queryCoord, collInfo: map[string]map[string]*collectionInfo{}, collLeader: map[string]map[string]*shardLeaders{}, + dbInfo: map[string]map[typeutil.UniqueID]string{}, credMap: map[string]*internalpb.CredentialInfo{}, shardMgr: shardMgr, privilegeInfos: map[string]struct{}{}, @@ -471,6 +475,90 @@ func (m *MetaCache) GetCollectionInfo(ctx context.Context, database string, coll return collInfo.getBasicInfo(), nil } +func (m *MetaCache) GetCollectionNamesByID(ctx context.Context, collectionIDs []UniqueID) ([]string, []string, error) { + hasUpdate := false + + dbNames := make([]string, 0) + collectionNames := make([]string, 0) + for _, collectionID := range collectionIDs { + dbName, collectionName := m.innerGetCollectionByID(collectionID) + if dbName != "" { + dbNames = append(dbNames, dbName) + collectionNames = append(collectionNames, collectionName) + continue + } + if hasUpdate { + return nil, nil, errors.New("collection not found after meta cache has been updated") + } + hasUpdate = true + err := m.updateDBInfo(ctx) + if err != nil { + return nil, nil, err + } + dbName, collectionName = m.innerGetCollectionByID(collectionID) + if dbName == "" { + return nil, nil, errors.New("collection not found") + } + dbNames = append(dbNames, dbName) + collectionNames = append(collectionNames, collectionName) + } + + return dbNames, collectionNames, nil +} + +func (m *MetaCache) innerGetCollectionByID(collectionID int64) (string, string) { + m.mu.RLock() + defer m.mu.RUnlock() + + for database, db := range m.dbInfo { + name, ok := db[collectionID] + if ok { + return database, name + } + } + return "", "" +} + +func (m *MetaCache) updateDBInfo(ctx context.Context) error { + databaseResp, err := m.rootCoord.ListDatabases(ctx, &milvuspb.ListDatabasesRequest{ + Base: commonpbutil.NewMsgBase(commonpbutil.WithMsgType(commonpb.MsgType_ListDatabases)), + }) + + if err := merr.CheckRPCCall(databaseResp, err); err != nil { + log.Warn("failed to ListDatabases", zap.Error(err)) + return err + } + + dbInfo := make(map[string]map[int64]string) + for _, dbName := range databaseResp.DbNames { + resp, err := m.rootCoord.ShowCollections(ctx, &milvuspb.ShowCollectionsRequest{ + Base: commonpbutil.NewMsgBase( + commonpbutil.WithMsgType(commonpb.MsgType_ShowCollections), + ), + DbName: dbName, + }) + + if err := merr.CheckRPCCall(resp, err); err != nil { + log.Warn("failed to ShowCollections", + zap.String("dbName", dbName), + zap.Error(err)) + return err + } + + collections := make(map[int64]string) + for i, collection := range resp.CollectionNames { + collections[resp.CollectionIds[i]] = collection + } + dbInfo[dbName] = collections + } + + m.mu.Lock() + defer m.mu.Unlock() + m.dbInfo = dbInfo + + return nil +} + // GetCollectionInfo returns the collection information related to provided collection name // If the information is not found, proxy will try to fetch information for other source (RootCoord for now) // TODO: may cause data race of this implementation, should be refactored in future. diff --git a/internal/proxy/meta_cache_test.go b/internal/proxy/meta_cache_test.go index ac8d9b7cfe..df2a65fbf6 100644 --- a/internal/proxy/meta_cache_test.go +++ b/internal/proxy/meta_cache_test.go @@ -881,3 +881,149 @@ func TestMetaCache_AllocID(t *testing.T) { assert.Equal(t, id, int64(0)) }) } + +func TestGlobalMetaCache_UpdateDBInfo(t *testing.T) { + rootCoord := mocks.NewMockRootCoordClient(t) + queryCoord := mocks.NewMockQueryCoordClient(t) + shardMgr := newShardClientMgr() + ctx := context.Background() + + cache, err := NewMetaCache(rootCoord, queryCoord, shardMgr) + assert.NoError(t, err) + + t.Run("fail to list db", func(t *testing.T) { + rootCoord.EXPECT().ListDatabases(mock.Anything, mock.Anything).Return(&milvuspb.ListDatabasesResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Code: 500, + }, + }, nil).Once() + err := cache.updateDBInfo(ctx) + assert.Error(t, err) + }) + + t.Run("fail to list collection", func(t *testing.T) { + rootCoord.EXPECT().ListDatabases(mock.Anything, mock.Anything).Return(&milvuspb.ListDatabasesResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + DbNames: []string{"db1"}, + }, nil).Once() + rootCoord.EXPECT().ShowCollections(mock.Anything, mock.Anything).Return(&milvuspb.ShowCollectionsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Code: 500, + }, + }, nil).Once() + err := cache.updateDBInfo(ctx) + assert.Error(t, err) + }) + + t.Run("success", func(t *testing.T) { + rootCoord.EXPECT().ListDatabases(mock.Anything, mock.Anything).Return(&milvuspb.ListDatabasesResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + DbNames: []string{"db1"}, + }, nil).Once() + rootCoord.EXPECT().ShowCollections(mock.Anything, mock.Anything).Return(&milvuspb.ShowCollectionsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + CollectionNames: []string{"collection1"}, + CollectionIds: []int64{1}, + }, nil).Once() + err := cache.updateDBInfo(ctx) + assert.NoError(t, err) + assert.Len(t, cache.dbInfo, 1) + assert.Len(t, cache.dbInfo["db1"], 1) + assert.Equal(t, "collection1", cache.dbInfo["db1"][1]) + }) +} + +func TestGlobalMetaCache_GetCollectionNamesByID(t *testing.T) { + rootCoord := mocks.NewMockRootCoordClient(t) + queryCoord := mocks.NewMockQueryCoordClient(t) + shardMgr := newShardClientMgr() + ctx := context.Background() + + t.Run("fail to update db info", func(t *testing.T) { + rootCoord.EXPECT().ListDatabases(mock.Anything, mock.Anything).Return(&milvuspb.ListDatabasesResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Code: 500, + }, + }, nil).Once() + + cache, err := NewMetaCache(rootCoord, queryCoord, shardMgr) + assert.NoError(t, err) + + _, _, err = cache.GetCollectionNamesByID(ctx, []int64{1}) + assert.Error(t, err) + }) + + t.Run("not found collection", func(t *testing.T) { + rootCoord.EXPECT().ListDatabases(mock.Anything, mock.Anything).Return(&milvuspb.ListDatabasesResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + DbNames: []string{"db1"}, + }, nil).Once() + rootCoord.EXPECT().ShowCollections(mock.Anything, mock.Anything).Return(&milvuspb.ShowCollectionsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + CollectionNames: []string{"collection1"}, + CollectionIds: []int64{1}, + }, nil).Once() + + cache, err := NewMetaCache(rootCoord, queryCoord, shardMgr) + assert.NoError(t, err) + _, _, err = cache.GetCollectionNamesByID(ctx, []int64{2}) + assert.Error(t, err) + }) + + t.Run("not found collection 2", func(t *testing.T) { + rootCoord.EXPECT().ListDatabases(mock.Anything, mock.Anything).Return(&milvuspb.ListDatabasesResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + DbNames: []string{"db1"}, + }, nil).Once() + rootCoord.EXPECT().ShowCollections(mock.Anything, mock.Anything).Return(&milvuspb.ShowCollectionsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + CollectionNames: []string{"collection1"}, + CollectionIds: []int64{1}, + }, nil).Once() + + cache, err := NewMetaCache(rootCoord, queryCoord, shardMgr) + assert.NoError(t, err) + _, _, err = cache.GetCollectionNamesByID(ctx, []int64{1, 2}) + assert.Error(t, err) + }) + + t.Run("success", func(t *testing.T) { + rootCoord.EXPECT().ListDatabases(mock.Anything, mock.Anything).Return(&milvuspb.ListDatabasesResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + DbNames: []string{"db1"}, + }, nil).Once() + rootCoord.EXPECT().ShowCollections(mock.Anything, mock.Anything).Return(&milvuspb.ShowCollectionsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + CollectionNames: []string{"collection1", "collection2"}, + CollectionIds: []int64{1, 2}, + }, nil).Once() + + cache, err := NewMetaCache(rootCoord, queryCoord, shardMgr) + assert.NoError(t, err) + dbNames, collectionNames, err := cache.GetCollectionNamesByID(ctx, []int64{1, 2}) + assert.NoError(t, err) + assert.Equal(t, []string{"collection1", "collection2"}, collectionNames) + assert.Equal(t, []string{"db1", "db1"}, dbNames) + }) +} diff --git a/internal/proxy/mock_cache.go b/internal/proxy/mock_cache.go index 3deb177a5c..6d61e17df0 100644 --- a/internal/proxy/mock_cache.go +++ b/internal/proxy/mock_cache.go @@ -275,6 +275,70 @@ func (_c *MockCache_GetCollectionName_Call) RunAndReturn(run func(context.Contex return _c } +// GetCollectionNamesByID provides a mock function with given fields: ctx, collectionID +func (_m *MockCache) GetCollectionNamesByID(ctx context.Context, collectionID []int64) ([]string, []string, error) { + ret := _m.Called(ctx, collectionID) + + var r0 []string + var r1 []string + var r2 error + if rf, ok := ret.Get(0).(func(context.Context, []int64) ([]string, []string, error)); ok { + return rf(ctx, collectionID) + } + if rf, ok := ret.Get(0).(func(context.Context, []int64) []string); ok { + r0 = rf(ctx, collectionID) + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]string) + } + } + + if rf, ok := ret.Get(1).(func(context.Context, []int64) []string); ok { + r1 = rf(ctx, collectionID) + } else { + if ret.Get(1) != nil { + r1 = ret.Get(1).([]string) + } + } + + if rf, ok := ret.Get(2).(func(context.Context, []int64) error); ok { + r2 = rf(ctx, collectionID) + } else { + r2 = ret.Error(2) + } + + return r0, r1, r2 +} + +// MockCache_GetCollectionNamesByID_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetCollectionNamesByID' +type MockCache_GetCollectionNamesByID_Call struct { + *mock.Call +} + +// GetCollectionNamesByID is a helper method to define mock.On call +// - ctx context.Context +// - collectionID []int64 +func (_e *MockCache_Expecter) GetCollectionNamesByID(ctx interface{}, collectionID interface{}) *MockCache_GetCollectionNamesByID_Call { + return &MockCache_GetCollectionNamesByID_Call{Call: _e.mock.On("GetCollectionNamesByID", ctx, collectionID)} +} + +func (_c *MockCache_GetCollectionNamesByID_Call) Run(run func(ctx context.Context, collectionID []int64)) *MockCache_GetCollectionNamesByID_Call { + _c.Call.Run(func(args mock.Arguments) { + run(args[0].(context.Context), args[1].([]int64)) + }) + return _c +} + +func (_c *MockCache_GetCollectionNamesByID_Call) Return(_a0 []string, _a1 []string, _a2 error) *MockCache_GetCollectionNamesByID_Call { + _c.Call.Return(_a0, _a1, _a2) + return _c +} + +func (_c *MockCache_GetCollectionNamesByID_Call) RunAndReturn(run func(context.Context, []int64) ([]string, []string, error)) *MockCache_GetCollectionNamesByID_Call { + _c.Call.Return(run) + return _c +} + // GetCollectionSchema provides a mock function with given fields: ctx, database, collectionName func (_m *MockCache) GetCollectionSchema(ctx context.Context, database string, collectionName string) (*schemaInfo, error) { ret := _m.Called(ctx, database, collectionName) diff --git a/internal/proxy/proxy.go b/internal/proxy/proxy.go index aa18485337..efa2c0e53f 100644 --- a/internal/proxy/proxy.go +++ b/internal/proxy/proxy.go @@ -26,11 +26,13 @@ import ( "time" "github.com/cockroachdb/errors" + "github.com/samber/lo" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/atomic" "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus-proto/go-api/v2/hook" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/proto/internalpb" @@ -38,6 +40,7 @@ import ( "github.com/milvus-io/milvus/internal/proxy/connection" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" + "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" @@ -45,6 +48,7 @@ import ( "github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/expr" "github.com/milvus-io/milvus/pkg/util/logutil" + "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/ratelimitutil" @@ -65,10 +69,11 @@ type Timestamp = typeutil.Timestamp // make sure Proxy implements types.Proxy var _ types.Proxy = (*Proxy)(nil) -var Params *paramtable.ComponentParam = paramtable.Get() - -// rateCol is global rateCollector in Proxy. -var rateCol *ratelimitutil.RateCollector +var ( + Params = paramtable.Get() + Extension hook.Extension + rateCol *ratelimitutil.RateCollector +) // Proxy of milvus type Proxy struct { @@ -151,6 +156,8 @@ func NewProxy(ctx context.Context, factory dependency.Factory) (*Proxy, error) { } node.UpdateStateCode(commonpb.StateCode_Abnormal) expr.Register("proxy", node) + hookutil.InitOnceHook() + Extension = hookutil.Extension logutil.Logger(ctx).Debug("create a new Proxy instance", zap.Any("state", node.stateCode.Load())) return node, nil } @@ -415,6 +422,12 @@ func (node *Proxy) Start() error { cb() } + Extension.Report(map[string]any{ + hookutil.OpTypeKey: hookutil.OpTypeNodeID, + hookutil.NodeIDKey: paramtable.GetNodeID(), + }) + node.startReportCollectionStorage() + log.Debug("update state code", zap.String("role", typeutil.ProxyRole), zap.String("State", commonpb.StateCode_Healthy.String())) node.UpdateStateCode(commonpb.StateCode_Healthy) @@ -537,3 +550,87 @@ func (node *Proxy) GetRateLimiter() (types.Limiter, error) { } return node.multiRateLimiter, nil } + +func (node *Proxy) startReportCollectionStorage() { + go func() { + tick := time.NewTicker(30 * time.Second) + defer tick.Stop() + for { + select { + case <-node.ctx.Done(): + return + case <-tick.C: + _ = node.reportCollectionStorage() + } + } + }() +} + +func (node *Proxy) reportCollectionStorage() error { + if node.dataCoord == nil { + return errors.New("nil datacoord") + } + req, err := metricsinfo.ConstructRequestByMetricType(metricsinfo.CollectionStorageMetrics) + if err != nil { + return err + } + + rsp, err := node.dataCoord.GetMetrics(node.ctx, req) + if err = merr.CheckRPCCall(rsp, err); err != nil { + log.Warn("failed to get metrics", zap.Error(err)) + return err + } + + dataCoordTopology := &metricsinfo.DataCoordTopology{} + err = metricsinfo.UnmarshalTopology(rsp.GetResponse(), dataCoordTopology) + if err != nil { + log.Warn("failed to unmarshal topology", zap.Error(err)) + return err + } + + quotaMetric := dataCoordTopology.Cluster.Self.QuotaMetrics + if quotaMetric == nil { + log.Warn("quota metric is nil") + return errors.New("quota metric is nil") + } + + ctx, cancelFunc := context.WithTimeout(node.ctx, 5*time.Second) + defer cancelFunc() + + ids := lo.Keys(quotaMetric.CollectionBinlogSize) + dbNames, collectionNames, err := globalMetaCache.GetCollectionNamesByID(ctx, ids) + if err != nil { + log.Warn("failed to get collection names", zap.Error(err)) + return err + } + + if len(ids) != len(dbNames) || len(ids) != len(collectionNames) { + log.Warn("failed to get collection names", + zap.Int("len(ids)", len(ids)), + zap.Int("len(dbNames)", len(dbNames)), + zap.Int("len(collectionNames)", len(collectionNames))) + return errors.New("failed to get collection names") + } + + nameInfos := make(map[typeutil.UniqueID]lo.Tuple2[string, string]) + for i, k := range ids { + nameInfos[k] = lo.Tuple2[string, string]{A: dbNames[i], B: collectionNames[i]} + } + + storeInfo := make(map[string]int64) + for collectionID, dataSize := range quotaMetric.CollectionBinlogSize { + nameTuple, ok := nameInfos[collectionID] + if !ok { + continue + } + storeInfo[nameTuple.A] += dataSize + } + + if len(storeInfo) > 0 { + Extension.Report(map[string]any{ + hookutil.OpTypeKey: hookutil.OpTypeStorage, + hookutil.StorageDetailKey: lo.MapValues(storeInfo, func(v int64, _ string) any { return v }), + }) + } + return nil +} diff --git a/internal/proxy/proxy_test.go b/internal/proxy/proxy_test.go index 30df1c3af0..48830d7bbe 100644 --- a/internal/proxy/proxy_test.go +++ b/internal/proxy/proxy_test.go @@ -60,6 +60,7 @@ import ( "github.com/milvus-io/milvus/internal/proto/rootcoordpb" "github.com/milvus-io/milvus/internal/util/componentutil" "github.com/milvus-io/milvus/internal/util/dependency" + "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" @@ -1085,7 +1086,7 @@ func TestProxy(t *testing.T) { assert.NotEqual(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode()) }) - var insertedIds []int64 + var insertedIDs []int64 wg.Add(1) t.Run("insert", func(t *testing.T) { defer wg.Done() @@ -1100,7 +1101,7 @@ func TestProxy(t *testing.T) { switch field := resp.GetIDs().GetIdField().(type) { case *schemapb.IDs_IntId: - insertedIds = field.IntId.GetData() + insertedIDs = field.IntId.GetData() default: t.Fatalf("Unexpected ID type") } @@ -1611,7 +1612,7 @@ func TestProxy(t *testing.T) { nq = 10 constructPrimaryKeysPlaceholderGroup := func() *commonpb.PlaceholderGroup { - expr := fmt.Sprintf("%v in [%v]", int64Field, insertedIds[0]) + expr := fmt.Sprintf("%v in [%v]", int64Field, insertedIDs[0]) exprBytes := []byte(expr) return &commonpb.PlaceholderGroup{ @@ -4803,3 +4804,227 @@ func TestUnhealthProxy_GetIndexStatistics(t *testing.T) { assert.Equal(t, commonpb.ErrorCode_NotReadyServe, resp.GetStatus().GetErrorCode()) }) } + +func TestProxy_ReportCollectionStorage(t *testing.T) { + t.Run("nil datacoord", func(t *testing.T) { + proxy := &Proxy{} + err := proxy.reportCollectionStorage() + assert.Error(t, err) + }) + + t.Run("fail to get metric", func(t *testing.T) { + datacoord := mocks.NewMockDataCoordClient(t) + datacoord.EXPECT().GetMetrics(mock.Anything, mock.Anything).Return(&milvuspb.GetMetricsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_UnexpectedError, + Code: 500, + }, + }, nil).Once() + + ctx := context.Background() + proxy := &Proxy{ + ctx: ctx, + dataCoord: datacoord, + } + err := proxy.reportCollectionStorage() + assert.Error(t, err) + }) + + t.Run("fail to unmarshal metric", func(t *testing.T) { + datacoord := mocks.NewMockDataCoordClient(t) + datacoord.EXPECT().GetMetrics(mock.Anything, mock.Anything).Return(&milvuspb.GetMetricsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Response: "invalid", + }, nil).Once() + + ctx := context.Background() + proxy := &Proxy{ + ctx: ctx, + dataCoord: datacoord, + } + err := proxy.reportCollectionStorage() + assert.Error(t, err) + }) + + t.Run("empty metric", func(t *testing.T) { + datacoord := mocks.NewMockDataCoordClient(t) + + r, _ := json.Marshal(&metricsinfo.DataCoordTopology{ + Cluster: metricsinfo.DataClusterTopology{ + Self: metricsinfo.DataCoordInfos{}, + }, + }) + + datacoord.EXPECT().GetMetrics(mock.Anything, mock.Anything).Return(&milvuspb.GetMetricsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Response: string(r), + ComponentName: "DataCoord", + }, nil).Once() + + ctx := context.Background() + proxy := &Proxy{ + ctx: ctx, + dataCoord: datacoord, + } + err := proxy.reportCollectionStorage() + assert.Error(t, err) + }) + + t.Run("fail to get cache", func(t *testing.T) { + origin := globalMetaCache + defer func() { + globalMetaCache = origin + }() + mockCache := NewMockCache(t) + globalMetaCache = mockCache + + datacoord := mocks.NewMockDataCoordClient(t) + r, _ := json.Marshal(&metricsinfo.DataCoordTopology{ + Cluster: metricsinfo.DataClusterTopology{ + Self: metricsinfo.DataCoordInfos{ + QuotaMetrics: &metricsinfo.DataCoordQuotaMetrics{ + TotalBinlogSize: 200, + CollectionBinlogSize: map[int64]int64{ + 1: 100, + 2: 50, + 3: 50, + }, + }, + }, + }, + }) + + datacoord.EXPECT().GetMetrics(mock.Anything, mock.Anything).Return(&milvuspb.GetMetricsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Response: string(r), + ComponentName: "DataCoord", + }, nil).Once() + mockCache.EXPECT().GetCollectionNamesByID(mock.Anything, mock.Anything).Return(nil, nil, errors.New("mock get collection names by id error")).Once() + + ctx := context.Background() + proxy := &Proxy{ + ctx: ctx, + dataCoord: datacoord, + } + err := proxy.reportCollectionStorage() + assert.Error(t, err) + }) + + t.Run("not match data", func(t *testing.T) { + origin := globalMetaCache + defer func() { + globalMetaCache = origin + }() + mockCache := NewMockCache(t) + globalMetaCache = mockCache + + datacoord := mocks.NewMockDataCoordClient(t) + r, _ := json.Marshal(&metricsinfo.DataCoordTopology{ + Cluster: metricsinfo.DataClusterTopology{ + Self: metricsinfo.DataCoordInfos{ + QuotaMetrics: &metricsinfo.DataCoordQuotaMetrics{ + TotalBinlogSize: 200, + CollectionBinlogSize: map[int64]int64{ + 1: 100, + 2: 50, + 3: 50, + }, + }, + }, + }, + }) + + datacoord.EXPECT().GetMetrics(mock.Anything, mock.Anything).Return(&milvuspb.GetMetricsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Response: string(r), + ComponentName: "DataCoord", + }, nil).Once() + mockCache.EXPECT().GetCollectionNamesByID(mock.Anything, mock.Anything).Return( + []string{"db1", "db1"}, []string{"col1", "col2"}, nil).Once() + + ctx := context.Background() + proxy := &Proxy{ + ctx: ctx, + dataCoord: datacoord, + } + err := proxy.reportCollectionStorage() + assert.Error(t, err) + }) + + t.Run("success", func(t *testing.T) { + origin := globalMetaCache + defer func() { + globalMetaCache = origin + }() + mockCache := NewMockCache(t) + globalMetaCache = mockCache + + datacoord := mocks.NewMockDataCoordClient(t) + r, _ := json.Marshal(&metricsinfo.DataCoordTopology{ + Cluster: metricsinfo.DataClusterTopology{ + Self: metricsinfo.DataCoordInfos{ + QuotaMetrics: &metricsinfo.DataCoordQuotaMetrics{ + TotalBinlogSize: 200, + CollectionBinlogSize: map[int64]int64{ + 1: 100, + 2: 50, + 3: 50, + }, + }, + }, + }, + }) + + datacoord.EXPECT().GetMetrics(mock.Anything, mock.Anything).Return(&milvuspb.GetMetricsResponse{ + Status: &commonpb.Status{ + ErrorCode: commonpb.ErrorCode_Success, + }, + Response: string(r), + ComponentName: "DataCoord", + }, nil).Once() + mockCache.EXPECT().GetCollectionNamesByID(mock.Anything, mock.Anything).Return( + []string{"db1", "db1", "db2"}, []string{"col1", "col2", "col3"}, nil).Once() + + originExtension := Extension + defer func() { + Extension = originExtension + }() + hasCheck := false + Extension = CheckExtension{ + reportChecker: func(info any) { + infoMap := info.(map[string]any) + storage := infoMap[hookutil.StorageDetailKey].(map[string]any) + log.Info("storage map", zap.Any("storage", storage)) + assert.EqualValues(t, 150, storage["db1"]) + assert.EqualValues(t, 50, storage["db2"]) + hasCheck = true + }, + } + + ctx := context.Background() + proxy := &Proxy{ + ctx: ctx, + dataCoord: datacoord, + } + err := proxy.reportCollectionStorage() + assert.NoError(t, err) + assert.True(t, hasCheck) + }) +} + +type CheckExtension struct { + reportChecker func(info any) +} + +func (c CheckExtension) Report(info any) int { + c.reportChecker(info) + return 0 +} diff --git a/internal/proxy/search_reduce_util.go b/internal/proxy/search_reduce_util.go index 44a59f795e..e0903bf5da 100644 --- a/internal/proxy/search_reduce_util.go +++ b/internal/proxy/search_reduce_util.go @@ -117,6 +117,7 @@ func reduceSearchResultDataWithGroupBy(ctx context.Context, subSearchResultData zap.Int64("topk", sData.TopK), zap.Int("length of pks", pkLength), zap.Int("length of FieldsData", len(sData.FieldsData))) + ret.Results.AllSearchCount += sData.GetAllSearchCount() if err := checkSearchResultData(sData, nq, topk); err != nil { log.Ctx(ctx).Warn("invalid search results", zap.Error(err)) return ret, err @@ -280,6 +281,7 @@ func reduceSearchResultDataNoGroupBy(ctx context.Context, subSearchResultData [] zap.Int64("topk", sData.TopK), zap.Int("length of pks", pkLength), zap.Int("length of FieldsData", len(sData.FieldsData))) + ret.Results.AllSearchCount += sData.GetAllSearchCount() if err := checkSearchResultData(sData, nq, topk); err != nil { log.Ctx(ctx).Warn("invalid search results", zap.Error(err)) return ret, err diff --git a/internal/proxy/task_delete.go b/internal/proxy/task_delete.go index a64758b485..145de2adce 100644 --- a/internal/proxy/task_delete.go +++ b/internal/proxy/task_delete.go @@ -61,7 +61,8 @@ type deleteTask struct { msgID UniqueID // result - count int64 + count int64 + allQueryCnt int64 } func (dt *deleteTask) TraceCtx() context.Context { @@ -246,6 +247,8 @@ type deleteRunner struct { // task queue queue *dmTaskQueue + + allQueryCnt atomic.Int64 } func (dr *deleteRunner) Init(ctx context.Context) error { @@ -422,6 +425,7 @@ func (dr *deleteRunner) getStreamingQueryAndDelteFunc(plan *planpb.PlanNode) exe taskCh := make(chan *deleteTask, 256) go dr.receiveQueryResult(ctx, client, taskCh) + var allQueryCnt int64 // wait all task finish for task := range taskCh { err := task.WaitToFinish() @@ -429,12 +433,14 @@ func (dr *deleteRunner) getStreamingQueryAndDelteFunc(plan *planpb.PlanNode) exe return err } dr.count.Add(task.count) + allQueryCnt += task.allQueryCnt } // query or produce task failed if dr.err != nil { return dr.err } + dr.allQueryCnt.Add(allQueryCnt) return nil } } @@ -468,6 +474,7 @@ func (dr *deleteRunner) receiveQueryResult(ctx context.Context, client querypb.Q log.Warn("produce delete task failed", zap.Error(err)) return } + task.allQueryCnt = result.GetAllRetrieveCount() taskCh <- task } diff --git a/internal/proxy/task_hybrid_search.go b/internal/proxy/task_hybrid_search.go index ede26d7075..5ceead077e 100644 --- a/internal/proxy/task_hybrid_search.go +++ b/internal/proxy/task_hybrid_search.go @@ -37,6 +37,7 @@ type hybridSearchTask struct { ctx context.Context *internalpb.HybridSearchRequest + dimension int64 result *milvuspb.SearchResults request *milvuspb.HybridSearchRequest searchTasks []*searchTask @@ -101,6 +102,11 @@ func (t *hybridSearchTask) PreExecute(ctx context.Context) error { log.Warn("get collection schema failed", zap.Error(err)) return err } + t.dimension, err = typeutil.GetCollectionDim(t.schema.CollectionSchema) + if err != nil { + log.Warn("get collection dimension failed", zap.Error(err)) + return err + } t.partitionKeyMode, err = isPartitionKeyMode(ctx, t.request.GetDbName(), collectionName) if err != nil { @@ -529,6 +535,7 @@ func rankSearchResultData(ctx context.Context, } for _, result := range searchResults { + ret.Results.AllSearchCount += result.GetResults().GetAllSearchCount() scores := result.GetResults().GetScores() start := int64(0) for i := int64(0); i < nq; i++ { diff --git a/internal/proxy/task_query.go b/internal/proxy/task_query.go index 8b95e7add0..9e26630898 100644 --- a/internal/proxy/task_query.go +++ b/internal/proxy/task_query.go @@ -54,6 +54,7 @@ type queryTask struct { collectionName string queryParams *queryParams schema *schemaInfo + dimension int64 userOutputFields []string @@ -65,7 +66,8 @@ type queryTask struct { channelsMvcc map[string]Timestamp fastSkip bool - reQuery bool + reQuery bool + allQueryCnt int64 } type queryParams struct { @@ -333,8 +335,17 @@ func (t *queryTask) PreExecute(ctx context.Context) error { t.queryParams = queryParams t.RetrieveRequest.Limit = queryParams.limit + queryParams.offset - schema, _ := globalMetaCache.GetCollectionSchema(ctx, t.request.GetDbName(), t.collectionName) + schema, err := globalMetaCache.GetCollectionSchema(ctx, t.request.GetDbName(), t.collectionName) + if err != nil { + log.Warn("get collection schema failed", zap.Error(err)) + return err + } t.schema = schema + t.dimension, err = typeutil.GetCollectionDim(t.schema.CollectionSchema) + if err != nil { + log.Warn("get collection dimension failed", zap.Error(err)) + return err + } if t.ids != nil { pkField := "" @@ -469,6 +480,7 @@ func (t *queryTask) PostExecute(ctx context.Context) error { var err error toReduceResults := make([]*internalpb.RetrieveResults, 0) + t.allQueryCnt = 0 select { case <-t.TraceCtx().Done(): log.Warn("proxy", zap.Int64("Query: wait to finish failed, timeout!, msgID:", t.ID())) @@ -477,6 +489,7 @@ func (t *queryTask) PostExecute(ctx context.Context) error { log.Debug("all queries are finished or canceled") t.resultBuf.Range(func(res *internalpb.RetrieveResults) bool { toReduceResults = append(toReduceResults, res) + t.allQueryCnt += res.GetAllRetrieveCount() log.Debug("proxy receives one query result", zap.Int64("sourceID", res.GetBase().GetSourceID())) return true }) diff --git a/internal/proxy/task_scheduler.go b/internal/proxy/task_scheduler.go index 6479262eb2..9d8b899f81 100644 --- a/internal/proxy/task_scheduler.go +++ b/internal/proxy/task_scheduler.go @@ -472,7 +472,6 @@ func (sched *taskScheduler) processTask(t task, q taskQueue) { span.AddEvent("scheduler process PostExecute") err = t.PostExecute(ctx) - if err != nil { span.RecordError(err) log.Ctx(ctx).Warn("Failed to post-execute task: ", zap.Error(err)) diff --git a/internal/proxy/task_search.go b/internal/proxy/task_search.go index b08a057943..a062d77f13 100644 --- a/internal/proxy/task_search.go +++ b/internal/proxy/task_search.go @@ -66,6 +66,7 @@ type searchTask struct { userOutputFields []string offset int64 + dimension int64 resultBuf *typeutil.ConcurrentSet[*internalpb.SearchResults] qc types.QueryCoordClient @@ -304,6 +305,11 @@ func (t *searchTask) PreExecute(ctx context.Context) error { log.Warn("get collection schema failed", zap.Error(err)) return err } + t.dimension, err = typeutil.GetCollectionDim(t.schema.CollectionSchema) + if err != nil { + log.Warn("get collection dimension failed", zap.Error(err)) + return err + } t.partitionKeyMode, err = isPartitionKeyMode(ctx, t.request.GetDbName(), collectionName) if err != nil { diff --git a/internal/proxy/util.go b/internal/proxy/util.go index 08ef4c0a0a..d27597d1ba 100644 --- a/internal/proxy/util.go +++ b/internal/proxy/util.go @@ -686,15 +686,15 @@ func autoGenPrimaryFieldData(fieldSchema *schemapb.FieldSchema, data interface{} }, } case schemapb.DataType_VarChar: - strIds := make([]string, len(data)) + strIDs := make([]string, len(data)) for i, v := range data { - strIds[i] = strconv.FormatInt(v, 10) + strIDs[i] = strconv.FormatInt(v, 10) } fieldData.Field = &schemapb.FieldData_Scalars{ Scalars: &schemapb.ScalarField{ Data: &schemapb.ScalarField_StringData{ StringData: &schemapb.StringArray{ - Data: strIds, + Data: strIDs, }, }, }, @@ -903,6 +903,11 @@ func GetCurUserFromContext(ctx context.Context) (string, error) { return username, nil } +func GetCurUserFromContextOrDefault(ctx context.Context) string { + username, _ := GetCurUserFromContext(ctx) + return username +} + func GetCurDBNameFromContextOrDefault(ctx context.Context) string { md, ok := metadata.FromIncomingContext(ctx) if !ok { @@ -1634,3 +1639,16 @@ func CheckDatabase(ctx context.Context, dbName string) bool { } return false } + +func SetReportValue(status *commonpb.Status, value int) { + if value <= 0 { + return + } + if !merr.Ok(status) { + return + } + if status.ExtraInfo == nil { + status.ExtraInfo = make(map[string]string) + } + status.ExtraInfo["report_value"] = strconv.Itoa(value) +} diff --git a/internal/querynodev2/segments/count_reducer.go b/internal/querynodev2/segments/count_reducer.go index 70a5f0dfb8..99134147b7 100644 --- a/internal/querynodev2/segments/count_reducer.go +++ b/internal/querynodev2/segments/count_reducer.go @@ -12,26 +12,34 @@ type cntReducer struct{} func (r *cntReducer) Reduce(ctx context.Context, results []*internalpb.RetrieveResults) (*internalpb.RetrieveResults, error) { cnt := int64(0) + allRetrieveCount := int64(0) for _, res := range results { + allRetrieveCount += res.GetAllRetrieveCount() c, err := funcutil.CntOfInternalResult(res) if err != nil { return nil, err } cnt += c } - return funcutil.WrapCntToInternalResult(cnt), nil + res := funcutil.WrapCntToInternalResult(cnt) + res.AllRetrieveCount = allRetrieveCount + return res, nil } type cntReducerSegCore struct{} func (r *cntReducerSegCore) Reduce(ctx context.Context, results []*segcorepb.RetrieveResults) (*segcorepb.RetrieveResults, error) { cnt := int64(0) + allRetrieveCount := int64(0) for _, res := range results { + allRetrieveCount += res.GetAllRetrieveCount() c, err := funcutil.CntOfSegCoreResult(res) if err != nil { return nil, err } cnt += c } - return funcutil.WrapCntToSegCoreResult(cnt), nil + res := funcutil.WrapCntToSegCoreResult(cnt) + res.AllRetrieveCount = allRetrieveCount + return res, nil } diff --git a/internal/querynodev2/segments/result.go b/internal/querynodev2/segments/result.go index 54e274e4e5..56cb1c9b83 100644 --- a/internal/querynodev2/segments/result.go +++ b/internal/querynodev2/segments/result.go @@ -130,6 +130,7 @@ func ReduceSearchResultData(ctx context.Context, searchResultData []*schemapb.Se for j := int64(1); j < nq; j++ { resultOffsets[i][j] = resultOffsets[i][j-1] + searchResultData[i].Topks[j-1] } + ret.AllSearchCount += searchResultData[i].GetAllSearchCount() } var skipDupCnt int64 @@ -284,6 +285,7 @@ func MergeInternalRetrieveResult(ctx context.Context, retrieveResults []*interna validRetrieveResults := []*internalpb.RetrieveResults{} for _, r := range retrieveResults { + ret.AllRetrieveCount += r.GetAllRetrieveCount() size := typeutil.GetSizeOfIDs(r.GetIds()) if r == nil || len(r.GetFieldsData()) == 0 || size == 0 { continue @@ -388,6 +390,7 @@ func MergeSegcoreRetrieveResults(ctx context.Context, retrieveResults []*segcore validRetrieveResults := []*segcorepb.RetrieveResults{} for _, r := range retrieveResults { size := typeutil.GetSizeOfIDs(r.GetIds()) + ret.AllRetrieveCount += r.GetAllRetrieveCount() if r == nil || len(r.GetOffset()) == 0 || size == 0 { log.Debug("filter out invalid retrieve result") continue diff --git a/internal/querynodev2/segments/retrieve.go b/internal/querynodev2/segments/retrieve.go index aea684fcfa..b4575debb6 100644 --- a/internal/querynodev2/segments/retrieve.go +++ b/internal/querynodev2/segments/retrieve.go @@ -118,9 +118,10 @@ func retrieveOnSegmentsWithStream(ctx context.Context, segments []Segment, segTy if len(result.GetOffset()) != 0 { if err = svr.Send(&internalpb.RetrieveResults{ - Status: merr.Success(), - Ids: result.GetIds(), - FieldsData: result.GetFieldsData(), + Status: merr.Success(), + Ids: result.GetIds(), + FieldsData: result.GetFieldsData(), + AllRetrieveCount: result.GetAllRetrieveCount(), }); err != nil { errs[i] = err } diff --git a/internal/querynodev2/segments/segment.go b/internal/querynodev2/segments/segment.go index 826ba48421..3db9cf0a8a 100644 --- a/internal/querynodev2/segments/segment.go +++ b/internal/querynodev2/segments/segment.go @@ -626,7 +626,7 @@ func (s *LocalSegment) Insert(ctx context.Context, rowIDs []int64, timestamps [] numOfRow := len(rowIDs) cOffset := C.int64_t(offset) cNumOfRows := C.int64_t(numOfRow) - cEntityIdsPtr := (*C.int64_t)(&(rowIDs)[0]) + cEntityIDsPtr := (*C.int64_t)(&(rowIDs)[0]) cTimestampsPtr := (*C.uint64_t)(&(timestamps)[0]) var status C.CStatus @@ -635,7 +635,7 @@ func (s *LocalSegment) Insert(ctx context.Context, rowIDs []int64, timestamps [] status = C.Insert(s.ptr, cOffset, cNumOfRows, - cEntityIdsPtr, + cEntityIDsPtr, cTimestampsPtr, (*C.uint8_t)(unsafe.Pointer(&insertRecordBlob[0])), (C.uint64_t)(len(insertRecordBlob)), diff --git a/internal/querynodev2/tasks/query_task.go b/internal/querynodev2/tasks/query_task.go index b495c2b1a8..3645705392 100644 --- a/internal/querynodev2/tasks/query_task.go +++ b/internal/querynodev2/tasks/query_task.go @@ -135,6 +135,7 @@ func (t *QueryTask) Execute() error { CostAggregation: &internalpb.CostAggregation{ ServiceTime: tr.ElapseSpan().Milliseconds(), }, + AllRetrieveCount: reducedResult.GetAllRetrieveCount(), } return nil } diff --git a/internal/rootcoord/constrant.go b/internal/rootcoord/constrant.go index 0aed96c8c5..b54b5d8bec 100644 --- a/internal/rootcoord/constrant.go +++ b/internal/rootcoord/constrant.go @@ -48,10 +48,10 @@ func checkGeneralCapacity(ctx context.Context, newColNum int, var generalNum int64 = 0 collectionsMap := core.meta.ListAllAvailCollections(ctx) - for dbId, collectionIds := range collectionsMap { + for dbId, collectionIDs := range collectionsMap { db, err := core.meta.GetDatabaseByID(ctx, dbId, ts) if err == nil { - for _, collectionId := range collectionIds { + for _, collectionId := range collectionIDs { collection, err := core.meta.GetCollectionByID(ctx, db.Name, collectionId, ts, true) if err == nil { partNum := int64(collection.GetPartitionNum(false)) diff --git a/internal/storage/utils.go b/internal/storage/utils.go index 631da3bfdd..991dae414c 100644 --- a/internal/storage/utils.go +++ b/internal/storage/utils.go @@ -164,7 +164,7 @@ func TransferColumnBasedInsertDataToRowBased(data *InsertData) ( } tss := data.Data[common.TimeStampField].(*Int64FieldData) - rowIds := data.Data[common.RowIDField].(*Int64FieldData) + rowIDs := data.Data[common.RowIDField].(*Int64FieldData) ls := fieldDataList{} for fieldID := range data.Data { @@ -176,8 +176,8 @@ func TransferColumnBasedInsertDataToRowBased(data *InsertData) ( ls.datas = append(ls.datas, data.Data[fieldID]) } - // checkNumRows(tss, rowIds, ls.datas...) // don't work - all := []FieldData{tss, rowIds} + // checkNumRows(tss, rowIDs, ls.datas...) // don't work + all := []FieldData{tss, rowIDs} all = append(all, ls.datas...) if !checkNumRows(all...) { return nil, nil, nil, @@ -210,7 +210,7 @@ func TransferColumnBasedInsertDataToRowBased(data *InsertData) ( utss[i] = uint64(tss.Data[i]) } - return utss, rowIds.Data, rows, nil + return utss, rowIDs.Data, rows, nil } /////////////////////////////////////////////////////////////////////////////////////////// diff --git a/internal/storage/utils_test.go b/internal/storage/utils_test.go index f845310c51..a08c15bdc2 100644 --- a/internal/storage/utils_test.go +++ b/internal/storage/utils_test.go @@ -125,10 +125,10 @@ func TestTransferColumnBasedInsertDataToRowBased(t *testing.T) { _, _, _, err = TransferColumnBasedInsertDataToRowBased(data) assert.Error(t, err) - rowIdsF := &Int64FieldData{ + rowIDsF := &Int64FieldData{ Data: []int64{1, 2, 3, 4}, } - data.Data[common.RowIDField] = rowIdsF + data.Data[common.RowIDField] = rowIDsF // row num mismatch _, _, _, err = TransferColumnBasedInsertDataToRowBased(data) @@ -193,10 +193,10 @@ func TestTransferColumnBasedInsertDataToRowBased(t *testing.T) { data.Data[111] = f11 data.Data[112] = f12 - utss, rowIds, rows, err := TransferColumnBasedInsertDataToRowBased(data) + utss, rowIDs, rows, err := TransferColumnBasedInsertDataToRowBased(data) assert.NoError(t, err) assert.ElementsMatch(t, []uint64{1, 2, 3}, utss) - assert.ElementsMatch(t, []int64{1, 2, 3}, rowIds) + assert.ElementsMatch(t, []int64{1, 2, 3}, rowIDs) assert.Equal(t, 3, len(rows)) // b := []byte("1")[0] if common.Endian == binary.LittleEndian { diff --git a/internal/util/hookutil/constant.go b/internal/util/hookutil/constant.go new file mode 100644 index 0000000000..4e9edb05ee --- /dev/null +++ b/internal/util/hookutil/constant.go @@ -0,0 +1,43 @@ +/* + * Licensed to the LF AI & Data foundation under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package hookutil + +var ( + // WARN: Please DO NOT modify all constants. + + OpTypeKey = "op_type" + DatabaseKey = "database" + UsernameKey = "username" + DataSizeKey = "data_size" + SuccessCntKey = "success_cnt" + FailCntKey = "fail_cnt" + RelatedCntKey = "related_cnt" + StorageDetailKey = "storage_detail" + NodeIDKey = "id" + DimensionKey = "dim" + + OpTypeInsert = "insert" + OpTypeDelete = "delete" + OpTypeUpsert = "upsert" + OpTypeQuery = "query" + OpTypeSearch = "search" + OpTypeHybridSearch = "hybrid_search" + OpTypeStorage = "storage" + OpTypeNodeID = "node_id" +) diff --git a/internal/util/hookutil/default.go b/internal/util/hookutil/default.go new file mode 100644 index 0000000000..6083e9d450 --- /dev/null +++ b/internal/util/hookutil/default.go @@ -0,0 +1,72 @@ +/* + * Licensed to the LF AI & Data foundation under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package hookutil + +import ( + "context" + + "github.com/cockroachdb/errors" + + "github.com/milvus-io/milvus-proto/go-api/v2/hook" +) + +type DefaultHook struct{} + +var _ hook.Hook = (*DefaultHook)(nil) + +func (d DefaultHook) VerifyAPIKey(key string) (string, error) { + return "", errors.New("default hook, can't verify api key") +} + +func (d DefaultHook) Init(params map[string]string) error { + return nil +} + +func (d DefaultHook) Mock(ctx context.Context, req interface{}, fullMethod string) (bool, interface{}, error) { + return false, nil, nil +} + +func (d DefaultHook) Before(ctx context.Context, req interface{}, fullMethod string) (context.Context, error) { + return ctx, nil +} + +func (d DefaultHook) After(ctx context.Context, result interface{}, err error, fullMethod string) error { + return nil +} + +// MockAPIHook is a mock hook for api key verification, ONLY FOR TEST +type MockAPIHook struct { + DefaultHook + MockErr error + User string +} + +func (m MockAPIHook) VerifyAPIKey(apiKey string) (string, error) { + return m.User, m.MockErr +} + +func (d DefaultHook) Release() {} + +type DefaultExtension struct{} + +var _ hook.Extension = (*DefaultExtension)(nil) + +func (d DefaultExtension) Report(info any) int { + return 0 +} diff --git a/internal/util/hookutil/hook.go b/internal/util/hookutil/hook.go new file mode 100644 index 0000000000..2f38e0a60b --- /dev/null +++ b/internal/util/hookutil/hook.go @@ -0,0 +1,102 @@ +/* + * Licensed to the LF AI & Data foundation under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package hookutil + +import ( + "fmt" + "plugin" + "sync" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus-proto/go-api/v2/hook" + "github.com/milvus-io/milvus/pkg/config" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/paramtable" +) + +var ( + Hoo hook.Hook + Extension hook.Extension + initOnce sync.Once +) + +func initHook() error { + Hoo = DefaultHook{} + Extension = DefaultExtension{} + + path := paramtable.Get().ProxyCfg.SoPath.GetValue() + if path == "" { + log.Info("empty so path, skip to load plugin") + return nil + } + + log.Info("start to load plugin", zap.String("path", path)) + p, err := plugin.Open(path) + if err != nil { + return fmt.Errorf("fail to open the plugin, error: %s", err.Error()) + } + log.Info("plugin open") + + h, err := p.Lookup("MilvusHook") + if err != nil { + return fmt.Errorf("fail to the 'MilvusHook' object in the plugin, error: %s", err.Error()) + } + + var ok bool + Hoo, ok = h.(hook.Hook) + if !ok { + return fmt.Errorf("fail to convert the `Hook` interface") + } + if err = Hoo.Init(paramtable.GetHookParams().SoConfig.GetValue()); err != nil { + return fmt.Errorf("fail to init configs for the hook, error: %s", err.Error()) + } + paramtable.GetHookParams().WatchHookWithPrefix("watch_hook", "", func(event *config.Event) { + log.Info("receive the hook refresh event", zap.Any("event", event)) + go func() { + soConfig := paramtable.GetHookParams().SoConfig.GetValue() + log.Info("refresh hook configs", zap.Any("config", soConfig)) + if err = Hoo.Init(soConfig); err != nil { + log.Panic("fail to init configs for the hook when refreshing", zap.Error(err)) + } + }() + }) + + e, err := p.Lookup("MilvusExtension") + if err != nil { + return fmt.Errorf("fail to the 'MilvusExtension' object in the plugin, error: %s", err.Error()) + } + Extension, ok = e.(hook.Extension) + if !ok { + return fmt.Errorf("fail to convert the `Extension` interface") + } + + return nil +} + +func InitOnceHook() { + initOnce.Do(func() { + err := initHook() + if err != nil { + log.Warn("fail to init hook", + zap.String("so_path", paramtable.Get().ProxyCfg.SoPath.GetValue()), + zap.Error(err)) + } + }) +} diff --git a/internal/util/hookutil/hook_test.go b/internal/util/hookutil/hook_test.go new file mode 100644 index 0000000000..0a2be25c82 --- /dev/null +++ b/internal/util/hookutil/hook_test.go @@ -0,0 +1,52 @@ +/* + * Licensed to the LF AI & Data foundation under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package hookutil + +import ( + "testing" + + "github.com/stretchr/testify/assert" + + "github.com/milvus-io/milvus/pkg/util/paramtable" +) + +func TestInitHook(t *testing.T) { + paramtable.Init() + Params := paramtable.Get() + paramtable.Get().Save(Params.ProxyCfg.SoPath.Key, "") + initHook() + assert.IsType(t, DefaultHook{}, Hoo) + + paramtable.Get().Save(Params.ProxyCfg.SoPath.Key, "/a/b/hook.so") + err := initHook() + assert.Error(t, err) + paramtable.Get().Save(Params.ProxyCfg.SoPath.Key, "") +} + +func TestDefaultHook(t *testing.T) { + d := &DefaultHook{} + assert.NoError(t, d.Init(nil)) + { + _, err := d.VerifyAPIKey("key") + assert.Error(t, err) + } + assert.NotPanics(t, func() { + d.Release() + }) +} diff --git a/internal/util/typeutil/result_helper.go b/internal/util/typeutil/result_helper.go index 56ce231f16..671dabb491 100644 --- a/internal/util/typeutil/result_helper.go +++ b/internal/util/typeutil/result_helper.go @@ -13,7 +13,7 @@ func appendFieldData(result RetrieveResults, fieldData *schemapb.FieldData) { result.AppendFieldData(fieldData) } -func FillRetrieveResultIfEmpty(result RetrieveResults, outputFieldIds []int64, schema *schemapb.CollectionSchema) error { +func FillRetrieveResultIfEmpty(result RetrieveResults, outputFieldIDs []int64, schema *schemapb.CollectionSchema) error { if !result.ResultEmpty() { return nil } @@ -24,7 +24,7 @@ func FillRetrieveResultIfEmpty(result RetrieveResults, outputFieldIds []int64, s if err != nil { return err } - for _, outputFieldID := range outputFieldIds { + for _, outputFieldID := range outputFieldIDs { field, err := helper.GetFieldFromID(outputFieldID) if err != nil { return err diff --git a/pkg/go.mod b/pkg/go.mod index 1f06ee7131..3525ac507c 100644 --- a/pkg/go.mod +++ b/pkg/go.mod @@ -14,7 +14,7 @@ require ( github.com/grpc-ecosystem/go-grpc-middleware v1.3.0 github.com/klauspost/compress v1.16.5 github.com/lingdor/stackerror v0.0.0-20191119040541-976d8885ed76 - github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240228061649-a922b16f2a46 + github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240313071055-2c89f346b00f github.com/nats-io/nats-server/v2 v2.9.17 github.com/nats-io/nats.go v1.24.0 github.com/panjf2000/ants/v2 v2.7.2 diff --git a/pkg/go.sum b/pkg/go.sum index 077f3ecff6..81a27741f4 100644 --- a/pkg/go.sum +++ b/pkg/go.sum @@ -483,8 +483,8 @@ github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfr github.com/mediocregopher/radix/v3 v3.4.2/go.mod h1:8FL3F6UQRXHXIBSPUs5h0RybMF8i4n7wVopoX3x7Bv8= github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= -github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240228061649-a922b16f2a46 h1:IgoGNTbsRPa2kdNI+IWuZrrortFEjTB42/gYDklZHVU= -github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240228061649-a922b16f2a46/go.mod h1:1OIl0v5PQeNxIJhCvY+K55CBUOYDZevw9g9380u1Wek= +github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240313071055-2c89f346b00f h1:f8rRJ5zatNq2WszAwy6S+J0Z2h7/CArqLDJ0gTHSFNs= +github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240313071055-2c89f346b00f/go.mod h1:1OIl0v5PQeNxIJhCvY+K55CBUOYDZevw9g9380u1Wek= github.com/milvus-io/pulsar-client-go v0.6.10 h1:eqpJjU+/QX0iIhEo3nhOqMNXL+TyInAs1IAHZCrCM/A= github.com/milvus-io/pulsar-client-go v0.6.10/go.mod h1:lQqCkgwDF8YFYjKA+zOheTk1tev2B+bKj5j7+nm8M1w= github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA/g= diff --git a/pkg/util/metricsinfo/metric_type.go b/pkg/util/metricsinfo/metric_type.go index 60e0503152..0e140e0521 100644 --- a/pkg/util/metricsinfo/metric_type.go +++ b/pkg/util/metricsinfo/metric_type.go @@ -27,6 +27,9 @@ const ( // SystemInfoMetrics means users request for system information metrics. SystemInfoMetrics = "system_info" + + // CollectionStorageMetrics means users request for collection storage metrics. + CollectionStorageMetrics = "collection_storage" ) // ParseMetricType returns the metric type of req diff --git a/pkg/util/typeutil/get_dim.go b/pkg/util/typeutil/get_dim.go index 8d0b8086bd..668a5dbdba 100644 --- a/pkg/util/typeutil/get_dim.go +++ b/pkg/util/typeutil/get_dim.go @@ -27,3 +27,14 @@ func GetDim(field *schemapb.FieldSchema) (int64, error) { } return int64(dim), nil } + +func GetCollectionDim(collection *schemapb.CollectionSchema) (int64, error) { + for _, fieldSchema := range collection.GetFields() { + dim, err := GetDim(fieldSchema) + if err != nil { + continue + } + return dim, nil + } + return 0, fmt.Errorf("dim not found") +} diff --git a/tests/integration/hellomilvus/hello_milvus_test.go b/tests/integration/hellomilvus/hello_milvus_test.go index 3ac192d02a..dff1475dd1 100644 --- a/tests/integration/hellomilvus/hello_milvus_test.go +++ b/tests/integration/hellomilvus/hello_milvus_test.go @@ -20,6 +20,7 @@ import ( "context" "fmt" "testing" + "time" "github.com/golang/protobuf/proto" "github.com/stretchr/testify/suite" @@ -28,6 +29,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/funcutil" @@ -77,6 +79,27 @@ func (s *HelloMilvusSuite) TestHelloMilvus() { fVecColumn := integration.NewFloatVectorFieldData(integration.FloatVecField, rowNum, dim) hashKeys := integration.GenerateHashKeys(rowNum) + insertCheckReport := func() { + timeoutCtx, cancelFunc := context.WithTimeout(ctx, 5*time.Second) + defer cancelFunc() + + for { + select { + case <-timeoutCtx.Done(): + s.Fail("insert check timeout") + case report := <-c.Extension.GetReportChan(): + reportInfo := report.(map[string]any) + log.Info("insert report info", zap.Any("reportInfo", reportInfo)) + if reportInfo[hookutil.OpTypeKey] == hookutil.OpTypeStorage { + continue + } + s.Equal(hookutil.OpTypeInsert, reportInfo[hookutil.OpTypeKey]) + s.NotEqualValues(0, reportInfo[hookutil.DataSizeKey]) + return + } + } + } + go insertCheckReport() insertResult, err := c.Proxy.Insert(ctx, &milvuspb.InsertRequest{ DbName: dbName, CollectionName: collectionName, @@ -145,11 +168,99 @@ func (s *HelloMilvusSuite) TestHelloMilvus() { searchReq := integration.ConstructSearchRequest("", collectionName, expr, integration.FloatVecField, schemapb.DataType_FloatVector, nil, metric.L2, params, nq, dim, topk, roundDecimal) - searchResult, err := c.Proxy.Search(ctx, searchReq) + searchCheckReport := func() { + timeoutCtx, cancelFunc := context.WithTimeout(ctx, 5*time.Second) + defer cancelFunc() + for { + select { + case <-timeoutCtx.Done(): + s.Fail("search check timeout") + case report := <-c.Extension.GetReportChan(): + reportInfo := report.(map[string]any) + log.Info("search report info", zap.Any("reportInfo", reportInfo)) + if reportInfo[hookutil.OpTypeKey] == hookutil.OpTypeStorage { + continue + } + s.Equal(hookutil.OpTypeSearch, reportInfo[hookutil.OpTypeKey]) + s.NotEqualValues(0, reportInfo[hookutil.DataSizeKey]) + s.EqualValues(rowNum, reportInfo[hookutil.RelatedCntKey]) + return + } + } + } + go searchCheckReport() + searchResult, err := c.Proxy.Search(ctx, searchReq) err = merr.CheckRPCCall(searchResult, err) s.NoError(err) + queryCheckReport := func() { + timeoutCtx, cancelFunc := context.WithTimeout(ctx, 5*time.Second) + defer cancelFunc() + + for { + select { + case <-timeoutCtx.Done(): + s.Fail("query check timeout") + case report := <-c.Extension.GetReportChan(): + reportInfo := report.(map[string]any) + log.Info("query report info", zap.Any("reportInfo", reportInfo)) + if reportInfo[hookutil.OpTypeKey] == hookutil.OpTypeStorage { + continue + } + s.Equal(hookutil.OpTypeQuery, reportInfo[hookutil.OpTypeKey]) + s.NotEqualValues(0, reportInfo[hookutil.DataSizeKey]) + s.EqualValues(rowNum, reportInfo[hookutil.RelatedCntKey]) + return + } + } + } + go queryCheckReport() + queryResult, err := c.Proxy.Query(ctx, &milvuspb.QueryRequest{ + DbName: dbName, + CollectionName: collectionName, + Expr: "", + OutputFields: []string{"count(*)"}, + }) + if queryResult.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success { + log.Warn("searchResult fail reason", zap.String("reason", queryResult.GetStatus().GetReason())) + } + s.NoError(err) + s.Equal(commonpb.ErrorCode_Success, queryResult.GetStatus().GetErrorCode()) + + deleteCheckReport := func() { + timeoutCtx, cancelFunc := context.WithTimeout(ctx, 5*time.Second) + defer cancelFunc() + + for { + select { + case <-timeoutCtx.Done(): + s.Fail("delete check timeout") + case report := <-c.Extension.GetReportChan(): + reportInfo := report.(map[string]any) + log.Info("delete report info", zap.Any("reportInfo", reportInfo)) + if reportInfo[hookutil.OpTypeKey] == hookutil.OpTypeStorage { + continue + } + s.Equal(hookutil.OpTypeDelete, reportInfo[hookutil.OpTypeKey]) + s.EqualValues(2, reportInfo[hookutil.SuccessCntKey]) + s.EqualValues(0, reportInfo[hookutil.RelatedCntKey]) + return + } + } + } + go deleteCheckReport() + deleteResult, err := c.Proxy.Delete(ctx, &milvuspb.DeleteRequest{ + DbName: dbName, + CollectionName: collectionName, + Expr: integration.Int64Field + " in [1, 2]", + }) + if deleteResult.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success { + log.Warn("deleteResult fail reason", zap.String("reason", deleteResult.GetStatus().GetReason())) + } + s.NoError(err) + s.Equal(commonpb.ErrorCode_Success, deleteResult.GetStatus().GetErrorCode()) + status, err := c.Proxy.ReleaseCollection(ctx, &milvuspb.ReleaseCollectionRequest{ CollectionName: collectionName, }) diff --git a/tests/integration/minicluster_v2.go b/tests/integration/minicluster_v2.go index 71386834fd..da9fa588a0 100644 --- a/tests/integration/minicluster_v2.go +++ b/tests/integration/minicluster_v2.go @@ -47,6 +47,7 @@ import ( "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/types" "github.com/milvus-io/milvus/internal/util/dependency" + "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -125,6 +126,8 @@ type MiniClusterV2 struct { qnid atomic.Int64 datanodes []*grpcdatanode.Server dnid atomic.Int64 + + Extension *ReportChanExtension } type OptionV2 func(cluster *MiniClusterV2) @@ -136,6 +139,8 @@ func StartMiniClusterV2(ctx context.Context, opts ...OptionV2) (*MiniClusterV2, dnid: *atomic.NewInt64(20000), } paramtable.Init() + cluster.Extension = InitReportExtension() + cluster.params = DefaultParams() cluster.clusterConfig = DefaultClusterConfig() for _, opt := range opts { @@ -445,3 +450,32 @@ func (cluster *MiniClusterV2) GetAvailablePort() (int, error) { defer listener.Close() return listener.Addr().(*net.TCPAddr).Port, nil } + +func InitReportExtension() *ReportChanExtension { + e := NewReportChanExtension() + hookutil.InitOnceHook() + hookutil.Extension = e + return e +} + +type ReportChanExtension struct { + reportChan chan any +} + +func NewReportChanExtension() *ReportChanExtension { + return &ReportChanExtension{ + reportChan: make(chan any), + } +} + +func (r *ReportChanExtension) Report(info any) int { + select { + case r.reportChan <- info: + default: + } + return 1 +} + +func (r *ReportChanExtension) GetReportChan() <-chan any { + return r.reportChan +} diff --git a/tests/integration/partitionkey/partition_key_test.go b/tests/integration/partitionkey/partition_key_test.go new file mode 100644 index 0000000000..afd2de0c23 --- /dev/null +++ b/tests/integration/partitionkey/partition_key_test.go @@ -0,0 +1,411 @@ +/* + * Licensed to the LF AI & Data foundation under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package partitionkey + +import ( + "context" + "fmt" + "testing" + "time" + + "github.com/golang/protobuf/proto" + "github.com/stretchr/testify/suite" + "go.uber.org/zap" + + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/util/hookutil" + "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/funcutil" + "github.com/milvus-io/milvus/pkg/util/metric" + "github.com/milvus-io/milvus/tests/integration" +) + +type PartitionKeySuite struct { + integration.MiniClusterSuite +} + +func (s *PartitionKeySuite) TestPartitionKey() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + c := s.Cluster + + const ( + dim = 128 + dbName = "" + rowNum = 1000 + ) + + collectionName := "TestPartitionKey" + funcutil.GenRandomStr() + schema := integration.ConstructSchema(collectionName, dim, false) + schema.Fields = append(schema.Fields, &schemapb.FieldSchema{ + FieldID: 102, + Name: "pid", + Description: "", + DataType: schemapb.DataType_Int64, + TypeParams: nil, + IndexParams: nil, + IsPartitionKey: true, + }) + marshaledSchema, err := proto.Marshal(schema) + s.NoError(err) + + createCollectionStatus, err := c.Proxy.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{ + DbName: dbName, + CollectionName: collectionName, + Schema: marshaledSchema, + ShardsNum: common.DefaultShardsNum, + }) + s.NoError(err) + if createCollectionStatus.GetErrorCode() != commonpb.ErrorCode_Success { + log.Warn("createCollectionStatus fail reason", zap.String("reason", createCollectionStatus.GetReason())) + } + s.Equal(createCollectionStatus.GetErrorCode(), commonpb.ErrorCode_Success) + + { + pkColumn := integration.NewInt64FieldDataWithStart(integration.Int64Field, rowNum, 0) + fVecColumn := integration.NewFloatVectorFieldData(integration.FloatVecField, rowNum, dim) + partitionKeyColumn := integration.NewInt64SameFieldData("pid", rowNum, 1) + hashKeys := integration.GenerateHashKeys(rowNum) + insertResult, err := c.Proxy.Insert(ctx, &milvuspb.InsertRequest{ + DbName: dbName, + CollectionName: collectionName, + FieldsData: []*schemapb.FieldData{pkColumn, fVecColumn, partitionKeyColumn}, + HashKeys: hashKeys, + NumRows: uint32(rowNum), + }) + s.NoError(err) + s.Equal(insertResult.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success) + } + + { + pkColumn := integration.NewInt64FieldDataWithStart(integration.Int64Field, rowNum, rowNum) + fVecColumn := integration.NewFloatVectorFieldData(integration.FloatVecField, rowNum, dim) + partitionKeyColumn := integration.NewInt64SameFieldData("pid", rowNum, 2) + hashKeys := integration.GenerateHashKeys(rowNum) + insertResult, err := c.Proxy.Insert(ctx, &milvuspb.InsertRequest{ + DbName: dbName, + CollectionName: collectionName, + FieldsData: []*schemapb.FieldData{pkColumn, fVecColumn, partitionKeyColumn}, + HashKeys: hashKeys, + NumRows: uint32(rowNum), + }) + s.NoError(err) + s.Equal(insertResult.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success) + } + + { + pkColumn := integration.NewInt64FieldDataWithStart(integration.Int64Field, rowNum, rowNum*2) + fVecColumn := integration.NewFloatVectorFieldData(integration.FloatVecField, rowNum, dim) + partitionKeyColumn := integration.NewInt64SameFieldData("pid", rowNum, 3) + hashKeys := integration.GenerateHashKeys(rowNum) + insertResult, err := c.Proxy.Insert(ctx, &milvuspb.InsertRequest{ + DbName: dbName, + CollectionName: collectionName, + FieldsData: []*schemapb.FieldData{pkColumn, fVecColumn, partitionKeyColumn}, + HashKeys: hashKeys, + NumRows: uint32(rowNum), + }) + s.NoError(err) + s.Equal(insertResult.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success) + } + + flushResp, err := c.Proxy.Flush(ctx, &milvuspb.FlushRequest{ + DbName: dbName, + CollectionNames: []string{collectionName}, + }) + s.NoError(err) + segmentIDs, has := flushResp.GetCollSegIDs()[collectionName] + ids := segmentIDs.GetData() + s.Require().NotEmpty(segmentIDs) + s.Require().True(has) + flushTs, has := flushResp.GetCollFlushTs()[collectionName] + s.True(has) + + segments, err := c.MetaWatcher.ShowSegments() + s.NoError(err) + s.NotEmpty(segments) + for _, segment := range segments { + log.Info("ShowSegments result", zap.String("segment", segment.String())) + } + s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName) + + // create index + createIndexStatus, err := c.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{ + CollectionName: collectionName, + FieldName: integration.FloatVecField, + IndexName: "_default", + ExtraParams: integration.ConstructIndexParam(dim, integration.IndexFaissIvfFlat, metric.L2), + }) + if createIndexStatus.GetErrorCode() != commonpb.ErrorCode_Success { + log.Warn("createIndexStatus fail reason", zap.String("reason", createIndexStatus.GetReason())) + } + s.NoError(err) + s.Equal(commonpb.ErrorCode_Success, createIndexStatus.GetErrorCode()) + + s.WaitForIndexBuilt(ctx, collectionName, integration.FloatVecField) + + // load + loadStatus, err := c.Proxy.LoadCollection(ctx, &milvuspb.LoadCollectionRequest{ + DbName: dbName, + CollectionName: collectionName, + }) + s.NoError(err) + if loadStatus.GetErrorCode() != commonpb.ErrorCode_Success { + log.Warn("loadStatus fail reason", zap.String("reason", loadStatus.GetReason())) + } + s.Equal(commonpb.ErrorCode_Success, loadStatus.GetErrorCode()) + s.WaitForLoad(ctx, collectionName) + + { + // search without partition key + expr := fmt.Sprintf("%s > 0", integration.Int64Field) + nq := 10 + topk := 10 + roundDecimal := -1 + + params := integration.GetSearchParams(integration.IndexFaissIvfFlat, metric.L2) + searchReq := integration.ConstructSearchRequest("", collectionName, expr, + integration.FloatVecField, schemapb.DataType_FloatVector, nil, metric.L2, params, nq, dim, topk, roundDecimal) + + searchCheckReport := func() { + timeoutCtx, cancelFunc := context.WithTimeout(ctx, 5*time.Second) + defer cancelFunc() + + for { + select { + case <-timeoutCtx.Done(): + s.Fail("search check timeout") + case report := <-c.Extension.GetReportChan(): + reportInfo := report.(map[string]any) + log.Info("search report info", zap.Any("reportInfo", reportInfo)) + if reportInfo[hookutil.OpTypeKey] == hookutil.OpTypeStorage { + continue + } + s.Equal(hookutil.OpTypeSearch, reportInfo[hookutil.OpTypeKey]) + s.NotEqualValues(0, reportInfo[hookutil.DataSizeKey]) + s.EqualValues(rowNum*3, reportInfo[hookutil.RelatedCntKey]) + return + } + } + } + go searchCheckReport() + searchResult, err := c.Proxy.Search(ctx, searchReq) + + if searchResult.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success { + log.Warn("searchResult fail reason", zap.String("reason", searchResult.GetStatus().GetReason())) + } + s.NoError(err) + s.Equal(commonpb.ErrorCode_Success, searchResult.GetStatus().GetErrorCode()) + } + + { + // search without partition key + expr := fmt.Sprintf("%s > 0 && pid == 1", integration.Int64Field) + nq := 10 + topk := 10 + roundDecimal := -1 + + params := integration.GetSearchParams(integration.IndexFaissIvfFlat, metric.L2) + searchReq := integration.ConstructSearchRequest("", collectionName, expr, + integration.FloatVecField, schemapb.DataType_FloatVector, nil, metric.L2, params, nq, dim, topk, roundDecimal) + + searchCheckReport := func() { + timeoutCtx, cancelFunc := context.WithTimeout(ctx, 5*time.Second) + defer cancelFunc() + + for { + select { + case <-timeoutCtx.Done(): + s.Fail("search check timeout") + case report := <-c.Extension.GetReportChan(): + reportInfo := report.(map[string]any) + log.Info("search report info", zap.Any("reportInfo", reportInfo)) + if reportInfo[hookutil.OpTypeKey] == hookutil.OpTypeStorage { + continue + } + s.Equal(hookutil.OpTypeSearch, reportInfo[hookutil.OpTypeKey]) + s.NotEqualValues(0, reportInfo[hookutil.DataSizeKey]) + s.EqualValues(rowNum, reportInfo[hookutil.RelatedCntKey]) + return + } + } + } + go searchCheckReport() + searchResult, err := c.Proxy.Search(ctx, searchReq) + + if searchResult.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success { + log.Warn("searchResult fail reason", zap.String("reason", searchResult.GetStatus().GetReason())) + } + s.NoError(err) + s.Equal(commonpb.ErrorCode_Success, searchResult.GetStatus().GetErrorCode()) + } + + { + // query without partition key + queryCheckReport := func() { + timeoutCtx, cancelFunc := context.WithTimeout(ctx, 5*time.Second) + defer cancelFunc() + + for { + select { + case <-timeoutCtx.Done(): + s.Fail("query check timeout") + case report := <-c.Extension.GetReportChan(): + reportInfo := report.(map[string]any) + log.Info("query report info", zap.Any("reportInfo", reportInfo)) + if reportInfo[hookutil.OpTypeKey] == hookutil.OpTypeStorage { + continue + } + s.Equal(hookutil.OpTypeQuery, reportInfo[hookutil.OpTypeKey]) + s.NotEqualValues(0, reportInfo[hookutil.DataSizeKey]) + s.EqualValues(3*rowNum, reportInfo[hookutil.RelatedCntKey]) + return + } + } + } + go queryCheckReport() + queryResult, err := c.Proxy.Query(ctx, &milvuspb.QueryRequest{ + DbName: dbName, + CollectionName: collectionName, + Expr: "", + OutputFields: []string{"count(*)"}, + }) + if queryResult.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success { + log.Warn("searchResult fail reason", zap.String("reason", queryResult.GetStatus().GetReason())) + } + s.NoError(err) + s.Equal(commonpb.ErrorCode_Success, queryResult.GetStatus().GetErrorCode()) + } + + { + // query with partition key + queryCheckReport := func() { + timeoutCtx, cancelFunc := context.WithTimeout(ctx, 5*time.Second) + defer cancelFunc() + + for { + select { + case <-timeoutCtx.Done(): + s.Fail("query check timeout") + case report := <-c.Extension.GetReportChan(): + reportInfo := report.(map[string]any) + log.Info("query report info", zap.Any("reportInfo", reportInfo)) + if reportInfo[hookutil.OpTypeKey] == hookutil.OpTypeStorage { + continue + } + s.Equal(hookutil.OpTypeQuery, reportInfo[hookutil.OpTypeKey]) + s.NotEqualValues(0, reportInfo[hookutil.DataSizeKey]) + s.EqualValues(rowNum, reportInfo[hookutil.RelatedCntKey]) + return + } + } + } + go queryCheckReport() + queryResult, err := c.Proxy.Query(ctx, &milvuspb.QueryRequest{ + DbName: dbName, + CollectionName: collectionName, + Expr: "pid == 1", + OutputFields: []string{"count(*)"}, + }) + if queryResult.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success { + log.Warn("searchResult fail reason", zap.String("reason", queryResult.GetStatus().GetReason())) + } + s.NoError(err) + s.Equal(commonpb.ErrorCode_Success, queryResult.GetStatus().GetErrorCode()) + } + + { + // delete without partition key + deleteCheckReport := func() { + timeoutCtx, cancelFunc := context.WithTimeout(ctx, 5*time.Second) + defer cancelFunc() + + for { + select { + case <-timeoutCtx.Done(): + s.Fail("delete check timeout") + case report := <-c.Extension.GetReportChan(): + reportInfo := report.(map[string]any) + log.Info("delete report info", zap.Any("reportInfo", reportInfo)) + if reportInfo[hookutil.OpTypeKey] == hookutil.OpTypeStorage { + continue + } + s.Equal(hookutil.OpTypeDelete, reportInfo[hookutil.OpTypeKey]) + s.EqualValues(rowNum, reportInfo[hookutil.SuccessCntKey]) + s.EqualValues(rowNum, reportInfo[hookutil.RelatedCntKey]) + return + } + } + } + go deleteCheckReport() + deleteResult, err := c.Proxy.Delete(ctx, &milvuspb.DeleteRequest{ + DbName: dbName, + CollectionName: collectionName, + Expr: integration.Int64Field + " < 1000", + }) + if deleteResult.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success { + log.Warn("deleteResult fail reason", zap.String("reason", deleteResult.GetStatus().GetReason())) + } + s.NoError(err) + s.Equal(commonpb.ErrorCode_Success, deleteResult.GetStatus().GetErrorCode()) + } + + { + // delete with partition key + deleteCheckReport := func() { + timeoutCtx, cancelFunc := context.WithTimeout(ctx, 5*time.Second) + defer cancelFunc() + + for { + select { + case <-timeoutCtx.Done(): + s.Fail("delete check timeout") + case report := <-c.Extension.GetReportChan(): + reportInfo := report.(map[string]any) + log.Info("delete report info", zap.Any("reportInfo", reportInfo)) + if reportInfo[hookutil.OpTypeKey] == hookutil.OpTypeStorage { + continue + } + s.Equal(hookutil.OpTypeDelete, reportInfo[hookutil.OpTypeKey]) + s.EqualValues(rowNum, reportInfo[hookutil.SuccessCntKey]) + s.EqualValues(rowNum, reportInfo[hookutil.RelatedCntKey]) + return + } + } + } + go deleteCheckReport() + deleteResult, err := c.Proxy.Delete(ctx, &milvuspb.DeleteRequest{ + DbName: dbName, + CollectionName: collectionName, + Expr: integration.Int64Field + " < 2000 && pid == 2", + }) + if deleteResult.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success { + log.Warn("deleteResult fail reason", zap.String("reason", deleteResult.GetStatus().GetReason())) + } + s.NoError(err) + s.Equal(commonpb.ErrorCode_Success, deleteResult.GetStatus().GetErrorCode()) + } +} + +func TestPartitionKey(t *testing.T) { + suite.Run(t, new(PartitionKeySuite)) +} diff --git a/tests/integration/suite.go b/tests/integration/suite.go index 4189a34a31..67a8cae77c 100644 --- a/tests/integration/suite.go +++ b/tests/integration/suite.go @@ -28,6 +28,7 @@ import ( "go.uber.org/zap/zapcore" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/util/hookutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/etcd" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -104,6 +105,24 @@ func (s *MiniClusterSuite) SetupTest() { s.Cluster = c // start mini cluster + nodeIDCheckReport := func() { + timeoutCtx, cancelFunc := context.WithTimeout(ctx, 5*time.Second) + defer cancelFunc() + + for { + select { + case <-timeoutCtx.Done(): + s.Fail("node id check timeout") + case report := <-c.Extension.GetReportChan(): + reportInfo := report.(map[string]any) + s.T().Log("node id report info: ", reportInfo) + s.Equal(hookutil.OpTypeNodeID, reportInfo[hookutil.OpTypeKey]) + s.NotEqualValues(0, reportInfo[hookutil.NodeIDKey]) + return + } + } + } + go nodeIDCheckReport() s.Require().NoError(s.Cluster.Start()) } diff --git a/tests/integration/util_insert.go b/tests/integration/util_insert.go index 729f8a87a2..6c527a7e23 100644 --- a/tests/integration/util_insert.go +++ b/tests/integration/util_insert.go @@ -58,7 +58,39 @@ func NewInt64FieldData(fieldName string, numRows int) *schemapb.FieldData { Scalars: &schemapb.ScalarField{ Data: &schemapb.ScalarField_LongData{ LongData: &schemapb.LongArray{ - Data: GenerateInt64Array(numRows), + Data: GenerateInt64Array(numRows, 0), + }, + }, + }, + }, + } +} + +func NewInt64FieldDataWithStart(fieldName string, numRows int, start int64) *schemapb.FieldData { + return &schemapb.FieldData{ + Type: schemapb.DataType_Int64, + FieldName: fieldName, + Field: &schemapb.FieldData_Scalars{ + Scalars: &schemapb.ScalarField{ + Data: &schemapb.ScalarField_LongData{ + LongData: &schemapb.LongArray{ + Data: GenerateInt64Array(numRows, start), + }, + }, + }, + }, + } +} + +func NewInt64SameFieldData(fieldName string, numRows int, value int64) *schemapb.FieldData { + return &schemapb.FieldData{ + Type: schemapb.DataType_Int64, + FieldName: fieldName, + Field: &schemapb.FieldData_Scalars{ + Scalars: &schemapb.ScalarField{ + Data: &schemapb.ScalarField_LongData{ + LongData: &schemapb.LongArray{ + Data: GenerateSameInt64Array(numRows, value), }, }, }, @@ -144,10 +176,18 @@ func NewBinaryVectorFieldData(fieldName string, numRows, dim int) *schemapb.Fiel } } -func GenerateInt64Array(numRows int) []int64 { +func GenerateInt64Array(numRows int, start int64) []int64 { ret := make([]int64, numRows) for i := 0; i < numRows; i++ { - ret[i] = int64(i) + ret[i] = int64(i) + start + } + return ret +} + +func GenerateSameInt64Array(numRows int, value int64) []int64 { + ret := make([]int64, numRows) + for i := 0; i < numRows; i++ { + ret[i] = value } return ret }