From b83b376cfc9352f1e14edaccdf30a67fc0e98386 Mon Sep 17 00:00:00 2001 From: wei liu Date: Tue, 5 Nov 2024 10:52:23 +0800 Subject: [PATCH 01/40] fix: Search/Query may failed during updating delegator cache. (#37116) issue: #37115 casue init query node client is too heavy, so we remove updateShardClient from leader mutex, which cause much more concurrent cornor cases. This PR delay query node client's init operation until `getClient` is called, then use leader mutex to protect updating shard client progress to avoid concurrent issues. --------- Signed-off-by: Wei Liu --- internal/proxy/lb_policy.go | 21 ++++++++++- internal/proxy/meta_cache.go | 6 +-- internal/proxy/shard_client.go | 67 ++++++++++++++++++++-------------- 3 files changed, 61 insertions(+), 33 deletions(-) diff --git a/internal/proxy/lb_policy.go b/internal/proxy/lb_policy.go index e0bb9794d2eff..0201bfec2b480 100644 --- a/internal/proxy/lb_policy.go +++ b/internal/proxy/lb_policy.go @@ -98,6 +98,23 @@ func (lb *LBPolicyImpl) Start(ctx context.Context) { } } +// GetShardLeaders should always retry until ctx done, except the collection is not loaded. +func (lb *LBPolicyImpl) GetShardLeaders(ctx context.Context, dbName string, collName string, collectionID int64, withCache bool) (map[string][]nodeInfo, error) { + var shardLeaders map[string][]nodeInfo + // use retry to handle query coord service not ready + err := retry.Handle(ctx, func() (bool, error) { + var err error + shardLeaders, err = globalMetaCache.GetShards(ctx, withCache, dbName, collName, collectionID) + if err != nil { + return !errors.Is(err, merr.ErrCollectionLoaded), err + } + + return false, nil + }) + + return shardLeaders, err +} + // try to select the best node from the available nodes func (lb *LBPolicyImpl) selectNode(ctx context.Context, balancer LBBalancer, workload ChannelWorkload, excludeNodes typeutil.UniqueSet) (int64, error) { availableNodes := lo.FilterMap(workload.shardLeaders, func(node int64, _ int) (int64, bool) { return node, !excludeNodes.Contain(node) }) @@ -105,7 +122,7 @@ func (lb *LBPolicyImpl) selectNode(ctx context.Context, balancer LBBalancer, wor if err != nil { log := log.Ctx(ctx) globalMetaCache.DeprecateShardCache(workload.db, workload.collectionName) - shardLeaders, err := globalMetaCache.GetShards(ctx, false, workload.db, workload.collectionName, workload.collectionID) + shardLeaders, err := lb.GetShardLeaders(ctx, workload.db, workload.collectionName, workload.collectionID, false) if err != nil { log.Warn("failed to get shard delegator", zap.Int64("collectionID", workload.collectionID), @@ -195,7 +212,7 @@ func (lb *LBPolicyImpl) ExecuteWithRetry(ctx context.Context, workload ChannelWo // Execute will execute collection workload in parallel func (lb *LBPolicyImpl) Execute(ctx context.Context, workload CollectionWorkLoad) error { - dml2leaders, err := globalMetaCache.GetShards(ctx, true, workload.db, workload.collectionName, workload.collectionID) + dml2leaders, err := lb.GetShardLeaders(ctx, workload.db, workload.collectionName, workload.collectionID, true) if err != nil { log.Ctx(ctx).Warn("failed to get shards", zap.Error(err)) return err diff --git a/internal/proxy/meta_cache.go b/internal/proxy/meta_cache.go index 1d360f758c734..34c6db0817723 100644 --- a/internal/proxy/meta_cache.go +++ b/internal/proxy/meta_cache.go @@ -1004,9 +1004,7 @@ func (m *MetaCache) GetShards(ctx context.Context, withCache bool, database, col if _, ok := m.collLeader[database]; !ok { m.collLeader[database] = make(map[string]*shardLeaders) } - m.collLeader[database][collectionName] = newShardLeaders - m.leaderMut.Unlock() iterator := newShardLeaders.GetReader() ret := iterator.Shuffle() @@ -1016,8 +1014,10 @@ func (m *MetaCache) GetShards(ctx context.Context, withCache bool, database, col oldLeaders = cacheShardLeaders.shardLeaders } // update refcnt in shardClientMgr - // and create new client for new leaders + // update shard leader's just create a empty client pool + // and init new client will be execute in getClient _ = m.shardMgr.UpdateShardLeaders(oldLeaders, ret) + m.leaderMut.Unlock() metrics.ProxyUpdateCacheLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method).Observe(float64(tr.ElapseSpan().Milliseconds())) return ret, nil diff --git a/internal/proxy/shard_client.go b/internal/proxy/shard_client.go index 237b3b3e8a0a3..8475494e6659c 100644 --- a/internal/proxy/shard_client.go +++ b/internal/proxy/shard_client.go @@ -31,26 +31,38 @@ var errClosed = errors.New("client is closed") type shardClient struct { sync.RWMutex info nodeInfo - client types.QueryNodeClient isClosed bool refCnt int clients []types.QueryNodeClient idx atomic.Int64 poolSize int pooling bool + + initialized atomic.Bool + creator queryNodeCreatorFunc } func (n *shardClient) getClient(ctx context.Context) (types.QueryNodeClient, error) { + if !n.initialized.Load() { + n.Lock() + if !n.initialized.Load() { + if err := n.initClients(); err != nil { + n.Unlock() + return nil, err + } + n.initialized.Store(true) + } + n.Unlock() + } + n.RLock() defer n.RUnlock() if n.isClosed { return nil, errClosed } - if n.pooling { - idx := n.idx.Inc() - return n.clients[int(idx)%n.poolSize], nil - } - return n.client, nil + + idx := n.idx.Inc() + return n.clients[int(idx)%n.poolSize], nil } func (n *shardClient) inc() { @@ -65,12 +77,13 @@ func (n *shardClient) inc() { func (n *shardClient) close() { n.isClosed = true n.refCnt = 0 - if n.client != nil { - if err := n.client.Close(); err != nil { + + for _, client := range n.clients { + if err := client.Close(); err != nil { log.Warn("close grpc client failed", zap.Error(err)) } - n.client = nil } + n.clients = nil } func (n *shardClient) dec() bool { @@ -94,41 +107,39 @@ func (n *shardClient) Close() { n.close() } -func newShardClient(info *nodeInfo, client types.QueryNodeClient) *shardClient { - ret := &shardClient{ +func newPoolingShardClient(info *nodeInfo, creator queryNodeCreatorFunc) (*shardClient, error) { + return &shardClient{ info: nodeInfo{ nodeID: info.nodeID, address: info.address, }, - client: client, - refCnt: 1, - } - return ret + refCnt: 1, + pooling: true, + creator: creator, + }, nil } -func newPoolingShardClient(info *nodeInfo, creator queryNodeCreatorFunc) (*shardClient, error) { +func (n *shardClient) initClients() error { num := paramtable.Get().ProxyCfg.QueryNodePoolingSize.GetAsInt() if num <= 0 { num = 1 } clients := make([]types.QueryNodeClient, 0, num) for i := 0; i < num; i++ { - client, err := creator(context.Background(), info.address, info.nodeID) + client, err := n.creator(context.Background(), n.info.address, n.info.nodeID) if err != nil { - return nil, err + // roll back already created clients + for _, c := range clients[:i] { + c.Close() + } + return errors.Wrap(err, fmt.Sprintf("create client for node=%d failed", n.info.nodeID)) } clients = append(clients, client) } - return &shardClient{ - info: nodeInfo{ - nodeID: info.nodeID, - address: info.address, - }, - refCnt: 1, - pooling: true, - clients: clients, - poolSize: num, - }, nil + + n.clients = clients + n.poolSize = num + return nil } type shardClientMgr interface { From 9a0e1c82bc74e5ba88048065a1788716570ef3c4 Mon Sep 17 00:00:00 2001 From: Zhen Ye Date: Tue, 5 Nov 2024 16:28:23 +0800 Subject: [PATCH 02/40] fix: repeated error code in milvus and segcore (#37359) issue: #37357 Signed-off-by: chyezh --- internal/core/src/common/EasyAssert.h | 12 ++++++------ pkg/util/merr/errors.go | 7 +++++-- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/internal/core/src/common/EasyAssert.h b/internal/core/src/common/EasyAssert.h index 763bd65a138a2..22726938ffbdc 100644 --- a/internal/core/src/common/EasyAssert.h +++ b/internal/core/src/common/EasyAssert.h @@ -64,13 +64,13 @@ enum ErrorCode { MemAllocateFailed = 2034, MemAllocateSizeNotMatch = 2035, MmapError = 2036, - OutOfRange = 2037, - GcpNativeError = 2038, - KnowhereError = 2100, + // timeout or cancel related + FollyOtherException = 2037, + FollyCancel = 2038, + OutOfRange = 2039, + GcpNativeError = 2040, - // timeout or cancel related. - FollyOtherException = 2200, - FollyCancel = 2201 + KnowhereError = 2099 }; namespace impl { void diff --git a/pkg/util/merr/errors.go b/pkg/util/merr/errors.go index 30bd26ec49f8f..a0cec3408096d 100644 --- a/pkg/util/merr/errors.go +++ b/pkg/util/merr/errors.go @@ -180,8 +180,11 @@ var ( ErrSegcore = newMilvusError("segcore error", 2000, false) ErrSegcoreUnsupported = newMilvusError("segcore unsupported error", 2001, false) ErrSegcorePretendFinished = newMilvusError("segcore pretend finished", 2002, false) - ErrSegcoreFollyOtherException = newMilvusError("segcore folly other exception", 2200, false) // throw from segcore. - ErrSegcoreFollyCancel = newMilvusError("segcore Future was canceled", 2201, false) // throw from segcore. + ErrSegcoreFollyOtherException = newMilvusError("segcore folly other exception", 2037, false) // throw from segcore. + ErrSegcoreFollyCancel = newMilvusError("segcore Future was canceled", 2038, false) // throw from segcore. + ErrSegcoreOutOfRange = newMilvusError("segcore out of range", 2039, false) // throw from segcore. + ErrSegcoreGCPNativeError = newMilvusError("segcore GCP native error", 2040, false) // throw from segcore. + KnowhereError = newMilvusError("knowhere error", 2099, false) // throw from segcore. // Do NOT export this, // never allow programmer using this, keep only for converting unknown error to milvusError From bd04cac4b3fe1e301635619331914f667a43a0c9 Mon Sep 17 00:00:00 2001 From: Bingyi Sun Date: Tue, 5 Nov 2024 17:12:23 +0800 Subject: [PATCH 03/40] fix: fix group by on chunked segment (#37292) https://github.com/milvus-io/milvus/issues/37244 --------- Signed-off-by: sunby --- internal/core/src/common/QueryResult.h | 15 +++++----- .../operator/groupby/SearchGroupByOperator.h | 2 +- internal/core/src/mmap/ChunkedColumn.h | 5 ++++ internal/core/src/query/SearchOnGrowing.cpp | 9 +++--- internal/core/src/query/SearchOnSealed.cpp | 8 +++-- .../core/unittest/test_chunked_segment.cpp | 29 ++++++++++++++++++- 6 files changed, 52 insertions(+), 16 deletions(-) diff --git a/internal/core/src/common/QueryResult.h b/internal/core/src/common/QueryResult.h index b5298d01b4346..75d54987ea607 100644 --- a/internal/core/src/common/QueryResult.h +++ b/internal/core/src/common/QueryResult.h @@ -66,8 +66,9 @@ struct OffsetDisPairComparator { }; struct VectorIterator { public: - VectorIterator(int chunk_count, int64_t chunk_rows = -1) - : chunk_rows_(chunk_rows) { + VectorIterator(int chunk_count, + const std::vector& total_rows_until_chunk = {}) + : total_rows_until_chunk_(total_rows_until_chunk) { iterators_.reserve(chunk_count); } @@ -119,7 +120,7 @@ struct VectorIterator { private: int64_t convert_to_segment_offset(int64_t chunk_offset, int chunk_idx) { - if (chunk_rows_ == -1) { + if (total_rows_until_chunk_.size() == 0) { AssertInfo( iterators_.size() == 1, "Wrong state for vectorIterators, which having incorrect " @@ -129,7 +130,7 @@ struct VectorIterator { iterators_.size()); return chunk_offset; } - return chunk_idx * chunk_rows_ + chunk_offset; + return total_rows_until_chunk_[chunk_idx] + chunk_offset; } private: @@ -139,7 +140,7 @@ struct VectorIterator { OffsetDisPairComparator> heap_; bool sealed = false; - int64_t chunk_rows_ = -1; + std::vector total_rows_until_chunk_; //currently, VectorIterator is guaranteed to be used serially without concurrent problem, in the future //we may need to add mutex to protect the variable sealed }; @@ -163,7 +164,7 @@ struct SearchResult { AssembleChunkVectorIterators( int64_t nq, int chunk_count, - int64_t rows_per_chunk, + const std::vector& total_rows_until_chunk, const std::vector& kw_iterators) { AssertInfo(kw_iterators.size() == nq * chunk_count, "kw_iterators count:{} is not equal to nq*chunk_count:{}, " @@ -176,7 +177,7 @@ struct SearchResult { vec_iter_idx = vec_iter_idx % nq; if (vector_iterators.size() < nq) { auto vector_iterator = std::make_shared( - chunk_count, rows_per_chunk); + chunk_count, total_rows_until_chunk); vector_iterators.emplace_back(vector_iterator); } auto kw_iterator = kw_iterators[i]; diff --git a/internal/core/src/exec/operator/groupby/SearchGroupByOperator.h b/internal/core/src/exec/operator/groupby/SearchGroupByOperator.h index e6a95c6603809..640789518cdf1 100644 --- a/internal/core/src/exec/operator/groupby/SearchGroupByOperator.h +++ b/internal/core/src/exec/operator/groupby/SearchGroupByOperator.h @@ -140,7 +140,7 @@ PrepareVectorIteratorsFromIndex(const SearchInfo& search_info, index.VectorIterators(dataset, search_conf, bitset); if (iterators_val.has_value()) { search_result.AssembleChunkVectorIterators( - nq, 1, -1, iterators_val.value()); + nq, 1, {0}, iterators_val.value()); } else { LOG_ERROR( "Returned knowhere iterator has non-ready iterators " diff --git a/internal/core/src/mmap/ChunkedColumn.h b/internal/core/src/mmap/ChunkedColumn.h index 91a7bf230b3ca..2a9e3ff6db40b 100644 --- a/internal/core/src/mmap/ChunkedColumn.h +++ b/internal/core/src/mmap/ChunkedColumn.h @@ -165,6 +165,11 @@ class ChunkedColumnBase : public ColumnBase { return num_rows_until_chunk_[chunk_id]; } + const std::vector& + GetNumRowsUntilChunk() const { + return num_rows_until_chunk_; + } + protected: bool nullable_{false}; size_t num_rows_{0}; diff --git a/internal/core/src/query/SearchOnGrowing.cpp b/internal/core/src/query/SearchOnGrowing.cpp index 0222ce2cd8b74..f71efb7562a6b 100644 --- a/internal/core/src/query/SearchOnGrowing.cpp +++ b/internal/core/src/query/SearchOnGrowing.cpp @@ -150,11 +150,12 @@ SearchOnGrowing(const segcore::SegmentGrowingImpl& segment, } } if (info.group_by_field_id_.has_value()) { + std::vector chunk_rows(max_chunk, 0); + for (int i = 1; i < max_chunk; ++i) { + chunk_rows[i] = i * vec_size_per_chunk; + } search_result.AssembleChunkVectorIterators( - num_queries, - max_chunk, - vec_size_per_chunk, - final_qr.chunk_iterators()); + num_queries, max_chunk, chunk_rows, final_qr.chunk_iterators()); } else { search_result.distances_ = std::move(final_qr.mutable_distances()); search_result.seg_offsets_ = diff --git a/internal/core/src/query/SearchOnSealed.cpp b/internal/core/src/query/SearchOnSealed.cpp index 9b3a4df287599..2bd7e8edb8ac6 100644 --- a/internal/core/src/query/SearchOnSealed.cpp +++ b/internal/core/src/query/SearchOnSealed.cpp @@ -161,8 +161,10 @@ SearchOnSealed(const Schema& schema, offset += chunk_size; } if (search_info.group_by_field_id_.has_value()) { - result.AssembleChunkVectorIterators( - num_queries, 1, -1, final_qr.chunk_iterators()); + result.AssembleChunkVectorIterators(num_queries, + num_chunk, + column->GetNumRowsUntilChunk(), + final_qr.chunk_iterators()); } else { result.distances_ = std::move(final_qr.mutable_distances()); result.seg_offsets_ = std::move(final_qr.mutable_seg_offsets()); @@ -201,7 +203,7 @@ SearchOnSealed(const Schema& schema, auto sub_qr = BruteForceSearchIterators( dataset, vec_data, row_count, search_info, bitset, data_type); result.AssembleChunkVectorIterators( - num_queries, 1, -1, sub_qr.chunk_iterators()); + num_queries, 1, {0}, sub_qr.chunk_iterators()); } else { auto sub_qr = BruteForceSearch( dataset, vec_data, row_count, search_info, bitset, data_type); diff --git a/internal/core/unittest/test_chunked_segment.cpp b/internal/core/unittest/test_chunked_segment.cpp index b0b624b68a793..d9b34218bdc7b 100644 --- a/internal/core/unittest/test_chunked_segment.cpp +++ b/internal/core/unittest/test_chunked_segment.cpp @@ -92,7 +92,7 @@ TEST(test_chunk_segment, TestSearchOnSealed) { search_info, query_data, 1, - chunk_size * chunk_num, + total_row_count, bv, search_result); @@ -107,4 +107,31 @@ TEST(test_chunk_segment, TestSearchOnSealed) { for (int i = 0; i < total_row_count; i++) { ASSERT_TRUE(offsets.find(i) != offsets.end()); } + + // test with group by + search_info.group_by_field_id_ = fakevec_id; + std::fill(bitset_data, bitset_data + bitset_size, 0); + query::SearchOnSealed(*schema, + column, + search_info, + query_data, + 1, + total_row_count, + bv, + search_result); + + ASSERT_EQ(1, search_result.vector_iterators_->size()); + + auto iter = search_result.vector_iterators_->at(0); + // collect all offsets + offsets.clear(); + while (iter->HasNext()) { + auto [offset, distance] = iter->Next().value(); + offsets.insert(offset); + } + + ASSERT_EQ(total_row_count, offsets.size()); + for (int i = 0; i < total_row_count; i++) { + ASSERT_TRUE(offsets.find(i) != offsets.end()); + } } From 9a9de3df5cf6f306e1876c2b19a597244222afb4 Mon Sep 17 00:00:00 2001 From: congqixia Date: Tue, 5 Nov 2024 18:26:23 +0800 Subject: [PATCH 04/40] enhance: Pass rpc stats via gin.Context (#37439) Related #37223 RPC stats worked in middleware but faild to get method & collection info Signed-off-by: Congqi Xia --- .../distributed/proxy/httpserver/handler_v2.go | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/internal/distributed/proxy/httpserver/handler_v2.go b/internal/distributed/proxy/httpserver/handler_v2.go index fced1d4877ed1..88a533637e984 100644 --- a/internal/distributed/proxy/httpserver/handler_v2.go +++ b/internal/distributed/proxy/httpserver/handler_v2.go @@ -192,7 +192,7 @@ func wrapperPost(newReq newReqFunc, v2 handlerFuncV2) gin.HandlerFunc { } } username, _ := c.Get(ContextUsername) - ctx, span := otel.Tracer(typeutil.ProxyRole).Start(c, c.Request.URL.Path) + ctx, span := otel.Tracer(typeutil.ProxyRole).Start(getCtx(c), c.Request.URL.Path) defer span.End() ctx = proxy.NewContextWithMetadata(ctx, username.(string), dbName) traceID := span.SpanContext().TraceID().String() @@ -204,10 +204,23 @@ func wrapperPost(newReq newReqFunc, v2 handlerFuncV2) gin.HandlerFunc { } } +const ( + v2CtxKey = `milvus_restful_v2_ctxkey` +) + +func getCtx(ctx *gin.Context) context.Context { + v, ok := ctx.Get(v2CtxKey) + if !ok { + return ctx + } + return v.(context.Context) +} + // restfulSizeMiddleware is the middleware fetchs metrics stats from gin struct. func restfulSizeMiddleware(handler gin.HandlerFunc, observeOutbound bool) gin.HandlerFunc { return func(ctx *gin.Context) { h := metrics.WrapRestfulContext(ctx, ctx.Request.ContentLength) + ctx.Set(v2CtxKey, h) handler(ctx) metrics.RecordRestfulMetrics(h, int64(ctx.Writer.Size()), observeOutbound) } From 0645d46ec6b5154e2ae1921a5ffd900bba436620 Mon Sep 17 00:00:00 2001 From: congqixia Date: Tue, 5 Nov 2024 19:26:24 +0800 Subject: [PATCH 05/40] fix: Skip EOF error when default empty yaml file (#37445) Related to #37404 --------- Signed-off-by: Congqi Xia --- pkg/config/file_source.go | 3 ++- pkg/config/file_source_test.go | 32 ++++++++++++++++++++++++++++++++ 2 files changed, 34 insertions(+), 1 deletion(-) create mode 100644 pkg/config/file_source_test.go diff --git a/pkg/config/file_source.go b/pkg/config/file_source.go index 4eace878d890f..535e87d5c2963 100644 --- a/pkg/config/file_source.go +++ b/pkg/config/file_source.go @@ -19,6 +19,7 @@ package config import ( "bytes" "fmt" + "io" "os" "path/filepath" "sync" @@ -146,7 +147,7 @@ func (fs *FileSource) loadFromFile() error { var node yaml.Node decoder := yaml.NewDecoder(bytes.NewReader(data)) - if err := decoder.Decode(&node); err != nil { + if err := decoder.Decode(&node); err != nil && !errors.Is(err, io.EOF) { return errors.Wrap(err, "YAML unmarshal failed: "+configFile) } diff --git a/pkg/config/file_source_test.go b/pkg/config/file_source_test.go new file mode 100644 index 0000000000000..60abdb110ed7d --- /dev/null +++ b/pkg/config/file_source_test.go @@ -0,0 +1,32 @@ +package config + +import ( + "os" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestEmptyYaml(t *testing.T) { + file, err := os.CreateTemp(os.TempDir(), "milvus_ut_config_fs_*.yaml") + require.NoError(t, err) + + filepath := file.Name() + + file.WriteString("#") + file.Close() + + defer os.Remove(filepath) + + fs := NewFileSource(&FileInfo{ + Files: []string{filepath}, + RefreshInterval: time.Hour, + }) + + _, err = fs.GetConfigurations() + assert.NoError(t, err) + + fs.Close() +} From 625b6176cd3eda38d87327de3504827675f19a16 Mon Sep 17 00:00:00 2001 From: "cai.zhang" Date: Tue, 5 Nov 2024 20:36:24 +0800 Subject: [PATCH 06/40] fix: Search for pk using raw data to reduce the overhead caused by views (#37202) issue: #37152 Signed-off-by: Cai Zhang --- internal/core/src/common/Chunk.h | 37 +++- internal/core/src/mmap/ChunkedColumn.h | 5 + internal/core/src/mmap/Column.h | 24 +++ .../src/segcore/ChunkedSegmentSealedImpl.cpp | 182 ++++-------------- .../src/segcore/ChunkedSegmentSealedImpl.h | 8 +- .../core/src/segcore/SegmentSealedImpl.cpp | 87 +++------ internal/core/src/segcore/SegmentSealedImpl.h | 4 + internal/core/unittest/test_sealed.cpp | 22 +++ internal/core/unittest/test_utils/DataGen.h | 1 + 9 files changed, 163 insertions(+), 207 deletions(-) diff --git a/internal/core/src/common/Chunk.h b/internal/core/src/common/Chunk.h index 5f9a40c0e43d5..7cfaa7fad466e 100644 --- a/internal/core/src/common/Chunk.h +++ b/internal/core/src/common/Chunk.h @@ -133,13 +133,46 @@ class StringChunk : public Chunk { offsets_ = reinterpret_cast(data + null_bitmap_bytes_num); } + std::string_view + operator[](const int i) const { + if (i < 0 || i > row_nums_) { + PanicInfo(ErrorCode::OutOfRange, "index out of range"); + } + + return {data_ + offsets_[i], offsets_[i + 1] - offsets_[i]}; + } + std::pair, FixedVector> StringViews(); + int + binary_search_string(std::string_view target) { + // only supported sorted pk + int left = 0; + int right = row_nums_ - 1; // `right` should be num_rows_ - 1 + int result = + -1; // Initialize result to store the first occurrence index + + while (left <= right) { + int mid = left + (right - left) / 2; + std::string_view midString = (*this)[mid]; + if (midString == target) { + result = mid; // Store the index of match + right = mid - 1; // Continue searching in the left half + } else if (midString < target) { + // midString < target + left = mid + 1; + } else { + // midString > target + right = mid - 1; + } + } + return result; + } + const char* ValueAt(int64_t idx) const override { - PanicInfo(ErrorCode::Unsupported, - "StringChunk::ValueAt is not supported"); + return (*this)[idx].data(); } uint64_t* diff --git a/internal/core/src/mmap/ChunkedColumn.h b/internal/core/src/mmap/ChunkedColumn.h index 2a9e3ff6db40b..31255b60b53a8 100644 --- a/internal/core/src/mmap/ChunkedColumn.h +++ b/internal/core/src/mmap/ChunkedColumn.h @@ -317,6 +317,11 @@ class ChunkedVariableColumn : public ChunkedColumnBase { ->StringViews(); } + std::shared_ptr + GetChunk(int64_t chunk_id) const { + return chunks_[chunk_id]; + } + BufferView GetBatchBuffer(int64_t start_offset, int64_t length) override { if (start_offset < 0 || start_offset > num_rows_ || diff --git a/internal/core/src/mmap/Column.h b/internal/core/src/mmap/Column.h index 2800d849d5cf3..b50a582e90312 100644 --- a/internal/core/src/mmap/Column.h +++ b/internal/core/src/mmap/Column.h @@ -749,6 +749,30 @@ class SingleChunkVariableColumn : public SingleChunkColumnBase { return ViewType(pos + sizeof(uint32_t), size); } + int + binary_search_string(std::string_view target) { + int left = 0; + int right = num_rows_ - 1; // `right` should be num_rows_ - 1 + int result = + -1; // Initialize result to store the first occurrence index + + while (left <= right) { + int mid = left + (right - left) / 2; + std::string_view midString = this->RawAt(mid); + if (midString == target) { + result = mid; // Store the index of match + right = mid - 1; // Continue searching in the left half + } else if (midString < target) { + // midString < target + left = mid + 1; + } else { + // midString > target + right = mid - 1; + } + } + return result; + } + std::string_view RawAt(const int i) const { return std::string_view((*this)[i]); diff --git a/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp b/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp index a77579c1dbcc7..c7d64161ea9e0 100644 --- a/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp +++ b/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp @@ -863,16 +863,17 @@ ChunkedSegmentSealedImpl::mask_with_delete(BitsetTypeView& bitset, auto bitmap_holder = std::shared_ptr(); - if (!is_sorted_by_pk_) { - bitmap_holder = get_deleted_bitmap(del_barrier, - ins_barrier, - deleted_record_, - insert_record_, - timestamp); - } else { - bitmap_holder = get_deleted_bitmap_s( - del_barrier, ins_barrier, deleted_record_, timestamp); - } + auto search_fn = [this](const PkType& pk, int64_t barrier) { + return this->search_pk(pk, barrier); + }; + bitmap_holder = get_deleted_bitmap(del_barrier, + ins_barrier, + deleted_record_, + insert_record_, + timestamp, + is_sorted_by_pk_, + search_fn); + if (!bitmap_holder || !bitmap_holder->bitmap_ptr) { return; } @@ -1182,72 +1183,34 @@ ChunkedSegmentSealedImpl::check_search(const query::Plan* plan) const { std::vector ChunkedSegmentSealedImpl::search_pk(const PkType& pk, Timestamp timestamp) const { - auto pk_field_id = schema_->get_primary_field_id().value_or(FieldId(-1)); - AssertInfo(pk_field_id.get() != -1, "Primary key is -1"); - auto pk_column = fields_.at(pk_field_id); - std::vector pk_offsets; - switch (schema_->get_fields().at(pk_field_id).get_data_type()) { - case DataType::INT64: { - auto target = std::get(pk); - // get int64 pks - auto num_chunk = pk_column->num_chunks(); - for (int i = 0; i < num_chunk; ++i) { - auto src = reinterpret_cast(pk_column->Data(i)); - auto chunk_row_num = pk_column->chunk_row_nums(i); - auto it = std::lower_bound( - src, - src + chunk_row_num, - target, - [](const int64_t& elem, const int64_t& value) { - return elem < value; - }); - for (; it != src + chunk_row_num && *it == target; it++) { - auto offset = it - src; - if (insert_record_.timestamps_[offset] <= timestamp) { - pk_offsets.emplace_back(offset); - } - } - } - break; - } - case DataType::VARCHAR: { - auto target = std::get(pk); - // get varchar pks - auto var_column = - std::dynamic_pointer_cast>( - pk_column); - auto num_chunk = var_column->num_chunks(); - for (int i = 0; i < num_chunk; ++i) { - auto views = var_column->StringViews(i).first; - auto it = std::lower_bound(views.begin(), views.end(), target); - for (; it != views.end() && *it == target; it++) { - auto offset = std::distance(views.begin(), it); - if (insert_record_.timestamps_[offset] <= timestamp) { - pk_offsets.emplace_back(offset); - } - } - } - break; - } - default: { - PanicInfo( - DataTypeInvalid, - fmt::format( - "unsupported type {}", - schema_->get_fields().at(pk_field_id).get_data_type())); - } + if (!is_sorted_by_pk_) { + return insert_record_.search_pk(pk, timestamp); } - - return pk_offsets; + return search_sorted_pk(pk, [this, timestamp](int64_t offset) { + return insert_record_.timestamps_[offset] <= timestamp; + }); } std::vector ChunkedSegmentSealedImpl::search_pk(const PkType& pk, int64_t insert_barrier) const { + if (!is_sorted_by_pk_) { + return insert_record_.search_pk(pk, insert_barrier); + } + return search_sorted_pk(pk, [insert_barrier](int64_t offset) { + return offset < insert_barrier; + }); +} + +template +std::vector +ChunkedSegmentSealedImpl::search_sorted_pk(const PkType& pk, + Condition condition) const { auto pk_field_id = schema_->get_primary_field_id().value_or(FieldId(-1)); AssertInfo(pk_field_id.get() != -1, "Primary key is -1"); auto pk_column = fields_.at(pk_field_id); std::vector pk_offsets; + switch (schema_->get_fields().at(pk_field_id).get_data_type()) { case DataType::INT64: { auto target = std::get(pk); @@ -1264,9 +1227,10 @@ ChunkedSegmentSealedImpl::search_pk(const PkType& pk, [](const int64_t& elem, const int64_t& value) { return elem < value; }); - for (; it != src + chunk_row_num && *it == target; it++) { + for (; it != src + pk_column->NumRows() && *it == target; + ++it) { auto offset = it - src; - if (offset < insert_barrier) { + if (condition(offset)) { pk_offsets.emplace_back(offset); } } @@ -1283,11 +1247,14 @@ ChunkedSegmentSealedImpl::search_pk(const PkType& pk, auto num_chunk = var_column->num_chunks(); for (int i = 0; i < num_chunk; ++i) { - auto views = var_column->StringViews(i).first; - auto it = std::lower_bound(views.begin(), views.end(), target); - for (; it != views.end() && *it == target; it++) { - auto offset = std::distance(views.begin(), it); - if (offset < insert_barrier) { + // TODO @xiaocai2333, @sunby: chunk need to record the min/max. + auto string_chunk = std::dynamic_pointer_cast( + var_column->GetChunk(i)); + auto offset = string_chunk->binary_search_string(target); + for (; offset != -1 && offset < var_column->NumRows() && + var_column->RawAt(offset) == target; + ++offset) { + if (condition(offset)) { pk_offsets.emplace_back(offset); } } @@ -1306,75 +1273,6 @@ ChunkedSegmentSealedImpl::search_pk(const PkType& pk, return pk_offsets; } -std::shared_ptr -ChunkedSegmentSealedImpl::get_deleted_bitmap_s( - int64_t del_barrier, - int64_t insert_barrier, - DeletedRecord& delete_record, - Timestamp query_timestamp) const { - // if insert_barrier and del_barrier have not changed, use cache data directly - bool hit_cache = false; - int64_t old_del_barrier = 0; - auto current = delete_record.clone_lru_entry( - insert_barrier, del_barrier, old_del_barrier, hit_cache); - if (hit_cache) { - return current; - } - - auto bitmap = current->bitmap_ptr; - - int64_t start, end; - if (del_barrier < old_del_barrier) { - // in this case, ts of delete record[current_del_barrier : old_del_barrier] > query_timestamp - // so these deletion records do not take effect in query/search - // so bitmap corresponding to those pks in delete record[current_del_barrier:old_del_barrier] will be reset to 0 - // for example, current_del_barrier = 2, query_time = 120, the bitmap will be reset to [0, 1, 1, 0, 0, 0, 0, 0] - start = del_barrier; - end = old_del_barrier; - } else { - // the cache is not enough, so update bitmap using new pks in delete record[old_del_barrier:current_del_barrier] - // for example, current_del_barrier = 4, query_time = 300, bitmap will be updated to [0, 1, 1, 0, 1, 1, 0, 0] - start = old_del_barrier; - end = del_barrier; - } - - // Avoid invalid calculations when there are a lot of repeated delete pks - std::unordered_map delete_timestamps; - for (auto del_index = start; del_index < end; ++del_index) { - auto pk = delete_record.pks()[del_index]; - auto timestamp = delete_record.timestamps()[del_index]; - - delete_timestamps[pk] = timestamp > delete_timestamps[pk] - ? timestamp - : delete_timestamps[pk]; - } - - for (auto& [pk, timestamp] : delete_timestamps) { - auto segOffsets = search_pk(pk, insert_barrier); - for (auto offset : segOffsets) { - int64_t insert_row_offset = offset.get(); - - // The deletion record do not take effect in search/query, - // and reset bitmap to 0 - if (timestamp > query_timestamp) { - bitmap->reset(insert_row_offset); - continue; - } - // Insert after delete with same pk, delete will not task effect on this insert record, - // and reset bitmap to 0 - if (insert_record_.timestamps_[offset.get()] >= timestamp) { - bitmap->reset(insert_row_offset); - continue; - } - // insert data corresponding to the insert_row_offset will be ignored in search/query - bitmap->set(insert_row_offset); - } - } - - delete_record.insert_lru_entry(current); - return current; -} - std::pair, bool> ChunkedSegmentSealedImpl::find_first(int64_t limit, const BitsetType& bitset) const { diff --git a/internal/core/src/segcore/ChunkedSegmentSealedImpl.h b/internal/core/src/segcore/ChunkedSegmentSealedImpl.h index fb07c1594b553..60314f019ec26 100644 --- a/internal/core/src/segcore/ChunkedSegmentSealedImpl.h +++ b/internal/core/src/segcore/ChunkedSegmentSealedImpl.h @@ -119,11 +119,9 @@ class ChunkedSegmentSealedImpl : public SegmentSealed { std::vector search_pk(const PkType& pk, int64_t insert_barrier) const; - std::shared_ptr - get_deleted_bitmap_s(int64_t del_barrier, - int64_t insert_barrier, - DeletedRecord& delete_record, - Timestamp query_timestamp) const; + template + std::vector + search_sorted_pk(const PkType& pk, Condition condition) const; std::unique_ptr get_vector(FieldId field_id, diff --git a/internal/core/src/segcore/SegmentSealedImpl.cpp b/internal/core/src/segcore/SegmentSealedImpl.cpp index 4371735fa57b5..2ffb9d9e8ae82 100644 --- a/internal/core/src/segcore/SegmentSealedImpl.cpp +++ b/internal/core/src/segcore/SegmentSealedImpl.cpp @@ -821,66 +821,36 @@ const Schema& SegmentSealedImpl::get_schema() const { return *schema_; } - std::vector SegmentSealedImpl::search_pk(const PkType& pk, Timestamp timestamp) const { - auto pk_field_id = schema_->get_primary_field_id().value_or(FieldId(-1)); - AssertInfo(pk_field_id.get() != -1, "Primary key is -1"); - auto pk_column = fields_.at(pk_field_id); - std::vector pk_offsets; - switch (schema_->get_fields().at(pk_field_id).get_data_type()) { - case DataType::INT64: { - auto target = std::get(pk); - // get int64 pks - auto src = reinterpret_cast(pk_column->Data()); - auto it = - std::lower_bound(src, - src + pk_column->NumRows(), - target, - [](const int64_t& elem, const int64_t& value) { - return elem < value; - }); - for (; it != src + pk_column->NumRows() && *it == target; it++) { - auto offset = it - src; - if (insert_record_.timestamps_[offset] <= timestamp) { - pk_offsets.emplace_back(it - src); - } - } - break; - } - case DataType::VARCHAR: { - auto target = std::get(pk); - // get varchar pks - auto var_column = std::dynamic_pointer_cast< - SingleChunkVariableColumn>(pk_column); - auto views = var_column->Views(); - auto it = std::lower_bound(views.begin(), views.end(), target); - for (; it != views.end() && *it == target; it++) { - auto offset = std::distance(views.begin(), it); - if (insert_record_.timestamps_[offset] <= timestamp) { - pk_offsets.emplace_back(offset); - } - } - break; - } - default: { - PanicInfo( - DataTypeInvalid, - fmt::format( - "unsupported type {}", - schema_->get_fields().at(pk_field_id).get_data_type())); - } + if (!is_sorted_by_pk_) { + return insert_record_.search_pk(pk, timestamp); } - - return pk_offsets; + return search_sorted_pk(pk, [this, timestamp](int64_t offset) { + return insert_record_.timestamps_[offset] <= timestamp; + }); } std::vector SegmentSealedImpl::search_pk(const PkType& pk, int64_t insert_barrier) const { + if (!is_sorted_by_pk_) { + return insert_record_.search_pk(pk, insert_barrier); + } + return search_sorted_pk(pk, [insert_barrier](int64_t offset) { + return offset < insert_barrier; + }); +} + +template +std::vector +SegmentSealedImpl::search_sorted_pk(const PkType& pk, + Condition condition) const { + AssertInfo(is_sorted_by_pk_, "segment is not sorted"); auto pk_field_id = schema_->get_primary_field_id().value_or(FieldId(-1)); AssertInfo(pk_field_id.get() != -1, "Primary key is -1"); auto pk_column = fields_.at(pk_field_id); std::vector pk_offsets; + switch (schema_->get_fields().at(pk_field_id).get_data_type()) { case DataType::INT64: { auto target = std::get(pk); @@ -893,9 +863,11 @@ SegmentSealedImpl::search_pk(const PkType& pk, int64_t insert_barrier) const { [](const int64_t& elem, const int64_t& value) { return elem < value; }); - for (; it != src + pk_column->NumRows() && *it == target; it++) { - if (it - src < insert_barrier) { - pk_offsets.emplace_back(it - src); + + for (; it != src + pk_column->NumRows() && *it == target; ++it) { + auto offset = it - src; + if (condition(offset)) { + pk_offsets.emplace_back(offset); } } break; @@ -905,14 +877,13 @@ SegmentSealedImpl::search_pk(const PkType& pk, int64_t insert_barrier) const { // get varchar pks auto var_column = std::dynamic_pointer_cast< SingleChunkVariableColumn>(pk_column); - auto views = var_column->Views(); - auto it = std::lower_bound(views.begin(), views.end(), target); - while (it != views.end() && *it == target) { - auto offset = std::distance(views.begin(), it); - if (offset < insert_barrier) { + auto offset = var_column->binary_search_string(target); + for (; offset != -1 && offset < var_column->NumRows() && + var_column->RawAt(offset) == target; + ++offset) { + if (condition(offset)) { pk_offsets.emplace_back(offset); } - ++it; } break; } diff --git a/internal/core/src/segcore/SegmentSealedImpl.h b/internal/core/src/segcore/SegmentSealedImpl.h index d5d633992535d..520e82ec0d34f 100644 --- a/internal/core/src/segcore/SegmentSealedImpl.h +++ b/internal/core/src/segcore/SegmentSealedImpl.h @@ -121,6 +121,10 @@ class SegmentSealedImpl : public SegmentSealed { std::vector search_pk(const PkType& pk, int64_t insert_barrier) const; + template + std::vector + search_sorted_pk(const PkType& pk, Condition condition) const; + std::unique_ptr get_vector(FieldId field_id, const int64_t* ids, diff --git a/internal/core/unittest/test_sealed.cpp b/internal/core/unittest/test_sealed.cpp index 68aec85c752a3..5aaba895da676 100644 --- a/internal/core/unittest/test_sealed.cpp +++ b/internal/core/unittest/test_sealed.cpp @@ -2473,3 +2473,25 @@ TEST(Sealed, QueryAllNullableFields) { EXPECT_EQ(double_array_result->valid_data_size(), dataset_size); EXPECT_EQ(float_array_result->valid_data_size(), dataset_size); } + +TEST(Sealed, SearchSortedPk) { + auto schema = std::make_shared(); + auto varchar_pk_field = schema->AddDebugField("pk", DataType::VARCHAR); + schema->set_primary_field_id(varchar_pk_field); + auto segment_sealed = CreateSealedSegment( + schema, nullptr, 999, SegcoreConfig::default_config(), false, true); + auto segment = dynamic_cast(segment_sealed.get()); + + int64_t dataset_size = 1000; + auto dataset = DataGen(schema, dataset_size, 42, 0, 10); + SealedLoadFieldData(dataset, *segment); + + auto pk_values = dataset.get_col(varchar_pk_field); + auto offsets = segment->search_pk(PkType(pk_values[100]), Timestamp(99999)); + EXPECT_EQ(10, offsets.size()); + EXPECT_EQ(100, offsets[0].get()); + + auto offsets2 = segment->search_pk(PkType(pk_values[100]), int64_t(105)); + EXPECT_EQ(5, offsets2.size()); + EXPECT_EQ(100, offsets2[0].get()); +} diff --git a/internal/core/unittest/test_utils/DataGen.h b/internal/core/unittest/test_utils/DataGen.h index 48af55d6ef7d6..db306a4d97390 100644 --- a/internal/core/unittest/test_utils/DataGen.h +++ b/internal/core/unittest/test_utils/DataGen.h @@ -498,6 +498,7 @@ inline GeneratedData DataGen(SchemaPtr schema, data[i * repeat_count + j] = str; } } + std::sort(data.begin(), data.end()); insert_cols(data, N, field_meta); break; } From b2c36efb5feb4ee48047602ebc00dfa33dbe54cb Mon Sep 17 00:00:00 2001 From: sre-ci-robot <56469371+sre-ci-robot@users.noreply.github.com> Date: Tue, 5 Nov 2024 21:18:32 +0800 Subject: [PATCH 07/40] [automated] Bump milvus version to v2.4.15 (#37458) Bump milvus version to v2.4.15 Signed-off-by: sre-ci-robot sre-ci-robot@users.noreply.github.com Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> --- scripts/standalone_embed.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/standalone_embed.sh b/scripts/standalone_embed.sh index 13fb097b4c69a..4f6e7a9ed34b0 100755 --- a/scripts/standalone_embed.sh +++ b/scripts/standalone_embed.sh @@ -47,7 +47,7 @@ EOF --health-start-period=90s \ --health-timeout=20s \ --health-retries=3 \ - milvusdb/milvus:v2.4.14 \ + milvusdb/milvus:v2.4.15 \ milvus run standalone 1> /dev/null } From b3de4b0eddb38b9eca9f961f6d15b542d0b46508 Mon Sep 17 00:00:00 2001 From: yellow-shine Date: Wed, 6 Nov 2024 10:24:30 +0800 Subject: [PATCH 08/40] enhance: refine the pipeline (#37412) 1. enhance: allow to pass conan repo url 2. enhance: add throttle for cpu-e2e and cpp-unit-test pipeline --------- Signed-off-by: Yellow Shine --- ci/jenkins/Nightly2.groovy | 2 +- ci/jenkins/PR-for-go-sdk.groovy | 9 ++++++++- ci/jenkins/PR.groovy | 9 ++++++++- ci/jenkins/UT-CPP.groovy | 15 ++++++++++++--- scripts/3rdparty_build.sh | 7 ++++++- 5 files changed, 35 insertions(+), 7 deletions(-) diff --git a/ci/jenkins/Nightly2.groovy b/ci/jenkins/Nightly2.groovy index 25694ba1a4f77..f24c2129b53ac 100644 --- a/ci/jenkins/Nightly2.groovy +++ b/ci/jenkins/Nightly2.groovy @@ -1,4 +1,4 @@ -@Library('jenkins-shared-library@v0.62.0') _ +@Library('jenkins-shared-library@v0.67.0') _ def pod = libraryResource 'io/milvus/pod/tekton-4am.yaml' diff --git a/ci/jenkins/PR-for-go-sdk.groovy b/ci/jenkins/PR-for-go-sdk.groovy index 4b72b2d6bf7b8..704d16056e7f4 100644 --- a/ci/jenkins/PR-for-go-sdk.groovy +++ b/ci/jenkins/PR-for-go-sdk.groovy @@ -1,4 +1,4 @@ -@Library('jenkins-shared-library@v0.62.0') _ +@Library('jenkins-shared-library@v0.67.0') _ def pod = libraryResource 'io/milvus/pod/tekton-4am.yaml' @@ -11,6 +11,13 @@ pipeline { buildDiscarder logRotator(artifactDaysToKeepStr: '30') preserveStashes(buildCount: 5) disableConcurrentBuilds(abortPrevious: true) + timeout(time: 6, unit: 'HOURS') + throttleJobProperty( + categories: ['go-sdk'], + throttleEnabled: true, + throttleOption: 'category' + + ) } agent { kubernetes { diff --git a/ci/jenkins/PR.groovy b/ci/jenkins/PR.groovy index 3501e0cda6b0c..4bf934081b57c 100644 --- a/ci/jenkins/PR.groovy +++ b/ci/jenkins/PR.groovy @@ -1,4 +1,4 @@ -@Library('jenkins-shared-library@v0.62.0') _ +@Library('jenkins-shared-library@v0.67.0') _ def pod = libraryResource 'io/milvus/pod/tekton-4am.yaml' def milvus_helm_chart_version = '4.2.8' @@ -10,6 +10,13 @@ pipeline { buildDiscarder logRotator(artifactDaysToKeepStr: '30') preserveStashes(buildCount: 5) disableConcurrentBuilds(abortPrevious: true) + timeout(time: 6, unit: 'HOURS') + throttleJobProperty( + categories: ['cpu-e2e'], + throttleEnabled: true, + throttleOption: 'category' + + ) } agent { kubernetes { diff --git a/ci/jenkins/UT-CPP.groovy b/ci/jenkins/UT-CPP.groovy index ac571457f6244..adc2038e8757a 100644 --- a/ci/jenkins/UT-CPP.groovy +++ b/ci/jenkins/UT-CPP.groovy @@ -1,4 +1,4 @@ -@Library('jenkins-shared-library@v0.63.0') _ +@Library('jenkins-shared-library@v0.67.0') _ def pod = libraryResource 'io/milvus/pod/tekton-4am.yaml' def milvus_helm_chart_version = '4.2.8' @@ -10,6 +10,13 @@ pipeline { buildDiscarder logRotator(artifactDaysToKeepStr: '30') preserveStashes(buildCount: 5) disableConcurrentBuilds(abortPrevious: true) + timeout(time: 6, unit: 'HOURS') + throttleJobProperty( + categories: ['cpp-unit-test'], + throttleEnabled: true, + throttleOption: 'category' + + ) } agent { kubernetes { @@ -29,7 +36,7 @@ pipeline { } } } - stage('build') { + stage('build & test') { steps { container('tkn') { script { @@ -40,7 +47,9 @@ pipeline { pullRequestNumber: "$env.CHANGE_ID", make_cmd: "make clean && make USE_ASAN=ON build-cpp-with-coverage", test_entrypoint: "./scripts/run_cpp_codecov.sh", - codecov_files: "./lcov_output.info" + codecov_report_name: "cpp-unit-test", + codecov_files: "./lcov_output.info", + tekton_pipeline_timeout: '3h' } } } diff --git a/scripts/3rdparty_build.sh b/scripts/3rdparty_build.sh index 2c894ba972c63..ec8c6af25cac6 100644 --- a/scripts/3rdparty_build.sh +++ b/scripts/3rdparty_build.sh @@ -52,9 +52,14 @@ pushd ${BUILD_OUTPUT_DIR} export CONAN_REVISIONS_ENABLED=1 export CXXFLAGS="-Wno-error=address -Wno-error=deprecated-declarations" export CFLAGS="-Wno-error=address -Wno-error=deprecated-declarations" + +# Determine the Conan remote URL, using the environment variable if set, otherwise defaulting +CONAN_ARTIFACTORY_URL="${CONAN_ARTIFACTORY_URL:-https://milvus01.jfrog.io/artifactory/api/conan/default-conan-local}" + if [[ ! `conan remote list` == *default-conan-local* ]]; then - conan remote add default-conan-local https://milvus01.jfrog.io/artifactory/api/conan/default-conan-local + conan remote add default-conan-local $CONAN_ARTIFACTORY_URL fi + unameOut="$(uname -s)" case "${unameOut}" in Darwin*) From c83b93946efa9ce2ba2a07a71525adccebe5b21e Mon Sep 17 00:00:00 2001 From: congqixia Date: Wed, 6 Nov 2024 14:36:25 +0800 Subject: [PATCH 09/40] enhance: [GoSDK] Add alter collection API & expose options (#37365) Related to #31293 This PR: - Add `AlterCollection` API for collection property modification - Expose hidden or missing option methods --------- Signed-off-by: Congqi Xia --- client/collection.go | 9 ++++ client/collection_options.go | 47 ++++++++++++++++++++ client/collection_test.go | 43 +++++++++++++++++- client/read_options.go | 5 +++ client/read_test.go | 3 +- tests/go_client/testcases/collection_test.go | 26 +++-------- 6 files changed, 111 insertions(+), 22 deletions(-) diff --git a/client/collection.go b/client/collection.go index 44fd08588b604..c89e7cb71665b 100644 --- a/client/collection.go +++ b/client/collection.go @@ -138,3 +138,12 @@ func (c *Client) RenameCollection(ctx context.Context, option RenameCollectionOp return merr.CheckRPCCall(resp, err) }) } + +func (c *Client) AlterCollection(ctx context.Context, option AlterCollectionOption, callOptions ...grpc.CallOption) error { + req := option.Request() + + return c.callService(func(milvusService milvuspb.MilvusServiceClient) error { + resp, err := milvusService.AlterCollection(ctx, req, callOptions...) + return merr.CheckRPCCall(resp, err) + }) +} diff --git a/client/collection_options.go b/client/collection_options.go index 2263e60323b2b..88192fac02a10 100644 --- a/client/collection_options.go +++ b/client/collection_options.go @@ -17,6 +17,8 @@ package client import ( + "fmt" + "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" @@ -56,6 +58,8 @@ type createCollectionOption struct { // partition key numPartitions int64 + indexOptions []CreateIndexOption + // is fast create collection isFast bool // fast creation with index @@ -83,6 +87,21 @@ func (opt *createCollectionOption) WithVarcharPK(varcharPK bool, maxLen int) *cr return opt } +func (opt *createCollectionOption) WithIndexOptions(indexOpts ...CreateIndexOption) *createCollectionOption { + opt.indexOptions = append(opt.indexOptions, indexOpts...) + return opt +} + +func (opt *createCollectionOption) WithProperty(key string, value any) *createCollectionOption { + opt.properties[key] = fmt.Sprintf("%v", value) + return opt +} + +func (opt *createCollectionOption) WithConsistencyLevel(cl entity.ConsistencyLevel) *createCollectionOption { + opt.consistencyLevel = cl + return opt +} + func (opt *createCollectionOption) Request() *milvuspb.CreateCollectionRequest { // fast create collection if opt.isFast { @@ -103,6 +122,7 @@ func (opt *createCollectionOption) Request() *milvuspb.CreateCollectionRequest { var schemaBytes []byte if opt.schema != nil { + opt.schema.WithName(opt.name) schemaProto := opt.schema.ProtoMessage() schemaBytes, _ = proto.Marshal(schemaProto) } @@ -144,6 +164,7 @@ func SimpleCreateCollectionOptions(name string, dim int64) *createCollectionOpti dim: dim, enabledDynamicSchema: true, consistencyLevel: entity.DefaultConsistencyLevel, + properties: make(map[string]string), isFast: true, metricType: entity.COSINE, @@ -157,6 +178,7 @@ func NewCreateCollectionOption(name string, collectionSchema *entity.Schema) *cr shardNum: 1, schema: collectionSchema, consistencyLevel: entity.DefaultConsistencyLevel, + properties: make(map[string]string), metricType: entity.COSINE, } @@ -263,3 +285,28 @@ func NewRenameCollectionOption(oldName, newName string) *renameCollectionOption newCollectionName: newName, } } + +type AlterCollectionOption interface { + Request() *milvuspb.AlterCollectionRequest +} + +type alterCollectionOption struct { + collectionName string + properties map[string]string +} + +func (opt *alterCollectionOption) WithProperty(key string, value any) *alterCollectionOption { + opt.properties[key] = fmt.Sprintf("%v", value) + return opt +} + +func (opt *alterCollectionOption) Request() *milvuspb.AlterCollectionRequest { + return &milvuspb.AlterCollectionRequest{ + CollectionName: opt.collectionName, + Properties: entity.MapKvPairs(opt.properties), + } +} + +func NewAlterCollectionOption(collection string) *alterCollectionOption { + return &alterCollectionOption{collectionName: collection, properties: make(map[string]string)} +} diff --git a/client/collection_test.go b/client/collection_test.go index 1c14c5267b776..cef0bd9967178 100644 --- a/client/collection_test.go +++ b/client/collection_test.go @@ -30,6 +30,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/client/v2/entity" + "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/util/merr" ) @@ -117,11 +118,20 @@ func (s *CollectionSuite) TestCreateCollectionOptions() { WithField(entity.NewField().WithName("int64").WithDataType(entity.FieldTypeInt64).WithIsPrimaryKey(true)). WithField(entity.NewField().WithName("vector").WithDim(128).WithDataType(entity.FieldTypeFloatVector)) - opt = NewCreateCollectionOption(collectionName, schema).WithShardNum(2) + opt = NewCreateCollectionOption(collectionName, schema). + WithShardNum(2). + WithConsistencyLevel(entity.ClEventually). + WithProperty(common.CollectionTTLConfigKey, 86400) req = opt.Request() s.Equal(collectionName, req.GetCollectionName()) s.EqualValues(2, req.GetShardsNum()) + s.EqualValues(commonpb.ConsistencyLevel_Eventually, req.GetConsistencyLevel()) + if s.Len(req.GetProperties(), 1) { + kv := req.GetProperties()[0] + s.Equal(common.CollectionTTLConfigKey, kv.GetKey()) + s.Equal("86400", kv.GetValue()) + } collSchema = &schemapb.CollectionSchema{} err = proto.Unmarshal(req.GetSchema(), collSchema) @@ -274,6 +284,37 @@ func (s *CollectionSuite) TestRenameCollection() { }) } +func (s *CollectionSuite) TestAlterCollection() { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + collName := fmt.Sprintf("test_collection_%s", s.randString(6)) + key := s.randString(6) + value := s.randString(6) + + s.Run("success", func() { + s.mock.EXPECT().AlterCollection(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, acr *milvuspb.AlterCollectionRequest) (*commonpb.Status, error) { + s.Equal(collName, acr.GetCollectionName()) + if s.Len(acr.GetProperties(), 1) { + item := acr.GetProperties()[0] + s.Equal(key, item.GetKey()) + s.Equal(value, item.GetValue()) + } + return merr.Success(), nil + }).Once() + + err := s.client.AlterCollection(ctx, NewAlterCollectionOption(collName).WithProperty(key, value)) + s.NoError(err) + }) + + s.Run("failure", func() { + s.mock.EXPECT().AlterCollection(mock.Anything, mock.Anything).Return(nil, merr.WrapErrServiceInternal("mocked")).Once() + + err := s.client.AlterCollection(ctx, NewAlterCollectionOption(collName).WithProperty(key, value)) + s.Error(err) + }) +} + func TestCollection(t *testing.T) { suite.Run(t, new(CollectionSuite)) } diff --git a/client/read_options.go b/client/read_options.go index 71a06e2fbff24..b3b056e85c319 100644 --- a/client/read_options.go +++ b/client/read_options.go @@ -142,6 +142,11 @@ func (opt *searchOption) WithPartitions(partitionNames ...string) *searchOption return opt } +func (opt *searchOption) WithGroupByField(groupByField string) *searchOption { + opt.request.groupByField = groupByField + return opt +} + func NewSearchOption(collectionName string, limit int, vectors []entity.Vector) *searchOption { return &searchOption{ collectionName: collectionName, diff --git a/client/read_test.go b/client/read_test.go index 4bf8b05ef3851..443713ed64f47 100644 --- a/client/read_test.go +++ b/client/read_test.go @@ -45,6 +45,7 @@ func (s *ReadSuite) TestSearch() { s.mock.EXPECT().Search(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, sr *milvuspb.SearchRequest) (*milvuspb.SearchResults, error) { s.Equal(collectionName, sr.GetCollectionName()) s.ElementsMatch([]string{partitionName}, sr.GetPartitionNames()) + // s.Equal(s) return &milvuspb.SearchResults{ Status: merr.Success(), @@ -71,7 +72,7 @@ func (s *ReadSuite) TestSearch() { entity.FloatVector(lo.RepeatBy(128, func(_ int) float32 { return rand.Float32() })), - }).WithPartitions(partitionName)) + }).WithPartitions(partitionName).WithGroupByField("group_by")) s.NoError(err) }) diff --git a/tests/go_client/testcases/collection_test.go b/tests/go_client/testcases/collection_test.go index 95f4ce9b18ada..116b2dc96e2e1 100644 --- a/tests/go_client/testcases/collection_test.go +++ b/tests/go_client/testcases/collection_test.go @@ -445,35 +445,22 @@ func TestCreateCollectionWithInvalidCollectionName(t *testing.T) { // connect ctx := hp.CreateContext(t, time.Second*common.DefaultTimeout) mc := createDefaultMilvusClient(ctx, t) - collName := common.GenRandomString(prefix, 6) // create collection and schema no name schema := genDefaultSchema() - err2 := mc.CreateCollection(ctx, client.NewCreateCollectionOption(collName, schema)) + err2 := mc.CreateCollection(ctx, client.NewCreateCollectionOption("", schema)) common.CheckErr(t, err2, false, "collection name should not be empty") // create collection with invalid schema name for _, invalidName := range common.GenInvalidNames() { log.Debug("TestCreateCollectionWithInvalidCollectionName", zap.String("collectionName", invalidName)) - // schema has invalid name - schema.WithName(invalidName) - err := mc.CreateCollection(ctx, client.NewCreateCollectionOption(collName, schema)) + err := mc.CreateCollection(ctx, client.NewCreateCollectionOption(invalidName, schema)) common.CheckErr(t, err, false, "collection name should not be empty", "the first character of a collection name must be an underscore or letter", "collection name can only contain numbers, letters and underscores", fmt.Sprintf("the length of a collection name must be less than %d characters", common.MaxCollectionNameLen)) - - // collection option has invalid name - schema.WithName(collName) - err2 := mc.CreateCollection(ctx, client.NewCreateCollectionOption(invalidName, schema)) - common.CheckErr(t, err2, false, "collection name matches schema name") } - - // collection name not equal to schema name - schema.WithName(collName) - err3 := mc.CreateCollection(ctx, client.NewCreateCollectionOption(common.GenRandomString("pre", 4), schema)) - common.CheckErr(t, err3, false, "collection name matches schema name") } // create collection missing pk field or vector field @@ -937,11 +924,10 @@ func TestCreateCollectionInvalid(t *testing.T) { vecField := entity.NewField().WithName("vec").WithDataType(entity.FieldTypeFloatVector).WithDim(8) mSchemaErrs := []mSchemaErr{ {schema: nil, errMsg: "schema does not contain vector field"}, - {schema: entity.NewSchema().WithField(vecField), errMsg: "collection name should not be empty"}, // no collection name - {schema: entity.NewSchema().WithName("aaa").WithField(vecField), errMsg: "primary key is not specified"}, // no pk field - {schema: entity.NewSchema().WithName("aaa").WithField(vecField).WithField(entity.NewField()), errMsg: "primary key is not specified"}, - {schema: entity.NewSchema().WithName("aaa").WithField(vecField).WithField(entity.NewField().WithIsPrimaryKey(true)), errMsg: "the data type of primary key should be Int64 or VarChar"}, - {schema: entity.NewSchema().WithName("aaa").WithField(vecField).WithField(entity.NewField().WithIsPrimaryKey(true).WithDataType(entity.FieldTypeVarChar)), errMsg: "field name should not be empty"}, + {schema: entity.NewSchema().WithField(vecField), errMsg: "primary key is not specified"}, // no pk field + {schema: entity.NewSchema().WithField(vecField).WithField(entity.NewField()), errMsg: "primary key is not specified"}, + {schema: entity.NewSchema().WithField(vecField).WithField(entity.NewField().WithIsPrimaryKey(true)), errMsg: "the data type of primary key should be Int64 or VarChar"}, + {schema: entity.NewSchema().WithField(vecField).WithField(entity.NewField().WithIsPrimaryKey(true).WithDataType(entity.FieldTypeVarChar)), errMsg: "field name should not be empty"}, } for _, mSchema := range mSchemaErrs { err := mc.CreateCollection(ctx, client.NewCreateCollectionOption(collName, mSchema.schema)) From f1dd55e0c0530751b1410cdc8da932a20fd2ae02 Mon Sep 17 00:00:00 2001 From: SimFG Date: Wed, 6 Nov 2024 14:48:26 +0800 Subject: [PATCH 10/40] enhance: improve rootcoord task scheduling policy (#37352) - issue: #30301 Signed-off-by: SimFG --- go.sum | 2 - internal/rootcoord/alter_alias_task.go | 8 + internal/rootcoord/alter_collection_task.go | 9 + internal/rootcoord/alter_database_task.go | 7 + internal/rootcoord/create_alias_task.go | 8 + internal/rootcoord/create_collection_task.go | 7 + internal/rootcoord/create_db_task.go | 4 + internal/rootcoord/create_partition_task.go | 9 + .../rootcoord/describe_collection_task.go | 9 + internal/rootcoord/describe_db_task.go | 7 + internal/rootcoord/drop_alias_task.go | 9 + internal/rootcoord/drop_collection_task.go | 4 + internal/rootcoord/drop_db_task.go | 6 +- internal/rootcoord/drop_partition_task.go | 9 + internal/rootcoord/has_collection_task.go | 7 + internal/rootcoord/has_partition_task.go | 9 + internal/rootcoord/list_db_task.go | 4 + internal/rootcoord/rename_collection_task.go | 6 + internal/rootcoord/root_coord.go | 9 + internal/rootcoord/scheduler.go | 59 ++- internal/rootcoord/scheduler_test.go | 93 +++++ internal/rootcoord/show_collection_task.go | 4 + internal/rootcoord/show_partition_task.go | 9 + internal/rootcoord/task.go | 90 +++++ internal/rootcoord/task_test.go | 341 ++++++++++++++++++ pkg/util/paramtable/component_param.go | 10 + 26 files changed, 728 insertions(+), 11 deletions(-) create mode 100644 internal/rootcoord/task_test.go diff --git a/go.sum b/go.sum index 66de8bb5875c1..98ffb403f2456 100644 --- a/go.sum +++ b/go.sum @@ -627,8 +627,6 @@ github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119 h1:9VXijWu github.com/milvus-io/cgosymbolizer v0.0.0-20240722103217-b7dee0e50119/go.mod h1:DvXTE/K/RtHehxU8/GtDs4vFtfw64jJ3PaCnFri8CRg= github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b h1:TfeY0NxYxZzUfIfYe5qYDBzt4ZYRqzUjTR6CvUzjat8= github.com/milvus-io/gorocksdb v0.0.0-20220624081344-8c5f4212846b/go.mod h1:iwW+9cWfIzzDseEBCCeDSN5SD16Tidvy8cwQ7ZY8Qj4= -github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240930043709-0c23514e4c34 h1:Fwxpg98128gfWRbQ1A3PMP9o2IfYZk7RSEy8rcoCWDA= -github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20240930043709-0c23514e4c34/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs= github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20241025031121-4d5c88b00cf7 h1:HwAitQk+V59QdYUwwVVYHTujd4QZrebg2Cc2hmcjhAg= github.com/milvus-io/milvus-proto/go-api/v2 v2.3.4-0.20241025031121-4d5c88b00cf7/go.mod h1:/6UT4zZl6awVeXLeE7UGDWZvXj3IWkRsh3mqsn0DiAs= github.com/milvus-io/pulsar-client-go v0.12.1 h1:O2JZp1tsYiO7C0MQ4hrUY/aJXnn2Gry6hpm7UodghmE= diff --git a/internal/rootcoord/alter_alias_task.go b/internal/rootcoord/alter_alias_task.go index 61abe8437c4f8..ac2f79acbf8a5 100644 --- a/internal/rootcoord/alter_alias_task.go +++ b/internal/rootcoord/alter_alias_task.go @@ -43,3 +43,11 @@ func (t *alterAliasTask) Execute(ctx context.Context) error { // alter alias is atomic enough. return t.core.meta.AlterAlias(ctx, t.Req.GetDbName(), t.Req.GetAlias(), t.Req.GetCollectionName(), t.GetTs()) } + +func (t *alterAliasTask) GetLockerKey() LockerKey { + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(t.Req.GetDbName(), false), + NewCollectionLockerKey(t.Req.GetCollectionName(), true), + ) +} diff --git a/internal/rootcoord/alter_collection_task.go b/internal/rootcoord/alter_collection_task.go index 8fd05e11cb8a8..9d7381c7f5668 100644 --- a/internal/rootcoord/alter_collection_task.go +++ b/internal/rootcoord/alter_collection_task.go @@ -124,3 +124,12 @@ func (a *alterCollectionTask) Execute(ctx context.Context) error { return redoTask.Execute(ctx) } + +func (a *alterCollectionTask) GetLockerKey() LockerKey { + collectionName := a.core.getRealCollectionName(a.ctx, a.Req.GetDbName(), a.Req.GetCollectionName()) + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(a.Req.GetDbName(), false), + NewCollectionLockerKey(collectionName, true), + ) +} diff --git a/internal/rootcoord/alter_database_task.go b/internal/rootcoord/alter_database_task.go index 292ba8bf3c8db..f87ec79d177dc 100644 --- a/internal/rootcoord/alter_database_task.go +++ b/internal/rootcoord/alter_database_task.go @@ -118,6 +118,13 @@ func (a *alterDatabaseTask) Execute(ctx context.Context) error { return redoTask.Execute(ctx) } +func (a *alterDatabaseTask) GetLockerKey() LockerKey { + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(a.Req.GetDbName(), true), + ) +} + func MergeProperties(oldProps []*commonpb.KeyValuePair, updatedProps []*commonpb.KeyValuePair) []*commonpb.KeyValuePair { props := make(map[string]string) for _, prop := range oldProps { diff --git a/internal/rootcoord/create_alias_task.go b/internal/rootcoord/create_alias_task.go index 0f3327a022f57..5675704703255 100644 --- a/internal/rootcoord/create_alias_task.go +++ b/internal/rootcoord/create_alias_task.go @@ -39,3 +39,11 @@ func (t *createAliasTask) Execute(ctx context.Context) error { // create alias is atomic enough. return t.core.meta.CreateAlias(ctx, t.Req.GetDbName(), t.Req.GetAlias(), t.Req.GetCollectionName(), t.GetTs()) } + +func (t *createAliasTask) GetLockerKey() LockerKey { + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(t.Req.GetDbName(), false), + NewCollectionLockerKey(t.Req.GetCollectionName(), true), + ) +} diff --git a/internal/rootcoord/create_collection_task.go b/internal/rootcoord/create_collection_task.go index 2621b02c6433f..9c83bc7018f51 100644 --- a/internal/rootcoord/create_collection_task.go +++ b/internal/rootcoord/create_collection_task.go @@ -658,3 +658,10 @@ func (t *createCollectionTask) Execute(ctx context.Context) error { return undoTask.Execute(ctx) } + +func (t *createCollectionTask) GetLockerKey() LockerKey { + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(t.Req.GetDbName(), true), + ) +} diff --git a/internal/rootcoord/create_db_task.go b/internal/rootcoord/create_db_task.go index 31de0c5f5afc5..35f1b67ddcf50 100644 --- a/internal/rootcoord/create_db_task.go +++ b/internal/rootcoord/create_db_task.go @@ -53,3 +53,7 @@ func (t *createDatabaseTask) Execute(ctx context.Context) error { db := model.NewDatabase(t.dbID, t.Req.GetDbName(), etcdpb.DatabaseState_DatabaseCreated, t.Req.GetProperties()) return t.core.meta.CreateDatabase(ctx, db, t.GetTs()) } + +func (t *createDatabaseTask) GetLockerKey() LockerKey { + return NewLockerKeyChain(NewClusterLockerKey(true)) +} diff --git a/internal/rootcoord/create_partition_task.go b/internal/rootcoord/create_partition_task.go index 76a5c7a718359..7e3fe097aa935 100644 --- a/internal/rootcoord/create_partition_task.go +++ b/internal/rootcoord/create_partition_task.go @@ -128,3 +128,12 @@ func (t *createPartitionTask) Execute(ctx context.Context) error { return undoTask.Execute(ctx) } + +func (t *createPartitionTask) GetLockerKey() LockerKey { + collectionName := t.core.getRealCollectionName(t.ctx, t.Req.GetDbName(), t.Req.GetCollectionName()) + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(t.Req.GetDbName(), false), + NewCollectionLockerKey(collectionName, true), + ) +} diff --git a/internal/rootcoord/describe_collection_task.go b/internal/rootcoord/describe_collection_task.go index 8a9da97a9f1ee..cc49a6cef5b88 100644 --- a/internal/rootcoord/describe_collection_task.go +++ b/internal/rootcoord/describe_collection_task.go @@ -53,3 +53,12 @@ func (t *describeCollectionTask) Execute(ctx context.Context) (err error) { t.Rsp = convertModelToDesc(coll, aliases, db.Name) return nil } + +func (t *describeCollectionTask) GetLockerKey() LockerKey { + collectionName := t.core.getRealCollectionName(t.ctx, t.Req.GetDbName(), t.Req.GetCollectionName()) + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(t.Req.GetDbName(), false), + NewCollectionLockerKey(collectionName, false), + ) +} diff --git a/internal/rootcoord/describe_db_task.go b/internal/rootcoord/describe_db_task.go index 603d1a46be90b..d711316b95238 100644 --- a/internal/rootcoord/describe_db_task.go +++ b/internal/rootcoord/describe_db_task.go @@ -55,3 +55,10 @@ func (t *describeDBTask) Execute(ctx context.Context) (err error) { } return nil } + +func (t *describeDBTask) GetLockerKey() LockerKey { + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(t.Req.GetDbName(), false), + ) +} diff --git a/internal/rootcoord/drop_alias_task.go b/internal/rootcoord/drop_alias_task.go index 28caceafc9eca..4f9fcbfe644a9 100644 --- a/internal/rootcoord/drop_alias_task.go +++ b/internal/rootcoord/drop_alias_task.go @@ -43,3 +43,12 @@ func (t *dropAliasTask) Execute(ctx context.Context) error { } return t.core.meta.DropAlias(ctx, t.Req.GetDbName(), t.Req.GetAlias(), t.GetTs()) } + +func (t *dropAliasTask) GetLockerKey() LockerKey { + collectionName := t.core.getRealCollectionName(t.ctx, t.Req.GetDbName(), t.Req.GetAlias()) + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(t.Req.GetDbName(), false), + NewCollectionLockerKey(collectionName, true), + ) +} diff --git a/internal/rootcoord/drop_collection_task.go b/internal/rootcoord/drop_collection_task.go index ce458e5cff059..4d443a1ad3ad7 100644 --- a/internal/rootcoord/drop_collection_task.go +++ b/internal/rootcoord/drop_collection_task.go @@ -119,3 +119,7 @@ func (t *dropCollectionTask) Execute(ctx context.Context) error { return redoTask.Execute(ctx) } + +func (t *dropCollectionTask) GetLockerKey() LockerKey { + return NewLockerKeyChain(NewClusterLockerKey(false), NewDatabaseLockerKey(t.Req.GetDbName(), true)) +} diff --git a/internal/rootcoord/drop_db_task.go b/internal/rootcoord/drop_db_task.go index 15096c4e3057c..bdc1cc035db32 100644 --- a/internal/rootcoord/drop_db_task.go +++ b/internal/rootcoord/drop_db_task.go @@ -47,7 +47,7 @@ func (t *dropDatabaseTask) Execute(ctx context.Context) error { databaseName: dbName, ts: ts, }) - redoTask.AddAsyncStep(&expireCacheStep{ + redoTask.AddSyncStep(&expireCacheStep{ baseStep: baseStep{core: t.core}, dbName: dbName, ts: ts, @@ -60,3 +60,7 @@ func (t *dropDatabaseTask) Execute(ctx context.Context) error { }) return redoTask.Execute(ctx) } + +func (t *dropDatabaseTask) GetLockerKey() LockerKey { + return NewLockerKeyChain(NewClusterLockerKey(true)) +} diff --git a/internal/rootcoord/drop_partition_task.go b/internal/rootcoord/drop_partition_task.go index cc45422db7f7e..c65c91f9c1099 100644 --- a/internal/rootcoord/drop_partition_task.go +++ b/internal/rootcoord/drop_partition_task.go @@ -111,3 +111,12 @@ func (t *dropPartitionTask) Execute(ctx context.Context) error { return redoTask.Execute(ctx) } + +func (t *dropPartitionTask) GetLockerKey() LockerKey { + collectionName := t.core.getRealCollectionName(t.ctx, t.Req.GetDbName(), t.Req.GetCollectionName()) + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(t.Req.GetDbName(), false), + NewCollectionLockerKey(collectionName, true), + ) +} diff --git a/internal/rootcoord/has_collection_task.go b/internal/rootcoord/has_collection_task.go index d9258a8f19607..52a4625ef1012 100644 --- a/internal/rootcoord/has_collection_task.go +++ b/internal/rootcoord/has_collection_task.go @@ -47,3 +47,10 @@ func (t *hasCollectionTask) Execute(ctx context.Context) error { t.Rsp.Value = err == nil return nil } + +func (t *hasCollectionTask) GetLockerKey() LockerKey { + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(t.Req.GetDbName(), false), + ) +} diff --git a/internal/rootcoord/has_partition_task.go b/internal/rootcoord/has_partition_task.go index 77ef717b47c84..1e9f32da740b4 100644 --- a/internal/rootcoord/has_partition_task.go +++ b/internal/rootcoord/has_partition_task.go @@ -57,3 +57,12 @@ func (t *hasPartitionTask) Execute(ctx context.Context) error { } return nil } + +func (t *hasPartitionTask) GetLockerKey() LockerKey { + collectionName := t.core.getRealCollectionName(t.ctx, t.Req.GetDbName(), t.Req.GetCollectionName()) + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(t.Req.GetDbName(), false), + NewCollectionLockerKey(collectionName, false), + ) +} diff --git a/internal/rootcoord/list_db_task.go b/internal/rootcoord/list_db_task.go index 1b4e81a79519d..847b5928c5747 100644 --- a/internal/rootcoord/list_db_task.go +++ b/internal/rootcoord/list_db_task.go @@ -124,3 +124,7 @@ func (t *listDatabaseTask) Execute(ctx context.Context) error { t.Resp.CreatedTimestamp = createdTimes return nil } + +func (t *listDatabaseTask) GetLockerKey() LockerKey { + return NewLockerKeyChain(NewClusterLockerKey(false)) +} diff --git a/internal/rootcoord/rename_collection_task.go b/internal/rootcoord/rename_collection_task.go index 50bdc61713693..fc915302cc912 100644 --- a/internal/rootcoord/rename_collection_task.go +++ b/internal/rootcoord/rename_collection_task.go @@ -42,3 +42,9 @@ func (t *renameCollectionTask) Execute(ctx context.Context) error { } return t.core.meta.RenameCollection(ctx, t.Req.GetDbName(), t.Req.GetOldName(), t.Req.GetNewDBName(), t.Req.GetNewName(), t.GetTs()) } + +func (t *renameCollectionTask) GetLockerKey() LockerKey { + return NewLockerKeyChain( + NewClusterLockerKey(true), + ) +} diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index ea267b577e590..ad6ef5fc597f2 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -1118,6 +1118,15 @@ func (c *Core) HasCollection(ctx context.Context, in *milvuspb.HasCollectionRequ return t.Rsp, nil } +// getRealCollectionName get origin collection name to avoid the alias name +func (c *Core) getRealCollectionName(ctx context.Context, db, collection string) string { + realName, err := c.meta.DescribeAlias(ctx, db, collection, 0) + if err != nil { + return collection + } + return realName +} + func (c *Core) describeCollection(ctx context.Context, in *milvuspb.DescribeCollectionRequest, allowUnavailable bool) (*model.Collection, error) { ts := getTravelTs(in) if in.GetCollectionName() != "" { diff --git a/internal/rootcoord/scheduler.go b/internal/rootcoord/scheduler.go index 1b3f8ab139e99..1f5e3d0e478ec 100644 --- a/internal/rootcoord/scheduler.go +++ b/internal/rootcoord/scheduler.go @@ -27,6 +27,7 @@ import ( "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/tso" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/lock" ) type IScheduler interface { @@ -48,21 +49,34 @@ type scheduler struct { lock sync.Mutex - minDdlTs atomic.Uint64 + minDdlTs atomic.Uint64 + clusterLock *lock.KeyLock[string] + databaseLock *lock.KeyLock[string] + collectionLock *lock.KeyLock[string] + lockMapping map[LockLevel]*lock.KeyLock[string] } func newScheduler(ctx context.Context, idAllocator allocator.Interface, tsoAllocator tso.Allocator) *scheduler { ctx1, cancel := context.WithCancel(ctx) // TODO n := 1024 * 10 - return &scheduler{ - ctx: ctx1, - cancel: cancel, - idAllocator: idAllocator, - tsoAllocator: tsoAllocator, - taskChan: make(chan task, n), - minDdlTs: *atomic.NewUint64(0), + s := &scheduler{ + ctx: ctx1, + cancel: cancel, + idAllocator: idAllocator, + tsoAllocator: tsoAllocator, + taskChan: make(chan task, n), + minDdlTs: *atomic.NewUint64(0), + clusterLock: lock.NewKeyLock[string](), + databaseLock: lock.NewKeyLock[string](), + collectionLock: lock.NewKeyLock[string](), } + s.lockMapping = map[LockLevel]*lock.KeyLock[string]{ + ClusterLock: s.clusterLock, + DatabaseLock: s.databaseLock, + CollectionLock: s.collectionLock, + } + return s } func (s *scheduler) Start() { @@ -147,6 +161,13 @@ func (s *scheduler) enqueue(task task) { } func (s *scheduler) AddTask(task task) error { + if Params.RootCoordCfg.UseLockScheduler.GetAsBool() { + lockKey := task.GetLockerKey() + if lockKey != nil { + return s.executeTaskWithLock(task, lockKey) + } + } + // make sure that setting ts and enqueue is atomic. s.lock.Lock() defer s.lock.Unlock() @@ -168,3 +189,25 @@ func (s *scheduler) GetMinDdlTs() Timestamp { func (s *scheduler) setMinDdlTs(ts Timestamp) { s.minDdlTs.Store(ts) } + +func (s *scheduler) executeTaskWithLock(task task, lockerKey LockerKey) error { + if lockerKey == nil { + if err := s.setID(task); err != nil { + return err + } + if err := s.setTs(task); err != nil { + return err + } + s.execute(task) + return nil + } + taskLock := s.lockMapping[lockerKey.Level()] + if lockerKey.IsWLock() { + taskLock.Lock(lockerKey.LockKey()) + defer taskLock.Unlock(lockerKey.LockKey()) + } else { + taskLock.RLock(lockerKey.LockKey()) + defer taskLock.RUnlock(lockerKey.LockKey()) + } + return s.executeTaskWithLock(task, lockerKey.Next()) +} diff --git a/internal/rootcoord/scheduler_test.go b/internal/rootcoord/scheduler_test.go index b48c4c8963bdb..99c0806baf008 100644 --- a/internal/rootcoord/scheduler_test.go +++ b/internal/rootcoord/scheduler_test.go @@ -20,13 +20,19 @@ import ( "context" "fmt" "math/rand" + "sync" "testing" "time" "github.com/cockroachdb/errors" "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" "go.uber.org/atomic" + "go.uber.org/zap" + "github.com/milvus-io/milvus/internal/allocator" + mocktso "github.com/milvus-io/milvus/internal/tso/mocks" + "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/paramtable" ) @@ -242,3 +248,90 @@ func Test_scheduler_updateDdlMinTsLoop(t *testing.T) { s.Stop() }) } + +type WithLockKeyTask struct { + baseTask + lockKey LockerKey + workDuration time.Duration + newTime time.Time + name string +} + +func NewWithLockKeyTask(lockKey LockerKey, duration time.Duration, name string) *WithLockKeyTask { + task := &WithLockKeyTask{ + baseTask: newBaseTask(context.Background(), nil), + lockKey: lockKey, + workDuration: duration, + newTime: time.Now(), + name: name, + } + return task +} + +func (t *WithLockKeyTask) GetLockerKey() LockerKey { + return t.lockKey +} + +func (t *WithLockKeyTask) Execute(ctx context.Context) error { + log.Info("execute task", zap.String("name", t.name), zap.Duration("duration", time.Since(t.newTime))) + time.Sleep(t.workDuration) + return nil +} + +func TestExecuteTaskWithLock(t *testing.T) { + paramtable.Init() + Params.Save(Params.RootCoordCfg.UseLockScheduler.Key, "true") + defer Params.Reset(Params.RootCoordCfg.UseLockScheduler.Key) + idMock := allocator.NewMockAllocator(t) + tsMock := mocktso.NewAllocator(t) + idMock.EXPECT().AllocOne().Return(1000, nil) + tsMock.EXPECT().GenerateTSO(mock.Anything).Return(10000, nil) + s := newScheduler(context.Background(), idMock, tsMock) + w := &sync.WaitGroup{} + w.Add(4) + { + go func() { + defer w.Done() + time.Sleep(1500 * time.Millisecond) + lockKey := NewLockerKeyChain(NewClusterLockerKey(false), NewDatabaseLockerKey("test", false)) + t1 := NewWithLockKeyTask(lockKey, time.Second*2, "t1-1") + err := s.AddTask(t1) + assert.NoError(t, err) + }() + } + { + go func() { + defer w.Done() + time.Sleep(1500 * time.Millisecond) + lockKey := NewLockerKeyChain(NewClusterLockerKey(false), NewDatabaseLockerKey("test", false)) + t1 := NewWithLockKeyTask(lockKey, time.Second*3, "t1-2") + err := s.AddTask(t1) + assert.NoError(t, err) + }() + } + { + go func() { + defer w.Done() + time.Sleep(500 * time.Millisecond) + lockKey := NewLockerKeyChain(NewClusterLockerKey(false), NewDatabaseLockerKey("test", true)) + t2 := NewWithLockKeyTask(lockKey, time.Second*2, "t2") + err := s.AddTask(t2) + assert.NoError(t, err) + }() + } + { + go func() { + defer w.Done() + lockKey := NewLockerKeyChain(NewClusterLockerKey(true)) + t3 := NewWithLockKeyTask(lockKey, time.Second, "t3") + err := s.AddTask(t3) + assert.NoError(t, err) + }() + } + + startTime := time.Now() + w.Wait() + delta := time.Since(startTime) + assert.True(t, delta > 6*time.Second) + assert.True(t, delta < 8*time.Second) +} diff --git a/internal/rootcoord/show_collection_task.go b/internal/rootcoord/show_collection_task.go index 090d4ada5b561..ab853266bbc36 100644 --- a/internal/rootcoord/show_collection_task.go +++ b/internal/rootcoord/show_collection_task.go @@ -147,3 +147,7 @@ func (t *showCollectionTask) Execute(ctx context.Context) error { } return nil } + +func (t *showCollectionTask) GetLockerKey() LockerKey { + return NewLockerKeyChain(NewClusterLockerKey(false), NewDatabaseLockerKey(t.Req.GetDbName(), false)) +} diff --git a/internal/rootcoord/show_partition_task.go b/internal/rootcoord/show_partition_task.go index 5e1f8214b4824..f023c60f60016 100644 --- a/internal/rootcoord/show_partition_task.go +++ b/internal/rootcoord/show_partition_task.go @@ -67,3 +67,12 @@ func (t *showPartitionTask) Execute(ctx context.Context) error { return nil } + +func (t *showPartitionTask) GetLockerKey() LockerKey { + collectionName := t.core.getRealCollectionName(t.ctx, t.Req.GetDbName(), t.Req.GetCollectionName()) + return NewLockerKeyChain( + NewClusterLockerKey(false), + NewDatabaseLockerKey(t.Req.GetDbName(), false), + NewCollectionLockerKey(collectionName, false), + ) +} diff --git a/internal/rootcoord/task.go b/internal/rootcoord/task.go index bbabf12bb6f57..4b0927b3bf77f 100644 --- a/internal/rootcoord/task.go +++ b/internal/rootcoord/task.go @@ -20,9 +20,27 @@ import ( "context" "time" + "go.uber.org/zap" + + "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/timerecord" ) +type LockLevel int + +const ( + ClusterLock LockLevel = iota + DatabaseLock + CollectionLock +) + +type LockerKey interface { + LockKey() string + Level() LockLevel + IsWLock() bool + Next() LockerKey +} + type task interface { GetCtx() context.Context SetCtx(context.Context) @@ -35,6 +53,7 @@ type task interface { WaitToFinish() error NotifyDone(err error) SetInQueueDuration() + GetLockerKey() LockerKey } type baseTask struct { @@ -101,3 +120,74 @@ func (b *baseTask) NotifyDone(err error) { func (b *baseTask) SetInQueueDuration() { b.queueDur = b.tr.ElapseSpan() } + +func (b *baseTask) GetLockerKey() LockerKey { + return nil +} + +type taskLockerKey struct { + key string + rw bool + level LockLevel + next LockerKey +} + +func (t *taskLockerKey) LockKey() string { + return t.key +} + +func (t *taskLockerKey) Level() LockLevel { + return t.level +} + +func (t *taskLockerKey) IsWLock() bool { + return t.rw +} + +func (t *taskLockerKey) Next() LockerKey { + return t.next +} + +func NewClusterLockerKey(rw bool) LockerKey { + return &taskLockerKey{ + key: "$", + rw: rw, + level: ClusterLock, + } +} + +func NewDatabaseLockerKey(db string, rw bool) LockerKey { + return &taskLockerKey{ + key: db, + rw: rw, + level: DatabaseLock, + } +} + +func NewCollectionLockerKey(collection string, rw bool) LockerKey { + return &taskLockerKey{ + key: collection, + rw: rw, + level: CollectionLock, + } +} + +func NewLockerKeyChain(lockerKeys ...LockerKey) LockerKey { + log.Info("NewLockerKeyChain", zap.Any("lockerKeys", len(lockerKeys))) + if len(lockerKeys) == 0 { + return nil + } + if lockerKeys[0] == nil || lockerKeys[0].Level() != ClusterLock { + log.Warn("Invalid locker key chain", zap.Stack("stack")) + return nil + } + + for i := 0; i < len(lockerKeys)-1; i++ { + if lockerKeys[i] == nil || lockerKeys[i].Level() >= lockerKeys[i+1].Level() { + log.Warn("Invalid locker key chain", zap.Stack("stack")) + return nil + } + lockerKeys[i].(*taskLockerKey).next = lockerKeys[i+1] + } + return lockerKeys[0] +} diff --git a/internal/rootcoord/task_test.go b/internal/rootcoord/task_test.go new file mode 100644 index 0000000000000..cf03907d3dcfe --- /dev/null +++ b/internal/rootcoord/task_test.go @@ -0,0 +1,341 @@ +/* + * 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 rootcoord + +import ( + "context" + "fmt" + "testing" + + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + + "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/proto/rootcoordpb" + mockrootcoord "github.com/milvus-io/milvus/internal/rootcoord/mocks" +) + +func TestLockerKey(t *testing.T) { + clusterLock := NewClusterLockerKey(true) + assert.Equal(t, clusterLock.IsWLock(), true) + assert.Equal(t, clusterLock.Level(), ClusterLock) + assert.Equal(t, clusterLock.LockKey(), "$") + + dbLock := NewDatabaseLockerKey("foo", true) + assert.Equal(t, dbLock.IsWLock(), true) + assert.Equal(t, dbLock.Level(), DatabaseLock) + assert.Equal(t, dbLock.LockKey(), "foo") + + collectionLock := NewCollectionLockerKey("foo", true) + assert.Equal(t, collectionLock.IsWLock(), true) + assert.Equal(t, collectionLock.Level(), CollectionLock) + assert.Equal(t, collectionLock.LockKey(), "foo") + + { + lockerChain := NewLockerKeyChain(nil) + assert.Nil(t, lockerChain) + } + + { + lockerChain := NewLockerKeyChain(dbLock) + assert.Nil(t, lockerChain) + } + + { + lockerChain := NewLockerKeyChain(clusterLock, collectionLock, dbLock) + assert.Nil(t, lockerChain) + } + + { + lockerChain := NewLockerKeyChain(clusterLock, dbLock, collectionLock) + assert.NotNil(t, lockerChain) + assert.Equal(t, lockerChain.Next(), dbLock) + assert.Equal(t, lockerChain.Next().Next(), collectionLock) + } +} + +func GetLockerKeyString(k LockerKey) string { + key := k.LockKey() + level := k.Level() + wLock := k.IsWLock() + if k.Next() == nil { + return fmt.Sprintf("%s-%d-%t", key, level, wLock) + } + return fmt.Sprintf("%s-%d-%t|%s", key, level, wLock, GetLockerKeyString(k.Next())) +} + +func TestGetLockerKey(t *testing.T) { + t.Run("alter alias task locker key", func(t *testing.T) { + tt := &alterAliasTask{ + Req: &milvuspb.AlterAliasRequest{ + DbName: "foo", + CollectionName: "bar", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false|bar-2-true") + }) + t.Run("alter collection task locker key", func(t *testing.T) { + metaMock := mockrootcoord.NewIMetaTable(t) + metaMock.EXPECT().DescribeAlias(mock.Anything, mock.Anything, mock.Anything, mock.Anything). + RunAndReturn(func(ctx context.Context, s string, s2 string, u uint64) (string, error) { + return s2, nil + }) + c := &Core{ + meta: metaMock, + } + tt := &alterCollectionTask{ + baseTask: baseTask{ + core: c, + }, + Req: &milvuspb.AlterCollectionRequest{ + DbName: "foo", + CollectionName: "bar", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false|bar-2-true") + }) + t.Run("alter database task locker key", func(t *testing.T) { + tt := &alterDatabaseTask{ + Req: &rootcoordpb.AlterDatabaseRequest{ + DbName: "foo", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-true") + }) + t.Run("create alias task locker key", func(t *testing.T) { + metaMock := mockrootcoord.NewIMetaTable(t) + c := &Core{ + meta: metaMock, + } + tt := &createAliasTask{ + baseTask: baseTask{ + core: c, + }, + Req: &milvuspb.CreateAliasRequest{ + DbName: "foo", + CollectionName: "bar", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false|bar-2-true") + }) + t.Run("create collection task locker key", func(t *testing.T) { + tt := &createCollectionTask{ + Req: &milvuspb.CreateCollectionRequest{ + DbName: "foo", + CollectionName: "bar", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-true") + }) + t.Run("create database task locker key", func(t *testing.T) { + tt := &createDatabaseTask{ + Req: &milvuspb.CreateDatabaseRequest{ + DbName: "foo", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-true") + }) + t.Run("create partition task locker key", func(t *testing.T) { + metaMock := mockrootcoord.NewIMetaTable(t) + metaMock.EXPECT().DescribeAlias(mock.Anything, mock.Anything, mock.Anything, mock.Anything). + RunAndReturn(func(ctx context.Context, s string, s2 string, u uint64) (string, error) { + return "real" + s2, nil + }) + c := &Core{ + meta: metaMock, + } + tt := &createPartitionTask{ + baseTask: baseTask{core: c}, + Req: &milvuspb.CreatePartitionRequest{ + DbName: "foo", + CollectionName: "bar", + PartitionName: "baz", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false|realbar-2-true") + }) + t.Run("describe collection task locker key", func(t *testing.T) { + metaMock := mockrootcoord.NewIMetaTable(t) + metaMock.EXPECT().DescribeAlias(mock.Anything, mock.Anything, mock.Anything, mock.Anything). + RunAndReturn(func(ctx context.Context, s string, s2 string, u uint64) (string, error) { + return "", errors.New("not found") + }) + c := &Core{ + meta: metaMock, + } + tt := &describeCollectionTask{ + baseTask: baseTask{core: c}, + Req: &milvuspb.DescribeCollectionRequest{ + DbName: "foo", + CollectionName: "bar", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false|bar-2-false") + }) + t.Run("describe database task locker key", func(t *testing.T) { + tt := &describeDBTask{ + Req: &rootcoordpb.DescribeDatabaseRequest{ + DbName: "foo", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false") + }) + t.Run("drop alias task locker key", func(t *testing.T) { + metaMock := mockrootcoord.NewIMetaTable(t) + metaMock.EXPECT().DescribeAlias(mock.Anything, mock.Anything, mock.Anything, mock.Anything). + RunAndReturn(func(ctx context.Context, s string, s2 string, u uint64) (string, error) { + return "real" + s2, nil + }) + c := &Core{ + meta: metaMock, + } + tt := &dropAliasTask{ + baseTask: baseTask{core: c}, + Req: &milvuspb.DropAliasRequest{ + DbName: "foo", + Alias: "bar", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false|realbar-2-true") + }) + t.Run("drop collection task locker key", func(t *testing.T) { + tt := &dropCollectionTask{ + Req: &milvuspb.DropCollectionRequest{ + DbName: "foo", + CollectionName: "bar", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-true") + }) + t.Run("drop database task locker key", func(t *testing.T) { + tt := &dropDatabaseTask{ + Req: &milvuspb.DropDatabaseRequest{ + DbName: "foo", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-true") + }) + t.Run("drop partition task locker key", func(t *testing.T) { + metaMock := mockrootcoord.NewIMetaTable(t) + metaMock.EXPECT().DescribeAlias(mock.Anything, mock.Anything, mock.Anything, mock.Anything). + RunAndReturn(func(ctx context.Context, s string, s2 string, u uint64) (string, error) { + return "real" + s2, nil + }) + c := &Core{ + meta: metaMock, + } + tt := &dropPartitionTask{ + baseTask: baseTask{core: c}, + Req: &milvuspb.DropPartitionRequest{ + DbName: "foo", + CollectionName: "bar", + PartitionName: "baz", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false|realbar-2-true") + }) + t.Run("has collection task locker key", func(t *testing.T) { + tt := &hasCollectionTask{ + Req: &milvuspb.HasCollectionRequest{ + DbName: "foo", + CollectionName: "bar", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false") + }) + t.Run("has partition task locker key", func(t *testing.T) { + metaMock := mockrootcoord.NewIMetaTable(t) + metaMock.EXPECT().DescribeAlias(mock.Anything, mock.Anything, mock.Anything, mock.Anything). + RunAndReturn(func(ctx context.Context, s string, s2 string, u uint64) (string, error) { + return "real" + s2, nil + }) + c := &Core{ + meta: metaMock, + } + tt := &hasPartitionTask{ + baseTask: baseTask{core: c}, + Req: &milvuspb.HasPartitionRequest{ + DbName: "foo", + CollectionName: "bar", + PartitionName: "baz", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false|realbar-2-false") + }) + t.Run("list db task locker key", func(t *testing.T) { + tt := &listDatabaseTask{} + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false") + }) + t.Run("rename collection task locker key", func(t *testing.T) { + tt := &renameCollectionTask{ + Req: &milvuspb.RenameCollectionRequest{ + DbName: "foo", + OldName: "bar", + NewName: "baz", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-true") + }) + t.Run("show collection task locker key", func(t *testing.T) { + tt := &showCollectionTask{ + Req: &milvuspb.ShowCollectionsRequest{ + DbName: "foo", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false") + }) + t.Run("show partition task locker key", func(t *testing.T) { + metaMock := mockrootcoord.NewIMetaTable(t) + metaMock.EXPECT().DescribeAlias(mock.Anything, mock.Anything, mock.Anything, mock.Anything). + RunAndReturn(func(ctx context.Context, s string, s2 string, u uint64) (string, error) { + return "real" + s2, nil + }) + c := &Core{ + meta: metaMock, + } + tt := &showPartitionTask{ + baseTask: baseTask{core: c}, + Req: &milvuspb.ShowPartitionsRequest{ + DbName: "foo", + CollectionName: "bar", + }, + } + key := tt.GetLockerKey() + assert.Equal(t, GetLockerKeyString(key), "$-0-false|foo-1-false|realbar-2-false") + }) +} diff --git a/pkg/util/paramtable/component_param.go b/pkg/util/paramtable/component_param.go index 9e0ff8bdc30a4..d7137f0061184 100644 --- a/pkg/util/paramtable/component_param.go +++ b/pkg/util/paramtable/component_param.go @@ -1117,6 +1117,7 @@ type rootCoordConfig struct { MaxDatabaseNum ParamItem `refreshable:"false"` MaxGeneralCapacity ParamItem `refreshable:"true"` GracefulStopTimeout ParamItem `refreshable:"true"` + UseLockScheduler ParamItem `refreshable:"true"` } func (p *rootCoordConfig) init(base *BaseTable) { @@ -1191,6 +1192,15 @@ Segments with smaller size than this parameter will not be indexed, and will be Export: true, } p.GracefulStopTimeout.Init(base.mgr) + + p.UseLockScheduler = ParamItem{ + Key: "rootCoord.useLockScheduler", + Version: "2.4.15", + DefaultValue: "false", + Doc: "use lock to schedule the task", + Export: false, + } + p.UseLockScheduler.Init(base.mgr) } // ///////////////////////////////////////////////////////////////////////////// From 8714774305ff4a5b9e5bfc739a1e6654da728abb Mon Sep 17 00:00:00 2001 From: wei liu Date: Wed, 6 Nov 2024 15:10:25 +0800 Subject: [PATCH 11/40] fix: search/query failed due to segment not loaded (#37403) issue: #36970 cause release segment and balance channel may happen at same time, and before new delegator become serviceable, if release segment exeuctes on new delegator, and search/query comes on old delegator, then release segment and query segment happens in parallel, if release segment execute first in worker, then search/query will got a SegmentNodeLoaded error. This PR add serviceable filter on delegator, then all load/release segment operation will happens on serviceable delegator. Signed-off-by: Wei Liu --- internal/querycoordv2/meta/leader_view_manager.go | 6 ++++++ internal/querycoordv2/task/executor.go | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/internal/querycoordv2/meta/leader_view_manager.go b/internal/querycoordv2/meta/leader_view_manager.go index 963e115b69506..fdb56671e5a73 100644 --- a/internal/querycoordv2/meta/leader_view_manager.go +++ b/internal/querycoordv2/meta/leader_view_manager.go @@ -109,6 +109,12 @@ func WithSegment2LeaderView(segmentID int64, isGrowing bool) LeaderViewFilter { }) } +func WithServiceable() LeaderViewFilter { + return lvFilterFunc(func(view *LeaderView) bool { + return view.UnServiceableError == nil + }) +} + type LeaderView struct { ID int64 CollectionID int64 diff --git a/internal/querycoordv2/task/executor.go b/internal/querycoordv2/task/executor.go index 375a290942ff1..764a7afd4f0fc 100644 --- a/internal/querycoordv2/task/executor.go +++ b/internal/querycoordv2/task/executor.go @@ -282,7 +282,7 @@ func (ex *Executor) releaseSegment(task *SegmentTask, step int) { dstNode = action.Node() req.NeedTransfer = false } else { - view := ex.dist.LeaderViewManager.GetLatestShardLeaderByFilter(meta.WithReplica2LeaderView(replica), meta.WithChannelName2LeaderView(action.Shard)) + view := ex.dist.LeaderViewManager.GetLatestShardLeaderByFilter(meta.WithReplica2LeaderView(replica), meta.WithChannelName2LeaderView(action.Shard), meta.WithServiceable()) if view == nil { msg := "no shard leader for the segment to execute releasing" err := merr.WrapErrChannelNotFound(task.Shard(), "shard delegator not found") From d67853fa8907a027d6698c133a1fbc62a35091a5 Mon Sep 17 00:00:00 2001 From: aoiasd <45024769+aoiasd@users.noreply.github.com> Date: Wed, 6 Nov 2024 17:48:24 +0800 Subject: [PATCH 12/40] feat: Tokenizer support build with params and clone for concurrency (#37048) relate: https://github.com/milvus-io/milvus/issues/35853 https://github.com/milvus-io/milvus/issues/36751 --------- Signed-off-by: aoiasd --- internal/core/src/common/FieldMeta.cpp | 13 +- internal/core/src/common/FieldMeta.h | 2 +- internal/core/src/index/TextMatchIndex.cpp | 26 +- internal/core/src/index/TextMatchIndex.h | 25 +- internal/core/src/indexbuilder/index_c.cpp | 2 +- .../src/segcore/ChunkedSegmentSealedImpl.cpp | 8 +- .../core/src/segcore/SegmentGrowingImpl.cpp | 4 +- .../core/src/segcore/SegmentSealedImpl.cpp | 8 +- internal/core/src/segcore/tokenizer_c.cpp | 17 +- internal/core/src/segcore/tokenizer_c.h | 5 +- .../tantivy/tantivy-binding/Cargo.lock | 6 +- .../tantivy/tantivy-binding/Cargo.toml | 1 + .../tantivy-binding/include/tantivy-binding.h | 10 +- .../tantivy/tantivy-binding/src/error.rs | 40 +++ .../tantivy-binding/src/index_reader_text.rs | 4 +- .../src/index_reader_text_c.rs | 27 +- .../src/index_writer_text_c.rs | 19 +- .../tantivy/tantivy-binding/src/lib.rs | 2 + .../tantivy/tantivy-binding/src/tokenizer.rs | 260 ++++++++++++++++-- .../tantivy-binding/src/tokenizer_c.rs | 29 +- .../tantivy-binding/src/tokenizer_filter.rs | 154 +++++++++++ .../tantivy/tantivy-binding/src/util.rs | 18 ++ .../core/thirdparty/tantivy/tantivy-wrapper.h | 18 +- internal/core/thirdparty/tantivy/tokenizer.h | 16 +- internal/core/unittest/test_c_tokenizer.cpp | 7 +- internal/core/unittest/test_text_match.cpp | 16 +- internal/proxy/task_query.go | 1 - internal/proxy/task_search.go | 1 - internal/util/ctokenizer/c_tokenizer.go | 9 + .../util/ctokenizer/c_tokenizer_factory.go | 11 +- internal/util/ctokenizer/c_tokenizer_test.go | 5 +- .../ctokenizer/text_schema_validator_test.go | 4 +- internal/util/function/bm25_function.go | 19 +- internal/util/tokenizerapi/tokenizer.go | 1 + tests/python_client/common/common_func.py | 2 +- .../testcases/test_full_text_search.py | 51 ++-- tests/python_client/testcases/test_query.py | 40 ++- tests/python_client/testcases/test_search.py | 2 +- .../testcases/test_vector_operations.py | 4 +- 39 files changed, 663 insertions(+), 224 deletions(-) create mode 100644 internal/core/thirdparty/tantivy/tantivy-binding/src/error.rs create mode 100644 internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer_filter.rs diff --git a/internal/core/src/common/FieldMeta.cpp b/internal/core/src/common/FieldMeta.cpp index ca55d45e67ac2..1506ddd507a92 100644 --- a/internal/core/src/common/FieldMeta.cpp +++ b/internal/core/src/common/FieldMeta.cpp @@ -22,18 +22,9 @@ TokenizerParams ParseTokenizerParams(const TypeParams& params) { auto iter = params.find("tokenizer_params"); if (iter == params.end()) { - return {}; + return "{}"; } - nlohmann::json j = nlohmann::json::parse(iter->second); - std::map ret; - for (const auto& [k, v] : j.items()) { - try { - ret[k] = v.get(); - } catch (std::exception& e) { - ret[k] = v.dump(); - } - } - return ret; + return iter->second; } bool diff --git a/internal/core/src/common/FieldMeta.h b/internal/core/src/common/FieldMeta.h index ed040902a54d6..0d920452bf10d 100644 --- a/internal/core/src/common/FieldMeta.h +++ b/internal/core/src/common/FieldMeta.h @@ -25,7 +25,7 @@ namespace milvus { using TypeParams = std::map; -using TokenizerParams = std::map; +using TokenizerParams = std::string; TokenizerParams ParseTokenizerParams(const TypeParams& params); diff --git a/internal/core/src/index/TextMatchIndex.cpp b/internal/core/src/index/TextMatchIndex.cpp index f21e5b319e006..8b12e02280b82 100644 --- a/internal/core/src/index/TextMatchIndex.cpp +++ b/internal/core/src/index/TextMatchIndex.cpp @@ -19,10 +19,9 @@ namespace milvus::index { constexpr const char* TMP_TEXT_LOG_PREFIX = "/tmp/milvus/text-log/"; -TextMatchIndex::TextMatchIndex( - int64_t commit_interval_in_ms, - const char* tokenizer_name, - const std::map& tokenizer_params) +TextMatchIndex::TextMatchIndex(int64_t commit_interval_in_ms, + const char* tokenizer_name, + const char* tokenizer_params) : commit_interval_in_ms_(commit_interval_in_ms), last_commit_time_(stdclock::now()) { d_type_ = TantivyDataType::Text; @@ -31,10 +30,9 @@ TextMatchIndex::TextMatchIndex( field_name.c_str(), true, "", tokenizer_name, tokenizer_params); } -TextMatchIndex::TextMatchIndex( - const std::string& path, - const char* tokenizer_name, - const std::map& tokenizer_params) +TextMatchIndex::TextMatchIndex(const std::string& path, + const char* tokenizer_name, + const char* tokenizer_params) : commit_interval_in_ms_(std::numeric_limits::max()), last_commit_time_(stdclock::now()) { path_ = path; @@ -47,10 +45,9 @@ TextMatchIndex::TextMatchIndex( tokenizer_params); } -TextMatchIndex::TextMatchIndex( - const storage::FileManagerContext& ctx, - const char* tokenizer_name, - const std::map& tokenizer_params) +TextMatchIndex::TextMatchIndex(const storage::FileManagerContext& ctx, + const char* tokenizer_name, + const char* tokenizer_params) : commit_interval_in_ms_(std::numeric_limits::max()), last_commit_time_(stdclock::now()) { schema_ = ctx.fieldDataMeta.field_schema; @@ -174,9 +171,8 @@ TextMatchIndex::CreateReader() { } void -TextMatchIndex::RegisterTokenizer( - const char* tokenizer_name, - const std::map& tokenizer_params) { +TextMatchIndex::RegisterTokenizer(const char* tokenizer_name, + const char* tokenizer_params) { wrapper_->register_tokenizer(tokenizer_name, tokenizer_params); } diff --git a/internal/core/src/index/TextMatchIndex.h b/internal/core/src/index/TextMatchIndex.h index 570668a0304e0..248bb63932712 100644 --- a/internal/core/src/index/TextMatchIndex.h +++ b/internal/core/src/index/TextMatchIndex.h @@ -22,20 +22,17 @@ using stdclock = std::chrono::high_resolution_clock; class TextMatchIndex : public InvertedIndexTantivy { public: // for growing segment. - explicit TextMatchIndex( - int64_t commit_interval_in_ms, - const char* tokenizer_name, - const std::map& tokenizer_params); + explicit TextMatchIndex(int64_t commit_interval_in_ms, + const char* tokenizer_name, + const char* tokenizer_params); // for sealed segment. - explicit TextMatchIndex( - const std::string& path, - const char* tokenizer_name, - const std::map& tokenizer_params); + explicit TextMatchIndex(const std::string& path, + const char* tokenizer_name, + const char* tokenizer_params); // for building index. - explicit TextMatchIndex( - const storage::FileManagerContext& ctx, - const char* tokenizer_name, - const std::map& tokenizer_params); + explicit TextMatchIndex(const storage::FileManagerContext& ctx, + const char* tokenizer_name, + const char* tokenizer_params); // for loading index explicit TextMatchIndex(const storage::FileManagerContext& ctx); @@ -67,9 +64,7 @@ class TextMatchIndex : public InvertedIndexTantivy { CreateReader(); void - RegisterTokenizer( - const char* tokenizer_name, - const std::map& tokenizer_params); + RegisterTokenizer(const char* tokenizer_name, const char* tokenizer_params); TargetBitmap MatchQuery(const std::string& query); diff --git a/internal/core/src/indexbuilder/index_c.cpp b/internal/core/src/indexbuilder/index_c.cpp index f4f4613c72259..d9029955bcaf3 100644 --- a/internal/core/src/indexbuilder/index_c.cpp +++ b/internal/core/src/indexbuilder/index_c.cpp @@ -284,7 +284,7 @@ BuildTextIndex(CBinarySet* c_binary_set, auto index = std::make_unique( fileManagerContext, "milvus_tokenizer", - field_schema.get_tokenizer_params()); + field_schema.get_tokenizer_params().c_str()); index->Build(config); auto binary = std::make_unique(index->Upload(config)); diff --git a/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp b/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp index c7d64161ea9e0..b3719ed38df2b 100644 --- a/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp +++ b/internal/core/src/segcore/ChunkedSegmentSealedImpl.cpp @@ -1511,13 +1511,13 @@ ChunkedSegmentSealedImpl::CreateTextIndex(FieldId field_id) { index = std::make_unique( std::numeric_limits::max(), "milvus_tokenizer", - field_meta.get_tokenizer_params()); + field_meta.get_tokenizer_params().c_str()); } else { // build text index using mmap. index = std::make_unique( cfg.GetMmapPath(), "milvus_tokenizer", - field_meta.get_tokenizer_params()); + field_meta.get_tokenizer_params().c_str()); } { @@ -1567,7 +1567,7 @@ ChunkedSegmentSealedImpl::CreateTextIndex(FieldId field_id) { index->Reload(); index->RegisterTokenizer("milvus_tokenizer", - field_meta.get_tokenizer_params()); + field_meta.get_tokenizer_params().c_str()); text_indexes_[field_id] = std::move(index); } @@ -1578,7 +1578,7 @@ ChunkedSegmentSealedImpl::LoadTextIndex( std::unique_lock lck(mutex_); const auto& field_meta = schema_->operator[](field_id); index->RegisterTokenizer("milvus_tokenizer", - field_meta.get_tokenizer_params()); + field_meta.get_tokenizer_params().c_str()); text_indexes_[field_id] = std::move(index); } diff --git a/internal/core/src/segcore/SegmentGrowingImpl.cpp b/internal/core/src/segcore/SegmentGrowingImpl.cpp index b90953c858066..bacfdab588774 100644 --- a/internal/core/src/segcore/SegmentGrowingImpl.cpp +++ b/internal/core/src/segcore/SegmentGrowingImpl.cpp @@ -859,11 +859,11 @@ SegmentGrowingImpl::CreateTextIndex(FieldId field_id) { "cannot create text index on non-string type"); // todo: make this(200) configurable. auto index = std::make_unique( - 200, "milvus_tokenizer", field_meta.get_tokenizer_params()); + 200, "milvus_tokenizer", field_meta.get_tokenizer_params().c_str()); index->Commit(); index->CreateReader(); index->RegisterTokenizer("milvus_tokenizer", - field_meta.get_tokenizer_params()); + field_meta.get_tokenizer_params().c_str()); text_indexes_[field_id] = std::move(index); } diff --git a/internal/core/src/segcore/SegmentSealedImpl.cpp b/internal/core/src/segcore/SegmentSealedImpl.cpp index 2ffb9d9e8ae82..be2b2635e2917 100644 --- a/internal/core/src/segcore/SegmentSealedImpl.cpp +++ b/internal/core/src/segcore/SegmentSealedImpl.cpp @@ -2014,13 +2014,13 @@ SegmentSealedImpl::CreateTextIndex(FieldId field_id) { index = std::make_unique( std::numeric_limits::max(), "milvus_tokenizer", - field_meta.get_tokenizer_params()); + field_meta.get_tokenizer_params().c_str()); } else { // build text index using mmap. index = std::make_unique( cfg.GetMmapPath(), "milvus_tokenizer", - field_meta.get_tokenizer_params()); + field_meta.get_tokenizer_params().c_str()); } { @@ -2069,7 +2069,7 @@ SegmentSealedImpl::CreateTextIndex(FieldId field_id) { index->Reload(); index->RegisterTokenizer("milvus_tokenizer", - field_meta.get_tokenizer_params()); + field_meta.get_tokenizer_params().c_str()); text_indexes_[field_id] = std::move(index); } @@ -2080,7 +2080,7 @@ SegmentSealedImpl::LoadTextIndex(FieldId field_id, std::unique_lock lck(mutex_); const auto& field_meta = schema_->operator[](field_id); index->RegisterTokenizer("milvus_tokenizer", - field_meta.get_tokenizer_params()); + field_meta.get_tokenizer_params().c_str()); text_indexes_[field_id] = std::move(index); } diff --git a/internal/core/src/segcore/tokenizer_c.cpp b/internal/core/src/segcore/tokenizer_c.cpp index 85a3cc39d4f55..781192f269c63 100644 --- a/internal/core/src/segcore/tokenizer_c.cpp +++ b/internal/core/src/segcore/tokenizer_c.cpp @@ -10,6 +10,7 @@ // or implied. See the License for the specific language governing permissions and limitations under the License #include "segcore/tokenizer_c.h" +#include #include "common/FieldMeta.h" #include "common/protobuf_utils.h" #include "pb/schema.pb.h" @@ -19,10 +20,9 @@ using Map = std::map; CStatus -create_tokenizer(CMap m, CTokenizer* tokenizer) { +create_tokenizer(const char* params, CTokenizer* tokenizer) { try { - auto mm = reinterpret_cast(m); - auto impl = std::make_unique(*mm); + auto impl = std::make_unique(params); *tokenizer = impl.release(); return milvus::SuccessCStatus(); } catch (std::exception& e) { @@ -30,6 +30,17 @@ create_tokenizer(CMap m, CTokenizer* tokenizer) { } } +CStatus +clone_tokenizer(CTokenizer* tokenizer, CTokenizer* rst) { + try { + auto impl = reinterpret_cast(*tokenizer); + *rst = impl->Clone().release(); + return milvus::SuccessCStatus(); + } catch (std::exception& e) { + return milvus::FailureCStatus(&e); + } +} + void free_tokenizer(CTokenizer tokenizer) { auto impl = reinterpret_cast(tokenizer); diff --git a/internal/core/src/segcore/tokenizer_c.h b/internal/core/src/segcore/tokenizer_c.h index 901689c5337ef..422449c0cab50 100644 --- a/internal/core/src/segcore/tokenizer_c.h +++ b/internal/core/src/segcore/tokenizer_c.h @@ -24,7 +24,10 @@ extern "C" { typedef void* CTokenizer; CStatus -create_tokenizer(CMap m, CTokenizer* tokenizer); +create_tokenizer(const char* params, CTokenizer* tokenizer); + +CStatus +clone_tokenizer(CTokenizer* tokenizer, CTokenizer* rst); void free_tokenizer(CTokenizer tokenizer); diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/Cargo.lock b/internal/core/thirdparty/tantivy/tantivy-binding/Cargo.lock index 47872ac8120b8..a72e056522e8d 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/Cargo.lock +++ b/internal/core/thirdparty/tantivy/tantivy-binding/Cargo.lock @@ -1021,11 +1021,12 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.115" +version = "1.0.128" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "12dc5c46daa8e9fdf4f5e71b6cf9a53f2487da0e86e55808e2d35539666497dd" +checksum = "6ff5456707a1de34e7e37f2a6fd3d3f808c318259cbd01ab6377795054b483d8" dependencies = [ "itoa", + "memchr", "ryu", "serde", ] @@ -1166,6 +1167,7 @@ dependencies = [ "libc", "log", "scopeguard", + "serde_json", "tantivy", "tantivy-jieba", "zstd-sys", diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/Cargo.toml b/internal/core/thirdparty/tantivy/tantivy-binding/Cargo.toml index 3bf9759d470f8..6b26b3ab67e7e 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/Cargo.toml +++ b/internal/core/thirdparty/tantivy/tantivy-binding/Cargo.toml @@ -15,6 +15,7 @@ env_logger = "0.11.3" log = "0.4.21" tantivy-jieba = "0.10.0" lazy_static = "1.4.0" +serde_json = "1.0.128" [build-dependencies] cbindgen = "0.26.0" diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/include/tantivy-binding.h b/internal/core/thirdparty/tantivy/tantivy-binding/include/tantivy-binding.h index c443ec7fc7a0e..5c494f215ece9 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/include/tantivy-binding.h +++ b/internal/core/thirdparty/tantivy/tantivy-binding/include/tantivy-binding.h @@ -88,7 +88,9 @@ RustArray tantivy_regex_query(void *ptr, const char *pattern); RustArray tantivy_match_query(void *ptr, const char *query); -void tantivy_register_tokenizer(void *ptr, const char *tokenizer_name, void *tokenizer_params); +void tantivy_register_tokenizer(void *ptr, + const char *tokenizer_name, + const char *tokenizer_params); void *tantivy_create_index(const char *field_name, TantivyDataType data_type, @@ -142,7 +144,7 @@ void tantivy_index_add_multi_keywords(void *ptr, void *tantivy_create_text_writer(const char *field_name, const char *path, const char *tokenizer_name, - void *tokenizer_params, + const char *tokenizer_params, uintptr_t num_threads, uintptr_t overall_memory_budget_in_bytes, bool in_ram); @@ -157,7 +159,9 @@ bool tantivy_token_stream_advance(void *token_stream); const char *tantivy_token_stream_get_token(void *token_stream); -void *tantivy_create_tokenizer(void *tokenizer_params); +void *tantivy_create_tokenizer(const char *tokenizer_params); + +void *tantivy_clone_tokenizer(void *ptr); void tantivy_free_tokenizer(void *tokenizer); diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/src/error.rs b/internal/core/thirdparty/tantivy/tantivy-binding/src/error.rs new file mode 100644 index 0000000000000..d3ddb125cc8fe --- /dev/null +++ b/internal/core/thirdparty/tantivy/tantivy-binding/src/error.rs @@ -0,0 +1,40 @@ +use serde_json as json; + +#[derive(Debug)] +pub struct TantivyError{ + reason: String, +} + +impl TantivyError{ + fn new(reason:String) -> Self{ + TantivyError{reason:reason} + } + + pub fn reason(&self) -> String{ + return self.reason.clone() + } +} + +impl From<&str> for TantivyError{ + fn from(value: &str) -> Self { + Self::new(value.to_string()) + } +} + +impl From for TantivyError{ + fn from(value: String) -> Self { + Self::new(value) + } +} + +impl From for TantivyError{ + fn from(value: json::Error) -> Self { + Self::new(value.to_string()) + } +} + +impl ToString for TantivyError{ + fn to_string(&self) -> String { + return self.reason() + } +} \ No newline at end of file diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text.rs b/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text.rs index 654346fc868c4..ef6e2d6cb6552 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text.rs +++ b/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text.rs @@ -4,7 +4,7 @@ use tantivy::{ Term, }; -use crate::{index_reader::IndexReaderWrapper, tokenizer::default_tokenizer}; +use crate::{index_reader::IndexReaderWrapper, tokenizer::standard_analyzer}; impl IndexReaderWrapper { // split the query string into multiple tokens using index's default tokenizer, @@ -14,7 +14,7 @@ impl IndexReaderWrapper { let mut tokenizer = self .index .tokenizer_for_field(self.field) - .unwrap_or(default_tokenizer()) + .unwrap_or(standard_analyzer(vec![])) .clone(); let mut token_stream = tokenizer.token_stream(q); let mut terms: Vec = Vec::new(); diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text_c.rs b/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text_c.rs index eb0653c90357b..fc7e00e7672e7 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text_c.rs +++ b/internal/core/thirdparty/tantivy/tantivy-binding/src/index_reader_text_c.rs @@ -1,8 +1,14 @@ -use std::{collections::HashMap, ffi::CStr}; +use std::{ffi::CStr}; use libc::{c_char, c_void}; -use crate::{array::RustArray, index_reader::IndexReaderWrapper, tokenizer::create_tokenizer}; +use crate::{ + array::RustArray, + string_c::c_str_to_str, + index_reader::IndexReaderWrapper, + tokenizer::create_tokenizer, + log::init_log, +}; #[no_mangle] pub extern "C" fn tantivy_match_query(ptr: *mut c_void, query: *const c_char) -> RustArray { @@ -18,23 +24,22 @@ pub extern "C" fn tantivy_match_query(ptr: *mut c_void, query: *const c_char) -> pub extern "C" fn tantivy_register_tokenizer( ptr: *mut c_void, tokenizer_name: *const c_char, - tokenizer_params: *mut c_void, + tokenizer_params: *const c_char, ) { + init_log(); let real = ptr as *mut IndexReaderWrapper; let tokenizer_name_str = unsafe { CStr::from_ptr(tokenizer_name) }; - let analyzer = unsafe { - let m = tokenizer_params as *const HashMap; - create_tokenizer(&(*m)) - }; + let params = unsafe{c_str_to_str(tokenizer_params).to_string()}; + let analyzer = create_tokenizer(¶ms); match analyzer { - Some(text_analyzer) => unsafe { + Ok(text_analyzer) => unsafe { (*real).register_tokenizer( String::from(tokenizer_name_str.to_str().unwrap()), text_analyzer, ); }, - None => { - panic!("unsupported tokenizer"); - } + Err(err) => { + panic!("create tokenizer failed with error: {} param: {}", err.to_string(), params); + }, } } diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/src/index_writer_text_c.rs b/internal/core/thirdparty/tantivy/tantivy-binding/src/index_writer_text_c.rs index 1ca70ac232c9b..5443fe14afb88 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/src/index_writer_text_c.rs +++ b/internal/core/thirdparty/tantivy/tantivy-binding/src/index_writer_text_c.rs @@ -1,4 +1,3 @@ -use std::collections::HashMap; use std::ffi::c_char; use std::ffi::c_void; use std::ffi::CStr; @@ -6,26 +5,27 @@ use std::ffi::CStr; use crate::index_writer::IndexWriterWrapper; use crate::tokenizer::create_tokenizer; use crate::util::create_binding; +use crate::string_c::c_str_to_str; +use crate::log::init_log; #[no_mangle] pub extern "C" fn tantivy_create_text_writer( field_name: *const c_char, path: *const c_char, tokenizer_name: *const c_char, - tokenizer_params: *mut c_void, + tokenizer_params: *const c_char, num_threads: usize, overall_memory_budget_in_bytes: usize, in_ram: bool, ) -> *mut c_void { + init_log(); let field_name_str = unsafe { CStr::from_ptr(field_name).to_str().unwrap() }; let path_str = unsafe { CStr::from_ptr(path).to_str().unwrap() }; let tokenizer_name_str = unsafe { CStr::from_ptr(tokenizer_name).to_str().unwrap() }; - let analyzer = unsafe { - let m = tokenizer_params as *const HashMap; - create_tokenizer(&(*m)) - }; + let params = unsafe{c_str_to_str(tokenizer_params).to_string()}; + let analyzer = create_tokenizer(¶ms); match analyzer { - Some(text_analyzer) => { + Ok(text_analyzer) => { let wrapper = IndexWriterWrapper::create_text_writer( String::from(field_name_str), String::from(path_str), @@ -37,8 +37,9 @@ pub extern "C" fn tantivy_create_text_writer( ); create_binding(wrapper) } - None => { + Err(err) => { + log::warn!("create tokenizer failed with error: {} param: {}", err.to_string(), params); std::ptr::null_mut() - } + }, } } diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/src/lib.rs b/internal/core/thirdparty/tantivy/tantivy-binding/src/lib.rs index fd73108fd4954..90bfa80fd11c7 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/src/lib.rs +++ b/internal/core/thirdparty/tantivy/tantivy-binding/src/lib.rs @@ -15,8 +15,10 @@ mod log; mod string_c; mod token_stream_c; mod tokenizer; +mod tokenizer_filter; mod tokenizer_c; mod util; +mod error; mod util_c; mod vec_collector; diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer.rs b/internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer.rs index 2e0d283947377..d831c9d918c6f 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer.rs +++ b/internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer.rs @@ -1,54 +1,254 @@ -use lazy_static::lazy_static; -use log::{info, warn}; +use log::warn; use std::collections::HashMap; -use tantivy::tokenizer::{TextAnalyzer, TokenizerManager}; -use crate::log::init_log; +use tantivy::tokenizer::*; +use serde_json as json; -lazy_static! { - static ref DEFAULT_TOKENIZER_MANAGER: TokenizerManager = TokenizerManager::default(); +use crate::tokenizer_filter::*; +use crate::error::TantivyError; +use crate::util::*; + + +// default build-in analyzer +pub(crate) fn standard_analyzer(stop_words: Vec) -> TextAnalyzer { + let builder = standard_builder() + .filter(LowerCaser) + .filter(RemoveLongFilter::limit(40)); + + if stop_words.len() > 0{ + return builder.filter(StopWordFilter::remove(stop_words)).build(); + } + + builder.build() } -pub(crate) fn default_tokenizer() -> TextAnalyzer { - DEFAULT_TOKENIZER_MANAGER.get("default").unwrap() +fn standard_builder() -> TextAnalyzerBuilder{ + TextAnalyzer::builder(SimpleTokenizer::default()).dynamic() } -fn jieba_tokenizer() -> TextAnalyzer { - tantivy_jieba::JiebaTokenizer {}.into() +fn whitespace_builder()-> TextAnalyzerBuilder{ + TextAnalyzer::builder(WhitespaceTokenizer::default()).dynamic() } -pub(crate) fn create_tokenizer(params: &HashMap) -> Option { - init_log(); +fn get_builder_by_name(name:&String) -> Result{ + match name.as_str() { + "standard" => Ok(standard_builder()), + "whitespace" => Ok(whitespace_builder()), + other => { + warn!("unsupported tokenizer: {}", other); + Err(format!("unsupported tokenizer: {}", other).into()) + } + } +} - match params.get("tokenizer") { - Some(tokenizer_name) => match tokenizer_name.as_str() { - "default" => { - Some(default_tokenizer()) +struct AnalyzerBuilder<'a>{ + // builder: TextAnalyzerBuilder + filters:HashMap, + params:&'a json::Map +} + +impl AnalyzerBuilder<'_>{ + fn new(params: &json::Map) -> AnalyzerBuilder{ + AnalyzerBuilder{ + filters: HashMap::new(), + params:params, + } + } + + fn get_tokenizer_name(&self) -> Result{ + let tokenizer=self.params.get("tokenizer"); + if tokenizer.is_none(){ + return Ok("standard".to_string()); + } + if !tokenizer.unwrap().is_string(){ + return Err(format!("tokenizer name should be string").into()); + } + + Ok(tokenizer.unwrap().as_str().unwrap().to_string()) + } + + fn add_custom_filter(&mut self, name: &String, params: &json::Map) -> Result<(),TantivyError>{ + match SystemFilter::try_from(params){ + Ok(filter) => { + self.filters.insert(name.to_string(), filter); + Ok(()) + }, + Err(e) => {Err(e)}, + } + } + + fn add_custom_filters(&mut self, params:&json::Map) -> Result<(),TantivyError>{ + for (name, value) in params{ + if !value.is_object(){ + continue; } - "jieba" => { - Some(jieba_tokenizer()) + self.add_custom_filter(name, value.as_object().unwrap())?; + } + Ok(()) + } + + fn build_filter(&mut self,mut builder: TextAnalyzerBuilder, params: &json::Value) -> Result{ + if !params.is_array(){ + return Err("filter params should be array".into()); + } + + let filters = params.as_array().unwrap(); + for filter in filters{ + if filter.is_string(){ + let filter_name = filter.as_str().unwrap(); + let costum = self.filters.remove(filter_name); + if !costum.is_none(){ + builder = costum.unwrap().transform(builder); + continue; + } + + // check if filter was system filter + let system = SystemFilter::from(filter_name); + match system { + SystemFilter::Invalid => { + return Err(format!("build analyzer failed, filter not found :{}", filter_name).into()) + } + other => { + builder = other.transform(builder); + }, + } + }else if filter.is_object(){ + let filter=SystemFilter::try_from(filter.as_object().unwrap())?; + builder = filter.transform(builder); } - s => { - warn!("unsupported tokenizer: {}", s); - None + }; + Ok(builder) + } + + fn build_option(&mut self, mut builder: TextAnalyzerBuilder) -> Result{ + for (key, value) in self.params{ + match key.as_str(){ + "tokenizer" => {}, + "filter" => { + // build with filter if filter param exist + builder=self.build_filter(builder, value)?; + }, + "max_token_length" => { + if !value.is_u64(){ + return Err("max token length should be int type".into()); + } + builder = builder.filter_dynamic(RemoveLongFilter::limit(value.as_u64().unwrap() as usize)); + } + other => return Err(format!("unknown analyzer option key: {}", other).into()), } - }, - None => { - Some(default_tokenizer()) } + Ok(builder) } + + fn build_template(self, type_: &str)-> Result{ + match type_{ + "standard" => { + let value = self.params.get("stop_words"); + match value{ + Some(value)=>{ + let str_list = get_string_list(value, "filter stop_words")?; + Ok(standard_analyzer(str_list)) + } + None => Ok(standard_analyzer(vec![])) + } + }, + other_ => Err(format!("unknown build-in analyzer type: {}", other_).into()) + } + } + + fn build(mut self) -> Result{ + // build base build-in analyzer + match self.params.get("type"){ + Some(type_) =>{ + if !type_.is_string(){ + return Err(format!("analyzer type shoud be string").into()) + } + return self.build_template(type_.as_str().unwrap()); + }, + None => {} + }; + + //build custom analyzer + let tokenizer_name = self.get_tokenizer_name()?; + + // jieba analyzer can't add filter. + if tokenizer_name == "jieba"{ + return Ok(tantivy_jieba::JiebaTokenizer{}.into()); + } + + let mut builder=get_builder_by_name(&tokenizer_name)?; + + // build with option + builder = self.build_option(builder)?; + Ok(builder.build()) + } +} + +pub(crate) fn create_tokenizer_with_filter(params: &String) -> Result { + match json::from_str::(¶ms){ + Ok(value) =>{ + if value.is_null(){ + return Ok(standard_analyzer(vec![])); + } + if !value.is_object(){ + return Err("tokenizer params should be a json map".into()); + } + let json_params = value.as_object().unwrap(); + + // create builder + let analyzer_params=json_params.get("analyzer"); + if analyzer_params.is_none(){ + return Ok(standard_analyzer(vec![])); + } + if !analyzer_params.unwrap().is_object(){ + return Err("analyzer params should be a json map".into()); + } + let mut builder = AnalyzerBuilder::new(analyzer_params.unwrap().as_object().unwrap()); + + // build custom filter + let filter_params=json_params.get("filter"); + if !filter_params.is_none() && filter_params.unwrap().is_object(){ + builder.add_custom_filters(filter_params.unwrap().as_object().unwrap())?; + } + + // build analyzer + builder.build() + }, + Err(err) => Err(err.into()), + } +} + +pub(crate) fn create_tokenizer(params: &String) -> Result { + if params.len()==0{ + return Ok(standard_analyzer(vec![])); + } + create_tokenizer_with_filter(&format!("{{\"analyzer\":{}}}", params)) } #[cfg(test)] mod tests { - use std::collections::HashMap; use crate::tokenizer::create_tokenizer; #[test] fn test_create_tokenizer() { - let mut params : HashMap = HashMap::new(); - params.insert("tokenizer".parse().unwrap(), "jieba".parse().unwrap()); + let params = r#"{"tokenizer": "standard"}"#; - let tokenizer = create_tokenizer(¶ms); - assert!(tokenizer.is_some()); + let tokenizer = create_tokenizer(¶ms.to_string()); + assert!(tokenizer.is_ok()); } -} + + #[test] + fn test_jieba_tokenizer() { + let params = r#"{"tokenizer": "jieba"}"#; + + let tokenizer = create_tokenizer(¶ms.to_string()); + assert!(tokenizer.is_ok()); + let mut bining = tokenizer.unwrap(); + + let mut stream = bining.token_stream("系统安全"); + while stream.advance(){ + let token = stream.token(); + let text = token.text.clone(); + print!("test token :{}\n", text.as_str()) + } + } +} \ No newline at end of file diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer_c.rs b/internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer_c.rs index c2caf097fc34c..86449699d5ab1 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer_c.rs +++ b/internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer_c.rs @@ -1,25 +1,34 @@ -use std::collections::HashMap; - -use libc::c_void; +use libc::{c_void,c_char}; use tantivy::tokenizer::TextAnalyzer; use crate::{ + string_c::c_str_to_str, tokenizer::create_tokenizer, util::{create_binding, free_binding}, + log::init_log, }; #[no_mangle] -pub extern "C" fn tantivy_create_tokenizer(tokenizer_params: *mut c_void) -> *mut c_void { - let analyzer = unsafe { - let m = tokenizer_params as *const HashMap; - create_tokenizer(&(*m)) - }; +pub extern "C" fn tantivy_create_tokenizer(tokenizer_params: *const c_char) -> *mut c_void { + init_log(); + let params = unsafe{c_str_to_str(tokenizer_params).to_string()}; + let analyzer = create_tokenizer(¶ms); match analyzer { - Some(text_analyzer) => create_binding(text_analyzer), - None => std::ptr::null_mut(), + Ok(text_analyzer) => create_binding(text_analyzer), + Err(err) => { + log::warn!("create tokenizer failed with error: {} param: {}", err.to_string(), params); + std::ptr::null_mut() + }, } } +#[no_mangle] +pub extern "C" fn tantivy_clone_tokenizer(ptr: *mut c_void) -> *mut c_void { + let analyzer=ptr as *mut TextAnalyzer; + let clone = unsafe {(*analyzer).clone()}; + create_binding(clone) +} + #[no_mangle] pub extern "C" fn tantivy_free_tokenizer(tokenizer: *mut c_void) { free_binding::(tokenizer); diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer_filter.rs b/internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer_filter.rs new file mode 100644 index 0000000000000..41dcbda8f210e --- /dev/null +++ b/internal/core/thirdparty/tantivy/tantivy-binding/src/tokenizer_filter.rs @@ -0,0 +1,154 @@ +use tantivy::tokenizer::*; +use serde_json as json; + +use crate::error::TantivyError; +use crate::util::*; + +pub(crate) enum SystemFilter{ + Invalid, + LowerCase(LowerCaser), + AsciiFolding(AsciiFoldingFilter), + AlphaNumOnly(AlphaNumOnlyFilter), + Length(RemoveLongFilter), + Stop(StopWordFilter), + Decompounder(SplitCompoundWords), + Stemmer(Stemmer) +} + +impl SystemFilter{ + pub(crate) fn transform(self, builder: TextAnalyzerBuilder) -> TextAnalyzerBuilder{ + match self{ + Self::LowerCase(filter) => builder.filter(filter).dynamic(), + Self::AsciiFolding(filter) => builder.filter(filter).dynamic(), + Self::AlphaNumOnly(filter) => builder.filter(filter).dynamic(), + Self::Length(filter) => builder.filter(filter).dynamic(), + Self::Stop(filter) => builder.filter(filter).dynamic(), + Self::Decompounder(filter) => builder.filter(filter).dynamic(), + Self::Stemmer(filter) => builder.filter(filter).dynamic(), + Self::Invalid => builder, + } + } +} + +// create length filter from params +// { +// "type": "length", +// "max": 10, // length +// } +// TODO support min length +fn get_length_filter(params: &json::Map) -> Result{ + let limit_str = params.get("max"); + if limit_str.is_none() || !limit_str.unwrap().is_u64(){ + return Err("lenth max param was none or not uint".into()) + } + let limit = limit_str.unwrap().as_u64().unwrap() as usize; + Ok(SystemFilter::Length(RemoveLongFilter::limit(limit))) +} + +fn get_stop_words_filter(params: &json::Map)-> Result{ + let value = params.get("stop_words"); + if value.is_none(){ + return Err("stop filter stop_words can't be empty".into()); + } + let str_list = get_string_list(value.unwrap(), "stop_words filter")?; + Ok(SystemFilter::Stop(StopWordFilter::remove(str_list))) +} + +fn get_decompounder_filter(params: &json::Map)-> Result{ + let value = params.get("word_list"); + if value.is_none() || !value.unwrap().is_array(){ + return Err("decompounder word list should be array".into()) + } + + let stop_words = value.unwrap().as_array().unwrap(); + let mut str_list = Vec::::new(); + for element in stop_words{ + match element.as_str(){ + Some(word) => str_list.push(word.to_string()), + None => return Err("decompounder word list item should be string".into()) + } + }; + + match SplitCompoundWords::from_dictionary(str_list){ + Ok(f) => Ok(SystemFilter::Decompounder(f)), + Err(e) => Err(format!("create decompounder failed: {}", e.to_string()).into()) + } +} + +fn get_stemmer_filter(params: &json::Map)-> Result{ + let value = params.get("language"); + if value.is_none() || !value.unwrap().is_string(){ + return Err("stemmer language field should be string".into()) + } + + match value.unwrap().as_str().unwrap().into_language(){ + Ok(language) => Ok(SystemFilter::Stemmer(Stemmer::new(language))), + Err(e) => Err(format!("create stemmer failed : {}", e.to_string()).into()), + } +} + +trait LanguageParser { + type Error; + fn into_language(self) -> Result; +} + +impl LanguageParser for &str { + type Error = TantivyError; + fn into_language(self) -> Result { + match self.to_lowercase().as_str() { + "arabig" => Ok(Language::Arabic), + "danish" => Ok(Language::Danish), + "dutch" => Ok(Language::Dutch), + "english" => Ok(Language::English), + "finnish" => Ok(Language::Finnish), + "french" => Ok(Language::French), + "german" => Ok(Language::German), + "greek" => Ok(Language::Greek), + "hungarian" => Ok(Language::Hungarian), + "italian" => Ok(Language::Italian), + "norwegian" => Ok(Language::Norwegian), + "portuguese" => Ok(Language::Portuguese), + "romanian" => Ok(Language::Romanian), + "russian" => Ok(Language::Russian), + "spanish" => Ok(Language::Spanish), + "swedish" => Ok(Language::Swedish), + "tamil" => Ok(Language::Tamil), + "turkish" => Ok(Language::Turkish), + other => Err(format!("unsupport language: {}", other).into()), + } + } +} + +impl From<&str> for SystemFilter{ + fn from(value: &str) -> Self { + match value{ + "lowercase" => Self::LowerCase(LowerCaser), + "asciifolding" => Self::AsciiFolding(AsciiFoldingFilter), + "alphanumonly" => Self::AlphaNumOnly(AlphaNumOnlyFilter), + _ => Self::Invalid, + } + } +} + +impl TryFrom<&json::Map> for SystemFilter { + type Error = TantivyError; + + fn try_from(params: &json::Map) -> Result { + match params.get(&"type".to_string()){ + Some(value) =>{ + if !value.is_string(){ + return Err("filter type should be string".into()); + }; + + match value.as_str().unwrap(){ + "length" => get_length_filter(params), + "stop" => get_stop_words_filter(params), + "decompounder" => get_decompounder_filter(params), + "stemmer" => get_stemmer_filter(params), + other=> Err(format!("unsupport filter type: {}", other).into()), + } + } + None => Err("no type field in filter params".into()), + } + } +} diff --git a/internal/core/thirdparty/tantivy/tantivy-binding/src/util.rs b/internal/core/thirdparty/tantivy/tantivy-binding/src/util.rs index 1f1c1655c1032..e705b5df072b1 100644 --- a/internal/core/thirdparty/tantivy/tantivy-binding/src/util.rs +++ b/internal/core/thirdparty/tantivy/tantivy-binding/src/util.rs @@ -1,5 +1,7 @@ use std::ffi::c_void; use std::ops::Bound; +use serde_json as json; +use crate::error::TantivyError; use tantivy::{directory::MmapDirectory, Index}; @@ -28,3 +30,19 @@ pub fn free_binding(ptr: *mut c_void) { drop(Box::from_raw(real)); } } + +pub(crate) fn get_string_list(value: &json::Value, label: &str) -> Result, TantivyError>{ + if !value.is_array(){ + return Err(format!("{} should be array", label).into()) + } + + let stop_words = value.as_array().unwrap(); + let mut str_list = Vec::::new(); + for element in stop_words{ + match element.as_str(){ + Some(word) => str_list.push(word.to_string()), + None => return Err(format!("{} list item should be string", label).into()) + } + }; + Ok(str_list) +} \ No newline at end of file diff --git a/internal/core/thirdparty/tantivy/tantivy-wrapper.h b/internal/core/thirdparty/tantivy/tantivy-wrapper.h index 17822d1bbdfb3..ee45e9d1958ff 100644 --- a/internal/core/thirdparty/tantivy/tantivy-wrapper.h +++ b/internal/core/thirdparty/tantivy/tantivy-wrapper.h @@ -14,7 +14,7 @@ namespace milvus::tantivy { using Map = std::map; static constexpr const char* DEFAULT_TOKENIZER_NAME = "milvus_tokenizer"; -static Map DEFAULT_TOKENIZER_PARAMS = {}; +static const char* DEFAULT_TOKENIZER_PARAMS = "{}"; static constexpr uintptr_t DEFAULT_NUM_THREADS = 4; static constexpr uintptr_t DEFAULT_OVERALL_MEMORY_BUDGET_IN_BYTES = DEFAULT_NUM_THREADS * 15 * 1024 * 1024; @@ -101,17 +101,14 @@ struct TantivyIndexWrapper { bool in_ram, const char* path, const char* tokenizer_name = DEFAULT_TOKENIZER_NAME, - const std::map& - tokenizer_params = DEFAULT_TOKENIZER_PARAMS, + const char* tokenizer_params = DEFAULT_TOKENIZER_PARAMS, uintptr_t num_threads = DEFAULT_NUM_THREADS, uintptr_t overall_memory_budget_in_bytes = DEFAULT_OVERALL_MEMORY_BUDGET_IN_BYTES) { - RustHashMap m; - m.from(tokenizer_params); writer_ = tantivy_create_text_writer(field_name, path, tokenizer_name, - m.get_pointer(), + tokenizer_params, num_threads, overall_memory_budget_in_bytes, in_ram); @@ -134,14 +131,11 @@ struct TantivyIndexWrapper { } void - register_tokenizer( - const char* tokenizer_name, - const std::map& tokenizer_params) { - RustHashMap m; - m.from(tokenizer_params); + register_tokenizer(const char* tokenizer_name, + const char* tokenizer_params) { if (reader_ != nullptr) { tantivy_register_tokenizer( - reader_, tokenizer_name, m.get_pointer()); + reader_, tokenizer_name, tokenizer_params); } } diff --git a/internal/core/thirdparty/tantivy/tokenizer.h b/internal/core/thirdparty/tantivy/tokenizer.h index dd753205aa196..eeeec4db6de3e 100644 --- a/internal/core/thirdparty/tantivy/tokenizer.h +++ b/internal/core/thirdparty/tantivy/tokenizer.h @@ -11,15 +11,17 @@ struct Tokenizer { public: NO_COPY_OR_ASSIGN(Tokenizer); - explicit Tokenizer(const std::map& params) { - RustHashMap m; - m.from(params); - ptr_ = tantivy_create_tokenizer(m.get_pointer()); + explicit Tokenizer(std::string&& params) { + auto shared_params = std::make_shared(std::move(params)); + ptr_ = tantivy_create_tokenizer(shared_params->c_str()); if (ptr_ == nullptr) { throw std::invalid_argument("invalid tokenizer parameters"); } } + explicit Tokenizer(void* _ptr) : ptr_(_ptr) { + } + ~Tokenizer() { if (ptr_ != nullptr) { tantivy_free_tokenizer(ptr_); @@ -34,6 +36,12 @@ struct Tokenizer { return std::make_unique(token_stream, shared_text); } + std::unique_ptr + Clone() { + auto newptr = tantivy_clone_tokenizer(ptr_); + return std::make_unique(newptr); + } + // CreateTokenStreamCopyText will copy the text and then create token stream based on the text. std::unique_ptr CreateTokenStreamCopyText(const std::string& text) { diff --git a/internal/core/unittest/test_c_tokenizer.cpp b/internal/core/unittest/test_c_tokenizer.cpp index 7e5c9e2a40df6..ac92370eaaaff 100644 --- a/internal/core/unittest/test_c_tokenizer.cpp +++ b/internal/core/unittest/test_c_tokenizer.cpp @@ -47,12 +47,10 @@ set_cmap(CMap m, const std::string& key, const std::string& value) { } TEST(CTokenizer, Default) { - auto m = create_cmap(); - set_cmap(m, "tokenizer", "default"); - + auto tokenizer_params = R"({"tokenizer": "standard"})"; CTokenizer tokenizer; { - auto status = create_tokenizer(m, &tokenizer); + auto status = create_tokenizer(tokenizer_params, &tokenizer); ASSERT_EQ(milvus::ErrorCode::Success, status.error_code); } @@ -71,5 +69,4 @@ TEST(CTokenizer, Default) { free_token_stream(token_stream); free_tokenizer(tokenizer); - free_cmap(m); } diff --git a/internal/core/unittest/test_text_match.cpp b/internal/core/unittest/test_text_match.cpp index 55b85cad1d118..574ebadb354ce 100644 --- a/internal/core/unittest/test_text_match.cpp +++ b/internal/core/unittest/test_text_match.cpp @@ -10,9 +10,9 @@ // or implied. See the License for the specific language governing permissions and limitations under the License #include +#include #include "common/Schema.h" -#include "segcore/segment_c.h" #include "segcore/SegmentGrowing.h" #include "segcore/SegmentGrowingImpl.h" #include "test_utils/DataGen.h" @@ -80,23 +80,19 @@ TEST(ParseJson, Naive) { TEST(ParseTokenizerParams, NoTokenizerParams) { TypeParams params{{"k", "v"}}; auto p = ParseTokenizerParams(params); - ASSERT_EQ(0, p.size()); + ASSERT_EQ("{}", std::string(p)); } TEST(ParseTokenizerParams, Default) { - TypeParams params{{"tokenizer_params", R"({"tokenizer": "default"})"}}; + TypeParams params{{"tokenizer_params", R"({"tokenizer": "standard"})"}}; auto p = ParseTokenizerParams(params); - ASSERT_EQ(1, p.size()); - auto iter = p.find("tokenizer"); - ASSERT_NE(p.end(), iter); - ASSERT_EQ("default", iter->second); + ASSERT_EQ(params.at("tokenizer_params"), p); } TEST(TextMatch, Index) { using Index = index::TextMatchIndex; - auto index = std::make_unique(std::numeric_limits::max(), - "milvus_tokenizer", - std::map{}); + auto index = std::make_unique( + std::numeric_limits::max(), "milvus_tokenizer", "{}"); index->CreateReader(); index->AddText("football, basketball, pingpang", 0); index->AddText("swimming, football", 1); diff --git a/internal/proxy/task_query.go b/internal/proxy/task_query.go index 29f3606f84532..3bbdaffd9ae29 100644 --- a/internal/proxy/task_query.go +++ b/internal/proxy/task_query.go @@ -297,7 +297,6 @@ func (t *queryTask) CanSkipAllocTimestamp() bool { } consistencyLevel = collectionInfo.consistencyLevel } - return consistencyLevel != commonpb.ConsistencyLevel_Strong } diff --git a/internal/proxy/task_search.go b/internal/proxy/task_search.go index 279387f5e6af4..511bbf74855e2 100644 --- a/internal/proxy/task_search.go +++ b/internal/proxy/task_search.go @@ -111,7 +111,6 @@ func (t *searchTask) CanSkipAllocTimestamp() bool { } consistencyLevel = collectionInfo.consistencyLevel } - return consistencyLevel != commonpb.ConsistencyLevel_Strong } diff --git a/internal/util/ctokenizer/c_tokenizer.go b/internal/util/ctokenizer/c_tokenizer.go index 915aa4cfa1938..e9f44aeb23a79 100644 --- a/internal/util/ctokenizer/c_tokenizer.go +++ b/internal/util/ctokenizer/c_tokenizer.go @@ -33,6 +33,15 @@ func (impl *CTokenizer) NewTokenStream(text string) tokenizerapi.TokenStream { return NewCTokenStream(ptr) } +func (impl *CTokenizer) Clone() (tokenizerapi.Tokenizer, error) { + var newptr C.CTokenizer + status := C.clone_tokenizer(&impl.ptr, &newptr) + if err := HandleCStatus(&status, "failed to clone tokenizer"); err != nil { + return nil, err + } + return NewCTokenizer(newptr), nil +} + func (impl *CTokenizer) Destroy() { C.free_tokenizer(impl.ptr) } diff --git a/internal/util/ctokenizer/c_tokenizer_factory.go b/internal/util/ctokenizer/c_tokenizer_factory.go index c5690d8861600..1bd3177917741 100644 --- a/internal/util/ctokenizer/c_tokenizer_factory.go +++ b/internal/util/ctokenizer/c_tokenizer_factory.go @@ -9,16 +9,17 @@ package ctokenizer import "C" import ( + "unsafe" + "github.com/milvus-io/milvus/internal/util/tokenizerapi" ) -func NewTokenizer(m map[string]string) (tokenizerapi.Tokenizer, error) { - mm := NewCMap() - defer mm.Destroy() - mm.From(m) +func NewTokenizer(param string) (tokenizerapi.Tokenizer, error) { + paramPtr := C.CString(param) + defer C.free(unsafe.Pointer(paramPtr)) var ptr C.CTokenizer - status := C.create_tokenizer(mm.GetPointer(), &ptr) + status := C.create_tokenizer(paramPtr, &ptr) if err := HandleCStatus(&status, "failed to create tokenizer"); err != nil { return nil, err } diff --git a/internal/util/ctokenizer/c_tokenizer_test.go b/internal/util/ctokenizer/c_tokenizer_test.go index 9b9517020d69e..f15b032a22e27 100644 --- a/internal/util/ctokenizer/c_tokenizer_test.go +++ b/internal/util/ctokenizer/c_tokenizer_test.go @@ -10,7 +10,7 @@ import ( func TestTokenizer(t *testing.T) { // default tokenizer. { - m := make(map[string]string) + m := "{\"tokenizer\": \"standard\"}" tokenizer, err := NewTokenizer(m) assert.NoError(t, err) defer tokenizer.Destroy() @@ -24,8 +24,7 @@ func TestTokenizer(t *testing.T) { // jieba tokenizer. { - m := make(map[string]string) - m["tokenizer"] = "jieba" + m := "{\"tokenizer\": \"jieba\"}" tokenizer, err := NewTokenizer(m) assert.NoError(t, err) defer tokenizer.Destroy() diff --git a/internal/util/ctokenizer/text_schema_validator_test.go b/internal/util/ctokenizer/text_schema_validator_test.go index 56e3ba668c5cb..9c202a849a7fa 100644 --- a/internal/util/ctokenizer/text_schema_validator_test.go +++ b/internal/util/ctokenizer/text_schema_validator_test.go @@ -33,7 +33,7 @@ func TestValidateTextSchema(t *testing.T) { DataType: schemapb.DataType_VarChar, TypeParams: []*commonpb.KeyValuePair{ {Key: "enable_match", Value: "true"}, - {Key: "tokenizer_params", Value: `{"tokenizer": "default"}`}, + {Key: "tokenizer_params", Value: `{"tokenizer": "standard"}`}, }, }, { @@ -41,7 +41,7 @@ func TestValidateTextSchema(t *testing.T) { DataType: schemapb.DataType_VarChar, TypeParams: []*commonpb.KeyValuePair{ {Key: "enable_match", Value: "true"}, - {Key: "tokenizer_params", Value: `{"tokenizer": "jieba"}`}, + {Key: "tokenizer_params", Value: `{"tokenizer": "standard"}`}, }, }, } diff --git a/internal/util/function/bm25_function.go b/internal/util/function/bm25_function.go index 275be8e412f29..ff8e046b1f504 100644 --- a/internal/util/function/bm25_function.go +++ b/internal/util/function/bm25_function.go @@ -40,6 +40,15 @@ type BM25FunctionRunner struct { concurrency int } +func getTokenizerParams(field *schemapb.FieldSchema) string { + for _, param := range field.GetTypeParams() { + if param.Key == "tokenizer_params" { + return param.Value + } + } + return "{}" +} + func NewBM25FunctionRunner(coll *schemapb.CollectionSchema, schema *schemapb.FunctionSchema) (*BM25FunctionRunner, error) { if len(schema.GetOutputFieldIds()) != 1 { return nil, fmt.Errorf("bm25 function should only have one output field, but now %d", len(schema.GetOutputFieldIds())) @@ -49,17 +58,22 @@ func NewBM25FunctionRunner(coll *schemapb.CollectionSchema, schema *schemapb.Fun schema: schema, concurrency: 8, } + var params string for _, field := range coll.GetFields() { if field.GetFieldID() == schema.GetOutputFieldIds()[0] { runner.outputField = field break } + + if field.GetFieldID() == schema.GetInputFieldIds()[0] { + params = getTokenizerParams(field) + } } if runner.outputField == nil { return nil, fmt.Errorf("no output field") } - tokenizer, err := ctokenizer.NewTokenizer(map[string]string{}) + tokenizer, err := ctokenizer.NewTokenizer(params) if err != nil { return nil, err } @@ -69,8 +83,7 @@ func NewBM25FunctionRunner(coll *schemapb.CollectionSchema, schema *schemapb.Fun } func (v *BM25FunctionRunner) run(data []string, dst []map[uint32]float32) error { - // TODO AOIASD Support single Tokenizer concurrency - tokenizer, err := ctokenizer.NewTokenizer(map[string]string{}) + tokenizer, err := v.tokenizer.Clone() if err != nil { return err } diff --git a/internal/util/tokenizerapi/tokenizer.go b/internal/util/tokenizerapi/tokenizer.go index 2b6debbec71f6..6dab31257122c 100644 --- a/internal/util/tokenizerapi/tokenizer.go +++ b/internal/util/tokenizerapi/tokenizer.go @@ -3,5 +3,6 @@ package tokenizerapi //go:generate mockery --name=Tokenizer --with-expecter type Tokenizer interface { NewTokenStream(text string) TokenStream + Clone() (Tokenizer, error) Destroy() } diff --git a/tests/python_client/common/common_func.py b/tests/python_client/common/common_func.py index b47d6301085f2..365bfe5f3599d 100644 --- a/tests/python_client/common/common_func.py +++ b/tests/python_client/common/common_func.py @@ -778,7 +778,7 @@ def gen_default_collection_schema(description=ct.default_desc, primary_field=ct. def gen_all_datatype_collection_schema(description=ct.default_desc, primary_field=ct.default_int64_field_name, auto_id=False, dim=ct.default_dim, enable_dynamic_field=True, **kwargs): tokenizer_params = { - "tokenizer": "default", + "tokenizer": "standard", } fields = [ gen_int64_field(), diff --git a/tests/python_client/testcases/test_full_text_search.py b/tests/python_client/testcases/test_full_text_search.py index c54933c7e7373..e7d674bb37591 100644 --- a/tests/python_client/testcases/test_full_text_search.py +++ b/tests/python_client/testcases/test_full_text_search.py @@ -33,7 +33,7 @@ class TestCreateCollectionWIthFullTextSearch(TestcaseBase): """ @pytest.mark.tags(CaseLabel.L0) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_create_collection_for_full_text_search(self, tokenizer): """ target: test create collection with full text search @@ -97,7 +97,7 @@ def test_create_collection_for_full_text_search(self, tokenizer): assert len(res["functions"]) == len(text_fields) @pytest.mark.tags(CaseLabel.L0) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_create_collection_for_full_text_search_twice_with_same_schema(self, tokenizer): """ target: test create collection with full text search twice with same schema @@ -175,7 +175,7 @@ class TestCreateCollectionWithFullTextSearchNegative(TestcaseBase): @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("tokenizer", ["unsupported"]) - @pytest.mark.xfail(reason="") + @pytest.mark.skip(reason="check not implement may cause panic") def test_create_collection_for_full_text_search_with_unsupported_tokenizer(self, tokenizer): """ target: test create collection with full text search with unsupported tokenizer @@ -249,7 +249,7 @@ def test_create_collection_for_full_text_search_with_invalid_input_output(self, expected: create collection failed """ tokenizer_params = { - "tokenizer": "default", + "tokenizer": "standard", } dim = 128 fields = [ @@ -327,7 +327,7 @@ def test_create_collection_for_full_text_search_with_field_not_tokenized(self): expected: create collection failed """ tokenizer_params = { - "tokenizer": "default", + "tokenizer": "standard", } dim = 128 fields = [ @@ -397,7 +397,7 @@ class TestInsertWithFullTextSearch(TestcaseBase): @pytest.mark.tags(CaseLabel.L0) @pytest.mark.parametrize("nullable", [False, True]) @pytest.mark.parametrize("text_lang", ["en", "zh", "hybrid"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_insert_for_full_text_search_default(self, tokenizer, text_lang, nullable): """ target: test insert data with full text search @@ -542,7 +542,7 @@ def test_insert_for_full_text_search_default(self, tokenizer, text_lang, nullabl @pytest.mark.parametrize("enable_dynamic_field", [True]) @pytest.mark.parametrize("nullable", [False]) @pytest.mark.parametrize("text_lang", ["en"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_insert_for_full_text_search_enable_dynamic_field(self, tokenizer, text_lang, nullable, enable_dynamic_field): """ target: test insert data with full text search and enable dynamic field @@ -692,7 +692,7 @@ def test_insert_for_full_text_search_enable_dynamic_field(self, tokenizer, text_ @pytest.mark.tags(CaseLabel.L0) @pytest.mark.parametrize("nullable", [True]) @pytest.mark.parametrize("text_lang", ["en"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_insert_for_full_text_search_with_dataframe(self, tokenizer, text_lang, nullable): """ target: test insert data for full text search with dataframe @@ -831,7 +831,7 @@ def test_insert_for_full_text_search_with_dataframe(self, tokenizer, text_lang, assert len(data) == count @pytest.mark.tags(CaseLabel.L2) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_insert_for_full_text_search_with_part_of_empty_string(self, tokenizer): """ target: test insert data with full text search with part of empty string @@ -990,7 +990,7 @@ class TestInsertWithFullTextSearchNegative(TestcaseBase): @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("nullable", [True]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_insert_with_full_text_search_with_non_varchar_data(self, tokenizer, nullable): """ target: test insert data with full text search with non varchar data @@ -1089,7 +1089,7 @@ class TestUpsertWithFullTextSearch(TestcaseBase): @pytest.mark.tags(CaseLabel.L0) @pytest.mark.parametrize("nullable", [False, True]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) @pytest.mark.xfail(reason="issue: https://github.com/milvus-io/milvus/issues/37021") def test_upsert_for_full_text_search(self, tokenizer, nullable): """ @@ -1260,7 +1260,7 @@ class TestUpsertWithFullTextSearchNegative(TestcaseBase): @pytest.mark.tags(CaseLabel.L1) @pytest.mark.parametrize("nullable", [False]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) @pytest.mark.xfail(reason="issue: https://github.com/milvus-io/milvus/issues/37021") def test_upsert_for_full_text_search_with_no_varchar_data(self, tokenizer, nullable): """ @@ -1402,7 +1402,7 @@ class TestDeleteWithFullTextSearch(TestcaseBase): """ @pytest.mark.tags(CaseLabel.L1) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_delete_for_full_text_search(self, tokenizer): """ target: test delete data for full text search @@ -1564,7 +1564,7 @@ class TestCreateIndexWithFullTextSearch(TestcaseBase): @pytest.mark.parametrize("b", [0.1]) @pytest.mark.parametrize("k", [1.2]) @pytest.mark.parametrize("index_type", ["SPARSE_INVERTED_INDEX", "SPARSE_WAND"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_create_index_for_full_text_search_default( self, tokenizer, index_type, k, b ): @@ -1688,7 +1688,7 @@ class TestCreateIndexWithFullTextSearchNegative(TestcaseBase): @pytest.mark.parametrize("b", [0.5]) @pytest.mark.parametrize("k", [1.5]) @pytest.mark.parametrize("index_type", ["HNSW", "INVALID_INDEX_TYPE"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_create_full_text_search_with_invalid_index_type( self, tokenizer, index_type, k, b ): @@ -1796,7 +1796,7 @@ def test_create_full_text_search_with_invalid_index_type( @pytest.mark.parametrize("k", [1.5]) @pytest.mark.parametrize("index_type", ["SPARSE_INVERTED_INDEX"]) @pytest.mark.parametrize("metric_type", ["COSINE", "L2", "IP"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_create_full_text_search_index_with_invalid_metric_type( self, tokenizer, index_type, metric_type, k, b ): @@ -1903,7 +1903,7 @@ def test_create_full_text_search_index_with_invalid_metric_type( @pytest.mark.parametrize("b", [0.5]) @pytest.mark.parametrize("k", [1.5]) @pytest.mark.parametrize("index_type", ["SPARSE_INVERTED_INDEX"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_create_index_using_bm25_metric_type_for_non_bm25_output_field( self, tokenizer, index_type, k, b ): @@ -2000,7 +2000,7 @@ def test_create_index_using_bm25_metric_type_for_non_bm25_output_field( @pytest.mark.parametrize("b", [-1]) @pytest.mark.parametrize("k", [-1]) @pytest.mark.parametrize("index_type", ["SPARSE_INVERTED_INDEX"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_create_full_text_search_with_invalid_bm25_params( self, tokenizer, index_type, k, b ): @@ -2121,7 +2121,7 @@ class TestSearchWithFullTextSearch(TestcaseBase): @pytest.mark.parametrize("enable_inverted_index", [True]) @pytest.mark.parametrize("index_type", ["SPARSE_INVERTED_INDEX", "SPARSE_WAND"]) @pytest.mark.parametrize("expr", ["text_match", "id_range"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) @pytest.mark.parametrize("offset", [10, 0]) def test_full_text_search_default( self, offset, tokenizer, expr, enable_inverted_index, enable_partition_key, empty_percent, index_type, nq @@ -2317,7 +2317,6 @@ def test_full_text_search_default( @pytest.mark.parametrize("expr", ["text_match"]) @pytest.mark.parametrize("offset", [10]) @pytest.mark.parametrize("tokenizer", ["jieba"]) - @pytest.mark.xfail(reason="issue: https://github.com/milvus-io/milvus/issues/36751") def test_full_text_search_with_jieba_tokenizer( self, offset, tokenizer, expr, enable_inverted_index, enable_partition_key, empty_percent, index_type, nq ): @@ -2329,7 +2328,7 @@ def test_full_text_search_with_jieba_tokenizer( expected: full text search successfully and result is correct """ tokenizer_params = { - "tokenizer": tokenizer, + "tokenizer": tokenizer, } dim = 128 fields = [ @@ -2511,7 +2510,7 @@ def test_full_text_search_with_jieba_tokenizer( @pytest.mark.parametrize("enable_inverted_index", [True]) @pytest.mark.parametrize("index_type", ["SPARSE_INVERTED_INDEX"]) @pytest.mark.parametrize("expr", [None]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_full_text_search_with_range_search( self, tokenizer, expr, enable_inverted_index, enable_partition_key, empty_percent, index_type, nq ): @@ -2676,7 +2675,7 @@ def test_full_text_search_with_range_search( @pytest.mark.parametrize("enable_inverted_index", [True]) @pytest.mark.parametrize("index_type", ["SPARSE_INVERTED_INDEX"]) @pytest.mark.parametrize("expr", [None]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_full_text_search_with_search_iterator( self, tokenizer, expr, enable_inverted_index, enable_partition_key, empty_percent, index_type, nq ): @@ -2829,7 +2828,7 @@ class TestSearchWithFullTextSearchNegative(TestcaseBase): @pytest.mark.parametrize("enable_inverted_index", [True]) @pytest.mark.parametrize("index_type", ["SPARSE_INVERTED_INDEX"]) @pytest.mark.parametrize("invalid_search_data", ["empty_text"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) @pytest.mark.xfail(reason="issue: https://github.com/milvus-io/milvus/issues/37022") def test_search_for_full_text_search_with_empty_string_search_data( self, tokenizer, enable_inverted_index, enable_partition_key, empty_percent, index_type, invalid_search_data @@ -2959,7 +2958,7 @@ def test_search_for_full_text_search_with_empty_string_search_data( @pytest.mark.parametrize("enable_inverted_index", [True]) @pytest.mark.parametrize("index_type", ["SPARSE_INVERTED_INDEX", "SPARSE_WAND"]) @pytest.mark.parametrize("invalid_search_data", ["sparse_vector", "dense_vector"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_search_for_full_text_search_with_invalid_search_data( self, tokenizer, enable_inverted_index, enable_partition_key, empty_percent, index_type, invalid_search_data ): @@ -3106,7 +3105,7 @@ class TestHybridSearchWithFullTextSearch(TestcaseBase): @pytest.mark.parametrize("enable_partition_key", [True]) @pytest.mark.parametrize("enable_inverted_index", [True]) @pytest.mark.parametrize("index_type", ["SPARSE_INVERTED_INDEX"]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_hybrid_search_with_full_text_search( self, tokenizer, enable_inverted_index, enable_partition_key, empty_percent, index_type ): diff --git a/tests/python_client/testcases/test_query.py b/tests/python_client/testcases/test_query.py index dc1e6622c2e62..03ff8a7578b7b 100644 --- a/tests/python_client/testcases/test_query.py +++ b/tests/python_client/testcases/test_query.py @@ -4441,7 +4441,7 @@ class TestQueryTextMatch(TestcaseBase): @pytest.mark.tags(CaseLabel.L0) @pytest.mark.parametrize("enable_partition_key", [True, False]) @pytest.mark.parametrize("enable_inverted_index", [True, False]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_query_text_match_en_normal( self, tokenizer, enable_inverted_index, enable_partition_key ): @@ -4724,24 +4724,16 @@ def test_query_text_match_custom_analyzer(self): expected: get the correct token, text match successfully and result is correct """ tokenizer_params = { - "tokenizer": "standard", - "alpha_num_only": True, - "ascii_folding": True, - "lower_case": True, - "max_token_length": 40, - "split_compound_words": [ - "dampf", - "schiff", - "fahrt", - "brot", - "backen", - "automat", - ], - "stemmer": "English", - "stop": { - "language": "English", - "words": ["an", "the"], - }, + "tokenizer": "standard", + # "lowercase", "asciifolding", "alphanumonly" was system filter + "filter":["lowercase", "asciifolding", "alphanumonly", + { + "type": "stop", + "stop_words": ["in", "of"], + }, { + "type": "stemmer", + "language": "english", + }], } dim = 128 fields = [ @@ -4852,7 +4844,7 @@ def test_query_text_match_with_combined_expression_for_single_field(self): expected: query successfully and result is correct """ tokenizer_params = { - "tokenizer": "default", + "tokenizer": "standard", } # 1. initialize with data dim = 128 @@ -4966,7 +4958,7 @@ def test_query_text_match_with_combined_expression_for_multi_field(self): expected: query successfully and result is correct """ tokenizer_params = { - "tokenizer": "default", + "tokenizer": "standard", } # 1. initialize with data dim = 128 @@ -5109,7 +5101,7 @@ def test_query_text_match_with_multi_lang(self): # 1. initialize with data tokenizer_params = { - "tokenizer": "default", + "tokenizer": "standard", } # 1. initialize with data dim = 128 @@ -5254,7 +5246,7 @@ def test_query_text_match_with_addition_inverted_index(self): # 1. initialize with data fake_en = Faker("en_US") tokenizer_params = { - "tokenizer": "default", + "tokenizer": "standard", } dim = 128 default_fields = [ @@ -5481,7 +5473,7 @@ def test_query_text_match_with_some_empty_string(self): """ # 1. initialize with data tokenizer_params = { - "tokenizer": "default", + "tokenizer": "standard", } # 1. initialize with data dim = 128 diff --git a/tests/python_client/testcases/test_search.py b/tests/python_client/testcases/test_search.py index ee21195a10a8c..a42d5c44958b8 100644 --- a/tests/python_client/testcases/test_search.py +++ b/tests/python_client/testcases/test_search.py @@ -13290,7 +13290,7 @@ class TestSearchWithTextMatchFilter(TestcaseBase): @pytest.mark.tags(CaseLabel.L0) @pytest.mark.parametrize("enable_partition_key", [True, False]) @pytest.mark.parametrize("enable_inverted_index", [True, False]) - @pytest.mark.parametrize("tokenizer", ["default"]) + @pytest.mark.parametrize("tokenizer", ["standard"]) def test_search_with_text_match_filter_normal_en( self, tokenizer, enable_inverted_index, enable_partition_key ): diff --git a/tests/restful_client_v2/testcases/test_vector_operations.py b/tests/restful_client_v2/testcases/test_vector_operations.py index 0f6f6c640e770..b6fc35f6855b9 100644 --- a/tests/restful_client_v2/testcases/test_vector_operations.py +++ b/tests/restful_client_v2/testcases/test_vector_operations.py @@ -1881,7 +1881,7 @@ def test_search_vector_with_ignore_growing(self, ignore_growing): assert len(res) == limit - @pytest.mark.parametrize("tokenizer", ["jieba", "default"]) + @pytest.mark.parametrize("tokenizer", ["jieba", "standard"]) def test_search_vector_with_text_match_filter(self, tokenizer): """ Query a vector with a simple payload @@ -2718,7 +2718,7 @@ def test_query_vector_with_large_sum_of_limit_offset(self, sum_of_limit_offset): if "like" in filter_expr: assert name.startswith(prefix) - @pytest.mark.parametrize("tokenizer", ["jieba", "default"]) + @pytest.mark.parametrize("tokenizer", ["jieba", "standard"]) def test_query_vector_with_text_match_filter(self, tokenizer): """ Query a vector with a simple payload From cf66fbee8a55aeeecbc47e31fd522fe97a1e5561 Mon Sep 17 00:00:00 2001 From: nico <109071306+NicoYuan1986@users.noreply.github.com> Date: Wed, 6 Nov 2024 17:50:25 +0800 Subject: [PATCH 13/40] test: update test cases (#37215) Signed-off-by: nico --- tests/python_client/testcases/test_index.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/python_client/testcases/test_index.py b/tests/python_client/testcases/test_index.py index 1bbfc13f18f97..9ebeb41f9c252 100644 --- a/tests/python_client/testcases/test_index.py +++ b/tests/python_client/testcases/test_index.py @@ -2947,6 +2947,7 @@ def test_bitmap_alter_cardinality_limit(self, request): @pytest.mark.tags(CaseLabel.L2) @pytest.mark.parametrize("bitmap_cardinality_limit", [-10, 0, 1001]) + # @pytest.mark.skip("valid now") def test_bitmap_cardinality_limit_invalid(self, request, bitmap_cardinality_limit): """ target: @@ -2973,8 +2974,9 @@ def test_bitmap_cardinality_limit_invalid(self, request, bitmap_cardinality_limi # build scalar index and check failed self.collection_wrap.create_index( field_name=DataType.INT64.name, index_name=DataType.INT64.name, - index_params={"index_type": IndexName.AUTOINDEX, "bitmap_cardinality_limit": bitmap_cardinality_limit}, - check_task=CheckTasks.err_res, check_items={ct.err_code: 1100, ct.err_msg: iem.CheckBitmapCardinality}) + index_params={"index_type": IndexName.AUTOINDEX, "bitmap_cardinality_limit": bitmap_cardinality_limit}) + assert self.collection_wrap.index()[0].params == {'bitmap_cardinality_limit': str(bitmap_cardinality_limit), + 'index_type': IndexName.AUTOINDEX} @pytest.mark.tags(CaseLabel.L2) @pytest.mark.parametrize("index_params, name", [({"index_type": IndexName.AUTOINDEX}, "AUTOINDEX"), ({}, "None")]) From 92028b7ff7d843921364e2d9cc604dbd882b70db Mon Sep 17 00:00:00 2001 From: congqixia Date: Wed, 6 Nov 2024 19:38:24 +0800 Subject: [PATCH 14/40] enhance: Use cancel label for ctx canceled storage op (#37468) Previously failed label is used for canceled storage op, which may cause wrong alarm when user cancel load operation or etc. This PR utilizes cancel label when such case happens. Signed-off-by: Congqi Xia --- internal/storage/remote_chunk_manager.go | 24 ++++++++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/internal/storage/remote_chunk_manager.go b/internal/storage/remote_chunk_manager.go index f510fe2e5008b..451b043f892e3 100644 --- a/internal/storage/remote_chunk_manager.go +++ b/internal/storage/remote_chunk_manager.go @@ -338,7 +338,11 @@ func (mcm *RemoteChunkManager) getObject(ctx context.Context, bucketName, object if err == nil && reader != nil { metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataGetLabel, metrics.SuccessLabel).Inc() } else { - metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataGetLabel, metrics.FailLabel).Inc() + if errors.Is(err, context.Canceled) { + metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataGetLabel, metrics.CancelLabel).Inc() + } else { + metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataGetLabel, metrics.FailLabel).Inc() + } } return reader, err @@ -354,7 +358,11 @@ func (mcm *RemoteChunkManager) putObject(ctx context.Context, bucketName, object Observe(float64(start.ElapseSpan().Milliseconds())) metrics.PersistentDataOpCounter.WithLabelValues(metrics.MetaPutLabel, metrics.SuccessLabel).Inc() } else { - metrics.PersistentDataOpCounter.WithLabelValues(metrics.MetaPutLabel, metrics.FailLabel).Inc() + if errors.Is(err, context.Canceled) { + metrics.PersistentDataOpCounter.WithLabelValues(metrics.MetaPutLabel, metrics.CancelLabel).Inc() + } else { + metrics.PersistentDataOpCounter.WithLabelValues(metrics.MetaPutLabel, metrics.FailLabel).Inc() + } } return err @@ -370,7 +378,11 @@ func (mcm *RemoteChunkManager) getObjectSize(ctx context.Context, bucketName, ob Observe(float64(start.ElapseSpan().Milliseconds())) metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataStatLabel, metrics.SuccessLabel).Inc() } else { - metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataStatLabel, metrics.FailLabel).Inc() + if errors.Is(err, context.Canceled) { + metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataStatLabel, metrics.CancelLabel).Inc() + } else { + metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataStatLabel, metrics.FailLabel).Inc() + } } return info, err @@ -386,7 +398,11 @@ func (mcm *RemoteChunkManager) removeObject(ctx context.Context, bucketName, obj Observe(float64(start.ElapseSpan().Milliseconds())) metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.SuccessLabel).Inc() } else { - metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.FailLabel).Inc() + if errors.Is(err, context.Canceled) { + metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.CancelLabel).Inc() + } else { + metrics.PersistentDataOpCounter.WithLabelValues(metrics.DataRemoveLabel, metrics.FailLabel).Inc() + } } return err From 8275e40f9462558603a458d5db4b6a4d5f62d5f0 Mon Sep 17 00:00:00 2001 From: sre-ci-robot Date: Wed, 6 Nov 2024 12:01:37 +0000 Subject: [PATCH 15/40] Update all contributors Signed-off-by: sre-ci-robot --- README.md | 3 ++- README_CN.md | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 9ca9f7a5d8550..75a9ee4d67d78 100644 --- a/README.md +++ b/README.md @@ -174,7 +174,7 @@ Contributions to Milvus are welcome from everyone. See [Guidelines for Contribut ### All contributors
-
+
@@ -572,6 +572,7 @@ Contributions to Milvus are welcome from everyone. See [Guidelines for Contribut + diff --git a/README_CN.md b/README_CN.md index afd7dc9af62ec..30b1bdaa146c9 100644 --- a/README_CN.md +++ b/README_CN.md @@ -156,7 +156,7 @@ Milvus [训练营](https://github.com/milvus-io/bootcamp)能够帮助你了解 ### All contributors
-
+
@@ -554,6 +554,7 @@ Milvus [训练营](https://github.com/milvus-io/bootcamp)能够帮助你了解 + From a2ecff1fb71c5a1228a76cf761db57c5beccbc64 Mon Sep 17 00:00:00 2001 From: sre-ci-robot Date: Thu, 7 Nov 2024 00:01:34 +0000 Subject: [PATCH 16/40] Update all contributors Signed-off-by: sre-ci-robot --- README.md | 3 ++- README_CN.md | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 75a9ee4d67d78..54fe6c65b8d07 100644 --- a/README.md +++ b/README.md @@ -174,10 +174,11 @@ Contributions to Milvus are welcome from everyone. See [Guidelines for Contribut ### All contributors
-
+
+ diff --git a/README_CN.md b/README_CN.md index 30b1bdaa146c9..089315d5e6bbf 100644 --- a/README_CN.md +++ b/README_CN.md @@ -156,10 +156,11 @@ Milvus [训练营](https://github.com/milvus-io/bootcamp)能够帮助你了解 ### All contributors
-
+
+ From 7cfd609ebc45789638b2726c8ee6099da42c2f38 Mon Sep 17 00:00:00 2001 From: wei liu Date: Thu, 7 Nov 2024 09:08:21 +0800 Subject: [PATCH 17/40] fix: [skip e2e]unstable integration test TestNodeDownOnSingleReplica (#37480) issue: #37289 cause pr #37116 introduce retry on get shard leader, which make search won't fail during query node down. Signed-off-by: Wei Liu --- .../replicas/balance/replica_test.go | 39 ++++--------------- 1 file changed, 8 insertions(+), 31 deletions(-) diff --git a/tests/integration/replicas/balance/replica_test.go b/tests/integration/replicas/balance/replica_test.go index 4bece315b1c97..a6bf3f7d4b80e 100644 --- a/tests/integration/replicas/balance/replica_test.go +++ b/tests/integration/replicas/balance/replica_test.go @@ -28,7 +28,6 @@ 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/proto/querypb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/funcutil" "github.com/milvus-io/milvus/pkg/util/merr" @@ -89,15 +88,6 @@ func (s *ReplicaTestSuit) TestNodeDownOnSingleReplica() { ctx := context.Background() - qn := s.Cluster.AddQueryNode() - // check segment number on new querynode - s.Eventually(func() bool { - resp, err := qn.GetDataDistribution(ctx, &querypb.GetDataDistributionRequest{}) - s.NoError(err) - s.True(merr.Ok(resp.GetStatus())) - return len(resp.Channels) == 1 && len(resp.Segments) == 2 - }, 30*time.Second, 1*time.Second) - stopSearchCh := make(chan struct{}) failCounter := atomic.NewInt64(0) go func() { @@ -116,7 +106,9 @@ func (s *ReplicaTestSuit) TestNodeDownOnSingleReplica() { searchReq := integration.ConstructSearchRequest("", name, expr, integration.FloatVecField, schemapb.DataType_FloatVector, nil, metric.L2, params, nq, dim, topk, roundDecimal) - searchResult, err := s.Cluster.Proxy.Search(ctx, searchReq) + searchCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + defer cancel() + searchResult, err := s.Cluster.Proxy.Search(searchCtx, searchReq) err = merr.CheckRPCCall(searchResult, err) if err != nil { @@ -130,7 +122,7 @@ func (s *ReplicaTestSuit) TestNodeDownOnSingleReplica() { s.Equal(failCounter.Load(), int64(0)) // stop qn in single replica expected got search failures - qn.Stop() + s.Cluster.QueryNode.Stop() time.Sleep(10 * time.Second) s.True(failCounter.Load() > 0) @@ -150,23 +142,6 @@ func (s *ReplicaTestSuit) TestNodeDownOnMultiReplica() { s.NoError(err) s.Len(resp.Replicas, 2) - // add a querynode, expected balance happens - qn1 := s.Cluster.AddQueryNode() - qn2 := s.Cluster.AddQueryNode() - - // check segment num on new query node - s.Eventually(func() bool { - resp, err := qn1.GetDataDistribution(ctx, &querypb.GetDataDistributionRequest{}) - s.NoError(err) - return len(resp.Channels) == 1 && len(resp.Segments) == 2 - }, 30*time.Second, 1*time.Second) - - s.Eventually(func() bool { - resp, err := qn2.GetDataDistribution(ctx, &querypb.GetDataDistributionRequest{}) - s.NoError(err) - return len(resp.Channels) == 1 && len(resp.Segments) == 2 - }, 30*time.Second, 1*time.Second) - stopSearchCh := make(chan struct{}) failCounter := atomic.NewInt64(0) go func() { @@ -185,7 +160,9 @@ func (s *ReplicaTestSuit) TestNodeDownOnMultiReplica() { searchReq := integration.ConstructSearchRequest("", name, expr, integration.FloatVecField, schemapb.DataType_FloatVector, nil, metric.L2, params, nq, dim, topk, roundDecimal) - searchResult, err := s.Cluster.Proxy.Search(ctx, searchReq) + searchCtx, cancel := context.WithTimeout(ctx, 5*time.Second) + defer cancel() + searchResult, err := s.Cluster.Proxy.Search(searchCtx, searchReq) err = merr.CheckRPCCall(searchResult, err) if err != nil { @@ -199,7 +176,7 @@ func (s *ReplicaTestSuit) TestNodeDownOnMultiReplica() { s.Equal(failCounter.Load(), int64(0)) // stop qn in multi replica replica expected no search failures - qn1.Stop() + s.Cluster.QueryNode.Stop() time.Sleep(20 * time.Second) s.Equal(failCounter.Load(), int64(0)) From ee54a98578eb7a0eca4b3f9cde61c5fce34b6b06 Mon Sep 17 00:00:00 2001 From: congqixia Date: Thu, 7 Nov 2024 10:06:25 +0800 Subject: [PATCH 18/40] enhance: Add cgo call metrics for load/write API (#37405) Cgo API cost is not observerable since not metrics is related to them. This PR add metrics for some sync cgo call related to load & write --------- Signed-off-by: Congqi Xia --- .../querynodev2/segments/load_index_info.go | 12 ++++++ internal/querynodev2/segments/segment.go | 41 +++++++++++++++++++ pkg/metrics/metrics.go | 2 + pkg/metrics/querynode_metrics.go | 14 +++++++ 4 files changed, 69 insertions(+) diff --git a/internal/querynodev2/segments/load_index_info.go b/internal/querynodev2/segments/load_index_info.go index ae05130211a47..07b2e3205cb4f 100644 --- a/internal/querynodev2/segments/load_index_info.go +++ b/internal/querynodev2/segments/load_index_info.go @@ -26,13 +26,17 @@ import "C" import ( "context" + "fmt" "runtime" + "time" "unsafe" "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/proto/cgopb" + "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/util/paramtable" ) // LoadIndexInfo is a wrapper of the underlying C-structure C.CLoadIndexInfo @@ -191,6 +195,14 @@ func (li *LoadIndexInfo) appendLoadIndexInfo(ctx context.Context, info *cgopb.Lo func (li *LoadIndexInfo) loadIndex(ctx context.Context) error { var status C.CStatus _, _ = GetLoadPool().Submit(func() (any, error) { + start := time.Now() + defer func() { + metrics.QueryNodeCGOCallLatency.WithLabelValues( + fmt.Sprint(paramtable.GetNodeID()), + "AppendIndexV2", + "Sync", + ).Observe(float64(time.Since(start).Milliseconds())) + }() traceCtx := ParseCTraceContext(ctx) status = C.AppendIndexV2(traceCtx.ctx, li.cLoadIndexInfo) runtime.KeepAlive(traceCtx) diff --git a/internal/querynodev2/segments/segment.go b/internal/querynodev2/segments/segment.go index 20578e32f91df..37544475f09d1 100644 --- a/internal/querynodev2/segments/segment.go +++ b/internal/querynodev2/segments/segment.go @@ -31,6 +31,7 @@ import ( "fmt" "runtime" "strings" + "time" "unsafe" "github.com/cockroachdb/errors" @@ -747,6 +748,14 @@ func (s *LocalSegment) Insert(ctx context.Context, rowIDs []int64, timestamps [] var status C.CStatus GetDynamicPool().Submit(func() (any, error) { + start := time.Now() + defer func() { + metrics.QueryNodeCGOCallLatency.WithLabelValues( + fmt.Sprint(paramtable.GetNodeID()), + "Insert", + "Sync", + ).Observe(float64(time.Since(start).Milliseconds())) + }() status = C.Insert(s.ptr, cOffset, cNumOfRows, @@ -822,6 +831,14 @@ func (s *LocalSegment) Delete(ctx context.Context, primaryKeys []storage.Primary } var status C.CStatus GetDynamicPool().Submit(func() (any, error) { + start := time.Now() + defer func() { + metrics.QueryNodeCGOCallLatency.WithLabelValues( + fmt.Sprint(paramtable.GetNodeID()), + "Delete", + "Sync", + ).Observe(float64(time.Since(start).Milliseconds())) + }() status = C.Delete(s.ptr, cOffset, cSize, @@ -884,6 +901,14 @@ func (s *LocalSegment) LoadMultiFieldData(ctx context.Context) error { var status C.CStatus GetLoadPool().Submit(func() (any, error) { + start := time.Now() + defer func() { + metrics.QueryNodeCGOCallLatency.WithLabelValues( + fmt.Sprint(paramtable.GetNodeID()), + "LoadFieldData", + "Sync", + ).Observe(float64(time.Since(start).Milliseconds())) + }() status = C.LoadFieldData(s.ptr, loadFieldDataInfo.cLoadFieldDataInfo) return nil, nil }).Await() @@ -951,6 +976,14 @@ func (s *LocalSegment) LoadFieldData(ctx context.Context, fieldID int64, rowCoun var status C.CStatus GetLoadPool().Submit(func() (any, error) { + start := time.Now() + defer func() { + metrics.QueryNodeCGOCallLatency.WithLabelValues( + fmt.Sprint(paramtable.GetNodeID()), + "LoadFieldData", + "Sync", + ).Observe(float64(time.Since(start).Milliseconds())) + }() log.Info("submitted loadFieldData task to load pool") status = C.LoadFieldData(s.ptr, loadFieldDataInfo.cLoadFieldDataInfo) return nil, nil @@ -1055,6 +1088,14 @@ func (s *LocalSegment) LoadDeltaData(ctx context.Context, deltaData *storage.Del */ var status C.CStatus GetDynamicPool().Submit(func() (any, error) { + start := time.Now() + defer func() { + metrics.QueryNodeCGOCallLatency.WithLabelValues( + fmt.Sprint(paramtable.GetNodeID()), + "LoadDeletedRecord", + "Sync", + ).Observe(float64(time.Since(start).Milliseconds())) + }() status = C.LoadDeletedRecord(s.ptr, loadInfo) return nil, nil }).Await() diff --git a/pkg/metrics/metrics.go b/pkg/metrics/metrics.go index daceb8c550a8b..723ff02864ba2 100644 --- a/pkg/metrics/metrics.go +++ b/pkg/metrics/metrics.go @@ -122,6 +122,8 @@ const ( lockOp = "lock_op" loadTypeName = "load_type" pathLabelName = "path" + cgoNameLabelName = `cgo_name` + cgoTypeLabelName = `cgo_type` // entities label LoadedLabel = "loaded" diff --git a/pkg/metrics/querynode_metrics.go b/pkg/metrics/querynode_metrics.go index b8e8b55c0e4a4..102f795ae4964 100644 --- a/pkg/metrics/querynode_metrics.go +++ b/pkg/metrics/querynode_metrics.go @@ -790,6 +790,19 @@ var ( channelNameLabelName, }, ) + + QueryNodeCGOCallLatency = prometheus.NewHistogramVec( + prometheus.HistogramOpts{ + Namespace: milvusNamespace, + Subsystem: typeutil.QueryNodeRole, + Name: "cgo_latency", + Help: "latency of each cgo call", + Buckets: buckets, + }, []string{ + nodeIDLabelName, + cgoNameLabelName, + cgoTypeLabelName, + }) ) // RegisterQueryNode registers QueryNode metrics @@ -859,6 +872,7 @@ func RegisterQueryNode(registry *prometheus.Registry) { registry.MustRegister(QueryNodeSearchHitSegmentNum) registry.MustRegister(QueryNodeDeleteBufferSize) registry.MustRegister(QueryNodeDeleteBufferRowNum) + registry.MustRegister(QueryNodeCGOCallLatency) // Add cgo metrics RegisterCGOMetrics(registry) From 6325d025048672bb1e31d616bfe4ef862729713e Mon Sep 17 00:00:00 2001 From: congqixia Date: Thu, 7 Nov 2024 10:08:24 +0800 Subject: [PATCH 19/40] fix: Add IP address validation from paramtable (#37416) See also #37404 #37402 IP address in paramtable need validation and fail fast with reasonable error message --------- Signed-off-by: Congqi Xia --- pkg/util/funcutil/func.go | 15 +++++++++++++++ pkg/util/funcutil/func_test.go | 32 +++++++++++++++++++++++++++----- 2 files changed, 42 insertions(+), 5 deletions(-) diff --git a/pkg/util/funcutil/func.go b/pkg/util/funcutil/func.go index b1b2882c53e3a..acd9aeb062455 100644 --- a/pkg/util/funcutil/func.go +++ b/pkg/util/funcutil/func.go @@ -30,12 +30,14 @@ import ( "time" "github.com/cockroachdb/errors" + "go.uber.org/zap" "google.golang.org/grpc/codes" grpcStatus "google.golang.org/grpc/status" "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/pkg/log" "github.com/milvus-io/milvus/pkg/util" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -57,6 +59,19 @@ func GetIP(ip string) string { if len(ip) == 0 { return GetLocalIP() } + netIP := net.ParseIP(ip) + // not a valid ip addr + if netIP == nil { + log.Warn("cannot parse input ip, treat it as hostname/service name", zap.String("ip", ip)) + return ip + } + // only localhost or unicast is acceptable + if netIP.IsUnspecified() { + panic(errors.Newf(`"%s" in param table is Unspecified IP address and cannot be used`)) + } + if netIP.IsMulticast() || netIP.IsLinkLocalMulticast() || netIP.IsInterfaceLocalMulticast() { + panic(errors.Newf(`"%s" in param table is Multicast IP address and cannot be used`)) + } return ip } diff --git a/pkg/util/funcutil/func_test.go b/pkg/util/funcutil/func_test.go index aec82cdcc5817..e2de736e4a1c3 100644 --- a/pkg/util/funcutil/func_test.go +++ b/pkg/util/funcutil/func_test.go @@ -91,11 +91,33 @@ func Test_GetLocalIP(t *testing.T) { } func Test_GetIP(t *testing.T) { - ip := GetIP("") - assert.NotNil(t, ip) - assert.NotZero(t, len(ip)) - ip = GetIP("127.0.0") - assert.Equal(t, ip, "127.0.0") + t.Run("empty_fallback_auto", func(t *testing.T) { + ip := GetIP("") + assert.NotNil(t, ip) + assert.NotZero(t, len(ip)) + }) + + t.Run("valid_ip", func(t *testing.T) { + assert.NotPanics(t, func() { + ip := GetIP("8.8.8.8") + assert.Equal(t, "8.8.8.8", ip) + }) + }) + + t.Run("invalid_ip", func(t *testing.T) { + assert.NotPanics(t, func() { + ip := GetIP("null") + assert.Equal(t, "null", ip) + }, "non ip format, could be hostname or service name") + + assert.Panics(t, func() { + GetIP("0.0.0.0") + }, "input is unspecified ip address, panicking") + + assert.Panics(t, func() { + GetIP("224.0.0.1") + }, "input is multicast ip address, panicking") + }) } func Test_ParseIndexParamsMap(t *testing.T) { From cae9e1c732d908c07b2757218a6f248646dce5f0 Mon Sep 17 00:00:00 2001 From: Zhen Ye Date: Thu, 7 Nov 2024 10:26:26 +0800 Subject: [PATCH 20/40] fix: drop collection failed if enable streaming service (#37444) issue: #36858 - Start channel manager on datacoord, but with empty assign policy in streaming service. - Make collection at dropping state can be recovered by flusher to make sure that milvus consume the dropCollection message. - Add backoff for flusher lifetime. - remove the proxy watcher from timetick at rootcoord in streaming service. Also see the better fixup: #37176 --------- Signed-off-by: chyezh --- internal/datacoord/channel_manager.go | 31 ++++++--- internal/datacoord/channel_manager_factory.go | 14 ++++ internal/datacoord/channel_store.go | 7 +- internal/datacoord/channel_store_test.go | 3 +- internal/datacoord/compaction.go | 17 +---- internal/datacoord/compaction_test.go | 52 +------------- internal/datacoord/policy.go | 4 ++ internal/datacoord/server.go | 8 ++- internal/datacoord/server_test.go | 2 +- internal/datacoord/services.go | 19 ++++-- internal/datacoord/services_test.go | 19 +++--- .../flushcommon/pipeline/data_sync_service.go | 8 ++- .../pipeline/flow_graph_time_tick_node.go | 13 ++++ internal/rootcoord/create_collection_task.go | 18 ++--- .../rootcoord/create_collection_task_test.go | 14 +++- internal/rootcoord/root_coord.go | 29 +++++--- internal/rootcoord/timeticksync.go | 2 +- .../flusher/flusherimpl/channel_lifetime.go | 4 +- .../flusher/flusherimpl/flusher_impl.go | 68 +++++++++++++------ .../flusher/flusherimpl/flusher_impl_test.go | 8 +++ .../server/wal/adaptor/wal_test.go | 7 +- .../wal/interceptors/ddl/ddl_interceptor.go | 13 +++- 22 files changed, 213 insertions(+), 147 deletions(-) diff --git a/internal/datacoord/channel_manager.go b/internal/datacoord/channel_manager.go index 01a9a0162552d..c413cb77b0cee 100644 --- a/internal/datacoord/channel_manager.go +++ b/internal/datacoord/channel_manager.go @@ -88,8 +88,8 @@ type ChannelBGChecker func(ctx context.Context) // ChannelmanagerOpt is to set optional parameters in channel manager. type ChannelmanagerOpt func(c *ChannelManagerImpl) -func withFactoryV2(f ChannelPolicyFactory) ChannelmanagerOpt { - return func(c *ChannelManagerImpl) { c.factory = f } +func withEmptyPolicyFactory() ChannelmanagerOpt { + return func(c *ChannelManagerImpl) { c.factory = NewEmptyChannelPolicyFactory() } } func withCheckerV2() ChannelmanagerOpt { @@ -161,7 +161,7 @@ func (m *ChannelManagerImpl) Startup(ctx context.Context, legacyNodes, allNodes m.finishRemoveChannel(info.NodeID, lo.Values(info.Channels)...) } - if m.balanceCheckLoop != nil && !streamingutil.IsStreamingServiceEnabled() { + if m.balanceCheckLoop != nil { log.Info("starting channel balance loop") m.wg.Add(1) go func() { @@ -330,20 +330,19 @@ func (m *ChannelManagerImpl) Balance() { } func (m *ChannelManagerImpl) Match(nodeID UniqueID, channel string) bool { + m.mu.RLock() + defer m.mu.RUnlock() + if streamingutil.IsStreamingServiceEnabled() { // Skip the channel matching check since the // channel manager no longer manages channels in streaming mode. - return true + // Only check if the channel exists. + return m.store.HasChannel(channel) } - - m.mu.RLock() - defer m.mu.RUnlock() - info := m.store.GetNode(nodeID) if info == nil { return false } - _, ok := info.Channels[channel] return ok } @@ -413,6 +412,7 @@ func (m *ChannelManagerImpl) FindWatcher(channel string) (UniqueID, error) { func (m *ChannelManagerImpl) removeChannel(nodeID int64, ch RWChannel) error { op := NewChannelOpSet(NewChannelOp(nodeID, Delete, ch)) log.Info("remove channel assignment", + zap.Int64("nodeID", nodeID), zap.String("channel", ch.GetName()), zap.Int64("assignment", nodeID), zap.Int64("collectionID", ch.GetCollectionID())) @@ -448,7 +448,8 @@ func (m *ChannelManagerImpl) AdvanceChannelState(ctx context.Context) { m.mu.RUnlock() // Processing standby channels - updatedStandbys := m.advanceStandbys(ctx, standbys) + updatedStandbys := false + updatedStandbys = m.advanceStandbys(ctx, standbys) updatedToCheckes := m.advanceToChecks(ctx, toChecks) updatedToNotifies := m.advanceToNotifies(ctx, toNotifies) @@ -485,8 +486,15 @@ func (m *ChannelManagerImpl) advanceStandbys(_ context.Context, standbys []*Node } validChannels[chName] = ch } - nodeAssign.Channels = validChannels + // If streaming service is enabled, the channel manager no longer manages channels. + // So the standby channels shouldn't be processed by channel manager, + // but the remove channel operation should be executed. + // TODO: ChannelManager can be removed in future at 3.0.0. + if streamingutil.IsStreamingServiceEnabled() { + continue + } + nodeAssign.Channels = validChannels if len(nodeAssign.Channels) == 0 { continue } @@ -496,6 +504,7 @@ func (m *ChannelManagerImpl) advanceStandbys(_ context.Context, standbys []*Node log.Warn("Reassign channels fail", zap.Int64("nodeID", nodeAssign.NodeID), zap.Strings("channels", chNames), + zap.Error(err), ) continue } diff --git a/internal/datacoord/channel_manager_factory.go b/internal/datacoord/channel_manager_factory.go index 88171beef3e80..3b5ae326cedef 100644 --- a/internal/datacoord/channel_manager_factory.go +++ b/internal/datacoord/channel_manager_factory.go @@ -39,3 +39,17 @@ func (f *ChannelPolicyFactoryV1) NewBalancePolicy() BalanceChannelPolicy { func (f *ChannelPolicyFactoryV1) NewAssignPolicy() AssignPolicy { return AvgAssignByCountPolicy } + +func NewEmptyChannelPolicyFactory() *EmptyChannelPolicyFactory { + return &EmptyChannelPolicyFactory{} +} + +type EmptyChannelPolicyFactory struct{} + +func (f *EmptyChannelPolicyFactory) NewBalancePolicy() BalanceChannelPolicy { + return EmptyBalancePolicy +} + +func (f *EmptyChannelPolicyFactory) NewAssignPolicy() AssignPolicy { + return EmptyAssignPolicy +} diff --git a/internal/datacoord/channel_store.go b/internal/datacoord/channel_store.go index cc4140ed65b9d..d3b6a2b3914ac 100644 --- a/internal/datacoord/channel_store.go +++ b/internal/datacoord/channel_store.go @@ -74,6 +74,8 @@ type RWChannelStore interface { UpdateState(isSuccessful bool, channels ...RWChannel) // SegLegacyChannelByNode is used by StateChannelStore only SetLegacyChannelByNode(nodeIDs ...int64) + + HasChannel(channel string) bool } // ChannelOpTypeNames implements zap log marshaller for ChannelOpSet. @@ -545,10 +547,7 @@ func (c *StateChannelStore) updateMetaMemoryForSingleOp(op *ChannelOp) error { storedChannel.setState(ToWatch) } case Delete: // Remove Channel - // if not Delete from bufferID, remove from channel - if op.NodeID != bufferID { - c.removeAssignment(op.NodeID, ch.GetName()) - } + c.removeAssignment(op.NodeID, ch.GetName()) default: log.Error("unknown opType in updateMetaMemoryForSingleOp", zap.Any("type", op.Type)) } diff --git a/internal/datacoord/channel_store_test.go b/internal/datacoord/channel_store_test.go index 191a0e040f024..0d8d263a1bbbf 100644 --- a/internal/datacoord/channel_store_test.go +++ b/internal/datacoord/channel_store_test.go @@ -37,7 +37,8 @@ func (s *StateChannelStoreSuite) SetupTest() { func generateWatchInfo(name string, state datapb.ChannelWatchState) *datapb.ChannelWatchInfo { return &datapb.ChannelWatchInfo{ Vchan: &datapb.VchannelInfo{ - ChannelName: name, + CollectionID: 1, + ChannelName: name, }, Schema: &schemapb.CollectionSchema{}, State: state, diff --git a/internal/datacoord/compaction.go b/internal/datacoord/compaction.go index 7a4093b4b3065..c20457289cbe4 100644 --- a/internal/datacoord/compaction.go +++ b/internal/datacoord/compaction.go @@ -86,7 +86,6 @@ type compactionPlanHandler struct { meta CompactionMeta allocator allocator.Allocator - chManager ChannelManager sessions session.DataNodeManager cluster Cluster analyzeScheduler *taskScheduler @@ -180,7 +179,7 @@ func (c *compactionPlanHandler) getCompactionTasksNumBySignalID(triggerID int64) return cnt } -func newCompactionPlanHandler(cluster Cluster, sessions session.DataNodeManager, cm ChannelManager, meta CompactionMeta, +func newCompactionPlanHandler(cluster Cluster, sessions session.DataNodeManager, meta CompactionMeta, allocator allocator.Allocator, analyzeScheduler *taskScheduler, handler Handler, ) *compactionPlanHandler { // Higher capacity will have better ordering in priority, but consumes more memory. @@ -188,7 +187,6 @@ func newCompactionPlanHandler(cluster Cluster, sessions session.DataNodeManager, capacity := paramtable.Get().DataCoordCfg.CompactionTaskQueueCapacity.GetAsInt() return &compactionPlanHandler{ queueTasks: *NewCompactionQueue(capacity, getPrioritizer()), - chManager: cm, meta: meta, sessions: sessions, allocator: allocator, @@ -680,19 +678,6 @@ func (c *compactionPlanHandler) pickAnyNode(nodeSlots map[int64]int64, task Comp return nodeID, useSlot } -func (c *compactionPlanHandler) pickShardNode(nodeSlots map[int64]int64, t CompactionTask) int64 { - nodeID, err := c.chManager.FindWatcher(t.GetTaskProto().GetChannel()) - if err != nil { - log.Info("failed to find watcher", zap.Int64("planID", t.GetTaskProto().GetPlanID()), zap.Error(err)) - return NullNodeID - } - - if nodeSlots[nodeID] > 0 { - return nodeID - } - return NullNodeID -} - // isFull return true if the task pool is full func (c *compactionPlanHandler) isFull() bool { return c.queueTasks.Len() >= c.queueTasks.capacity diff --git a/internal/datacoord/compaction_test.go b/internal/datacoord/compaction_test.go index b0fb0c97f0eca..19d33f8a016b2 100644 --- a/internal/datacoord/compaction_test.go +++ b/internal/datacoord/compaction_test.go @@ -57,7 +57,7 @@ func (s *CompactionPlanHandlerSuite) SetupTest() { s.mockCm = NewMockChannelManager(s.T()) s.mockSessMgr = session.NewMockDataNodeManager(s.T()) s.cluster = NewMockCluster(s.T()) - s.handler = newCompactionPlanHandler(s.cluster, s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc, nil, nil) + s.handler = newCompactionPlanHandler(s.cluster, s.mockSessMgr, s.mockMeta, s.mockAlloc, nil, nil) } func (s *CompactionPlanHandlerSuite) TestScheduleEmpty() { @@ -449,54 +449,6 @@ func (s *CompactionPlanHandlerSuite) TestPickAnyNodeForClusteringTask() { s.Equal(int64(NullNodeID), node) } -func (s *CompactionPlanHandlerSuite) TestPickShardNode() { - s.SetupTest() - nodeSlots := map[int64]int64{ - 100: 2, - 101: 6, - } - - t1 := newMixCompactionTask(&datapb.CompactionTask{ - PlanID: 19530, - Type: datapb.CompactionType_MixCompaction, - Channel: "ch-01", - NodeID: 1, - }, nil, s.mockMeta, s.mockSessMgr) - t1.plan = &datapb.CompactionPlan{ - PlanID: 19530, - Channel: "ch-01", - Type: datapb.CompactionType_MixCompaction, - } - - t2 := newMixCompactionTask(&datapb.CompactionTask{ - PlanID: 19531, - Type: datapb.CompactionType_MixCompaction, - Channel: "ch-02", - NodeID: 1, - }, nil, s.mockMeta, s.mockSessMgr) - t2.plan = &datapb.CompactionPlan{ - PlanID: 19531, - Channel: "ch-02", - Type: datapb.CompactionType_Level0DeleteCompaction, - } - - s.mockCm.EXPECT().FindWatcher(mock.Anything).RunAndReturn(func(channel string) (int64, error) { - if channel == "ch-01" { - return 100, nil - } - if channel == "ch-02" { - return 101, nil - } - return 1, nil - }).Twice() - - node := s.handler.pickShardNode(nodeSlots, t1) - s.Equal(int64(100), node) - - node = s.handler.pickShardNode(nodeSlots, t2) - s.Equal(int64(101), node) -} - func (s *CompactionPlanHandlerSuite) TestRemoveTasksByChannel() { s.SetupTest() ch := "ch1" @@ -604,7 +556,7 @@ func (s *CompactionPlanHandlerSuite) TestExecCompactionPlan() { s.SetupTest() s.mockMeta.EXPECT().CheckAndSetSegmentsCompacting(mock.Anything).Return(true, true).Maybe() s.mockMeta.EXPECT().SaveCompactionTask(mock.Anything).Return(nil) - handler := newCompactionPlanHandler(nil, s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc, nil, nil) + handler := newCompactionPlanHandler(nil, s.mockSessMgr, s.mockMeta, s.mockAlloc, nil, nil) task := &datapb.CompactionTask{ TriggerID: 1, diff --git a/internal/datacoord/policy.go b/internal/datacoord/policy.go index 434fef0434107..43c256807bfa3 100644 --- a/internal/datacoord/policy.go +++ b/internal/datacoord/policy.go @@ -145,6 +145,10 @@ func AvgBalanceChannelPolicy(cluster Assignments) *ChannelOpSet { // ExclusiveNodes means donot assign channels to these nodes. type AssignPolicy func(currentCluster Assignments, toAssign *NodeChannelInfo, exclusiveNodes []int64) *ChannelOpSet +func EmptyAssignPolicy(currentCluster Assignments, toAssign *NodeChannelInfo, execlusiveNodes []int64) *ChannelOpSet { + return nil +} + func AvgAssignByCountPolicy(currentCluster Assignments, toAssign *NodeChannelInfo, execlusiveNodes []int64) *ChannelOpSet { var ( toCluster Assignments diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 3046183c2de68..a1a9db46bea9c 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -506,7 +506,11 @@ func (s *Server) initCluster() error { s.sessionManager = session.NewDataNodeManagerImpl(session.WithDataNodeCreator(s.dataNodeCreator)) var err error - s.channelManager, err = NewChannelManager(s.watchClient, s.handler, s.sessionManager, s.idAllocator, withCheckerV2()) + channelManagerOpts := []ChannelmanagerOpt{withCheckerV2()} + if streamingutil.IsStreamingServiceEnabled() { + channelManagerOpts = append(channelManagerOpts, withEmptyPolicyFactory()) + } + s.channelManager, err = NewChannelManager(s.watchClient, s.handler, s.sessionManager, s.idAllocator, channelManagerOpts...) if err != nil { return err } @@ -705,7 +709,7 @@ func (s *Server) initIndexNodeManager() { } func (s *Server) initCompaction() { - s.compactionHandler = newCompactionPlanHandler(s.cluster, s.sessionManager, s.channelManager, s.meta, s.allocator, s.taskScheduler, s.handler) + s.compactionHandler = newCompactionPlanHandler(s.cluster, s.sessionManager, s.meta, s.allocator, s.taskScheduler, s.handler) s.compactionTriggerManager = NewCompactionTriggerManager(s.allocator, s.handler, s.compactionHandler, s.meta) s.compactionTrigger = newCompactionTrigger(s.meta, s.compactionHandler, s.allocator, s.handler, s.indexEngineVersionManager) } diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index 1c39070793e70..d8595570887ab 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -1680,7 +1680,7 @@ func TestGetCompactionState(t *testing.T) { {State: datapb.CompactionTaskState_timeout}, {State: datapb.CompactionTaskState_timeout}, }) - mockHandler := newCompactionPlanHandler(nil, nil, nil, mockMeta, nil, nil, nil) + mockHandler := newCompactionPlanHandler(nil, nil, mockMeta, nil, nil, nil) svr.compactionHandler = mockHandler resp, err := svr.GetCompactionState(context.Background(), &milvuspb.GetCompactionStateRequest{CompactionID: 1}) assert.NoError(t, err) diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index 2fd524fddde91..79b63d964bb11 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -637,10 +637,14 @@ func (s *Server) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtual return resp, nil } - log.Info("DropVChannel plan to remove", zap.String("channel", channel)) - err = s.channelManager.Release(nodeID, channel) - if err != nil { - log.Warn("DropVChannel failed to ReleaseAndRemove", zap.String("channel", channel), zap.Error(err)) + if !streamingutil.IsStreamingServiceEnabled() { + // if streaming service is enabled, the channel manager will never manage the channel. + // so we don't need to release the channel anymore. + log.Info("DropVChannel plan to remove", zap.String("channel", channel)) + err = s.channelManager.Release(nodeID, channel) + if err != nil { + log.Warn("DropVChannel failed to ReleaseAndRemove", zap.String("channel", channel), zap.Error(err)) + } } s.segmentManager.DropSegmentsOfChannel(ctx, channel) s.compactionHandler.removeTasksByChannel(channel) @@ -937,11 +941,14 @@ func (s *Server) GetChannelRecoveryInfo(ctx context.Context, req *datapb.GetChan return resp, nil } collectionID := funcutil.GetCollectionIDFromVChannel(req.GetVchannel()) - collection, err := s.handler.GetCollection(ctx, collectionID) - if err != nil { + // `handler.GetCollection` cannot fetch dropping collection, + // so we use `broker.DescribeCollectionInternal` to get collection info to help fetch dropping collection to get the recovery info. + collection, err := s.broker.DescribeCollectionInternal(ctx, collectionID) + if err := merr.CheckRPCCall(collection, err); err != nil { resp.Status = merr.Status(err) return resp, nil } + channel := NewRWChannel(req.GetVchannel(), collectionID, nil, collection.Schema, 0) // TODO: remove RWChannel, just use vchannel + collectionID channelInfo := s.handler.GetDataVChanPositions(channel, allPartitionID) log.Info("datacoord get channel recovery info", diff --git a/internal/datacoord/services_test.go b/internal/datacoord/services_test.go index 23c620181c58c..50709772a345d 100644 --- a/internal/datacoord/services_test.go +++ b/internal/datacoord/services_test.go @@ -20,6 +20,7 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datacoord/allocator" + "github.com/milvus-io/milvus/internal/datacoord/broker" "github.com/milvus-io/milvus/internal/datacoord/session" "github.com/milvus-io/milvus/internal/metastore/mocks" "github.com/milvus-io/milvus/internal/metastore/model" @@ -1525,11 +1526,9 @@ func TestGetChannelRecoveryInfo(t *testing.T) { s.stateCode.Store(commonpb.StateCode_Healthy) // get collection failed - handler := NewNMockHandler(t) - handler.EXPECT().GetCollection(mock.Anything, mock.Anything). - Return(nil, errors.New("mock err")) - s.handler = handler - assert.NoError(t, err) + broker := broker.NewMockBroker(t) + broker.EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).Return(nil, errors.New("mock err")) + s.broker = broker resp, err = s.GetChannelRecoveryInfo(ctx, &datapb.GetChannelRecoveryInfoRequest{ Vchannel: "ch-1", }) @@ -1547,9 +1546,13 @@ func TestGetChannelRecoveryInfo(t *testing.T) { IndexedSegmentIds: []int64{4}, } - handler = NewNMockHandler(t) - handler.EXPECT().GetCollection(mock.Anything, mock.Anything). - Return(&collectionInfo{Schema: &schemapb.CollectionSchema{}}, nil) + broker.EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything).Unset() + broker.EXPECT().DescribeCollectionInternal(mock.Anything, mock.Anything). + Return(&milvuspb.DescribeCollectionResponse{ + Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, + Schema: &schemapb.CollectionSchema{}, + }, nil) + handler := NewNMockHandler(t) handler.EXPECT().GetDataVChanPositions(mock.Anything, mock.Anything).Return(channelInfo) s.handler = handler diff --git a/internal/flushcommon/pipeline/data_sync_service.go b/internal/flushcommon/pipeline/data_sync_service.go index 983b66b0e6b8d..cba3d2fcac087 100644 --- a/internal/flushcommon/pipeline/data_sync_service.go +++ b/internal/flushcommon/pipeline/data_sync_service.go @@ -77,6 +77,7 @@ type nodeConfig struct { vChannelName string metacache metacache.MetaCache serverID typeutil.UniqueID + dropCallback func() } // Start the flow graph in dataSyncService @@ -222,6 +223,7 @@ func getServiceWithChannel(initCtx context.Context, params *util.PipelineParams, info *datapb.ChannelWatchInfo, metacache metacache.MetaCache, unflushed, flushed []*datapb.SegmentInfo, input <-chan *msgstream.MsgPack, wbTaskObserverCallback writebuffer.TaskObserverCallback, + dropCallback func(), ) (*DataSyncService, error) { var ( channelName = info.GetVchan().GetChannelName() @@ -238,6 +240,7 @@ func getServiceWithChannel(initCtx context.Context, params *util.PipelineParams, vChannelName: channelName, metacache: metacache, serverID: serverID, + dropCallback: dropCallback, } ctx, cancel := context.WithCancel(params.Ctx) @@ -355,7 +358,7 @@ func NewDataSyncService(initCtx context.Context, pipelineParams *util.PipelinePa if metaCache, err = getMetaCacheWithTickler(initCtx, pipelineParams, info, tickler, unflushedSegmentInfos, flushedSegmentInfos); err != nil { return nil, err } - return getServiceWithChannel(initCtx, pipelineParams, info, metaCache, unflushedSegmentInfos, flushedSegmentInfos, nil, nil) + return getServiceWithChannel(initCtx, pipelineParams, info, metaCache, unflushedSegmentInfos, flushedSegmentInfos, nil, nil, nil) } func NewStreamingNodeDataSyncService( @@ -364,6 +367,7 @@ func NewStreamingNodeDataSyncService( info *datapb.ChannelWatchInfo, input <-chan *msgstream.MsgPack, wbTaskObserverCallback writebuffer.TaskObserverCallback, + dropCallback func(), ) (*DataSyncService, error) { // recover segment checkpoints var ( @@ -389,7 +393,7 @@ func NewStreamingNodeDataSyncService( if metaCache, err = getMetaCacheForStreaming(initCtx, pipelineParams, info, unflushedSegmentInfos, flushedSegmentInfos); err != nil { return nil, err } - return getServiceWithChannel(initCtx, pipelineParams, info, metaCache, unflushedSegmentInfos, flushedSegmentInfos, input, wbTaskObserverCallback) + return getServiceWithChannel(initCtx, pipelineParams, info, metaCache, unflushedSegmentInfos, flushedSegmentInfos, input, wbTaskObserverCallback, dropCallback) } func NewDataSyncServiceWithMetaCache(metaCache metacache.MetaCache) *DataSyncService { diff --git a/internal/flushcommon/pipeline/flow_graph_time_tick_node.go b/internal/flushcommon/pipeline/flow_graph_time_tick_node.go index 985dd39c2a06b..9f720cbaa4e5a 100644 --- a/internal/flushcommon/pipeline/flow_graph_time_tick_node.go +++ b/internal/flushcommon/pipeline/flow_graph_time_tick_node.go @@ -45,6 +45,7 @@ type ttNode struct { lastUpdateTime *atomic.Time cpUpdater *util.ChannelCheckpointUpdater dropMode *atomic.Bool + dropCallback func() } // Name returns node name, implementing flowgraph.Node @@ -72,6 +73,13 @@ func (ttn *ttNode) Operate(in []Msg) []Msg { fgMsg := in[0].(*FlowGraphMsg) if fgMsg.dropCollection { ttn.dropMode.Store(true) + if ttn.dropCallback != nil { + defer func() { + // if drop collection setup, call drop callback + // For streaming node to cleanup the resources. + ttn.dropCallback() + }() + } } // skip updating checkpoint for drop collection @@ -83,6 +91,10 @@ func (ttn *ttNode) Operate(in []Msg) []Msg { curTs, _ := tsoutil.ParseTS(fgMsg.TimeRange.TimestampMax) if fgMsg.IsCloseMsg() { + if ttn.dropMode.Load() { + // if drop collection setup, skip update checkpoint to avoid update dirty checkpoint. + return in + } if len(fgMsg.EndPositions) > 0 { channelPos, _, err := ttn.writeBufferManager.GetCheckpoint(ttn.vChannelName) if err != nil { @@ -141,6 +153,7 @@ func newTTNode(config *nodeConfig, wbManager writebuffer.BufferManager, cpUpdate lastUpdateTime: atomic.NewTime(time.Time{}), // set to Zero to update channel checkpoint immediately after fg started cpUpdater: cpUpdater, dropMode: atomic.NewBool(false), + dropCallback: config.dropCallback, } return tt, nil diff --git a/internal/rootcoord/create_collection_task.go b/internal/rootcoord/create_collection_task.go index 9c83bc7018f51..511adbbc66005 100644 --- a/internal/rootcoord/create_collection_task.go +++ b/internal/rootcoord/create_collection_task.go @@ -550,13 +550,6 @@ func (t *createCollectionTask) Execute(ctx context.Context) error { vchanNames := t.channels.virtualChannels chanNames := t.channels.physicalChannels - startPositions, err := t.addChannelsAndGetStartPositions(ctx, ts) - if err != nil { - // ugly here, since we must get start positions first. - t.core.chanTimeTick.removeDmlChannels(t.channels.physicalChannels...) - return err - } - partitions := make([]*model.Partition, len(partIDs)) for i, partID := range partIDs { partitions[i] = &model.Partition{ @@ -580,7 +573,6 @@ func (t *createCollectionTask) Execute(ctx context.Context) error { PhysicalChannelNames: chanNames, ShardsNum: t.Req.ShardsNum, ConsistencyLevel: t.Req.ConsistencyLevel, - StartPositions: toKeyDataPairs(startPositions), CreateTime: ts, State: pb.CollectionState_CollectionCreating, Partitions: partitions, @@ -604,6 +596,16 @@ func (t *createCollectionTask) Execute(ctx context.Context) error { return nil } + // TODO: The create collection is not idempotent for other component, such as wal. + // we need to make the create collection operation must success after some persistent operation, refactor it in future. + startPositions, err := t.addChannelsAndGetStartPositions(ctx, ts) + if err != nil { + // ugly here, since we must get start positions first. + t.core.chanTimeTick.removeDmlChannels(t.channels.physicalChannels...) + return err + } + collInfo.StartPositions = toKeyDataPairs(startPositions) + undoTask := newBaseUndoTask(t.core.stepExecutor) undoTask.AddStep(&expireCacheStep{ baseStep: baseStep{core: t.core}, diff --git a/internal/rootcoord/create_collection_task_test.go b/internal/rootcoord/create_collection_task_test.go index 1986b99ce4921..46bad4b284e1a 100644 --- a/internal/rootcoord/create_collection_task_test.go +++ b/internal/rootcoord/create_collection_task_test.go @@ -930,13 +930,25 @@ func Test_createCollectionTask_Execute(t *testing.T) { ticker := newTickerWithMockFailStream() shardNum := 2 pchans := ticker.getDmlChannelNames(shardNum) - core := newTestCore(withTtSynchronizer(ticker)) + meta := newMockMetaTable() + meta.GetCollectionByNameFunc = func(ctx context.Context, collectionName string, ts Timestamp) (*model.Collection, error) { + return nil, errors.New("error mock GetCollectionByName") + } + core := newTestCore(withTtSynchronizer(ticker), withMeta(meta)) + schema := &schemapb.CollectionSchema{Name: "", Fields: []*schemapb.FieldSchema{{}}} task := &createCollectionTask{ baseTask: newBaseTask(context.Background(), core), channels: collectionChannels{ physicalChannels: pchans, virtualChannels: []string{funcutil.GenRandomStr(), funcutil.GenRandomStr()}, }, + Req: &milvuspb.CreateCollectionRequest{ + Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection}, + CollectionName: "", + Schema: []byte{}, + ShardsNum: int32(shardNum), + }, + schema: schema, } err := task.Execute(context.Background()) assert.Error(t, err) diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index ad6ef5fc597f2..3efd75721c8ad 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -462,13 +462,22 @@ func (c *Core) initInternal() error { c.garbageCollector = newBgGarbageCollector(c) c.stepExecutor = newBgStepExecutor(c.ctx) - c.proxyWatcher = proxyutil.NewProxyWatcher( - c.etcdCli, - c.chanTimeTick.initSessions, - c.proxyClientManager.AddProxyClients, - ) - c.proxyWatcher.AddSessionFunc(c.chanTimeTick.addSession, c.proxyClientManager.AddProxyClient) - c.proxyWatcher.DelSessionFunc(c.chanTimeTick.delSession, c.proxyClientManager.DelProxyClient) + if !streamingutil.IsStreamingServiceEnabled() { + c.proxyWatcher = proxyutil.NewProxyWatcher( + c.etcdCli, + c.chanTimeTick.initSessions, + c.proxyClientManager.AddProxyClients, + ) + c.proxyWatcher.AddSessionFunc(c.chanTimeTick.addSession, c.proxyClientManager.AddProxyClient) + c.proxyWatcher.DelSessionFunc(c.chanTimeTick.delSession, c.proxyClientManager.DelProxyClient) + } else { + c.proxyWatcher = proxyutil.NewProxyWatcher( + c.etcdCli, + c.proxyClientManager.AddProxyClients, + ) + c.proxyWatcher.AddSessionFunc(c.proxyClientManager.AddProxyClient) + c.proxyWatcher.DelSessionFunc(c.proxyClientManager.DelProxyClient) + } log.Info("init proxy manager done") c.metricsCacheManager = metricsinfo.NewMetricsCacheManager() @@ -730,11 +739,11 @@ func (c *Core) startInternal() error { } func (c *Core) startServerLoop() { - c.wg.Add(2) - go c.startTimeTickLoop() + c.wg.Add(1) go c.tsLoop() if !streamingutil.IsStreamingServiceEnabled() { - c.wg.Add(1) + c.wg.Add(2) + go c.startTimeTickLoop() go c.chanTimeTick.startWatch(&c.wg) } } diff --git a/internal/rootcoord/timeticksync.go b/internal/rootcoord/timeticksync.go index 22eed18acb057..7dc0f81fc77e3 100644 --- a/internal/rootcoord/timeticksync.go +++ b/internal/rootcoord/timeticksync.go @@ -163,7 +163,7 @@ func (t *timetickSync) sendToChannel() bool { } } - if len(idleSessionList) > 0 && !streamingutil.IsStreamingServiceEnabled() { + if len(idleSessionList) > 0 { // give warning every 2 second if not get ttMsg from source sessions if maxCnt%10 == 0 { log.Warn("session idle for long time", zap.Any("idle list", idleSessionList), diff --git a/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go b/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go index 15819bc6591c5..afcbace6a8dfc 100644 --- a/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go +++ b/internal/streamingnode/server/flusher/flusherimpl/channel_lifetime.go @@ -129,7 +129,9 @@ func (c *channelLifetime) Run() error { BinLogFileCounterIncr: uint64(len(insertLogs)), }) } - }) + }, + func() { go func() { c.Cancel() }() }, + ) if err != nil { return err } diff --git a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go index 15b03e1d9f71d..f87acd7353c5e 100644 --- a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go +++ b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl.go @@ -22,6 +22,7 @@ import ( "time" "github.com/cockroachdb/errors" + "go.uber.org/atomic" "go.uber.org/zap" "github.com/milvus-io/milvus/internal/flushcommon/broker" @@ -130,37 +131,64 @@ func (f *flusherImpl) Start() { go f.cpUpdater.Start() go func() { defer f.stopWg.Done() + backoff := typeutil.NewBackoffTimer(typeutil.BackoffTimerConfig{ + Default: 5 * time.Second, + Backoff: typeutil.BackoffConfig{ + InitialInterval: 50 * time.Millisecond, + Multiplier: 2.0, + MaxInterval: 5 * time.Second, + }, + }) + + var nextTimer <-chan time.Time for { select { case <-f.stopChan.CloseCh(): log.Info("flusher exited") return case <-f.notifyCh: - futures := make([]*conc.Future[any], 0) - f.channelLifetimes.Range(func(vchannel string, lifetime ChannelLifetime) bool { - future := GetExecPool().Submit(func() (any, error) { - err := lifetime.Run() - if errors.Is(err, errChannelLifetimeUnrecoverable) { - log.Warn("channel lifetime is unrecoverable, removed", zap.String("vchannel", vchannel)) - f.channelLifetimes.Remove(vchannel) - return nil, nil - } - if err != nil { - log.Warn("build pipeline failed", zap.String("vchannel", vchannel), zap.Error(err)) - f.notify() // Notify to trigger retry. - return nil, err - } - return nil, nil - }) - futures = append(futures, future) - return true - }) - _ = conc.AwaitAll(futures...) + nextTimer = f.handle(backoff) + case <-nextTimer: + nextTimer = f.handle(backoff) } } }() } +func (f *flusherImpl) handle(backoff *typeutil.BackoffTimer) <-chan time.Time { + futures := make([]*conc.Future[any], 0) + failureCnt := atomic.NewInt64(0) + f.channelLifetimes.Range(func(vchannel string, lifetime ChannelLifetime) bool { + future := GetExecPool().Submit(func() (any, error) { + err := lifetime.Run() + if errors.Is(err, errChannelLifetimeUnrecoverable) { + log.Warn("channel lifetime is unrecoverable, removed", zap.String("vchannel", vchannel)) + f.channelLifetimes.Remove(vchannel) + return nil, nil + } + if err != nil { + log.Warn("build pipeline failed", zap.String("vchannel", vchannel), zap.Error(err)) + failureCnt.Inc() + return nil, err + } + return nil, nil + }) + futures = append(futures, future) + return true + }) + _ = conc.BlockOnAll(futures...) + + if failureCnt.Load() > 0 { + backoff.EnableBackoff() + nextTimer, interval := backoff.NextTimer() + log.Warn("flusher lifetime trasition failed, retry with backoff...", zap.Int64("failureCnt", failureCnt.Load()), zap.Duration("interval", interval)) + return nextTimer + } + // There's a failure, do no backoff. + backoff.DisableBackoff() + return nil +} + func (f *flusherImpl) Stop() { f.stopChan.Close() f.stopWg.Wait() diff --git a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go index b737da1fef606..aef723e7a59f6 100644 --- a/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go +++ b/internal/streamingnode/server/flusher/flusherimpl/flusher_impl_test.go @@ -25,6 +25,7 @@ import ( "github.com/samber/lo" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" + "go.uber.org/atomic" "google.golang.org/grpc" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" @@ -39,6 +40,7 @@ import ( "github.com/milvus-io/milvus/internal/streamingnode/server/resource" "github.com/milvus-io/milvus/internal/streamingnode/server/wal" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/paramtable" ) @@ -48,9 +50,15 @@ func init() { func newMockDatacoord(t *testing.T, maybe bool) *mocks.MockDataCoordClient { datacoord := mocks.NewMockDataCoordClient(t) + failureCnt := atomic.NewInt32(20) expect := datacoord.EXPECT().GetChannelRecoveryInfo(mock.Anything, mock.Anything).RunAndReturn( func(ctx context.Context, request *datapb.GetChannelRecoveryInfoRequest, option ...grpc.CallOption, ) (*datapb.GetChannelRecoveryInfoResponse, error) { + if failureCnt.Dec() > 0 { + return &datapb.GetChannelRecoveryInfoResponse{ + Status: merr.Status(merr.ErrCollectionNotFound), + }, nil + } messageID := 1 b := make([]byte, 8) common.Endian.PutUint64(b, uint64(messageID)) diff --git a/internal/streamingnode/server/wal/adaptor/wal_test.go b/internal/streamingnode/server/wal/adaptor/wal_test.go index 3f3f2cdc7b81e..7f996ff58c2af 100644 --- a/internal/streamingnode/server/wal/adaptor/wal_test.go +++ b/internal/streamingnode/server/wal/adaptor/wal_test.go @@ -60,8 +60,8 @@ func initResourceForTest(t *testing.T) { flusher := mock_flusher.NewMockFlusher(t) flusher.EXPECT().RegisterPChannel(mock.Anything, mock.Anything).Return(nil).Maybe() flusher.EXPECT().UnregisterPChannel(mock.Anything).Return().Maybe() - flusher.EXPECT().RegisterVChannel(mock.Anything, mock.Anything).Return() - flusher.EXPECT().UnregisterVChannel(mock.Anything).Return() + flusher.EXPECT().RegisterVChannel(mock.Anything, mock.Anything).Return().Maybe() + flusher.EXPECT().UnregisterVChannel(mock.Anything).Return().Maybe() resource.InitForTest( t, @@ -194,6 +194,9 @@ func (f *testOneWALFramework) testSendCreateCollection(ctx context.Context, w wa } func (f *testOneWALFramework) testSendDropCollection(ctx context.Context, w wal.WAL) { + // Here, the drop colllection is conflict with the txn message, so we need to wait for a while. + // TODO: fix it in the redo interceptor. + time.Sleep(2 * time.Second) // drop collection after test dropMsg, err := message.NewDropCollectionMessageBuilderV1(). WithHeader(&message.DropCollectionMessageHeader{ diff --git a/internal/streamingnode/server/wal/interceptors/ddl/ddl_interceptor.go b/internal/streamingnode/server/wal/interceptors/ddl/ddl_interceptor.go index 8f2a17b6277ec..7cb1ee12384ab 100644 --- a/internal/streamingnode/server/wal/interceptors/ddl/ddl_interceptor.go +++ b/internal/streamingnode/server/wal/interceptors/ddl/ddl_interceptor.go @@ -34,14 +34,21 @@ type ddlAppendInterceptor struct { } // DoAppend implements AppendInterceptor. -func (d *ddlAppendInterceptor) DoAppend(ctx context.Context, msg message.MutableMessage, append interceptors.Append) (msgID message.MessageID, err error) { +func (d *ddlAppendInterceptor) DoAppend(ctx context.Context, msg message.MutableMessage, append interceptors.Append) (message.MessageID, error) { + // send the create collection message. + msgID, err := append(ctx, msg) + if err != nil { + return msgID, err + } + switch msg.MessageType() { case message.MessageTypeCreateCollection: resource.Resource().Flusher().RegisterVChannel(msg.VChannel(), d.wal.Get()) case message.MessageTypeDropCollection: - resource.Resource().Flusher().UnregisterVChannel(msg.VChannel()) + // TODO: unregister vchannel, cannot unregister vchannel now. + // Wait for PR: https://github.com/milvus-io/milvus/pull/37176 } - return append(ctx, msg) + return msgID, nil } // Close implements BasicInterceptor. From aed3b94b5d777c11195b9091ceb775de1beb9002 Mon Sep 17 00:00:00 2001 From: "cai.zhang" Date: Thu, 7 Nov 2024 10:44:25 +0800 Subject: [PATCH 21/40] enhance: Refine error message for contains array (#37383) issue: #36221 Signed-off-by: Cai Zhang --- internal/parser/planparserv2/plan_parser_v2_test.go | 9 +++++---- internal/parser/planparserv2/utils.go | 11 ++++++++--- tests/python_client/testcases/test_mix_scenes.py | 4 ++-- 3 files changed, 15 insertions(+), 9 deletions(-) diff --git a/internal/parser/planparserv2/plan_parser_v2_test.go b/internal/parser/planparserv2/plan_parser_v2_test.go index d1e55baf17d37..febff4be83180 100644 --- a/internal/parser/planparserv2/plan_parser_v2_test.go +++ b/internal/parser/planparserv2/plan_parser_v2_test.go @@ -997,7 +997,6 @@ func Test_JSONContains(t *testing.T) { `JSON_contains(JSONField, 5)`, `json_contains(A, [1,2,3])`, `array_contains(A, [1,2,3])`, - `array_contains(ArrayField, [1,2,3])`, `array_contains(ArrayField, 1)`, `json_contains(JSONField, 5)`, `json_contains($meta, 1)`, @@ -1030,6 +1029,8 @@ func Test_InvalidJSONContains(t *testing.T) { `json_contains(A, StringField > 5)`, `json_contains(A)`, `json_contains(A, 5, C)`, + `json_contains(ArrayField, "abc")`, + `json_contains(ArrayField, [1,2])`, } for _, expr = range exprs { _, err = CreateSearchPlan(schema, expr, "FloatVectorField", &planpb.QueryInfo{ @@ -1135,7 +1136,7 @@ func Test_JSONContainsAll(t *testing.T) { `JSON_CONTAINS_ALL(A, [1,"2",3.0])`, `array_contains_all(ArrayField, [1,2,3])`, `array_contains_all(ArrayField, [1])`, - `json_contains_all(ArrayField, [1,2,3])`, + `array_contains_all(ArrayField, [1,2,3])`, } for _, expr = range exprs { plan, err = CreateSearchPlan(schema, expr, "FloatVectorField", &planpb.QueryInfo{ @@ -1157,6 +1158,7 @@ func Test_JSONContainsAll(t *testing.T) { `JSON_CONTAINS_ALL(A[""], [1,2,3])`, `JSON_CONTAINS_ALL(Int64Field, [1,2,3])`, `JSON_CONTAINS_ALL(A, B)`, + `JSON_CONTAINS_ALL(ArrayField, [[1,2,3]])`, } for _, expr = range invalidExprs { _, err = CreateSearchPlan(schema, expr, "FloatVectorField", &planpb.QueryInfo{ @@ -1180,8 +1182,6 @@ func Test_JSONContainsAny(t *testing.T) { `json_contains_any(A, [1,"2",3.0])`, `JSON_CONTAINS_ANY(A, [1,"2",3.0])`, `JSON_CONTAINS_ANY(ArrayField, [1,2,3])`, - `JSON_CONTAINS_ANY(ArrayField, [3,4,5])`, - `JSON_CONTAINS_ANY(ArrayField, [1,2,3])`, } for _, expr = range exprs { plan, err = CreateSearchPlan(schema, expr, "FloatVectorField", &planpb.QueryInfo{ @@ -1202,6 +1202,7 @@ func Test_JSONContainsAny(t *testing.T) { `JSON_CONTAINS_ANY(A, [2>>a])`, `JSON_CONTAINS_ANY(A[""], [1,2,3])`, `JSON_CONTAINS_ANY(Int64Field, [1,2,3])`, + `JSON_CONTAINS_ANY(ArrayField, [[1,2,3]])`, `JSON_CONTAINS_ANY(A, B)`, } for _, expr = range invalidExprs { diff --git a/internal/parser/planparserv2/utils.go b/internal/parser/planparserv2/utils.go index 4faef470dd7c1..61eeca1b86875 100644 --- a/internal/parser/planparserv2/utils.go +++ b/internal/parser/planparserv2/utils.go @@ -665,16 +665,21 @@ func checkContainsElement(columnExpr *ExprWithType, op planpb.JSONContainsExpr_J if typeutil.IsArrayType(columnExpr.expr.GetColumnExpr().GetInfo().GetDataType()) { var elements []*planpb.GenericValue if op == planpb.JSONContainsExpr_Contains { - elements = []*planpb.GenericValue{elementValue} + castedValue, err := castValue(columnExpr.expr.GetColumnExpr().GetInfo().GetElementType(), elementValue) + if err != nil { + return err + } + elements = []*planpb.GenericValue{castedValue} } else { elements = elementValue.GetArrayVal().GetArray() } + arrayElementType := columnExpr.expr.GetColumnExpr().GetInfo().GetElementType() for _, value := range elements { valExpr := toValueExpr(value) - if !canBeCompared(columnExpr, valExpr) { + if !canBeComparedDataType(arrayElementType, valExpr.dataType) { return fmt.Errorf("%s operation can't compare between array element type: %s and %s", op.String(), - columnExpr.expr.GetColumnExpr().GetInfo().GetElementType(), + arrayElementType, valExpr.dataType) } } diff --git a/tests/python_client/testcases/test_mix_scenes.py b/tests/python_client/testcases/test_mix_scenes.py index 7fa5b5770b5a0..fce7d816d8e5c 100644 --- a/tests/python_client/testcases/test_mix_scenes.py +++ b/tests/python_client/testcases/test_mix_scenes.py @@ -889,11 +889,11 @@ def test_bitmap_index_query_with_invalid_array_params(self): # query self.collection_wrap.query( expr=Expr.array_contains_any('ARRAY_VARCHAR', [['a', 'b']]).value, limit=1, check_task=CheckTasks.err_res, - check_items={ct.err_code: 65535, ct.err_msg: "fail to Query on QueryNode"}) + check_items={ct.err_code: 1100, ct.err_msg: qem.ParseExpressionFailed}) self.collection_wrap.query( expr=Expr.array_contains_all('ARRAY_VARCHAR', [['a', 'b']]).value, limit=1, check_task=CheckTasks.err_res, - check_items={ct.err_code: 65535, ct.err_msg: "fail to Query on QueryNode"}) + check_items={ct.err_code: 1100, ct.err_msg: qem.ParseExpressionFailed}) self.collection_wrap.query( expr=Expr.array_contains('ARRAY_VARCHAR', [['a', 'b']]).value, limit=1, check_task=CheckTasks.err_res, From f813fb45637633f61ba73ace5d1a0acabd815897 Mon Sep 17 00:00:00 2001 From: congqixia Date: Thu, 7 Nov 2024 10:50:25 +0800 Subject: [PATCH 22/40] enhance: [GoSDK] Remove example in main pkg to make ut script work (#37472) Related to #31293 Example with main func/package causes `go list` command failed with vcs error. This PR removes example files with main package. The example snippet shall be added back in runnable test format. --------- Signed-off-by: Congqi Xia --- client/column/columns_test.go | 4 +- client/example/database/main.go | 89 -------- client/example/playground/main.go | 326 ------------------------------ client/example/rowbase/main.go | 87 -------- scripts/run_go_codecov.sh | 4 +- 5 files changed, 4 insertions(+), 506 deletions(-) delete mode 100644 client/example/database/main.go delete mode 100644 client/example/playground/main.go delete mode 100644 client/example/rowbase/main.go diff --git a/client/column/columns_test.go b/client/column/columns_test.go index 1a4b3f1605bf5..bba30794bd446 100644 --- a/client/column/columns_test.go +++ b/client/column/columns_test.go @@ -40,9 +40,9 @@ func TestIDColumns(t *testing.T) { t.Run("nil id", func(t *testing.T) { _, err := IDColumns(intPKCol, nil, 0, -1) - assert.Error(t, err) + assert.NoError(t, err) _, err = IDColumns(strPKCol, nil, 0, -1) - assert.Error(t, err) + assert.NoError(t, err) idField := &schemapb.IDs{} col, err := IDColumns(intPKCol, idField, 0, -1) diff --git a/client/example/database/main.go b/client/example/database/main.go deleted file mode 100644 index 0069923d9a2c6..0000000000000 --- a/client/example/database/main.go +++ /dev/null @@ -1,89 +0,0 @@ -package main - -import ( - "context" - "log" - - milvusclient "github.com/milvus-io/milvus/client/v2" - "github.com/milvus-io/milvus/client/v2/entity" -) - -const ( - milvusAddr = `localhost:19530` - nEntities, dim = 3000, 128 - collectionName = "hello_milvus" - - msgFmt = "==== %s ====\n" - idCol, randomCol, embeddingCol = "ID", "random", "embeddings" - topK = 3 -) - -func main() { - ctx := context.Background() - - log.Printf(msgFmt, "start connecting to Milvus") - c, err := milvusclient.New(ctx, &milvusclient.ClientConfig{ - Address: milvusAddr, - }) - if err != nil { - log.Fatal("failed to connect to milvus, err: ", err.Error()) - } - defer c.Close(ctx) - - dbNames, err := c.ListDatabase(ctx, milvusclient.NewListDatabaseOption()) - if err != nil { - log.Fatal("failed to list databases", err.Error()) - } - log.Println("=== Databases: ", dbNames) - - schema := entity.NewSchema().WithName("hello_milvus"). - WithField(entity.NewField().WithName("ID").WithDataType(entity.FieldTypeInt64).WithIsPrimaryKey(true)). - WithField(entity.NewField().WithName("Vector").WithDataType(entity.FieldTypeFloatVector).WithDim(128)) - - if err := c.CreateCollection(ctx, milvusclient.NewCreateCollectionOption("hello_milvus", schema)); err != nil { - log.Fatal("failed to create collection:", err.Error()) - } - - collections, err := c.ListCollections(ctx, milvusclient.NewListCollectionOption()) - if err != nil { - log.Fatal("failed to list collections,", err.Error()) - } - - for _, collectionName := range collections { - collection, err := c.DescribeCollection(ctx, milvusclient.NewDescribeCollectionOption(collectionName)) - if err != nil { - log.Fatal(err.Error()) - } - log.Println(collection.Name) - for _, field := range collection.Schema.Fields { - log.Println("=== Field: ", field.Name, field.DataType, field.AutoID) - } - } - - c.CreateDatabase(ctx, milvusclient.NewCreateDatabaseOption("test")) - c.UsingDatabase(ctx, milvusclient.NewUsingDatabaseOption("test")) - - schema = entity.NewSchema().WithName("hello_milvus"). - WithField(entity.NewField().WithName("ID").WithDataType(entity.FieldTypeVarChar).WithMaxLength(64).WithIsPrimaryKey(true)). - WithField(entity.NewField().WithName("Vector").WithDataType(entity.FieldTypeFloatVector).WithDim(128)) - - if err := c.CreateCollection(ctx, milvusclient.NewCreateCollectionOption("hello_milvus", schema)); err != nil { - log.Fatal("failed to create collection:", err.Error()) - } - - collections, err = c.ListCollections(ctx, milvusclient.NewListCollectionOption()) - if err != nil { - log.Fatal("failed to list collections,", err.Error()) - } - - for _, collectionName := range collections { - collection, err := c.DescribeCollection(ctx, milvusclient.NewDescribeCollectionOption(collectionName)) - if err != nil { - log.Fatal(err.Error()) - } - log.Println(collection.Name) - for _, field := range collection.Schema.Fields { - log.Println("=== Field: ", field.Name, field.DataType, field.AutoID) - } - } -} diff --git a/client/example/playground/main.go b/client/example/playground/main.go deleted file mode 100644 index 10b3256bcdfe5..0000000000000 --- a/client/example/playground/main.go +++ /dev/null @@ -1,326 +0,0 @@ -package main - -import ( - "context" - "flag" - "log" - "math/rand" - "time" - - milvusclient "github.com/milvus-io/milvus/client/v2" - "github.com/milvus-io/milvus/client/v2/entity" - "github.com/milvus-io/milvus/client/v2/index" -) - -var cmd = flag.String("cmd", helloMilvusCmd, "command to run") - -const ( - helloMilvusCmd = `hello_milvus` - partitionsCmd = `partitions` - indexCmd = `indexes` - countCmd = `count` - - milvusAddr = `localhost:19530` - nEntities, dim = 3000, 128 - collectionName = "hello_milvus" - - msgFmt = "==== %s ====\n" - idCol, randomCol, embeddingCol = "ID", "random", "embeddings" - topK = 3 -) - -func main() { - flag.Parse() - - switch *cmd { - case helloMilvusCmd: - HelloMilvus() - case partitionsCmd: - Partitions() - case indexCmd: - Indexes() - case countCmd: - Count() - } -} - -func Count() { - ctx := context.Background() - - collectionName := "hello_count_inverted" - - c, err := milvusclient.New(ctx, &milvusclient.ClientConfig{ - Address: "127.0.0.1:19530", - }) - if err != nil { - log.Fatal("failed to connect to milvus, err: ", err.Error()) - } - - schema := entity.NewSchema().WithName(collectionName). - WithField(entity.NewField().WithName("id").WithDataType(entity.FieldTypeInt64).WithIsAutoID(true).WithIsPrimaryKey(true)). - WithField(entity.NewField().WithName("vector").WithDataType(entity.FieldTypeFloatVector).WithDim(128)) - - err = c.CreateCollection(ctx, milvusclient.NewCreateCollectionOption(collectionName, schema)) - if err != nil { - log.Fatal("failed to connect to milvus, err: ", err.Error()) - } - - indexTask, err := c.CreateIndex(ctx, milvusclient.NewCreateIndexOption(collectionName, "id", index.NewGenericIndex("inverted", map[string]string{}))) - if err != nil { - log.Fatal("failed to connect to milvus, err: ", err.Error()) - } - - indexTask.Await(ctx) - - indexTask, err = c.CreateIndex(ctx, milvusclient.NewCreateIndexOption(collectionName, "vector", index.NewHNSWIndex(entity.L2, 16, 32))) - if err != nil { - log.Fatal("failed to connect to milvus, err: ", err.Error()) - } - - indexTask.Await(ctx) - - loadTask, err := c.LoadCollection(ctx, milvusclient.NewLoadCollectionOption(collectionName)) - if err != nil { - log.Fatal("faied to load collection, err: ", err.Error()) - } - loadTask.Await(ctx) - - for i := 0; i < 100; i++ { - // randomData := make([]int64, 0, nEntities) - vectorData := make([][]float32, 0, nEntities) - // generate data - for i := 0; i < nEntities; i++ { - // randomData = append(randomData, rand.Int63n(1000)) - vec := make([]float32, 0, dim) - for j := 0; j < dim; j++ { - vec = append(vec, rand.Float32()) - } - vectorData = append(vectorData, vec) - } - - _, err = c.Insert(ctx, milvusclient.NewColumnBasedInsertOption(collectionName).WithFloatVectorColumn("vector", dim, vectorData)) - if err != nil { - log.Fatal("failed to insert data") - } - - log.Println("start flush collection") - flushTask, err := c.Flush(ctx, milvusclient.NewFlushOption(collectionName)) - if err != nil { - log.Fatal("failed to flush", err.Error()) - } - start := time.Now() - err = flushTask.Await(ctx) - if err != nil { - log.Fatal("failed to flush", err.Error()) - } - log.Println("flush done, elapsed", time.Since(start)) - - result, err := c.Query(ctx, milvusclient.NewQueryOption(collectionName). - WithOutputFields("count(*)"). - WithConsistencyLevel(entity.ClStrong)) - if err != nil { - log.Fatal("failed to connect to milvus, err: ", err.Error()) - } - for _, rs := range result.Fields { - log.Println(rs) - } - result, err = c.Query(ctx, milvusclient.NewQueryOption(collectionName). - WithOutputFields("count(*)"). - WithFilter("id > 0"). - WithConsistencyLevel(entity.ClStrong)) - if err != nil { - log.Fatal("failed to connect to milvus, err: ", err.Error()) - } - for _, rs := range result.Fields { - log.Println(rs) - } - } - - err = c.DropCollection(ctx, milvusclient.NewDropCollectionOption(collectionName)) - if err != nil { - log.Fatal("=== Failed to drop collection", err.Error()) - } -} - -func HelloMilvus() { - ctx := context.Background() - - log.Printf(msgFmt, "start connecting to Milvus") - c, err := milvusclient.New(ctx, &milvusclient.ClientConfig{ - Address: milvusAddr, - }) - if err != nil { - log.Fatal("failed to connect to milvus, err: ", err.Error()) - } - defer c.Close(ctx) - - if has, err := c.HasCollection(ctx, milvusclient.NewHasCollectionOption(collectionName)); err != nil { - log.Fatal("failed to check collection exists or not", err.Error()) - } else if has { - c.DropCollection(ctx, milvusclient.NewDropCollectionOption(collectionName)) - } - - err = c.CreateCollection(ctx, milvusclient.SimpleCreateCollectionOptions(collectionName, dim).WithVarcharPK(true, 128)) - if err != nil { - log.Fatal("failed to create collection", err.Error()) - } - - collections, err := c.ListCollections(ctx, milvusclient.NewListCollectionOption()) - if err != nil { - log.Fatal("failed to list collections,", err.Error()) - } - - for _, collectionName := range collections { - collection, err := c.DescribeCollection(ctx, milvusclient.NewDescribeCollectionOption(collectionName)) - if err != nil { - log.Fatal(err.Error()) - } - log.Println(collection.Name) - for _, field := range collection.Schema.Fields { - log.Println("=== Field: ", field.Name, field.DataType, field.AutoID) - } - } - - // randomData := make([]int64, 0, nEntities) - vectorData := make([][]float32, 0, nEntities) - // generate data - for i := 0; i < nEntities; i++ { - // randomData = append(randomData, rand.Int63n(1000)) - vec := make([]float32, 0, dim) - for j := 0; j < dim; j++ { - vec = append(vec, rand.Float32()) - } - vectorData = append(vectorData, vec) - } - - _, err = c.Insert(ctx, milvusclient.NewColumnBasedInsertOption(collectionName).WithFloatVectorColumn("vector", dim, vectorData)) - if err != nil { - log.Fatal("failed to insert data") - } - - log.Println("start flush collection") - flushTask, err := c.Flush(ctx, milvusclient.NewFlushOption(collectionName)) - if err != nil { - log.Fatal("failed to flush", err.Error()) - } - start := time.Now() - err = flushTask.Await(ctx) - if err != nil { - log.Fatal("failed to flush", err.Error()) - } - log.Println("flush done, elapsed", time.Since(start)) - - vec2search := []entity.Vector{ - entity.FloatVector(vectorData[len(vectorData)-2]), - entity.FloatVector(vectorData[len(vectorData)-1]), - } - - resultSets, err := c.Search(ctx, milvusclient.NewSearchOption(collectionName, 3, vec2search).WithConsistencyLevel(entity.ClEventually)) - if err != nil { - log.Fatal("failed to search collection", err.Error()) - } - for _, resultSet := range resultSets { - for i := 0; i < resultSet.ResultCount; i++ { - log.Print(resultSet.IDs.Get(i)) - } - log.Println() - } - - err = c.DropCollection(ctx, milvusclient.NewDropCollectionOption(collectionName)) - if err != nil { - log.Fatal("=== Failed to drop collection", err.Error()) - } -} - -func Partitions() { - ctx := context.Background() - - log.Printf(msgFmt, "start connecting to Milvus") - c, err := milvusclient.New(ctx, &milvusclient.ClientConfig{ - Address: milvusAddr, - }) - if err != nil { - log.Fatal("failed to connect to milvus, err: ", err.Error()) - } - defer c.Close(ctx) - - has, err := c.HasCollection(ctx, milvusclient.NewHasCollectionOption(collectionName)) - if err != nil { - log.Fatal(err) - } - if has { - c.DropCollection(ctx, milvusclient.NewDropCollectionOption(collectionName)) - } - - err = c.CreateCollection(ctx, milvusclient.SimpleCreateCollectionOptions(collectionName, dim)) - if err != nil { - log.Fatal("failed to create collection", err.Error()) - } - - partitions, err := c.ListPartitions(ctx, milvusclient.NewListPartitionOption(collectionName)) - if err != nil { - log.Fatal("failed to create collection", err.Error()) - } - - for _, partitionName := range partitions { - err := c.DropPartition(ctx, milvusclient.NewDropPartitionOption(collectionName, partitionName)) - if err != nil { - log.Println(err.Error()) - } - } - - c.CreatePartition(ctx, milvusclient.NewCreatePartitionOption(collectionName, "new_partition")) - partitions, err = c.ListPartitions(ctx, milvusclient.NewListPartitionOption(collectionName)) - if err != nil { - log.Fatal("failed to create collection", err.Error()) - } - log.Println(partitions) - - err = c.DropCollection(ctx, milvusclient.NewDropCollectionOption(collectionName)) - if err != nil { - log.Fatal("=== Failed to drop collection", err.Error()) - } -} - -func Indexes() { - ctx := context.Background() - - log.Printf(msgFmt, "start connecting to Milvus") - c, err := milvusclient.New(ctx, &milvusclient.ClientConfig{ - Address: milvusAddr, - }) - if err != nil { - log.Fatal("failed to connect to milvus, err: ", err.Error()) - } - defer c.Close(ctx) - - has, err := c.HasCollection(ctx, milvusclient.NewHasCollectionOption(collectionName)) - if err != nil { - log.Fatal(err) - } - if has { - c.DropCollection(ctx, milvusclient.NewDropCollectionOption(collectionName)) - } - - err = c.CreateCollection(ctx, milvusclient.SimpleCreateCollectionOptions(collectionName, dim)) - if err != nil { - log.Fatal("failed to create collection", err.Error()) - } - - index := index.NewHNSWIndex(entity.COSINE, 16, 64) - - createIdxOpt := milvusclient.NewCreateIndexOption(collectionName, "vector", index) - task, err := c.CreateIndex(ctx, createIdxOpt) - if err != nil { - log.Fatal("failed to create index", err.Error()) - } - task.Await(ctx) - - indexes, err := c.ListIndexes(ctx, milvusclient.NewListIndexOption(collectionName)) - if err != nil { - log.Fatal("failed to list indexes", err.Error()) - } - for _, indexName := range indexes { - log.Println(indexName) - } -} diff --git a/client/example/rowbase/main.go b/client/example/rowbase/main.go deleted file mode 100644 index 2b43378bb8071..0000000000000 --- a/client/example/rowbase/main.go +++ /dev/null @@ -1,87 +0,0 @@ -package main - -import ( - "context" - "log" - "math/rand" - - "github.com/samber/lo" - - milvusclient "github.com/milvus-io/milvus/client/v2" - "github.com/milvus-io/milvus/client/v2/row" -) - -type Data struct { - ID int64 `milvus:"name:id;primary_key;auto_id"` - Vector []float32 `milvus:"name:vector;dim:128"` -} - -const ( - milvusAddr = `localhost:19530` - nEntities, dim = 10, 128 - collectionName = "hello_row_base" - - msgFmt = "==== %s ====\n" - idCol, randomCol, embeddingCol = "id", "random", "vector" - topK = 3 -) - -func main() { - schema, err := row.ParseSchema(&Data{}) - if err != nil { - log.Fatal("failed to parse schema from struct", err.Error()) - } - - for _, field := range schema.Fields { - log.Printf("Field name: %s, FieldType %s, IsPrimaryKey: %t", field.Name, field.DataType, field.PrimaryKey) - } - schema.WithName(collectionName) - - ctx := context.Background() - - log.Printf(msgFmt, "start connecting to Milvus") - c, err := milvusclient.New(ctx, &milvusclient.ClientConfig{ - Address: milvusAddr, - }) - if err != nil { - log.Fatal("failed to connect to milvus, err: ", err.Error()) - } - defer c.Close(ctx) - - if has, err := c.HasCollection(ctx, milvusclient.NewHasCollectionOption(collectionName)); err != nil { - log.Fatal("failed to check collection exists or not", err.Error()) - } else if has { - log.Printf("collection %s alread exists, dropping it now\n", collectionName) - c.DropCollection(ctx, milvusclient.NewDropCollectionOption(collectionName)) - } - - err = c.CreateCollection(ctx, milvusclient.NewCreateCollectionOption(collectionName, schema)) - if err != nil { - log.Fatal("failed to create collection", err.Error()) - } - - var rows []*Data - for i := 0; i < nEntities; i++ { - vec := make([]float32, 0, dim) - for j := 0; j < dim; j++ { - vec = append(vec, rand.Float32()) - } - rows = append(rows, &Data{ - Vector: vec, - }) - } - - insertResult, err := c.Insert(ctx, milvusclient.NewRowBasedInsertOption(collectionName, lo.Map(rows, func(data *Data, _ int) any { - return data - })...)) - if err != nil { - log.Fatal("failed to insert data: ", err.Error()) - } - log.Println(insertResult.IDs) - for _, row := range rows { - // id shall be written back - log.Println(row.ID) - } - - c.DropCollection(ctx, milvusclient.NewDropCollectionOption(collectionName)) -} diff --git a/scripts/run_go_codecov.sh b/scripts/run_go_codecov.sh index 1e8ca87e5f928..57231534341f3 100755 --- a/scripts/run_go_codecov.sh +++ b/scripts/run_go_codecov.sh @@ -73,5 +73,5 @@ endTime=`date +%s` echo "Total time for go unittest:" $(($endTime-$beginTime)) "s" # generate html report -go tool cover -html=./${FILE_COVERAGE_INFO} -o ./${FILE_COVERAGE_HTML} -echo "Generate go coverage report to ${FILE_COVERAGE_HTML}" +# go tool cover -html=./${FILE_COVERAGE_INFO} -o ./${FILE_COVERAGE_HTML} +# echo "Generate go coverage report to ${FILE_COVERAGE_HTML}" From 51ed2a61c8596f7a1576aa9dc9824031fb16dc08 Mon Sep 17 00:00:00 2001 From: XuanYang-cn Date: Thu, 7 Nov 2024 11:16:24 +0800 Subject: [PATCH 23/40] fix: Correct dropped segment num metrics (#37410) See also: #31891 Signed-off-by: yangxuan --- internal/datacoord/meta.go | 10 +- internal/datacoord/meta_test.go | 243 +++++++++++++++++++++----------- 2 files changed, 163 insertions(+), 90 deletions(-) diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index dec075915c11d..7aaaecd14c609 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -1551,14 +1551,13 @@ func (m *meta) completeMixCompactionMutation(t *datapb.CompactionTask, result *d IsSorted: compactToSegment.GetIsSorted(), }) - // L1 segment with NumRows=0 will be discarded, so no need to change the metric - if compactToSegmentInfo.GetNumOfRows() > 0 { - // metrics mutation for compactTo segments - metricMutation.addNewSeg(compactToSegmentInfo.GetState(), compactToSegmentInfo.GetLevel(), compactToSegmentInfo.GetIsSorted(), compactToSegmentInfo.GetNumOfRows()) - } else { + if compactToSegmentInfo.GetNumOfRows() == 0 { compactToSegmentInfo.State = commonpb.SegmentState_Dropped } + // metrics mutation for compactTo segments + metricMutation.addNewSeg(compactToSegmentInfo.GetState(), compactToSegmentInfo.GetLevel(), compactToSegmentInfo.GetIsSorted(), compactToSegmentInfo.GetNumOfRows()) + log.Info("Add a new compactTo segment", zap.Int64("compactTo", compactToSegmentInfo.GetID()), zap.Int64("compactTo segment numRows", compactToSegmentInfo.GetNumOfRows()), @@ -1582,6 +1581,7 @@ func (m *meta) completeMixCompactionMutation(t *datapb.CompactionTask, result *d for _, seg := range compactToInfos { binlogs = append(binlogs, metastore.BinlogsIncrement{Segment: seg}) } + // alter compactTo before compactFrom segments to avoid data lost if service crash during AlterSegments if err := m.catalog.AlterSegments(m.ctx, compactToInfos, binlogs...); err != nil { log.Warn("fail to alter compactTo segments", zap.Error(err)) diff --git a/internal/datacoord/meta_test.go b/internal/datacoord/meta_test.go index 8492173123b7b..741c227ec6e6d 100644 --- a/internal/datacoord/meta_test.go +++ b/internal/datacoord/meta_test.go @@ -182,103 +182,176 @@ func (suite *MetaBasicSuite) TestCollection() { } func (suite *MetaBasicSuite) TestCompleteCompactionMutation() { - latestSegments := NewSegmentsInfo() - for segID, segment := range map[UniqueID]*SegmentInfo{ - 1: {SegmentInfo: &datapb.SegmentInfo{ - ID: 1, - CollectionID: 100, - PartitionID: 10, - State: commonpb.SegmentState_Flushed, - Level: datapb.SegmentLevel_L1, - Binlogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 10000, 10001)}, - Statslogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 20000, 20001)}, - // latest segment has 2 deltalogs, one submit for compaction, one is appended before compaction done - Deltalogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 30000), getFieldBinlogIDs(0, 30001)}, - NumOfRows: 2, - }}, - 2: {SegmentInfo: &datapb.SegmentInfo{ - ID: 2, - CollectionID: 100, - PartitionID: 10, - State: commonpb.SegmentState_Flushed, - Level: datapb.SegmentLevel_L1, - Binlogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 11000)}, - Statslogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 21000)}, - // latest segment has 2 deltalogs, one submit for compaction, one is appended before compaction done - Deltalogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 31000), getFieldBinlogIDs(0, 31001)}, - NumOfRows: 2, - }}, - } { - latestSegments.SetSegment(segID, segment) + getLatestSegments := func() *SegmentsInfo { + latestSegments := NewSegmentsInfo() + for segID, segment := range map[UniqueID]*SegmentInfo{ + 1: {SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + CollectionID: 100, + PartitionID: 10, + State: commonpb.SegmentState_Flushed, + Level: datapb.SegmentLevel_L1, + Binlogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 10000, 10001)}, + Statslogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 20000, 20001)}, + // latest segment has 2 deltalogs, one submit for compaction, one is appended before compaction done + Deltalogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 30000), getFieldBinlogIDs(0, 30001)}, + NumOfRows: 2, + }}, + 2: {SegmentInfo: &datapb.SegmentInfo{ + ID: 2, + CollectionID: 100, + PartitionID: 10, + State: commonpb.SegmentState_Flushed, + Level: datapb.SegmentLevel_L1, + Binlogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 11000)}, + Statslogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 21000)}, + // latest segment has 2 deltalogs, one submit for compaction, one is appended before compaction done + Deltalogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 31000), getFieldBinlogIDs(0, 31001)}, + NumOfRows: 2, + }}, + } { + latestSegments.SetSegment(segID, segment) + } + + return latestSegments } mockChMgr := mocks.NewChunkManager(suite.T()) - m := &meta{ - catalog: &datacoord.Catalog{MetaKv: NewMetaMemoryKV()}, - segments: latestSegments, - chunkManager: mockChMgr, - } - compactToSeg := &datapb.CompactionSegment{ - SegmentID: 3, - InsertLogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 50000)}, - Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 50001)}, - NumOfRows: 2, - } + suite.Run("test complete with compactTo 0 num of rows", func() { + latestSegments := getLatestSegments() + compactToSeg := &datapb.CompactionSegment{ + SegmentID: 4, + InsertLogs: []*datapb.FieldBinlog{}, + Field2StatslogPaths: []*datapb.FieldBinlog{}, + NumOfRows: 0, + } - result := &datapb.CompactionPlanResult{ - Segments: []*datapb.CompactionSegment{compactToSeg}, - } - task := &datapb.CompactionTask{ - InputSegments: []UniqueID{1, 2}, - Type: datapb.CompactionType_MixCompaction, - } + result := &datapb.CompactionPlanResult{ + Segments: []*datapb.CompactionSegment{compactToSeg}, + } + task := &datapb.CompactionTask{ + InputSegments: []UniqueID{1, 2}, + Type: datapb.CompactionType_MixCompaction, + } + m := &meta{ + catalog: &datacoord.Catalog{MetaKv: NewMetaMemoryKV()}, + segments: latestSegments, + chunkManager: mockChMgr, + } - infos, mutation, err := m.CompleteCompactionMutation(task, result) - assert.NoError(suite.T(), err) - suite.Equal(1, len(infos)) - info := infos[0] - suite.NoError(err) - suite.NotNil(info) - suite.NotNil(mutation) - - // check newSegment - suite.EqualValues(3, info.GetID()) - suite.Equal(datapb.SegmentLevel_L1, info.GetLevel()) - suite.Equal(commonpb.SegmentState_Flushed, info.GetState()) - - binlogs := info.GetBinlogs() - for _, fbinlog := range binlogs { - for _, blog := range fbinlog.GetBinlogs() { - suite.Empty(blog.GetLogPath()) - suite.EqualValues(50000, blog.GetLogID()) + infos, mutation, err := m.CompleteCompactionMutation(task, result) + assert.NoError(suite.T(), err) + suite.Equal(1, len(infos)) + info := infos[0] + suite.NoError(err) + suite.NotNil(info) + suite.NotNil(mutation) + + // check compact to segments + suite.EqualValues(4, info.GetID()) + suite.Equal(datapb.SegmentLevel_L1, info.GetLevel()) + suite.Equal(commonpb.SegmentState_Dropped, info.GetState()) + + suite.Empty(info.GetBinlogs()) + suite.Empty(info.GetStatslogs()) + + // check compactFrom segments + for _, segID := range []int64{1, 2} { + seg := m.GetSegment(segID) + suite.Equal(commonpb.SegmentState_Dropped, seg.GetState()) + suite.NotEmpty(seg.GetDroppedAt()) + + suite.EqualValues(segID, seg.GetID()) + suite.ElementsMatch(latestSegments.segments[segID].GetBinlogs(), seg.GetBinlogs()) + suite.ElementsMatch(latestSegments.segments[segID].GetStatslogs(), seg.GetStatslogs()) + suite.ElementsMatch(latestSegments.segments[segID].GetDeltalogs(), seg.GetDeltalogs()) } - } - statslogs := info.GetStatslogs() - for _, fbinlog := range statslogs { - for _, blog := range fbinlog.GetBinlogs() { - suite.Empty(blog.GetLogPath()) - suite.EqualValues(50001, blog.GetLogID()) + // check mutation metrics + suite.EqualValues(2, len(mutation.stateChange[datapb.SegmentLevel_L1.String()])) + suite.EqualValues(-4, mutation.rowCountChange) + suite.EqualValues(0, mutation.rowCountAccChange) + flushedUnsorted := mutation.stateChange[datapb.SegmentLevel_L1.String()][commonpb.SegmentState_Flushed.String()][getSortStatus(false)] + suite.EqualValues(-2, flushedUnsorted) + + droppedUnsorted := mutation.stateChange[datapb.SegmentLevel_L1.String()][commonpb.SegmentState_Dropped.String()][getSortStatus(false)] + suite.EqualValues(3, droppedUnsorted) + }) + + suite.Run("test complete compaction mutation", func() { + latestSegments := getLatestSegments() + compactToSeg := &datapb.CompactionSegment{ + SegmentID: 3, + InsertLogs: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 50000)}, + Field2StatslogPaths: []*datapb.FieldBinlog{getFieldBinlogIDs(0, 50001)}, + NumOfRows: 2, } - } - // check compactFrom segments - for _, segID := range []int64{1, 2} { - seg := m.GetSegment(segID) - suite.Equal(commonpb.SegmentState_Dropped, seg.GetState()) - suite.NotEmpty(seg.GetDroppedAt()) + result := &datapb.CompactionPlanResult{ + Segments: []*datapb.CompactionSegment{compactToSeg}, + } + task := &datapb.CompactionTask{ + InputSegments: []UniqueID{1, 2}, + Type: datapb.CompactionType_MixCompaction, + } + m := &meta{ + catalog: &datacoord.Catalog{MetaKv: NewMetaMemoryKV()}, + segments: latestSegments, + chunkManager: mockChMgr, + } - suite.EqualValues(segID, seg.GetID()) - suite.ElementsMatch(latestSegments.segments[segID].GetBinlogs(), seg.GetBinlogs()) - suite.ElementsMatch(latestSegments.segments[segID].GetStatslogs(), seg.GetStatslogs()) - suite.ElementsMatch(latestSegments.segments[segID].GetDeltalogs(), seg.GetDeltalogs()) - } + infos, mutation, err := m.CompleteCompactionMutation(task, result) + assert.NoError(suite.T(), err) + suite.Equal(1, len(infos)) + info := infos[0] + suite.NoError(err) + suite.NotNil(info) + suite.NotNil(mutation) + + // check newSegment + suite.EqualValues(3, info.GetID()) + suite.Equal(datapb.SegmentLevel_L1, info.GetLevel()) + suite.Equal(commonpb.SegmentState_Flushed, info.GetState()) + + binlogs := info.GetBinlogs() + for _, fbinlog := range binlogs { + for _, blog := range fbinlog.GetBinlogs() { + suite.Empty(blog.GetLogPath()) + suite.EqualValues(50000, blog.GetLogID()) + } + } + + statslogs := info.GetStatslogs() + for _, fbinlog := range statslogs { + for _, blog := range fbinlog.GetBinlogs() { + suite.Empty(blog.GetLogPath()) + suite.EqualValues(50001, blog.GetLogID()) + } + } + + // check compactFrom segments + for _, segID := range []int64{1, 2} { + seg := m.GetSegment(segID) + suite.Equal(commonpb.SegmentState_Dropped, seg.GetState()) + suite.NotEmpty(seg.GetDroppedAt()) - // check mutation metrics - suite.Equal(2, len(mutation.stateChange[datapb.SegmentLevel_L1.String()])) - suite.EqualValues(-2, mutation.rowCountChange) - suite.EqualValues(2, mutation.rowCountAccChange) + suite.EqualValues(segID, seg.GetID()) + suite.ElementsMatch(latestSegments.segments[segID].GetBinlogs(), seg.GetBinlogs()) + suite.ElementsMatch(latestSegments.segments[segID].GetStatslogs(), seg.GetStatslogs()) + suite.ElementsMatch(latestSegments.segments[segID].GetDeltalogs(), seg.GetDeltalogs()) + } + + // check mutation metrics + suite.EqualValues(2, len(mutation.stateChange[datapb.SegmentLevel_L1.String()])) + suite.EqualValues(-2, mutation.rowCountChange) + suite.EqualValues(2, mutation.rowCountAccChange) + flushedCount := mutation.stateChange[datapb.SegmentLevel_L1.String()][commonpb.SegmentState_Flushed.String()][getSortStatus(false)] + suite.EqualValues(-1, flushedCount) + + droppedCount := mutation.stateChange[datapb.SegmentLevel_L1.String()][commonpb.SegmentState_Dropped.String()][getSortStatus(false)] + suite.EqualValues(2, droppedCount) + }) } func (suite *MetaBasicSuite) TestSetSegment() { From b4c749dcd5dfed51204fc4b9014e0114b206ea1d Mon Sep 17 00:00:00 2001 From: aoiasd <45024769+aoiasd@users.noreply.github.com> Date: Thu, 7 Nov 2024 11:18:26 +0800 Subject: [PATCH 24/40] fix: merge sort segment loss data (#37400) relate: https://github.com/milvus-io/milvus/issues/37238 Signed-off-by: aoiasd --- internal/datanode/compaction/merge_sort.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/internal/datanode/compaction/merge_sort.go b/internal/datanode/compaction/merge_sort.go index bcf8dd8bb71e4..a43cc0c1796b3 100644 --- a/internal/datanode/compaction/merge_sort.go +++ b/internal/datanode/compaction/merge_sort.go @@ -105,6 +105,12 @@ func mergeSortMultipleSegments(ctx context.Context, deletedRowCount++ continue } + + // Filtering expired entity + if isExpiredEntity(collectionTtl, currentTs, typeutil.Timestamp(v.Timestamp)) { + expiredRowCount++ + continue + } return v, nil } } @@ -125,12 +131,6 @@ func mergeSortMultipleSegments(ctx context.Context, smallest := heap.Pop(&pq).(*PQItem) v := smallest.Value - // Filtering expired entity - if isExpiredEntity(collectionTtl, currentTs, typeutil.Timestamp(v.Timestamp)) { - expiredRowCount++ - continue - } - err := mWriter.Write(v) if err != nil { log.Warn("compact wrong, failed to writer row", zap.Error(err)) From 9b6dd23f8ede46cf874c97a1aacf2ce04ca35404 Mon Sep 17 00:00:00 2001 From: smellthemoon <64083300+smellthemoon@users.noreply.github.com> Date: Thu, 7 Nov 2024 11:22:25 +0800 Subject: [PATCH 25/40] fix: wrong path spelling when use rootpath in segcore (#37453) #36532 Signed-off-by: lixinguo Co-authored-by: lixinguo --- .../core/src/clustering/KmeansClustering.h | 37 +++++++++++-------- .../core/src/index/InvertedIndexTantivy.cpp | 14 +++---- internal/core/src/storage/ChunkManager.h | 1 + internal/core/src/storage/FileManager.h | 31 ++++++++++------ internal/core/src/storage/Util.cpp | 34 +++++++++++------ 5 files changed, 72 insertions(+), 45 deletions(-) diff --git a/internal/core/src/clustering/KmeansClustering.h b/internal/core/src/clustering/KmeansClustering.h index 500613ea0a0f5..151ef5e3f5788 100644 --- a/internal/core/src/clustering/KmeansClustering.h +++ b/internal/core/src/clustering/KmeansClustering.h @@ -17,9 +17,11 @@ #pragma once #include +#include #include #include +#include "boost/filesystem/path.hpp" #include "storage/MemFileManagerImpl.h" #include "pb/clustering.pb.h" #include "knowhere/cluster/cluster_factory.h" @@ -61,27 +63,32 @@ class KmeansClustering { GetRemoteCentroidsObjectPrefix() const { auto index_meta_ = file_manager_->GetIndexMeta(); auto field_meta_ = file_manager_->GetFieldDataMeta(); - return file_manager_->GetChunkManager()->GetRootPath() + "/" + - std::string(ANALYZE_ROOT_PATH) + "/" + - std::to_string(index_meta_.build_id) + "/" + - std::to_string(index_meta_.index_version) + "/" + - std::to_string(field_meta_.collection_id) + "/" + - std::to_string(field_meta_.partition_id) + "/" + - std::to_string(field_meta_.field_id); + boost::filesystem::path prefix = + file_manager_->GetChunkManager()->GetRootPath(); + boost::filesystem::path path = + std::to_string(index_meta_.build_id) + "/" + + std::to_string(index_meta_.index_version) + "/" + + std::to_string(field_meta_.collection_id) + "/" + + std::to_string(field_meta_.partition_id) + "/" + + std::to_string(field_meta_.field_id); + return (prefix / path).string(); } inline std::string GetRemoteCentroidIdMappingObjectPrefix(int64_t segment_id) const { auto index_meta_ = file_manager_->GetIndexMeta(); auto field_meta_ = file_manager_->GetFieldDataMeta(); - return file_manager_->GetChunkManager()->GetRootPath() + "/" + - std::string(ANALYZE_ROOT_PATH) + "/" + - std::to_string(index_meta_.build_id) + "/" + - std::to_string(index_meta_.index_version) + "/" + - std::to_string(field_meta_.collection_id) + "/" + - std::to_string(field_meta_.partition_id) + "/" + - std::to_string(field_meta_.field_id) + "/" + - std::to_string(segment_id); + boost::filesystem::path prefix = + file_manager_->GetChunkManager()->GetRootPath(); + boost::filesystem::path path = std::string(ANALYZE_ROOT_PATH); + boost::filesystem::path path1 = + std::to_string(index_meta_.build_id) + "/" + + std::to_string(index_meta_.index_version) + "/" + + std::to_string(field_meta_.collection_id) + "/" + + std::to_string(field_meta_.partition_id) + "/" + + std::to_string(field_meta_.field_id) + "/" + + std::to_string(segment_id); + return (prefix / path / path1).string(); } ~KmeansClustering() = default; diff --git a/internal/core/src/index/InvertedIndexTantivy.cpp b/internal/core/src/index/InvertedIndexTantivy.cpp index 1db1fa9d01bff..663564dd95b23 100644 --- a/internal/core/src/index/InvertedIndexTantivy.cpp +++ b/internal/core/src/index/InvertedIndexTantivy.cpp @@ -188,15 +188,15 @@ InvertedIndexTantivy::Load(milvus::tracer::TraceContext ctx, }), files_value.end()); - auto index_valid_data_file = - mem_file_manager_->GetRemoteIndexObjectPrefix() + - std::string("/index_null_offset"); - auto it = std::find( - files_value.begin(), files_value.end(), index_valid_data_file); + auto it = std::find_if( + files_value.begin(), files_value.end(), [](const std::string& file) { + return file.substr(file.find_last_of('/') + 1) == + "index_null_offset"; + }); if (it != files_value.end()) { - files_value.erase(it); std::vector file; - file.push_back(index_valid_data_file); + file.push_back(*it); + files_value.erase(it); auto index_datas = mem_file_manager_->LoadIndexToMemory(file); AssembleIndexDatas(index_datas); BinarySet binary_set; diff --git a/internal/core/src/storage/ChunkManager.h b/internal/core/src/storage/ChunkManager.h index 9f51154ee6f69..fa4a39b2ec965 100644 --- a/internal/core/src/storage/ChunkManager.h +++ b/internal/core/src/storage/ChunkManager.h @@ -117,6 +117,7 @@ class ChunkManager { /** * @brief Get the Root Path * @return std::string + * Note: when join path, please check the training '/' */ virtual std::string GetRootPath() const = 0; diff --git a/internal/core/src/storage/FileManager.h b/internal/core/src/storage/FileManager.h index 18f4b798c8ed3..d2e71a39ee903 100644 --- a/internal/core/src/storage/FileManager.h +++ b/internal/core/src/storage/FileManager.h @@ -21,6 +21,7 @@ #include #include "common/Consts.h" +#include "boost/filesystem/path.hpp" #include "knowhere/file_manager.h" #include "log/Log.h" #include "storage/ChunkManager.h" @@ -129,11 +130,14 @@ class FileManagerImpl : public knowhere::FileManager { virtual std::string GetRemoteIndexObjectPrefix() const { - return rcm_->GetRootPath() + "/" + std::string(INDEX_ROOT_PATH) + "/" + - std::to_string(index_meta_.build_id) + "/" + - std::to_string(index_meta_.index_version) + "/" + - std::to_string(field_meta_.partition_id) + "/" + - std::to_string(field_meta_.segment_id); + boost::filesystem::path prefix = rcm_->GetRootPath(); + boost::filesystem::path path = std::string(INDEX_ROOT_PATH); + boost::filesystem::path path1 = + std::to_string(index_meta_.build_id) + "/" + + std::to_string(index_meta_.index_version) + "/" + + std::to_string(field_meta_.partition_id) + "/" + + std::to_string(field_meta_.segment_id); + return (prefix / path / path1).string(); } virtual std::string @@ -147,13 +151,16 @@ class FileManagerImpl : public knowhere::FileManager { virtual std::string GetRemoteTextLogPrefix() const { - return rcm_->GetRootPath() + "/" + std::string(TEXT_LOG_ROOT_PATH) + - "/" + std::to_string(index_meta_.build_id) + "/" + - std::to_string(index_meta_.index_version) + "/" + - std::to_string(field_meta_.collection_id) + "/" + - std::to_string(field_meta_.partition_id) + "/" + - std::to_string(field_meta_.segment_id) + "/" + - std::to_string(field_meta_.field_id); + boost::filesystem::path prefix = rcm_->GetRootPath(); + boost::filesystem::path path = std::string(TEXT_LOG_ROOT_PATH); + boost::filesystem::path path1 = + std::to_string(index_meta_.build_id) + "/" + + std::to_string(index_meta_.index_version) + "/" + + std::to_string(field_meta_.collection_id) + "/" + + std::to_string(field_meta_.partition_id) + "/" + + std::to_string(field_meta_.segment_id) + "/" + + std::to_string(field_meta_.field_id); + return (prefix / path / path1).string(); } protected: diff --git a/internal/core/src/storage/Util.cpp b/internal/core/src/storage/Util.cpp index 0ccf13b45fe9d..5e137207722f7 100644 --- a/internal/core/src/storage/Util.cpp +++ b/internal/core/src/storage/Util.cpp @@ -502,8 +502,11 @@ std::string GenIndexPathPrefix(ChunkManagerPtr cm, int64_t build_id, int64_t index_version) { - return cm->GetRootPath() + "/" + std::string(INDEX_ROOT_PATH) + "/" + - GenIndexPathIdentifier(build_id, index_version); + boost::filesystem::path prefix = cm->GetRootPath(); + boost::filesystem::path path = std::string(INDEX_ROOT_PATH); + boost::filesystem::path path1 = + GenIndexPathIdentifier(build_id, index_version); + return (prefix / path / path1).string(); } std::string @@ -512,29 +515,38 @@ GenTextIndexPathPrefix(ChunkManagerPtr cm, int64_t index_version, int64_t segment_id, int64_t field_id) { - return cm->GetRootPath() + "/" + std::string(TEXT_LOG_ROOT_PATH) + "/" + - GenTextIndexPathIdentifier( - build_id, index_version, segment_id, field_id); + boost::filesystem::path prefix = cm->GetRootPath(); + boost::filesystem::path path = std::string(TEXT_LOG_ROOT_PATH); + boost::filesystem::path path1 = GenTextIndexPathIdentifier( + build_id, index_version, segment_id, field_id); + return (prefix / path / path1).string(); } std::string GetIndexPathPrefixWithBuildID(ChunkManagerPtr cm, int64_t build_id) { - return cm->GetRootPath() + "/" + std::string(INDEX_ROOT_PATH) + "/" + - std::to_string(build_id); + boost::filesystem::path prefix = cm->GetRootPath(); + boost::filesystem::path path = std::string(INDEX_ROOT_PATH); + boost::filesystem::path path1 = std::to_string(build_id); + return (prefix / path / path1).string(); } std::string GenFieldRawDataPathPrefix(ChunkManagerPtr cm, int64_t segment_id, int64_t field_id) { - return cm->GetRootPath() + "/" + std::string(RAWDATA_ROOT_PATH) + "/" + - std::to_string(segment_id) + "/" + std::to_string(field_id) + "/"; + boost::filesystem::path prefix = cm->GetRootPath(); + boost::filesystem::path path = std::string(RAWDATA_ROOT_PATH); + boost::filesystem::path path1 = + std::to_string(segment_id) + "/" + std::to_string(field_id) + "/"; + return (prefix / path / path1).string(); } std::string GetSegmentRawDataPathPrefix(ChunkManagerPtr cm, int64_t segment_id) { - return cm->GetRootPath() + "/" + std::string(RAWDATA_ROOT_PATH) + "/" + - std::to_string(segment_id); + boost::filesystem::path prefix = cm->GetRootPath(); + boost::filesystem::path path = std::string(RAWDATA_ROOT_PATH); + boost::filesystem::path path1 = std::to_string(segment_id); + return (prefix / path / path1).string(); } std::unique_ptr From f348bd94413afe52f7b9a7d471ce78d7e9f47537 Mon Sep 17 00:00:00 2001 From: jaime Date: Thu, 7 Nov 2024 11:52:25 +0800 Subject: [PATCH 26/40] feat: add segment,pipeline, replica and resourcegroup api for WebUI (#37344) issue: #36621 Signed-off-by: jaime --- internal/datacoord/channel_manager.go | 19 + internal/datacoord/channel_manager_test.go | 48 +++ internal/datacoord/index_meta.go | 2 +- internal/datacoord/meta.go | 25 ++ internal/datacoord/meta_test.go | 61 +++ internal/datacoord/metrics_info.go | 166 +++++--- internal/datacoord/metrics_info_test.go | 385 +++++++++++++++++- internal/datacoord/mock_channelmanager.go | 48 +++ internal/datacoord/server.go | 18 +- internal/datacoord/stats_task_meta.go | 2 +- internal/datanode/data_node.go | 11 + internal/flushcommon/metacache/segment.go | 8 + .../pipeline/flow_graph_manager.go | 57 +++ .../pipeline/flow_graph_manager_test.go | 111 ++++- .../flushcommon/pipeline/mock_fgmanager.go | 112 ++++- internal/flushcommon/syncmgr/task.go | 7 +- internal/flushcommon/syncmgr/task_test.go | 32 +- internal/flushcommon/util/timetick_sender.go | 12 + internal/http/router.go | 51 ++- internal/http/webui/header.html | 3 - internal/http/webui/index.html | 25 +- internal/http/webui/nodes.html | 62 --- internal/http/webui/static/js/mockdata.js | 373 ++++++++++++++++- internal/proxy/impl.go | 38 +- internal/proxy/impl_test.go | 16 +- internal/querycoordv2/handlers.go | 49 +++ internal/querycoordv2/handlers_test.go | 108 +++++ .../querycoordv2/meta/channel_dist_manager.go | 22 + .../meta/channel_dist_manager_test.go | 41 ++ internal/querycoordv2/meta/dist_manager.go | 36 ++ .../querycoordv2/meta/dist_manager_test.go | 94 +++++ .../querycoordv2/meta/leader_view_manager.go | 44 ++ .../meta/leader_view_manager_test.go | 78 ++++ .../querycoordv2/meta/mock_target_manager.go | 46 +++ internal/querycoordv2/meta/replica_manager.go | 32 ++ .../querycoordv2/meta/replica_manager_test.go | 56 +++ .../querycoordv2/meta/resource_manager.go | 22 + .../meta/resource_manager_test.go | 34 ++ .../querycoordv2/meta/segment_dist_manager.go | 31 ++ .../meta/segment_dist_manager_test.go | 63 +++ internal/querycoordv2/meta/target.go | 20 + internal/querycoordv2/meta/target_manager.go | 27 ++ .../querycoordv2/meta/target_manager_test.go | 61 +++ internal/querycoordv2/server.go | 33 ++ internal/querynodev2/metrics_info.go | 51 +++ internal/querynodev2/metrics_info_test.go | 131 ++++++ internal/querynodev2/pipeline/manager.go | 23 ++ internal/querynodev2/pipeline/pipeline.go | 5 + internal/querynodev2/server.go | 10 + internal/util/flowgraph/flow_graph.go | 13 +- internal/util/flowgraph/node.go | 14 +- internal/util/flowgraph/node_test.go | 6 +- internal/util/metrics/utils.go | 58 ++- internal/util/pipeline/stream_pipeline.go | 25 +- pkg/util/metricsinfo/metric_request.go | 20 +- pkg/util/metricsinfo/metrics_info.go | 113 ++++- pkg/util/metricsinfo/utils.go | 22 + pkg/util/typeutil/time.go | 5 + 58 files changed, 2834 insertions(+), 251 deletions(-) delete mode 100644 internal/http/webui/nodes.html create mode 100644 internal/querycoordv2/handlers_test.go create mode 100644 internal/querycoordv2/meta/dist_manager_test.go create mode 100644 internal/querynodev2/metrics_info_test.go diff --git a/internal/datacoord/channel_manager.go b/internal/datacoord/channel_manager.go index c413cb77b0cee..61ca79848537b 100644 --- a/internal/datacoord/channel_manager.go +++ b/internal/datacoord/channel_manager.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/errors" "github.com/samber/lo" "go.uber.org/zap" + "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/proto/datapb" @@ -53,6 +54,8 @@ type ChannelManager interface { GetNodeChannelsByCollectionID(collectionID int64) map[int64][]string GetChannelsByCollectionID(collectionID int64) []RWChannel GetChannelNamesByCollectionID(collectionID int64) []string + + GetChannelWatchInfos() map[int64]map[string]*datapb.ChannelWatchInfo } // An interface sessionManager implments @@ -739,6 +742,22 @@ func (m *ChannelManagerImpl) fillChannelWatchInfo(op *ChannelOp) error { return nil } +func (m *ChannelManagerImpl) GetChannelWatchInfos() map[int64]map[string]*datapb.ChannelWatchInfo { + m.mu.RLock() + defer m.mu.RUnlock() + infos := make(map[int64]map[string]*datapb.ChannelWatchInfo) + for _, nc := range m.store.GetNodesChannels() { + for _, ch := range nc.Channels { + watchInfo := proto.Clone(ch.GetWatchInfo()).(*datapb.ChannelWatchInfo) + if _, ok := infos[nc.NodeID]; !ok { + infos[nc.NodeID] = make(map[string]*datapb.ChannelWatchInfo) + } + infos[nc.NodeID][watchInfo.Vchan.ChannelName] = watchInfo + } + } + return infos +} + func inferStateByOpType(opType ChannelOpType) datapb.ChannelWatchState { switch opType { case Watch: diff --git a/internal/datacoord/channel_manager_test.go b/internal/datacoord/channel_manager_test.go index 919fbd1831061..450a260754eef 100644 --- a/internal/datacoord/channel_manager_test.go +++ b/internal/datacoord/channel_manager_test.go @@ -805,3 +805,51 @@ func (s *ChannelManagerSuite) TestStartupRootCoordFailed() { func (s *ChannelManagerSuite) TestCheckLoop() {} func (s *ChannelManagerSuite) TestGet() {} + +func (s *ChannelManagerSuite) TestGetChannelWatchInfos() { + store := NewMockRWChannelStore(s.T()) + store.EXPECT().GetNodesChannels().Return([]*NodeChannelInfo{ + { + NodeID: 1, + Channels: map[string]RWChannel{ + "ch1": &channelMeta{ + WatchInfo: &datapb.ChannelWatchInfo{ + Vchan: &datapb.VchannelInfo{ + ChannelName: "ch1", + }, + StartTs: 100, + State: datapb.ChannelWatchState_ToWatch, + OpID: 1, + }, + }, + }, + }, + { + NodeID: 2, + Channels: map[string]RWChannel{ + "ch2": &channelMeta{ + WatchInfo: &datapb.ChannelWatchInfo{ + Vchan: &datapb.VchannelInfo{ + ChannelName: "ch2", + }, + StartTs: 10, + State: datapb.ChannelWatchState_WatchSuccess, + OpID: 1, + }, + }, + }, + }, + }) + + cm := &ChannelManagerImpl{store: store} + infos := cm.GetChannelWatchInfos() + s.Equal(2, len(infos)) + s.Equal("ch1", infos[1]["ch1"].GetVchan().ChannelName) + s.Equal("ch2", infos[2]["ch2"].GetVchan().ChannelName) + + // test empty value + store.EXPECT().GetNodesChannels().Unset() + store.EXPECT().GetNodesChannels().Return([]*NodeChannelInfo{}) + infos = cm.GetChannelWatchInfos() + s.Equal(0, len(infos)) +} diff --git a/internal/datacoord/index_meta.go b/internal/datacoord/index_meta.go index cf8eb7d42d7a2..dbc60101d00cd 100644 --- a/internal/datacoord/index_meta.go +++ b/internal/datacoord/index_meta.go @@ -25,11 +25,11 @@ import ( "sync" "time" - "github.com/golang/protobuf/proto" "github.com/hashicorp/golang-lru/v2/expirable" "github.com/prometheus/client_golang/prometheus" "github.com/samber/lo" "go.uber.org/zap" + "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index 7aaaecd14c609..257e8980cb6cf 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -2025,3 +2025,28 @@ func (m *meta) SaveStatsResultSegment(oldSegmentID int64, result *workerpb.Stats return metricMutation, nil } + +func (m *meta) getSegmentsMetrics() []*metricsinfo.Segment { + m.RLock() + defer m.RUnlock() + + segments := make([]*metricsinfo.Segment, 0, len(m.segments.segments)) + for _, s := range m.segments.segments { + segments = append(segments, &metricsinfo.Segment{ + SegmentID: s.ID, + CollectionID: s.CollectionID, + PartitionID: s.PartitionID, + Channel: s.InsertChannel, + NumOfRows: s.NumOfRows, + State: s.State.String(), + MemSize: s.size.Load(), + Level: s.Level.String(), + IsImporting: s.IsImporting, + Compacted: s.Compacted, + IsSorted: s.IsSorted, + NodeID: paramtable.GetNodeID(), + }) + } + + return segments +} diff --git a/internal/datacoord/meta_test.go b/internal/datacoord/meta_test.go index 741c227ec6e6d..83f2a3d07892e 100644 --- a/internal/datacoord/meta_test.go +++ b/internal/datacoord/meta_test.go @@ -26,6 +26,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/suite" + "golang.org/x/exp/slices" "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" @@ -43,6 +44,7 @@ import ( "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/util" "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/testutils" ) @@ -1319,3 +1321,62 @@ func Test_meta_ReloadCollectionsFromRootcoords(t *testing.T) { assert.NotNil(t, c) }) } + +func TestMeta_GetSegmentsJSON(t *testing.T) { + // Create a mock meta object + m := &meta{ + segments: &SegmentsInfo{ + segments: map[int64]*SegmentInfo{ + 1: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + CollectionID: 1, + PartitionID: 1, + InsertChannel: "channel1", + NumOfRows: 100, + State: commonpb.SegmentState_Growing, + MaxRowNum: 1000, + Compacted: false, + }, + }, + 2: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 2, + CollectionID: 2, + PartitionID: 2, + InsertChannel: "channel2", + NumOfRows: 200, + State: commonpb.SegmentState_Sealed, + MaxRowNum: 2000, + Compacted: true, + }, + }, + }, + }, + } + + segments := m.getSegmentsMetrics() + + // Check the length of the segments + assert.Equal(t, 2, len(segments)) + + slices.SortFunc(segments, func(i, j *metricsinfo.Segment) int { return int(i.SegmentID - j.SegmentID) }) + + // Check the first segment + assert.Equal(t, int64(1), segments[0].SegmentID) + assert.Equal(t, int64(1), segments[0].CollectionID) + assert.Equal(t, int64(1), segments[0].PartitionID) + assert.Equal(t, "channel1", segments[0].Channel) + assert.Equal(t, int64(100), segments[0].NumOfRows) + assert.Equal(t, "Growing", segments[0].State) + assert.False(t, segments[0].Compacted) + + // Check the second segment + assert.Equal(t, int64(2), segments[1].SegmentID) + assert.Equal(t, int64(2), segments[1].CollectionID) + assert.Equal(t, int64(2), segments[1].PartitionID) + assert.Equal(t, "channel2", segments[1].Channel) + assert.Equal(t, int64(200), segments[1].NumOfRows) + assert.Equal(t, "Sealed", segments[1].State) + assert.True(t, segments[1].Compacted) +} diff --git a/internal/datacoord/metrics_info.go b/internal/datacoord/metrics_info.go index 1728aee08d0d9..60ca296679dcb 100644 --- a/internal/datacoord/metrics_info.go +++ b/internal/datacoord/metrics_info.go @@ -19,6 +19,7 @@ package datacoord import ( "context" "encoding/json" + "sync" "github.com/cockroachdb/errors" "go.uber.org/zap" @@ -27,8 +28,10 @@ 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/internal/datacoord/session" + "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/indexpb" "github.com/milvus-io/milvus/internal/types" + "github.com/milvus-io/milvus/internal/util/metrics" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/hardware" "github.com/milvus-io/milvus/pkg/util/merr" @@ -82,74 +85,90 @@ func (s *Server) getCollectionMetrics(ctx context.Context) *metricsinfo.DataCoor return ret } -// GetSyncTaskMetrics retrieves and aggregates the sync task metrics of the datanode. -func (s *Server) GetSyncTaskMetrics( - ctx context.Context, - req *milvuspb.GetMetricsRequest, -) (string, error) { - resp, err := s.requestDataNodeGetMetrics(ctx, req) - if err != nil { - return "", err +func (s *Server) getChannelsJSON(ctx context.Context, req *milvuspb.GetMetricsRequest) (string, error) { + channels, err := getMetrics[*metricsinfo.Channel](s, ctx, req) + // fill checkpoint timestamp + channel2Checkpoints := s.meta.GetChannelCheckpoints() + for _, channel := range channels { + if cp, ok := channel2Checkpoints[channel.Name]; ok { + channel.CheckpointTS = typeutil.TimestampToString(cp.GetTimestamp()) + } else { + log.Warn("channel not found in meta cache", zap.String("channel", channel.Name)) + } } + return metricsinfo.MarshalGetMetricsValues(channels, err) +} - tasks := make(map[string][]*metricsinfo.SyncTask, resp.Len()) - resp.Range(func(key string, value *milvuspb.GetMetricsResponse) bool { - if value.Response != "" { - var sts []*metricsinfo.SyncTask - if err1 := json.Unmarshal([]byte(value.Response), &sts); err1 != nil { - log.Warn("failed to unmarshal sync task metrics") - err = err1 - return false +// mergeChannels merges the channel metrics from data nodes and channel watch infos from channel manager +// dnChannels: a slice of Channel metrics from data nodes +// dcChannels: a map of channel watch infos from the channel manager, keyed by node ID and channel name +func mergeChannels(dnChannels []*metricsinfo.Channel, dcChannels map[int64]map[string]*datapb.ChannelWatchInfo) []*metricsinfo.Channel { + mergedChannels := make([]*metricsinfo.Channel, 0) + + // Add or update channels from data nodes + for _, dnChannel := range dnChannels { + if dcChannelMap, ok := dcChannels[dnChannel.NodeID]; ok { + if dcChannel, ok := dcChannelMap[dnChannel.Name]; ok { + dnChannel.WatchState = dcChannel.State.String() + delete(dcChannelMap, dnChannel.Name) } - tasks[key] = sts } - return true - }) - - if err != nil { - return "", err + mergedChannels = append(mergedChannels, dnChannel) } - if len(tasks) == 0 { - return "", nil + // Add remaining channels from channel manager + for nodeID, dcChannelMap := range dcChannels { + for _, dcChannel := range dcChannelMap { + mergedChannels = append(mergedChannels, &metricsinfo.Channel{ + Name: dcChannel.Vchan.ChannelName, + CollectionID: dcChannel.Vchan.CollectionID, + WatchState: dcChannel.State.String(), + NodeID: nodeID, + }) + } } - bs, err := json.Marshal(tasks) - if err != nil { - return "", err - } - return (string)(bs), nil + return mergedChannels } -func (s *Server) requestDataNodeGetMetrics( - ctx context.Context, - req *milvuspb.GetMetricsRequest, -) (*typeutil.ConcurrentMap[string, *milvuspb.GetMetricsResponse], error) { - nodes := s.cluster.GetSessions() +func (s *Server) getDistJSON(ctx context.Context, req *milvuspb.GetMetricsRequest) string { + segments := s.meta.getSegmentsMetrics() + var channels []*metricsinfo.DmChannel + for nodeID, ch := range s.channelManager.GetChannelWatchInfos() { + for _, chInfo := range ch { + dmChannel := metrics.NewDMChannelFrom(chInfo.GetVchan()) + dmChannel.NodeID = nodeID + dmChannel.WatchState = chInfo.State.String() + dmChannel.StartWatchTS = chInfo.GetStartTs() + channels = append(channels, dmChannel) + } + } - rets := typeutil.NewConcurrentMap[string, *milvuspb.GetMetricsResponse]() - wg, ctx := errgroup.WithContext(ctx) - for _, node := range nodes { - wg.Go(func() error { - cli, err := node.GetOrCreateClient(ctx) - if err != nil { - return err - } - ret, err := cli.GetMetrics(ctx, req) - if err != nil { - return err - } - key := metricsinfo.ConstructComponentName(typeutil.DataNodeRole, node.NodeID()) - rets.Insert(key, ret) - return nil - }) + if len(segments) == 0 && len(channels) == 0 { + return "" } - err := wg.Wait() + dist := &metricsinfo.DataCoordDist{ + Segments: segments, + DMChannels: channels, + } + + bs, err := json.Marshal(dist) if err != nil { - return nil, err + log.Warn("marshal dist value failed", zap.String("err", err.Error())) + return "" } - return rets, nil + return string(bs) +} + +func (s *Server) getSegmentsJSON(ctx context.Context, req *milvuspb.GetMetricsRequest) (string, error) { + ret, err := getMetrics[*metricsinfo.Segment](s, ctx, req) + return metricsinfo.MarshalGetMetricsValues(ret, err) +} + +func (s *Server) getSyncTaskJSON(ctx context.Context, req *milvuspb.GetMetricsRequest) (string, error) { + ret, err := getMetrics[*metricsinfo.SyncTask](s, ctx, req) + return metricsinfo.MarshalGetMetricsValues(ret, err) } // getSystemInfoMetrics composes data cluster metrics @@ -322,3 +341,44 @@ func (s *Server) getIndexNodeMetrics(ctx context.Context, req *milvuspb.GetMetri infos.BaseComponentInfos.HasError = false return infos, nil } + +// getMetrics retrieves and aggregates the metrics of the datanode to a slice +func getMetrics[T any](s *Server, ctx context.Context, req *milvuspb.GetMetricsRequest) ([]T, error) { + var metrics []T + var mu sync.Mutex + errorGroup, ctx := errgroup.WithContext(ctx) + + nodes := s.cluster.GetSessions() + for _, node := range nodes { + errorGroup.Go(func() error { + cli, err := node.GetOrCreateClient(ctx) + if err != nil { + return err + } + resp, err := cli.GetMetrics(ctx, req) + if err != nil { + log.Warn("failed to get metric from DataNode", zap.Int64("nodeID", node.NodeID())) + return err + } + + if resp.Response == "" { + return nil + } + + var infos []T + err = json.Unmarshal([]byte(resp.Response), &infos) + if err != nil { + log.Warn("invalid metrics of data node was found", zap.Error(err)) + return err + } + + mu.Lock() + metrics = append(metrics, infos...) + mu.Unlock() + return nil + }) + } + + err := errorGroup.Wait() + return metrics, err +} diff --git a/internal/datacoord/metrics_info_test.go b/internal/datacoord/metrics_info_test.go index f85d43998ef95..fe6114c067052 100644 --- a/internal/datacoord/metrics_info_test.go +++ b/internal/datacoord/metrics_info_test.go @@ -18,17 +18,22 @@ package datacoord import ( "context" + "encoding/json" "testing" "github.com/cockroachdb/errors" "github.com/stretchr/testify/assert" "google.golang.org/grpc" + "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/msgpb" "github.com/milvus-io/milvus/internal/datacoord/session" + "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/types" "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/typeutil" ) @@ -206,10 +211,25 @@ func TestGetSyncTaskMetrics(t *testing.T) { req := &milvuspb.GetMetricsRequest{} ctx := context.Background() - task := `[{"segment_id": 1, "batch_rows": 100, "segment_level": "L0", "ts_from": 1000, "ts_to": 2000,"delta_row_count": 10, "flush_size": 1024, "running_time": 2000000000}]` + tasks := []metricsinfo.SyncTask{ + { + SegmentID: 1, + BatchRows: 100, + SegmentLevel: "L0", + TSFrom: 1000, + TSTo: 2000, + DeltaRowCount: 10, + FlushSize: 1024, + RunningTime: "2h", + }, + } + tasksBytes, err := json.Marshal(tasks) + assert.NoError(t, err) + expectedJSON := string(tasksBytes) + mockResp := &milvuspb.GetMetricsResponse{ Status: merr.Success(), - Response: task, + Response: expectedJSON, } mockClient := &mockMetricDataNodeClient{ @@ -226,9 +246,8 @@ func TestGetSyncTaskMetrics(t *testing.T) { mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)}) svr.cluster = mockCluster - actualJSON, err := svr.GetSyncTaskMetrics(ctx, req) + actualJSON, err := svr.getSyncTaskJSON(ctx, req) assert.NoError(t, err) - expectedJSON := `{"datanode1":[{"segment_id":1,"batch_rows":100,"segment_level":"L0","ts_from":1000,"ts_to":2000,"delta_row_count":10,"flush_size":1024,"running_time":2000000000}]}` assert.Equal(t, expectedJSON, actualJSON) }) @@ -250,7 +269,7 @@ func TestGetSyncTaskMetrics(t *testing.T) { mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)}) svr.cluster = mockCluster - actualJSON, err := svr.GetSyncTaskMetrics(ctx, req) + actualJSON, err := svr.getSyncTaskJSON(ctx, req) assert.Error(t, err) assert.Equal(t, "", actualJSON) }) @@ -278,7 +297,7 @@ func TestGetSyncTaskMetrics(t *testing.T) { mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)}) svr.cluster = mockCluster - actualJSON, err := svr.GetSyncTaskMetrics(ctx, req) + actualJSON, err := svr.getSyncTaskJSON(ctx, req) assert.Error(t, err) assert.Equal(t, "", actualJSON) }) @@ -307,8 +326,360 @@ func TestGetSyncTaskMetrics(t *testing.T) { svr.cluster = mockCluster expectedJSON := "" - actualJSON, err := svr.GetSyncTaskMetrics(ctx, req) + actualJSON, err := svr.getSyncTaskJSON(ctx, req) + assert.NoError(t, err) + assert.Equal(t, expectedJSON, actualJSON) + }) +} + +func TestGetSegmentsJSON(t *testing.T) { + svr := Server{} + t.Run("ReturnsCorrectJSON", func(t *testing.T) { + req := &milvuspb.GetMetricsRequest{} + ctx := context.Background() + + segments := []*metricsinfo.Segment{ + { + SegmentID: 1, + CollectionID: 100, + PartitionID: 10, + NumOfRows: 1000, + State: "Flushed", + }, + } + segmentsBytes, err := json.Marshal(segments) + assert.NoError(t, err) + expectedJSON := string(segmentsBytes) + + mockResp := &milvuspb.GetMetricsResponse{ + Status: merr.Success(), + Response: expectedJSON, + } + + mockClient := &mockMetricDataNodeClient{ + mock: func() (*milvuspb.GetMetricsResponse, error) { + return mockResp, nil + }, + } + + dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) { + return mockClient, nil + } + + mockCluster := NewMockCluster(t) + mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)}) + svr.cluster = mockCluster + + actualJSON, err := svr.getSegmentsJSON(ctx, req) + assert.NoError(t, err) + assert.Equal(t, expectedJSON, actualJSON) + }) + + t.Run("ReturnsErrorOnRequestFailure", func(t *testing.T) { + req := &milvuspb.GetMetricsRequest{} + ctx := context.Background() + + mockClient := &mockMetricDataNodeClient{ + mock: func() (*milvuspb.GetMetricsResponse, error) { + return nil, errors.New("request failed") + }, + } + + dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) { + return mockClient, nil + } + + mockCluster := NewMockCluster(t) + mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)}) + svr.cluster = mockCluster + + actualJSON, err := svr.getSegmentsJSON(ctx, req) + assert.Error(t, err) + assert.Equal(t, "", actualJSON) + }) + + t.Run("ReturnsErrorOnUnmarshalFailure", func(t *testing.T) { + req := &milvuspb.GetMetricsRequest{} + ctx := context.Background() + + mockResp := &milvuspb.GetMetricsResponse{ + Status: merr.Success(), + Response: `invalid json`, + } + + mockClient := &mockMetricDataNodeClient{ + mock: func() (*milvuspb.GetMetricsResponse, error) { + return mockResp, nil + }, + } + + dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) { + return mockClient, nil + } + + mockCluster := NewMockCluster(t) + mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)}) + svr.cluster = mockCluster + + actualJSON, err := svr.getSegmentsJSON(ctx, req) + assert.Error(t, err) + assert.Equal(t, "", actualJSON) + }) + + t.Run("ReturnsEmptyJSONWhenNoSegments", func(t *testing.T) { + req := &milvuspb.GetMetricsRequest{} + ctx := context.Background() + + mockResp := &milvuspb.GetMetricsResponse{ + Status: merr.Success(), + Response: "", + } + + mockClient := &mockMetricDataNodeClient{ + mock: func() (*milvuspb.GetMetricsResponse, error) { + return mockResp, nil + }, + } + + dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) { + return mockClient, nil + } + + mockCluster := NewMockCluster(t) + mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)}) + svr.cluster = mockCluster + + expectedJSON := "" + actualJSON, err := svr.getSegmentsJSON(ctx, req) + assert.NoError(t, err) + assert.Equal(t, expectedJSON, actualJSON) + }) +} + +func TestGetChannelsJSON(t *testing.T) { + svr := Server{} + t.Run("ReturnsCorrectJSON", func(t *testing.T) { + req := &milvuspb.GetMetricsRequest{} + + channels := []*metricsinfo.Channel{ + { + Name: "channel1", + CollectionID: 100, + NodeID: 1, + }, + } + channelsBytes, err := json.Marshal(channels) + assert.NoError(t, err) + channelJSON := string(channelsBytes) + + mockResp := &milvuspb.GetMetricsResponse{ + Status: merr.Success(), + Response: channelJSON, + } + + mockClient := &mockMetricDataNodeClient{ + mock: func() (*milvuspb.GetMetricsResponse, error) { + return mockResp, nil + }, + } + + dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) { + return mockClient, nil + } + + mockCluster := NewMockCluster(t) + mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)}) + svr.cluster = mockCluster + + svr.meta = &meta{channelCPs: newChannelCps()} + svr.meta.channelCPs.checkpoints["channel1"] = &msgpb.MsgPosition{Timestamp: 1000} + + actualJSON, err := svr.getChannelsJSON(context.TODO(), req) + assert.NoError(t, err) + + channels = []*metricsinfo.Channel{ + { + Name: "channel1", + CollectionID: 100, + NodeID: 1, + CheckpointTS: typeutil.TimestampToString(1000), + }, + } + channelsBytes, err = json.Marshal(channels) + assert.NoError(t, err) + expectedJSON := string(channelsBytes) + + assert.Equal(t, expectedJSON, actualJSON) + }) + + t.Run("ReturnsErrorOnRequestFailure", func(t *testing.T) { + req := &milvuspb.GetMetricsRequest{} + ctx := context.Background() + + mockClient := &mockMetricDataNodeClient{ + mock: func() (*milvuspb.GetMetricsResponse, error) { + return nil, errors.New("request failed") + }, + } + + dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) { + return mockClient, nil + } + + mockCluster := NewMockCluster(t) + mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)}) + svr.cluster = mockCluster + + svr.meta = &meta{channelCPs: newChannelCps()} + + actualJSON, err := svr.getChannelsJSON(ctx, req) + assert.Error(t, err) + assert.Equal(t, "", actualJSON) + }) + + t.Run("ReturnsErrorOnUnmarshalFailure", func(t *testing.T) { + req := &milvuspb.GetMetricsRequest{} + ctx := context.Background() + + mockResp := &milvuspb.GetMetricsResponse{ + Status: merr.Success(), + Response: `invalid json`, + } + + mockClient := &mockMetricDataNodeClient{ + mock: func() (*milvuspb.GetMetricsResponse, error) { + return mockResp, nil + }, + } + + dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) { + return mockClient, nil + } + + mockCluster := NewMockCluster(t) + mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)}) + svr.cluster = mockCluster + + svr.meta = &meta{channelCPs: newChannelCps()} + + actualJSON, err := svr.getChannelsJSON(ctx, req) + assert.Error(t, err) + assert.Equal(t, "", actualJSON) + }) + + t.Run("ReturnsEmptyJSONWhenNoChannels", func(t *testing.T) { + req := &milvuspb.GetMetricsRequest{} + ctx := context.Background() + + mockResp := &milvuspb.GetMetricsResponse{ + Status: merr.Success(), + Response: "", + } + + mockClient := &mockMetricDataNodeClient{ + mock: func() (*milvuspb.GetMetricsResponse, error) { + return mockResp, nil + }, + } + + dataNodeCreator := func(ctx context.Context, addr string, nodeID int64) (types.DataNodeClient, error) { + return mockClient, nil + } + + mockCluster := NewMockCluster(t) + mockCluster.EXPECT().GetSessions().Return([]*session.Session{session.NewSession(&session.NodeInfo{NodeID: 1}, dataNodeCreator)}) + svr.cluster = mockCluster + svr.meta = &meta{channelCPs: newChannelCps()} + + expectedJSON := "" + actualJSON, err := svr.getChannelsJSON(ctx, req) + assert.NoError(t, err) + assert.Equal(t, expectedJSON, actualJSON) + }) +} + +func TestGetDistJSON(t *testing.T) { + svr := Server{} + nodeID := paramtable.GetNodeID() + paramtable.SetNodeID(1) + defer paramtable.SetNodeID(nodeID) + + t.Run("ReturnsCorrectJSON", func(t *testing.T) { + req := &milvuspb.GetMetricsRequest{} + ctx := context.Background() + + svr.meta = &meta{ + segments: &SegmentsInfo{ + segments: map[int64]*SegmentInfo{ + 1: { + SegmentInfo: &datapb.SegmentInfo{ + ID: 1, + CollectionID: 1, + PartitionID: 1, + InsertChannel: "channel1", + Level: datapb.SegmentLevel_L1, + State: commonpb.SegmentState_Flushed, + }, + }, + }, + }, + } + + cm := NewMockChannelManager(t) + cm.EXPECT().GetChannelWatchInfos().Return(map[int64]map[string]*datapb.ChannelWatchInfo{ + 1: { + "channel1": { + State: datapb.ChannelWatchState_ToWatch, + Vchan: &datapb.VchannelInfo{ + ChannelName: "channel1", + }, + }, + }, + }) + + svr.channelManager = cm + + segments := []*metricsinfo.Segment{ + { + SegmentID: 1, + State: commonpb.SegmentState_Flushed.String(), + CollectionID: 1, + PartitionID: 1, + Channel: "channel1", + Level: datapb.SegmentLevel_L1.String(), + NodeID: 1, + }, + } + channels := []*metricsinfo.DmChannel{ + { + ChannelName: "channel1", + NodeID: 1, + WatchState: datapb.ChannelWatchState_ToWatch.String(), + }, + } + dist := &metricsinfo.DataCoordDist{ + Segments: segments, + DMChannels: channels, + } + distBytes, err := json.Marshal(dist) assert.NoError(t, err) + expectedJSON := string(distBytes) + + actualJSON := svr.getDistJSON(ctx, req) + assert.Equal(t, expectedJSON, actualJSON) + }) + + t.Run("ReturnsEmptyJSONWhenNoDist", func(t *testing.T) { + req := &milvuspb.GetMetricsRequest{} + ctx := context.Background() + + svr.meta = &meta{segments: &SegmentsInfo{segments: map[int64]*SegmentInfo{}}} + cm := NewMockChannelManager(t) + cm.EXPECT().GetChannelWatchInfos().Return(map[int64]map[string]*datapb.ChannelWatchInfo{}) + + svr.channelManager = cm + expectedJSON := "" + actualJSON := svr.getDistJSON(ctx, req) assert.Equal(t, expectedJSON, actualJSON) }) } diff --git a/internal/datacoord/mock_channelmanager.go b/internal/datacoord/mock_channelmanager.go index 9a61852db87c3..fccecee419de2 100644 --- a/internal/datacoord/mock_channelmanager.go +++ b/internal/datacoord/mock_channelmanager.go @@ -5,6 +5,7 @@ package datacoord import ( context "context" + datapb "github.com/milvus-io/milvus/internal/proto/datapb" mock "github.com/stretchr/testify/mock" ) @@ -308,6 +309,53 @@ func (_c *MockChannelManager_GetChannelNamesByCollectionID_Call) RunAndReturn(ru return _c } +// GetChannelWatchInfos provides a mock function with given fields: +func (_m *MockChannelManager) GetChannelWatchInfos() map[int64]map[string]*datapb.ChannelWatchInfo { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetChannelWatchInfos") + } + + var r0 map[int64]map[string]*datapb.ChannelWatchInfo + if rf, ok := ret.Get(0).(func() map[int64]map[string]*datapb.ChannelWatchInfo); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(map[int64]map[string]*datapb.ChannelWatchInfo) + } + } + + return r0 +} + +// MockChannelManager_GetChannelWatchInfos_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetChannelWatchInfos' +type MockChannelManager_GetChannelWatchInfos_Call struct { + *mock.Call +} + +// GetChannelWatchInfos is a helper method to define mock.On call +func (_e *MockChannelManager_Expecter) GetChannelWatchInfos() *MockChannelManager_GetChannelWatchInfos_Call { + return &MockChannelManager_GetChannelWatchInfos_Call{Call: _e.mock.On("GetChannelWatchInfos")} +} + +func (_c *MockChannelManager_GetChannelWatchInfos_Call) Run(run func()) *MockChannelManager_GetChannelWatchInfos_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockChannelManager_GetChannelWatchInfos_Call) Return(_a0 map[int64]map[string]*datapb.ChannelWatchInfo) *MockChannelManager_GetChannelWatchInfos_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockChannelManager_GetChannelWatchInfos_Call) RunAndReturn(run func() map[int64]map[string]*datapb.ChannelWatchInfo) *MockChannelManager_GetChannelWatchInfos_Call { + _c.Call.Return(run) + return _c +} + // GetChannelsByCollectionID provides a mock function with given fields: collectionID func (_m *MockChannelManager) GetChannelsByCollectionID(collectionID int64) []RWChannel { ret := _m.Called(collectionID) diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index a1a9db46bea9c..21e34ce602814 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -1141,6 +1141,11 @@ func (s *Server) registerMetricsRequest() { return s.getSystemInfoMetrics(ctx, req) }) + s.metricsRequest.RegisterMetricsRequest(metricsinfo.DataDist, + func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { + return s.getDistJSON(ctx, req), nil + }) + s.metricsRequest.RegisterMetricsRequest(metricsinfo.ImportTasks, func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { return s.importMeta.TaskStatsJSON(), nil @@ -1158,8 +1163,19 @@ func (s *Server) registerMetricsRequest() { s.metricsRequest.RegisterMetricsRequest(metricsinfo.SyncTasks, func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { - return s.GetSyncTaskMetrics(ctx, req) + return s.getSyncTaskJSON(ctx, req) + }) + + s.metricsRequest.RegisterMetricsRequest(metricsinfo.DataSegments, + func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { + return s.getSegmentsJSON(ctx, req) }) + + s.metricsRequest.RegisterMetricsRequest(metricsinfo.DataChannels, + func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { + return s.getChannelsJSON(ctx, req) + }) + log.Info("register metrics actions finished") } diff --git a/internal/datacoord/stats_task_meta.go b/internal/datacoord/stats_task_meta.go index 2c46d68cfd6c2..4734ee45c76b9 100644 --- a/internal/datacoord/stats_task_meta.go +++ b/internal/datacoord/stats_task_meta.go @@ -22,8 +22,8 @@ import ( "strconv" "sync" - "github.com/golang/protobuf/proto" "go.uber.org/zap" + "google.golang.org/protobuf/proto" "github.com/milvus-io/milvus/internal/metastore" "github.com/milvus-io/milvus/internal/proto/indexpb" diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index eccba38a1580c..bbf62f484db9d 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -284,10 +284,21 @@ func (node *DataNode) registerMetricsRequest() { func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { return node.getSystemInfoMetrics(ctx, req) }) + node.metricsRequest.RegisterMetricsRequest(metricsinfo.SyncTasks, func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { return node.syncMgr.TaskStatsJSON(), nil }) + + node.metricsRequest.RegisterMetricsRequest(metricsinfo.DataSegments, + func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { + return node.flowgraphManager.GetSegmentsJSON(), nil + }) + + node.metricsRequest.RegisterMetricsRequest(metricsinfo.DataChannels, + func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { + return node.flowgraphManager.GetChannelsJSON(), nil + }) log.Info("register metrics actions finished") } diff --git a/internal/flushcommon/metacache/segment.go b/internal/flushcommon/metacache/segment.go index 8c4906ff7201e..4bdadda6177ad 100644 --- a/internal/flushcommon/metacache/segment.go +++ b/internal/flushcommon/metacache/segment.go @@ -87,6 +87,14 @@ func (s *SegmentInfo) Level() datapb.SegmentLevel { return s.level } +func (s *SegmentInfo) BufferRows() int64 { + return s.bufferRows +} + +func (s *SegmentInfo) SyncingRows() int64 { + return s.syncingRows +} + func (s *SegmentInfo) Clone() *SegmentInfo { return &SegmentInfo{ segmentID: s.segmentID, diff --git a/internal/flushcommon/pipeline/flow_graph_manager.go b/internal/flushcommon/pipeline/flow_graph_manager.go index 2d8b930442f81..1cf36a177417f 100644 --- a/internal/flushcommon/pipeline/flow_graph_manager.go +++ b/internal/flushcommon/pipeline/flow_graph_manager.go @@ -18,6 +18,7 @@ package pipeline import ( "context" + "encoding/json" "fmt" "go.uber.org/zap" @@ -25,6 +26,7 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/typeutil" ) @@ -40,6 +42,8 @@ type FlowgraphManager interface { GetFlowgraphCount() int GetCollectionIDs() []int64 + GetChannelsJSON() string + GetSegmentsJSON() string Close() } @@ -115,6 +119,59 @@ func (fm *fgManagerImpl) GetCollectionIDs() []int64 { return collectionSet.Collect() } +// GetChannelsJSON returns all channels in json format. +func (fm *fgManagerImpl) GetChannelsJSON() string { + var channels []*metricsinfo.Channel + fm.flowgraphs.Range(func(ch string, ds *DataSyncService) bool { + latestTimeTick := ds.timetickSender.GetLatestTimestamp(ch) + channels = append(channels, &metricsinfo.Channel{ + Name: ch, + WatchState: ds.fg.Status(), + LatestTimeTick: typeutil.TimestampToString(latestTimeTick), + NodeID: paramtable.GetNodeID(), + CollectionID: ds.metacache.Collection(), + }) + return true + }) + + ret, err := json.Marshal(channels) + if err != nil { + log.Warn("failed to marshal channels", zap.Error(err)) + return "" + } + return string(ret) +} + +func (fm *fgManagerImpl) GetSegmentsJSON() string { + var segments []*metricsinfo.Segment + fm.flowgraphs.Range(func(ch string, ds *DataSyncService) bool { + meta := ds.metacache + for _, segment := range meta.GetSegmentsBy() { + segments = append(segments, &metricsinfo.Segment{ + SegmentID: segment.SegmentID(), + CollectionID: meta.Collection(), + PartitionID: segment.PartitionID(), + Channel: ch, + State: segment.State().String(), + Level: segment.Level().String(), + NodeID: paramtable.GetNodeID(), + NumOfRows: segment.NumOfRows(), + FlushedRows: segment.FlushedRows(), + SyncBufferRows: segment.BufferRows(), + SyncingRows: segment.SyncingRows(), + }) + } + return true + }) + + ret, err := json.Marshal(segments) + if err != nil { + log.Warn("failed to marshal segments", zap.Error(err)) + return "" + } + return string(ret) +} + func (fm *fgManagerImpl) Close() { fm.cancelFunc() } diff --git a/internal/flushcommon/pipeline/flow_graph_manager_test.go b/internal/flushcommon/pipeline/flow_graph_manager_test.go index aaa25cb4f5196..3cb27e74459f7 100644 --- a/internal/flushcommon/pipeline/flow_graph_manager_test.go +++ b/internal/flushcommon/pipeline/flow_graph_manager_test.go @@ -18,8 +18,8 @@ package pipeline import ( "context" + "encoding/json" "fmt" - "math/rand" "os" "testing" @@ -30,15 +30,20 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datanode/allocator" "github.com/milvus-io/milvus/internal/flushcommon/broker" + "github.com/milvus-io/milvus/internal/flushcommon/metacache" + "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" "github.com/milvus-io/milvus/internal/flushcommon/syncmgr" "github.com/milvus-io/milvus/internal/flushcommon/util" "github.com/milvus-io/milvus/internal/flushcommon/writebuffer" "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/util/metrics" "github.com/milvus-io/milvus/internal/util/sessionutil" "github.com/milvus-io/milvus/pkg/common" "github.com/milvus-io/milvus/pkg/mq/msgdispatcher" "github.com/milvus-io/milvus/pkg/mq/msgstream" + "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) func TestMain(t *testing.M) { @@ -98,7 +103,7 @@ func TestFlowGraphManager(t *testing.T) { } func generateChannelWatchInfo() *datapb.ChannelWatchInfo { - collectionID := int64(rand.Uint32()) + collectionID := int64(1) dmChannelName := fmt.Sprintf("%s_%d", "fake-ch-", collectionID) schema := &schemapb.CollectionSchema{ Name: fmt.Sprintf("%s_%d", "collection_", collectionID), @@ -124,3 +129,105 @@ func generateChannelWatchInfo() *datapb.ChannelWatchInfo { Schema: schema, } } + +type mockTimeSender struct{} + +func (m *mockTimeSender) Update(channel string, ts typeutil.Timestamp, stats []*commonpb.SegmentStats) { + panic("implement me") +} + +func (m *mockTimeSender) GetLatestTimestamp(channel string) typeutil.Timestamp { + return 0 +} + +func newFlowGraphManager(t *testing.T) (string, FlowgraphManager) { + mockBroker := broker.NewMockBroker(t) + mockBroker.EXPECT().ReportTimeTick(mock.Anything, mock.Anything).Return(nil).Maybe() + mockBroker.EXPECT().SaveBinlogPaths(mock.Anything, mock.Anything).Return(nil).Maybe() + mockBroker.EXPECT().GetSegmentInfo(mock.Anything, mock.Anything).Return([]*datapb.SegmentInfo{}, nil).Maybe() + + wbm := writebuffer.NewMockBufferManager(t) + wbm.EXPECT().Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) + + dispClient := msgdispatcher.NewMockClient(t) + dispClient.EXPECT().Register(mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(make(chan *msgstream.MsgPack), nil) + + pipelineParams := &util.PipelineParams{ + Ctx: context.TODO(), + Session: &sessionutil.Session{SessionRaw: sessionutil.SessionRaw{ServerID: 0}}, + Broker: mockBroker, + TimeTickSender: &mockTimeSender{}, + DispClient: dispClient, + WriteBufferManager: wbm, + } + + chanWatchInfo := generateChannelWatchInfo() + ds, err := NewDataSyncService( + context.TODO(), + pipelineParams, + chanWatchInfo, + util.NewTickler(), + ) + assert.NoError(t, err) + + fm := NewFlowgraphManager() + fm.AddFlowgraph(ds) + return ds.vchannelName, fm +} + +func TestGetChannelsJSON(t *testing.T) { + paramtable.SetNodeID(1) + _, fm := newFlowGraphManager(t) + obj := []*metricsinfo.Channel{ + { + Name: "fake-ch-_1", + WatchState: "Healthy", + LatestTimeTick: typeutil.TimestampToString(0), + NodeID: paramtable.GetNodeID(), + CollectionID: 1, + }, + } + + expectedBytes, err := json.Marshal(obj) + assert.NoError(t, err) + expectedJSON := string(expectedBytes) + + jsonResult := fm.GetChannelsJSON() + assert.JSONEq(t, expectedJSON, jsonResult) +} + +func TestGetSegmentJSON(t *testing.T) { + ch, fm := newFlowGraphManager(t) + ds, ok := fm.GetFlowgraphService(ch) + assert.True(t, ok) + + nodeID := paramtable.GetNodeID() + paramtable.SetNodeID(1) + defer paramtable.SetNodeID(nodeID) + + pkStatsFactory := func(*datapb.SegmentInfo) pkoracle.PkStat { + return pkoracle.NewBloomFilterSet() + } + segment := &datapb.SegmentInfo{ + ID: 1, + PartitionID: 10, + State: commonpb.SegmentState_Flushed, + Level: datapb.SegmentLevel_L1, + NumOfRows: 10240, + CollectionID: 1, + } + + s := metrics.NewSegmentFrom(segment) + s.NodeID = 1 + s.Channel = "fake-ch-_1" + s.FlushedRows = 10240 + + expectedBytes, err := json.Marshal([]*metricsinfo.Segment{s}) + assert.NoError(t, err) + expectedJSON := string(expectedBytes) + + ds.metacache.AddSegment(segment, pkStatsFactory, metacache.NoneBm25StatsFactory) + jsonResult := fm.GetSegmentsJSON() + fmt.Println(jsonResult) + assert.JSONEq(t, expectedJSON, jsonResult) +} diff --git a/internal/flushcommon/pipeline/mock_fgmanager.go b/internal/flushcommon/pipeline/mock_fgmanager.go index 6945e21ff271f..cf8cd6b2aa1ca 100644 --- a/internal/flushcommon/pipeline/mock_fgmanager.go +++ b/internal/flushcommon/pipeline/mock_fgmanager.go @@ -1,4 +1,4 @@ -// Code generated by mockery v2.32.4. DO NOT EDIT. +// Code generated by mockery v2.46.0. DO NOT EDIT. package pipeline @@ -114,10 +114,59 @@ func (_c *MockFlowgraphManager_Close_Call) RunAndReturn(run func()) *MockFlowgra return _c } +// GetChannelsJSON provides a mock function with given fields: +func (_m *MockFlowgraphManager) GetChannelsJSON() string { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetChannelsJSON") + } + + var r0 string + if rf, ok := ret.Get(0).(func() string); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(string) + } + + return r0 +} + +// MockFlowgraphManager_GetChannelsJSON_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetChannelsJSON' +type MockFlowgraphManager_GetChannelsJSON_Call struct { + *mock.Call +} + +// GetChannelsJSON is a helper method to define mock.On call +func (_e *MockFlowgraphManager_Expecter) GetChannelsJSON() *MockFlowgraphManager_GetChannelsJSON_Call { + return &MockFlowgraphManager_GetChannelsJSON_Call{Call: _e.mock.On("GetChannelsJSON")} +} + +func (_c *MockFlowgraphManager_GetChannelsJSON_Call) Run(run func()) *MockFlowgraphManager_GetChannelsJSON_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockFlowgraphManager_GetChannelsJSON_Call) Return(_a0 string) *MockFlowgraphManager_GetChannelsJSON_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockFlowgraphManager_GetChannelsJSON_Call) RunAndReturn(run func() string) *MockFlowgraphManager_GetChannelsJSON_Call { + _c.Call.Return(run) + return _c +} + // GetCollectionIDs provides a mock function with given fields: func (_m *MockFlowgraphManager) GetCollectionIDs() []int64 { ret := _m.Called() + if len(ret) == 0 { + panic("no return value specified for GetCollectionIDs") + } + var r0 []int64 if rf, ok := ret.Get(0).(func() []int64); ok { r0 = rf() @@ -161,6 +210,10 @@ func (_c *MockFlowgraphManager_GetCollectionIDs_Call) RunAndReturn(run func() [] func (_m *MockFlowgraphManager) GetFlowgraphCount() int { ret := _m.Called() + if len(ret) == 0 { + panic("no return value specified for GetFlowgraphCount") + } + var r0 int if rf, ok := ret.Get(0).(func() int); ok { r0 = rf() @@ -202,6 +255,10 @@ func (_c *MockFlowgraphManager_GetFlowgraphCount_Call) RunAndReturn(run func() i func (_m *MockFlowgraphManager) GetFlowgraphService(channel string) (*DataSyncService, bool) { ret := _m.Called(channel) + if len(ret) == 0 { + panic("no return value specified for GetFlowgraphService") + } + var r0 *DataSyncService var r1 bool if rf, ok := ret.Get(0).(func(string) (*DataSyncService, bool)); ok { @@ -252,10 +309,59 @@ func (_c *MockFlowgraphManager_GetFlowgraphService_Call) RunAndReturn(run func(s return _c } +// GetSegmentsJSON provides a mock function with given fields: +func (_m *MockFlowgraphManager) GetSegmentsJSON() string { + ret := _m.Called() + + if len(ret) == 0 { + panic("no return value specified for GetSegmentsJSON") + } + + var r0 string + if rf, ok := ret.Get(0).(func() string); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(string) + } + + return r0 +} + +// MockFlowgraphManager_GetSegmentsJSON_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetSegmentsJSON' +type MockFlowgraphManager_GetSegmentsJSON_Call struct { + *mock.Call +} + +// GetSegmentsJSON is a helper method to define mock.On call +func (_e *MockFlowgraphManager_Expecter) GetSegmentsJSON() *MockFlowgraphManager_GetSegmentsJSON_Call { + return &MockFlowgraphManager_GetSegmentsJSON_Call{Call: _e.mock.On("GetSegmentsJSON")} +} + +func (_c *MockFlowgraphManager_GetSegmentsJSON_Call) Run(run func()) *MockFlowgraphManager_GetSegmentsJSON_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockFlowgraphManager_GetSegmentsJSON_Call) Return(_a0 string) *MockFlowgraphManager_GetSegmentsJSON_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockFlowgraphManager_GetSegmentsJSON_Call) RunAndReturn(run func() string) *MockFlowgraphManager_GetSegmentsJSON_Call { + _c.Call.Return(run) + return _c +} + // HasFlowgraph provides a mock function with given fields: channel func (_m *MockFlowgraphManager) HasFlowgraph(channel string) bool { ret := _m.Called(channel) + if len(ret) == 0 { + panic("no return value specified for HasFlowgraph") + } + var r0 bool if rf, ok := ret.Get(0).(func(string) bool); ok { r0 = rf(channel) @@ -298,6 +404,10 @@ func (_c *MockFlowgraphManager_HasFlowgraph_Call) RunAndReturn(run func(string) func (_m *MockFlowgraphManager) HasFlowgraphWithOpID(channel string, opID int64) bool { ret := _m.Called(channel, opID) + if len(ret) == 0 { + panic("no return value specified for HasFlowgraphWithOpID") + } + var r0 bool if rf, ok := ret.Get(0).(func(string, int64) bool); ok { r0 = rf(channel, opID) diff --git a/internal/flushcommon/syncmgr/task.go b/internal/flushcommon/syncmgr/task.go index 86ae63df67a86..9ebcbaae38291 100644 --- a/internal/flushcommon/syncmgr/task.go +++ b/internal/flushcommon/syncmgr/task.go @@ -422,10 +422,11 @@ func (t *SyncTask) MarshalJSON() ([]byte, error) { SegmentID: t.segmentID, BatchRows: t.batchRows, SegmentLevel: t.level.String(), - TsFrom: t.tsFrom, - TsTo: t.tsTo, + TSFrom: t.tsFrom, + TSTo: t.tsTo, DeltaRowCount: t.deltaRowCount, FlushSize: t.flushedSize, - RunningTime: t.execTime, + RunningTime: t.execTime.String(), + NodeID: paramtable.GetNodeID(), }) } diff --git a/internal/flushcommon/syncmgr/task_test.go b/internal/flushcommon/syncmgr/task_test.go index e132cdb963b96..bb2c334e785d9 100644 --- a/internal/flushcommon/syncmgr/task_test.go +++ b/internal/flushcommon/syncmgr/task_test.go @@ -35,10 +35,12 @@ import ( "github.com/milvus-io/milvus/internal/flushcommon/broker" "github.com/milvus-io/milvus/internal/flushcommon/metacache" "github.com/milvus-io/milvus/internal/flushcommon/metacache/pkoracle" + "github.com/milvus-io/milvus/internal/json" "github.com/milvus-io/milvus/internal/mocks" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/common" + "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/retry" "github.com/milvus-io/milvus/pkg/util/tsoutil" @@ -383,7 +385,7 @@ func (s *SyncTaskSuite) TestNextID() { } func (s *SyncTaskSuite) TestSyncTask_MarshalJSON() { - task := &SyncTask{ + t := &SyncTask{ segmentID: 12345, batchRows: 100, level: datapb.SegmentLevel_L0, @@ -394,18 +396,22 @@ func (s *SyncTaskSuite) TestSyncTask_MarshalJSON() { execTime: 2 * time.Second, } - expectedJSON := `{ - "segment_id": 12345, - "batch_rows": 100, - "segment_level": "L0", - "ts_from": 1000, - "ts_to": 2000, - "delta_row_count": 10, - "flush_size": 1024, - "running_time": 2000000000 - }` - - data, err := task.MarshalJSON() + tm := &metricsinfo.SyncTask{ + SegmentID: t.segmentID, + BatchRows: t.batchRows, + SegmentLevel: t.level.String(), + TSFrom: t.tsFrom, + TSTo: t.tsTo, + DeltaRowCount: t.deltaRowCount, + FlushSize: t.flushedSize, + RunningTime: t.execTime.String(), + NodeID: paramtable.GetNodeID(), + } + expectedBytes, err := json.Marshal(tm) + s.NoError(err) + expectedJSON := string(expectedBytes) + + data, err := t.MarshalJSON() s.NoError(err) s.JSONEq(expectedJSON, string(data)) } diff --git a/internal/flushcommon/util/timetick_sender.go b/internal/flushcommon/util/timetick_sender.go index 25e889b5a2e4c..d18e4644338c0 100644 --- a/internal/flushcommon/util/timetick_sender.go +++ b/internal/flushcommon/util/timetick_sender.go @@ -36,6 +36,7 @@ import ( type StatsUpdater interface { Update(channel string, ts typeutil.Timestamp, stats []*commonpb.SegmentStats) + GetLatestTimestamp(channel string) typeutil.Timestamp } // TimeTickSender is to merge channel states updated by flow graph node and send to datacoord periodically @@ -126,6 +127,17 @@ func (m *TimeTickSender) Update(channelName string, timestamp uint64, segStats [ m.statsCache[channelName].lastTs = timestamp } +func (m *TimeTickSender) GetLatestTimestamp(channel string) typeutil.Timestamp { + m.mu.RLock() + defer m.mu.RUnlock() + chStats, ok := m.statsCache[channel] + if !ok { + log.Warn("channel not found in TimeTickSender", zap.String("channel", channel)) + return 0 + } + return chStats.lastTs +} + func (m *TimeTickSender) assembleDatanodeTtMsg() ([]*msgpb.DataNodeTtMsg, map[string]uint64) { m.mu.RLock() defer m.mu.RUnlock() diff --git a/internal/http/router.go b/internal/http/router.go index 4d6acec3f7adc..2859704f480d8 100644 --- a/internal/http/router.go +++ b/internal/http/router.go @@ -65,19 +65,46 @@ const ( // for WebUI restful api root path const ( - ClusterInfoPath = "/_cluster/info" - ClusterConfigsPath = "/_cluster/configs" - ClusterClientsPath = "/_cluster/clients" + // ClusterInfoPath is the path to get cluster information. + ClusterInfoPath = "/_cluster/info" + // ClusterConfigsPath is the path to get cluster configurations. + ClusterConfigsPath = "/_cluster/configs" + // ClusterClientsPath is the path to get connected clients. + ClusterClientsPath = "/_cluster/clients" + // ClusterDependenciesPath is the path to get cluster dependencies. ClusterDependenciesPath = "/_cluster/dependencies" - HookConfigsPath = "/_hook/configs" - QCoordSegmentsPath = "/_qcoord/segments" - QCoordChannelsPath = "/_qcoord/channels" - QCoordAllTasksPath = "/_qcoord/tasks/all" + // HookConfigsPath is the path to get hook configurations. + HookConfigsPath = "/_hook/configs" - DCoordAllTasksPath = "/_dcoord/tasks/all" - DCoordImportTasksPath = "/_dcoord/tasks/import" - DCoordCompactionTasksPath = "/_dcoord/tasks/compaction" - DCoordBuildIndexTasksPath = "/_dcoord/tasks/build_index" + // QCDistPath is the path to get QueryCoord distribution. + QCDistPath = "/_qc/dist" + // QCTargetPath is the path to get QueryCoord target. + QCTargetPath = "/_qc/target" + // QCReplicaPath is the path to get QueryCoord replica. + QCReplicaPath = "/_qc/replica" + // QCResourceGroupPath is the path to get QueryCoord resource group. + QCResourceGroupPath = "/_qc/resource_group" + // QCAllTasksPath is the path to get all tasks in QueryCoord. + QCAllTasksPath = "/_qc/tasks" - DNodeSyncTasksPath = "/_dnode/tasks/sync" + // QNSegmentsPath is the path to get segments in QueryNode. + QNSegmentsPath = "/_qn/segments" + // QNChannelsPath is the path to get channels in QueryNode. + QNChannelsPath = "/_qn/channels" + + // DCDistPath is the path to get all segments and channels distribution in DataCoord. + DCDistPath = "/_dc/dist" + // DCImportTasksPath is the path to get import tasks in DataCoord. + DCImportTasksPath = "/_dc/tasks/import" + // DCCompactionTasksPath is the path to get compaction tasks in DataCoord. + DCCompactionTasksPath = "/_dc/tasks/compaction" + // DCBuildIndexTasksPath is the path to get build index tasks in DataCoord. + DCBuildIndexTasksPath = "/_dc/tasks/build_index" + + // DNSyncTasksPath is the path to get sync tasks in DataNode. + DNSyncTasksPath = "/_dn/tasks/sync" + // DNSegmentsPath is the path to get segments in DataNode. + DNSegmentsPath = "/_dn/segments" + // DNChannelsPath is the path to get channels in DataNode. + DNChannelsPath = "/_dn/channels" ) diff --git a/internal/http/webui/header.html b/internal/http/webui/header.html index 4fda9e47362df..e87bd4472adfb 100644 --- a/internal/http/webui/header.html +++ b/internal/http/webui/header.html @@ -7,9 +7,6 @@ Home