2023-03-27 00:42:00 +08:00
|
|
|
package tasks
|
|
|
|
|
2023-07-03 18:24:25 +08:00
|
|
|
// TODO: rename this file into search_task.go
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
import (
|
|
|
|
"bytes"
|
|
|
|
"context"
|
|
|
|
"fmt"
|
2023-07-03 18:24:25 +08:00
|
|
|
"strconv"
|
2023-03-27 00:42:00 +08:00
|
|
|
|
2023-04-14 18:18:29 +08:00
|
|
|
"github.com/golang/protobuf/proto"
|
2023-04-06 19:14:32 +08:00
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2023-06-09 01:28:37 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
2023-03-27 00:42:00 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
2023-06-08 17:24:36 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/collector"
|
2023-03-27 00:42:00 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/segments"
|
|
|
|
"github.com/milvus-io/milvus/internal/util"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/log"
|
|
|
|
"github.com/milvus-io/milvus/pkg/metrics"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
2023-06-08 17:24:36 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
2023-04-06 19:14:32 +08:00
|
|
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
2023-03-27 00:42:00 +08:00
|
|
|
)
|
|
|
|
|
2023-07-03 18:24:25 +08:00
|
|
|
var (
|
|
|
|
_ Task = &SearchTask{}
|
|
|
|
_ MergeTask = &SearchTask{}
|
|
|
|
)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
type SearchTask struct {
|
2023-04-14 18:18:29 +08:00
|
|
|
ctx context.Context
|
|
|
|
collection *segments.Collection
|
|
|
|
segmentManager *segments.Manager
|
|
|
|
req *querypb.SearchRequest
|
|
|
|
result *internalpb.SearchResults
|
2023-06-16 16:02:39 +08:00
|
|
|
merged bool
|
|
|
|
groupSize int64
|
2023-04-14 18:18:29 +08:00
|
|
|
topk int64
|
|
|
|
nq int64
|
|
|
|
placeholderGroup []byte
|
|
|
|
originTopks []int64
|
|
|
|
originNqs []int64
|
|
|
|
others []*SearchTask
|
|
|
|
notifier chan error
|
2023-04-03 15:24:24 +08:00
|
|
|
|
|
|
|
tr *timerecord.TimeRecorder
|
2023-03-27 00:42:00 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func NewSearchTask(ctx context.Context,
|
|
|
|
collection *segments.Collection,
|
|
|
|
manager *segments.Manager,
|
|
|
|
req *querypb.SearchRequest,
|
|
|
|
) *SearchTask {
|
|
|
|
return &SearchTask{
|
2023-04-14 18:18:29 +08:00
|
|
|
ctx: ctx,
|
|
|
|
collection: collection,
|
|
|
|
segmentManager: manager,
|
|
|
|
req: req,
|
2023-06-16 16:02:39 +08:00
|
|
|
merged: false,
|
|
|
|
groupSize: 1,
|
2023-04-14 18:18:29 +08:00
|
|
|
topk: req.GetReq().GetTopk(),
|
|
|
|
nq: req.GetReq().GetNq(),
|
|
|
|
placeholderGroup: req.GetReq().GetPlaceholderGroup(),
|
|
|
|
originTopks: []int64{req.GetReq().GetTopk()},
|
|
|
|
originNqs: []int64{req.GetReq().GetNq()},
|
|
|
|
notifier: make(chan error, 1),
|
2023-06-15 14:24:38 +08:00
|
|
|
tr: timerecord.NewTimeRecorderWithTrace(ctx, "searchTask"),
|
2023-03-27 00:42:00 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2023-07-03 18:24:25 +08:00
|
|
|
// Return the username which task is belong to.
|
|
|
|
// Return "" if the task do not contain any user info.
|
|
|
|
func (t *SearchTask) Username() string {
|
|
|
|
return t.req.Req.GetUsername()
|
|
|
|
}
|
|
|
|
|
2023-06-08 17:24:36 +08:00
|
|
|
func (t *SearchTask) PreExecute() error {
|
2023-07-03 18:24:25 +08:00
|
|
|
// Update task wait time metric before execute
|
|
|
|
nodeID := strconv.FormatInt(paramtable.GetNodeID(), 10)
|
|
|
|
inQueueDuration := t.tr.ElapseSpan()
|
|
|
|
|
|
|
|
// Update in queue metric for prometheus.
|
|
|
|
metrics.QueryNodeSQLatencyInQueue.WithLabelValues(
|
|
|
|
nodeID,
|
|
|
|
metrics.SearchLabel).
|
|
|
|
Observe(float64(inQueueDuration.Milliseconds()))
|
|
|
|
|
|
|
|
username := t.Username()
|
|
|
|
metrics.QueryNodeSQPerUserLatencyInQueue.WithLabelValues(
|
|
|
|
nodeID,
|
|
|
|
metrics.SearchLabel,
|
|
|
|
username).
|
|
|
|
Observe(float64(inQueueDuration.Milliseconds()))
|
|
|
|
|
|
|
|
// Update collector for query node quota.
|
|
|
|
collector.Average.Add(metricsinfo.SearchQueueMetric, float64(inQueueDuration.Microseconds()))
|
|
|
|
|
|
|
|
// Execute merged task's PreExecute.
|
2023-06-08 17:24:36 +08:00
|
|
|
for _, subTask := range t.others {
|
|
|
|
err := subTask.PreExecute()
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
func (t *SearchTask) Execute() error {
|
|
|
|
log := log.Ctx(t.ctx).With(
|
|
|
|
zap.Int64("collectionID", t.collection.ID()),
|
|
|
|
zap.String("shard", t.req.GetDmlChannels()[0]),
|
|
|
|
)
|
2023-04-14 18:18:29 +08:00
|
|
|
|
2023-06-15 14:24:38 +08:00
|
|
|
executeRecord := timerecord.NewTimeRecorderWithTrace(t.ctx, "searchTaskExecute")
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
req := t.req
|
2023-04-14 18:18:29 +08:00
|
|
|
t.combinePlaceHolderGroups()
|
|
|
|
searchReq, err := segments.NewSearchRequest(t.collection, req, t.placeholderGroup)
|
2023-03-27 00:42:00 +08:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
defer searchReq.Delete()
|
|
|
|
|
|
|
|
var results []*segments.SearchResult
|
|
|
|
if req.GetScope() == querypb.DataScope_Historical {
|
|
|
|
results, _, _, err = segments.SearchHistorical(
|
|
|
|
t.ctx,
|
|
|
|
t.segmentManager,
|
|
|
|
searchReq,
|
|
|
|
req.GetReq().GetCollectionID(),
|
|
|
|
nil,
|
|
|
|
req.GetSegmentIDs(),
|
|
|
|
)
|
|
|
|
} else if req.GetScope() == querypb.DataScope_Streaming {
|
|
|
|
results, _, _, err = segments.SearchStreaming(
|
|
|
|
t.ctx,
|
|
|
|
t.segmentManager,
|
|
|
|
searchReq,
|
|
|
|
req.GetReq().GetCollectionID(),
|
|
|
|
nil,
|
|
|
|
req.GetSegmentIDs(),
|
|
|
|
)
|
|
|
|
}
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
defer segments.DeleteSearchResults(results)
|
|
|
|
|
|
|
|
if len(results) == 0 {
|
2023-04-14 18:18:29 +08:00
|
|
|
for i := range t.originNqs {
|
|
|
|
var task *SearchTask
|
|
|
|
if i == 0 {
|
|
|
|
task = t
|
|
|
|
} else {
|
|
|
|
task = t.others[i-1]
|
|
|
|
}
|
|
|
|
|
|
|
|
task.result = &internalpb.SearchResults{
|
|
|
|
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success},
|
|
|
|
MetricType: req.GetReq().GetMetricType(),
|
|
|
|
NumQueries: t.originNqs[i],
|
|
|
|
TopK: t.originTopks[i],
|
|
|
|
SlicedOffset: 1,
|
|
|
|
SlicedNumCount: 1,
|
2023-06-15 14:24:38 +08:00
|
|
|
CostAggregation: &internalpb.CostAggregation{
|
|
|
|
ServiceTime: executeRecord.ElapseSpan().Milliseconds(),
|
|
|
|
},
|
2023-04-14 18:18:29 +08:00
|
|
|
}
|
2023-03-27 00:42:00 +08:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2023-06-15 14:24:38 +08:00
|
|
|
reduceRecord := timerecord.NewTimeRecorderWithTrace(t.ctx, "searchTaskReduce")
|
2023-03-27 00:42:00 +08:00
|
|
|
blobs, err := segments.ReduceSearchResultsAndFillData(
|
|
|
|
searchReq.Plan(),
|
|
|
|
results,
|
|
|
|
int64(len(results)),
|
2023-04-14 18:18:29 +08:00
|
|
|
t.originNqs,
|
|
|
|
t.originTopks,
|
2023-03-27 00:42:00 +08:00
|
|
|
)
|
|
|
|
if err != nil {
|
|
|
|
log.Warn("failed to reduce search results", zap.Error(err))
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
defer segments.DeleteSearchResultDataBlobs(blobs)
|
|
|
|
|
2023-04-14 18:18:29 +08:00
|
|
|
for i := range t.originNqs {
|
|
|
|
blob, err := segments.GetSearchResultDataBlob(blobs, i)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
var task *SearchTask
|
|
|
|
if i == 0 {
|
|
|
|
task = t
|
|
|
|
} else {
|
|
|
|
task = t.others[i-1]
|
|
|
|
}
|
|
|
|
|
|
|
|
// Note: blob is unsafe because get from C
|
|
|
|
bs := make([]byte, len(blob))
|
|
|
|
copy(bs, blob)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
2023-04-14 18:18:29 +08:00
|
|
|
metrics.QueryNodeReduceLatency.WithLabelValues(
|
|
|
|
fmt.Sprint(paramtable.GetNodeID()),
|
2023-07-11 18:18:28 +08:00
|
|
|
metrics.SearchLabel,
|
|
|
|
metrics.ReduceSegments).
|
2023-06-15 14:24:38 +08:00
|
|
|
Observe(float64(reduceRecord.ElapseSpan().Milliseconds()))
|
2023-04-14 18:18:29 +08:00
|
|
|
|
|
|
|
task.result = &internalpb.SearchResults{
|
|
|
|
Status: util.WrapStatus(commonpb.ErrorCode_Success, ""),
|
|
|
|
MetricType: req.GetReq().GetMetricType(),
|
|
|
|
NumQueries: t.originNqs[i],
|
|
|
|
TopK: t.originTopks[i],
|
|
|
|
SlicedBlob: bs,
|
|
|
|
SlicedOffset: 1,
|
|
|
|
SlicedNumCount: 1,
|
2023-06-15 14:24:38 +08:00
|
|
|
CostAggregation: &internalpb.CostAggregation{
|
|
|
|
ServiceTime: executeRecord.ElapseSpan().Milliseconds(),
|
|
|
|
},
|
2023-04-14 18:18:29 +08:00
|
|
|
}
|
2023-03-27 00:42:00 +08:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (t *SearchTask) Merge(other *SearchTask) bool {
|
|
|
|
var (
|
2023-04-14 18:18:29 +08:00
|
|
|
nq = t.nq
|
|
|
|
topk = t.topk
|
2023-06-16 16:02:39 +08:00
|
|
|
otherNq = other.nq
|
|
|
|
otherTopk = other.topk
|
2023-03-27 00:42:00 +08:00
|
|
|
)
|
|
|
|
|
2023-04-27 18:26:35 +08:00
|
|
|
diffTopk := topk != otherTopk
|
2023-03-27 00:42:00 +08:00
|
|
|
pre := funcutil.Min(nq*topk, otherNq*otherTopk)
|
|
|
|
maxTopk := funcutil.Max(topk, otherTopk)
|
|
|
|
after := (nq + otherNq) * maxTopk
|
|
|
|
ratio := float64(after) / float64(pre)
|
|
|
|
|
|
|
|
// Check mergeable
|
|
|
|
if t.req.GetReq().GetDbID() != other.req.GetReq().GetDbID() ||
|
|
|
|
t.req.GetReq().GetCollectionID() != other.req.GetReq().GetCollectionID() ||
|
|
|
|
t.req.GetReq().GetTravelTimestamp() != other.req.GetReq().GetTravelTimestamp() ||
|
|
|
|
t.req.GetReq().GetDslType() != other.req.GetReq().GetDslType() ||
|
|
|
|
t.req.GetDmlChannels()[0] != other.req.GetDmlChannels()[0] ||
|
|
|
|
nq+otherNq > paramtable.Get().QueryNodeCfg.MaxGroupNQ.GetAsInt64() ||
|
2023-04-27 18:26:35 +08:00
|
|
|
diffTopk && ratio > paramtable.Get().QueryNodeCfg.TopKMergeRatio.GetAsFloat() ||
|
2023-03-27 00:42:00 +08:00
|
|
|
!funcutil.SliceSetEqual(t.req.GetReq().GetPartitionIDs(), other.req.GetReq().GetPartitionIDs()) ||
|
|
|
|
!funcutil.SliceSetEqual(t.req.GetSegmentIDs(), other.req.GetSegmentIDs()) ||
|
|
|
|
!bytes.Equal(t.req.GetReq().GetSerializedExprPlan(), other.req.GetReq().GetSerializedExprPlan()) {
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
|
|
|
// Merge
|
2023-06-16 16:02:39 +08:00
|
|
|
t.groupSize += other.groupSize
|
2023-04-14 18:18:29 +08:00
|
|
|
t.topk = maxTopk
|
|
|
|
t.nq += otherNq
|
2023-03-27 00:42:00 +08:00
|
|
|
t.originTopks = append(t.originTopks, other.originTopks...)
|
|
|
|
t.originNqs = append(t.originNqs, other.originNqs...)
|
|
|
|
t.others = append(t.others, other)
|
2023-06-16 16:02:39 +08:00
|
|
|
other.merged = true
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
|
|
|
|
func (t *SearchTask) Done(err error) {
|
2023-06-16 16:02:39 +08:00
|
|
|
if !t.merged {
|
|
|
|
metrics.QueryNodeSearchGroupSize.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(t.groupSize))
|
2023-06-15 19:50:39 +08:00
|
|
|
metrics.QueryNodeSearchGroupNQ.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(t.nq))
|
|
|
|
metrics.QueryNodeSearchGroupTopK.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(t.topk))
|
2023-03-27 00:42:00 +08:00
|
|
|
}
|
2023-07-03 18:24:25 +08:00
|
|
|
t.notifier <- err
|
2023-03-27 00:42:00 +08:00
|
|
|
for _, other := range t.others {
|
|
|
|
other.Done(err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (t *SearchTask) Canceled() error {
|
|
|
|
return t.ctx.Err()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (t *SearchTask) Wait() error {
|
|
|
|
return <-t.notifier
|
|
|
|
}
|
|
|
|
|
|
|
|
func (t *SearchTask) Result() *internalpb.SearchResults {
|
|
|
|
return t.result
|
|
|
|
}
|
|
|
|
|
2023-07-03 18:24:25 +08:00
|
|
|
func (t *SearchTask) NQ() int64 {
|
|
|
|
return t.nq
|
|
|
|
}
|
|
|
|
|
|
|
|
func (t *SearchTask) MergeWith(other Task) bool {
|
|
|
|
switch other := other.(type) {
|
|
|
|
case *SearchTask:
|
|
|
|
return t.Merge(other)
|
|
|
|
}
|
|
|
|
return false
|
|
|
|
}
|
|
|
|
|
2023-04-14 18:18:29 +08:00
|
|
|
// combinePlaceHolderGroups combine all the placeholder groups.
|
|
|
|
func (t *SearchTask) combinePlaceHolderGroups() {
|
|
|
|
if len(t.others) > 0 {
|
|
|
|
ret := &commonpb.PlaceholderGroup{}
|
|
|
|
_ = proto.Unmarshal(t.placeholderGroup, ret)
|
|
|
|
for _, t := range t.others {
|
|
|
|
x := &commonpb.PlaceholderGroup{}
|
|
|
|
_ = proto.Unmarshal(t.placeholderGroup, x)
|
|
|
|
ret.Placeholders[0].Values = append(ret.Placeholders[0].Values, x.Placeholders[0].Values...)
|
|
|
|
}
|
|
|
|
t.placeholderGroup, _ = proto.Marshal(ret)
|
|
|
|
}
|
|
|
|
}
|