// 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 querynode import ( "context" "errors" "fmt" "path/filepath" "strconv" "sync" "github.com/golang/protobuf/proto" etcdkv "github.com/milvus-io/milvus/internal/kv/etcd" "go.etcd.io/etcd/api/v3/mvccpb" "go.uber.org/zap" "github.com/milvus-io/milvus/internal/log" "github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/segcorepb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/internal/util" ) // historical is in charge of historical data in query node type historical struct { ctx context.Context replica ReplicaInterface tSafeReplica TSafeReplicaInterface mu sync.Mutex // guards globalSealedSegments globalSealedSegments map[UniqueID]*querypb.SegmentInfo etcdKV *etcdkv.EtcdKV } // newHistorical returns a new historical func newHistorical(ctx context.Context, replica ReplicaInterface, etcdKV *etcdkv.EtcdKV, tSafeReplica TSafeReplicaInterface) *historical { return &historical{ ctx: ctx, replica: replica, globalSealedSegments: make(map[UniqueID]*querypb.SegmentInfo), etcdKV: etcdKV, tSafeReplica: tSafeReplica, } } func (h *historical) start() { go h.watchGlobalSegmentMeta() } // close would release all resources in historical func (h *historical) close() { // free collectionReplica h.replica.freeAll() } func (h *historical) watchGlobalSegmentMeta() { log.Debug("query node watchGlobalSegmentMeta start") watchChan := h.etcdKV.WatchWithPrefix(util.SegmentMetaPrefix) for { select { case <-h.ctx.Done(): log.Debug("query node watchGlobalSegmentMeta close") return case resp := <-watchChan: for _, event := range resp.Events { segmentID, err := strconv.ParseInt(filepath.Base(string(event.Kv.Key)), 10, 64) if err != nil { log.Warn("watchGlobalSegmentMeta failed", zap.Any("error", err.Error())) continue } switch event.Type { case mvccpb.PUT: log.Debug("globalSealedSegments add segment", zap.Any("segmentID", segmentID), ) segmentInfo := &querypb.SegmentInfo{} err = proto.Unmarshal(event.Kv.Value, segmentInfo) if err != nil { log.Warn("watchGlobalSegmentMeta failed", zap.Any("error", err.Error())) continue } h.addGlobalSegmentInfo(segmentID, segmentInfo) case mvccpb.DELETE: log.Debug("globalSealedSegments delete segment", zap.Any("segmentID", segmentID), ) h.removeGlobalSegmentInfo(segmentID) } } } } } func (h *historical) addGlobalSegmentInfo(segmentID UniqueID, segmentInfo *querypb.SegmentInfo) { h.mu.Lock() defer h.mu.Unlock() h.globalSealedSegments[segmentID] = segmentInfo } func (h *historical) removeGlobalSegmentInfo(segmentID UniqueID) { h.mu.Lock() defer h.mu.Unlock() delete(h.globalSealedSegments, segmentID) } func (h *historical) getGlobalSegmentIDsByCollectionID(collectionID UniqueID) []UniqueID { h.mu.Lock() defer h.mu.Unlock() resIDs := make([]UniqueID, 0) for _, v := range h.globalSealedSegments { if v.CollectionID == collectionID { resIDs = append(resIDs, v.SegmentID) } } return resIDs } func (h *historical) getGlobalSegmentIDsByPartitionIds(partitionIDs []UniqueID) []UniqueID { h.mu.Lock() defer h.mu.Unlock() resIDs := make([]UniqueID, 0) for _, v := range h.globalSealedSegments { for _, partitionID := range partitionIDs { if v.PartitionID == partitionID { resIDs = append(resIDs, v.SegmentID) } } } return resIDs } func (h *historical) removeGlobalSegmentIDsByCollectionID(collectionID UniqueID) { h.mu.Lock() defer h.mu.Unlock() for _, v := range h.globalSealedSegments { if v.CollectionID == collectionID { delete(h.globalSealedSegments, v.SegmentID) } } } func (h *historical) removeGlobalSegmentIDsByPartitionIds(partitionIDs []UniqueID) { h.mu.Lock() defer h.mu.Unlock() for _, v := range h.globalSealedSegments { for _, partitionID := range partitionIDs { if v.PartitionID == partitionID { delete(h.globalSealedSegments, v.SegmentID) } } } } func (h *historical) retrieve(collID UniqueID, partIDs []UniqueID, vcm storage.ChunkManager, plan *RetrievePlan) ([]*segcorepb.RetrieveResults, []UniqueID, []UniqueID, error) { retrieveResults := make([]*segcorepb.RetrieveResults, 0) retrieveSegmentIDs := make([]UniqueID, 0) // get historical partition ids var retrievePartIDs []UniqueID if len(partIDs) == 0 { hisPartIDs, err := h.replica.getPartitionIDs(collID) if err != nil { return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err } retrievePartIDs = hisPartIDs } else { for _, id := range partIDs { _, err := h.replica.getPartitionByID(id) if err == nil { retrievePartIDs = append(retrievePartIDs, id) } } } for _, partID := range retrievePartIDs { segIDs, err := h.replica.getSegmentIDs(partID) if err != nil { return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err } for _, segID := range segIDs { seg, err := h.replica.getSegmentByID(segID) if err != nil { return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err } result, err := seg.retrieve(plan) if err != nil { return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err } if err = seg.fillVectorFieldsData(collID, vcm, result); err != nil { return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err } retrieveResults = append(retrieveResults, result) retrieveSegmentIDs = append(retrieveSegmentIDs, segID) } } return retrieveResults, retrieveSegmentIDs, retrievePartIDs, nil } // search will search all the target segments in historical func (h *historical) search(searchReqs []*searchRequest, collID UniqueID, partIDs []UniqueID, plan *SearchPlan, searchTs Timestamp) ([]*SearchResult, []UniqueID, []UniqueID, error) { searchResults := make([]*SearchResult, 0) searchSegmentIDs := make([]UniqueID, 0) // get historical partition ids var searchPartIDs []UniqueID if len(partIDs) == 0 { hisPartIDs, err := h.replica.getPartitionIDs(collID) if err != nil { return searchResults, searchSegmentIDs, searchPartIDs, err } log.Debug("no partition specified, search all partitions", zap.Any("collectionID", collID), zap.Any("all partitions", hisPartIDs), ) searchPartIDs = hisPartIDs } else { for _, id := range partIDs { _, err := h.replica.getPartitionByID(id) if err == nil { log.Debug("append search partition id", zap.Any("collectionID", collID), zap.Any("partitionID", id), ) searchPartIDs = append(searchPartIDs, id) } } } col, err := h.replica.getCollectionByID(collID) if err != nil { return searchResults, searchSegmentIDs, searchPartIDs, err } // all partitions have been released if len(searchPartIDs) == 0 && col.getLoadType() == loadTypePartition { return searchResults, searchSegmentIDs, searchPartIDs, errors.New("partitions have been released , collectionID = " + fmt.Sprintln(collID) + "target partitionIDs = " + fmt.Sprintln(partIDs)) } if len(searchPartIDs) == 0 && col.getLoadType() == loadTypeCollection { if err = col.checkReleasedPartitions(partIDs); err != nil { return searchResults, searchSegmentIDs, searchPartIDs, err } return searchResults, searchSegmentIDs, searchPartIDs, nil } var segmentLock sync.RWMutex for _, partID := range searchPartIDs { segIDs, err := h.replica.getSegmentIDs(partID) if err != nil { return searchResults, searchSegmentIDs, searchPartIDs, err } var err2 error var wg sync.WaitGroup for _, segID := range segIDs { segID2 := segID wg.Add(1) go func() { defer wg.Done() seg, err := h.replica.getSegmentByID(segID2) if err != nil { err2 = err return } if !seg.getOnService() { return } searchResult, err := seg.search(plan, searchReqs, []Timestamp{searchTs}) if err != nil { err2 = err return } segmentLock.Lock() searchResults = append(searchResults, searchResult) searchSegmentIDs = append(searchSegmentIDs, seg.segmentID) segmentLock.Unlock() }() } wg.Wait() if err2 != nil { return searchResults, searchSegmentIDs, searchPartIDs, err2 } } return searchResults, searchSegmentIDs, searchPartIDs, nil }