2023-03-27 00:42:00 +08:00
|
|
|
|
// 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 querynodev2
|
|
|
|
|
|
|
|
|
|
import (
|
|
|
|
|
"context"
|
|
|
|
|
"fmt"
|
|
|
|
|
"strconv"
|
|
|
|
|
"sync"
|
2024-05-17 10:11:37 +08:00
|
|
|
|
"time"
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
|
|
"github.com/golang/protobuf/proto"
|
|
|
|
|
"github.com/samber/lo"
|
2023-05-23 16:01:26 +08:00
|
|
|
|
"go.opentelemetry.io/otel/trace"
|
2023-03-27 00:42:00 +08:00
|
|
|
|
"go.uber.org/zap"
|
|
|
|
|
"golang.org/x/sync/errgroup"
|
|
|
|
|
|
2023-06-09 01:28:37 +08:00
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
|
2023-07-17 19:23:19 +08:00
|
|
|
|
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
2024-03-22 10:27:17 +08:00
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
2023-07-17 19:23:19 +08:00
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
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-04-26 10:14:41 +08:00
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/segcorepb"
|
2023-03-27 00:42:00 +08:00
|
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/collector"
|
|
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/delegator"
|
|
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/segments"
|
2023-05-23 16:01:26 +08:00
|
|
|
|
"github.com/milvus-io/milvus/internal/querynodev2/tasks"
|
2023-03-27 00:42:00 +08:00
|
|
|
|
"github.com/milvus-io/milvus/internal/storage"
|
2023-09-12 10:19:17 +08:00
|
|
|
|
"github.com/milvus-io/milvus/internal/util/streamrpc"
|
2023-04-06 19:14:32 +08:00
|
|
|
|
"github.com/milvus-io/milvus/pkg/common"
|
|
|
|
|
"github.com/milvus-io/milvus/pkg/log"
|
|
|
|
|
"github.com/milvus-io/milvus/pkg/metrics"
|
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/merr"
|
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/timerecord"
|
|
|
|
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
2023-03-27 00:42:00 +08:00
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
// GetComponentStates returns information about whether the node is healthy
|
2023-09-26 09:57:25 +08:00
|
|
|
|
func (node *QueryNode) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
|
2023-03-27 00:42:00 +08:00
|
|
|
|
stats := &milvuspb.ComponentStates{
|
2023-10-11 21:01:35 +08:00
|
|
|
|
Status: merr.Success(),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
code := node.lifetime.GetState()
|
|
|
|
|
nodeID := common.NotRegisteredID
|
|
|
|
|
|
|
|
|
|
if node.session != nil && node.session.Registered() {
|
2024-02-21 11:54:53 +08:00
|
|
|
|
nodeID = node.GetNodeID()
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2024-02-28 13:13:12 +08:00
|
|
|
|
log.Debug("QueryNode current state", zap.Int64("NodeID", nodeID), zap.String("StateCode", code.String()))
|
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
info := &milvuspb.ComponentInfo{
|
|
|
|
|
NodeID: nodeID,
|
|
|
|
|
Role: typeutil.QueryNodeRole,
|
|
|
|
|
StateCode: code,
|
|
|
|
|
}
|
|
|
|
|
stats.State = info
|
|
|
|
|
return stats, nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// GetTimeTickChannel returns the time tick channel
|
|
|
|
|
// TimeTickChannel contains many time tick messages, which will be sent by query nodes
|
2023-09-26 09:57:25 +08:00
|
|
|
|
func (node *QueryNode) GetTimeTickChannel(ctx context.Context, req *internalpb.GetTimeTickChannelRequest) (*milvuspb.StringResponse, error) {
|
2023-03-27 00:42:00 +08:00
|
|
|
|
return &milvuspb.StringResponse{
|
2023-10-11 21:01:35 +08:00
|
|
|
|
Status: merr.Success(),
|
2023-09-04 09:57:09 +08:00
|
|
|
|
Value: paramtable.Get().CommonCfg.QueryCoordTimeTick.GetValue(),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// GetStatisticsChannel returns the statistics channel
|
|
|
|
|
// Statistics channel contains statistics infos of query nodes, such as segment infos, memory infos
|
2023-09-26 09:57:25 +08:00
|
|
|
|
func (node *QueryNode) GetStatisticsChannel(ctx context.Context, req *internalpb.GetStatisticsChannelRequest) (*milvuspb.StringResponse, error) {
|
2023-03-27 00:42:00 +08:00
|
|
|
|
return &milvuspb.StringResponse{
|
2023-10-11 21:01:35 +08:00
|
|
|
|
Status: merr.Success(),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// GetStatistics returns loaded statistics of collection.
|
|
|
|
|
func (node *QueryNode) GetStatistics(ctx context.Context, req *querypb.GetStatisticsRequest) (*internalpb.GetStatisticsResponse, error) {
|
2023-10-26 19:14:10 +08:00
|
|
|
|
log := log.Ctx(ctx).With(
|
2023-03-27 00:42:00 +08:00
|
|
|
|
zap.Int64("msgID", req.GetReq().GetBase().GetMsgID()),
|
|
|
|
|
zap.Strings("vChannels", req.GetDmlChannels()),
|
|
|
|
|
zap.Int64s("segmentIDs", req.GetSegmentIDs()),
|
|
|
|
|
zap.Uint64("guaranteeTimestamp", req.GetReq().GetGuaranteeTimestamp()),
|
2023-10-26 19:14:10 +08:00
|
|
|
|
zap.Uint64("timeTravel", req.GetReq().GetTravelTimestamp()),
|
|
|
|
|
)
|
|
|
|
|
log.Debug("received GetStatisticsRequest")
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-03-27 00:42:00 +08:00
|
|
|
|
return &internalpb.GetStatisticsResponse{
|
2023-05-10 09:31:19 +08:00
|
|
|
|
Status: merr.Status(err),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
|
|
|
|
failRet := &internalpb.GetStatisticsResponse{
|
2023-10-11 21:01:35 +08:00
|
|
|
|
Status: merr.Success(),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
var toReduceResults []*internalpb.GetStatisticsResponse
|
|
|
|
|
var mu sync.Mutex
|
|
|
|
|
runningGp, runningCtx := errgroup.WithContext(ctx)
|
|
|
|
|
for _, ch := range req.GetDmlChannels() {
|
|
|
|
|
ch := ch
|
|
|
|
|
req := &querypb.GetStatisticsRequest{
|
|
|
|
|
Req: req.Req,
|
|
|
|
|
DmlChannels: []string{ch},
|
|
|
|
|
SegmentIDs: req.SegmentIDs,
|
|
|
|
|
FromShardLeader: req.FromShardLeader,
|
|
|
|
|
Scope: req.Scope,
|
|
|
|
|
}
|
|
|
|
|
runningGp.Go(func() error {
|
|
|
|
|
ret, err := node.getChannelStatistics(runningCtx, req, ch)
|
2023-09-15 10:09:21 +08:00
|
|
|
|
if err == nil {
|
|
|
|
|
err = merr.Error(ret.GetStatus())
|
|
|
|
|
}
|
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
mu.Lock()
|
|
|
|
|
defer mu.Unlock()
|
|
|
|
|
if err != nil {
|
2023-09-12 16:07:18 +08:00
|
|
|
|
failRet.Status = merr.Status(err)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
return err
|
|
|
|
|
}
|
|
|
|
|
toReduceResults = append(toReduceResults, ret)
|
|
|
|
|
return nil
|
|
|
|
|
})
|
|
|
|
|
}
|
|
|
|
|
if err := runningGp.Wait(); err != nil {
|
|
|
|
|
return failRet, nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
ret, err := reduceStatisticResponse(toReduceResults)
|
|
|
|
|
if err != nil {
|
2023-09-12 16:07:18 +08:00
|
|
|
|
failRet.Status = merr.Status(err)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
return failRet, nil
|
|
|
|
|
}
|
|
|
|
|
log.Debug("reduce statistic result done")
|
|
|
|
|
|
|
|
|
|
return ret, nil
|
|
|
|
|
}
|
|
|
|
|
|
2023-07-17 19:23:19 +08:00
|
|
|
|
func (node *QueryNode) composeIndexMeta(indexInfos []*indexpb.IndexInfo, schema *schemapb.CollectionSchema) *segcorepb.CollectionIndexMeta {
|
|
|
|
|
fieldIndexMetas := make([]*segcorepb.FieldIndexMeta, 0)
|
|
|
|
|
for _, info := range indexInfos {
|
|
|
|
|
fieldIndexMetas = append(fieldIndexMetas, &segcorepb.FieldIndexMeta{
|
|
|
|
|
CollectionID: info.GetCollectionID(),
|
|
|
|
|
FieldID: info.GetFieldID(),
|
|
|
|
|
IndexName: info.GetIndexName(),
|
|
|
|
|
TypeParams: info.GetTypeParams(),
|
|
|
|
|
IndexParams: info.GetIndexParams(),
|
|
|
|
|
IsAutoIndex: info.GetIsAutoIndex(),
|
|
|
|
|
UserIndexParams: info.GetUserIndexParams(),
|
|
|
|
|
})
|
|
|
|
|
}
|
|
|
|
|
sizePerRecord, err := typeutil.EstimateSizePerRecord(schema)
|
|
|
|
|
maxIndexRecordPerSegment := int64(0)
|
|
|
|
|
if err != nil || sizePerRecord == 0 {
|
|
|
|
|
log.Warn("failed to transfer segment size to collection, because failed to estimate size per record", zap.Error(err))
|
|
|
|
|
} else {
|
|
|
|
|
threshold := paramtable.Get().DataCoordCfg.SegmentMaxSize.GetAsFloat() * 1024 * 1024
|
|
|
|
|
proportion := paramtable.Get().DataCoordCfg.SegmentSealProportion.GetAsFloat()
|
|
|
|
|
maxIndexRecordPerSegment = int64(threshold * proportion / float64(sizePerRecord))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return &segcorepb.CollectionIndexMeta{
|
|
|
|
|
IndexMetas: fieldIndexMetas,
|
|
|
|
|
MaxIndexRowCount: maxIndexRecordPerSegment,
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
// WatchDmChannels create consumers on dmChannels to receive Incremental data,which is the important part of real-time query
|
2024-02-21 22:12:53 +08:00
|
|
|
|
func (node *QueryNode) WatchDmChannels(ctx context.Context, req *querypb.WatchDmChannelsRequest) (status *commonpb.Status, e error) {
|
2024-05-17 10:11:37 +08:00
|
|
|
|
defer node.updateDistributionModifyTS()
|
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
channel := req.GetInfos()[0]
|
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("collectionID", req.GetCollectionID()),
|
|
|
|
|
zap.String("channel", channel.GetChannelName()),
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.Int64("currentNodeID", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
log.Info("received watch channel request",
|
|
|
|
|
zap.Int64("version", req.GetVersion()),
|
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
// check node healthy
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-05-10 09:31:19 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
2023-10-31 15:50:13 +08:00
|
|
|
|
// check index
|
|
|
|
|
if len(req.GetIndexInfoList()) == 0 {
|
|
|
|
|
err := merr.WrapErrIndexNotFoundForCollection(req.GetSchema().GetName())
|
2023-06-13 22:08:38 +08:00
|
|
|
|
return merr.Status(err), nil
|
|
|
|
|
}
|
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
if !node.subscribingChannels.Insert(channel.GetChannelName()) {
|
|
|
|
|
msg := "channel subscribing..."
|
|
|
|
|
log.Warn(msg)
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.subscribingChannels.Remove(channel.GetChannelName())
|
|
|
|
|
|
2023-08-23 10:10:22 +08:00
|
|
|
|
// to avoid concurrent watch/unwatch
|
|
|
|
|
if node.unsubscribingChannels.Contain(channel.GetChannelName()) {
|
2023-09-04 09:57:09 +08:00
|
|
|
|
err := merr.WrapErrChannelReduplicate(channel.GetChannelName(), "the other same channel is unsubscribing")
|
|
|
|
|
log.Warn("failed to unsubscribe channel", zap.Error(err))
|
2023-08-23 10:10:22 +08:00
|
|
|
|
return merr.Status(err), nil
|
|
|
|
|
}
|
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
_, exist := node.delegators.Get(channel.GetChannelName())
|
|
|
|
|
if exist {
|
|
|
|
|
log.Info("channel already subscribed")
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2023-05-23 17:17:26 +08:00
|
|
|
|
|
2023-07-17 19:23:19 +08:00
|
|
|
|
node.manager.Collection.PutOrRef(req.GetCollectionID(), req.GetSchema(),
|
|
|
|
|
node.composeIndexMeta(req.GetIndexInfoList(), req.Schema), req.GetLoadMeta())
|
2024-02-21 22:12:53 +08:00
|
|
|
|
defer func() {
|
|
|
|
|
if !merr.Ok(status) {
|
|
|
|
|
node.manager.Collection.Unref(req.GetCollectionID(), 1)
|
|
|
|
|
}
|
|
|
|
|
}()
|
2024-01-08 15:34:48 +08:00
|
|
|
|
|
2023-11-08 16:42:21 +08:00
|
|
|
|
delegator, err := delegator.NewShardDelegator(
|
|
|
|
|
ctx,
|
|
|
|
|
req.GetCollectionID(),
|
|
|
|
|
req.GetReplicaID(),
|
|
|
|
|
channel.GetChannelName(),
|
|
|
|
|
req.GetVersion(),
|
|
|
|
|
node.clusterManager,
|
|
|
|
|
node.manager,
|
|
|
|
|
node.tSafeManager,
|
|
|
|
|
node.loader,
|
|
|
|
|
node.factory,
|
|
|
|
|
channel.GetSeekPosition().GetTimestamp(),
|
fix: correct autoindex segment num (#28387)
Fix #28386
Current code snippet
```
// get delegator
sd, ok := node.delegators.Get(channel)
if !ok {
err := merr.WrapErrChannelNotFound(channel)
log.Warn("Query failed, failed to get shard delegator for search", zap.Error(err))
return nil, err
}
req, err = node.optimizeSearchParams(ctx, req, sd)
if err != nil {
log.Warn("failed to optimize search params", zap.Error(err))
return nil, err
}
// do search
results, err := sd.Search(searchCtx, req)
```
We could move these into `ShardDelegator`, and directly use sealed
segment num in `Search` methods, also segment num got outside could be
wrong when we specify partitions.
Signed-off-by: chasingegg <chao.gao@zilliz.com>
2023-11-22 11:12:22 +08:00
|
|
|
|
node.queryHook,
|
2024-03-22 13:57:06 +08:00
|
|
|
|
node.chunkManager,
|
2023-11-08 16:42:21 +08:00
|
|
|
|
)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
if err != nil {
|
|
|
|
|
log.Warn("failed to create shard delegator", zap.Error(err))
|
2023-09-26 17:15:27 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
node.delegators.Insert(channel.GetChannelName(), delegator)
|
|
|
|
|
defer func() {
|
|
|
|
|
if err != nil {
|
|
|
|
|
node.delegators.GetAndRemove(channel.GetChannelName())
|
|
|
|
|
}
|
|
|
|
|
}()
|
|
|
|
|
|
|
|
|
|
// create tSafe
|
2023-10-26 19:14:10 +08:00
|
|
|
|
node.tSafeManager.Add(ctx, channel.ChannelName, channel.GetSeekPosition().GetTimestamp())
|
2023-03-27 00:42:00 +08:00
|
|
|
|
defer func() {
|
|
|
|
|
if err != nil {
|
2023-10-26 19:14:10 +08:00
|
|
|
|
node.tSafeManager.Remove(ctx, channel.ChannelName)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
}()
|
|
|
|
|
|
|
|
|
|
pipeline, err := node.pipelineManager.Add(req.GetCollectionID(), channel.GetChannelName())
|
|
|
|
|
if err != nil {
|
|
|
|
|
msg := "failed to create pipeline"
|
|
|
|
|
log.Warn(msg, zap.Error(err))
|
2023-09-26 17:15:27 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
defer func() {
|
|
|
|
|
if err != nil {
|
|
|
|
|
node.pipelineManager.Remove(channel.GetChannelName())
|
|
|
|
|
}
|
|
|
|
|
}()
|
|
|
|
|
|
2023-12-28 18:18:54 +08:00
|
|
|
|
growingInfo := lo.SliceToMap(channel.GetUnflushedSegmentIds(), func(id int64) (int64, uint64) {
|
|
|
|
|
info := req.GetSegmentInfos()[id]
|
|
|
|
|
return id, info.GetDmlPosition().GetTimestamp()
|
|
|
|
|
})
|
2024-04-10 15:29:17 +08:00
|
|
|
|
delegator.AddExcludedSegments(growingInfo)
|
2023-12-28 18:18:54 +08:00
|
|
|
|
|
2024-02-23 10:02:55 +08:00
|
|
|
|
defer func() {
|
|
|
|
|
if err != nil {
|
|
|
|
|
// remove legacy growing
|
2024-05-06 20:29:30 +08:00
|
|
|
|
node.manager.Segment.RemoveBy(ctx, segments.WithChannel(channel.GetChannelName()),
|
2024-02-23 10:02:55 +08:00
|
|
|
|
segments.WithType(segments.SegmentTypeGrowing))
|
2024-05-07 10:03:42 +08:00
|
|
|
|
// remove legacy l0 segments
|
2024-05-07 10:31:49 +08:00
|
|
|
|
node.manager.Segment.RemoveBy(ctx, segments.WithChannel(channel.GetChannelName()),
|
2024-05-07 10:03:42 +08:00
|
|
|
|
segments.WithLevel(datapb.SegmentLevel_L0))
|
2024-02-23 10:02:55 +08:00
|
|
|
|
}
|
|
|
|
|
}()
|
2024-05-07 10:03:42 +08:00
|
|
|
|
|
|
|
|
|
err = loadL0Segments(ctx, delegator, req)
|
|
|
|
|
if err != nil {
|
|
|
|
|
log.Warn("failed to load l0 segments", zap.Error(err))
|
|
|
|
|
return merr.Status(err), nil
|
|
|
|
|
}
|
2023-03-27 00:42:00 +08:00
|
|
|
|
err = loadGrowingSegments(ctx, delegator, req)
|
|
|
|
|
if err != nil {
|
|
|
|
|
msg := "failed to load growing segments"
|
|
|
|
|
log.Warn(msg, zap.Error(err))
|
2023-09-26 17:15:27 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
position := &msgpb.MsgPosition{
|
|
|
|
|
ChannelName: channel.SeekPosition.ChannelName,
|
|
|
|
|
MsgID: channel.SeekPosition.MsgID,
|
|
|
|
|
Timestamp: channel.SeekPosition.Timestamp,
|
|
|
|
|
}
|
|
|
|
|
err = pipeline.ConsumeMsgStream(position)
|
|
|
|
|
if err != nil {
|
2023-04-26 17:06:34 +08:00
|
|
|
|
err = merr.WrapErrServiceUnavailable(err.Error(), "InitPipelineFailed")
|
2023-03-27 00:42:00 +08:00
|
|
|
|
log.Warn(err.Error(),
|
|
|
|
|
zap.Int64("collectionID", channel.CollectionID),
|
|
|
|
|
zap.String("channel", channel.ChannelName),
|
|
|
|
|
)
|
2023-04-26 17:06:34 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// start pipeline
|
|
|
|
|
pipeline.Start()
|
|
|
|
|
// delegator after all steps done
|
|
|
|
|
delegator.Start()
|
|
|
|
|
log.Info("watch dml channel success")
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (node *QueryNode) UnsubDmChannel(ctx context.Context, req *querypb.UnsubDmChannelRequest) (*commonpb.Status, error) {
|
2024-05-17 10:11:37 +08:00
|
|
|
|
defer node.updateDistributionModifyTS()
|
2023-03-27 00:42:00 +08:00
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("collectionID", req.GetCollectionID()),
|
|
|
|
|
zap.String("channel", req.GetChannelName()),
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.Int64("currentNodeID", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
log.Info("received unsubscribe channel request")
|
|
|
|
|
|
|
|
|
|
// check node healthy
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-05-10 09:31:19 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
2023-08-23 10:10:22 +08:00
|
|
|
|
node.unsubscribingChannels.Insert(req.GetChannelName())
|
|
|
|
|
defer node.unsubscribingChannels.Remove(req.GetChannelName())
|
2023-07-06 14:18:26 +08:00
|
|
|
|
delegator, ok := node.delegators.GetAndRemove(req.GetChannelName())
|
2023-04-20 19:30:33 +08:00
|
|
|
|
if ok {
|
|
|
|
|
// close the delegator first to block all coming query/search requests
|
2023-03-27 00:42:00 +08:00
|
|
|
|
delegator.Close()
|
|
|
|
|
|
2023-04-20 19:30:33 +08:00
|
|
|
|
node.pipelineManager.Remove(req.GetChannelName())
|
2024-05-06 20:29:30 +08:00
|
|
|
|
node.manager.Segment.RemoveBy(ctx, segments.WithChannel(req.GetChannelName()), segments.WithType(segments.SegmentTypeGrowing))
|
|
|
|
|
node.manager.Segment.RemoveBy(ctx, segments.WithChannel(req.GetChannelName()), segments.WithLevel(datapb.SegmentLevel_L0))
|
2023-10-26 19:14:10 +08:00
|
|
|
|
node.tSafeManager.Remove(ctx, req.GetChannelName())
|
2023-07-14 10:28:30 +08:00
|
|
|
|
|
|
|
|
|
node.manager.Collection.Unref(req.GetCollectionID(), 1)
|
2023-04-20 19:30:33 +08:00
|
|
|
|
}
|
2023-03-27 00:42:00 +08:00
|
|
|
|
log.Info("unsubscribed channel")
|
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (node *QueryNode) LoadPartitions(ctx context.Context, req *querypb.LoadPartitionsRequest) (*commonpb.Status, error) {
|
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("collectionID", req.GetCollectionID()),
|
|
|
|
|
zap.Int64s("partitionIDs", req.GetPartitionIDs()),
|
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
log.Info("received load partitions request")
|
|
|
|
|
// check node healthy
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthyOrStopping); err != nil {
|
2023-05-10 09:31:19 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2023-05-01 16:36:38 +08:00
|
|
|
|
defer node.lifetime.Done()
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
|
|
collection := node.manager.Collection.Get(req.GetCollectionID())
|
2023-05-01 16:36:38 +08:00
|
|
|
|
if collection != nil {
|
|
|
|
|
collection.AddPartition(req.GetPartitionIDs()...)
|
2023-05-18 09:17:23 +08:00
|
|
|
|
}
|
2023-05-06 17:42:39 +08:00
|
|
|
|
|
|
|
|
|
log.Info("load partitions done")
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// LoadSegments load historical data into query node, historical data can be vector data or index
|
|
|
|
|
func (node *QueryNode) LoadSegments(ctx context.Context, req *querypb.LoadSegmentsRequest) (*commonpb.Status, error) {
|
2024-05-17 10:11:37 +08:00
|
|
|
|
defer node.updateDistributionModifyTS()
|
2023-03-27 00:42:00 +08:00
|
|
|
|
segment := req.GetInfos()[0]
|
|
|
|
|
|
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("collectionID", segment.GetCollectionID()),
|
|
|
|
|
zap.Int64("partitionID", segment.GetPartitionID()),
|
|
|
|
|
zap.String("shard", segment.GetInsertChannel()),
|
|
|
|
|
zap.Int64("segmentID", segment.GetSegmentID()),
|
2024-06-07 14:17:20 +08:00
|
|
|
|
zap.String("level", segment.GetLevel().String()),
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.Int64("currentNodeID", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
log.Info("received load segments request",
|
|
|
|
|
zap.Int64("version", req.GetVersion()),
|
|
|
|
|
zap.Bool("needTransfer", req.GetNeedTransfer()),
|
2024-02-07 14:56:47 +08:00
|
|
|
|
zap.String("loadScope", req.GetLoadScope().String()))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
// check node healthy
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-05-10 09:31:19 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2024-01-18 22:18:55 +08:00
|
|
|
|
defer node.lifetime.Done()
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
2023-11-23 19:42:30 +08:00
|
|
|
|
// check index
|
|
|
|
|
if len(req.GetIndexInfoList()) == 0 {
|
|
|
|
|
err := merr.WrapErrIndexNotFoundForCollection(req.GetSchema().GetName())
|
|
|
|
|
return merr.Status(err), nil
|
|
|
|
|
}
|
|
|
|
|
|
2024-05-15 12:59:34 +08:00
|
|
|
|
// fallback binlog memory size to log size when it is zero
|
|
|
|
|
fallbackBinlogMemorySize := func(binlogs []*datapb.FieldBinlog) {
|
|
|
|
|
for _, insertBinlogs := range binlogs {
|
|
|
|
|
for _, b := range insertBinlogs.GetBinlogs() {
|
|
|
|
|
if b.GetMemorySize() == 0 {
|
|
|
|
|
b.MemorySize = b.GetLogSize()
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
for _, s := range req.GetInfos() {
|
|
|
|
|
fallbackBinlogMemorySize(s.GetBinlogPaths())
|
|
|
|
|
fallbackBinlogMemorySize(s.GetStatslogs())
|
|
|
|
|
fallbackBinlogMemorySize(s.GetDeltalogs())
|
|
|
|
|
}
|
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
// Delegates request to workers
|
|
|
|
|
if req.GetNeedTransfer() {
|
|
|
|
|
delegator, ok := node.delegators.Get(segment.GetInsertChannel())
|
|
|
|
|
if !ok {
|
|
|
|
|
msg := "failed to load segments, delegator not found"
|
|
|
|
|
log.Warn(msg)
|
2023-09-26 17:15:27 +08:00
|
|
|
|
err := merr.WrapErrChannelNotFound(segment.GetInsertChannel())
|
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
req.NeedTransfer = false
|
|
|
|
|
err := delegator.LoadSegments(ctx, req)
|
|
|
|
|
if err != nil {
|
|
|
|
|
log.Warn("delegator failed to load segments", zap.Error(err))
|
2023-09-26 17:15:27 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
2023-11-02 23:52:16 +08:00
|
|
|
|
node.manager.Collection.PutOrRef(req.GetCollectionID(), req.GetSchema(),
|
|
|
|
|
node.composeIndexMeta(req.GetIndexInfoList(), req.GetSchema()), req.GetLoadMeta())
|
|
|
|
|
defer node.manager.Collection.Unref(req.GetCollectionID(), 1)
|
|
|
|
|
|
2023-06-19 17:14:41 +08:00
|
|
|
|
if req.GetLoadScope() == querypb.LoadScope_Delta {
|
|
|
|
|
return node.loadDeltaLogs(ctx, req), nil
|
|
|
|
|
}
|
2023-07-18 10:51:19 +08:00
|
|
|
|
if req.GetLoadScope() == querypb.LoadScope_Index {
|
|
|
|
|
return node.loadIndex(ctx, req), nil
|
|
|
|
|
}
|
2023-06-19 17:14:41 +08:00
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
// Actual load segment
|
|
|
|
|
log.Info("start to load segments...")
|
|
|
|
|
loaded, err := node.loader.Load(ctx,
|
|
|
|
|
req.GetCollectionID(),
|
|
|
|
|
segments.SegmentTypeSealed,
|
|
|
|
|
req.GetVersion(),
|
|
|
|
|
req.GetInfos()...,
|
|
|
|
|
)
|
|
|
|
|
if err != nil {
|
2023-09-12 16:07:18 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
2023-07-14 10:28:30 +08:00
|
|
|
|
node.manager.Collection.Ref(req.GetCollectionID(), uint32(len(loaded)))
|
|
|
|
|
|
2023-05-11 15:33:24 +08:00
|
|
|
|
log.Info("load segments done...",
|
2023-03-27 00:42:00 +08:00
|
|
|
|
zap.Int64s("segments", lo.Map(loaded, func(s segments.Segment, _ int) int64 { return s.ID() })))
|
2023-07-14 10:28:30 +08:00
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// ReleaseCollection clears all data related to this collection on the querynode
|
|
|
|
|
func (node *QueryNode) ReleaseCollection(ctx context.Context, in *querypb.ReleaseCollectionRequest) (*commonpb.Status, error) {
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthyOrStopping); err != nil {
|
2023-05-10 09:31:19 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// ReleasePartitions clears all data related to this partition on the querynode
|
2023-05-06 17:42:39 +08:00
|
|
|
|
func (node *QueryNode) ReleasePartitions(ctx context.Context, req *querypb.ReleasePartitionsRequest) (*commonpb.Status, error) {
|
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("collection", req.GetCollectionID()),
|
|
|
|
|
zap.Int64s("partitions", req.GetPartitionIDs()),
|
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
log.Info("received release partitions request")
|
|
|
|
|
|
|
|
|
|
// check node healthy
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-05-10 09:31:19 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
2023-05-06 17:42:39 +08:00
|
|
|
|
collection := node.manager.Collection.Get(req.GetCollectionID())
|
|
|
|
|
if collection != nil {
|
|
|
|
|
for _, partition := range req.GetPartitionIDs() {
|
|
|
|
|
collection.RemovePartition(partition)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
log.Info("release partitions done")
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// ReleaseSegments remove the specified segments from query node according segmentIDs, partitionIDs, and collectionID
|
|
|
|
|
func (node *QueryNode) ReleaseSegments(ctx context.Context, req *querypb.ReleaseSegmentsRequest) (*commonpb.Status, error) {
|
2024-05-17 10:11:37 +08:00
|
|
|
|
defer node.updateDistributionModifyTS()
|
2023-03-27 00:42:00 +08:00
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("collectionID", req.GetCollectionID()),
|
|
|
|
|
zap.String("shard", req.GetShard()),
|
|
|
|
|
zap.Int64s("segmentIDs", req.GetSegmentIDs()),
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.Int64("currentNodeID", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
log.Info("received release segment request",
|
|
|
|
|
zap.String("scope", req.GetScope().String()),
|
|
|
|
|
zap.Bool("needTransfer", req.GetNeedTransfer()),
|
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
// check node healthy
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthyOrStopping); err != nil {
|
2023-05-10 09:31:19 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
|
|
|
|
if req.GetNeedTransfer() {
|
|
|
|
|
delegator, ok := node.delegators.Get(req.GetShard())
|
|
|
|
|
if !ok {
|
|
|
|
|
msg := "failed to release segment, delegator not found"
|
|
|
|
|
log.Warn(msg)
|
2023-09-26 17:15:27 +08:00
|
|
|
|
err := merr.WrapErrChannelNotFound(req.GetShard())
|
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
req.NeedTransfer = false
|
|
|
|
|
err := delegator.ReleaseSegments(ctx, req, false)
|
|
|
|
|
if err != nil {
|
|
|
|
|
log.Warn("delegator failed to release segment", zap.Error(err))
|
2023-09-26 17:15:27 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
log.Info("start to release segments")
|
2023-07-14 10:28:30 +08:00
|
|
|
|
sealedCount := 0
|
2023-03-27 00:42:00 +08:00
|
|
|
|
for _, id := range req.GetSegmentIDs() {
|
2024-05-06 20:29:30 +08:00
|
|
|
|
_, count := node.manager.Segment.Remove(ctx, id, req.GetScope())
|
2023-07-14 10:28:30 +08:00
|
|
|
|
sealedCount += count
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2023-07-14 10:28:30 +08:00
|
|
|
|
node.manager.Collection.Unref(req.GetCollectionID(), uint32(sealedCount))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// GetSegmentInfo returns segment information of the collection on the queryNode, and the information includes memSize, numRow, indexName, indexID ...
|
|
|
|
|
func (node *QueryNode) GetSegmentInfo(ctx context.Context, in *querypb.GetSegmentInfoRequest) (*querypb.GetSegmentInfoResponse, error) {
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-03-27 00:42:00 +08:00
|
|
|
|
return &querypb.GetSegmentInfoResponse{
|
2023-05-10 09:31:19 +08:00
|
|
|
|
Status: merr.Status(err),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
|
|
|
|
var segmentInfos []*querypb.SegmentInfo
|
|
|
|
|
for _, segmentID := range in.GetSegmentIDs() {
|
|
|
|
|
segment := node.manager.Segment.Get(segmentID)
|
|
|
|
|
if segment == nil {
|
|
|
|
|
continue
|
|
|
|
|
}
|
|
|
|
|
collection := node.manager.Collection.Get(segment.Collection())
|
|
|
|
|
if collection == nil {
|
|
|
|
|
continue
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// TODO(yah01): now Milvus supports only 1 vector field
|
|
|
|
|
vecFields := funcutil.GetVecFieldIDs(collection.Schema())
|
|
|
|
|
var (
|
|
|
|
|
indexName string
|
|
|
|
|
indexID int64
|
|
|
|
|
indexInfos []*querypb.FieldIndexInfo
|
|
|
|
|
)
|
|
|
|
|
for _, field := range vecFields {
|
|
|
|
|
index := segment.GetIndex(field)
|
|
|
|
|
if index != nil {
|
|
|
|
|
indexName = index.IndexInfo.GetIndexName()
|
|
|
|
|
indexID = index.IndexInfo.GetIndexID()
|
|
|
|
|
indexInfos = append(indexInfos, index.IndexInfo)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
info := &querypb.SegmentInfo{
|
|
|
|
|
SegmentID: segment.ID(),
|
|
|
|
|
SegmentState: segment.Type(),
|
2024-05-07 19:13:35 +08:00
|
|
|
|
DmChannel: segment.Shard().VirtualName(),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
PartitionID: segment.Partition(),
|
|
|
|
|
CollectionID: segment.Collection(),
|
2024-02-21 11:54:53 +08:00
|
|
|
|
NodeID: node.GetNodeID(),
|
|
|
|
|
NodeIds: []int64{node.GetNodeID()},
|
2023-03-27 00:42:00 +08:00
|
|
|
|
MemSize: segment.MemSize(),
|
|
|
|
|
NumRows: segment.InsertCount(),
|
|
|
|
|
IndexName: indexName,
|
|
|
|
|
IndexID: indexID,
|
|
|
|
|
IndexInfos: indexInfos,
|
|
|
|
|
}
|
|
|
|
|
segmentInfos = append(segmentInfos, info)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return &querypb.GetSegmentInfoResponse{
|
2023-10-11 21:01:35 +08:00
|
|
|
|
Status: merr.Success(),
|
2023-09-04 09:57:09 +08:00
|
|
|
|
Infos: segmentInfos,
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
|
2023-05-23 16:01:26 +08:00
|
|
|
|
// only used for shard delegator search segments from worker
|
|
|
|
|
func (node *QueryNode) SearchSegments(ctx context.Context, req *querypb.SearchRequest) (*internalpb.SearchResults, error) {
|
|
|
|
|
channel := req.GetDmlChannels()[0]
|
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("msgID", req.GetReq().GetBase().GetMsgID()),
|
|
|
|
|
zap.Int64("collectionID", req.Req.GetCollectionID()),
|
|
|
|
|
zap.String("channel", channel),
|
|
|
|
|
zap.String("scope", req.GetScope().String()),
|
|
|
|
|
)
|
2024-01-09 11:38:48 +08:00
|
|
|
|
channelsMvcc := make(map[string]uint64)
|
|
|
|
|
for _, ch := range req.GetDmlChannels() {
|
|
|
|
|
channelsMvcc[ch] = req.GetReq().GetMvccTimestamp()
|
|
|
|
|
}
|
|
|
|
|
resp := &internalpb.SearchResults{
|
|
|
|
|
ChannelsMvcc: channelsMvcc,
|
|
|
|
|
}
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
|
|
|
|
resp.Status = merr.Status(err)
|
2023-09-12 16:07:18 +08:00
|
|
|
|
return resp, nil
|
2023-05-23 16:01:26 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
2024-02-21 11:54:53 +08:00
|
|
|
|
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.SearchLabel, metrics.TotalLabel, metrics.FromLeader).Inc()
|
2023-07-06 08:46:26 +08:00
|
|
|
|
defer func() {
|
2024-01-10 16:46:49 +08:00
|
|
|
|
if !merr.Ok(resp.GetStatus()) {
|
2024-02-21 11:54:53 +08:00
|
|
|
|
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.SearchLabel, metrics.FailLabel, metrics.FromLeader).Inc()
|
2023-07-06 08:46:26 +08:00
|
|
|
|
}
|
|
|
|
|
}()
|
|
|
|
|
|
2023-05-23 16:01:26 +08:00
|
|
|
|
log.Debug("start to search segments on worker",
|
|
|
|
|
zap.Int64s("segmentIDs", req.GetSegmentIDs()),
|
|
|
|
|
)
|
|
|
|
|
searchCtx, cancel := context.WithCancel(ctx)
|
|
|
|
|
defer cancel()
|
|
|
|
|
|
2023-06-15 14:24:38 +08:00
|
|
|
|
tr := timerecord.NewTimeRecorder("searchSegments")
|
|
|
|
|
log.Debug("search segments...")
|
2023-05-23 16:01:26 +08:00
|
|
|
|
|
|
|
|
|
collection := node.manager.Collection.Get(req.Req.GetCollectionID())
|
|
|
|
|
if collection == nil {
|
2023-07-12 19:48:29 +08:00
|
|
|
|
err := merr.WrapErrCollectionNotLoaded(req.GetReq().GetCollectionID())
|
|
|
|
|
log.Warn("failed to search segments", zap.Error(err))
|
2023-09-12 16:07:18 +08:00
|
|
|
|
resp.Status = merr.Status(err)
|
|
|
|
|
return resp, nil
|
2023-05-23 16:01:26 +08:00
|
|
|
|
}
|
|
|
|
|
|
2024-05-06 20:29:30 +08:00
|
|
|
|
var task tasks.Task
|
|
|
|
|
if paramtable.Get().QueryNodeCfg.UseStreamComputing.GetAsBool() {
|
|
|
|
|
task = tasks.NewStreamingSearchTask(searchCtx, collection, node.manager, req, node.serverID)
|
|
|
|
|
} else {
|
|
|
|
|
task = tasks.NewSearchTask(searchCtx, collection, node.manager, req, node.serverID)
|
|
|
|
|
}
|
|
|
|
|
|
2023-07-03 18:24:25 +08:00
|
|
|
|
if err := node.scheduler.Add(task); err != nil {
|
|
|
|
|
log.Warn("failed to search channel", zap.Error(err))
|
2023-09-12 16:07:18 +08:00
|
|
|
|
resp.Status = merr.Status(err)
|
|
|
|
|
return resp, nil
|
2023-05-23 16:01:26 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
err := task.Wait()
|
|
|
|
|
if err != nil {
|
2023-06-15 14:24:38 +08:00
|
|
|
|
log.Warn("failed to search segments", zap.Error(err))
|
2023-09-12 16:07:18 +08:00
|
|
|
|
resp.Status = merr.Status(err)
|
|
|
|
|
return resp, nil
|
2023-05-23 16:01:26 +08:00
|
|
|
|
}
|
|
|
|
|
|
2023-06-15 14:24:38 +08:00
|
|
|
|
tr.CtxElapse(ctx, fmt.Sprintf("search segments done, channel = %s, segmentIDs = %v",
|
2023-05-23 16:01:26 +08:00
|
|
|
|
channel,
|
|
|
|
|
req.GetSegmentIDs(),
|
|
|
|
|
))
|
|
|
|
|
|
|
|
|
|
latency := tr.ElapseSpan()
|
2024-02-21 11:54:53 +08:00
|
|
|
|
metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.SearchLabel, metrics.FromLeader).Observe(float64(latency.Milliseconds()))
|
|
|
|
|
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.SearchLabel, metrics.SuccessLabel, metrics.FromLeader).Inc()
|
2023-06-15 14:24:38 +08:00
|
|
|
|
|
2024-05-06 20:29:30 +08:00
|
|
|
|
resp = task.SearchResult()
|
2023-09-12 16:07:18 +08:00
|
|
|
|
resp.GetCostAggregation().ResponseTime = tr.ElapseSpan().Milliseconds()
|
|
|
|
|
resp.GetCostAggregation().TotalNQ = node.scheduler.GetWaitingTaskTotalNQ()
|
|
|
|
|
return resp, nil
|
2023-05-23 16:01:26 +08:00
|
|
|
|
}
|
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
// Search performs replica search tasks.
|
|
|
|
|
func (node *QueryNode) Search(ctx context.Context, req *querypb.SearchRequest) (*internalpb.SearchResults, error) {
|
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("collectionID", req.GetReq().GetCollectionID()),
|
|
|
|
|
zap.Strings("channels", req.GetDmlChannels()),
|
2023-11-30 10:42:27 +08:00
|
|
|
|
zap.Int64("nq", req.GetReq().GetNq()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
)
|
|
|
|
|
|
2023-06-29 14:04:22 +08:00
|
|
|
|
log.Debug("Received SearchRequest",
|
2023-03-27 00:42:00 +08:00
|
|
|
|
zap.Int64s("segmentIDs", req.GetSegmentIDs()),
|
2024-01-09 11:38:48 +08:00
|
|
|
|
zap.Uint64("guaranteeTimestamp", req.GetReq().GetGuaranteeTimestamp()),
|
|
|
|
|
zap.Uint64("mvccTimestamp", req.GetReq().GetMvccTimestamp()))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
2023-07-25 18:51:01 +08:00
|
|
|
|
tr := timerecord.NewTimeRecorderWithTrace(ctx, "SearchRequest")
|
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-05-10 09:31:19 +08:00
|
|
|
|
return &internalpb.SearchResults{
|
|
|
|
|
Status: merr.Status(err),
|
|
|
|
|
}, nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
2023-11-30 18:34:32 +08:00
|
|
|
|
resp := &internalpb.SearchResults{
|
2023-10-11 21:01:35 +08:00
|
|
|
|
Status: merr.Success(),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2023-06-07 15:58:36 +08:00
|
|
|
|
collection := node.manager.Collection.Get(req.GetReq().GetCollectionID())
|
|
|
|
|
if collection == nil {
|
2023-11-30 18:34:32 +08:00
|
|
|
|
resp.Status = merr.Status(merr.WrapErrCollectionNotFound(req.GetReq().GetCollectionID()))
|
|
|
|
|
return resp, nil
|
2023-06-07 15:58:36 +08:00
|
|
|
|
}
|
|
|
|
|
|
2023-11-30 18:34:32 +08:00
|
|
|
|
toReduceResults := make([]*internalpb.SearchResults, len(req.GetDmlChannels()))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
runningGp, runningCtx := errgroup.WithContext(ctx)
|
2024-01-09 11:38:48 +08:00
|
|
|
|
|
2023-11-30 18:34:32 +08:00
|
|
|
|
for i, ch := range req.GetDmlChannels() {
|
2023-03-27 00:42:00 +08:00
|
|
|
|
ch := ch
|
|
|
|
|
req := &querypb.SearchRequest{
|
|
|
|
|
Req: req.Req,
|
|
|
|
|
DmlChannels: []string{ch},
|
|
|
|
|
SegmentIDs: req.SegmentIDs,
|
|
|
|
|
Scope: req.Scope,
|
2023-08-11 18:35:29 +08:00
|
|
|
|
TotalChannelNum: req.TotalChannelNum,
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2023-04-06 14:32:29 +08:00
|
|
|
|
|
2023-11-30 18:34:32 +08:00
|
|
|
|
i := i
|
2023-03-27 00:42:00 +08:00
|
|
|
|
runningGp.Go(func() error {
|
|
|
|
|
ret, err := node.searchChannel(runningCtx, req, ch)
|
|
|
|
|
if err != nil {
|
|
|
|
|
return err
|
|
|
|
|
}
|
2023-11-30 18:34:32 +08:00
|
|
|
|
if err := merr.Error(ret.GetStatus()); err != nil {
|
|
|
|
|
return err
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2023-11-30 18:34:32 +08:00
|
|
|
|
toReduceResults[i] = ret
|
2023-03-27 00:42:00 +08:00
|
|
|
|
return nil
|
|
|
|
|
})
|
|
|
|
|
}
|
|
|
|
|
if err := runningGp.Wait(); err != nil {
|
2023-11-30 18:34:32 +08:00
|
|
|
|
resp.Status = merr.Status(err)
|
|
|
|
|
return resp, nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2023-04-03 15:24:24 +08:00
|
|
|
|
|
2023-07-25 18:51:01 +08:00
|
|
|
|
tr.RecordSpan()
|
2024-04-09 14:21:18 +08:00
|
|
|
|
var result *internalpb.SearchResults
|
|
|
|
|
var err2 error
|
|
|
|
|
if req.GetReq().GetIsAdvanced() {
|
|
|
|
|
result, err2 = segments.ReduceAdvancedSearchResults(ctx, toReduceResults, req.Req.GetNq())
|
|
|
|
|
} else {
|
|
|
|
|
result, err2 = segments.ReduceSearchResults(ctx, toReduceResults, req.Req.GetNq(), req.Req.GetTopk(), req.Req.GetMetricType())
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if err2 != nil {
|
|
|
|
|
log.Warn("failed to reduce search results", zap.Error(err2))
|
|
|
|
|
resp.Status = merr.Status(err2)
|
2023-11-30 18:34:32 +08:00
|
|
|
|
return resp, nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2024-04-09 14:21:18 +08:00
|
|
|
|
|
2023-07-25 18:51:01 +08:00
|
|
|
|
reduceLatency := tr.RecordSpan()
|
2024-05-06 20:29:30 +08:00
|
|
|
|
metrics.QueryNodeReduceLatency.
|
|
|
|
|
WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.SearchLabel, metrics.ReduceShards, metrics.BatchReduce).
|
2023-07-25 18:51:01 +08:00
|
|
|
|
Observe(float64(reduceLatency.Milliseconds()))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
2023-06-15 14:24:38 +08:00
|
|
|
|
collector.Rate.Add(metricsinfo.NQPerSecond, float64(req.GetReq().GetNq()))
|
|
|
|
|
collector.Rate.Add(metricsinfo.SearchThroughput, float64(proto.Size(req)))
|
2024-02-21 11:54:53 +08:00
|
|
|
|
metrics.QueryNodeExecuteCounter.WithLabelValues(strconv.FormatInt(node.GetNodeID(), 10), metrics.SearchLabel).
|
2023-06-15 14:24:38 +08:00
|
|
|
|
Add(float64(proto.Size(req)))
|
|
|
|
|
|
2023-07-25 18:51:01 +08:00
|
|
|
|
if result.GetCostAggregation() != nil {
|
|
|
|
|
result.GetCostAggregation().ResponseTime = tr.ElapseSpan().Milliseconds()
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2023-06-15 14:24:38 +08:00
|
|
|
|
return result, nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
2023-05-23 16:01:26 +08:00
|
|
|
|
// only used for delegator query segments from worker
|
|
|
|
|
func (node *QueryNode) QuerySegments(ctx context.Context, req *querypb.QueryRequest) (*internalpb.RetrieveResults, error) {
|
2023-09-26 17:15:27 +08:00
|
|
|
|
resp := &internalpb.RetrieveResults{
|
2023-10-11 21:01:35 +08:00
|
|
|
|
Status: merr.Success(),
|
2023-09-12 16:07:18 +08:00
|
|
|
|
}
|
2023-05-23 16:01:26 +08:00
|
|
|
|
msgID := req.Req.Base.GetMsgID()
|
|
|
|
|
traceID := trace.SpanFromContext(ctx).SpanContext().TraceID()
|
|
|
|
|
channel := req.GetDmlChannels()[0]
|
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("msgID", msgID),
|
|
|
|
|
zap.Int64("collectionID", req.GetReq().GetCollectionID()),
|
|
|
|
|
zap.String("channel", channel),
|
|
|
|
|
zap.String("scope", req.GetScope().String()),
|
|
|
|
|
)
|
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-09-26 17:15:27 +08:00
|
|
|
|
resp.Status = merr.Status(err)
|
|
|
|
|
return resp, nil
|
2023-05-23 16:01:26 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
2024-02-21 11:54:53 +08:00
|
|
|
|
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.QueryLabel, metrics.TotalLabel, metrics.FromLeader).Inc()
|
2023-07-06 08:46:26 +08:00
|
|
|
|
defer func() {
|
2023-09-26 17:15:27 +08:00
|
|
|
|
if resp.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
|
2024-02-21 11:54:53 +08:00
|
|
|
|
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.QueryLabel, metrics.FailLabel, metrics.FromLeader).Inc()
|
2023-07-06 08:46:26 +08:00
|
|
|
|
}
|
|
|
|
|
}()
|
|
|
|
|
|
2024-04-10 20:11:19 +08:00
|
|
|
|
log.Debug("start do query segments", zap.Int64s("segmentIDs", req.GetSegmentIDs()))
|
2023-05-23 16:01:26 +08:00
|
|
|
|
// add cancel when error occurs
|
|
|
|
|
queryCtx, cancel := context.WithCancel(ctx)
|
|
|
|
|
defer cancel()
|
|
|
|
|
|
2023-06-15 14:24:38 +08:00
|
|
|
|
tr := timerecord.NewTimeRecorder("querySegments")
|
2023-07-25 18:51:01 +08:00
|
|
|
|
collection := node.manager.Collection.Get(req.Req.GetCollectionID())
|
|
|
|
|
if collection == nil {
|
2023-09-26 17:15:27 +08:00
|
|
|
|
resp.Status = merr.Status(merr.WrapErrCollectionNotLoaded(req.Req.GetCollectionID()))
|
|
|
|
|
return resp, nil
|
2023-07-25 18:51:01 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Send task to scheduler and wait until it finished.
|
|
|
|
|
task := tasks.NewQueryTask(queryCtx, collection, node.manager, req)
|
|
|
|
|
if err := node.scheduler.Add(task); err != nil {
|
|
|
|
|
log.Warn("failed to add query task into scheduler", zap.Error(err))
|
2023-09-26 17:15:27 +08:00
|
|
|
|
resp.Status = merr.Status(err)
|
|
|
|
|
return resp, nil
|
2023-07-25 18:51:01 +08:00
|
|
|
|
}
|
|
|
|
|
err := task.Wait()
|
2023-05-23 16:01:26 +08:00
|
|
|
|
if err != nil {
|
|
|
|
|
log.Warn("failed to query channel", zap.Error(err))
|
2023-09-26 17:15:27 +08:00
|
|
|
|
resp.Status = merr.Status(err)
|
|
|
|
|
return resp, nil
|
2023-05-23 16:01:26 +08:00
|
|
|
|
}
|
|
|
|
|
|
2024-04-10 20:11:19 +08:00
|
|
|
|
tr.CtxElapse(ctx, fmt.Sprintf("do query done, traceID = %s, vChannel = %s, segmentIDs = %v",
|
2023-05-23 16:01:26 +08:00
|
|
|
|
traceID,
|
|
|
|
|
channel,
|
|
|
|
|
req.GetSegmentIDs(),
|
|
|
|
|
))
|
|
|
|
|
|
|
|
|
|
// TODO QueryNodeSQLatencyInQueue QueryNodeReduceLatency
|
|
|
|
|
latency := tr.ElapseSpan()
|
2024-02-21 11:54:53 +08:00
|
|
|
|
metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.QueryLabel, metrics.FromLeader).Observe(float64(latency.Milliseconds()))
|
|
|
|
|
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.QueryLabel, metrics.SuccessLabel, metrics.FromLeader).Inc()
|
2023-07-25 18:51:01 +08:00
|
|
|
|
result := task.Result()
|
|
|
|
|
result.GetCostAggregation().ResponseTime = latency.Milliseconds()
|
|
|
|
|
result.GetCostAggregation().TotalNQ = node.scheduler.GetWaitingTaskTotalNQ()
|
|
|
|
|
return result, nil
|
2023-05-23 16:01:26 +08:00
|
|
|
|
}
|
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
// Query performs replica query tasks.
|
|
|
|
|
func (node *QueryNode) Query(ctx context.Context, req *querypb.QueryRequest) (*internalpb.RetrieveResults, error) {
|
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("collectionID", req.GetReq().GetCollectionID()),
|
|
|
|
|
zap.Strings("shards", req.GetDmlChannels()),
|
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
log.Debug("received query request",
|
|
|
|
|
zap.Int64s("outputFields", req.GetReq().GetOutputFieldsId()),
|
|
|
|
|
zap.Int64s("segmentIDs", req.GetSegmentIDs()),
|
|
|
|
|
zap.Uint64("guaranteeTimestamp", req.GetReq().GetGuaranteeTimestamp()),
|
2023-08-10 18:53:17 +08:00
|
|
|
|
zap.Uint64("mvccTimestamp", req.GetReq().GetMvccTimestamp()),
|
2023-04-11 15:40:31 +08:00
|
|
|
|
zap.Bool("isCount", req.GetReq().GetIsCount()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
)
|
2023-07-25 18:51:01 +08:00
|
|
|
|
tr := timerecord.NewTimeRecorderWithTrace(ctx, "QueryRequest")
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-05-10 09:31:19 +08:00
|
|
|
|
return &internalpb.RetrieveResults{
|
|
|
|
|
Status: merr.Status(err),
|
|
|
|
|
}, nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
|
|
|
|
toMergeResults := make([]*internalpb.RetrieveResults, len(req.GetDmlChannels()))
|
|
|
|
|
runningGp, runningCtx := errgroup.WithContext(ctx)
|
|
|
|
|
|
|
|
|
|
for i, ch := range req.GetDmlChannels() {
|
|
|
|
|
ch := ch
|
|
|
|
|
req := &querypb.QueryRequest{
|
2024-04-10 20:11:19 +08:00
|
|
|
|
Req: req.Req,
|
|
|
|
|
DmlChannels: []string{ch},
|
|
|
|
|
SegmentIDs: req.SegmentIDs,
|
|
|
|
|
Scope: req.Scope,
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
idx := i
|
|
|
|
|
runningGp.Go(func() error {
|
|
|
|
|
ret, err := node.queryChannel(runningCtx, req, ch)
|
2023-09-15 10:09:21 +08:00
|
|
|
|
if err == nil {
|
|
|
|
|
err = merr.Error(ret.GetStatus())
|
|
|
|
|
}
|
2023-03-27 00:42:00 +08:00
|
|
|
|
if err != nil {
|
|
|
|
|
return err
|
|
|
|
|
}
|
|
|
|
|
toMergeResults[idx] = ret
|
|
|
|
|
return nil
|
|
|
|
|
})
|
|
|
|
|
}
|
|
|
|
|
if err := runningGp.Wait(); err != nil {
|
2023-09-12 16:07:18 +08:00
|
|
|
|
return &internalpb.RetrieveResults{
|
|
|
|
|
Status: merr.Status(err),
|
|
|
|
|
}, nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2023-04-11 15:40:31 +08:00
|
|
|
|
|
2023-07-25 18:51:01 +08:00
|
|
|
|
tr.RecordSpan()
|
2023-04-11 15:40:31 +08:00
|
|
|
|
reducer := segments.CreateInternalReducer(req, node.manager.Collection.Get(req.GetReq().GetCollectionID()).Schema())
|
|
|
|
|
ret, err := reducer.Reduce(ctx, toMergeResults)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
if err != nil {
|
2023-09-12 16:07:18 +08:00
|
|
|
|
return &internalpb.RetrieveResults{
|
|
|
|
|
Status: merr.Status(err),
|
|
|
|
|
}, nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2023-07-25 18:51:01 +08:00
|
|
|
|
reduceLatency := tr.RecordSpan()
|
2024-05-06 20:29:30 +08:00
|
|
|
|
metrics.QueryNodeReduceLatency.WithLabelValues(fmt.Sprint(node.GetNodeID()),
|
|
|
|
|
metrics.QueryLabel, metrics.ReduceShards, metrics.BatchReduce).
|
2023-07-25 18:51:01 +08:00
|
|
|
|
Observe(float64(reduceLatency.Milliseconds()))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
2024-04-10 20:11:19 +08:00
|
|
|
|
collector.Rate.Add(metricsinfo.NQPerSecond, 1)
|
|
|
|
|
metrics.QueryNodeExecuteCounter.WithLabelValues(strconv.FormatInt(node.GetNodeID(), 10), metrics.QueryLabel).Add(float64(proto.Size(req)))
|
2024-04-10 15:07:17 +08:00
|
|
|
|
relatedDataSize := lo.Reduce(toMergeResults, func(acc int64, result *internalpb.RetrieveResults, _ int) int64 {
|
|
|
|
|
return acc + result.GetCostAggregation().GetTotalRelatedDataSize()
|
|
|
|
|
}, 0)
|
2023-06-16 18:38:39 +08:00
|
|
|
|
|
2024-04-10 15:07:17 +08:00
|
|
|
|
if ret.CostAggregation == nil {
|
|
|
|
|
ret.CostAggregation = &internalpb.CostAggregation{}
|
2023-06-16 18:38:39 +08:00
|
|
|
|
}
|
2024-04-10 15:07:17 +08:00
|
|
|
|
ret.CostAggregation.ResponseTime = tr.ElapseSpan().Milliseconds()
|
|
|
|
|
ret.CostAggregation.TotalRelatedDataSize = relatedDataSize
|
2023-03-27 00:42:00 +08:00
|
|
|
|
return ret, nil
|
|
|
|
|
}
|
|
|
|
|
|
2023-09-26 09:57:25 +08:00
|
|
|
|
func (node *QueryNode) QueryStream(req *querypb.QueryRequest, srv querypb.QueryNode_QueryStreamServer) error {
|
|
|
|
|
ctx := srv.Context()
|
2023-09-12 10:19:17 +08:00
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("collectionID", req.GetReq().GetCollectionID()),
|
|
|
|
|
zap.Strings("shards", req.GetDmlChannels()),
|
|
|
|
|
)
|
2023-09-26 09:57:25 +08:00
|
|
|
|
concurrentSrv := streamrpc.NewConcurrentQueryStreamServer(srv)
|
2023-09-12 10:19:17 +08:00
|
|
|
|
|
|
|
|
|
log.Debug("received query stream request",
|
|
|
|
|
zap.Int64s("outputFields", req.GetReq().GetOutputFieldsId()),
|
|
|
|
|
zap.Int64s("segmentIDs", req.GetSegmentIDs()),
|
|
|
|
|
zap.Uint64("guaranteeTimestamp", req.GetReq().GetGuaranteeTimestamp()),
|
|
|
|
|
zap.Uint64("mvccTimestamp", req.GetReq().GetMvccTimestamp()),
|
|
|
|
|
zap.Bool("isCount", req.GetReq().GetIsCount()),
|
|
|
|
|
)
|
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-09-26 09:57:25 +08:00
|
|
|
|
concurrentSrv.Send(&internalpb.RetrieveResults{Status: merr.Status(err)})
|
2023-09-12 10:19:17 +08:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
|
|
|
|
runningGp, runningCtx := errgroup.WithContext(ctx)
|
|
|
|
|
|
|
|
|
|
for _, ch := range req.GetDmlChannels() {
|
|
|
|
|
ch := ch
|
|
|
|
|
req := &querypb.QueryRequest{
|
2024-04-10 20:11:19 +08:00
|
|
|
|
Req: req.Req,
|
|
|
|
|
DmlChannels: []string{ch},
|
|
|
|
|
SegmentIDs: req.SegmentIDs,
|
|
|
|
|
Scope: req.Scope,
|
2023-09-12 10:19:17 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
runningGp.Go(func() error {
|
2023-09-26 09:57:25 +08:00
|
|
|
|
err := node.queryChannelStream(runningCtx, req, ch, concurrentSrv)
|
2023-09-12 10:19:17 +08:00
|
|
|
|
if err != nil {
|
|
|
|
|
return err
|
|
|
|
|
}
|
|
|
|
|
return nil
|
|
|
|
|
})
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if err := runningGp.Wait(); err != nil {
|
2023-09-26 09:57:25 +08:00
|
|
|
|
concurrentSrv.Send(&internalpb.RetrieveResults{
|
2023-09-12 16:07:18 +08:00
|
|
|
|
Status: merr.Status(err),
|
|
|
|
|
})
|
2023-09-12 10:19:17 +08:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
collector.Rate.Add(metricsinfo.NQPerSecond, 1)
|
2024-02-21 11:54:53 +08:00
|
|
|
|
metrics.QueryNodeExecuteCounter.WithLabelValues(strconv.FormatInt(node.GetNodeID(), 10), metrics.QueryLabel).Add(float64(proto.Size(req)))
|
2023-09-12 10:19:17 +08:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2023-09-26 09:57:25 +08:00
|
|
|
|
func (node *QueryNode) QueryStreamSegments(req *querypb.QueryRequest, srv querypb.QueryNode_QueryStreamSegmentsServer) error {
|
|
|
|
|
ctx := srv.Context()
|
|
|
|
|
msgID := req.Req.Base.GetMsgID()
|
|
|
|
|
traceID := trace.SpanFromContext(ctx).SpanContext().TraceID()
|
|
|
|
|
channel := req.GetDmlChannels()[0]
|
|
|
|
|
concurrentSrv := streamrpc.NewConcurrentQueryStreamServer(srv)
|
|
|
|
|
|
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("msgID", msgID),
|
|
|
|
|
zap.Int64("collectionID", req.GetReq().GetCollectionID()),
|
|
|
|
|
zap.String("channel", channel),
|
|
|
|
|
zap.String("scope", req.GetScope().String()),
|
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
resp := &internalpb.RetrieveResults{}
|
2024-02-21 11:54:53 +08:00
|
|
|
|
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.QueryLabel, metrics.TotalLabel, metrics.FromLeader).Inc()
|
2023-09-26 09:57:25 +08:00
|
|
|
|
defer func() {
|
|
|
|
|
if resp.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
|
2024-02-21 11:54:53 +08:00
|
|
|
|
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.QueryLabel, metrics.FailLabel, metrics.FromLeader).Inc()
|
2023-09-26 09:57:25 +08:00
|
|
|
|
}
|
|
|
|
|
}()
|
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
|
|
|
|
resp.Status = merr.Status(err)
|
2023-09-26 09:57:25 +08:00
|
|
|
|
concurrentSrv.Send(resp)
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
2024-04-10 20:11:19 +08:00
|
|
|
|
log.Debug("start do query with channel", zap.Int64s("segmentIDs", req.GetSegmentIDs()))
|
2023-09-26 09:57:25 +08:00
|
|
|
|
|
|
|
|
|
tr := timerecord.NewTimeRecorder("queryChannel")
|
|
|
|
|
|
|
|
|
|
err := node.queryStreamSegments(ctx, req, concurrentSrv)
|
|
|
|
|
if err != nil {
|
|
|
|
|
resp.Status = merr.Status(err)
|
|
|
|
|
concurrentSrv.Send(resp)
|
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2024-04-10 20:11:19 +08:00
|
|
|
|
tr.CtxElapse(ctx, fmt.Sprintf("do query done, traceID = %s, vChannel = %s, segmentIDs = %v",
|
2023-09-26 09:57:25 +08:00
|
|
|
|
traceID,
|
|
|
|
|
channel,
|
|
|
|
|
req.GetSegmentIDs(),
|
|
|
|
|
))
|
|
|
|
|
|
|
|
|
|
// TODO QueryNodeSQLatencyInQueue QueryNodeReduceLatency
|
|
|
|
|
latency := tr.ElapseSpan()
|
2024-02-21 11:54:53 +08:00
|
|
|
|
metrics.QueryNodeSQReqLatency.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.QueryLabel, metrics.FromLeader).Observe(float64(latency.Milliseconds()))
|
|
|
|
|
metrics.QueryNodeSQCount.WithLabelValues(fmt.Sprint(node.GetNodeID()), metrics.QueryLabel, metrics.SuccessLabel, metrics.FromLeader).Inc()
|
2023-09-26 09:57:25 +08:00
|
|
|
|
return nil
|
|
|
|
|
}
|
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
// SyncReplicaSegments syncs replica node & segments states
|
|
|
|
|
func (node *QueryNode) SyncReplicaSegments(ctx context.Context, req *querypb.SyncReplicaSegmentsRequest) (*commonpb.Status, error) {
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// ShowConfigurations returns the configurations of queryNode matching req.Pattern
|
|
|
|
|
func (node *QueryNode) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-03-27 00:42:00 +08:00
|
|
|
|
log.Warn("QueryNode.ShowConfigurations failed",
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.Int64("nodeId", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
zap.String("req", req.Pattern),
|
2023-04-26 17:06:34 +08:00
|
|
|
|
zap.Error(err))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
|
|
return &internalpb.ShowConfigurationsResponse{
|
2023-04-26 17:06:34 +08:00
|
|
|
|
Status: merr.Status(err),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
Configuations: nil,
|
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
|
|
|
|
configList := make([]*commonpb.KeyValuePair, 0)
|
|
|
|
|
for key, value := range paramtable.Get().GetComponentConfigurations("querynode", req.Pattern) {
|
|
|
|
|
configList = append(configList,
|
|
|
|
|
&commonpb.KeyValuePair{
|
|
|
|
|
Key: key,
|
|
|
|
|
Value: value,
|
|
|
|
|
})
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return &internalpb.ShowConfigurationsResponse{
|
2023-10-11 21:01:35 +08:00
|
|
|
|
Status: merr.Success(),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
Configuations: configList,
|
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// GetMetrics return system infos of the query node, such as total memory, memory usage, cpu usage ...
|
|
|
|
|
func (node *QueryNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-03-27 00:42:00 +08:00
|
|
|
|
log.Warn("QueryNode.GetMetrics failed",
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.Int64("nodeId", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
zap.String("req", req.Request),
|
2023-04-26 17:06:34 +08:00
|
|
|
|
zap.Error(err))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
2023-04-26 17:06:34 +08:00
|
|
|
|
Status: merr.Status(err),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
Response: "",
|
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
|
|
|
|
metricType, err := metricsinfo.ParseMetricType(req.Request)
|
|
|
|
|
if err != nil {
|
|
|
|
|
log.Warn("QueryNode.GetMetrics failed to parse metric type",
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.Int64("nodeId", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
zap.String("req", req.Request),
|
|
|
|
|
zap.Error(err))
|
|
|
|
|
|
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
2023-09-12 16:07:18 +08:00
|
|
|
|
Status: merr.Status(err),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if metricType == metricsinfo.SystemInfoMetrics {
|
|
|
|
|
queryNodeMetrics, err := getSystemInfoMetrics(ctx, req, node)
|
|
|
|
|
if err != nil {
|
|
|
|
|
log.Warn("QueryNode.GetMetrics failed",
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.Int64("nodeId", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
zap.String("req", req.Request),
|
|
|
|
|
zap.String("metricType", metricType),
|
|
|
|
|
zap.Error(err))
|
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
2023-09-12 16:07:18 +08:00
|
|
|
|
Status: merr.Status(err),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
log.RatedDebug(50, "QueryNode.GetMetrics",
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.Int64("nodeID", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
zap.String("req", req.Request),
|
|
|
|
|
zap.String("metricType", metricType),
|
|
|
|
|
zap.Any("queryNodeMetrics", queryNodeMetrics))
|
|
|
|
|
|
|
|
|
|
return queryNodeMetrics, nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
log.Debug("QueryNode.GetMetrics failed, request metric type is not implemented yet",
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.Int64("nodeID", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
zap.String("req", req.Request),
|
|
|
|
|
zap.String("metricType", metricType))
|
|
|
|
|
|
|
|
|
|
return &milvuspb.GetMetricsResponse{
|
2023-09-12 16:07:18 +08:00
|
|
|
|
Status: merr.Status(merr.WrapErrMetricNotFound(metricType)),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (node *QueryNode) GetDataDistribution(ctx context.Context, req *querypb.GetDataDistributionRequest) (*querypb.GetDataDistributionResponse, error) {
|
2023-10-26 19:14:10 +08:00
|
|
|
|
log := log.Ctx(ctx).With(
|
2023-03-27 00:42:00 +08:00
|
|
|
|
zap.Int64("msgID", req.GetBase().GetMsgID()),
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.Int64("nodeID", node.GetNodeID()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
)
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
|
|
|
|
log.Warn("QueryNode.GetDataDistribution failed",
|
2023-04-26 17:06:34 +08:00
|
|
|
|
zap.Error(err))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
|
|
|
|
return &querypb.GetDataDistributionResponse{
|
2023-04-26 17:06:34 +08:00
|
|
|
|
Status: merr.Status(err),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
2024-05-17 10:11:37 +08:00
|
|
|
|
lastModifyTs := node.getDistributionModifyTS()
|
|
|
|
|
distributionChange := func() bool {
|
|
|
|
|
if req.GetLastUpdateTs() == 0 {
|
|
|
|
|
return true
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
return req.GetLastUpdateTs() < lastModifyTs
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
if !distributionChange() {
|
|
|
|
|
return &querypb.GetDataDistributionResponse{
|
|
|
|
|
Status: merr.Success(),
|
|
|
|
|
NodeID: node.GetNodeID(),
|
|
|
|
|
LastModifyTs: lastModifyTs,
|
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
|
2023-03-27 00:42:00 +08:00
|
|
|
|
sealedSegments := node.manager.Segment.GetBy(segments.WithType(commonpb.SegmentState_Sealed))
|
|
|
|
|
segmentVersionInfos := make([]*querypb.SegmentVersionInfo, 0, len(sealedSegments))
|
|
|
|
|
for _, s := range sealedSegments {
|
|
|
|
|
segmentVersionInfos = append(segmentVersionInfos, &querypb.SegmentVersionInfo{
|
|
|
|
|
ID: s.ID(),
|
|
|
|
|
Collection: s.Collection(),
|
|
|
|
|
Partition: s.Partition(),
|
2024-05-07 19:13:35 +08:00
|
|
|
|
Channel: s.Shard().VirtualName(),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
Version: s.Version(),
|
|
|
|
|
LastDeltaTimestamp: s.LastDeltaTimestamp(),
|
2023-07-11 11:22:29 +08:00
|
|
|
|
IndexInfo: lo.SliceToMap(s.Indexes(), func(info *segments.IndexedFieldInfo) (int64, *querypb.FieldIndexInfo) {
|
|
|
|
|
return info.IndexInfo.FieldID, info.IndexInfo
|
|
|
|
|
}),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
})
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
channelVersionInfos := make([]*querypb.ChannelVersionInfo, 0)
|
|
|
|
|
leaderViews := make([]*querypb.LeaderView, 0)
|
|
|
|
|
|
2023-09-06 12:57:15 +08:00
|
|
|
|
node.delegators.Range(func(key string, delegator delegator.ShardDelegator) bool {
|
|
|
|
|
if !delegator.Serviceable() {
|
2023-03-27 00:42:00 +08:00
|
|
|
|
return true
|
|
|
|
|
}
|
|
|
|
|
channelVersionInfos = append(channelVersionInfos, &querypb.ChannelVersionInfo{
|
|
|
|
|
Channel: key,
|
2023-09-06 12:57:15 +08:00
|
|
|
|
Collection: delegator.Collection(),
|
|
|
|
|
Version: delegator.Version(),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
})
|
|
|
|
|
|
2023-09-06 12:57:15 +08:00
|
|
|
|
sealed, growing := delegator.GetSegmentInfo(false)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
sealedSegments := make(map[int64]*querypb.SegmentDist)
|
|
|
|
|
for _, item := range sealed {
|
|
|
|
|
for _, segment := range item.Segments {
|
|
|
|
|
sealedSegments[segment.SegmentID] = &querypb.SegmentDist{
|
|
|
|
|
NodeID: item.NodeID,
|
|
|
|
|
Version: segment.Version,
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2023-11-27 14:58:26 +08:00
|
|
|
|
numOfGrowingRows := int64(0)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
growingSegments := make(map[int64]*msgpb.MsgPosition)
|
|
|
|
|
for _, entry := range growing {
|
2023-05-30 12:25:28 +08:00
|
|
|
|
segment := node.manager.Segment.GetWithType(entry.SegmentID, segments.SegmentTypeGrowing)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
if segment == nil {
|
|
|
|
|
log.Warn("leader view growing not found", zap.String("channel", key), zap.Int64("segmentID", entry.SegmentID))
|
|
|
|
|
growingSegments[entry.SegmentID] = &msgpb.MsgPosition{}
|
2023-07-07 18:26:25 +08:00
|
|
|
|
continue
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
growingSegments[entry.SegmentID] = segment.StartPosition()
|
2023-11-27 14:58:26 +08:00
|
|
|
|
numOfGrowingRows += segment.InsertCount()
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
leaderViews = append(leaderViews, &querypb.LeaderView{
|
2024-06-10 21:34:08 +08:00
|
|
|
|
Collection: delegator.Collection(),
|
|
|
|
|
Channel: key,
|
|
|
|
|
SegmentDist: sealedSegments,
|
|
|
|
|
GrowingSegments: growingSegments,
|
|
|
|
|
TargetVersion: delegator.GetTargetVersion(),
|
|
|
|
|
NumOfGrowingRows: numOfGrowingRows,
|
|
|
|
|
PartitionStatsVersions: delegator.GetPartitionStatsVersions(ctx),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
})
|
|
|
|
|
return true
|
|
|
|
|
})
|
|
|
|
|
|
|
|
|
|
return &querypb.GetDataDistributionResponse{
|
2024-05-17 10:11:37 +08:00
|
|
|
|
Status: merr.Success(),
|
|
|
|
|
NodeID: node.GetNodeID(),
|
|
|
|
|
Segments: segmentVersionInfos,
|
|
|
|
|
Channels: channelVersionInfos,
|
|
|
|
|
LeaderViews: leaderViews,
|
|
|
|
|
LastModifyTs: lastModifyTs,
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}, nil
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (node *QueryNode) SyncDistribution(ctx context.Context, req *querypb.SyncDistributionRequest) (*commonpb.Status, error) {
|
2024-05-17 10:11:37 +08:00
|
|
|
|
defer node.updateDistributionModifyTS()
|
|
|
|
|
|
2023-08-31 12:03:00 +08:00
|
|
|
|
log := log.Ctx(ctx).With(zap.Int64("collectionID", req.GetCollectionID()),
|
2024-02-21 11:54:53 +08:00
|
|
|
|
zap.String("channel", req.GetChannel()), zap.Int64("currentNodeID", node.GetNodeID()))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
// check node healthy
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-05-10 09:31:19 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
2023-04-07 19:32:29 +08:00
|
|
|
|
// get shard delegator
|
2023-03-27 00:42:00 +08:00
|
|
|
|
shardDelegator, ok := node.delegators.Get(req.GetChannel())
|
|
|
|
|
if !ok {
|
2023-09-12 16:07:18 +08:00
|
|
|
|
err := merr.WrapErrChannelNotFound(req.GetChannel())
|
2023-08-31 12:03:00 +08:00
|
|
|
|
log.Warn("failed to find shard cluster when sync")
|
2023-09-12 16:07:18 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
2023-07-03 18:24:25 +08:00
|
|
|
|
// translate segment action
|
2023-03-29 14:06:02 +08:00
|
|
|
|
removeActions := make([]*querypb.SyncAction, 0)
|
2023-11-03 19:02:18 +08:00
|
|
|
|
group, ctx := errgroup.WithContext(ctx)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
for _, action := range req.GetActions() {
|
|
|
|
|
log := log.With(zap.String("Action",
|
2023-10-11 06:51:33 +08:00
|
|
|
|
action.GetType().String()))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
switch action.GetType() {
|
|
|
|
|
case querypb.SyncType_Remove:
|
2023-10-11 06:51:33 +08:00
|
|
|
|
log.Info("sync action", zap.Int64("segmentID", action.SegmentID))
|
2023-03-29 14:06:02 +08:00
|
|
|
|
removeActions = append(removeActions, action)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
case querypb.SyncType_Set:
|
2023-10-11 06:51:33 +08:00
|
|
|
|
log.Info("sync action", zap.Int64("segmentID", action.SegmentID))
|
2023-09-13 09:35:18 +08:00
|
|
|
|
if action.GetInfo() == nil {
|
|
|
|
|
log.Warn("sync request from legacy querycoord without load info, skip")
|
|
|
|
|
continue
|
|
|
|
|
}
|
2023-11-03 19:02:18 +08:00
|
|
|
|
|
|
|
|
|
// to pass segment'version, we call load segment one by one
|
|
|
|
|
action := action
|
|
|
|
|
group.Go(func() error {
|
|
|
|
|
return shardDelegator.LoadSegments(ctx, &querypb.LoadSegmentsRequest{
|
|
|
|
|
Base: commonpbutil.NewMsgBase(
|
|
|
|
|
commonpbutil.WithMsgType(commonpb.MsgType_LoadSegments),
|
|
|
|
|
commonpbutil.WithMsgID(req.Base.GetMsgID()),
|
|
|
|
|
),
|
2023-12-04 18:14:34 +08:00
|
|
|
|
Infos: []*querypb.SegmentLoadInfo{action.GetInfo()},
|
|
|
|
|
Schema: req.GetSchema(),
|
|
|
|
|
LoadMeta: req.GetLoadMeta(),
|
|
|
|
|
CollectionID: req.GetCollectionID(),
|
|
|
|
|
ReplicaID: req.GetReplicaID(),
|
|
|
|
|
DstNodeID: action.GetNodeID(),
|
|
|
|
|
Version: action.GetVersion(),
|
|
|
|
|
NeedTransfer: false,
|
|
|
|
|
LoadScope: querypb.LoadScope_Delta,
|
|
|
|
|
IndexInfoList: req.GetIndexInfoList(),
|
2023-11-03 19:02:18 +08:00
|
|
|
|
})
|
|
|
|
|
})
|
2023-06-27 11:48:45 +08:00
|
|
|
|
case querypb.SyncType_UpdateVersion:
|
2023-10-11 06:51:33 +08:00
|
|
|
|
log.Info("sync action", zap.Int64("TargetVersion", action.GetTargetVersion()))
|
2024-04-10 15:29:17 +08:00
|
|
|
|
droppedInfos := lo.SliceToMap(action.GetDroppedInTarget(), func(id int64) (int64, uint64) {
|
2024-04-12 10:39:19 +08:00
|
|
|
|
if action.GetCheckpoint() == nil {
|
|
|
|
|
return id, typeutil.MaxTimestamp
|
|
|
|
|
}
|
|
|
|
|
return id, action.GetCheckpoint().Timestamp
|
2024-04-10 15:29:17 +08:00
|
|
|
|
})
|
|
|
|
|
shardDelegator.AddExcludedSegments(droppedInfos)
|
2023-07-24 14:09:00 +08:00
|
|
|
|
shardDelegator.SyncTargetVersion(action.GetTargetVersion(), action.GetGrowingInTarget(),
|
2024-01-04 17:02:46 +08:00
|
|
|
|
action.GetSealedInTarget(), action.GetDroppedInTarget(), action.GetCheckpoint())
|
2024-06-10 21:34:08 +08:00
|
|
|
|
case querypb.SyncType_UpdatePartitionStats:
|
|
|
|
|
log.Info("sync update partition stats versions")
|
|
|
|
|
shardDelegator.SyncPartitionStats(ctx, action.PartitionStatsVersions)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
default:
|
2023-09-12 16:07:18 +08:00
|
|
|
|
return merr.Status(merr.WrapErrServiceInternal("unknown action type", action.GetType().String())), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2023-11-03 19:02:18 +08:00
|
|
|
|
err := group.Wait()
|
|
|
|
|
if err != nil {
|
|
|
|
|
log.Warn("failed to sync distribution", zap.Error(err))
|
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
2024-04-01 10:39:12 +08:00
|
|
|
|
// in case of target node offline, when try to remove segment from leader's distribution, use wildcardNodeID(-1) to skip nodeID check
|
2023-03-29 14:06:02 +08:00
|
|
|
|
for _, action := range removeActions {
|
|
|
|
|
shardDelegator.ReleaseSegments(ctx, &querypb.ReleaseSegmentsRequest{
|
2024-04-01 10:39:12 +08:00
|
|
|
|
NodeID: -1,
|
2023-03-31 11:28:23 +08:00
|
|
|
|
SegmentIDs: []int64{action.GetSegmentID()},
|
|
|
|
|
Scope: querypb.DataScope_Historical,
|
|
|
|
|
CollectionID: req.GetCollectionID(),
|
2023-03-29 14:06:02 +08:00
|
|
|
|
}, true)
|
|
|
|
|
}
|
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Delete is used to forward delete message between delegator and workers.
|
|
|
|
|
func (node *QueryNode) Delete(ctx context.Context, req *querypb.DeleteRequest) (*commonpb.Status, error) {
|
|
|
|
|
log := log.Ctx(ctx).With(
|
|
|
|
|
zap.Int64("collectionID", req.GetCollectionId()),
|
|
|
|
|
zap.String("channel", req.GetVchannelName()),
|
|
|
|
|
zap.Int64("segmentID", req.GetSegmentId()),
|
2023-12-26 14:28:47 +08:00
|
|
|
|
zap.String("scope", req.GetScope().String()),
|
2023-03-27 00:42:00 +08:00
|
|
|
|
)
|
|
|
|
|
|
|
|
|
|
// check node healthy
|
2023-10-11 21:01:35 +08:00
|
|
|
|
if err := node.lifetime.Add(merr.IsHealthy); err != nil {
|
2023-05-10 09:31:19 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
defer node.lifetime.Done()
|
|
|
|
|
|
|
|
|
|
log.Info("QueryNode received worker delete request")
|
2024-01-12 14:40:59 +08:00
|
|
|
|
log.Debug("Worker delete detail", zap.Stringer("info", &deleteRequestStringer{DeleteRequest: req}))
|
2023-03-27 00:42:00 +08:00
|
|
|
|
|
2023-12-26 14:28:47 +08:00
|
|
|
|
filters := []segments.SegmentFilter{
|
|
|
|
|
segments.WithID(req.GetSegmentId()),
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// do not add filter for Unknown & All scope, for backward cap
|
|
|
|
|
switch req.GetScope() {
|
|
|
|
|
case querypb.DataScope_Historical:
|
|
|
|
|
filters = append(filters, segments.WithType(segments.SegmentTypeSealed))
|
|
|
|
|
case querypb.DataScope_Streaming:
|
|
|
|
|
filters = append(filters, segments.WithType(segments.SegmentTypeGrowing))
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
segments := node.manager.Segment.GetBy(filters...)
|
2023-03-27 00:42:00 +08:00
|
|
|
|
if len(segments) == 0 {
|
|
|
|
|
err := merr.WrapErrSegmentNotFound(req.GetSegmentId())
|
|
|
|
|
log.Warn("segment not found for delete")
|
2023-09-26 22:09:28 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
pks := storage.ParseIDs2PrimaryKeys(req.GetPrimaryKeys())
|
|
|
|
|
for _, segment := range segments {
|
2023-12-27 16:10:47 +08:00
|
|
|
|
err := segment.Delete(ctx, pks, req.GetTimestamps())
|
2023-03-27 00:42:00 +08:00
|
|
|
|
if err != nil {
|
|
|
|
|
log.Warn("segment delete failed", zap.Error(err))
|
2023-09-12 16:07:18 +08:00
|
|
|
|
return merr.Status(err), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2023-10-11 21:01:35 +08:00
|
|
|
|
return merr.Success(), nil
|
2023-03-27 00:42:00 +08:00
|
|
|
|
}
|
2024-01-12 14:40:59 +08:00
|
|
|
|
|
|
|
|
|
type deleteRequestStringer struct {
|
|
|
|
|
*querypb.DeleteRequest
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (req *deleteRequestStringer) String() string {
|
|
|
|
|
var pkInfo string
|
|
|
|
|
switch {
|
|
|
|
|
case req.GetPrimaryKeys().GetIntId() != nil:
|
|
|
|
|
ids := req.GetPrimaryKeys().GetIntId().GetData()
|
|
|
|
|
pkInfo = fmt.Sprintf("Pks range[%d-%d], len: %d", ids[0], ids[len(ids)-1], len(ids))
|
|
|
|
|
case req.GetPrimaryKeys().GetStrId() != nil:
|
|
|
|
|
ids := req.GetPrimaryKeys().GetStrId().GetData()
|
|
|
|
|
pkInfo = fmt.Sprintf("Pks range[%s-%s], len: %d", ids[0], ids[len(ids)-1], len(ids))
|
|
|
|
|
}
|
|
|
|
|
tss := req.GetTimestamps()
|
|
|
|
|
return fmt.Sprintf("%s, timestamp range: [%d-%d]", pkInfo, tss[0], tss[len(tss)-1])
|
|
|
|
|
}
|
2024-05-17 10:11:37 +08:00
|
|
|
|
|
|
|
|
|
func (node *QueryNode) updateDistributionModifyTS() {
|
|
|
|
|
node.lastModifyLock.Lock()
|
|
|
|
|
defer node.lastModifyLock.Unlock()
|
|
|
|
|
|
|
|
|
|
node.lastModifyTs = time.Now().UnixNano()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
func (node *QueryNode) getDistributionModifyTS() int64 {
|
|
|
|
|
node.lastModifyLock.RLock()
|
|
|
|
|
defer node.lastModifyLock.RUnlock()
|
|
|
|
|
return node.lastModifyTs
|
|
|
|
|
}
|