mirror of https://github.com/milvus-io/milvus.git
Add metrics for proxy and querynode (#17205)
Signed-off-by: zhenshan.cao <zhenshan.cao@zilliz.com>pull/17248/head
parent
e196caf604
commit
ad9276a440
|
@ -70,8 +70,8 @@ var (
|
|||
Buckets: buckets, // unit: ms
|
||||
}, []string{nodeIDLabelName, queryTypeLabelName})
|
||||
|
||||
// ProxyReduceSearchResultLatency record the time that the proxy reduces search result.
|
||||
ProxyReduceSearchResultLatency = prometheus.NewHistogramVec(
|
||||
// ProxyReduceResultLatency record the time that the proxy reduces search result.
|
||||
ProxyReduceResultLatency = prometheus.NewHistogramVec(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: milvusNamespace,
|
||||
Subsystem: typeutil.ProxyRole,
|
||||
|
@ -80,8 +80,8 @@ var (
|
|||
Buckets: buckets, // unit: ms
|
||||
}, []string{nodeIDLabelName, queryTypeLabelName})
|
||||
|
||||
// ProxyDecodeSearchResultLatency record the time that the proxy decodes the search result.
|
||||
ProxyDecodeSearchResultLatency = prometheus.NewHistogramVec(
|
||||
// ProxyDecodeResultLatency record the time that the proxy decodes the search result.
|
||||
ProxyDecodeResultLatency = prometheus.NewHistogramVec(
|
||||
prometheus.HistogramOpts{
|
||||
Namespace: milvusNamespace,
|
||||
Subsystem: typeutil.ProxyRole,
|
||||
|
@ -233,8 +233,8 @@ func RegisterProxy(registry *prometheus.Registry) {
|
|||
registry.MustRegister(ProxySearchLatency)
|
||||
registry.MustRegister(ProxySendSQReqLatency)
|
||||
registry.MustRegister(ProxyWaitForSearchResultLatency)
|
||||
registry.MustRegister(ProxyReduceSearchResultLatency)
|
||||
registry.MustRegister(ProxyDecodeSearchResultLatency)
|
||||
registry.MustRegister(ProxyReduceResultLatency)
|
||||
registry.MustRegister(ProxyDecodeResultLatency)
|
||||
|
||||
registry.MustRegister(ProxyMsgStreamObjectsForPChan)
|
||||
|
||||
|
|
|
@ -35,7 +35,7 @@ var (
|
|||
prometheus.GaugeOpts{
|
||||
Namespace: milvusNamespace,
|
||||
Subsystem: typeutil.QueryCoordRole,
|
||||
Name: "entitiy_num",
|
||||
Name: "entity_num",
|
||||
Help: "number of entities",
|
||||
}, []string{})
|
||||
|
||||
|
|
|
@ -167,18 +167,6 @@ var (
|
|||
nodeIDLabelName,
|
||||
})
|
||||
|
||||
/* Todo reimplement in query_shard.go
|
||||
QueryNodeServiceTime = prometheus.NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Namespace: milvusNamespace,
|
||||
Subsystem: typeutil.QueryNodeRole,
|
||||
Name: "sync_utc_time",
|
||||
Help: "ServiceTimes of collections in QueryNode.",
|
||||
}, []string{
|
||||
nodeIDLabelName,
|
||||
})
|
||||
*/
|
||||
|
||||
QueryNodeNumFlowGraphs = prometheus.NewGaugeVec(
|
||||
prometheus.GaugeOpts{
|
||||
Namespace: milvusNamespace,
|
||||
|
@ -205,6 +193,5 @@ func RegisterQueryNode(registry *prometheus.Registry) {
|
|||
registry.MustRegister(QueryNodeSQSegmentLatencyInCore)
|
||||
registry.MustRegister(QueryNodeReduceLatency)
|
||||
registry.MustRegister(QueryNodeLoadSegmentLatency)
|
||||
// registry.MustRegister(QueryNodeServiceTime)
|
||||
registry.MustRegister(QueryNodeNumFlowGraphs)
|
||||
}
|
||||
|
|
|
@ -5,6 +5,7 @@ import (
|
|||
"errors"
|
||||
"fmt"
|
||||
"regexp"
|
||||
"strconv"
|
||||
"strings"
|
||||
"sync"
|
||||
|
||||
|
@ -15,6 +16,7 @@ import (
|
|||
"golang.org/x/sync/errgroup"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/timerecord"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
|
@ -311,10 +313,14 @@ func (t *queryTask) PostExecute(ctx context.Context) error {
|
|||
}()
|
||||
|
||||
wg.Wait()
|
||||
|
||||
metrics.ProxyDecodeResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.GetNodeID(), 10), metrics.QueryLabel).Observe(0.0)
|
||||
tr.Record("reduceResultStart")
|
||||
t.result, err = mergeRetrieveResults(t.toReduceResults)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
metrics.ProxyReduceResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.GetNodeID(), 10), metrics.QueryLabel).Observe(float64(tr.RecordSpan().Milliseconds()))
|
||||
t.result.CollectionName = t.collectionName
|
||||
|
||||
if len(t.result.FieldsData) > 0 {
|
||||
|
|
|
@ -360,7 +360,8 @@ func (t *searchTask) PostExecute(ctx context.Context) error {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
metrics.ProxyDecodeSearchResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.GetNodeID(), 10), metrics.SearchLabel).Observe(float64(tr.RecordSpan().Milliseconds()))
|
||||
metrics.ProxyDecodeResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.GetNodeID(), 10),
|
||||
metrics.SearchLabel).Observe(float64(tr.RecordSpan().Milliseconds()))
|
||||
log.Debug("proxy search post execute stage 2", zap.Any("len(validSearchResults)", len(validSearchResults)))
|
||||
if len(validSearchResults) <= 0 {
|
||||
log.Warn("search result is empty", zap.Any("requestID", t.Base.MsgID), zap.String("requestType", "search"))
|
||||
|
@ -392,7 +393,7 @@ func (t *searchTask) PostExecute(ctx context.Context) error {
|
|||
return err
|
||||
}
|
||||
|
||||
metrics.ProxyReduceSearchResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.GetNodeID(), 10), metrics.SuccessLabel).Observe(float64(tr.RecordSpan().Milliseconds()))
|
||||
metrics.ProxyReduceResultLatency.WithLabelValues(strconv.FormatInt(Params.ProxyCfg.GetNodeID(), 10), metrics.SearchLabel).Observe(float64(tr.RecordSpan().Milliseconds()))
|
||||
t.result.CollectionName = t.collectionName
|
||||
|
||||
schema, err := globalMetaCache.GetCollectionSchema(ctx, t.request.CollectionName)
|
||||
|
|
|
@ -26,12 +26,14 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/common"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/metrics"
|
||||
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
queryPb "github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/timerecord"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
)
|
||||
|
||||
|
@ -560,11 +562,18 @@ func (node *QueryNode) isHealthy() bool {
|
|||
|
||||
// Search performs replica search tasks.
|
||||
func (node *QueryNode) Search(ctx context.Context, req *queryPb.SearchRequest) (*internalpb.SearchResults, error) {
|
||||
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.SearchLabel, metrics.TotalLabel).Inc()
|
||||
failRet := &internalpb.SearchResults{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
},
|
||||
}
|
||||
|
||||
defer func() {
|
||||
if failRet.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.QueryLabel, metrics.FailLabel).Inc()
|
||||
}
|
||||
}()
|
||||
if !node.isHealthy() {
|
||||
failRet.Status.Reason = msgQueryNodeIsUnhealthy(Params.QueryNodeCfg.GetNodeID())
|
||||
return failRet, nil
|
||||
|
@ -577,14 +586,6 @@ func (node *QueryNode) Search(ctx context.Context, req *queryPb.SearchRequest) (
|
|||
return failRet, nil
|
||||
}
|
||||
|
||||
if !node.queryShardService.hasQueryShard(req.GetDmlChannel()) {
|
||||
// TODO: add replicaID in request or remove it in query shard
|
||||
err := node.queryShardService.addQueryShard(req.Req.CollectionID, req.GetDmlChannel(), 0)
|
||||
if err != nil {
|
||||
failRet.Status.Reason = err.Error()
|
||||
return failRet, nil
|
||||
}
|
||||
}
|
||||
qs, err := node.queryShardService.getQueryShard(req.GetDmlChannel())
|
||||
if err != nil {
|
||||
log.Warn("Search failed, failed to get query shard", zap.String("dml channel", req.GetDmlChannel()), zap.Error(err))
|
||||
|
@ -593,6 +594,8 @@ func (node *QueryNode) Search(ctx context.Context, req *queryPb.SearchRequest) (
|
|||
return failRet, nil
|
||||
}
|
||||
|
||||
tr := timerecord.NewTimeRecorder(fmt.Sprintf("search %d", req.Req.CollectionID))
|
||||
|
||||
if req.FromShardLeader {
|
||||
historicalTask, err2 := newSearchTask(ctx, req)
|
||||
if err2 != nil {
|
||||
|
@ -612,6 +615,15 @@ func (node *QueryNode) Search(ctx context.Context, req *queryPb.SearchRequest) (
|
|||
failRet.Status.Reason = err2.Error()
|
||||
return failRet, nil
|
||||
}
|
||||
|
||||
failRet.Status.ErrorCode = commonpb.ErrorCode_Success
|
||||
metrics.QueryNodeSQLatencyInQueue.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
|
||||
metrics.SearchLabel).Observe(float64(historicalTask.queueDur.Milliseconds()))
|
||||
metrics.QueryNodeReduceLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
|
||||
metrics.SearchLabel).Observe(float64(historicalTask.reduceDur.Milliseconds()))
|
||||
latency := tr.ElapseSpan()
|
||||
metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.SearchLabel).Observe(float64(latency.Milliseconds()))
|
||||
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.SearchLabel, metrics.SuccessLabel).Inc()
|
||||
return historicalTask.Ret, nil
|
||||
}
|
||||
|
||||
|
@ -671,6 +683,10 @@ func (node *QueryNode) Search(ctx context.Context, req *queryPb.SearchRequest) (
|
|||
errStreaming = err2
|
||||
return
|
||||
}
|
||||
metrics.QueryNodeSQLatencyInQueue.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
|
||||
metrics.SearchLabel).Observe(float64(streamingTask.queueDur.Milliseconds()))
|
||||
metrics.QueryNodeReduceLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
|
||||
metrics.SearchLabel).Observe(float64(streamingTask.reduceDur.Milliseconds()))
|
||||
streamingResult = streamingTask.Ret
|
||||
}()
|
||||
wg.Wait()
|
||||
|
@ -697,16 +713,28 @@ func (node *QueryNode) Search(ctx context.Context, req *queryPb.SearchRequest) (
|
|||
failRet.Status.Reason = err2.Error()
|
||||
return failRet, nil
|
||||
}
|
||||
|
||||
failRet.Status.ErrorCode = commonpb.ErrorCode_Success
|
||||
latency := tr.ElapseSpan()
|
||||
metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.SearchLabel).Observe(float64(latency.Milliseconds()))
|
||||
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.SearchLabel, metrics.SuccessLabel).Inc()
|
||||
return ret, nil
|
||||
}
|
||||
|
||||
// Query performs replica query tasks.
|
||||
func (node *QueryNode) Query(ctx context.Context, req *queryPb.QueryRequest) (*internalpb.RetrieveResults, error) {
|
||||
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.QueryLabel, metrics.TotalLabel).Inc()
|
||||
failRet := &internalpb.RetrieveResults{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UnexpectedError,
|
||||
},
|
||||
}
|
||||
|
||||
defer func() {
|
||||
if failRet.Status.ErrorCode != commonpb.ErrorCode_Success {
|
||||
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.SearchLabel, metrics.FailLabel).Inc()
|
||||
}
|
||||
}()
|
||||
if !node.isHealthy() {
|
||||
failRet.Status.Reason = msgQueryNodeIsUnhealthy(Params.QueryNodeCfg.GetNodeID())
|
||||
return failRet, nil
|
||||
|
@ -718,12 +746,6 @@ func (node *QueryNode) Query(ctx context.Context, req *queryPb.QueryRequest) (*i
|
|||
return failRet, nil
|
||||
}
|
||||
|
||||
if !node.queryShardService.hasQueryShard(req.GetDmlChannel()) {
|
||||
err := node.queryShardService.addQueryShard(req.Req.CollectionID, req.GetDmlChannel(), 0) // TODO: add replicaID in request or remove it in query shard
|
||||
failRet.Status.Reason = err.Error()
|
||||
return failRet, nil
|
||||
}
|
||||
|
||||
qs, err := node.queryShardService.getQueryShard(req.GetDmlChannel())
|
||||
if err != nil {
|
||||
log.Warn("Query failed, failed to get query shard", zap.String("dml channel", req.GetDmlChannel()), zap.Error(err))
|
||||
|
@ -731,6 +753,7 @@ func (node *QueryNode) Query(ctx context.Context, req *queryPb.QueryRequest) (*i
|
|||
return failRet, nil
|
||||
}
|
||||
|
||||
tr := timerecord.NewTimeRecorder(fmt.Sprintf("retrieve %d", req.Req.CollectionID))
|
||||
if req.FromShardLeader {
|
||||
// construct a queryTask
|
||||
queryTask := newQueryTask(ctx, req)
|
||||
|
@ -747,6 +770,14 @@ func (node *QueryNode) Query(ctx context.Context, req *queryPb.QueryRequest) (*i
|
|||
failRet.Status.Reason = err2.Error()
|
||||
return failRet, nil
|
||||
}
|
||||
failRet.Status.ErrorCode = commonpb.ErrorCode_Success
|
||||
metrics.QueryNodeSQLatencyInQueue.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
|
||||
metrics.QueryLabel).Observe(float64(queryTask.queueDur.Milliseconds()))
|
||||
metrics.QueryNodeReduceLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
|
||||
metrics.QueryLabel).Observe(float64(queryTask.reduceDur.Milliseconds()))
|
||||
latency := tr.ElapseSpan()
|
||||
metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.QueryLabel).Observe(float64(latency.Milliseconds()))
|
||||
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.QueryLabel, metrics.SuccessLabel).Inc()
|
||||
return queryTask.Ret, nil
|
||||
}
|
||||
|
||||
|
@ -801,6 +832,10 @@ func (node *QueryNode) Query(ctx context.Context, req *queryPb.QueryRequest) (*i
|
|||
if err2 != nil {
|
||||
return
|
||||
}
|
||||
metrics.QueryNodeSQLatencyInQueue.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
|
||||
metrics.QueryLabel).Observe(float64(streamingTask.queueDur.Milliseconds()))
|
||||
metrics.QueryNodeReduceLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()),
|
||||
metrics.QueryLabel).Observe(float64(streamingTask.reduceDur.Milliseconds()))
|
||||
streamingResult = streamingTask.Ret
|
||||
}()
|
||||
wg.Wait()
|
||||
|
@ -827,6 +862,10 @@ func (node *QueryNode) Query(ctx context.Context, req *queryPb.QueryRequest) (*i
|
|||
failRet.Status.Reason = err2.Error()
|
||||
return failRet, nil
|
||||
}
|
||||
failRet.Status.ErrorCode = commonpb.ErrorCode_Success
|
||||
latency := tr.ElapseSpan()
|
||||
metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.QueryLabel).Observe(float64(latency.Milliseconds()))
|
||||
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(Params.QueryNodeCfg.GetNodeID()), metrics.QueryLabel, metrics.SuccessLabel).Inc()
|
||||
return ret, nil
|
||||
}
|
||||
|
||||
|
|
|
@ -81,6 +81,7 @@ func (q *queryTask) queryOnStreaming() error {
|
|||
return sErr
|
||||
}
|
||||
|
||||
q.tr.RecordSpan()
|
||||
mergedResult, err := mergeSegcoreRetrieveResults(sResults)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -91,6 +92,7 @@ func (q *queryTask) queryOnStreaming() error {
|
|||
Ids: mergedResult.Ids,
|
||||
FieldsData: mergedResult.FieldsData,
|
||||
}
|
||||
q.reduceDur = q.tr.RecordSpan()
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -19,6 +19,7 @@ package querynode
|
|||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"time"
|
||||
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
|
@ -40,8 +41,10 @@ type readTask interface {
|
|||
Merge(readTask)
|
||||
CanMergeWith(readTask) bool
|
||||
CPUUsage() int32
|
||||
SetMaxCPUUSage(int32)
|
||||
Timeout() bool
|
||||
|
||||
SetMaxCPUUSage(int32)
|
||||
SetStep(step TaskStep)
|
||||
}
|
||||
|
||||
var _ readTask = (*baseReadTask)(nil)
|
||||
|
@ -59,17 +62,55 @@ type baseReadTask struct {
|
|||
TravelTimestamp uint64
|
||||
GuaranteeTimestamp uint64
|
||||
TimeoutTimestamp uint64
|
||||
step TaskStep
|
||||
queueDur time.Duration
|
||||
reduceDur time.Duration
|
||||
tr *timerecord.TimeRecorder
|
||||
}
|
||||
|
||||
func (b *baseReadTask) SetStep(step TaskStep) {
|
||||
b.step = step
|
||||
switch step {
|
||||
case TaskStepEnqueue:
|
||||
b.queueDur = 0
|
||||
b.tr.Record("enqueueStart")
|
||||
case TaskStepPreExecute:
|
||||
b.queueDur = b.tr.Record("enqueueEnd")
|
||||
}
|
||||
}
|
||||
|
||||
func (b *baseReadTask) OnEnqueue() error {
|
||||
b.SetStep(TaskStepEnqueue)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *baseReadTask) SetMaxCPUUSage(cpu int32) {
|
||||
b.maxCPU = cpu
|
||||
}
|
||||
|
||||
func (b *baseReadTask) Execute(ctx context.Context) error {
|
||||
func (b *baseReadTask) PreExecute(ctx context.Context) error {
|
||||
b.SetStep(TaskStepPreExecute)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *baseReadTask) Execute(ctx context.Context) error {
|
||||
b.SetStep(TaskStepExecute)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *baseReadTask) PostExecute(ctx context.Context) error {
|
||||
b.SetStep(TaskStepPostExecute)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *baseReadTask) Notify(err error) {
|
||||
switch b.step {
|
||||
case TaskStepEnqueue:
|
||||
b.queueDur = b.tr.Record("enqueueEnd")
|
||||
}
|
||||
b.baseTask.Notify(err)
|
||||
}
|
||||
|
||||
// GetCollectionID return CollectionID.
|
||||
func (b *baseReadTask) GetCollectionID() UniqueID {
|
||||
return b.CollectionID
|
||||
|
|
|
@ -205,6 +205,7 @@ func (s *taskScheduler) scheduleReadTasks() {
|
|||
|
||||
func (s *taskScheduler) AddReadTask(ctx context.Context, t readTask) error {
|
||||
t.SetMaxCPUUSage(s.maxCPUUsage)
|
||||
t.OnEnqueue()
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return fmt.Errorf("taskScheduler AddReadTask context is done")
|
||||
|
|
|
@ -58,6 +58,10 @@ type searchTask struct {
|
|||
}
|
||||
|
||||
func (s *searchTask) PreExecute(ctx context.Context) error {
|
||||
s.SetStep(TaskStepPreExecute)
|
||||
for _, t := range s.otherTasks {
|
||||
t.SetStep(TaskStepPreExecute)
|
||||
}
|
||||
s.combinePlaceHolderGroups()
|
||||
return nil
|
||||
}
|
||||
|
@ -187,6 +191,7 @@ func (s *searchTask) reduceResults(searchReq *searchRequest, results []*SearchRe
|
|||
isEmpty := len(results) == 0
|
||||
cnt := 1 + len(s.otherTasks)
|
||||
var t *searchTask
|
||||
s.tr.RecordSpan()
|
||||
if !isEmpty {
|
||||
sInfo := parseSliceInfo(s.OrigNQs, s.OrigTopKs, s.NQ)
|
||||
numSegment := int64(len(results))
|
||||
|
@ -240,6 +245,7 @@ func (s *searchTask) reduceResults(searchReq *searchRequest, results []*SearchRe
|
|||
}
|
||||
}
|
||||
}
|
||||
s.reduceDur = s.tr.RecordSpan()
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,43 @@
|
|||
// 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
|
||||
|
||||
type TaskStep int32
|
||||
|
||||
const (
|
||||
TaskStepUnissued TaskStep = 0
|
||||
TaskStepEnqueue TaskStep = 1
|
||||
TaskStepPreExecute TaskStep = 2
|
||||
TaskStepExecute TaskStep = 3
|
||||
TaskStepPostExecute TaskStep = 4
|
||||
)
|
||||
|
||||
var TaskStepNames = map[TaskStep]string{
|
||||
0: "Unissued",
|
||||
1: "Enqueue",
|
||||
2: "PreExecute",
|
||||
3: "Execute",
|
||||
4: "PostExecute",
|
||||
}
|
||||
|
||||
func (x TaskStep) String() string {
|
||||
ret, ok := TaskStepNames[x]
|
||||
if !ok {
|
||||
return "None"
|
||||
}
|
||||
return ret
|
||||
}
|
|
@ -0,0 +1,31 @@
|
|||
// 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 (
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
)
|
||||
|
||||
func TestTaskStep_String(t *testing.T) {
|
||||
assert.Equal(t, TaskStepUnissued.String(), "Unissued")
|
||||
assert.Equal(t, TaskStepEnqueue.String(), "Enqueue")
|
||||
assert.Equal(t, TaskStepPreExecute.String(), "PreExecute")
|
||||
assert.Equal(t, TaskStepExecute.String(), "Execute")
|
||||
assert.Equal(t, TaskStepPostExecute.String(), "PostExecute")
|
||||
}
|
Loading…
Reference in New Issue