mirror of
https://gitee.com/milvus-io/milvus.git
synced 2024-12-01 11:29:48 +08:00
97757405de
Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com>
276 lines
8.8 KiB
Go
276 lines
8.8 KiB
Go
// 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"
|
|
"sync"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
"github.com/milvus-io/milvus/internal/metrics"
|
|
"github.com/milvus-io/milvus/internal/proto/segcorepb"
|
|
"github.com/milvus-io/milvus/internal/storage"
|
|
"github.com/milvus-io/milvus/internal/util/timerecord"
|
|
)
|
|
|
|
// historical is in charge of historical data in query node
|
|
type historical struct {
|
|
ctx context.Context
|
|
|
|
replica ReplicaInterface
|
|
tSafeReplica TSafeReplicaInterface
|
|
}
|
|
|
|
// newHistorical returns a new historical
|
|
func newHistorical(ctx context.Context,
|
|
replica ReplicaInterface,
|
|
tSafeReplica TSafeReplicaInterface) *historical {
|
|
|
|
return &historical{
|
|
ctx: ctx,
|
|
replica: replica,
|
|
tSafeReplica: tSafeReplica,
|
|
}
|
|
}
|
|
|
|
// close would release all resources in historical
|
|
func (h *historical) close() {
|
|
// free collectionReplica
|
|
h.replica.freeAll()
|
|
}
|
|
|
|
// // retrieve will retrieve from the segments in historical
|
|
func (h *historical) retrieve(collID UniqueID, partIDs []UniqueID, vcm storage.ChunkManager,
|
|
plan *RetrievePlan) (retrieveResults []*segcorepb.RetrieveResults, retrieveSegmentIDs []UniqueID, retrievePartIDs []UniqueID, err error) {
|
|
|
|
// get historical partition ids
|
|
retrievePartIDs, err = h.getTargetPartIDs(collID, partIDs)
|
|
if err != nil {
|
|
return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err
|
|
}
|
|
|
|
log.Debug("retrieve target partitions", zap.Int64("collectionID", collID), zap.Int64s("partitionIDs", retrievePartIDs))
|
|
|
|
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.fillIndexedFieldsData(collID, vcm, result); err != nil {
|
|
return retrieveResults, retrieveSegmentIDs, retrievePartIDs, err
|
|
}
|
|
retrieveResults = append(retrieveResults, result)
|
|
retrieveSegmentIDs = append(retrieveSegmentIDs, segID)
|
|
}
|
|
}
|
|
|
|
return retrieveResults, retrieveSegmentIDs, retrievePartIDs, nil
|
|
}
|
|
|
|
// retrieveBySegmentIDs retrieves records from segments specified by their IDs
|
|
func (h *historical) retrieveBySegmentIDs(collID UniqueID, segmentIDs []UniqueID, vcm storage.ChunkManager, plan *RetrievePlan) (
|
|
retrieveResults []*segcorepb.RetrieveResults, err error) {
|
|
|
|
for _, segID := range segmentIDs {
|
|
seg, err := h.replica.getSegmentByID(segID)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
result, err := seg.retrieve(plan)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
err = seg.fillIndexedFieldsData(collID, vcm, result)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
retrieveResults = append(retrieveResults, result)
|
|
}
|
|
|
|
return retrieveResults, nil
|
|
}
|
|
|
|
// search will search all the target segments in historical
|
|
func (h *historical) search(searchReqs []*searchRequest, collID UniqueID, partIDs []UniqueID, plan *SearchPlan,
|
|
searchTs Timestamp) (searchResults []*SearchResult, searchSegmentIDs []UniqueID, searchPartIDs []UniqueID, err error) {
|
|
|
|
searchPartIDs, err = h.getTargetPartIDs(collID, partIDs)
|
|
if err != nil {
|
|
return searchResults, searchSegmentIDs, searchPartIDs, err
|
|
}
|
|
|
|
log.Debug("search target partitions", zap.Int64("collectionID", collID), zap.Int64s("partitionIDs", searchPartIDs))
|
|
|
|
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 nil, nil, searchPartIDs, errors.New("partitions have been released , collectionID = " +
|
|
fmt.Sprintln(collID) + "target partitionIDs = " + fmt.Sprintln(partIDs))
|
|
}
|
|
|
|
if len(searchPartIDs) == 0 && col.getLoadType() == loadTypeCollection {
|
|
return searchResults, searchSegmentIDs, searchPartIDs, nil
|
|
}
|
|
|
|
var segmentIDs []UniqueID
|
|
for _, partID := range searchPartIDs {
|
|
segIDs, err := h.replica.getSegmentIDs(partID)
|
|
if err != nil {
|
|
return searchResults, searchSegmentIDs, searchPartIDs, err
|
|
}
|
|
|
|
segmentIDs = append(segmentIDs, segIDs...)
|
|
}
|
|
|
|
searchResults, searchSegmentIDs, err = h.searchSegments(segmentIDs, searchReqs, plan, searchTs)
|
|
|
|
return searchResults, searchSegmentIDs, searchPartIDs, err
|
|
}
|
|
|
|
// validateSegmentIDs checks segments if their collection/partition(s) have been released
|
|
func (h *historical) validateSegmentIDs(segmentIDs []UniqueID, collectionID UniqueID, partitionIDs []UniqueID) (err error) {
|
|
// validate partitionIDs
|
|
validatedPartitionIDs, err := h.getTargetPartIDs(collectionID, partitionIDs)
|
|
if err != nil {
|
|
return
|
|
}
|
|
log.Debug("search validated partitions", zap.Int64("collectionID", collectionID), zap.Int64s("partitionIDs", validatedPartitionIDs))
|
|
|
|
col, err := h.replica.getCollectionByID(collectionID)
|
|
if err != nil {
|
|
return
|
|
}
|
|
|
|
// return if no partitions are loaded currently
|
|
if len(validatedPartitionIDs) == 0 {
|
|
switch col.getLoadType() {
|
|
case loadTypeCollection:
|
|
err = fmt.Errorf("partitions have been released, collectionID = %d, target paritition= %v", collectionID, partitionIDs)
|
|
case loadTypePartition:
|
|
err = fmt.Errorf("collection has been released, collectionID = %d, target paritition= %v", collectionID, partitionIDs)
|
|
default:
|
|
err = fmt.Errorf("got unknown loadType %d", col.getLoadType())
|
|
}
|
|
return
|
|
}
|
|
|
|
for _, segmentID := range segmentIDs {
|
|
var segment *Segment
|
|
if segment, err = h.replica.getSegmentByID(segmentID); err != nil {
|
|
return
|
|
}
|
|
if !inList(validatedPartitionIDs, segment.partitionID) {
|
|
err = fmt.Errorf("segment %d belongs to partition %d, which is not in %v", segmentID, segment.partitionID, validatedPartitionIDs)
|
|
return
|
|
}
|
|
}
|
|
return
|
|
}
|
|
|
|
// getSearchPartIDs fetches the partition ids to search from the request ids
|
|
func (h *historical) getTargetPartIDs(collID UniqueID, partIDs []UniqueID) ([]UniqueID, error) {
|
|
// no partition id specified, get all partition ids in collection
|
|
if len(partIDs) == 0 {
|
|
return h.replica.getPartitionIDs(collID)
|
|
}
|
|
|
|
var targetPartIDs []UniqueID
|
|
for _, id := range partIDs {
|
|
_, err := h.replica.getPartitionByID(id)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
targetPartIDs = append(targetPartIDs, id)
|
|
}
|
|
return targetPartIDs, nil
|
|
}
|
|
|
|
// searchSegments performs search on listed segments
|
|
// all segment ids are validated before calling this function
|
|
func (h *historical) searchSegments(segIDs []UniqueID, searchReqs []*searchRequest, plan *SearchPlan, searchTs Timestamp) ([]*SearchResult, []UniqueID, error) {
|
|
// pre-fetch all the segment
|
|
// if error found, return before executing segment search
|
|
segments := make([]*Segment, 0, len(segIDs))
|
|
for _, segID := range segIDs {
|
|
seg, err := h.replica.getSegmentByID(segID)
|
|
if err != nil {
|
|
return nil, nil, err
|
|
}
|
|
segments = append(segments, seg)
|
|
}
|
|
|
|
// results variables
|
|
var searchResults []*SearchResult
|
|
var searchSegmentIDs []UniqueID
|
|
var lock sync.Mutex
|
|
var serr error
|
|
|
|
// calling segment search in goroutines
|
|
var wg sync.WaitGroup
|
|
for _, seg := range segments {
|
|
wg.Add(1)
|
|
go func(seg *Segment) {
|
|
defer wg.Done()
|
|
if !seg.getOnService() {
|
|
log.Warn("segment no on service", zap.Int64("segmentID", seg.segmentID))
|
|
return
|
|
}
|
|
// record search time
|
|
tr := timerecord.NewTimeRecorder("searchOnSealed")
|
|
searchResult, err := seg.search(plan, searchReqs, []Timestamp{searchTs})
|
|
|
|
// update metrics
|
|
metrics.QueryNodeSQSegmentLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
|
|
metrics.SearchLabel,
|
|
metrics.SealedSegmentLabel).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
|
|
|
// write back result into list
|
|
lock.Lock()
|
|
defer lock.Unlock()
|
|
if err != nil {
|
|
serr = err
|
|
return
|
|
}
|
|
searchResults = append(searchResults, searchResult)
|
|
searchSegmentIDs = append(searchSegmentIDs, seg.segmentID)
|
|
}(seg)
|
|
}
|
|
wg.Wait()
|
|
if serr != nil {
|
|
return nil, nil, serr
|
|
}
|
|
return searchResults, searchSegmentIDs, nil
|
|
}
|