2024-05-14 14:59:33 +08:00

220 lines
6.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 segments
import (
"context"
"fmt"
"sync"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/segcorepb"
"github.com/milvus-io/milvus/internal/querynodev2/segments/metricsutil"
"github.com/milvus-io/milvus/internal/util/streamrpc"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/timerecord"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
type RetrieveSegmentResult struct {
Result *segcorepb.RetrieveResults
Segment Segment
}
// retrieveOnSegments performs retrieve on listed segments
// all segment ids are validated before calling this function
func retrieveOnSegments(ctx context.Context, mgr *Manager, segments []Segment, segType SegmentType, plan *RetrievePlan, req *querypb.QueryRequest) ([]RetrieveSegmentResult, error) {
resultCh := make(chan RetrieveSegmentResult, len(segments))
// TODO(longjiquan): remove this limit after two-phase retrieval can be applied on lru-segment.
plan.ignoreNonPk = !paramtable.Get().QueryNodeCfg.UseStreamComputing.GetAsBool() &&
len(segments) > 1 && req.GetReq().GetLimit() != typeutil.Unlimited && plan.ShouldIgnoreNonPk()
label := metrics.SealedSegmentLabel
if segType == commonpb.SegmentState_Growing {
label = metrics.GrowingSegmentLabel
}
retriever := func(ctx context.Context, s Segment) error {
tr := timerecord.NewTimeRecorder("retrieveOnSegments")
result, err := s.Retrieve(ctx, plan)
if err != nil {
return err
}
resultCh <- RetrieveSegmentResult{
result,
s,
}
metrics.QueryNodeSQSegmentLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()),
metrics.QueryLabel, label).Observe(float64(tr.ElapseSpan().Milliseconds()))
return nil
}
errGroup, ctx := errgroup.WithContext(ctx)
for _, segment := range segments {
seg := segment
errGroup.Go(func() error {
if ctx.Err() != nil {
return ctx.Err()
}
// record search time and cache miss
var err error
accessRecord := metricsutil.NewQuerySegmentAccessRecord(getSegmentMetricLabel(seg))
defer func() {
accessRecord.Finish(err)
}()
if seg.IsLazyLoad() {
ctx, cancel := withLazyLoadTimeoutContext(ctx)
defer cancel()
var missing bool
missing, err = mgr.DiskCache.Do(ctx, seg.ID(), retriever)
if missing {
accessRecord.CacheMissing()
}
if err != nil {
log.Warn("failed to do query disk cache", zap.Int64("segID", seg.ID()), zap.Error(err))
}
return err
}
return retriever(ctx, seg)
})
}
err := errGroup.Wait()
close(resultCh)
if err != nil {
return nil, err
}
results := make([]RetrieveSegmentResult, 0, len(segments))
for r := range resultCh {
results = append(results, r)
}
return results, nil
}
func retrieveOnSegmentsWithStream(ctx context.Context, segments []Segment, segType SegmentType, plan *RetrievePlan, svr streamrpc.QueryStreamServer) error {
var (
errs = make([]error, len(segments))
wg sync.WaitGroup
)
label := metrics.SealedSegmentLabel
if segType == commonpb.SegmentState_Growing {
label = metrics.GrowingSegmentLabel
}
for i, segment := range segments {
wg.Add(1)
go func(segment Segment, i int) {
defer wg.Done()
tr := timerecord.NewTimeRecorder("retrieveOnSegmentsWithStream")
result, err := segment.Retrieve(ctx, plan)
if err != nil {
errs[i] = err
return
}
if len(result.GetOffset()) != 0 {
if err = svr.Send(&internalpb.RetrieveResults{
Status: merr.Success(),
Ids: result.GetIds(),
FieldsData: result.GetFieldsData(),
CostAggregation: &internalpb.CostAggregation{
TotalRelatedDataSize: GetSegmentRelatedDataSize(segment),
},
AllRetrieveCount: result.GetAllRetrieveCount(),
}); err != nil {
errs[i] = err
}
}
errs[i] = nil
metrics.QueryNodeSQSegmentLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()),
metrics.QueryLabel, label).Observe(float64(tr.ElapseSpan().Milliseconds()))
}(segment, i)
}
wg.Wait()
return merr.Combine(errs...)
}
// retrieve will retrieve all the validate target segments
func Retrieve(ctx context.Context, manager *Manager, plan *RetrievePlan, req *querypb.QueryRequest) ([]RetrieveSegmentResult, []Segment, error) {
if ctx.Err() != nil {
return nil, nil, ctx.Err()
}
var err error
var SegType commonpb.SegmentState
var retrieveSegments []Segment
segIDs := req.GetSegmentIDs()
collID := req.Req.GetCollectionID()
log.Debug("retrieve on segments", zap.Int64s("segmentIDs", segIDs), zap.Int64("collectionID", collID))
if req.GetScope() == querypb.DataScope_Historical {
SegType = SegmentTypeSealed
retrieveSegments, err = validateOnHistorical(ctx, manager, collID, nil, segIDs)
} else {
SegType = SegmentTypeGrowing
retrieveSegments, err = validateOnStream(ctx, manager, collID, nil, segIDs)
}
if err != nil {
return nil, retrieveSegments, err
}
result, err := retrieveOnSegments(ctx, manager, retrieveSegments, SegType, plan, req)
return result, retrieveSegments, err
}
// retrieveStreaming will retrieve all the validate target segments and return by stream
func RetrieveStream(ctx context.Context, manager *Manager, plan *RetrievePlan, req *querypb.QueryRequest, srv streamrpc.QueryStreamServer) ([]Segment, error) {
var err error
var SegType commonpb.SegmentState
var retrieveSegments []Segment
segIDs := req.GetSegmentIDs()
collID := req.Req.GetCollectionID()
if req.GetScope() == querypb.DataScope_Historical {
SegType = SegmentTypeSealed
retrieveSegments, err = validateOnHistorical(ctx, manager, collID, nil, segIDs)
} else {
SegType = SegmentTypeGrowing
retrieveSegments, err = validateOnStream(ctx, manager, collID, nil, segIDs)
}
if err != nil {
return retrieveSegments, err
}
err = retrieveOnSegmentsWithStream(ctx, retrieveSegments, SegType, plan, srv)
return retrieveSegments, err
}