mirror of https://github.com/milvus-io/milvus.git
enhance: refine exists log print with ctx (#38080)
issue: #35917 Refines exists log print with ctx Signed-off-by: tinswzy <zhenyuan.wei@zilliz.com>pull/38439/head
parent
1e274384cd
commit
27229f7907
|
@ -58,10 +58,10 @@ func (s *DataCoord) Prepare() error {
|
|||
// Run starts service
|
||||
func (s *DataCoord) Run() error {
|
||||
if err := s.svr.Run(); err != nil {
|
||||
log.Error("DataCoord starts error", zap.Error(err))
|
||||
log.Ctx(s.ctx).Error("DataCoord starts error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Debug("DataCoord successfully started")
|
||||
log.Ctx(s.ctx).Debug("DataCoord successfully started")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -57,10 +57,10 @@ func (d *DataNode) Prepare() error {
|
|||
// Run starts service
|
||||
func (d *DataNode) Run() error {
|
||||
if err := d.svr.Run(); err != nil {
|
||||
log.Error("DataNode starts error", zap.Error(err))
|
||||
log.Ctx(d.ctx).Error("DataNode starts error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Debug("Datanode successfully started")
|
||||
log.Ctx(d.ctx).Info("Datanode successfully started")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -39,13 +39,13 @@ func (s *IndexCoord) Prepare() error {
|
|||
|
||||
// Run starts service
|
||||
func (s *IndexCoord) Run() error {
|
||||
log.Info("IndexCoord running ...")
|
||||
log.Ctx(context.TODO()).Info("IndexCoord running ...")
|
||||
return nil
|
||||
}
|
||||
|
||||
// Stop terminates service
|
||||
func (s *IndexCoord) Stop() error {
|
||||
log.Info("IndexCoord stopping ...")
|
||||
log.Ctx(context.TODO()).Info("IndexCoord stopping ...")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -33,13 +33,16 @@ import (
|
|||
|
||||
// IndexNode implements IndexNode grpc server
|
||||
type IndexNode struct {
|
||||
ctx context.Context
|
||||
svr *grpcindexnode.Server
|
||||
}
|
||||
|
||||
// NewIndexNode creates a new IndexNode
|
||||
func NewIndexNode(ctx context.Context, factory dependency.Factory) (*IndexNode, error) {
|
||||
var err error
|
||||
n := &IndexNode{}
|
||||
n := &IndexNode{
|
||||
ctx: ctx,
|
||||
}
|
||||
svr, err := grpcindexnode.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
|
@ -55,10 +58,10 @@ func (n *IndexNode) Prepare() error {
|
|||
// Run starts service
|
||||
func (n *IndexNode) Run() error {
|
||||
if err := n.svr.Run(); err != nil {
|
||||
log.Error("IndexNode starts error", zap.Error(err))
|
||||
log.Ctx(n.ctx).Error("IndexNode starts error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Debug("IndexNode successfully started")
|
||||
log.Ctx(n.ctx).Info("IndexNode successfully started")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -58,10 +58,10 @@ func (n *Proxy) Prepare() error {
|
|||
// Run starts service
|
||||
func (n *Proxy) Run() error {
|
||||
if err := n.svr.Run(); err != nil {
|
||||
log.Error("Proxy starts error", zap.Error(err))
|
||||
log.Ctx(context.TODO()).Error("Proxy starts error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Info("Proxy successfully started")
|
||||
log.Ctx(context.TODO()).Info("Proxy successfully started")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -57,10 +57,10 @@ func (qs *QueryCoord) Prepare() error {
|
|||
// Run starts service
|
||||
func (qs *QueryCoord) Run() error {
|
||||
if err := qs.svr.Run(); err != nil {
|
||||
log.Error("QueryCoord starts error", zap.Error(err))
|
||||
log.Ctx(qs.ctx).Error("QueryCoord starts error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Debug("QueryCoord successfully started")
|
||||
log.Ctx(qs.ctx).Info("QueryCoord successfully started")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -57,10 +57,10 @@ func (q *QueryNode) Prepare() error {
|
|||
// Run starts service
|
||||
func (q *QueryNode) Run() error {
|
||||
if err := q.svr.Run(); err != nil {
|
||||
log.Error("QueryNode starts error", zap.Error(err))
|
||||
log.Ctx(q.ctx).Error("QueryNode starts error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Debug("QueryNode successfully started")
|
||||
log.Ctx(q.ctx).Info("QueryNode successfully started")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -56,10 +56,10 @@ func (rc *RootCoord) Prepare() error {
|
|||
// Run starts service
|
||||
func (rc *RootCoord) Run() error {
|
||||
if err := rc.svr.Run(); err != nil {
|
||||
log.Error("RootCoord starts error", zap.Error(err))
|
||||
log.Ctx(rc.ctx).Error("RootCoord starts error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Info("RootCoord successfully started")
|
||||
log.Ctx(rc.ctx).Info("RootCoord successfully started")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -29,8 +29,8 @@ type StreamingNode struct {
|
|||
}
|
||||
|
||||
// NewStreamingNode creates a new StreamingNode
|
||||
func NewStreamingNode(_ context.Context, factory dependency.Factory) (*StreamingNode, error) {
|
||||
svr, err := streamingnode.NewServer(factory)
|
||||
func NewStreamingNode(ctx context.Context, factory dependency.Factory) (*StreamingNode, error) {
|
||||
svr, err := streamingnode.NewServer(ctx, factory)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
|
|
@ -106,6 +106,7 @@ func (c *mck) execute(args []string, flags *flag.FlagSet) {
|
|||
|
||||
func (c *mck) run() {
|
||||
c.connectMinio()
|
||||
log := log.Ctx(context.TODO())
|
||||
|
||||
_, values, err := c.metaKV.LoadWithPrefix(context.TODO(), segmentPrefix)
|
||||
if err != nil {
|
||||
|
@ -206,13 +207,14 @@ func (c *mck) formatFlags(args []string, flags *flag.FlagSet) {
|
|||
if err := flags.Parse(os.Args[2:]); err != nil {
|
||||
log.Fatal("failed to parse flags", zap.Error(err))
|
||||
}
|
||||
log.Info("args", zap.Strings("args", args))
|
||||
log.Ctx(context.TODO()).Info("args", zap.Strings("args", args))
|
||||
}
|
||||
|
||||
func (c *mck) connectEctd() {
|
||||
c.params.Init(paramtable.NewBaseTable())
|
||||
var etcdCli *clientv3.Client
|
||||
var err error
|
||||
log := log.Ctx(context.TODO())
|
||||
if c.etcdIP != "" {
|
||||
etcdCli, err = etcd.GetRemoteEtcdClient([]string{c.etcdIP})
|
||||
} else {
|
||||
|
@ -243,7 +245,7 @@ func (c *mck) connectMinio() {
|
|||
var err error
|
||||
c.minioChunkManager, err = chunkManagerFactory.NewPersistentStorageChunkManager(context.Background())
|
||||
if err != nil {
|
||||
log.Fatal("failed to connect to minio", zap.Error(err))
|
||||
log.Ctx(context.TODO()).Fatal("failed to connect to minio", zap.Error(err))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -254,11 +256,12 @@ func getConfigValue(a string, b string, name string) string {
|
|||
if b != "" {
|
||||
return b
|
||||
}
|
||||
log.Panic(fmt.Sprintf("the config '%s' is empty", name))
|
||||
log.Ctx(context.TODO()).Panic(fmt.Sprintf("the config '%s' is empty", name))
|
||||
return ""
|
||||
}
|
||||
|
||||
func (c *mck) cleanTrash() {
|
||||
log := log.Ctx(context.TODO())
|
||||
keys, _, err := c.metaKV.LoadWithPrefix(context.TODO(), MckTrash)
|
||||
if err != nil {
|
||||
log.Error("failed to load backup info", zap.Error(err))
|
||||
|
@ -367,6 +370,7 @@ func getTrashKey(taskType, key string) string {
|
|||
}
|
||||
|
||||
func (c *mck) extractTask(prefix string, keys []string, values []string) {
|
||||
log := log.Ctx(context.TODO())
|
||||
for i := range keys {
|
||||
taskID, err := strconv.ParseInt(filepath.Base(keys[i]), 10, 64)
|
||||
if err != nil {
|
||||
|
@ -393,6 +397,7 @@ func (c *mck) extractTask(prefix string, keys []string, values []string) {
|
|||
}
|
||||
|
||||
func (c *mck) removeTask(invalidTask int64) bool {
|
||||
log := log.Ctx(context.TODO())
|
||||
taskType := c.taskNameMap[invalidTask]
|
||||
key := c.taskKeyMap[invalidTask]
|
||||
err := c.metaKV.Save(context.TODO(), getTrashKey(taskType, key), c.allTaskInfo[key])
|
||||
|
@ -520,6 +525,7 @@ func (c *mck) extractVecFieldIndexInfo(taskID int64, infos []*querypb.FieldIndex
|
|||
|
||||
// return partitionIDs,segmentIDs,error
|
||||
func (c *mck) unmarshalTask(taskID int64, t string) (string, []int64, []int64, error) {
|
||||
log := log.Ctx(context.TODO())
|
||||
header := commonpb.MsgHeader{}
|
||||
err := proto.Unmarshal([]byte(t), &header)
|
||||
if err != nil {
|
||||
|
|
|
@ -242,7 +242,7 @@ func CleanSession(metaPath string, etcdEndpoints []string, sessionSuffix []strin
|
|||
for _, key := range keys {
|
||||
_, _ = etcdCli.Delete(ctx, key)
|
||||
}
|
||||
log.Info("clean sessions from etcd", zap.Any("keys", keys))
|
||||
log.Ctx(ctx).Info("clean sessions from etcd", zap.Any("keys", keys))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -259,6 +259,7 @@ func getSessionPaths(ctx context.Context, client *clientv3.Client, metaPath stri
|
|||
|
||||
// filterUnmatchedKey skip active keys that don't match completed key, the latest active key may from standby server
|
||||
func addActiveKeySuffix(ctx context.Context, client *clientv3.Client, sessionPathPrefix string, sessionSuffix []string) []string {
|
||||
log := log.Ctx(ctx)
|
||||
suffixSet := lo.SliceToMap(sessionSuffix, func(t string) (string, struct{}) {
|
||||
return t, struct{}{}
|
||||
})
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package main
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/csv"
|
||||
"fmt"
|
||||
"io"
|
||||
|
@ -59,7 +60,7 @@ func collectRecursive(params *paramtable.ComponentParam, data *[]DocContent, val
|
|||
if val.Kind() != reflect.Struct {
|
||||
return
|
||||
}
|
||||
log.Debug("enter", zap.Any("variable", val.String()))
|
||||
log.Ctx(context.TODO()).Debug("enter", zap.Any("variable", val.String()))
|
||||
for j := 0; j < val.NumField(); j++ {
|
||||
subVal := val.Field(j)
|
||||
tag := val.Type().Field(j).Tag
|
||||
|
@ -71,11 +72,11 @@ func collectRecursive(params *paramtable.ComponentParam, data *[]DocContent, val
|
|||
if strings.HasPrefix(item.DefaultValue, "\"") && strings.HasSuffix(item.DefaultValue, "\"") {
|
||||
defaultValue = fmt.Sprintf("\"%s\"", defaultValue)
|
||||
}
|
||||
log.Debug("got key", zap.String("key", item.Key), zap.Any("value", defaultValue), zap.String("variable", val.Type().Field(j).Name))
|
||||
log.Ctx(context.TODO()).Debug("got key", zap.String("key", item.Key), zap.Any("value", defaultValue), zap.String("variable", val.Type().Field(j).Name))
|
||||
*data = append(*data, DocContent{item.Key, defaultValue, item.Version, refreshable, item.Export, item.Doc})
|
||||
} else if t == "paramtable.ParamGroup" {
|
||||
item := subVal.Interface().(paramtable.ParamGroup)
|
||||
log.Debug("got key", zap.String("key", item.KeyPrefix), zap.String("variable", val.Type().Field(j).Name))
|
||||
log.Ctx(context.TODO()).Debug("got key", zap.String("key", item.KeyPrefix), zap.String("variable", val.Type().Field(j).Name))
|
||||
refreshable := tag.Get("refreshable")
|
||||
|
||||
// Sort group items to stablize the output order
|
||||
|
@ -87,7 +88,7 @@ func collectRecursive(params *paramtable.ComponentParam, data *[]DocContent, val
|
|||
sort.Strings(keys)
|
||||
for _, key := range keys {
|
||||
value := m[key]
|
||||
log.Debug("got group entry", zap.String("key", key), zap.String("value", value))
|
||||
log.Ctx(context.TODO()).Debug("got group entry", zap.String("key", key), zap.String("value", value))
|
||||
*data = append(*data, DocContent{fmt.Sprintf("%s%s", item.KeyPrefix, key), quoteIfNeeded(value), item.Version, refreshable, item.Export, item.GetDoc(key)})
|
||||
}
|
||||
} else {
|
||||
|
@ -148,7 +149,7 @@ func (m *YamlMarshaller) writeYamlRecursive(data []DocContent, level int) {
|
|||
for _, key := range keys {
|
||||
contents, ok := topLevels.Get(key)
|
||||
if !ok {
|
||||
log.Debug("didnot found config for " + key)
|
||||
log.Ctx(context.TODO()).Debug("didnot found config for " + key)
|
||||
continue
|
||||
}
|
||||
content := contents[0]
|
||||
|
|
|
@ -136,7 +136,7 @@ func prepareRootCoordMeta(ctx context.Context, allocator tso.Allocator) rootcoor
|
|||
}
|
||||
catalog = &kvmetestore.Catalog{Txn: metaKV, Snapshot: ss}
|
||||
case util.MetaStoreTypeTiKV:
|
||||
log.Info("Using tikv as meta storage.")
|
||||
log.Ctx(ctx).Info("Using tikv as meta storage.")
|
||||
var metaKV kv.MetaKv
|
||||
var ss *kvmetestore.SuffixSnapshot
|
||||
var err error
|
||||
|
|
|
@ -66,7 +66,7 @@ func RegisterQueryCoordServer(server querypb.QueryCoordServer) {
|
|||
}
|
||||
newLocalClient := grpcclient.NewLocalGRPCClient(&querypb.QueryCoord_ServiceDesc, server, querypb.NewQueryCoordClient)
|
||||
glocalClient.queryCoordClient.Set(&nopCloseQueryCoordClient{newLocalClient})
|
||||
log.Info("register query coord server", zap.Any("enableLocalClient", enableLocal))
|
||||
log.Ctx(context.TODO()).Info("register query coord server", zap.Any("enableLocalClient", enableLocal))
|
||||
}
|
||||
|
||||
// RegsterDataCoordServer register data coord server
|
||||
|
@ -76,7 +76,7 @@ func RegisterDataCoordServer(server datapb.DataCoordServer) {
|
|||
}
|
||||
newLocalClient := grpcclient.NewLocalGRPCClient(&datapb.DataCoord_ServiceDesc, server, datapb.NewDataCoordClient)
|
||||
glocalClient.dataCoordClient.Set(&nopCloseDataCoordClient{newLocalClient})
|
||||
log.Info("register data coord server", zap.Any("enableLocalClient", enableLocal))
|
||||
log.Ctx(context.TODO()).Info("register data coord server", zap.Any("enableLocalClient", enableLocal))
|
||||
}
|
||||
|
||||
// RegisterRootCoordServer register root coord server
|
||||
|
@ -86,7 +86,7 @@ func RegisterRootCoordServer(server rootcoordpb.RootCoordServer) {
|
|||
}
|
||||
newLocalClient := grpcclient.NewLocalGRPCClient(&rootcoordpb.RootCoord_ServiceDesc, server, rootcoordpb.NewRootCoordClient)
|
||||
glocalClient.rootCoordClient.Set(&nopCloseRootCoordClient{newLocalClient})
|
||||
log.Info("register root coord server", zap.Any("enableLocalClient", enableLocal))
|
||||
log.Ctx(context.TODO()).Info("register root coord server", zap.Any("enableLocalClient", enableLocal))
|
||||
}
|
||||
|
||||
// GetQueryCoordClient return query coord client
|
||||
|
|
|
@ -166,7 +166,7 @@ func (m *ChannelManagerImpl) Startup(ctx context.Context, legacyNodes, allNodes
|
|||
}
|
||||
|
||||
if m.balanceCheckLoop != nil {
|
||||
log.Info("starting channel balance loop")
|
||||
log.Ctx(ctx).Info("starting channel balance loop")
|
||||
m.wg.Add(1)
|
||||
go func() {
|
||||
defer m.wg.Done()
|
||||
|
@ -174,7 +174,7 @@ func (m *ChannelManagerImpl) Startup(ctx context.Context, legacyNodes, allNodes
|
|||
}()
|
||||
}
|
||||
|
||||
log.Info("cluster start up",
|
||||
log.Ctx(ctx).Info("cluster start up",
|
||||
zap.Int64s("allNodes", allNodes),
|
||||
zap.Int64s("legacyNodes", legacyNodes),
|
||||
zap.Int64s("oldNodes", oNodes),
|
||||
|
@ -242,6 +242,7 @@ func (m *ChannelManagerImpl) Watch(ctx context.Context, ch RWChannel) error {
|
|||
|
||||
log.Info("Add channel")
|
||||
updates := NewChannelOpSet(NewChannelOp(bufferID, Watch, ch))
|
||||
// TODO fill in traceID to channelOp's watchInfo
|
||||
err := m.execute(updates)
|
||||
if err != nil {
|
||||
log.Warn("fail to update new channel updates into meta",
|
||||
|
@ -255,6 +256,7 @@ func (m *ChannelManagerImpl) Watch(ctx context.Context, ch RWChannel) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// TODO fill in traceID to channelOp's watchInfo
|
||||
if err := m.execute(updates); err != nil {
|
||||
log.Warn("fail to assign channel, will retry later", zap.Array("updates", updates), zap.Error(err))
|
||||
return nil
|
||||
|
@ -489,7 +491,7 @@ func (m *ChannelManagerImpl) finishRemoveChannel(nodeID int64, channels ...RWCha
|
|||
}
|
||||
}
|
||||
|
||||
func (m *ChannelManagerImpl) advanceStandbys(_ context.Context, standbys []*NodeChannelInfo) bool {
|
||||
func (m *ChannelManagerImpl) advanceStandbys(ctx context.Context, standbys []*NodeChannelInfo) bool {
|
||||
var advanced bool = false
|
||||
for _, nodeAssign := range standbys {
|
||||
validChannels := make(map[string]RWChannel)
|
||||
|
@ -516,7 +518,7 @@ func (m *ChannelManagerImpl) advanceStandbys(_ context.Context, standbys []*Node
|
|||
|
||||
chNames := lo.Keys(validChannels)
|
||||
if err := m.reassign(nodeAssign); err != nil {
|
||||
log.Warn("Reassign channels fail",
|
||||
log.Ctx(ctx).Warn("Reassign channels fail",
|
||||
zap.Int64("nodeID", nodeAssign.NodeID),
|
||||
zap.Strings("channels", chNames),
|
||||
zap.Error(err),
|
||||
|
@ -524,7 +526,7 @@ func (m *ChannelManagerImpl) advanceStandbys(_ context.Context, standbys []*Node
|
|||
continue
|
||||
}
|
||||
|
||||
log.Info("Reassign standby channels to node",
|
||||
log.Ctx(ctx).Info("Reassign standby channels to node",
|
||||
zap.Int64("nodeID", nodeAssign.NodeID),
|
||||
zap.Strings("channels", chNames),
|
||||
)
|
||||
|
@ -550,7 +552,7 @@ func (m *ChannelManagerImpl) advanceToNotifies(ctx context.Context, toNotifies [
|
|||
)
|
||||
|
||||
chNames := lo.Keys(nodeAssign.Channels)
|
||||
log.Info("Notify channel operations to datanode",
|
||||
log.Ctx(ctx).Info("Notify channel operations to datanode",
|
||||
zap.Int64("assignment", nodeAssign.NodeID),
|
||||
zap.Int("total operation count", len(nodeAssign.Channels)),
|
||||
zap.Strings("channel names", chNames),
|
||||
|
@ -577,7 +579,7 @@ func (m *ChannelManagerImpl) advanceToNotifies(ctx context.Context, toNotifies [
|
|||
}
|
||||
}
|
||||
|
||||
log.Info("Finish to notify channel operations to datanode",
|
||||
log.Ctx(ctx).Info("Finish to notify channel operations to datanode",
|
||||
zap.Int64("assignment", nodeAssign.NodeID),
|
||||
zap.Int("operation count", channelCount),
|
||||
zap.Int("success count", len(succeededChannels)),
|
||||
|
@ -608,7 +610,7 @@ func (m *ChannelManagerImpl) advanceToChecks(ctx context.Context, toChecks []*No
|
|||
futures := make([]*conc.Future[any], 0, len(nodeAssign.Channels))
|
||||
|
||||
chNames := lo.Keys(nodeAssign.Channels)
|
||||
log.Info("Check ToWatch/ToRelease channel operations progress",
|
||||
log.Ctx(ctx).Info("Check ToWatch/ToRelease channel operations progress",
|
||||
zap.Int("channel count", len(nodeAssign.Channels)),
|
||||
zap.Strings("channel names", chNames),
|
||||
)
|
||||
|
@ -641,7 +643,7 @@ func (m *ChannelManagerImpl) advanceToChecks(ctx context.Context, toChecks []*No
|
|||
}
|
||||
}
|
||||
|
||||
log.Info("Finish to Check ToWatch/ToRelease channel operations progress",
|
||||
log.Ctx(ctx).Info("Finish to Check ToWatch/ToRelease channel operations progress",
|
||||
zap.Int("channel count", len(nodeAssign.Channels)),
|
||||
zap.Strings("channel names", chNames),
|
||||
)
|
||||
|
@ -650,7 +652,7 @@ func (m *ChannelManagerImpl) advanceToChecks(ctx context.Context, toChecks []*No
|
|||
}
|
||||
|
||||
func (m *ChannelManagerImpl) Notify(ctx context.Context, nodeID int64, info *datapb.ChannelWatchInfo) error {
|
||||
log := log.With(
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.String("channel", info.GetVchan().GetChannelName()),
|
||||
zap.Int64("assignment", nodeID),
|
||||
zap.String("operation", info.GetState().String()),
|
||||
|
@ -666,7 +668,7 @@ func (m *ChannelManagerImpl) Notify(ctx context.Context, nodeID int64, info *dat
|
|||
}
|
||||
|
||||
func (m *ChannelManagerImpl) Check(ctx context.Context, nodeID int64, info *datapb.ChannelWatchInfo) (successful bool, got bool) {
|
||||
log := log.With(
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.Int64("opID", info.GetOpID()),
|
||||
zap.Int64("nodeID", nodeID),
|
||||
zap.String("check operation", info.GetState().String()),
|
||||
|
|
|
@ -517,9 +517,9 @@ func (c *StateChannelStore) getChannel(nodeID int64, channelName string) *StateC
|
|||
if storedChannel, ok := cInfo.Channels[channelName]; ok {
|
||||
return storedChannel.(*StateChannel)
|
||||
}
|
||||
log.Debug("Channel doesn't exist in Node", zap.String("channel", channelName), zap.Int64("nodeID", nodeID))
|
||||
log.Ctx(context.TODO()).Debug("Channel doesn't exist in Node", zap.String("channel", channelName), zap.Int64("nodeID", nodeID))
|
||||
} else {
|
||||
log.Error("Node doesn't exist", zap.Int64("NodeID", nodeID))
|
||||
log.Ctx(context.TODO()).Error("Node doesn't exist", zap.Int64("NodeID", nodeID))
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -550,7 +550,7 @@ func (c *StateChannelStore) updateMetaMemoryForSingleOp(op *ChannelOp) error {
|
|||
case Delete: // Remove Channel
|
||||
c.removeAssignment(op.NodeID, ch.GetName())
|
||||
default:
|
||||
log.Error("unknown opType in updateMetaMemoryForSingleOp", zap.Any("type", op.Type))
|
||||
log.Ctx(context.TODO()).Error("unknown opType in updateMetaMemoryForSingleOp", zap.Any("type", op.Type))
|
||||
}
|
||||
})
|
||||
return nil
|
||||
|
@ -572,7 +572,7 @@ func (c *StateChannelStore) updateMeta(opSet *ChannelOpSet) error {
|
|||
} else if ops.Len() == 1 {
|
||||
c.updateMetaMemoryForSingleOp(ops.Collect()[0])
|
||||
} else {
|
||||
log.Error("unsupported ChannelOpSet", zap.Any("OpSet", ops))
|
||||
log.Ctx(context.TODO()).Error("unsupported ChannelOpSet", zap.Any("OpSet", ops))
|
||||
}
|
||||
}
|
||||
return nil
|
||||
|
|
|
@ -191,7 +191,7 @@ func (c *ClusterImpl) QuerySlots() map[int64]int64 {
|
|||
defer wg.Done()
|
||||
resp, err := c.sessionManager.QuerySlot(nodeID)
|
||||
if err != nil {
|
||||
log.Warn("query slot failed", zap.Int64("nodeID", nodeID), zap.Error(err))
|
||||
log.Ctx(context.TODO()).Warn("query slot failed", zap.Int64("nodeID", nodeID), zap.Error(err))
|
||||
return
|
||||
}
|
||||
mu.Lock()
|
||||
|
@ -200,7 +200,7 @@ func (c *ClusterImpl) QuerySlots() map[int64]int64 {
|
|||
}(nodeID)
|
||||
}
|
||||
wg.Wait()
|
||||
log.Debug("query slot done", zap.Any("nodeSlots", nodeSlots))
|
||||
log.Ctx(context.TODO()).Debug("query slot done", zap.Any("nodeSlots", nodeSlots))
|
||||
return nodeSlots
|
||||
}
|
||||
|
||||
|
|
|
@ -452,9 +452,9 @@ func (c *compactionPlanHandler) cleanCompactionTaskMeta() {
|
|||
if duration > float64(Params.DataCoordCfg.CompactionDropToleranceInSeconds.GetAsDuration(time.Second).Seconds()) {
|
||||
// try best to delete meta
|
||||
err := c.meta.DropCompactionTask(context.TODO(), task)
|
||||
log.Debug("drop compaction task meta", zap.Int64("planID", task.PlanID))
|
||||
log.Ctx(context.TODO()).Debug("drop compaction task meta", zap.Int64("planID", task.PlanID))
|
||||
if err != nil {
|
||||
log.Warn("fail to drop task", zap.Int64("planID", task.PlanID), zap.Error(err))
|
||||
log.Ctx(context.TODO()).Warn("fail to drop task", zap.Int64("planID", task.PlanID), zap.Error(err))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -463,6 +463,7 @@ func (c *compactionPlanHandler) cleanCompactionTaskMeta() {
|
|||
}
|
||||
|
||||
func (c *compactionPlanHandler) cleanPartitionStats() error {
|
||||
log := log.Ctx(context.TODO())
|
||||
log.Debug("start gc partitionStats meta and files")
|
||||
// gc partition stats
|
||||
channelPartitionStatsInfos := make(map[string][]*datapb.PartitionStatsInfo)
|
||||
|
@ -520,6 +521,7 @@ func (c *compactionPlanHandler) stop() {
|
|||
}
|
||||
|
||||
func (c *compactionPlanHandler) removeTasksByChannel(channel string) {
|
||||
log := log.Ctx(context.TODO())
|
||||
log.Info("removing tasks by channel", zap.String("channel", channel))
|
||||
c.queueTasks.RemoveAll(func(task CompactionTask) bool {
|
||||
if task.GetTaskProto().GetChannel() == channel {
|
||||
|
@ -590,7 +592,7 @@ func (c *compactionPlanHandler) getCompactionTask(planID int64) CompactionTask {
|
|||
}
|
||||
|
||||
func (c *compactionPlanHandler) enqueueCompaction(task *datapb.CompactionTask) error {
|
||||
log := log.With(zap.Int64("planID", task.GetPlanID()), zap.Int64("triggerID", task.GetTriggerID()), zap.Int64("collectionID", task.GetCollectionID()), zap.String("type", task.GetType().String()))
|
||||
log := log.Ctx(context.TODO()).With(zap.Int64("planID", task.GetPlanID()), zap.Int64("triggerID", task.GetTriggerID()), zap.Int64("collectionID", task.GetCollectionID()), zap.String("type", task.GetType().String()))
|
||||
t, err := c.createCompactTask(task)
|
||||
if err != nil {
|
||||
// Conflict is normal
|
||||
|
@ -646,6 +648,7 @@ func assignNodeID(slots map[int64]int64, t CompactionTask) int64 {
|
|||
return NullNodeID
|
||||
}
|
||||
|
||||
log := log.Ctx(context.TODO())
|
||||
nodeID, useSlot := pickAnyNode(slots, t)
|
||||
if nodeID == NullNodeID {
|
||||
log.Info("compactionHandler cannot find datanode for compaction task",
|
||||
|
@ -715,7 +718,7 @@ func pickAnyNode(nodeSlots map[int64]int64, task CompactionTask) (nodeID int64,
|
|||
|
||||
useSlot = task.GetSlotUsage()
|
||||
if useSlot <= 0 {
|
||||
log.Warn("task slot should not be 0", zap.Int64("planID", task.GetTaskProto().GetPlanID()), zap.String("type", task.GetTaskProto().GetType().String()))
|
||||
log.Ctx(context.TODO()).Warn("task slot should not be 0", zap.Int64("planID", task.GetTaskProto().GetPlanID()), zap.String("type", task.GetTaskProto().GetType().String()))
|
||||
return NullNodeID, useSlot
|
||||
}
|
||||
|
||||
|
|
|
@ -405,6 +405,7 @@ func (t *clusteringCompactionTask) regeneratePartitionStats(tmpToResultSegments
|
|||
}
|
||||
|
||||
func (t *clusteringCompactionTask) processIndexing() error {
|
||||
log := log.Ctx(context.TODO())
|
||||
// wait for segment indexed
|
||||
collectionIndexes := t.meta.GetIndexMeta().GetIndexesForCollection(t.GetTaskProto().GetCollectionID(), "")
|
||||
if len(collectionIndexes) == 0 {
|
||||
|
@ -439,7 +440,7 @@ func (t *clusteringCompactionTask) markResultSegmentsVisible() error {
|
|||
|
||||
err := t.meta.UpdateSegmentsInfo(context.TODO(), operators...)
|
||||
if err != nil {
|
||||
log.Warn("markResultSegmentVisible UpdateSegmentsInfo fail", zap.Error(err))
|
||||
log.Ctx(context.TODO()).Warn("markResultSegmentVisible UpdateSegmentsInfo fail", zap.Error(err))
|
||||
return merr.WrapErrClusteringCompactionMetaError("markResultSegmentVisible UpdateSegmentsInfo", err)
|
||||
}
|
||||
return nil
|
||||
|
@ -453,7 +454,7 @@ func (t *clusteringCompactionTask) markInputSegmentsDropped() error {
|
|||
}
|
||||
err := t.meta.UpdateSegmentsInfo(context.TODO(), operators...)
|
||||
if err != nil {
|
||||
log.Warn("markInputSegmentsDropped UpdateSegmentsInfo fail", zap.Error(err))
|
||||
log.Ctx(context.TODO()).Warn("markInputSegmentsDropped UpdateSegmentsInfo fail", zap.Error(err))
|
||||
return merr.WrapErrClusteringCompactionMetaError("markInputSegmentsDropped UpdateSegmentsInfo", err)
|
||||
}
|
||||
return nil
|
||||
|
@ -497,6 +498,7 @@ func (t *clusteringCompactionTask) completeTask() error {
|
|||
}
|
||||
|
||||
func (t *clusteringCompactionTask) processAnalyzing() error {
|
||||
log := log.Ctx(context.TODO())
|
||||
analyzeTask := t.meta.GetAnalyzeMeta().GetTask(t.GetTaskProto().GetAnalyzeTaskID())
|
||||
if analyzeTask == nil {
|
||||
log.Warn("analyzeTask not found", zap.Int64("id", t.GetTaskProto().GetAnalyzeTaskID()))
|
||||
|
@ -525,6 +527,7 @@ func (t *clusteringCompactionTask) resetSegmentCompacting() {
|
|||
}
|
||||
|
||||
func (t *clusteringCompactionTask) processFailedOrTimeout() error {
|
||||
log := log.Ctx(context.TODO())
|
||||
log.Info("clean task", zap.Int64("triggerID", t.GetTaskProto().GetTriggerID()), zap.Int64("planID", t.GetTaskProto().GetPlanID()), zap.String("state", t.GetTaskProto().GetState().String()))
|
||||
|
||||
if err := t.sessions.DropCompactionPlan(t.GetTaskProto().GetNodeID(), &datapb.DropCompactionPlanRequest{
|
||||
|
@ -604,6 +607,7 @@ func (t *clusteringCompactionTask) processFailedOrTimeout() error {
|
|||
}
|
||||
|
||||
func (t *clusteringCompactionTask) doAnalyze() error {
|
||||
log := log.Ctx(context.TODO())
|
||||
analyzeTask := &indexpb.AnalyzeTask{
|
||||
CollectionID: t.GetTaskProto().GetCollectionID(),
|
||||
PartitionID: t.GetTaskProto().GetPartitionID(),
|
||||
|
@ -627,7 +631,7 @@ func (t *clusteringCompactionTask) doAnalyze() error {
|
|||
}
|
||||
|
||||
func (t *clusteringCompactionTask) doCompact() error {
|
||||
log := log.With(zap.Int64("planID", t.GetTaskProto().GetPlanID()), zap.String("type", t.GetTaskProto().GetType().String()))
|
||||
log := log.Ctx(context.TODO()).With(zap.Int64("planID", t.GetTaskProto().GetPlanID()), zap.String("type", t.GetTaskProto().GetType().String()))
|
||||
if t.NeedReAssignNodeID() {
|
||||
log.RatedWarn(10, "not assign nodeID")
|
||||
return nil
|
||||
|
@ -683,7 +687,7 @@ func (t *clusteringCompactionTask) updateAndSaveTaskMeta(opts ...compactionTaskO
|
|||
task := t.ShadowClone(opts...)
|
||||
err := t.saveTaskMeta(task)
|
||||
if err != nil {
|
||||
log.Warn("Failed to saveTaskMeta", zap.Error(err))
|
||||
log.Ctx(context.TODO()).Warn("Failed to saveTaskMeta", zap.Error(err))
|
||||
return merr.WrapErrClusteringCompactionMetaError("updateAndSaveTaskMeta", err) // retryable
|
||||
}
|
||||
t.SetTask(task)
|
||||
|
@ -695,7 +699,7 @@ func (t *clusteringCompactionTask) checkTimeout() bool {
|
|||
if t.GetTaskProto().GetTimeoutInSeconds() > 0 {
|
||||
diff := time.Since(time.Unix(t.GetTaskProto().GetStartTime(), 0)).Seconds()
|
||||
if diff > float64(t.GetTaskProto().GetTimeoutInSeconds()) {
|
||||
log.Warn("compaction timeout",
|
||||
log.Ctx(context.TODO()).Warn("compaction timeout",
|
||||
zap.Int32("timeout in seconds", t.GetTaskProto().GetTimeoutInSeconds()),
|
||||
zap.Int64("startTime", t.GetTaskProto().GetStartTime()),
|
||||
)
|
||||
|
|
|
@ -675,7 +675,7 @@ func isDeleteRowsTooManySegment(segment *SegmentInfo) bool {
|
|||
is := float64(totalDeletedRows)/float64(segment.GetNumOfRows()) >= Params.DataCoordCfg.SingleCompactionRatioThreshold.GetAsFloat() ||
|
||||
totalDeleteLogSize > Params.DataCoordCfg.SingleCompactionDeltaLogMaxSize.GetAsInt64()
|
||||
if is {
|
||||
log.Info("total delete entities is too much",
|
||||
log.Ctx(context.TODO()).Info("total delete entities is too much",
|
||||
zap.Int64("segmentID", segment.ID),
|
||||
zap.Int64("numRows", segment.GetNumOfRows()),
|
||||
zap.Int("deleted rows", totalDeletedRows),
|
||||
|
@ -687,6 +687,7 @@ func isDeleteRowsTooManySegment(segment *SegmentInfo) bool {
|
|||
func (t *compactionTrigger) ShouldDoSingleCompaction(segment *SegmentInfo, compactTime *compactTime) bool {
|
||||
// no longer restricted binlog numbers because this is now related to field numbers
|
||||
|
||||
log := log.Ctx(context.TODO())
|
||||
binlogCount := GetBinlogCount(segment.GetBinlogs())
|
||||
deltaLogCount := GetBinlogCount(segment.GetDeltalogs())
|
||||
if isDeltalogTooManySegment(segment) {
|
||||
|
|
|
@ -108,6 +108,7 @@ func (m *CompactionTriggerManager) startLoop() {
|
|||
defer logutil.LogPanic()
|
||||
defer m.closeWg.Done()
|
||||
|
||||
log := log.Ctx(context.TODO())
|
||||
l0Ticker := time.NewTicker(Params.DataCoordCfg.L0CompactionTriggerInterval.GetAsDuration(time.Second))
|
||||
defer l0Ticker.Stop()
|
||||
clusteringTicker := time.NewTicker(Params.DataCoordCfg.ClusteringCompactionTriggerInterval.GetAsDuration(time.Second))
|
||||
|
@ -182,7 +183,7 @@ func (m *CompactionTriggerManager) startLoop() {
|
|||
}
|
||||
|
||||
func (m *CompactionTriggerManager) ManualTrigger(ctx context.Context, collectionID int64, clusteringCompaction bool) (UniqueID, error) {
|
||||
log.Info("receive manual trigger", zap.Int64("collectionID", collectionID))
|
||||
log.Ctx(ctx).Info("receive manual trigger", zap.Int64("collectionID", collectionID))
|
||||
views, triggerID, err := m.clusteringPolicy.triggerOneCollection(context.Background(), collectionID, true)
|
||||
if err != nil {
|
||||
return 0, err
|
||||
|
@ -198,6 +199,7 @@ func (m *CompactionTriggerManager) ManualTrigger(ctx context.Context, collection
|
|||
}
|
||||
|
||||
func (m *CompactionTriggerManager) notify(ctx context.Context, eventType CompactionTriggerType, views []CompactionView) {
|
||||
log := log.Ctx(ctx)
|
||||
for _, view := range views {
|
||||
switch eventType {
|
||||
case TriggerTypeLevelZeroViewChange:
|
||||
|
@ -246,7 +248,7 @@ func (m *CompactionTriggerManager) notify(ctx context.Context, eventType Compact
|
|||
}
|
||||
|
||||
func (m *CompactionTriggerManager) SubmitL0ViewToScheduler(ctx context.Context, view CompactionView) {
|
||||
log := log.With(zap.String("view", view.String()))
|
||||
log := log.Ctx(ctx).With(zap.String("view", view.String()))
|
||||
taskID, err := m.allocator.AllocID(ctx)
|
||||
if err != nil {
|
||||
log.Warn("Failed to submit compaction view to scheduler because allocate id fail", zap.Error(err))
|
||||
|
@ -296,7 +298,7 @@ func (m *CompactionTriggerManager) SubmitL0ViewToScheduler(ctx context.Context,
|
|||
}
|
||||
|
||||
func (m *CompactionTriggerManager) SubmitClusteringViewToScheduler(ctx context.Context, view CompactionView) {
|
||||
log := log.With(zap.String("view", view.String()))
|
||||
log := log.Ctx(ctx).With(zap.String("view", view.String()))
|
||||
taskID, _, err := m.allocator.AllocN(2)
|
||||
if err != nil {
|
||||
log.Warn("Failed to submit compaction view to scheduler because allocate id fail", zap.Error(err))
|
||||
|
@ -362,7 +364,7 @@ func (m *CompactionTriggerManager) SubmitClusteringViewToScheduler(ctx context.C
|
|||
}
|
||||
|
||||
func (m *CompactionTriggerManager) SubmitSingleViewToScheduler(ctx context.Context, view CompactionView) {
|
||||
log := log.With(zap.String("view", view.String()))
|
||||
log := log.Ctx(ctx).With(zap.String("view", view.String()))
|
||||
// TODO[GOOSE], 11 = 1 planID + 10 segmentID, this is a hack need to be removed.
|
||||
// Any plan that output segment number greater than 10 will be marked as invalid plan for now.
|
||||
startID, endID, err := m.allocator.AllocN(11)
|
||||
|
|
|
@ -489,6 +489,7 @@ func (gc *garbageCollector) recycleDroppedSegments(ctx context.Context) {
|
|||
}
|
||||
|
||||
func (gc *garbageCollector) recycleChannelCPMeta(ctx context.Context) {
|
||||
log := log.Ctx(ctx)
|
||||
channelCPs, err := gc.meta.catalog.ListChannelCheckpoint(ctx)
|
||||
if err != nil {
|
||||
log.Warn("list channel cp fail during GC", zap.Error(err))
|
||||
|
@ -596,7 +597,7 @@ func (gc *garbageCollector) removeObjectFiles(ctx context.Context, filePaths map
|
|||
if !errors.Is(err, merr.ErrIoKeyNotFound) {
|
||||
return struct{}{}, err
|
||||
}
|
||||
log.Info("remove log failed, key not found, may be removed at previous GC, ignore the error",
|
||||
log.Ctx(ctx).Info("remove log failed, key not found, may be removed at previous GC, ignore the error",
|
||||
zap.String("path", filePath),
|
||||
zap.Error(err))
|
||||
}
|
||||
|
@ -610,7 +611,7 @@ func (gc *garbageCollector) removeObjectFiles(ctx context.Context, filePaths map
|
|||
// recycleUnusedIndexes is used to delete those indexes that is deleted by collection.
|
||||
func (gc *garbageCollector) recycleUnusedIndexes(ctx context.Context) {
|
||||
start := time.Now()
|
||||
log := log.With(zap.String("gcName", "recycleUnusedIndexes"), zap.Time("startAt", start))
|
||||
log := log.Ctx(ctx).With(zap.String("gcName", "recycleUnusedIndexes"), zap.Time("startAt", start))
|
||||
log.Info("start recycleUnusedIndexes...")
|
||||
defer func() { log.Info("recycleUnusedIndexes done", zap.Duration("timeCost", time.Since(start))) }()
|
||||
|
||||
|
@ -633,7 +634,7 @@ func (gc *garbageCollector) recycleUnusedIndexes(ctx context.Context) {
|
|||
// recycleUnusedSegIndexes remove the index of segment if index is deleted or segment itself is deleted.
|
||||
func (gc *garbageCollector) recycleUnusedSegIndexes(ctx context.Context) {
|
||||
start := time.Now()
|
||||
log := log.With(zap.String("gcName", "recycleUnusedSegIndexes"), zap.Time("startAt", start))
|
||||
log := log.Ctx(ctx).With(zap.String("gcName", "recycleUnusedSegIndexes"), zap.Time("startAt", start))
|
||||
log.Info("start recycleUnusedSegIndexes...")
|
||||
defer func() { log.Info("recycleUnusedSegIndexes done", zap.Duration("timeCost", time.Since(start))) }()
|
||||
|
||||
|
@ -676,7 +677,7 @@ func (gc *garbageCollector) recycleUnusedSegIndexes(ctx context.Context) {
|
|||
// recycleUnusedIndexFiles is used to delete those index files that no longer exist in the meta.
|
||||
func (gc *garbageCollector) recycleUnusedIndexFiles(ctx context.Context) {
|
||||
start := time.Now()
|
||||
log := log.With(zap.String("gcName", "recycleUnusedIndexFiles"), zap.Time("startAt", start))
|
||||
log := log.Ctx(ctx).With(zap.String("gcName", "recycleUnusedIndexFiles"), zap.Time("startAt", start))
|
||||
log.Info("start recycleUnusedIndexFiles...")
|
||||
|
||||
prefix := path.Join(gc.option.cli.RootPath(), common.SegmentIndexPath) + "/"
|
||||
|
@ -774,6 +775,7 @@ func (gc *garbageCollector) getAllIndexFilesOfIndex(segmentIndex *model.SegmentI
|
|||
|
||||
// recycleUnusedAnalyzeFiles is used to delete those analyze stats files that no longer exist in the meta.
|
||||
func (gc *garbageCollector) recycleUnusedAnalyzeFiles(ctx context.Context) {
|
||||
log := log.Ctx(ctx)
|
||||
log.Info("start recycleUnusedAnalyzeFiles")
|
||||
startTs := time.Now()
|
||||
prefix := path.Join(gc.option.cli.RootPath(), common.AnalyzeStatsPath) + "/"
|
||||
|
@ -846,7 +848,7 @@ func (gc *garbageCollector) recycleUnusedAnalyzeFiles(ctx context.Context) {
|
|||
// if missing found, performs gc cleanup
|
||||
func (gc *garbageCollector) recycleUnusedTextIndexFiles(ctx context.Context) {
|
||||
start := time.Now()
|
||||
log := log.With(zap.String("gcName", "recycleUnusedTextIndexFiles"), zap.Time("startAt", start))
|
||||
log := log.Ctx(ctx).With(zap.String("gcName", "recycleUnusedTextIndexFiles"), zap.Time("startAt", start))
|
||||
log.Info("start recycleUnusedTextIndexFiles...")
|
||||
defer func() { log.Info("recycleUnusedTextIndexFiles done", zap.Duration("timeCost", time.Since(start))) }()
|
||||
|
||||
|
|
|
@ -147,7 +147,7 @@ func (s *importScheduler) peekSlots() map[int64]int64 {
|
|||
}(nodeID)
|
||||
}
|
||||
wg.Wait()
|
||||
log.Debug("peek slots done", zap.Any("nodeSlots", nodeSlots))
|
||||
log.Ctx(context.TODO()).Debug("peek slots done", zap.Any("nodeSlots", nodeSlots))
|
||||
return nodeSlots
|
||||
}
|
||||
|
||||
|
|
|
@ -366,7 +366,7 @@ func (m *indexMeta) HasSameReq(req *indexpb.CreateIndexRequest) (bool, UniqueID)
|
|||
if !checkParams(fieldIndex, req) {
|
||||
continue
|
||||
}
|
||||
log.Debug("has same index", zap.Int64("collectionID", req.CollectionID),
|
||||
log.Ctx(context.TODO()).Debug("has same index", zap.Int64("collectionID", req.CollectionID),
|
||||
zap.Int64("fieldID", req.FieldID), zap.String("indexName", req.IndexName),
|
||||
zap.Int64("indexID", fieldIndex.IndexID))
|
||||
return true, fieldIndex.IndexID
|
||||
|
@ -376,20 +376,20 @@ func (m *indexMeta) HasSameReq(req *indexpb.CreateIndexRequest) (bool, UniqueID)
|
|||
}
|
||||
|
||||
func (m *indexMeta) CreateIndex(ctx context.Context, index *model.Index) error {
|
||||
log.Info("meta update: CreateIndex", zap.Int64("collectionID", index.CollectionID),
|
||||
log.Ctx(ctx).Info("meta update: CreateIndex", zap.Int64("collectionID", index.CollectionID),
|
||||
zap.Int64("fieldID", index.FieldID), zap.Int64("indexID", index.IndexID), zap.String("indexName", index.IndexName))
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
|
||||
if err := m.catalog.CreateIndex(ctx, index); err != nil {
|
||||
log.Error("meta update: CreateIndex save meta fail", zap.Int64("collectionID", index.CollectionID),
|
||||
log.Ctx(ctx).Error("meta update: CreateIndex save meta fail", zap.Int64("collectionID", index.CollectionID),
|
||||
zap.Int64("fieldID", index.FieldID), zap.Int64("indexID", index.IndexID),
|
||||
zap.String("indexName", index.IndexName), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
m.updateCollectionIndex(index)
|
||||
log.Info("meta update: CreateIndex success", zap.Int64("collectionID", index.CollectionID),
|
||||
log.Ctx(ctx).Info("meta update: CreateIndex success", zap.Int64("collectionID", index.CollectionID),
|
||||
zap.Int64("fieldID", index.FieldID), zap.Int64("indexID", index.IndexID), zap.String("indexName", index.IndexName))
|
||||
return nil
|
||||
}
|
||||
|
@ -416,19 +416,19 @@ func (m *indexMeta) AddSegmentIndex(ctx context.Context, segIndex *model.Segment
|
|||
defer m.Unlock()
|
||||
|
||||
buildID := segIndex.BuildID
|
||||
log.Info("meta update: adding segment index", zap.Int64("collectionID", segIndex.CollectionID),
|
||||
log.Ctx(ctx).Info("meta update: adding segment index", zap.Int64("collectionID", segIndex.CollectionID),
|
||||
zap.Int64("segmentID", segIndex.SegmentID), zap.Int64("indexID", segIndex.IndexID),
|
||||
zap.Int64("buildID", buildID))
|
||||
|
||||
segIndex.IndexState = commonpb.IndexState_Unissued
|
||||
if err := m.catalog.CreateSegmentIndex(ctx, segIndex); err != nil {
|
||||
log.Warn("meta update: adding segment index failed",
|
||||
log.Ctx(ctx).Warn("meta update: adding segment index failed",
|
||||
zap.Int64("segmentID", segIndex.SegmentID), zap.Int64("indexID", segIndex.IndexID),
|
||||
zap.Int64("buildID", segIndex.BuildID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
m.updateSegmentIndex(segIndex)
|
||||
log.Info("meta update: adding segment index success", zap.Int64("collectionID", segIndex.CollectionID),
|
||||
log.Ctx(ctx).Info("meta update: adding segment index success", zap.Int64("collectionID", segIndex.CollectionID),
|
||||
zap.Int64("segmentID", segIndex.SegmentID), zap.Int64("indexID", segIndex.IndexID),
|
||||
zap.Int64("buildID", buildID))
|
||||
m.updateIndexTasksMetrics()
|
||||
|
@ -563,7 +563,7 @@ func (m *indexMeta) GetFieldIndexes(collID, fieldID UniqueID, indexName string)
|
|||
|
||||
// MarkIndexAsDeleted will mark the corresponding index as deleted, and recycleUnusedIndexFiles will recycle these tasks.
|
||||
func (m *indexMeta) MarkIndexAsDeleted(ctx context.Context, collID UniqueID, indexIDs []UniqueID) error {
|
||||
log.Info("IndexCoord metaTable MarkIndexAsDeleted", zap.Int64("collectionID", collID),
|
||||
log.Ctx(ctx).Info("IndexCoord metaTable MarkIndexAsDeleted", zap.Int64("collectionID", collID),
|
||||
zap.Int64s("indexIDs", indexIDs))
|
||||
|
||||
m.Lock()
|
||||
|
@ -588,14 +588,14 @@ func (m *indexMeta) MarkIndexAsDeleted(ctx context.Context, collID UniqueID, ind
|
|||
}
|
||||
err := m.catalog.AlterIndexes(ctx, indexes)
|
||||
if err != nil {
|
||||
log.Error("failed to alter index meta in meta store", zap.Int("indexes num", len(indexes)), zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to alter index meta in meta store", zap.Int("indexes num", len(indexes)), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
for _, index := range indexes {
|
||||
m.indexes[index.CollectionID][index.IndexID] = index
|
||||
}
|
||||
|
||||
log.Info("IndexCoord metaTable MarkIndexAsDeleted success", zap.Int64("collectionID", collID), zap.Int64s("indexIDs", indexIDs))
|
||||
log.Ctx(ctx).Info("IndexCoord metaTable MarkIndexAsDeleted success", zap.Int64("collectionID", collID), zap.Int64s("indexIDs", indexIDs))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -760,7 +760,7 @@ func (m *indexMeta) UpdateVersion(buildID, nodeID UniqueID) error {
|
|||
m.Lock()
|
||||
defer m.Unlock()
|
||||
|
||||
log.Info("IndexCoord metaTable UpdateVersion receive", zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
|
||||
log.Ctx(m.ctx).Info("IndexCoord metaTable UpdateVersion receive", zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
|
||||
segIdx, ok := m.segmentBuildInfo.Get(buildID)
|
||||
if !ok {
|
||||
return fmt.Errorf("there is no index with buildID: %d", buildID)
|
||||
|
@ -781,7 +781,7 @@ func (m *indexMeta) FinishTask(taskInfo *workerpb.IndexTaskInfo) error {
|
|||
|
||||
segIdx, ok := m.segmentBuildInfo.Get(taskInfo.GetBuildID())
|
||||
if !ok {
|
||||
log.Warn("there is no index with buildID", zap.Int64("buildID", taskInfo.GetBuildID()))
|
||||
log.Ctx(m.ctx).Warn("there is no index with buildID", zap.Int64("buildID", taskInfo.GetBuildID()))
|
||||
return nil
|
||||
}
|
||||
updateFunc := func(segIdx *model.SegmentIndex) error {
|
||||
|
@ -798,7 +798,7 @@ func (m *indexMeta) FinishTask(taskInfo *workerpb.IndexTaskInfo) error {
|
|||
return err
|
||||
}
|
||||
|
||||
log.Info("finish index task success", zap.Int64("buildID", taskInfo.GetBuildID()),
|
||||
log.Ctx(m.ctx).Info("finish index task success", zap.Int64("buildID", taskInfo.GetBuildID()),
|
||||
zap.String("state", taskInfo.GetState().String()), zap.String("fail reason", taskInfo.GetFailReason()),
|
||||
zap.Int32("current_index_version", taskInfo.GetCurrentIndexVersion()),
|
||||
)
|
||||
|
@ -813,7 +813,7 @@ func (m *indexMeta) DeleteTask(buildID int64) error {
|
|||
|
||||
segIdx, ok := m.segmentBuildInfo.Get(buildID)
|
||||
if !ok {
|
||||
log.Warn("there is no index with buildID", zap.Int64("buildID", buildID))
|
||||
log.Ctx(m.ctx).Warn("there is no index with buildID", zap.Int64("buildID", buildID))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -826,7 +826,7 @@ func (m *indexMeta) DeleteTask(buildID int64) error {
|
|||
return err
|
||||
}
|
||||
|
||||
log.Info("delete index task success", zap.Int64("buildID", buildID))
|
||||
log.Ctx(m.ctx).Info("delete index task success", zap.Int64("buildID", buildID))
|
||||
m.updateIndexTasksMetrics()
|
||||
return nil
|
||||
}
|
||||
|
@ -846,7 +846,7 @@ func (m *indexMeta) BuildIndex(buildID UniqueID) error {
|
|||
|
||||
err := m.alterSegmentIndexes([]*model.SegmentIndex{segIdx})
|
||||
if err != nil {
|
||||
log.Error("meta Update: segment index in progress fail", zap.Int64("buildID", segIdx.BuildID), zap.Error(err))
|
||||
log.Ctx(m.ctx).Error("meta Update: segment index in progress fail", zap.Int64("buildID", segIdx.BuildID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
|
@ -854,7 +854,7 @@ func (m *indexMeta) BuildIndex(buildID UniqueID) error {
|
|||
if err := m.updateSegIndexMeta(segIdx, updateFunc); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Info("meta update: segment index in progress success", zap.Int64("buildID", segIdx.BuildID),
|
||||
log.Ctx(m.ctx).Info("meta update: segment index in progress success", zap.Int64("buildID", segIdx.BuildID),
|
||||
zap.Int64("segmentID", segIdx.SegmentID))
|
||||
|
||||
m.updateIndexTasksMetrics()
|
||||
|
@ -932,10 +932,10 @@ func (m *indexMeta) GetDeletedIndexes() []*model.Index {
|
|||
func (m *indexMeta) RemoveIndex(ctx context.Context, collID, indexID UniqueID) error {
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
log.Info("IndexCoord meta table remove index", zap.Int64("collectionID", collID), zap.Int64("indexID", indexID))
|
||||
log.Ctx(ctx).Info("IndexCoord meta table remove index", zap.Int64("collectionID", collID), zap.Int64("indexID", indexID))
|
||||
err := m.catalog.DropIndex(ctx, collID, indexID)
|
||||
if err != nil {
|
||||
log.Info("IndexCoord meta table remove index fail", zap.Int64("collectionID", collID),
|
||||
log.Ctx(ctx).Info("IndexCoord meta table remove index fail", zap.Int64("collectionID", collID),
|
||||
zap.Int64("indexID", indexID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
@ -948,7 +948,7 @@ func (m *indexMeta) RemoveIndex(ctx context.Context, collID, indexID UniqueID) e
|
|||
metrics.IndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.FinishedIndexTaskLabel})
|
||||
metrics.IndexTaskNum.Delete(prometheus.Labels{"collection_id": strconv.FormatInt(collID, 10), "index_task_status": metrics.FailedIndexTaskLabel})
|
||||
}
|
||||
log.Info("IndexCoord meta table remove index success", zap.Int64("collectionID", collID), zap.Int64("indexID", indexID))
|
||||
log.Ctx(ctx).Info("IndexCoord meta table remove index success", zap.Int64("collectionID", collID), zap.Int64("indexID", indexID))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -82,11 +82,11 @@ func (s *Server) createIndexForSegment(ctx context.Context, segment *SegmentInfo
|
|||
|
||||
func (s *Server) createIndexesForSegment(ctx context.Context, segment *SegmentInfo) error {
|
||||
if Params.DataCoordCfg.EnableStatsTask.GetAsBool() && !segment.GetIsSorted() && !segment.GetIsImporting() {
|
||||
log.Debug("segment is not sorted by pk, skip create indexes", zap.Int64("segmentID", segment.GetID()))
|
||||
log.Ctx(ctx).Debug("segment is not sorted by pk, skip create indexes", zap.Int64("segmentID", segment.GetID()))
|
||||
return nil
|
||||
}
|
||||
if segment.GetLevel() == datapb.SegmentLevel_L0 {
|
||||
log.Debug("segment is level zero, skip create indexes", zap.Int64("segmentID", segment.GetID()))
|
||||
log.Ctx(ctx).Debug("segment is level zero, skip create indexes", zap.Int64("segmentID", segment.GetID()))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -95,7 +95,7 @@ func (s *Server) createIndexesForSegment(ctx context.Context, segment *SegmentIn
|
|||
for _, index := range indexes {
|
||||
if _, ok := indexIDToSegIndexes[index.IndexID]; !ok {
|
||||
if err := s.createIndexForSegment(ctx, segment, index.IndexID); err != nil {
|
||||
log.Warn("create index for segment fail", zap.Int64("segmentID", segment.ID),
|
||||
log.Ctx(ctx).Warn("create index for segment fail", zap.Int64("segmentID", segment.ID),
|
||||
zap.Int64("indexID", index.IndexID))
|
||||
return err
|
||||
}
|
||||
|
@ -119,6 +119,7 @@ func (s *Server) getUnIndexTaskSegments(ctx context.Context) []*SegmentInfo {
|
|||
}
|
||||
|
||||
func (s *Server) createIndexForSegmentLoop(ctx context.Context) {
|
||||
log := log.Ctx(ctx)
|
||||
log.Info("start create index for segment loop...")
|
||||
defer s.serverLoopWg.Done()
|
||||
|
||||
|
|
|
@ -374,7 +374,7 @@ func (m *meta) GetSegmentsChanPart(selector SegmentInfoSelector) []*chanPartSegm
|
|||
for _, entry := range mDimEntry {
|
||||
result = append(result, entry)
|
||||
}
|
||||
log.Debug("GetSegmentsChanPart", zap.Int("length", len(result)))
|
||||
log.Ctx(context.TODO()).Debug("GetSegmentsChanPart", zap.Int("length", len(result)))
|
||||
return result
|
||||
}
|
||||
|
||||
|
@ -446,7 +446,7 @@ func (m *meta) GetQuotaInfo() *metricsinfo.DataCoordQuotaMetrics {
|
|||
metrics.DataCoordSegmentBinLogFileCount.WithLabelValues(
|
||||
fmt.Sprint(segment.GetCollectionID())).Add(float64(getBinlogFileCount(segment.SegmentInfo)))
|
||||
} else {
|
||||
log.Warn("not found database name", zap.Int64("collectionID", segment.GetCollectionID()))
|
||||
log.Ctx(context.TODO()).Warn("not found database name", zap.Int64("collectionID", segment.GetCollectionID()))
|
||||
}
|
||||
|
||||
if _, ok := collectionRowsNum[segment.GetCollectionID()]; !ok {
|
||||
|
@ -527,6 +527,7 @@ func (m *meta) AddSegment(ctx context.Context, segment *SegmentInfo) error {
|
|||
|
||||
// DropSegment remove segment with provided id, etcd persistence also removed
|
||||
func (m *meta) DropSegment(ctx context.Context, segmentID UniqueID) error {
|
||||
log := log.Ctx(ctx)
|
||||
log.Debug("meta update: dropping segment", zap.Int64("segmentID", segmentID))
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
|
@ -599,7 +600,7 @@ func (m *meta) GetSegmentsTotalCurrentRows(segmentIDs []UniqueID) int64 {
|
|||
for _, segmentID := range segmentIDs {
|
||||
segment := m.segments.GetSegment(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("cannot find segment", zap.Int64("segmentID", segmentID))
|
||||
log.Ctx(context.TODO()).Warn("cannot find segment", zap.Int64("segmentID", segmentID))
|
||||
continue
|
||||
}
|
||||
sum += segment.currRows
|
||||
|
@ -623,6 +624,7 @@ func (m *meta) GetSegmentsChannels(segmentIDs []UniqueID) (map[int64]string, err
|
|||
|
||||
// SetState setting segment with provided ID state
|
||||
func (m *meta) SetState(ctx context.Context, segmentID UniqueID, targetState commonpb.SegmentState) error {
|
||||
log := log.Ctx(context.TODO())
|
||||
log.Debug("meta update: setting segment state",
|
||||
zap.Int64("segmentID", segmentID),
|
||||
zap.Any("target state", targetState))
|
||||
|
@ -668,6 +670,7 @@ func (m *meta) SetState(ctx context.Context, segmentID UniqueID, targetState com
|
|||
func (m *meta) UpdateSegment(segmentID int64, operators ...SegmentOperator) error {
|
||||
m.Lock()
|
||||
defer m.Unlock()
|
||||
log := log.Ctx(context.TODO())
|
||||
info := m.segments.GetSegment(segmentID)
|
||||
if info == nil {
|
||||
log.Warn("meta update: UpdateSegment - segment not found",
|
||||
|
@ -720,7 +723,7 @@ func (p *updateSegmentPack) Get(segmentID int64) *SegmentInfo {
|
|||
|
||||
segment := p.meta.segments.GetSegment(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: get segment failed - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: get segment failed - segment not found",
|
||||
zap.Int64("segmentID", segmentID),
|
||||
zap.Bool("segment nil", segment == nil),
|
||||
zap.Bool("segment unhealthy", !isSegmentHealthy(segment)))
|
||||
|
@ -737,7 +740,7 @@ func CreateL0Operator(collectionID, partitionID, segmentID int64, channel string
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.meta.segments.GetSegment(segmentID)
|
||||
if segment == nil {
|
||||
log.Info("meta update: add new l0 segment",
|
||||
log.Ctx(context.TODO()).Info("meta update: add new l0 segment",
|
||||
zap.Int64("collectionID", collectionID),
|
||||
zap.Int64("partitionID", partitionID),
|
||||
zap.Int64("segmentID", segmentID))
|
||||
|
@ -760,7 +763,7 @@ func UpdateStorageVersionOperator(segmentID int64, version int64) UpdateOperator
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Info("meta update: update storage version - segment not found",
|
||||
log.Ctx(context.TODO()).Info("meta update: update storage version - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
|
@ -776,7 +779,7 @@ func UpdateStatusOperator(segmentID int64, status commonpb.SegmentState) UpdateO
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: update status failed - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update status failed - segment not found",
|
||||
zap.Int64("segmentID", segmentID),
|
||||
zap.String("status", status.String()))
|
||||
return false
|
||||
|
@ -794,7 +797,7 @@ func UpdateCompactedOperator(segmentID int64) UpdateOperator {
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: update binlog failed - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update binlog failed - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
|
@ -807,7 +810,7 @@ func SetSegmentIsInvisible(segmentID int64, isInvisible bool) UpdateOperator {
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: update segment visible fail - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update segment visible fail - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
|
@ -820,12 +823,12 @@ func UpdateSegmentLevelOperator(segmentID int64, level datapb.SegmentLevel) Upda
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: update level fail - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update level fail - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
if segment.LastLevel == segment.Level && segment.Level == level {
|
||||
log.Debug("segment already is this level", zap.Int64("segID", segmentID), zap.String("level", level.String()))
|
||||
log.Ctx(context.TODO()).Debug("segment already is this level", zap.Int64("segID", segmentID), zap.String("level", level.String()))
|
||||
return true
|
||||
}
|
||||
segment.LastLevel = segment.Level
|
||||
|
@ -838,13 +841,13 @@ func UpdateSegmentPartitionStatsVersionOperator(segmentID int64, version int64)
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: update partition stats version fail - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update partition stats version fail - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
segment.LastPartitionStatsVersion = segment.PartitionStatsVersion
|
||||
segment.PartitionStatsVersion = version
|
||||
log.Debug("update segment version", zap.Int64("segmentID", segmentID), zap.Int64("PartitionStatsVersion", version), zap.Int64("LastPartitionStatsVersion", segment.LastPartitionStatsVersion))
|
||||
log.Ctx(context.TODO()).Debug("update segment version", zap.Int64("segmentID", segmentID), zap.Int64("PartitionStatsVersion", version), zap.Int64("LastPartitionStatsVersion", segment.LastPartitionStatsVersion))
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
@ -853,14 +856,14 @@ func RevertSegmentLevelOperator(segmentID int64) UpdateOperator {
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: revert level fail - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: revert level fail - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
// just for compatibility,
|
||||
if segment.GetLevel() != segment.GetLastLevel() && segment.GetLastLevel() != datapb.SegmentLevel_Legacy {
|
||||
segment.Level = segment.LastLevel
|
||||
log.Debug("revert segment level", zap.Int64("segmentID", segmentID), zap.String("LastLevel", segment.LastLevel.String()))
|
||||
log.Ctx(context.TODO()).Debug("revert segment level", zap.Int64("segmentID", segmentID), zap.String("LastLevel", segment.LastLevel.String()))
|
||||
return true
|
||||
}
|
||||
return false
|
||||
|
@ -871,12 +874,12 @@ func RevertSegmentPartitionStatsVersionOperator(segmentID int64) UpdateOperator
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: revert level fail - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: revert level fail - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
segment.PartitionStatsVersion = segment.LastPartitionStatsVersion
|
||||
log.Debug("revert segment partition stats version", zap.Int64("segmentID", segmentID), zap.Int64("LastPartitionStatsVersion", segment.LastPartitionStatsVersion))
|
||||
log.Ctx(context.TODO()).Debug("revert segment partition stats version", zap.Int64("segmentID", segmentID), zap.Int64("LastPartitionStatsVersion", segment.LastPartitionStatsVersion))
|
||||
return true
|
||||
}
|
||||
}
|
||||
|
@ -886,7 +889,7 @@ func AddBinlogsOperator(segmentID int64, binlogs, statslogs, deltalogs, bm25logs
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: add binlog failed - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: add binlog failed - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
|
@ -910,7 +913,7 @@ func UpdateBinlogsOperator(segmentID int64, binlogs, statslogs, deltalogs []*dat
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: update binlog failed - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update binlog failed - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
|
@ -946,14 +949,14 @@ func UpdateStartPosition(startPositions []*datapb.SegmentStartPosition) UpdateOp
|
|||
func UpdateDmlPosition(segmentID int64, dmlPosition *msgpb.MsgPosition) UpdateOperator {
|
||||
return func(modPack *updateSegmentPack) bool {
|
||||
if len(dmlPosition.GetMsgID()) == 0 {
|
||||
log.Warn("meta update: update dml position failed - nil position msg id",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update dml position failed - nil position msg id",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: update dml position failed - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update dml position failed - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
|
@ -968,7 +971,7 @@ func UpdateCheckPointOperator(segmentID int64, checkpoints []*datapb.CheckPoint)
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: update checkpoint failed - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update checkpoint failed - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
|
@ -976,12 +979,12 @@ func UpdateCheckPointOperator(segmentID int64, checkpoints []*datapb.CheckPoint)
|
|||
for _, cp := range checkpoints {
|
||||
if cp.SegmentID != segmentID {
|
||||
// Don't think this is gonna to happen, ignore for now.
|
||||
log.Warn("checkpoint in segment is not same as flush segment to update, igreo", zap.Int64("current", segmentID), zap.Int64("checkpoint segment", cp.SegmentID))
|
||||
log.Ctx(context.TODO()).Warn("checkpoint in segment is not same as flush segment to update, igreo", zap.Int64("current", segmentID), zap.Int64("checkpoint segment", cp.SegmentID))
|
||||
continue
|
||||
}
|
||||
|
||||
if segment.DmlPosition != nil && segment.DmlPosition.Timestamp >= cp.Position.Timestamp {
|
||||
log.Warn("checkpoint in segment is larger than reported", zap.Any("current", segment.GetDmlPosition()), zap.Any("reported", cp.GetPosition()))
|
||||
log.Ctx(context.TODO()).Warn("checkpoint in segment is larger than reported", zap.Any("current", segment.GetDmlPosition()), zap.Any("reported", cp.GetPosition()))
|
||||
// segment position in etcd is larger than checkpoint, then dont change it
|
||||
continue
|
||||
}
|
||||
|
@ -992,7 +995,7 @@ func UpdateCheckPointOperator(segmentID int64, checkpoints []*datapb.CheckPoint)
|
|||
|
||||
count := segmentutil.CalcRowCountFromBinLog(segment.SegmentInfo)
|
||||
if count != segment.currRows && count > 0 {
|
||||
log.Info("check point reported inconsistent with bin log row count",
|
||||
log.Ctx(context.TODO()).Info("check point reported inconsistent with bin log row count",
|
||||
zap.Int64("current rows (wrong)", segment.currRows),
|
||||
zap.Int64("segment bin log row count (correct)", count))
|
||||
segment.NumOfRows = count
|
||||
|
@ -1005,7 +1008,7 @@ func UpdateImportedRows(segmentID int64, rows int64) UpdateOperator {
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: update NumOfRows failed - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update NumOfRows failed - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
|
@ -1020,7 +1023,7 @@ func UpdateIsImporting(segmentID int64, isImporting bool) UpdateOperator {
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: update isImporting failed - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update isImporting failed - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
|
@ -1035,12 +1038,12 @@ func UpdateAsDroppedIfEmptyWhenFlushing(segmentID int64) UpdateOperator {
|
|||
return func(modPack *updateSegmentPack) bool {
|
||||
segment := modPack.Get(segmentID)
|
||||
if segment == nil {
|
||||
log.Warn("meta update: update as dropped if empty when flusing failed - segment not found",
|
||||
log.Ctx(context.TODO()).Warn("meta update: update as dropped if empty when flusing failed - segment not found",
|
||||
zap.Int64("segmentID", segmentID))
|
||||
return false
|
||||
}
|
||||
if segment.GetNumOfRows() == 0 && segment.GetState() == commonpb.SegmentState_Flushing {
|
||||
log.Info("meta update: update as dropped if empty when flusing", zap.Int64("segmentID", segmentID))
|
||||
log.Ctx(context.TODO()).Info("meta update: update as dropped if empty when flusing", zap.Int64("segmentID", segmentID))
|
||||
updateSegStateAndPrepareMetrics(segment, commonpb.SegmentState_Dropped, modPack.metricMutation)
|
||||
}
|
||||
return true
|
||||
|
@ -1074,7 +1077,7 @@ func (m *meta) UpdateSegmentsInfo(ctx context.Context, operators ...UpdateOperat
|
|||
increments := lo.Values(updatePack.increments)
|
||||
|
||||
if err := m.catalog.AlterSegments(ctx, segments, increments...); err != nil {
|
||||
log.Error("meta update: update flush segments info - failed to store flush segment info into Etcd",
|
||||
log.Ctx(ctx).Error("meta update: update flush segments info - failed to store flush segment info into Etcd",
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
@ -1084,13 +1087,14 @@ func (m *meta) UpdateSegmentsInfo(ctx context.Context, operators ...UpdateOperat
|
|||
for id, s := range updatePack.segments {
|
||||
m.segments.SetSegment(id, s)
|
||||
}
|
||||
log.Info("meta update: update flush segments info - update flush segments info successfully")
|
||||
log.Ctx(ctx).Info("meta update: update flush segments info - update flush segments info successfully")
|
||||
return nil
|
||||
}
|
||||
|
||||
// UpdateDropChannelSegmentInfo updates segment checkpoints and binlogs before drop
|
||||
// reusing segment info to pass segment id, binlogs, statslog, deltalog, start position and checkpoint
|
||||
func (m *meta) UpdateDropChannelSegmentInfo(ctx context.Context, channel string, segments []*SegmentInfo) error {
|
||||
log := log.Ctx(ctx)
|
||||
log.Debug("meta update: update drop channel segment info",
|
||||
zap.String("channel", channel))
|
||||
m.Lock()
|
||||
|
@ -1145,7 +1149,7 @@ func (m *meta) mergeDropSegment(seg2Drop *SegmentInfo) (*SegmentInfo, *segMetric
|
|||
segment := m.segments.GetSegment(seg2Drop.ID)
|
||||
// healthy check makes sure the Idempotence
|
||||
if segment == nil || !isSegmentHealthy(segment) {
|
||||
log.Warn("UpdateDropChannel skipping nil or unhealthy", zap.Bool("is nil", segment == nil),
|
||||
log.Ctx(context.TODO()).Warn("UpdateDropChannel skipping nil or unhealthy", zap.Bool("is nil", segment == nil),
|
||||
zap.Bool("isHealthy", isSegmentHealthy(segment)))
|
||||
return nil, metricMutation
|
||||
}
|
||||
|
@ -1212,7 +1216,7 @@ func (m *meta) batchSaveDropSegments(ctx context.Context, channel string, modSeg
|
|||
for k := range modSegments {
|
||||
modSegIDs = append(modSegIDs, k)
|
||||
}
|
||||
log.Info("meta update: batch save drop segments",
|
||||
log.Ctx(ctx).Info("meta update: batch save drop segments",
|
||||
zap.Int64s("drop segments", modSegIDs))
|
||||
segments := make([]*datapb.SegmentInfo, 0)
|
||||
for _, seg := range modSegments {
|
||||
|
@ -1333,7 +1337,7 @@ func (m *meta) GetRealSegmentsForChannel(channel string) []*SegmentInfo {
|
|||
|
||||
// AddAllocation add allocation in segment
|
||||
func (m *meta) AddAllocation(segmentID UniqueID, allocation *Allocation) error {
|
||||
log.Debug("meta update: add allocation",
|
||||
log.Ctx(m.ctx).Debug("meta update: add allocation",
|
||||
zap.Int64("segmentID", segmentID),
|
||||
zap.Any("allocation", allocation))
|
||||
m.Lock()
|
||||
|
@ -1341,13 +1345,13 @@ func (m *meta) AddAllocation(segmentID UniqueID, allocation *Allocation) error {
|
|||
curSegInfo := m.segments.GetSegment(segmentID)
|
||||
if curSegInfo == nil {
|
||||
// TODO: Error handling.
|
||||
log.Error("meta update: add allocation failed - segment not found", zap.Int64("segmentID", segmentID))
|
||||
log.Ctx(m.ctx).Error("meta update: add allocation failed - segment not found", zap.Int64("segmentID", segmentID))
|
||||
return errors.New("meta update: add allocation failed - segment not found")
|
||||
}
|
||||
// As we use global segment lastExpire to guarantee data correctness after restart
|
||||
// there is no need to persist allocation to meta store, only update allocation in-memory meta.
|
||||
m.segments.AddAllocation(segmentID, allocation)
|
||||
log.Info("meta update: add allocation - complete", zap.Int64("segmentID", segmentID))
|
||||
log.Ctx(m.ctx).Info("meta update: add allocation - complete", zap.Int64("segmentID", segmentID))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -1449,7 +1453,7 @@ func getMinPosition(positions []*msgpb.MsgPosition) *msgpb.MsgPosition {
|
|||
}
|
||||
|
||||
func (m *meta) completeClusterCompactionMutation(t *datapb.CompactionTask, result *datapb.CompactionPlanResult) ([]*SegmentInfo, *segMetricMutation, error) {
|
||||
log := log.With(zap.Int64("planID", t.GetPlanID()),
|
||||
log := log.Ctx(context.TODO()).With(zap.Int64("planID", t.GetPlanID()),
|
||||
zap.String("type", t.GetType().String()),
|
||||
zap.Int64("collectionID", t.CollectionID),
|
||||
zap.Int64("partitionID", t.PartitionID),
|
||||
|
@ -1527,7 +1531,7 @@ func (m *meta) completeClusterCompactionMutation(t *datapb.CompactionTask, resul
|
|||
}
|
||||
|
||||
func (m *meta) completeMixCompactionMutation(t *datapb.CompactionTask, result *datapb.CompactionPlanResult) ([]*SegmentInfo, *segMetricMutation, error) {
|
||||
log := log.With(zap.Int64("planID", t.GetPlanID()),
|
||||
log := log.Ctx(context.TODO()).With(zap.Int64("planID", t.GetPlanID()),
|
||||
zap.String("type", t.GetType().String()),
|
||||
zap.Int64("collectionID", t.CollectionID),
|
||||
zap.Int64("partitionID", t.PartitionID),
|
||||
|
@ -1705,7 +1709,7 @@ func (m *meta) UpdateChannelCheckpoint(ctx context.Context, vChannel string, pos
|
|||
}
|
||||
m.channelCPs.checkpoints[vChannel] = pos
|
||||
ts, _ := tsoutil.ParseTS(pos.Timestamp)
|
||||
log.Info("UpdateChannelCheckpoint done",
|
||||
log.Ctx(context.TODO()).Info("UpdateChannelCheckpoint done",
|
||||
zap.String("vChannel", vChannel),
|
||||
zap.Uint64("ts", pos.GetTimestamp()),
|
||||
zap.ByteString("msgID", pos.GetMsgID()),
|
||||
|
@ -1740,6 +1744,7 @@ func (m *meta) MarkChannelCheckpointDropped(ctx context.Context, channel string)
|
|||
|
||||
// UpdateChannelCheckpoints updates and saves channel checkpoints.
|
||||
func (m *meta) UpdateChannelCheckpoints(ctx context.Context, positions []*msgpb.MsgPosition) error {
|
||||
log := log.Ctx(ctx)
|
||||
m.channelCPs.Lock()
|
||||
defer m.channelCPs.Unlock()
|
||||
toUpdates := lo.Filter(positions, func(pos *msgpb.MsgPosition, _ int) bool {
|
||||
|
@ -1786,7 +1791,7 @@ func (m *meta) DropChannelCheckpoint(vChannel string) error {
|
|||
}
|
||||
delete(m.channelCPs.checkpoints, vChannel)
|
||||
metrics.DataCoordCheckpointUnixSeconds.DeleteLabelValues(fmt.Sprint(paramtable.GetNodeID()), vChannel)
|
||||
log.Info("DropChannelCheckpoint done", zap.String("vChannel", vChannel))
|
||||
log.Ctx(context.TODO()).Info("DropChannelCheckpoint done", zap.String("vChannel", vChannel))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -1899,7 +1904,7 @@ func isFlushState(state commonpb.SegmentState) bool {
|
|||
|
||||
// updateSegStateAndPrepareMetrics updates a segment's in-memory state and prepare for the corresponding metric update.
|
||||
func updateSegStateAndPrepareMetrics(segToUpdate *SegmentInfo, targetState commonpb.SegmentState, metricMutation *segMetricMutation) {
|
||||
log.Debug("updating segment state and updating metrics",
|
||||
log.Ctx(context.TODO()).Debug("updating segment state and updating metrics",
|
||||
zap.Int64("segmentID", segToUpdate.GetID()),
|
||||
zap.String("old state", segToUpdate.GetState().String()),
|
||||
zap.String("new state", targetState.String()),
|
||||
|
@ -1958,7 +1963,7 @@ func (m *meta) CleanPartitionStatsInfo(ctx context.Context, info *datapb.Partiti
|
|||
}
|
||||
}
|
||||
|
||||
log.Debug("remove clustering compaction stats files",
|
||||
log.Ctx(ctx).Debug("remove clustering compaction stats files",
|
||||
zap.Int64("collectionID", info.GetCollectionID()),
|
||||
zap.Int64("partitionID", info.GetPartitionID()),
|
||||
zap.String("vChannel", info.GetVChannel()),
|
||||
|
@ -1966,19 +1971,19 @@ func (m *meta) CleanPartitionStatsInfo(ctx context.Context, info *datapb.Partiti
|
|||
zap.Strings("removePaths", removePaths))
|
||||
err := m.chunkManager.MultiRemove(context.Background(), removePaths)
|
||||
if err != nil {
|
||||
log.Warn("remove clustering compaction stats files failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("remove clustering compaction stats files failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
// first clean analyze task
|
||||
if err = m.analyzeMeta.DropAnalyzeTask(ctx, info.GetAnalyzeTaskID()); err != nil {
|
||||
log.Warn("remove analyze task failed", zap.Int64("analyzeTaskID", info.GetAnalyzeTaskID()), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("remove analyze task failed", zap.Int64("analyzeTaskID", info.GetAnalyzeTaskID()), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
// finally, clean up the partition stats info, and make sure the analysis task is cleaned up
|
||||
err = m.partitionStatsMeta.DropPartitionStatsInfo(ctx, info)
|
||||
log.Debug("drop partition stats meta",
|
||||
log.Ctx(ctx).Debug("drop partition stats meta",
|
||||
zap.Int64("collectionID", info.GetCollectionID()),
|
||||
zap.Int64("partitionID", info.GetPartitionID()),
|
||||
zap.String("vChannel", info.GetVChannel()),
|
||||
|
@ -1993,7 +1998,7 @@ func (m *meta) SaveStatsResultSegment(oldSegmentID int64, result *workerpb.Stats
|
|||
m.Lock()
|
||||
defer m.Unlock()
|
||||
|
||||
log := log.With(zap.Int64("collectionID", result.GetCollectionID()),
|
||||
log := log.Ctx(m.ctx).With(zap.Int64("collectionID", result.GetCollectionID()),
|
||||
zap.Int64("partitionID", result.GetPartitionID()),
|
||||
zap.Int64("old segmentID", oldSegmentID),
|
||||
zap.Int64("target segmentID", result.GetSegmentID()))
|
||||
|
|
|
@ -248,6 +248,7 @@ func (s *Server) QuitSignal() <-chan struct{} {
|
|||
|
||||
// Register registers data service at etcd
|
||||
func (s *Server) Register() error {
|
||||
log := log.Ctx(s.ctx)
|
||||
// first register indexCoord
|
||||
s.icSession.Register()
|
||||
s.session.Register()
|
||||
|
@ -301,6 +302,7 @@ func (s *Server) initSession() error {
|
|||
|
||||
// Init change server state to Initializing
|
||||
func (s *Server) Init() error {
|
||||
log := log.Ctx(s.ctx)
|
||||
var err error
|
||||
s.registerMetricsRequest()
|
||||
s.factory.Init(Params)
|
||||
|
@ -345,6 +347,7 @@ func (s *Server) RegisterStreamingCoordGRPCService(server *grpc.Server) {
|
|||
}
|
||||
|
||||
func (s *Server) initDataCoord() error {
|
||||
log := log.Ctx(s.ctx)
|
||||
s.stateCode.Store(commonpb.StateCode_Initializing)
|
||||
var err error
|
||||
if err = s.initRootCoordClient(); err != nil {
|
||||
|
@ -438,6 +441,7 @@ func (s *Server) initDataCoord() error {
|
|||
// datanodes etcd watch, etcd alive check and flush completed status check
|
||||
// 4. set server state to Healthy
|
||||
func (s *Server) Start() error {
|
||||
log := log.Ctx(s.ctx)
|
||||
if !s.enableActiveStandBy {
|
||||
s.startDataCoord()
|
||||
log.Info("DataCoord startup successfully")
|
||||
|
@ -573,6 +577,7 @@ func (s *Server) initGarbageCollection(cli storage.ChunkManager) {
|
|||
}
|
||||
|
||||
func (s *Server) initServiceDiscovery() error {
|
||||
log := log.Ctx(s.ctx)
|
||||
r := semver.MustParseRange(">=2.2.3")
|
||||
sessions, rev, err := s.session.GetSessionsWithVersionRange(typeutil.DataNodeRole, r)
|
||||
if err != nil {
|
||||
|
@ -778,6 +783,7 @@ func (s *Server) startTaskScheduler() {
|
|||
}
|
||||
|
||||
func (s *Server) updateSegmentStatistics(ctx context.Context, stats []*commonpb.SegmentStats) {
|
||||
log := log.Ctx(ctx)
|
||||
for _, stat := range stats {
|
||||
segment := s.meta.GetSegment(ctx, stat.GetSegmentID())
|
||||
if segment == nil {
|
||||
|
@ -807,6 +813,7 @@ func (s *Server) updateSegmentStatistics(ctx context.Context, stats []*commonpb.
|
|||
}
|
||||
|
||||
func (s *Server) getFlushableSegmentsInfo(ctx context.Context, flushableIDs []int64) []*SegmentInfo {
|
||||
log := log.Ctx(ctx)
|
||||
res := make([]*SegmentInfo, 0, len(flushableIDs))
|
||||
for _, id := range flushableIDs {
|
||||
sinfo := s.meta.GetHealthySegment(ctx, id)
|
||||
|
@ -832,7 +839,7 @@ func (s *Server) startWatchService(ctx context.Context) {
|
|||
|
||||
func (s *Server) stopServiceWatch() {
|
||||
// ErrCompacted is handled inside SessionWatcher, which means there is some other error occurred, closing server.
|
||||
logutil.Logger(s.ctx).Error("watch service channel closed", zap.Int64("serverID", paramtable.GetNodeID()))
|
||||
log.Ctx(s.ctx).Error("watch service channel closed", zap.Int64("serverID", paramtable.GetNodeID()))
|
||||
go s.Stop()
|
||||
if s.session.IsTriggerKill() {
|
||||
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
||||
|
@ -843,6 +850,7 @@ func (s *Server) stopServiceWatch() {
|
|||
|
||||
// watchService watches services.
|
||||
func (s *Server) watchService(ctx context.Context) {
|
||||
log := log.Ctx(ctx)
|
||||
defer logutil.LogPanic()
|
||||
defer s.serverLoopWg.Done()
|
||||
for {
|
||||
|
@ -998,11 +1006,11 @@ func (s *Server) startFlushLoop(ctx context.Context) {
|
|||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
logutil.Logger(s.ctx).Info("flush loop shutdown")
|
||||
log.Ctx(s.ctx).Info("flush loop shutdown")
|
||||
return
|
||||
case segmentID := <-s.flushCh:
|
||||
// Ignore return error
|
||||
log.Info("flush successfully", zap.Any("segmentID", segmentID))
|
||||
log.Ctx(ctx).Info("flush successfully", zap.Any("segmentID", segmentID))
|
||||
err := s.postFlush(ctx, segmentID)
|
||||
if err != nil {
|
||||
log.Warn("failed to do post flush", zap.Int64("segmentID", segmentID), zap.Error(err))
|
||||
|
@ -1086,12 +1094,13 @@ func (s *Server) initRootCoordClient() error {
|
|||
//
|
||||
// stop message stream client and stop server loops
|
||||
func (s *Server) Stop() error {
|
||||
log := log.Ctx(s.ctx)
|
||||
if !s.stateCode.CompareAndSwap(commonpb.StateCode_Healthy, commonpb.StateCode_Abnormal) {
|
||||
return nil
|
||||
}
|
||||
logutil.Logger(s.ctx).Info("datacoord server shutdown")
|
||||
log.Info("datacoord server shutdown")
|
||||
s.garbageCollector.close()
|
||||
logutil.Logger(s.ctx).Info("datacoord garbage collector stopped")
|
||||
log.Info("datacoord garbage collector stopped")
|
||||
|
||||
if s.streamingCoord != nil {
|
||||
log.Info("StreamingCoord stoping...")
|
||||
|
@ -1106,16 +1115,16 @@ func (s *Server) Stop() error {
|
|||
s.syncSegmentsScheduler.Stop()
|
||||
|
||||
s.stopCompaction()
|
||||
logutil.Logger(s.ctx).Info("datacoord compaction stopped")
|
||||
log.Info("datacoord compaction stopped")
|
||||
|
||||
s.jobManager.Stop()
|
||||
logutil.Logger(s.ctx).Info("datacoord statsJobManager stopped")
|
||||
log.Info("datacoord statsJobManager stopped")
|
||||
|
||||
s.taskScheduler.Stop()
|
||||
logutil.Logger(s.ctx).Info("datacoord index builder stopped")
|
||||
log.Info("datacoord index builder stopped")
|
||||
|
||||
s.cluster.Close()
|
||||
logutil.Logger(s.ctx).Info("datacoord cluster stopped")
|
||||
log.Info("datacoord cluster stopped")
|
||||
|
||||
if s.session != nil {
|
||||
s.session.Stop()
|
||||
|
@ -1126,14 +1135,14 @@ func (s *Server) Stop() error {
|
|||
}
|
||||
|
||||
s.stopServerLoop()
|
||||
logutil.Logger(s.ctx).Info("datacoord serverloop stopped")
|
||||
logutil.Logger(s.ctx).Warn("datacoord stop successful")
|
||||
log.Info("datacoord serverloop stopped")
|
||||
log.Warn("datacoord stop successful")
|
||||
return nil
|
||||
}
|
||||
|
||||
// CleanMeta only for test
|
||||
func (s *Server) CleanMeta() error {
|
||||
log.Debug("clean meta", zap.Any("kv", s.kv))
|
||||
log.Ctx(s.ctx).Debug("clean meta", zap.Any("kv", s.kv))
|
||||
err := s.kv.RemoveWithPrefix(s.ctx, "")
|
||||
err2 := s.watchClient.RemoveWithPrefix(s.ctx, "")
|
||||
if err2 != nil {
|
||||
|
@ -1201,7 +1210,7 @@ func (s *Server) registerMetricsRequest() {
|
|||
}
|
||||
return s.meta.indexMeta.GetIndexJSON(collectionID), nil
|
||||
})
|
||||
log.Info("register metrics actions finished")
|
||||
log.Ctx(s.ctx).Info("register metrics actions finished")
|
||||
}
|
||||
|
||||
// loadCollectionFromRootCoord communicates with RootCoord and asks for collection information.
|
||||
|
@ -1258,7 +1267,7 @@ func (s *Server) updateBalanceConfigLoop(ctx context.Context) {
|
|||
for {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
log.Info("update balance config loop exit!")
|
||||
log.Ctx(ctx).Info("update balance config loop exit!")
|
||||
return
|
||||
|
||||
case <-ticker.C:
|
||||
|
@ -1272,6 +1281,7 @@ func (s *Server) updateBalanceConfigLoop(ctx context.Context) {
|
|||
}
|
||||
|
||||
func (s *Server) updateBalanceConfig() bool {
|
||||
log := log.Ctx(s.ctx)
|
||||
r := semver.MustParseRange("<2.3.0")
|
||||
sessions, _, err := s.session.GetSessionsWithVersionRange(typeutil.DataNodeRole, r)
|
||||
if err != nil {
|
||||
|
|
|
@ -690,7 +690,7 @@ func (s *Server) GetStateCode() commonpb.StateCode {
|
|||
// GetComponentStates returns DataCoord's current state
|
||||
func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
|
||||
code := s.GetStateCode()
|
||||
log.Debug("DataCoord current state", zap.String("StateCode", code.String()))
|
||||
log.Ctx(ctx).Debug("DataCoord current state", zap.String("StateCode", code.String()))
|
||||
nodeID := common.NotRegisteredID
|
||||
if s.session != nil && s.session.Registered() {
|
||||
nodeID = s.session.GetServerID() // or Params.NodeID
|
||||
|
@ -1528,12 +1528,12 @@ func (s *Server) handleDataNodeTtMsg(ctx context.Context, ttMsg *msgpb.DataNodeT
|
|||
// An error status will be returned and error will be logged, if we failed to mark *all* segments.
|
||||
// Deprecated, do not use it
|
||||
func (s *Server) MarkSegmentsDropped(ctx context.Context, req *datapb.MarkSegmentsDroppedRequest) (*commonpb.Status, error) {
|
||||
log.Info("marking segments dropped", zap.Int64s("segments", req.GetSegmentIds()))
|
||||
log.Ctx(ctx).Info("marking segments dropped", zap.Int64s("segments", req.GetSegmentIds()))
|
||||
var err error
|
||||
for _, segID := range req.GetSegmentIds() {
|
||||
if err = s.meta.SetState(ctx, segID, commonpb.SegmentState_Dropped); err != nil {
|
||||
// Fail-open.
|
||||
log.Error("failed to set segment state as dropped", zap.Int64("segmentID", segID))
|
||||
log.Ctx(ctx).Error("failed to set segment state as dropped", zap.Int64("segmentID", segID))
|
||||
break
|
||||
}
|
||||
}
|
||||
|
@ -1664,7 +1664,7 @@ func (s *Server) ImportV2(ctx context.Context, in *internalpb.ImportRequestInter
|
|||
Status: merr.Success(),
|
||||
}
|
||||
|
||||
log := log.With(zap.Int64("collection", in.GetCollectionID()),
|
||||
log := log.Ctx(ctx).With(zap.Int64("collection", in.GetCollectionID()),
|
||||
zap.Int64s("partitions", in.GetPartitionIDs()),
|
||||
zap.Strings("channels", in.GetChannelNames()))
|
||||
log.Info("receive import request", zap.Any("files", in.GetFiles()), zap.Any("options", in.GetOptions()))
|
||||
|
@ -1749,7 +1749,7 @@ func (s *Server) ImportV2(ctx context.Context, in *internalpb.ImportRequestInter
|
|||
}
|
||||
|
||||
func (s *Server) GetImportProgress(ctx context.Context, in *internalpb.GetImportProgressRequest) (*internalpb.GetImportProgressResponse, error) {
|
||||
log := log.With(zap.String("jobID", in.GetJobID()))
|
||||
log := log.Ctx(ctx).With(zap.String("jobID", in.GetJobID()))
|
||||
if err := merr.CheckHealthy(s.GetStateCode()); err != nil {
|
||||
return &internalpb.GetImportProgressResponse{
|
||||
Status: merr.Status(err),
|
||||
|
|
|
@ -71,6 +71,7 @@ func NewNodeManager(ctx context.Context, indexNodeCreator IndexNodeCreatorFunc)
|
|||
|
||||
// SetClient sets IndexNode client to node manager.
|
||||
func (nm *IndexNodeManager) SetClient(nodeID typeutil.UniqueID, client types.IndexNodeClient) {
|
||||
log := log.Ctx(nm.ctx)
|
||||
log.Debug("set IndexNode client", zap.Int64("nodeID", nodeID))
|
||||
nm.lock.Lock()
|
||||
defer nm.lock.Unlock()
|
||||
|
@ -81,7 +82,7 @@ func (nm *IndexNodeManager) SetClient(nodeID typeutil.UniqueID, client types.Ind
|
|||
|
||||
// RemoveNode removes the unused client of IndexNode.
|
||||
func (nm *IndexNodeManager) RemoveNode(nodeID typeutil.UniqueID) {
|
||||
log.Debug("remove IndexNode", zap.Int64("nodeID", nodeID))
|
||||
log.Ctx(nm.ctx).Debug("remove IndexNode", zap.Int64("nodeID", nodeID))
|
||||
nm.lock.Lock()
|
||||
defer nm.lock.Unlock()
|
||||
delete(nm.nodeClients, nodeID)
|
||||
|
@ -90,7 +91,7 @@ func (nm *IndexNodeManager) RemoveNode(nodeID typeutil.UniqueID) {
|
|||
}
|
||||
|
||||
func (nm *IndexNodeManager) StoppingNode(nodeID typeutil.UniqueID) {
|
||||
log.Debug("IndexCoord", zap.Int64("Stopping node with ID", nodeID))
|
||||
log.Ctx(nm.ctx).Debug("IndexCoord", zap.Int64("Stopping node with ID", nodeID))
|
||||
nm.lock.Lock()
|
||||
defer nm.lock.Unlock()
|
||||
nm.stoppingNodes[nodeID] = struct{}{}
|
||||
|
@ -98,7 +99,7 @@ func (nm *IndexNodeManager) StoppingNode(nodeID typeutil.UniqueID) {
|
|||
|
||||
// AddNode adds the client of IndexNode.
|
||||
func (nm *IndexNodeManager) AddNode(nodeID typeutil.UniqueID, address string) error {
|
||||
log.Debug("add IndexNode", zap.Int64("nodeID", nodeID), zap.String("node address", address))
|
||||
log.Ctx(nm.ctx).Debug("add IndexNode", zap.Int64("nodeID", nodeID), zap.String("node address", address))
|
||||
var (
|
||||
nodeClient types.IndexNodeClient
|
||||
err error
|
||||
|
@ -106,7 +107,7 @@ func (nm *IndexNodeManager) AddNode(nodeID typeutil.UniqueID, address string) er
|
|||
|
||||
nodeClient, err = nm.indexNodeCreator(context.TODO(), address, nodeID)
|
||||
if err != nil {
|
||||
log.Error("create IndexNode client fail", zap.Error(err))
|
||||
log.Ctx(nm.ctx).Error("create IndexNode client fail", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -126,6 +127,7 @@ func (nm *IndexNodeManager) PickClient() (typeutil.UniqueID, types.IndexNodeClie
|
|||
wg = sync.WaitGroup{}
|
||||
)
|
||||
|
||||
log := log.Ctx(ctx)
|
||||
for nodeID, client := range nm.nodeClients {
|
||||
if _, ok := nm.stoppingNodes[nodeID]; !ok {
|
||||
nodeID := nodeID
|
||||
|
@ -167,6 +169,7 @@ func (nm *IndexNodeManager) PickClient() (typeutil.UniqueID, types.IndexNodeClie
|
|||
}
|
||||
|
||||
func (nm *IndexNodeManager) ClientSupportDisk() bool {
|
||||
log := log.Ctx(nm.ctx)
|
||||
log.Debug("check if client support disk index")
|
||||
allClients := nm.GetAllClients()
|
||||
if len(allClients) == 0 {
|
||||
|
|
|
@ -432,6 +432,7 @@ func (t *clusteringCompactionTask) switchPolicyForVectorPlan(centroids *clusteri
|
|||
func (t *clusteringCompactionTask) getVectorAnalyzeResult(ctx context.Context) error {
|
||||
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, fmt.Sprintf("getVectorAnalyzeResult-%d", t.GetPlanID()))
|
||||
defer span.End()
|
||||
log := log.Ctx(ctx)
|
||||
analyzeResultPath := t.plan.AnalyzeResultPath
|
||||
centroidFilePath := path.Join(analyzeResultPath, metautil.JoinIDPath(t.collectionID, t.partitionID, t.clusteringKeyField.FieldID), common.Centroids)
|
||||
offsetMappingFiles := make(map[int64]string, 0)
|
||||
|
@ -464,6 +465,7 @@ func (t *clusteringCompactionTask) mapping(ctx context.Context,
|
|||
defer span.End()
|
||||
inputSegments := t.plan.GetSegmentBinlogs()
|
||||
mapStart := time.Now()
|
||||
log := log.Ctx(ctx)
|
||||
|
||||
// start flush goroutine
|
||||
go t.backgroundFlush(ctx)
|
||||
|
@ -928,6 +930,7 @@ func (t *clusteringCompactionTask) packBufferToSegment(ctx context.Context, buff
|
|||
return nil
|
||||
}
|
||||
|
||||
log := log.Ctx(ctx)
|
||||
binlogNum := 0
|
||||
numRows := buffer.flushedRowNum[segmentID]
|
||||
insertLogs := make([]*datapb.FieldBinlog, 0)
|
||||
|
@ -1375,6 +1378,7 @@ func (t *clusteringCompactionTask) GetSlotUsage() int64 {
|
|||
}
|
||||
|
||||
func (t *clusteringCompactionTask) checkBuffersAfterCompaction() error {
|
||||
log := log.Ctx(t.ctx)
|
||||
for _, buffer := range t.clusterBuffers {
|
||||
if len(buffer.flushedBinlogs) != 0 {
|
||||
log.Warn("there are some binlogs have leaked, please check", zap.Int("buffer id", buffer.id),
|
||||
|
|
|
@ -52,6 +52,7 @@ func isExpiredEntity(ttl int64, now, ts typeutil.Timestamp) bool {
|
|||
func mergeDeltalogs(ctx context.Context, io io.BinlogIO, paths []string) (map[interface{}]typeutil.Timestamp, error) {
|
||||
pk2ts := make(map[interface{}]typeutil.Timestamp)
|
||||
|
||||
log := log.Ctx(ctx)
|
||||
if len(paths) == 0 {
|
||||
log.Debug("compact with no deltalogs, skip merge deltalogs")
|
||||
return pk2ts, nil
|
||||
|
|
|
@ -58,7 +58,6 @@ import (
|
|||
"github.com/milvus-io/milvus/pkg/mq/msgdispatcher"
|
||||
"github.com/milvus-io/milvus/pkg/util/conc"
|
||||
"github.com/milvus-io/milvus/pkg/util/expr"
|
||||
"github.com/milvus-io/milvus/pkg/util/logutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/retry"
|
||||
|
@ -193,6 +192,7 @@ func (node *DataNode) SetDataCoordClient(ds types.DataCoordClient) error {
|
|||
|
||||
// Register register datanode to etcd
|
||||
func (node *DataNode) Register() error {
|
||||
log := log.Ctx(node.ctx)
|
||||
log.Debug("node begin to register to etcd", zap.String("serverName", node.session.ServerName), zap.Int64("ServerID", node.session.ServerID))
|
||||
node.session.Register()
|
||||
|
||||
|
@ -228,7 +228,7 @@ func (node *DataNode) Init() error {
|
|||
var initError error
|
||||
node.initOnce.Do(func() {
|
||||
node.registerMetricsRequest()
|
||||
logutil.Logger(node.ctx).Info("DataNode server initializing",
|
||||
log.Ctx(node.ctx).Info("DataNode server initializing",
|
||||
zap.String("TimeTickChannelName", Params.CommonCfg.DataCoordTimeTick.GetValue()),
|
||||
)
|
||||
if err := node.initSession(); err != nil {
|
||||
|
@ -299,12 +299,12 @@ func (node *DataNode) registerMetricsRequest() {
|
|||
func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) {
|
||||
return node.flowgraphManager.GetChannelsJSON(), nil
|
||||
})
|
||||
log.Info("register metrics actions finished")
|
||||
log.Ctx(node.ctx).Info("register metrics actions finished")
|
||||
}
|
||||
|
||||
// tryToReleaseFlowgraph tries to release a flowgraph
|
||||
func (node *DataNode) tryToReleaseFlowgraph(channel string) {
|
||||
log.Info("try to release flowgraph", zap.String("channel", channel))
|
||||
log.Ctx(node.ctx).Info("try to release flowgraph", zap.String("channel", channel))
|
||||
if node.compactionExecutor != nil {
|
||||
node.compactionExecutor.DiscardPlan(channel)
|
||||
}
|
||||
|
@ -318,6 +318,7 @@ func (node *DataNode) tryToReleaseFlowgraph(channel string) {
|
|||
|
||||
// Start will update DataNode state to HEALTHY
|
||||
func (node *DataNode) Start() error {
|
||||
log := log.Ctx(node.ctx)
|
||||
var startErr error
|
||||
node.startOnce.Do(func() {
|
||||
if err := node.allocator.Start(); err != nil {
|
||||
|
@ -402,7 +403,7 @@ func (node *DataNode) Stop() error {
|
|||
}
|
||||
|
||||
if node.allocator != nil {
|
||||
log.Info("close id allocator", zap.String("role", typeutil.DataNodeRole))
|
||||
log.Ctx(node.ctx).Info("close id allocator", zap.String("role", typeutil.DataNodeRole))
|
||||
node.allocator.Close()
|
||||
}
|
||||
|
||||
|
|
|
@ -51,7 +51,7 @@ import (
|
|||
|
||||
// WatchDmChannels is not in use
|
||||
func (node *DataNode) WatchDmChannels(ctx context.Context, in *datapb.WatchDmChannelsRequest) (*commonpb.Status, error) {
|
||||
log.Warn("DataNode WatchDmChannels is not in use")
|
||||
log.Ctx(ctx).Warn("DataNode WatchDmChannels is not in use")
|
||||
|
||||
// TODO ERROR OF GRPC NOT IN USE
|
||||
return merr.Success(), nil
|
||||
|
@ -61,7 +61,7 @@ func (node *DataNode) WatchDmChannels(ctx context.Context, in *datapb.WatchDmCha
|
|||
func (node *DataNode) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) {
|
||||
nodeID := common.NotRegisteredID
|
||||
state := node.stateCode.Load().(commonpb.StateCode)
|
||||
log.Debug("DataNode current state", zap.String("State", state.String()))
|
||||
log.Ctx(ctx).Debug("DataNode current state", zap.String("State", state.String()))
|
||||
if node.GetSession() != nil && node.session.Registered() {
|
||||
nodeID = node.GetSession().ServerID
|
||||
}
|
||||
|
@ -133,9 +133,9 @@ func (node *DataNode) GetStatisticsChannel(ctx context.Context, req *internalpb.
|
|||
|
||||
// ShowConfigurations returns the configurations of DataNode matching req.Pattern
|
||||
func (node *DataNode) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) {
|
||||
log.Debug("DataNode.ShowConfigurations", zap.String("pattern", req.Pattern))
|
||||
log.Ctx(ctx).Debug("DataNode.ShowConfigurations", zap.String("pattern", req.Pattern))
|
||||
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
||||
log.Warn("DataNode.ShowConfigurations failed", zap.Int64("nodeId", node.GetNodeID()), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("DataNode.ShowConfigurations failed", zap.Int64("nodeId", node.GetNodeID()), zap.Error(err))
|
||||
|
||||
return &internalpb.ShowConfigurationsResponse{
|
||||
Status: merr.Status(err),
|
||||
|
@ -160,7 +160,7 @@ func (node *DataNode) ShowConfigurations(ctx context.Context, req *internalpb.Sh
|
|||
// GetMetrics return datanode metrics
|
||||
func (node *DataNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) {
|
||||
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
||||
log.Warn("DataNode.GetMetrics failed", zap.Int64("nodeId", node.GetNodeID()), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("DataNode.GetMetrics failed", zap.Int64("nodeId", node.GetNodeID()), zap.Error(err))
|
||||
|
||||
return &milvuspb.GetMetricsResponse{
|
||||
Status: merr.Status(err),
|
||||
|
@ -252,7 +252,7 @@ func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPl
|
|||
// return status of all compaction plans
|
||||
func (node *DataNode) GetCompactionState(ctx context.Context, req *datapb.CompactionStateRequest) (*datapb.CompactionStateResponse, error) {
|
||||
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
|
||||
log.Warn("DataNode.GetCompactionState failed", zap.Int64("nodeId", node.GetNodeID()), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("DataNode.GetCompactionState failed", zap.Int64("nodeId", node.GetNodeID()), zap.Error(err))
|
||||
return &datapb.CompactionStateResponse{
|
||||
Status: merr.Status(err),
|
||||
}, nil
|
||||
|
@ -351,9 +351,9 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments
|
|||
}
|
||||
|
||||
func (node *DataNode) NotifyChannelOperation(ctx context.Context, req *datapb.ChannelOperationsRequest) (*commonpb.Status, error) {
|
||||
log.Ctx(ctx).Info("DataNode receives NotifyChannelOperation",
|
||||
zap.Int("operation count", len(req.GetInfos())))
|
||||
log := log.Ctx(ctx).With(zap.Int("operation count", len(req.GetInfos())))
|
||||
|
||||
log.Info("DataNode receives NotifyChannelOperation")
|
||||
if node.channelManager == nil {
|
||||
log.Warn("DataNode NotifyChannelOperation failed due to nil channelManager")
|
||||
return merr.Status(merr.WrapErrServiceInternal("channelManager is nil! Ignore if you are upgrading datanode/coord to rpc based watch")), nil
|
||||
|
|
|
@ -96,6 +96,7 @@ func (cm *ConnectionManager) AddDependency(roleName string) error {
|
|||
return errors.New("roleName is illegal")
|
||||
}
|
||||
|
||||
log := log.Ctx(context.TODO())
|
||||
_, ok := cm.dependencies[roleName]
|
||||
if ok {
|
||||
log.Warn("Dependency is already added", zap.String("roleName", roleName))
|
||||
|
@ -132,7 +133,7 @@ func (cm *ConnectionManager) GetRootCoordClient() (rootcoordpb.RootCoordClient,
|
|||
defer cm.rootCoordMu.RUnlock()
|
||||
_, ok := cm.dependencies[typeutil.RootCoordRole]
|
||||
if !ok {
|
||||
log.Error("RootCoord dependency has not been added yet")
|
||||
log.Ctx(context.TODO()).Error("RootCoord dependency has not been added yet")
|
||||
return nil, false
|
||||
}
|
||||
|
||||
|
@ -144,7 +145,7 @@ func (cm *ConnectionManager) GetQueryCoordClient() (querypb.QueryCoordClient, bo
|
|||
defer cm.queryCoordMu.RUnlock()
|
||||
_, ok := cm.dependencies[typeutil.QueryCoordRole]
|
||||
if !ok {
|
||||
log.Error("QueryCoord dependency has not been added yet")
|
||||
log.Ctx(context.TODO()).Error("QueryCoord dependency has not been added yet")
|
||||
return nil, false
|
||||
}
|
||||
|
||||
|
@ -156,7 +157,7 @@ func (cm *ConnectionManager) GetDataCoordClient() (datapb.DataCoordClient, bool)
|
|||
defer cm.dataCoordMu.RUnlock()
|
||||
_, ok := cm.dependencies[typeutil.DataCoordRole]
|
||||
if !ok {
|
||||
log.Error("DataCoord dependency has not been added yet")
|
||||
log.Ctx(context.TODO()).Error("DataCoord dependency has not been added yet")
|
||||
return nil, false
|
||||
}
|
||||
|
||||
|
@ -168,7 +169,7 @@ func (cm *ConnectionManager) GetQueryNodeClients() (map[int64]querypb.QueryNodeC
|
|||
defer cm.queryNodesMu.RUnlock()
|
||||
_, ok := cm.dependencies[typeutil.QueryNodeRole]
|
||||
if !ok {
|
||||
log.Error("QueryNode dependency has not been added yet")
|
||||
log.Ctx(context.TODO()).Error("QueryNode dependency has not been added yet")
|
||||
return nil, false
|
||||
}
|
||||
|
||||
|
@ -180,7 +181,7 @@ func (cm *ConnectionManager) GetDataNodeClients() (map[int64]datapb.DataNodeClie
|
|||
defer cm.dataNodesMu.RUnlock()
|
||||
_, ok := cm.dependencies[typeutil.DataNodeRole]
|
||||
if !ok {
|
||||
log.Error("DataNode dependency has not been added yet")
|
||||
log.Ctx(context.TODO()).Error("DataNode dependency has not been added yet")
|
||||
return nil, false
|
||||
}
|
||||
|
||||
|
@ -192,7 +193,7 @@ func (cm *ConnectionManager) GetIndexNodeClients() (map[int64]workerpb.IndexNode
|
|||
defer cm.indexNodesMu.RUnlock()
|
||||
_, ok := cm.dependencies[typeutil.IndexNodeRole]
|
||||
if !ok {
|
||||
log.Error("IndexNode dependency has not been added yet")
|
||||
log.Ctx(context.TODO()).Error("IndexNode dependency has not been added yet")
|
||||
return nil, false
|
||||
}
|
||||
|
||||
|
@ -223,7 +224,7 @@ func (cm *ConnectionManager) processEvent(channel <-chan *sessionutil.SessionEve
|
|||
}
|
||||
case ev, ok := <-channel:
|
||||
if !ok {
|
||||
log.Error("watch service channel closed", zap.Int64("serverID", cm.session.ServerID))
|
||||
log.Ctx(context.TODO()).Error("watch service channel closed", zap.Int64("serverID", cm.session.ServerID))
|
||||
go cm.Stop()
|
||||
if cm.session.TriggerKill {
|
||||
if p, err := os.FindProcess(os.Getpid()); err == nil {
|
||||
|
@ -234,7 +235,7 @@ func (cm *ConnectionManager) processEvent(channel <-chan *sessionutil.SessionEve
|
|||
}
|
||||
switch ev.EventType {
|
||||
case sessionutil.SessionAddEvent:
|
||||
log.Debug("ConnectionManager", zap.Any("add event", ev.Session))
|
||||
log.Ctx(context.TODO()).Debug("ConnectionManager", zap.Any("add event", ev.Session))
|
||||
cm.buildConnections(ev.Session)
|
||||
case sessionutil.SessionDelEvent:
|
||||
cm.removeTask(ev.Session.ServerID)
|
||||
|
@ -245,6 +246,7 @@ func (cm *ConnectionManager) processEvent(channel <-chan *sessionutil.SessionEve
|
|||
}
|
||||
|
||||
func (cm *ConnectionManager) receiveFinishTask() {
|
||||
log := log.Ctx(context.TODO())
|
||||
for {
|
||||
select {
|
||||
case _, ok := <-cm.closeCh:
|
||||
|
@ -377,7 +379,7 @@ func (bct *buildClientTask) Run() {
|
|||
defer bct.finish()
|
||||
connectGrpcFunc := func() error {
|
||||
opts := tracer.GetInterceptorOpts()
|
||||
log.Debug("Grpc connect", zap.String("Address", bct.sess.Address))
|
||||
log.Ctx(bct.ctx).Debug("Grpc connect", zap.String("Address", bct.sess.Address))
|
||||
ctx, cancel := context.WithTimeout(bct.ctx, 30*time.Second)
|
||||
defer cancel()
|
||||
conn, err := grpc.DialContext(ctx, bct.sess.Address,
|
||||
|
@ -410,9 +412,9 @@ func (bct *buildClientTask) Run() {
|
|||
}
|
||||
|
||||
err := retry.Do(bct.ctx, connectGrpcFunc, bct.retryOptions...)
|
||||
log.Debug("ConnectionManager", zap.Int64("build connection finish", bct.sess.ServerID))
|
||||
log.Ctx(bct.ctx).Debug("ConnectionManager", zap.Int64("build connection finish", bct.sess.ServerID))
|
||||
if err != nil {
|
||||
log.Debug("BuildClientTask try connect failed",
|
||||
log.Ctx(bct.ctx).Debug("BuildClientTask try connect failed",
|
||||
zap.String("roleName", bct.sess.ServerName), zap.Error(err))
|
||||
bct.state = buildClientFailed
|
||||
return
|
||||
|
@ -425,7 +427,7 @@ func (bct *buildClientTask) Stop() {
|
|||
}
|
||||
|
||||
func (bct *buildClientTask) finish() {
|
||||
log.Debug("ConnectionManager", zap.Int64("notify connection finish", bct.sess.ServerID))
|
||||
log.Ctx(bct.ctx).Debug("ConnectionManager", zap.Int64("notify connection finish", bct.sess.ServerID))
|
||||
bct.notify <- bct.sess.ServerID
|
||||
}
|
||||
|
||||
|
|
|
@ -284,8 +284,8 @@ func initSession(ctx context.Context) *sessionutil.Session {
|
|||
endpoints := baseTable.GetWithDefault("etcd.endpoints", paramtable.DefaultEtcdEndpoints)
|
||||
etcdEndpoints := strings.Split(endpoints, ",")
|
||||
|
||||
log.Debug("metaRootPath", zap.Any("metaRootPath", metaRootPath))
|
||||
log.Debug("etcdPoints", zap.Any("etcdPoints", etcdEndpoints))
|
||||
log.Ctx(context.TODO()).Debug("metaRootPath", zap.Any("metaRootPath", metaRootPath))
|
||||
log.Ctx(context.TODO()).Debug("etcdPoints", zap.Any("etcdPoints", etcdEndpoints))
|
||||
|
||||
etcdCli, err := etcd.GetRemoteEtcdClient(etcdEndpoints)
|
||||
if err != nil {
|
||||
|
|
|
@ -51,6 +51,7 @@ type Client struct {
|
|||
grpcClient grpcclient.GrpcClient[datapb.DataCoordClient]
|
||||
sess *sessionutil.Session
|
||||
sourceID int64
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
// NewClient creates a new client instance
|
||||
|
@ -58,7 +59,7 @@ func NewClient(ctx context.Context) (types.DataCoordClient, error) {
|
|||
sess := sessionutil.NewSession(ctx)
|
||||
if sess == nil {
|
||||
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
||||
log.Debug("DataCoordClient NewClient failed", zap.Error(err))
|
||||
log.Ctx(ctx).Debug("DataCoordClient NewClient failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -66,6 +67,7 @@ func NewClient(ctx context.Context) (types.DataCoordClient, error) {
|
|||
client := &Client{
|
||||
grpcClient: grpcclient.NewClientBase[datapb.DataCoordClient](config, "milvus.proto.data.DataCoord"),
|
||||
sess: sess,
|
||||
ctx: ctx,
|
||||
}
|
||||
client.grpcClient.SetRole(typeutil.DataCoordRole)
|
||||
client.grpcClient.SetGetAddrFunc(client.getDataCoordAddr)
|
||||
|
@ -76,7 +78,7 @@ func NewClient(ctx context.Context) (types.DataCoordClient, error) {
|
|||
client.grpcClient.EnableEncryption()
|
||||
cp, err := utils.CreateCertPoolforClient(Params.InternalTLSCfg.InternalTLSCaPemPath.GetValue(), "Datacoord")
|
||||
if err != nil {
|
||||
log.Error("Failed to create cert pool for Datacoord client")
|
||||
log.Ctx(ctx).Error("Failed to create cert pool for Datacoord client")
|
||||
return nil, err
|
||||
}
|
||||
client.grpcClient.SetInternalTLSCertPool(cp)
|
||||
|
@ -91,6 +93,7 @@ func (c *Client) newGrpcClient(cc *grpc.ClientConn) datapb.DataCoordClient {
|
|||
|
||||
func (c *Client) getDataCoordAddr() (string, error) {
|
||||
key := c.grpcClient.GetRole()
|
||||
log := log.Ctx(c.ctx)
|
||||
msess, _, err := c.sess.GetSessions(key)
|
||||
if err != nil {
|
||||
log.Debug("DataCoordClient, getSessions failed", zap.Any("key", key), zap.Error(err))
|
||||
|
|
|
@ -93,10 +93,10 @@ func (s *Server) Prepare() error {
|
|||
netutil.OptPort(paramtable.Get().DataCoordGrpcServerCfg.Port.GetAsInt()),
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("DataCoord fail to create net listener", zap.Error(err))
|
||||
log.Ctx(s.ctx).Warn("DataCoord fail to create net listener", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Info("DataCoord listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
log.Ctx(s.ctx).Info("DataCoord listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
s.listener = listener
|
||||
return nil
|
||||
}
|
||||
|
@ -104,6 +104,7 @@ func (s *Server) Prepare() error {
|
|||
func (s *Server) init() error {
|
||||
params := paramtable.Get()
|
||||
etcdConfig := ¶ms.EtcdCfg
|
||||
log := log.Ctx(s.ctx)
|
||||
|
||||
etcdCli, err := etcd.CreateEtcdClient(
|
||||
etcdConfig.UseEmbedEtcd.GetAsBool(),
|
||||
|
@ -223,13 +224,13 @@ func (s *Server) startGrpcLoop() {
|
|||
func (s *Server) start() error {
|
||||
err := s.dataCoord.Register()
|
||||
if err != nil {
|
||||
log.Debug("DataCoord register service failed", zap.Error(err))
|
||||
log.Ctx(s.ctx).Debug("DataCoord register service failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
err = s.dataCoord.Start()
|
||||
if err != nil {
|
||||
log.Error("DataCoord start failed", zap.Error(err))
|
||||
log.Ctx(s.ctx).Error("DataCoord start failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
|
@ -238,9 +239,9 @@ func (s *Server) start() error {
|
|||
// Stop stops the DataCoord server gracefully.
|
||||
// Need to call the GracefulStop interface of grpc server and call the stop method of the inner DataCoord object.
|
||||
func (s *Server) Stop() (err error) {
|
||||
logger := log.With()
|
||||
logger := log.Ctx(s.ctx)
|
||||
if s.listener != nil {
|
||||
logger = log.With(zap.String("address", s.listener.Address()))
|
||||
logger = logger.With(zap.String("address", s.listener.Address()))
|
||||
}
|
||||
logger.Info("Datacoord stopping")
|
||||
defer func() {
|
||||
|
@ -261,7 +262,7 @@ func (s *Server) Stop() (err error) {
|
|||
logger.Info("internal server[dataCoord] start to stop")
|
||||
err = s.dataCoord.Stop()
|
||||
if err != nil {
|
||||
log.Error("failed to close dataCoord", zap.Error(err))
|
||||
logger.Error("failed to close dataCoord", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
s.cancel()
|
||||
|
@ -278,12 +279,12 @@ func (s *Server) Run() error {
|
|||
if err := s.init(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Debug("DataCoord init done ...")
|
||||
log.Ctx(s.ctx).Debug("DataCoord init done ...")
|
||||
|
||||
if err := s.start(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Debug("DataCoord start done ...")
|
||||
log.Ctx(s.ctx).Debug("DataCoord start done ...")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -46,6 +46,7 @@ type Client struct {
|
|||
sess *sessionutil.Session
|
||||
addr string
|
||||
serverID int64
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
// NewClient creates a client for DataNode.
|
||||
|
@ -56,15 +57,17 @@ func NewClient(ctx context.Context, addr string, serverID int64) (types.DataNode
|
|||
sess := sessionutil.NewSession(ctx)
|
||||
if sess == nil {
|
||||
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
||||
log.Debug("DataNodeClient New Etcd Session failed", zap.Error(err))
|
||||
log.Ctx(ctx).Debug("DataNodeClient New Etcd Session failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
config := &Params.DataNodeGrpcClientCfg
|
||||
client := &Client{
|
||||
addr: addr,
|
||||
grpcClient: grpcclient.NewClientBase[datapb.DataNodeClient](config, "milvus.proto.data.DataNode"),
|
||||
sess: sess,
|
||||
serverID: serverID,
|
||||
ctx: ctx,
|
||||
}
|
||||
// node shall specify node id
|
||||
client.grpcClient.SetRole(fmt.Sprintf("%s-%d", typeutil.DataNodeRole, serverID))
|
||||
|
@ -77,7 +80,7 @@ func NewClient(ctx context.Context, addr string, serverID int64) (types.DataNode
|
|||
client.grpcClient.EnableEncryption()
|
||||
cp, err := utils.CreateCertPoolforClient(Params.InternalTLSCfg.InternalTLSCaPemPath.GetValue(), "DataNode")
|
||||
if err != nil {
|
||||
log.Error("Failed to create cert pool for DataNode client")
|
||||
log.Ctx(ctx).Error("Failed to create cert pool for DataNode client")
|
||||
return nil, err
|
||||
}
|
||||
client.grpcClient.SetInternalTLSCertPool(cp)
|
||||
|
|
|
@ -96,10 +96,10 @@ func (s *Server) Prepare() error {
|
|||
netutil.OptHighPriorityToUsePort(paramtable.Get().DataNodeGrpcServerCfg.Port.GetAsInt()),
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("DataNode fail to create net listener", zap.Error(err))
|
||||
log.Ctx(s.ctx).Warn("DataNode fail to create net listener", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Info("DataNode listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
log.Ctx(s.ctx).Info("DataNode listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
s.listener = listener
|
||||
paramtable.Get().Save(
|
||||
paramtable.Get().DataNodeGrpcServerCfg.Port.Key,
|
||||
|
@ -166,7 +166,7 @@ func (s *Server) startGrpcLoop() {
|
|||
|
||||
go funcutil.CheckGrpcReady(ctx, s.grpcErrChan)
|
||||
if err := s.grpcServer.Serve(s.listener); err != nil {
|
||||
log.Warn("DataNode failed to start gRPC")
|
||||
log.Ctx(s.ctx).Warn("DataNode failed to start gRPC")
|
||||
s.grpcErrChan <- err
|
||||
}
|
||||
}
|
||||
|
@ -189,20 +189,20 @@ func (s *Server) Run() error {
|
|||
// errors are propagated upstream as panic.
|
||||
return err
|
||||
}
|
||||
log.Info("DataNode gRPC services successfully initialized")
|
||||
log.Ctx(s.ctx).Info("DataNode gRPC services successfully initialized")
|
||||
if err := s.start(); err != nil {
|
||||
// errors are propagated upstream as panic.
|
||||
return err
|
||||
}
|
||||
log.Info("DataNode gRPC services successfully started")
|
||||
log.Ctx(s.ctx).Info("DataNode gRPC services successfully started")
|
||||
return nil
|
||||
}
|
||||
|
||||
// Stop stops Datanode's grpc service.
|
||||
func (s *Server) Stop() (err error) {
|
||||
logger := log.With()
|
||||
logger := log.Ctx(s.ctx)
|
||||
if s.listener != nil {
|
||||
logger = log.With(zap.String("address", s.listener.Address()))
|
||||
logger = logger.With(zap.String("address", s.listener.Address()))
|
||||
}
|
||||
logger.Info("datanode stopping")
|
||||
defer func() {
|
||||
|
@ -220,7 +220,7 @@ func (s *Server) Stop() (err error) {
|
|||
logger.Info("internal server[datanode] start to stop")
|
||||
err = s.datanode.Stop()
|
||||
if err != nil {
|
||||
log.Error("failed to close datanode", zap.Error(err))
|
||||
logger.Error("failed to close datanode", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
s.cancel()
|
||||
|
@ -234,7 +234,7 @@ func (s *Server) Stop() (err error) {
|
|||
// init initializes Datanode's grpc service.
|
||||
func (s *Server) init() error {
|
||||
etcdConfig := ¶mtable.Get().EtcdCfg
|
||||
ctx := context.Background()
|
||||
log := log.Ctx(s.ctx)
|
||||
|
||||
etcdCli, err := etcd.CreateEtcdClient(
|
||||
etcdConfig.UseEmbedEtcd.GetAsBool(),
|
||||
|
@ -255,7 +255,6 @@ func (s *Server) init() error {
|
|||
s.SetEtcdClient(s.etcdCli)
|
||||
s.datanode.SetAddress(s.listener.Address())
|
||||
log.Info("DataNode address", zap.String("address", s.listener.Address()))
|
||||
log.Info("DataNode serverID", zap.Int64("serverID", s.serverID.Load()))
|
||||
|
||||
err = s.startGrpc()
|
||||
if err != nil {
|
||||
|
@ -271,7 +270,7 @@ func (s *Server) init() error {
|
|||
panic(err)
|
||||
}
|
||||
|
||||
if err = componentutil.WaitForComponentHealthy(ctx, rootCoordClient, "RootCoord", 1000000, time.Millisecond*200); err != nil {
|
||||
if err = componentutil.WaitForComponentHealthy(s.ctx, rootCoordClient, "RootCoord", 1000000, time.Millisecond*200); err != nil {
|
||||
log.Error("failed to wait for RootCoord client to be ready", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
|
@ -290,7 +289,7 @@ func (s *Server) init() error {
|
|||
panic(err)
|
||||
}
|
||||
|
||||
if err = componentutil.WaitForComponentInitOrHealthy(ctx, dataCoordClient, "DataCoord", 1000000, time.Millisecond*200); err != nil {
|
||||
if err = componentutil.WaitForComponentInitOrHealthy(s.ctx, dataCoordClient, "DataCoord", 1000000, time.Millisecond*200); err != nil {
|
||||
log.Error("failed to wait for DataCoord client to be ready", zap.Error(err))
|
||||
panic(err)
|
||||
}
|
||||
|
@ -317,7 +316,7 @@ func (s *Server) start() error {
|
|||
}
|
||||
err := s.datanode.Register()
|
||||
if err != nil {
|
||||
log.Debug("failed to register to Etcd", zap.Error(err))
|
||||
log.Ctx(s.ctx).Debug("failed to register to Etcd", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
|
|
|
@ -55,7 +55,7 @@ func NewClient(ctx context.Context, addr string, nodeID int64, encryption bool)
|
|||
sess := sessionutil.NewSession(ctx)
|
||||
if sess == nil {
|
||||
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
||||
log.Debug("IndexNodeClient New Etcd Session failed", zap.Error(err))
|
||||
log.Ctx(ctx).Debug("IndexNodeClient New Etcd Session failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
config := &Params.IndexNodeGrpcClientCfg
|
||||
|
@ -77,7 +77,7 @@ func NewClient(ctx context.Context, addr string, nodeID int64, encryption bool)
|
|||
client.grpcClient.EnableEncryption()
|
||||
cp, err := utils.CreateCertPoolforClient(Params.InternalTLSCfg.InternalTLSCaPemPath.GetValue(), "IndexNode")
|
||||
if err != nil {
|
||||
log.Error("Failed to create cert pool for IndexNode client")
|
||||
log.Ctx(ctx).Error("Failed to create cert pool for IndexNode client")
|
||||
return nil, err
|
||||
}
|
||||
client.grpcClient.SetInternalTLSCertPool(cp)
|
||||
|
|
|
@ -71,11 +71,11 @@ func (s *Server) Prepare() error {
|
|||
netutil.OptHighPriorityToUsePort(paramtable.Get().IndexNodeGrpcServerCfg.Port.GetAsInt()),
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("IndexNode fail to create net listener", zap.Error(err))
|
||||
log.Ctx(s.loopCtx).Warn("IndexNode fail to create net listener", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
s.listener = listener
|
||||
log.Info("IndexNode listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
log.Ctx(s.loopCtx).Info("IndexNode listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
paramtable.Get().Save(
|
||||
paramtable.Get().IndexNodeGrpcServerCfg.Port.Key,
|
||||
strconv.FormatInt(int64(listener.Port()), 10))
|
||||
|
@ -87,11 +87,11 @@ func (s *Server) Run() error {
|
|||
if err := s.init(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Debug("IndexNode init done ...")
|
||||
log.Ctx(s.loopCtx).Debug("IndexNode init done ...")
|
||||
if err := s.start(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Debug("IndexNode start done ...")
|
||||
log.Ctx(s.loopCtx).Debug("IndexNode start done ...")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -155,6 +155,7 @@ func (s *Server) startGrpcLoop() {
|
|||
func (s *Server) init() error {
|
||||
etcdConfig := ¶mtable.Get().EtcdCfg
|
||||
var err error
|
||||
log := log.Ctx(s.loopCtx)
|
||||
|
||||
defer func() {
|
||||
if err != nil {
|
||||
|
@ -204,6 +205,7 @@ func (s *Server) init() error {
|
|||
|
||||
// start starts IndexNode's grpc service.
|
||||
func (s *Server) start() error {
|
||||
log := log.Ctx(s.loopCtx)
|
||||
err := s.indexnode.Start()
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -219,9 +221,9 @@ func (s *Server) start() error {
|
|||
|
||||
// Stop stops IndexNode's grpc service.
|
||||
func (s *Server) Stop() (err error) {
|
||||
logger := log.With()
|
||||
logger := log.Ctx(s.loopCtx)
|
||||
if s.listener != nil {
|
||||
logger = log.With(zap.String("address", s.listener.Address()))
|
||||
logger = logger.With(zap.String("address", s.listener.Address()))
|
||||
}
|
||||
logger.Info("IndexNode stopping")
|
||||
defer func() {
|
||||
|
|
|
@ -55,7 +55,7 @@ func NewClient(ctx context.Context, addr string, nodeID int64) (types.ProxyClien
|
|||
sess := sessionutil.NewSession(ctx)
|
||||
if sess == nil {
|
||||
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
||||
log.Debug("Proxy client new session failed", zap.Error(err))
|
||||
log.Ctx(ctx).Debug("Proxy client new session failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
config := &Params.ProxyGrpcClientCfg
|
||||
|
@ -74,7 +74,7 @@ func NewClient(ctx context.Context, addr string, nodeID int64) (types.ProxyClien
|
|||
client.grpcClient.EnableEncryption()
|
||||
cp, err := utils.CreateCertPoolforClient(Params.InternalTLSCfg.InternalTLSCaPemPath.GetValue(), "Proxy")
|
||||
if err != nil {
|
||||
log.Error("Failed to create cert pool for Proxy client")
|
||||
log.Ctx(ctx).Error("Failed to create cert pool for Proxy client")
|
||||
return nil, err
|
||||
}
|
||||
client.grpcClient.SetInternalTLSCertPool(cp)
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package grpcproxy
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/tls"
|
||||
"crypto/x509"
|
||||
"fmt"
|
||||
|
@ -33,13 +34,14 @@ import (
|
|||
)
|
||||
|
||||
// newListenerManager creates a new listener
|
||||
func newListenerManager() (l *listenerManager, err error) {
|
||||
func newListenerManager(ctx context.Context) (l *listenerManager, err error) {
|
||||
defer func() {
|
||||
if err != nil && l != nil {
|
||||
l.Close()
|
||||
}
|
||||
}()
|
||||
|
||||
log := log.Ctx(ctx)
|
||||
externalGrpcListener, err := netutil.NewListener(
|
||||
netutil.OptIP(paramtable.Get().ProxyGrpcServerCfg.IP),
|
||||
netutil.OptPort(paramtable.Get().ProxyGrpcServerCfg.Port.GetAsInt()),
|
||||
|
@ -64,14 +66,15 @@ func newListenerManager() (l *listenerManager, err error) {
|
|||
externalGrpcListener: externalGrpcListener,
|
||||
internalGrpcListener: internalGrpcListener,
|
||||
}
|
||||
if err = newHTTPListner(l); err != nil {
|
||||
if err = newHTTPListner(ctx, l); err != nil {
|
||||
return
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
// newHTTPListner creates a new http listener
|
||||
func newHTTPListner(l *listenerManager) error {
|
||||
func newHTTPListner(ctx context.Context, l *listenerManager) error {
|
||||
log := log.Ctx(ctx)
|
||||
HTTPParams := ¶mtable.Get().HTTPCfg
|
||||
if !HTTPParams.Enabled.GetAsBool() {
|
||||
// http server is disabled
|
||||
|
@ -192,6 +195,7 @@ func (l *listenerManager) HTTPListener() net.Listener {
|
|||
}
|
||||
|
||||
func (l *listenerManager) Close() {
|
||||
log := log.Ctx(context.TODO())
|
||||
if l.portShareMode {
|
||||
if l.cmux != nil {
|
||||
log.Info("Proxy close cmux grpc listener")
|
||||
|
|
|
@ -130,7 +130,7 @@ func authenticate(c *gin.Context) {
|
|||
username, password, ok := httpserver.ParseUsernamePassword(c)
|
||||
if ok {
|
||||
if proxy.PasswordVerify(c, username, password) {
|
||||
log.Debug("auth successful", zap.String("username", username))
|
||||
log.Ctx(context.TODO()).Debug("auth successful", zap.String("username", username))
|
||||
c.Set(httpserver.ContextUsername, username)
|
||||
return
|
||||
}
|
||||
|
@ -142,7 +142,7 @@ func authenticate(c *gin.Context) {
|
|||
c.Set(httpserver.ContextUsername, user)
|
||||
return
|
||||
}
|
||||
log.Warn("fail to verify apikey", zap.Error(err))
|
||||
log.Ctx(context.TODO()).Warn("fail to verify apikey", zap.Error(err))
|
||||
}
|
||||
|
||||
hookutil.GetExtension().ReportRefused(context.Background(), nil, &milvuspb.BoolResponse{
|
||||
|
@ -198,11 +198,11 @@ func (s *Server) startHTTPServer(errChan chan error) {
|
|||
s.httpServer = &http.Server{Handler: ginHandler, ReadHeaderTimeout: time.Second}
|
||||
errChan <- nil
|
||||
if err := s.httpServer.Serve(s.listenerManager.HTTPListener()); err != nil && err != cmux.ErrServerClosed {
|
||||
log.Error("start Proxy http server to listen failed", zap.Error(err))
|
||||
log.Ctx(s.ctx).Error("start Proxy http server to listen failed", zap.Error(err))
|
||||
errChan <- err
|
||||
return
|
||||
}
|
||||
log.Info("Proxy http server exited")
|
||||
log.Ctx(s.ctx).Info("Proxy http server exited")
|
||||
}
|
||||
|
||||
func (s *Server) startInternalRPCServer(errChan chan error) {
|
||||
|
@ -228,6 +228,8 @@ func (s *Server) startExternalGrpc(errChan chan error) {
|
|||
Timeout: 10 * time.Second, // Wait 10 second for the ping ack before assuming the connection is dead
|
||||
}
|
||||
|
||||
log := log.Ctx(s.ctx)
|
||||
|
||||
limiter, err := s.proxy.GetRateLimiter()
|
||||
if err != nil {
|
||||
log.Error("Get proxy rate limiter failed", zap.Error(err))
|
||||
|
@ -381,6 +383,7 @@ func (s *Server) startInternalGrpc(errChan chan error) {
|
|||
grpc_health_v1.RegisterHealthServer(s.grpcInternalServer, s)
|
||||
errChan <- nil
|
||||
|
||||
log := log.Ctx(s.ctx)
|
||||
log.Info("create Proxy internal grpc server",
|
||||
zap.Any("enforcement policy", kaep),
|
||||
zap.Any("server parameters", kasp))
|
||||
|
@ -394,7 +397,7 @@ func (s *Server) startInternalGrpc(errChan chan error) {
|
|||
}
|
||||
|
||||
func (s *Server) Prepare() error {
|
||||
listenerManager, err := newListenerManager()
|
||||
listenerManager, err := newListenerManager(s.ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -404,6 +407,7 @@ func (s *Server) Prepare() error {
|
|||
|
||||
// Start start the Proxy Server
|
||||
func (s *Server) Run() error {
|
||||
log := log.Ctx(s.ctx)
|
||||
log.Info("init Proxy server")
|
||||
if err := s.init(); err != nil {
|
||||
log.Warn("init Proxy server failed", zap.Error(err))
|
||||
|
@ -423,6 +427,7 @@ func (s *Server) Run() error {
|
|||
func (s *Server) init() error {
|
||||
etcdConfig := ¶mtable.Get().EtcdCfg
|
||||
Params := ¶mtable.Get().ProxyGrpcServerCfg
|
||||
log := log.Ctx(s.ctx)
|
||||
log.Info("Proxy init service's parameter table done")
|
||||
HTTPParams := ¶mtable.Get().HTTPCfg
|
||||
log.Info("Proxy init http server's parameter table done")
|
||||
|
@ -556,6 +561,7 @@ func (s *Server) init() error {
|
|||
}
|
||||
|
||||
func (s *Server) start() error {
|
||||
log := log.Ctx(s.ctx)
|
||||
if err := s.proxy.Start(); err != nil {
|
||||
log.Warn("failed to start Proxy server", zap.Error(err))
|
||||
return err
|
||||
|
@ -582,7 +588,7 @@ func (s *Server) start() error {
|
|||
|
||||
// Stop stop the Proxy Server
|
||||
func (s *Server) Stop() (err error) {
|
||||
logger := log.With()
|
||||
logger := log.Ctx(s.ctx)
|
||||
if s.listenerManager != nil {
|
||||
logger = log.With(
|
||||
zap.String("internal address", s.listenerManager.internalGrpcListener.Address()),
|
||||
|
@ -606,17 +612,17 @@ func (s *Server) Stop() (err error) {
|
|||
// try to close grpc server firstly, it has the same root listener with cmux server and
|
||||
// http listener that tls has not been enabled.
|
||||
if s.grpcExternalServer != nil {
|
||||
log.Info("Proxy stop external grpc server")
|
||||
logger.Info("Proxy stop external grpc server")
|
||||
utils.GracefulStopGRPCServer(s.grpcExternalServer)
|
||||
}
|
||||
|
||||
if s.httpServer != nil {
|
||||
log.Info("Proxy stop http server...")
|
||||
logger.Info("Proxy stop http server...")
|
||||
s.httpServer.Close()
|
||||
}
|
||||
|
||||
if s.grpcInternalServer != nil {
|
||||
log.Info("Proxy stop internal grpc server")
|
||||
logger.Info("Proxy stop internal grpc server")
|
||||
utils.GracefulStopGRPCServer(s.grpcInternalServer)
|
||||
}
|
||||
|
||||
|
@ -631,7 +637,7 @@ func (s *Server) Stop() (err error) {
|
|||
logger.Info("internal server[proxy] start to stop")
|
||||
err = s.proxy.Stop()
|
||||
if err != nil {
|
||||
log.Error("failed to close proxy", zap.Error(err))
|
||||
logger.Error("failed to close proxy", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
|
|
@ -44,6 +44,7 @@ var Params *paramtable.ComponentParam = paramtable.Get()
|
|||
type Client struct {
|
||||
grpcClient grpcclient.GrpcClient[querypb.QueryCoordClient]
|
||||
sess *sessionutil.Session
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
// NewClient creates a client for QueryCoord grpc call.
|
||||
|
@ -51,13 +52,15 @@ func NewClient(ctx context.Context) (types.QueryCoordClient, error) {
|
|||
sess := sessionutil.NewSession(ctx)
|
||||
if sess == nil {
|
||||
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
||||
log.Debug("QueryCoordClient NewClient failed", zap.Error(err))
|
||||
log.Ctx(ctx).Debug("QueryCoordClient NewClient failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
clientCtx := log.WithFields(ctx, zap.String("module", "QueryCoordClient"))
|
||||
config := &Params.QueryCoordGrpcClientCfg
|
||||
client := &Client{
|
||||
grpcClient: grpcclient.NewClientBase[querypb.QueryCoordClient](config, "milvus.proto.query.QueryCoord"),
|
||||
sess: sess,
|
||||
ctx: clientCtx,
|
||||
}
|
||||
client.grpcClient.SetRole(typeutil.QueryCoordRole)
|
||||
client.grpcClient.SetGetAddrFunc(client.getQueryCoordAddr)
|
||||
|
@ -68,7 +71,7 @@ func NewClient(ctx context.Context) (types.QueryCoordClient, error) {
|
|||
client.grpcClient.EnableEncryption()
|
||||
cp, err := utils.CreateCertPoolforClient(Params.InternalTLSCfg.InternalTLSCaPemPath.GetValue(), "QueryCoord")
|
||||
if err != nil {
|
||||
log.Error("Failed to create cert pool for QueryCoord client")
|
||||
log.Ctx(ctx).Error("Failed to create cert pool for QueryCoord client")
|
||||
return nil, err
|
||||
}
|
||||
client.grpcClient.SetInternalTLSCertPool(cp)
|
||||
|
@ -81,16 +84,16 @@ func (c *Client) getQueryCoordAddr() (string, error) {
|
|||
key := c.grpcClient.GetRole()
|
||||
msess, _, err := c.sess.GetSessions(key)
|
||||
if err != nil {
|
||||
log.Debug("QueryCoordClient GetSessions failed", zap.Error(err))
|
||||
log.Ctx(c.ctx).Debug("QueryCoordClient GetSessions failed", zap.Error(err))
|
||||
return "", err
|
||||
}
|
||||
ms, ok := msess[key]
|
||||
if !ok {
|
||||
log.Debug("QueryCoordClient msess key not existed", zap.Any("key", key))
|
||||
log.Ctx(c.ctx).Debug("QueryCoordClient msess key not existed", zap.Any("key", key))
|
||||
return "", fmt.Errorf("find no available querycoord, check querycoord state")
|
||||
}
|
||||
|
||||
log.Debug("QueryCoordClient GetSessions success",
|
||||
log.Ctx(c.ctx).Debug("QueryCoordClient GetSessions success",
|
||||
zap.String("address", ms.Address),
|
||||
zap.Int64("serverID", ms.ServerID),
|
||||
)
|
||||
|
|
|
@ -98,10 +98,10 @@ func (s *Server) Prepare() error {
|
|||
netutil.OptPort(paramtable.Get().QueryCoordGrpcServerCfg.Port.GetAsInt()),
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("QueryCoord fail to create net listener", zap.Error(err))
|
||||
log.Ctx(s.loopCtx).Warn("QueryCoord fail to create net listener", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Info("QueryCoord listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
log.Ctx(s.loopCtx).Info("QueryCoord listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
s.listener = listener
|
||||
return nil
|
||||
}
|
||||
|
@ -111,12 +111,12 @@ func (s *Server) Run() error {
|
|||
if err := s.init(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Info("QueryCoord init done ...")
|
||||
log.Ctx(s.loopCtx).Info("QueryCoord init done ...")
|
||||
|
||||
if err := s.start(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Info("QueryCoord start done ...")
|
||||
log.Ctx(s.loopCtx).Info("QueryCoord start done ...")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -124,6 +124,7 @@ var getTiKVClient = tikv.GetTiKVClient
|
|||
|
||||
// init initializes QueryCoord's grpc service.
|
||||
func (s *Server) init() error {
|
||||
log := log.Ctx(s.loopCtx)
|
||||
params := paramtable.Get()
|
||||
etcdConfig := ¶ms.EtcdCfg
|
||||
|
||||
|
@ -259,7 +260,7 @@ func (s *Server) GetQueryCoord() types.QueryCoordComponent {
|
|||
|
||||
// Stop stops QueryCoord's grpc service.
|
||||
func (s *Server) Stop() (err error) {
|
||||
logger := log.With()
|
||||
logger := log.Ctx(s.loopCtx)
|
||||
if s.listener != nil {
|
||||
logger = log.With(zap.String("address", s.listener.Address()))
|
||||
}
|
||||
|
|
|
@ -46,6 +46,7 @@ type Client struct {
|
|||
addr string
|
||||
sess *sessionutil.Session
|
||||
nodeID int64
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
// NewClient creates a new QueryNode client.
|
||||
|
@ -56,7 +57,7 @@ func NewClient(ctx context.Context, addr string, nodeID int64) (types.QueryNodeC
|
|||
sess := sessionutil.NewSession(ctx)
|
||||
if sess == nil {
|
||||
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
||||
log.Debug("QueryNodeClient NewClient failed", zap.Error(err))
|
||||
log.Ctx(ctx).Debug("QueryNodeClient NewClient failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
config := ¶mtable.Get().QueryNodeGrpcClientCfg
|
||||
|
@ -65,6 +66,7 @@ func NewClient(ctx context.Context, addr string, nodeID int64) (types.QueryNodeC
|
|||
grpcClient: grpcclient.NewClientBase[querypb.QueryNodeClient](config, "milvus.proto.query.QueryNode"),
|
||||
sess: sess,
|
||||
nodeID: nodeID,
|
||||
ctx: ctx,
|
||||
}
|
||||
// node shall specify node id
|
||||
client.grpcClient.SetRole(fmt.Sprintf("%s-%d", typeutil.QueryNodeRole, nodeID))
|
||||
|
@ -77,7 +79,7 @@ func NewClient(ctx context.Context, addr string, nodeID int64) (types.QueryNodeC
|
|||
client.grpcClient.EnableEncryption()
|
||||
cp, err := utils.CreateCertPoolforClient(Params.InternalTLSCfg.InternalTLSCaPemPath.GetValue(), "QueryNode")
|
||||
if err != nil {
|
||||
log.Error("Failed to create cert pool for QueryNode client")
|
||||
log.Ctx(ctx).Error("Failed to create cert pool for QueryNode client")
|
||||
return nil, err
|
||||
}
|
||||
client.grpcClient.SetInternalTLSCertPool(cp)
|
||||
|
|
|
@ -95,11 +95,11 @@ func (s *Server) Prepare() error {
|
|||
netutil.OptHighPriorityToUsePort(paramtable.Get().QueryNodeGrpcServerCfg.Port.GetAsInt()),
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("QueryNode fail to create net listener", zap.Error(err))
|
||||
log.Ctx(s.ctx).Warn("QueryNode fail to create net listener", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
s.listener = listener
|
||||
log.Info("QueryNode listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
log.Ctx(s.ctx).Info("QueryNode listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
paramtable.Get().Save(
|
||||
paramtable.Get().QueryNodeGrpcServerCfg.Port.Key,
|
||||
strconv.FormatInt(int64(listener.Port()), 10))
|
||||
|
@ -109,6 +109,7 @@ func (s *Server) Prepare() error {
|
|||
// init initializes QueryNode's grpc service.
|
||||
func (s *Server) init() error {
|
||||
etcdConfig := ¶mtable.Get().EtcdCfg
|
||||
log := log.Ctx(s.ctx)
|
||||
log.Debug("QueryNode", zap.Int("port", s.listener.Port()))
|
||||
|
||||
etcdCli, err := etcd.CreateEtcdClient(
|
||||
|
@ -151,6 +152,7 @@ func (s *Server) init() error {
|
|||
|
||||
// start starts QueryNode's grpc service.
|
||||
func (s *Server) start() error {
|
||||
log := log.Ctx(s.ctx)
|
||||
if err := s.querynode.Start(); err != nil {
|
||||
log.Error("QueryNode start failed", zap.Error(err))
|
||||
return err
|
||||
|
@ -215,7 +217,7 @@ func (s *Server) startGrpcLoop() {
|
|||
|
||||
go funcutil.CheckGrpcReady(ctx, s.grpcErrChan)
|
||||
if err := s.grpcServer.Serve(s.listener); err != nil {
|
||||
log.Debug("QueryNode Start Grpc Failed!!!!")
|
||||
log.Ctx(s.ctx).Debug("QueryNode Start Grpc Failed!!!!")
|
||||
s.grpcErrChan <- err
|
||||
}
|
||||
}
|
||||
|
@ -225,20 +227,20 @@ func (s *Server) Run() error {
|
|||
if err := s.init(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Debug("QueryNode init done ...")
|
||||
log.Ctx(s.ctx).Debug("QueryNode init done ...")
|
||||
|
||||
if err := s.start(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Debug("QueryNode start done ...")
|
||||
log.Ctx(s.ctx).Debug("QueryNode start done ...")
|
||||
return nil
|
||||
}
|
||||
|
||||
// Stop stops QueryNode's grpc service.
|
||||
func (s *Server) Stop() (err error) {
|
||||
logger := log.With()
|
||||
logger := log.Ctx(s.ctx)
|
||||
if s.listener != nil {
|
||||
logger = log.With(zap.String("address", s.listener.Address()))
|
||||
logger = logger.With(zap.String("address", s.listener.Address()))
|
||||
}
|
||||
logger.Info("QueryNode stopping")
|
||||
defer func() {
|
||||
|
@ -248,7 +250,7 @@ func (s *Server) Stop() (err error) {
|
|||
logger.Info("internal server[querynode] start to stop")
|
||||
err = s.querynode.Stop()
|
||||
if err != nil {
|
||||
log.Error("failed to close querynode", zap.Error(err))
|
||||
logger.Error("failed to close querynode", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
if s.etcdCli != nil {
|
||||
|
|
|
@ -47,6 +47,7 @@ var Params *paramtable.ComponentParam = paramtable.Get()
|
|||
type Client struct {
|
||||
grpcClient grpcclient.GrpcClient[rootcoordpb.RootCoordClient]
|
||||
sess *sessionutil.Session
|
||||
ctx context.Context
|
||||
}
|
||||
|
||||
// NewClient create root coordinator client with specified etcd info and timeout
|
||||
|
@ -58,13 +59,14 @@ func NewClient(ctx context.Context) (types.RootCoordClient, error) {
|
|||
sess := sessionutil.NewSession(ctx)
|
||||
if sess == nil {
|
||||
err := fmt.Errorf("new session error, maybe can not connect to etcd")
|
||||
log.Debug("QueryCoordClient NewClient failed", zap.Error(err))
|
||||
log.Ctx(ctx).Debug("New RootCoord Client failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
config := &Params.RootCoordGrpcClientCfg
|
||||
client := &Client{
|
||||
grpcClient: grpcclient.NewClientBase[rootcoordpb.RootCoordClient](config, "milvus.proto.rootcoord.RootCoord"),
|
||||
sess: sess,
|
||||
ctx: ctx,
|
||||
}
|
||||
client.grpcClient.SetRole(typeutil.RootCoordRole)
|
||||
client.grpcClient.SetGetAddrFunc(client.getRootCoordAddr)
|
||||
|
@ -75,7 +77,7 @@ func NewClient(ctx context.Context) (types.RootCoordClient, error) {
|
|||
client.grpcClient.EnableEncryption()
|
||||
cp, err := utils.CreateCertPoolforClient(Params.InternalTLSCfg.InternalTLSCaPemPath.GetValue(), "RootCoord")
|
||||
if err != nil {
|
||||
log.Error("Failed to create cert pool for RootCoord client")
|
||||
log.Ctx(ctx).Error("Failed to create cert pool for RootCoord client")
|
||||
return nil, err
|
||||
}
|
||||
client.grpcClient.SetInternalTLSCertPool(cp)
|
||||
|
@ -90,6 +92,7 @@ func (c *Client) newGrpcClient(cc *grpc.ClientConn) rootcoordpb.RootCoordClient
|
|||
}
|
||||
|
||||
func (c *Client) getRootCoordAddr() (string, error) {
|
||||
log := log.Ctx(c.ctx)
|
||||
key := c.grpcClient.GetRole()
|
||||
msess, _, err := c.sess.GetSessions(key)
|
||||
if err != nil {
|
||||
|
|
|
@ -142,6 +142,7 @@ func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error)
|
|||
}
|
||||
|
||||
func (s *Server) Prepare() error {
|
||||
log := log.Ctx(s.ctx)
|
||||
listener, err := netutil.NewListener(
|
||||
netutil.OptIP(paramtable.Get().RootCoordGrpcServerCfg.IP),
|
||||
netutil.OptPort(paramtable.Get().RootCoordGrpcServerCfg.Port.GetAsInt()),
|
||||
|
@ -165,12 +166,12 @@ func (s *Server) Run() error {
|
|||
if err := s.init(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Info("RootCoord init done ...")
|
||||
log.Ctx(s.ctx).Info("RootCoord init done ...")
|
||||
|
||||
if err := s.start(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Info("RootCoord start done ...")
|
||||
log.Ctx(s.ctx).Info("RootCoord start done ...")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -179,6 +180,7 @@ var getTiKVClient = tikv.GetTiKVClient
|
|||
func (s *Server) init() error {
|
||||
params := paramtable.Get()
|
||||
etcdConfig := ¶ms.EtcdCfg
|
||||
log := log.Ctx(s.ctx)
|
||||
log.Info("init params done..")
|
||||
|
||||
etcdCli, err := etcd.CreateEtcdClient(
|
||||
|
@ -259,6 +261,7 @@ func (s *Server) startGrpcLoop() {
|
|||
Time: 60 * time.Second, // Ping the client if it is idle for 60 seconds to ensure the connection is still active
|
||||
Timeout: 10 * time.Second, // Wait 10 second for the ping ack before assuming the connection is dead
|
||||
}
|
||||
log := log.Ctx(s.ctx)
|
||||
log.Info("start grpc ", zap.Int("port", s.listener.Port()))
|
||||
|
||||
ctx, cancel := context.WithCancel(s.ctx)
|
||||
|
@ -304,6 +307,7 @@ func (s *Server) startGrpcLoop() {
|
|||
}
|
||||
|
||||
func (s *Server) start() error {
|
||||
log := log.Ctx(s.ctx)
|
||||
log.Info("RootCoord Core start ...")
|
||||
if err := s.rootCoord.Register(); err != nil {
|
||||
log.Error("RootCoord registers service failed", zap.Error(err))
|
||||
|
@ -319,9 +323,9 @@ func (s *Server) start() error {
|
|||
}
|
||||
|
||||
func (s *Server) Stop() (err error) {
|
||||
logger := log.With()
|
||||
logger := log.Ctx(s.ctx)
|
||||
if s.listener != nil {
|
||||
logger = log.With(zap.String("address", s.listener.Address()))
|
||||
logger = logger.With(zap.String("address", s.listener.Address()))
|
||||
}
|
||||
logger.Info("Rootcoord stopping")
|
||||
defer func() {
|
||||
|
|
|
@ -68,6 +68,9 @@ type Server struct {
|
|||
session *sessionutil.Session
|
||||
metaKV kv.MetaKv
|
||||
|
||||
ctx context.Context
|
||||
cancel context.CancelFunc
|
||||
|
||||
// server
|
||||
streamingnode *streamingnodeserver.Server
|
||||
|
||||
|
@ -87,12 +90,15 @@ type Server struct {
|
|||
}
|
||||
|
||||
// NewServer create a new StreamingNode server.
|
||||
func NewServer(f dependency.Factory) (*Server, error) {
|
||||
func NewServer(ctx context.Context, f dependency.Factory) (*Server, error) {
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
return &Server{
|
||||
stopOnce: sync.Once{},
|
||||
factory: f,
|
||||
grpcServerChan: make(chan struct{}),
|
||||
componentState: componentutil.NewComponentStateService(typeutil.StreamingNodeRole),
|
||||
ctx: ctx1,
|
||||
cancel: cancel,
|
||||
}, nil
|
||||
}
|
||||
|
||||
|
@ -102,11 +108,11 @@ func (s *Server) Prepare() error {
|
|||
netutil.OptHighPriorityToUsePort(paramtable.Get().StreamingNodeGrpcServerCfg.Port.GetAsInt()),
|
||||
)
|
||||
if err != nil {
|
||||
log.Warn("StreamingNode fail to create net listener", zap.Error(err))
|
||||
log.Ctx(s.ctx).Warn("StreamingNode fail to create net listener", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
s.listener = listener
|
||||
log.Info("StreamingNode listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
log.Ctx(s.ctx).Info("StreamingNode listen on", zap.String("address", listener.Addr().String()), zap.Int("port", listener.Port()))
|
||||
paramtable.Get().Save(
|
||||
paramtable.Get().StreamingNodeGrpcServerCfg.Port.Key,
|
||||
strconv.FormatInt(int64(listener.Port()), 10))
|
||||
|
@ -115,19 +121,15 @@ func (s *Server) Prepare() error {
|
|||
|
||||
// Run runs the server.
|
||||
func (s *Server) Run() error {
|
||||
// TODO: We should set a timeout for the process startup.
|
||||
// But currently, we don't implement.
|
||||
ctx := context.Background()
|
||||
|
||||
if err := s.init(ctx); err != nil {
|
||||
if err := s.init(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Info("streamingnode init done ...")
|
||||
log.Ctx(s.ctx).Info("streamingnode init done ...")
|
||||
|
||||
if err := s.start(ctx); err != nil {
|
||||
if err := s.start(); err != nil {
|
||||
return err
|
||||
}
|
||||
log.Info("streamingnode start done ...")
|
||||
log.Ctx(s.ctx).Info("streamingnode start done ...")
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -140,6 +142,7 @@ func (s *Server) Stop() (err error) {
|
|||
// stop stops the server.
|
||||
func (s *Server) stop() {
|
||||
s.componentState.OnStopping()
|
||||
log := log.Ctx(s.ctx)
|
||||
|
||||
log.Info("streamingnode stop", zap.String("Address", s.listener.Address()))
|
||||
|
||||
|
@ -179,6 +182,7 @@ func (s *Server) stop() {
|
|||
<-s.grpcServerChan
|
||||
log.Info("streamingnode stop done")
|
||||
|
||||
s.cancel()
|
||||
if err := s.listener.Close(); err != nil {
|
||||
log.Warn("streamingnode stop listener failed", zap.Error(err))
|
||||
}
|
||||
|
@ -190,7 +194,8 @@ func (s *Server) Health(ctx context.Context) commonpb.StateCode {
|
|||
return resp.GetState().StateCode
|
||||
}
|
||||
|
||||
func (s *Server) init(ctx context.Context) (err error) {
|
||||
func (s *Server) init() (err error) {
|
||||
log := log.Ctx(s.ctx)
|
||||
defer func() {
|
||||
if err != nil {
|
||||
log.Error("StreamingNode init failed", zap.Error(err))
|
||||
|
@ -205,16 +210,16 @@ func (s *Server) init(ctx context.Context) (err error) {
|
|||
if err := s.initMeta(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := s.initChunkManager(ctx); err != nil {
|
||||
if err := s.initChunkManager(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := s.initSession(ctx); err != nil {
|
||||
if err := s.initSession(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := s.initRootCoord(ctx); err != nil {
|
||||
if err := s.initRootCoord(); err != nil {
|
||||
return err
|
||||
}
|
||||
if err := s.initDataCoord(ctx); err != nil {
|
||||
if err := s.initDataCoord(); err != nil {
|
||||
return err
|
||||
}
|
||||
s.initGRPCServer()
|
||||
|
@ -230,13 +235,14 @@ func (s *Server) init(ctx context.Context) (err error) {
|
|||
WithMetaKV(s.metaKV).
|
||||
WithChunkManager(s.chunkManager).
|
||||
Build()
|
||||
if err := s.streamingnode.Init(ctx); err != nil {
|
||||
if err := s.streamingnode.Init(s.ctx); err != nil {
|
||||
return errors.Wrap(err, "StreamingNode service init failed")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) start(ctx context.Context) (err error) {
|
||||
func (s *Server) start() (err error) {
|
||||
log := log.Ctx(s.ctx)
|
||||
defer func() {
|
||||
if err != nil {
|
||||
log.Error("StreamingNode start failed", zap.Error(err))
|
||||
|
@ -249,7 +255,7 @@ func (s *Server) start(ctx context.Context) (err error) {
|
|||
s.streamingnode.Start()
|
||||
|
||||
// Start grpc server.
|
||||
if err := s.startGPRCServer(ctx); err != nil {
|
||||
if err := s.startGPRCServer(s.ctx); err != nil {
|
||||
return errors.Wrap(err, "StreamingNode start gRPC server fail")
|
||||
}
|
||||
// Register current server to etcd.
|
||||
|
@ -259,20 +265,21 @@ func (s *Server) start(ctx context.Context) (err error) {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) initSession(ctx context.Context) error {
|
||||
s.session = sessionutil.NewSession(ctx)
|
||||
func (s *Server) initSession() error {
|
||||
s.session = sessionutil.NewSession(s.ctx)
|
||||
if s.session == nil {
|
||||
return errors.New("session is nil, the etcd client connection may have failed")
|
||||
}
|
||||
s.session.Init(typeutil.StreamingNodeRole, s.listener.Address(), false, true)
|
||||
paramtable.SetNodeID(s.session.ServerID)
|
||||
log.Info("StreamingNode init session", zap.Int64("nodeID", paramtable.GetNodeID()), zap.String("node address", s.listener.Address()))
|
||||
log.Ctx(s.ctx).Info("StreamingNode init session", zap.Int64("nodeID", paramtable.GetNodeID()), zap.String("node address", s.listener.Address()))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) initMeta() error {
|
||||
params := paramtable.Get()
|
||||
metaType := params.MetaStoreCfg.MetaStoreType.GetValue()
|
||||
log := log.Ctx(s.ctx)
|
||||
log.Info("data coordinator connecting to metadata store", zap.String("metaType", metaType))
|
||||
metaRootPath := ""
|
||||
if metaType == util.MetaStoreTypeTiKV {
|
||||
|
@ -293,40 +300,42 @@ func (s *Server) initMeta() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) initRootCoord(ctx context.Context) (err error) {
|
||||
func (s *Server) initRootCoord() (err error) {
|
||||
log := log.Ctx(s.ctx)
|
||||
log.Info("StreamingNode connect to rootCoord...")
|
||||
s.rootCoord, err = rcc.NewClient(ctx)
|
||||
s.rootCoord, err = rcc.NewClient(s.ctx)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "StreamingNode try to new RootCoord client failed")
|
||||
}
|
||||
|
||||
log.Info("StreamingNode try to wait for RootCoord ready")
|
||||
err = componentutil.WaitForComponentHealthy(ctx, s.rootCoord, "RootCoord", 1000000, time.Millisecond*200)
|
||||
err = componentutil.WaitForComponentHealthy(s.ctx, s.rootCoord, "RootCoord", 1000000, time.Millisecond*200)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "StreamingNode wait for RootCoord ready failed")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) initDataCoord(ctx context.Context) (err error) {
|
||||
func (s *Server) initDataCoord() (err error) {
|
||||
log := log.Ctx(s.ctx)
|
||||
log.Info("StreamingNode connect to dataCoord...")
|
||||
s.dataCoord, err = dcc.NewClient(ctx)
|
||||
s.dataCoord, err = dcc.NewClient(s.ctx)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "StreamingNode try to new DataCoord client failed")
|
||||
}
|
||||
|
||||
log.Info("StreamingNode try to wait for DataCoord ready")
|
||||
err = componentutil.WaitForComponentHealthy(ctx, s.dataCoord, "DataCoord", 1000000, time.Millisecond*200)
|
||||
err = componentutil.WaitForComponentHealthy(s.ctx, s.dataCoord, "DataCoord", 1000000, time.Millisecond*200)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "StreamingNode wait for DataCoord ready failed")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Server) initChunkManager(ctx context.Context) (err error) {
|
||||
log.Info("StreamingNode init chunk manager...")
|
||||
func (s *Server) initChunkManager() (err error) {
|
||||
log.Ctx(s.ctx).Info("StreamingNode init chunk manager...")
|
||||
s.factory.Init(paramtable.Get())
|
||||
manager, err := s.factory.NewPersistentStorageChunkManager(ctx)
|
||||
manager, err := s.factory.NewPersistentStorageChunkManager(s.ctx)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "StreamingNode try to new chunk manager failed")
|
||||
}
|
||||
|
@ -335,7 +344,7 @@ func (s *Server) initChunkManager(ctx context.Context) (err error) {
|
|||
}
|
||||
|
||||
func (s *Server) initGRPCServer() {
|
||||
log.Info("create StreamingNode server...")
|
||||
log.Ctx(s.ctx).Info("create StreamingNode server...")
|
||||
cfg := ¶mtable.Get().StreamingNodeGrpcServerCfg
|
||||
kaep := keepalive.EnforcementPolicy{
|
||||
MinTime: 5 * time.Second, // If a client pings more than once every 5 seconds, terminate the connection
|
||||
|
@ -396,7 +405,7 @@ func (s *Server) registerSessionToETCD() {
|
|||
s.session.Register()
|
||||
// start liveness check
|
||||
s.session.LivenessCheck(context.Background(), func() {
|
||||
log.Error("StreamingNode disconnected from etcd, process will exit", zap.Int64("Server Id", paramtable.GetNodeID()))
|
||||
log.Ctx(s.ctx).Error("StreamingNode disconnected from etcd, process will exit", zap.Int64("Server Id", paramtable.GetNodeID()))
|
||||
os.Exit(1)
|
||||
})
|
||||
}
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package utils
|
||||
|
||||
import (
|
||||
"context"
|
||||
"crypto/x509"
|
||||
"os"
|
||||
"time"
|
||||
|
@ -18,6 +19,7 @@ func GracefulStopGRPCServer(s *grpc.Server) {
|
|||
if s == nil {
|
||||
return
|
||||
}
|
||||
log := log.Ctx(context.TODO())
|
||||
ch := make(chan struct{})
|
||||
go func() {
|
||||
defer close(ch)
|
||||
|
@ -37,6 +39,7 @@ func GracefulStopGRPCServer(s *grpc.Server) {
|
|||
}
|
||||
|
||||
func getTLSCreds(certFile string, keyFile string, nodeType string) credentials.TransportCredentials {
|
||||
log := log.Ctx(context.TODO())
|
||||
log.Info("TLS Server PEM Path", zap.String("path", certFile))
|
||||
log.Info("TLS Server Key Path", zap.String("path", keyFile))
|
||||
creds, err := credentials.NewServerTLSFromFile(certFile, keyFile)
|
||||
|
@ -48,6 +51,7 @@ func getTLSCreds(certFile string, keyFile string, nodeType string) credentials.T
|
|||
}
|
||||
|
||||
func EnableInternalTLS(NodeType string) grpc.ServerOption {
|
||||
log := log.Ctx(context.TODO())
|
||||
var Params *paramtable.ComponentParam = paramtable.Get()
|
||||
certFile := Params.InternalTLSCfg.InternalTLSServerPemPath.GetValue()
|
||||
keyFile := Params.InternalTLSCfg.InternalTLSServerKeyPath.GetValue()
|
||||
|
@ -63,6 +67,7 @@ func EnableInternalTLS(NodeType string) grpc.ServerOption {
|
|||
}
|
||||
|
||||
func CreateCertPoolforClient(caFile string, nodeType string) (*x509.CertPool, error) {
|
||||
log := log.Ctx(context.TODO())
|
||||
log.Info("Creating cert pool for " + nodeType)
|
||||
log.Info("Cert file path:", zap.String("caFile", caFile))
|
||||
certPool := x509.NewCertPool()
|
||||
|
|
|
@ -71,7 +71,7 @@ func (b *BinlogIoImpl) AsyncDownload(ctx context.Context, paths []string) []*con
|
|||
var err error
|
||||
|
||||
start := time.Now()
|
||||
log.Debug("BinlogIO download", zap.String("path", path))
|
||||
log.Ctx(ctx).Debug("BinlogIO download", zap.String("path", path))
|
||||
err = retry.Do(ctx, func() error {
|
||||
val, err = b.Read(ctx, path)
|
||||
if err != nil {
|
||||
|
@ -80,7 +80,7 @@ func (b *BinlogIoImpl) AsyncDownload(ctx context.Context, paths []string) []*con
|
|||
return err
|
||||
})
|
||||
|
||||
log.Debug("BinlogIO download success", zap.String("path", path), zap.Int64("cost", time.Since(start).Milliseconds()),
|
||||
log.Ctx(ctx).Debug("BinlogIO download success", zap.String("path", path), zap.Int64("cost", time.Since(start).Milliseconds()),
|
||||
zap.Error(err))
|
||||
|
||||
return val, err
|
||||
|
@ -106,7 +106,7 @@ func (b *BinlogIoImpl) AsyncUpload(ctx context.Context, kvs map[string][]byte) [
|
|||
future := b.pool.Submit(func() (any, error) {
|
||||
var err error
|
||||
start := time.Now()
|
||||
log.Debug("BinlogIO upload", zap.String("paths", innerK))
|
||||
log.Ctx(ctx).Debug("BinlogIO upload", zap.String("paths", innerK))
|
||||
err = retry.Do(ctx, func() error {
|
||||
err = b.Write(ctx, innerK, innerV)
|
||||
if err != nil {
|
||||
|
@ -114,7 +114,7 @@ func (b *BinlogIoImpl) AsyncUpload(ctx context.Context, kvs map[string][]byte) [
|
|||
}
|
||||
return err
|
||||
})
|
||||
log.Debug("BinlogIO upload success", zap.String("paths", innerK), zap.Int64("cost", time.Since(start).Milliseconds()), zap.Error(err))
|
||||
log.Ctx(ctx).Debug("BinlogIO upload success", zap.String("paths", innerK), zap.Int64("cost", time.Since(start).Milliseconds()), zap.Error(err))
|
||||
return struct{}{}, err
|
||||
})
|
||||
futures = append(futures, future)
|
||||
|
|
|
@ -98,6 +98,7 @@ func (ddn *ddNode) IsValidInMsg(in []Msg) bool {
|
|||
|
||||
// Operate handles input messages, implementing flowgrpah.Node
|
||||
func (ddn *ddNode) Operate(in []Msg) []Msg {
|
||||
log := log.Ctx(ddn.ctx)
|
||||
msMsg, ok := in[0].(*MsgStreamMsg)
|
||||
if !ok {
|
||||
log.Warn("type assertion failed for MsgStreamMsg", zap.String("channel", ddn.vChannelName), zap.String("name", reflect.TypeOf(in[0]).Name()))
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
package pipeline
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"time"
|
||||
|
@ -131,7 +132,7 @@ func (ttn *ttNode) updateChannelCP(channelPos *msgpb.MsgPosition, curTs time.Tim
|
|||
channelCPTs, _ := tsoutil.ParseTS(channelPos.GetTimestamp())
|
||||
// reset flush ts to prevent frequent flush
|
||||
ttn.writeBufferManager.NotifyCheckpointUpdated(ttn.vChannelName, channelPos.GetTimestamp())
|
||||
log.Debug("UpdateChannelCheckpoint success",
|
||||
log.Ctx(context.TODO()).Debug("UpdateChannelCheckpoint success",
|
||||
zap.String("channel", ttn.vChannelName),
|
||||
zap.Uint64("cpTs", channelPos.GetTimestamp()),
|
||||
zap.Time("cpTime", channelCPTs))
|
||||
|
|
|
@ -471,7 +471,7 @@ func GenRowData() (rawData []byte) {
|
|||
panic(err)
|
||||
}
|
||||
rawData = append(rawData, bfloat64.Bytes()...)
|
||||
log.Debug("Rawdata length:", zap.Int("Length of rawData", len(rawData)))
|
||||
log.Ctx(context.TODO()).Debug("Rawdata length:", zap.Int("Length of rawData", len(rawData)))
|
||||
return
|
||||
}
|
||||
|
||||
|
|
|
@ -117,7 +117,7 @@ type IndexNode struct {
|
|||
|
||||
// NewIndexNode creates a new IndexNode component.
|
||||
func NewIndexNode(ctx context.Context, factory dependency.Factory) *IndexNode {
|
||||
log.Debug("New IndexNode ...")
|
||||
log.Ctx(ctx).Debug("New IndexNode ...")
|
||||
rand.Seed(time.Now().UnixNano())
|
||||
ctx1, cancel := context.WithCancel(ctx)
|
||||
b := &IndexNode{
|
||||
|
@ -144,7 +144,7 @@ func (i *IndexNode) Register() error {
|
|||
metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.IndexNodeRole).Inc()
|
||||
// start liveness check
|
||||
i.session.LivenessCheck(i.loopCtx, func() {
|
||||
log.Error("Index Node disconnected from etcd, process will exit", zap.Int64("Server Id", i.session.ServerID))
|
||||
log.Ctx(i.loopCtx).Error("Index Node disconnected from etcd, process will exit", zap.Int64("Server Id", i.session.ServerID))
|
||||
os.Exit(1)
|
||||
})
|
||||
return nil
|
||||
|
|
|
@ -108,7 +108,7 @@ func (queue *IndexTaskQueue) AddActiveTask(t task) {
|
|||
tName := t.Name()
|
||||
_, ok := queue.activeTasks[tName]
|
||||
if ok {
|
||||
log.Debug("IndexNode task already in active task list", zap.String("TaskID", tName))
|
||||
log.Ctx(context.TODO()).Debug("IndexNode task already in active task list", zap.String("TaskID", tName))
|
||||
}
|
||||
|
||||
queue.activeTasks[tName] = t
|
||||
|
@ -124,7 +124,7 @@ func (queue *IndexTaskQueue) PopActiveTask(tName string) task {
|
|||
delete(queue.activeTasks, tName)
|
||||
return t
|
||||
}
|
||||
log.Debug("IndexNode task was not found in the active task list", zap.String("TaskName", tName))
|
||||
log.Ctx(queue.sched.ctx).Debug("IndexNode task was not found in the active task list", zap.String("TaskName", tName))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -246,7 +246,7 @@ func (sched *TaskScheduler) processTask(t task, q TaskQueue) {
|
|||
}
|
||||
|
||||
func (sched *TaskScheduler) indexBuildLoop() {
|
||||
log.Debug("IndexNode TaskScheduler start build loop ...")
|
||||
log.Ctx(sched.ctx).Debug("IndexNode TaskScheduler start build loop ...")
|
||||
defer sched.wg.Done()
|
||||
for {
|
||||
select {
|
||||
|
|
|
@ -127,13 +127,13 @@ func (st *statsTask) PreExecute(ctx context.Context) error {
|
|||
|
||||
if err := binlog.DecompressBinLogWithRootPath(st.req.GetStorageConfig().GetRootPath(), storage.InsertBinlog, st.req.GetCollectionID(), st.req.GetPartitionID(),
|
||||
st.req.GetSegmentID(), st.req.GetInsertLogs()); err != nil {
|
||||
log.Warn("Decompress insert binlog error", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("Decompress insert binlog error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
if err := binlog.DecompressBinLogWithRootPath(st.req.GetStorageConfig().GetRootPath(), storage.DeleteBinlog, st.req.GetCollectionID(), st.req.GetPartitionID(),
|
||||
st.req.GetSegmentID(), st.req.GetDeltaLogs()); err != nil {
|
||||
log.Warn("Decompress delta binlog error", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("Decompress delta binlog error", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -162,7 +162,7 @@ func (st *statsTask) sortSegment(ctx context.Context) ([]*datapb.FieldBinlog, er
|
|||
bm25FieldIds := compaction.GetBM25FieldIDs(st.req.GetSchema())
|
||||
writer, err := compaction.NewSegmentWriter(st.req.GetSchema(), numRows, statsBatchSize, st.req.GetTargetSegmentID(), st.req.GetPartitionID(), st.req.GetCollectionID(), bm25FieldIds)
|
||||
if err != nil {
|
||||
log.Warn("sort segment wrong, unable to init segment writer",
|
||||
log.Ctx(ctx).Warn("sort segment wrong, unable to init segment writer",
|
||||
zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
@ -181,7 +181,7 @@ func (st *statsTask) sortSegment(ctx context.Context) ([]*datapb.FieldBinlog, er
|
|||
downloadStart := time.Now()
|
||||
values, err := st.downloadData(ctx, numRows, writer.GetPkID(), bm25FieldIds)
|
||||
if err != nil {
|
||||
log.Warn("download data failed", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("download data failed", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
downloadCost = time.Since(downloadStart)
|
||||
|
@ -195,7 +195,7 @@ func (st *statsTask) sortSegment(ctx context.Context) ([]*datapb.FieldBinlog, er
|
|||
for i, v := range values {
|
||||
err := writer.Write(v)
|
||||
if err != nil {
|
||||
log.Warn("write value wrong, failed to writer row", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("write value wrong, failed to writer row", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -203,7 +203,7 @@ func (st *statsTask) sortSegment(ctx context.Context) ([]*datapb.FieldBinlog, er
|
|||
serWriteStart := time.Now()
|
||||
binlogNum, kvs, partialBinlogs, err := serializeWrite(ctx, st.req.GetStartLogID()+st.logIDOffset, writer)
|
||||
if err != nil {
|
||||
log.Warn("stats wrong, failed to serialize writer", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("stats wrong, failed to serialize writer", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
serWriteTimeCost += time.Since(serWriteStart)
|
||||
|
@ -214,7 +214,7 @@ func (st *statsTask) sortSegment(ctx context.Context) ([]*datapb.FieldBinlog, er
|
|||
flushBatchCount++
|
||||
st.logIDOffset += binlogNum
|
||||
if st.req.GetStartLogID()+st.logIDOffset >= st.req.GetEndLogID() {
|
||||
log.Warn("binlog files too much, log is not enough", zap.Int64("taskID", st.req.GetTaskID()),
|
||||
log.Ctx(ctx).Warn("binlog files too much, log is not enough", zap.Int64("taskID", st.req.GetTaskID()),
|
||||
zap.Int64("binlog num", binlogNum), zap.Int64("startLogID", st.req.GetStartLogID()),
|
||||
zap.Int64("endLogID", st.req.GetEndLogID()), zap.Int64("logIDOffset", st.logIDOffset))
|
||||
return nil, fmt.Errorf("binlog files too much, log is not enough")
|
||||
|
@ -226,7 +226,7 @@ func (st *statsTask) sortSegment(ctx context.Context) ([]*datapb.FieldBinlog, er
|
|||
serWriteStart := time.Now()
|
||||
binlogNum, kvs, partialBinlogs, err := serializeWrite(ctx, st.req.GetStartLogID()+st.logIDOffset, writer)
|
||||
if err != nil {
|
||||
log.Warn("stats wrong, failed to serialize writer", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("stats wrong, failed to serialize writer", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
serWriteTimeCost += time.Since(serWriteStart)
|
||||
|
@ -239,14 +239,14 @@ func (st *statsTask) sortSegment(ctx context.Context) ([]*datapb.FieldBinlog, er
|
|||
|
||||
err = conc.AwaitAll(uploadFutures...)
|
||||
if err != nil {
|
||||
log.Warn("stats wrong, failed to upload kvs", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("stats wrong, failed to upload kvs", zap.Int64("taskID", st.req.GetTaskID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
serWriteStart := time.Now()
|
||||
binlogNums, sPath, err := statSerializeWrite(ctx, st.binlogIO, st.req.GetStartLogID()+st.logIDOffset, writer, numRows)
|
||||
if err != nil {
|
||||
log.Warn("stats wrong, failed to serialize write segment stats", zap.Int64("taskID", st.req.GetTaskID()),
|
||||
log.Ctx(ctx).Warn("stats wrong, failed to serialize write segment stats", zap.Int64("taskID", st.req.GetTaskID()),
|
||||
zap.Int64("remaining row count", numRows), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
@ -258,7 +258,7 @@ func (st *statsTask) sortSegment(ctx context.Context) ([]*datapb.FieldBinlog, er
|
|||
if len(bm25FieldIds) > 0 {
|
||||
binlogNums, bm25StatsLogs, err = bm25SerializeWrite(ctx, st.binlogIO, st.req.GetStartLogID()+st.logIDOffset, writer, numRows)
|
||||
if err != nil {
|
||||
log.Warn("compact wrong, failed to serialize write segment bm25 stats", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("compact wrong, failed to serialize write segment bm25 stats", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
st.logIDOffset += binlogNums
|
||||
|
@ -288,7 +288,7 @@ func (st *statsTask) sortSegment(ctx context.Context) ([]*datapb.FieldBinlog, er
|
|||
st.req.GetInsertChannel(),
|
||||
int64(len(values)), insertLogs, statsLogs, bm25StatsLogs)
|
||||
|
||||
log.Info("sort segment end",
|
||||
log.Ctx(ctx).Info("sort segment end",
|
||||
zap.String("clusterID", st.req.GetClusterID()),
|
||||
zap.Int64("taskID", st.req.GetTaskID()),
|
||||
zap.Int64("collectionID", st.req.GetCollectionID()),
|
||||
|
@ -330,7 +330,7 @@ func (st *statsTask) Execute(ctx context.Context) error {
|
|||
st.req.GetTaskID(),
|
||||
insertLogs)
|
||||
if err != nil {
|
||||
log.Warn("stats wrong, failed to create text index", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("stats wrong, failed to create text index", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
@ -569,7 +569,7 @@ func statSerializeWrite(ctx context.Context, io io.BinlogIO, startID int64, writ
|
|||
},
|
||||
}
|
||||
if err := io.Upload(ctx, kvs); err != nil {
|
||||
log.Warn("failed to upload insert log", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("failed to upload insert log", zap.Error(err))
|
||||
return binlogNum, nil, err
|
||||
}
|
||||
|
||||
|
@ -607,7 +607,7 @@ func bm25SerializeWrite(ctx context.Context, io io.BinlogIO, startID int64, writ
|
|||
}
|
||||
|
||||
if err := io.Upload(ctx, kvs); err != nil {
|
||||
log.Warn("failed to upload bm25 log", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("failed to upload bm25 log", zap.Error(err))
|
||||
return 0, nil, err
|
||||
}
|
||||
|
||||
|
|
|
@ -70,7 +70,7 @@ func (i *IndexNode) storeIndexTaskState(ClusterID string, buildID UniqueID, stat
|
|||
i.stateLock.Lock()
|
||||
defer i.stateLock.Unlock()
|
||||
if task, ok := i.indexTasks[key]; ok {
|
||||
log.Debug("IndexNode store task state", zap.String("clusterID", ClusterID), zap.Int64("buildID", buildID),
|
||||
log.Ctx(i.loopCtx).Debug("IndexNode store task state", zap.String("clusterID", ClusterID), zap.Int64("buildID", buildID),
|
||||
zap.String("state", state.String()), zap.String("fail reason", failReason))
|
||||
task.state = state
|
||||
task.failReason = failReason
|
||||
|
|
|
@ -67,7 +67,7 @@ func NewEtcdKV(client *clientv3.Client, rootPath string, options ...Option) *etc
|
|||
|
||||
// Close closes the connection to etcd.
|
||||
func (kv *etcdKV) Close() {
|
||||
log.Debug("etcd kv closed", zap.String("path", kv.rootPath))
|
||||
log.Ctx(context.TODO()).Debug("etcd kv closed", zap.String("path", kv.rootPath))
|
||||
}
|
||||
|
||||
// GetPath returns the path of the key.
|
||||
|
|
|
@ -277,14 +277,14 @@ func (kc *Catalog) AddSegment(ctx context.Context, segment *datapb.SegmentInfo)
|
|||
func (kc *Catalog) LoadFromSegmentPath(ctx context.Context, colID, partID, segID typeutil.UniqueID) (*datapb.SegmentInfo, error) {
|
||||
v, err := kc.MetaKv.Load(ctx, buildSegmentPath(colID, partID, segID))
|
||||
if err != nil {
|
||||
log.Error("(testing only) failed to load segment info by segment path")
|
||||
log.Ctx(context.TODO()).Error("(testing only) failed to load segment info by segment path")
|
||||
return nil, err
|
||||
}
|
||||
|
||||
segInfo := &datapb.SegmentInfo{}
|
||||
err = proto.Unmarshal([]byte(v), segInfo)
|
||||
if err != nil {
|
||||
log.Error("(testing only) failed to unmarshall segment info")
|
||||
log.Ctx(context.TODO()).Error("(testing only) failed to unmarshall segment info")
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -358,7 +358,7 @@ func (kc *Catalog) SaveByBatch(ctx context.Context, kvs map[string]string) error
|
|||
}
|
||||
err := etcd.SaveByBatchWithLimit(kvs, util.MaxEtcdTxnNum, saveFn)
|
||||
if err != nil {
|
||||
log.Error("failed to save by batch", zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to save by batch", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
|
@ -434,10 +434,10 @@ func (kc *Catalog) MarkChannelAdded(ctx context.Context, channel string) error {
|
|||
key := buildChannelRemovePath(channel)
|
||||
err := kc.MetaKv.Save(ctx, key, NonRemoveFlagTomestone)
|
||||
if err != nil {
|
||||
log.Error("failed to mark channel added", zap.String("channel", channel), zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to mark channel added", zap.String("channel", channel), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Info("NON remove flag tombstone added", zap.String("channel", channel))
|
||||
log.Ctx(ctx).Info("NON remove flag tombstone added", zap.String("channel", channel))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -445,10 +445,10 @@ func (kc *Catalog) MarkChannelDeleted(ctx context.Context, channel string) error
|
|||
key := buildChannelRemovePath(channel)
|
||||
err := kc.MetaKv.Save(ctx, key, RemoveFlagTomestone)
|
||||
if err != nil {
|
||||
log.Error("Failed to mark channel dropped", zap.String("channel", channel), zap.Error(err))
|
||||
log.Ctx(ctx).Error("Failed to mark channel dropped", zap.String("channel", channel), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Info("remove flag tombstone added", zap.String("channel", channel))
|
||||
log.Ctx(ctx).Info("remove flag tombstone added", zap.String("channel", channel))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -470,7 +470,7 @@ func (kc *Catalog) ChannelExists(ctx context.Context, channel string) bool {
|
|||
// DropChannel removes channel remove flag after whole procedure is finished
|
||||
func (kc *Catalog) DropChannel(ctx context.Context, channel string) error {
|
||||
key := buildChannelRemovePath(channel)
|
||||
log.Info("removing channel remove path", zap.String("channel", channel))
|
||||
log.Ctx(ctx).Info("removing channel remove path", zap.String("channel", channel))
|
||||
return kc.MetaKv.Remove(ctx, key)
|
||||
}
|
||||
|
||||
|
@ -480,7 +480,7 @@ func (kc *Catalog) ListChannelCheckpoint(ctx context.Context) (map[string]*msgpb
|
|||
channelCP := &msgpb.MsgPosition{}
|
||||
err := proto.Unmarshal(value, channelCP)
|
||||
if err != nil {
|
||||
log.Error("unmarshal channelCP failed when ListChannelCheckpoint", zap.Error(err))
|
||||
log.Ctx(ctx).Error("unmarshal channelCP failed when ListChannelCheckpoint", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
ss := strings.Split(string(key), "/")
|
||||
|
@ -566,7 +566,7 @@ func (kc *Catalog) ListIndexes(ctx context.Context) ([]*model.Index, error) {
|
|||
meta := &indexpb.FieldIndex{}
|
||||
err := proto.Unmarshal(value, meta)
|
||||
if err != nil {
|
||||
log.Warn("unmarshal index info failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("unmarshal index info failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -612,7 +612,7 @@ func (kc *Catalog) DropIndex(ctx context.Context, collID typeutil.UniqueID, drop
|
|||
|
||||
err := kc.MetaKv.Remove(ctx, key)
|
||||
if err != nil {
|
||||
log.Error("drop collection index meta fail", zap.Int64("collectionID", collID),
|
||||
log.Ctx(ctx).Error("drop collection index meta fail", zap.Int64("collectionID", collID),
|
||||
zap.Int64("indexID", dropIdxID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
@ -629,7 +629,7 @@ func (kc *Catalog) CreateSegmentIndex(ctx context.Context, segIdx *model.Segment
|
|||
}
|
||||
err = kc.MetaKv.Save(ctx, key, string(value))
|
||||
if err != nil {
|
||||
log.Error("failed to save segment index meta in etcd", zap.Int64("buildID", segIdx.BuildID),
|
||||
log.Ctx(ctx).Error("failed to save segment index meta in etcd", zap.Int64("buildID", segIdx.BuildID),
|
||||
zap.Int64("segmentID", segIdx.SegmentID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
@ -642,7 +642,7 @@ func (kc *Catalog) ListSegmentIndexes(ctx context.Context) ([]*model.SegmentInde
|
|||
segmentIndexInfo := &indexpb.SegmentIndex{}
|
||||
err := proto.Unmarshal(value, segmentIndexInfo)
|
||||
if err != nil {
|
||||
log.Warn("unmarshal segment index info failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("unmarshal segment index info failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -676,7 +676,7 @@ func (kc *Catalog) DropSegmentIndex(ctx context.Context, collID, partID, segID,
|
|||
|
||||
err := kc.MetaKv.Remove(ctx, key)
|
||||
if err != nil {
|
||||
log.Error("drop segment index meta fail", zap.Int64("buildID", buildID), zap.Error(err))
|
||||
log.Ctx(ctx).Error("drop segment index meta fail", zap.Int64("buildID", buildID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
|
|
@ -8,7 +8,6 @@ import (
|
|||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/klauspost/compress/zstd"
|
||||
"github.com/pingcap/log"
|
||||
"github.com/samber/lo"
|
||||
"go.uber.org/zap"
|
||||
"google.golang.org/protobuf/proto"
|
||||
|
@ -16,6 +15,7 @@ import (
|
|||
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/pkg/kv"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/util/compressor"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
)
|
||||
|
|
|
@ -322,13 +322,13 @@ func (kc *Catalog) CreateCredential(ctx context.Context, credential *model.Crede
|
|||
k := fmt.Sprintf("%s/%s", CredentialPrefix, credential.Username)
|
||||
v, err := json.Marshal(&internalpb.CredentialInfo{EncryptedPassword: credential.EncryptedPassword})
|
||||
if err != nil {
|
||||
log.Error("create credential marshal fail", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Error("create credential marshal fail", zap.String("key", k), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
err = kc.Txn.Save(ctx, k, string(v))
|
||||
if err != nil {
|
||||
log.Error("create credential persist meta fail", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Error("create credential persist meta fail", zap.String("key", k), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -562,9 +562,9 @@ func (kc *Catalog) GetCredential(ctx context.Context, username string) (*model.C
|
|||
v, err := kc.Txn.Load(ctx, k)
|
||||
if err != nil {
|
||||
if errors.Is(err, merr.ErrIoKeyNotFound) {
|
||||
log.Debug("not found the user", zap.String("key", k))
|
||||
log.Ctx(ctx).Debug("not found the user", zap.String("key", k))
|
||||
} else {
|
||||
log.Warn("get credential meta fail", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("get credential meta fail", zap.String("key", k), zap.Error(err))
|
||||
}
|
||||
return nil, err
|
||||
}
|
||||
|
@ -733,7 +733,7 @@ func (kc *Catalog) DropCredential(ctx context.Context, username string) error {
|
|||
k := fmt.Sprintf("%s/%s", CredentialPrefix, username)
|
||||
userResults, err := kc.ListUser(ctx, util.DefaultTenant, &milvuspb.UserEntity{Name: username}, true)
|
||||
if err != nil && !errors.Is(err, merr.ErrIoKeyNotFound) {
|
||||
log.Warn("fail to list user", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to list user", zap.String("key", k), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
deleteKeys := make([]string, 0, len(userResults)+1)
|
||||
|
@ -748,7 +748,7 @@ func (kc *Catalog) DropCredential(ctx context.Context, username string) error {
|
|||
}
|
||||
err = kc.Txn.MultiRemove(ctx, deleteKeys)
|
||||
if err != nil {
|
||||
log.Warn("fail to drop credential", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to drop credential", zap.String("key", k), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -766,7 +766,7 @@ func (kc *Catalog) GetCollectionByName(ctx context.Context, dbID int64, collecti
|
|||
prefix := getDatabasePrefix(dbID)
|
||||
_, vals, err := kc.Snapshot.LoadWithPrefix(ctx, prefix, ts)
|
||||
if err != nil {
|
||||
log.Warn("get collection meta fail", zap.String("collectionName", collectionName), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("get collection meta fail", zap.String("collectionName", collectionName), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -774,7 +774,7 @@ func (kc *Catalog) GetCollectionByName(ctx context.Context, dbID int64, collecti
|
|||
colMeta := pb.CollectionInfo{}
|
||||
err = proto.Unmarshal([]byte(val), &colMeta)
|
||||
if err != nil {
|
||||
log.Warn("get collection meta unmarshal fail", zap.String("collectionName", collectionName), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("get collection meta unmarshal fail", zap.String("collectionName", collectionName), zap.Error(err))
|
||||
continue
|
||||
}
|
||||
if colMeta.Schema.Name == collectionName {
|
||||
|
@ -790,7 +790,7 @@ func (kc *Catalog) ListCollections(ctx context.Context, dbID int64, ts typeutil.
|
|||
prefix := getDatabasePrefix(dbID)
|
||||
_, vals, err := kc.Snapshot.LoadWithPrefix(ctx, prefix, ts)
|
||||
if err != nil {
|
||||
log.Error("get collections meta fail",
|
||||
log.Ctx(ctx).Error("get collections meta fail",
|
||||
zap.String("prefix", prefix),
|
||||
zap.Uint64("timestamp", ts),
|
||||
zap.Error(err))
|
||||
|
@ -803,17 +803,17 @@ func (kc *Catalog) ListCollections(ctx context.Context, dbID int64, ts typeutil.
|
|||
collMeta := &pb.CollectionInfo{}
|
||||
err := proto.Unmarshal([]byte(val), collMeta)
|
||||
if err != nil {
|
||||
log.Warn("unmarshal collection info failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("unmarshal collection info failed", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
kc.fixDefaultDBIDConsistency(ctx, collMeta, ts)
|
||||
colls = append(colls, collMeta)
|
||||
}
|
||||
log.Info("unmarshal all collection details cost", zap.Int64("db", dbID), zap.Duration("cost", time.Since(start)))
|
||||
log.Ctx(ctx).Info("unmarshal all collection details cost", zap.Int64("db", dbID), zap.Duration("cost", time.Since(start)))
|
||||
|
||||
start = time.Now()
|
||||
ret, err := kc.batchAppendPartitionAndFieldsInfo(ctx, colls, ts)
|
||||
log.Info("append partition and fields info cost", zap.Int64("db", dbID), zap.Duration("cost", time.Since(start)))
|
||||
log.Ctx(ctx).Info("append partition and fields info cost", zap.Int64("db", dbID), zap.Duration("cost", time.Since(start)))
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -920,7 +920,7 @@ func (kc *Catalog) ListCredentials(ctx context.Context) ([]string, error) {
|
|||
func (kc *Catalog) ListCredentialsWithPasswd(ctx context.Context) (map[string]string, error) {
|
||||
keys, values, err := kc.Txn.LoadWithPrefix(ctx, CredentialPrefix)
|
||||
if err != nil {
|
||||
log.Error("list all credential usernames fail", zap.String("prefix", CredentialPrefix), zap.Error(err))
|
||||
log.Ctx(ctx).Error("list all credential usernames fail", zap.String("prefix", CredentialPrefix), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -928,13 +928,13 @@ func (kc *Catalog) ListCredentialsWithPasswd(ctx context.Context) (map[string]st
|
|||
for i := range keys {
|
||||
username := typeutil.After(keys[i], UserSubPrefix+"/")
|
||||
if len(username) == 0 {
|
||||
log.Warn("no username extract from path:", zap.String("path", keys[i]))
|
||||
log.Ctx(ctx).Warn("no username extract from path:", zap.String("path", keys[i]))
|
||||
continue
|
||||
}
|
||||
credential := &internalpb.CredentialInfo{}
|
||||
err := json.Unmarshal([]byte(values[i]), credential)
|
||||
if err != nil {
|
||||
log.Error("credential unmarshal fail", zap.String("key", keys[i]), zap.Error(err))
|
||||
log.Ctx(ctx).Error("credential unmarshal fail", zap.String("key", keys[i]), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
users[username] = credential.EncryptedPassword
|
||||
|
@ -949,7 +949,7 @@ func (kc *Catalog) save(ctx context.Context, k string) error {
|
|||
return err
|
||||
}
|
||||
if err == nil {
|
||||
log.Debug("the key has existed", zap.String("key", k))
|
||||
log.Ctx(ctx).Debug("the key has existed", zap.String("key", k))
|
||||
return common.NewIgnorableError(fmt.Errorf("the key[%s] has existed", k))
|
||||
}
|
||||
return kc.Txn.Save(ctx, k, "")
|
||||
|
@ -961,6 +961,7 @@ func (kc *Catalog) remove(ctx context.Context, k string) error {
|
|||
return err
|
||||
}
|
||||
if err != nil && errors.Is(err, merr.ErrIoKeyNotFound) {
|
||||
log.Ctx(ctx).Debug("the key isn't existed", zap.String("key", k))
|
||||
return common.NewIgnorableError(fmt.Errorf("the key[%s] isn't existed", k))
|
||||
}
|
||||
return kc.Txn.Remove(ctx, k)
|
||||
|
@ -970,7 +971,7 @@ func (kc *Catalog) CreateRole(ctx context.Context, tenant string, entity *milvus
|
|||
k := funcutil.HandleTenantForEtcdKey(RolePrefix, tenant, entity.Name)
|
||||
err := kc.save(ctx, k)
|
||||
if err != nil && !common.IsIgnorableError(err) {
|
||||
log.Warn("fail to save the role", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to save the role", zap.String("key", k), zap.Error(err))
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
@ -979,7 +980,7 @@ func (kc *Catalog) DropRole(ctx context.Context, tenant string, roleName string)
|
|||
k := funcutil.HandleTenantForEtcdKey(RolePrefix, tenant, roleName)
|
||||
roleResults, err := kc.ListRole(ctx, tenant, &milvuspb.RoleEntity{Name: roleName}, true)
|
||||
if err != nil && !errors.Is(err, merr.ErrIoKeyNotFound) {
|
||||
log.Warn("fail to list role", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to list role", zap.String("key", k), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -996,7 +997,7 @@ func (kc *Catalog) DropRole(ctx context.Context, tenant string, roleName string)
|
|||
|
||||
err = kc.Txn.MultiRemove(ctx, deleteKeys)
|
||||
if err != nil {
|
||||
log.Warn("fail to drop role", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to drop role", zap.String("key", k), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
|
@ -1008,12 +1009,12 @@ func (kc *Catalog) AlterUserRole(ctx context.Context, tenant string, userEntity
|
|||
if operateType == milvuspb.OperateUserRoleType_AddUserToRole {
|
||||
err = kc.save(ctx, k)
|
||||
if err != nil {
|
||||
log.Error("fail to save the user-role", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to save the user-role", zap.String("key", k), zap.Error(err))
|
||||
}
|
||||
} else if operateType == milvuspb.OperateUserRoleType_RemoveUserFromRole {
|
||||
err = kc.remove(ctx, k)
|
||||
if err != nil {
|
||||
log.Error("fail to remove the user-role", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to remove the user-role", zap.String("key", k), zap.Error(err))
|
||||
}
|
||||
} else {
|
||||
err = fmt.Errorf("invalid operate user role type, operate type: %d", operateType)
|
||||
|
@ -1029,14 +1030,14 @@ func (kc *Catalog) ListRole(ctx context.Context, tenant string, entity *milvuspb
|
|||
roleMappingKey := funcutil.HandleTenantForEtcdKey(RoleMappingPrefix, tenant, "")
|
||||
keys, _, err := kc.Txn.LoadWithPrefix(ctx, roleMappingKey)
|
||||
if err != nil {
|
||||
log.Error("fail to load role mappings", zap.String("key", roleMappingKey), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to load role mappings", zap.String("key", roleMappingKey), zap.Error(err))
|
||||
return results, err
|
||||
}
|
||||
|
||||
for _, key := range keys {
|
||||
roleMappingInfos := typeutil.AfterN(key, roleMappingKey+"/", "/")
|
||||
if len(roleMappingInfos) != 2 {
|
||||
log.Warn("invalid role mapping key", zap.String("string", key), zap.String("sub_string", roleMappingKey))
|
||||
log.Ctx(ctx).Warn("invalid role mapping key", zap.String("string", key), zap.String("sub_string", roleMappingKey))
|
||||
continue
|
||||
}
|
||||
username := roleMappingInfos[0]
|
||||
|
@ -1060,13 +1061,13 @@ func (kc *Catalog) ListRole(ctx context.Context, tenant string, entity *milvuspb
|
|||
roleKey := funcutil.HandleTenantForEtcdKey(RolePrefix, tenant, "")
|
||||
keys, _, err := kc.Txn.LoadWithPrefix(ctx, roleKey)
|
||||
if err != nil {
|
||||
log.Error("fail to load roles", zap.String("key", roleKey), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to load roles", zap.String("key", roleKey), zap.Error(err))
|
||||
return results, err
|
||||
}
|
||||
for _, key := range keys {
|
||||
infoArr := typeutil.AfterN(key, roleKey+"/", "/")
|
||||
if len(infoArr) != 1 || len(infoArr[0]) == 0 {
|
||||
log.Warn("invalid role key", zap.String("string", key), zap.String("sub_string", roleKey))
|
||||
log.Ctx(ctx).Warn("invalid role key", zap.String("string", key), zap.String("sub_string", roleKey))
|
||||
continue
|
||||
}
|
||||
appendRoleResult(infoArr[0])
|
||||
|
@ -1078,7 +1079,7 @@ func (kc *Catalog) ListRole(ctx context.Context, tenant string, entity *milvuspb
|
|||
roleKey := funcutil.HandleTenantForEtcdKey(RolePrefix, tenant, entity.Name)
|
||||
_, err := kc.Txn.Load(ctx, roleKey)
|
||||
if err != nil {
|
||||
log.Warn("fail to load a role", zap.String("key", roleKey), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to load a role", zap.String("key", roleKey), zap.Error(err))
|
||||
return results, err
|
||||
}
|
||||
appendRoleResult(entity.Name)
|
||||
|
@ -1092,13 +1093,13 @@ func (kc *Catalog) getRolesByUsername(ctx context.Context, tenant string, userna
|
|||
k := funcutil.HandleTenantForEtcdKey(RoleMappingPrefix, tenant, username)
|
||||
keys, _, err := kc.Txn.LoadWithPrefix(ctx, k)
|
||||
if err != nil {
|
||||
log.Error("fail to load role mappings by the username", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to load role mappings by the username", zap.String("key", k), zap.Error(err))
|
||||
return roles, err
|
||||
}
|
||||
for _, key := range keys {
|
||||
roleMappingInfos := typeutil.AfterN(key, k+"/", "/")
|
||||
if len(roleMappingInfos) != 1 {
|
||||
log.Warn("invalid role mapping key", zap.String("string", key), zap.String("sub_string", k))
|
||||
log.Ctx(ctx).Warn("invalid role mapping key", zap.String("string", key), zap.String("sub_string", k))
|
||||
continue
|
||||
}
|
||||
roles = append(roles, roleMappingInfos[0])
|
||||
|
@ -1114,7 +1115,7 @@ func (kc *Catalog) getUserResult(ctx context.Context, tenant string, username st
|
|||
}
|
||||
roleNames, err := kc.getRolesByUsername(ctx, tenant, username)
|
||||
if err != nil {
|
||||
log.Warn("fail to get roles by the username", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to get roles by the username", zap.Error(err))
|
||||
return result, err
|
||||
}
|
||||
var roles []*milvuspb.RoleEntity
|
||||
|
@ -1185,7 +1186,7 @@ func (kc *Catalog) AlterGrant(ctx context.Context, tenant string, entity *milvus
|
|||
if v, err = kc.Txn.Load(ctx, k); err == nil {
|
||||
idStr = v
|
||||
} else {
|
||||
log.Warn("fail to load grant privilege entity", zap.String("key", k), zap.Any("type", operateType), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to load grant privilege entity", zap.String("key", k), zap.Any("type", operateType), zap.Error(err))
|
||||
if funcutil.IsRevoke(operateType) {
|
||||
if errors.Is(err, merr.ErrIoKeyNotFound) {
|
||||
return common.NewIgnorableError(fmt.Errorf("the grant[%s] isn't existed", k))
|
||||
|
@ -1199,7 +1200,7 @@ func (kc *Catalog) AlterGrant(ctx context.Context, tenant string, entity *milvus
|
|||
idStr = crypto.MD5(k)
|
||||
err = kc.Txn.Save(ctx, k, idStr)
|
||||
if err != nil {
|
||||
log.Error("fail to allocate id when altering the grant", zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to allocate id when altering the grant", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
@ -1207,18 +1208,18 @@ func (kc *Catalog) AlterGrant(ctx context.Context, tenant string, entity *milvus
|
|||
k = funcutil.HandleTenantForEtcdKey(GranteeIDPrefix, tenant, fmt.Sprintf("%s/%s", idStr, privilegeName))
|
||||
_, err = kc.Txn.Load(ctx, k)
|
||||
if err != nil {
|
||||
log.Warn("fail to load the grantee id", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to load the grantee id", zap.String("key", k), zap.Error(err))
|
||||
if !errors.Is(err, merr.ErrIoKeyNotFound) {
|
||||
log.Warn("fail to load the grantee id", zap.String("key", k), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
log.Debug("not found the grantee id", zap.String("key", k))
|
||||
log.Ctx(ctx).Debug("not found the grantee id", zap.String("key", k))
|
||||
if funcutil.IsRevoke(operateType) {
|
||||
return common.NewIgnorableError(fmt.Errorf("the grantee-id[%s] isn't existed", k))
|
||||
}
|
||||
if funcutil.IsGrant(operateType) {
|
||||
if err = kc.Txn.Save(ctx, k, entity.Grantor.User.Name); err != nil {
|
||||
log.Error("fail to save the grantee id", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to save the grantee id", zap.String("key", k), zap.Error(err))
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
@ -1226,7 +1227,7 @@ func (kc *Catalog) AlterGrant(ctx context.Context, tenant string, entity *milvus
|
|||
}
|
||||
if funcutil.IsRevoke(operateType) {
|
||||
if err = kc.Txn.Remove(ctx, k); err != nil {
|
||||
log.Error("fail to remove the grantee id", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to remove the grantee id", zap.String("key", k), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return err
|
||||
|
@ -1247,13 +1248,13 @@ func (kc *Catalog) ListGrant(ctx context.Context, tenant string, entity *milvusp
|
|||
granteeIDKey := funcutil.HandleTenantForEtcdKey(GranteeIDPrefix, tenant, v)
|
||||
keys, values, err := kc.Txn.LoadWithPrefix(ctx, granteeIDKey)
|
||||
if err != nil {
|
||||
log.Error("fail to load the grantee ids", zap.String("key", granteeIDKey), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to load the grantee ids", zap.String("key", granteeIDKey), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
for i, key := range keys {
|
||||
granteeIDInfos := typeutil.AfterN(key, granteeIDKey+"/", "/")
|
||||
if len(granteeIDInfos) != 1 {
|
||||
log.Warn("invalid grantee id", zap.String("string", key), zap.String("sub_string", granteeIDKey))
|
||||
log.Ctx(ctx).Warn("invalid grantee id", zap.String("string", key), zap.String("sub_string", granteeIDKey))
|
||||
continue
|
||||
}
|
||||
privilegeName := util.PrivilegeNameForAPI(granteeIDInfos[0])
|
||||
|
@ -1297,7 +1298,7 @@ func (kc *Catalog) ListGrant(ctx context.Context, tenant string, entity *milvusp
|
|||
granteeKey = funcutil.HandleTenantForEtcdKey(GranteePrefix, tenant, fmt.Sprintf("%s/%s/%s", entity.Role.Name, entity.Object.Name, funcutil.CombineObjectName(entity.DbName, entity.ObjectName)))
|
||||
v, err := kc.Txn.Load(ctx, granteeKey)
|
||||
if err != nil {
|
||||
log.Error("fail to load the grant privilege entity", zap.String("key", granteeKey), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to load the grant privilege entity", zap.String("key", granteeKey), zap.Error(err))
|
||||
return entities, err
|
||||
}
|
||||
err = appendGrantEntity(v, entity.Object.Name, funcutil.CombineObjectName(entity.DbName, entity.ObjectName))
|
||||
|
@ -1308,13 +1309,13 @@ func (kc *Catalog) ListGrant(ctx context.Context, tenant string, entity *milvusp
|
|||
granteeKey = funcutil.HandleTenantForEtcdKey(GranteePrefix, tenant, entity.Role.Name)
|
||||
keys, values, err := kc.Txn.LoadWithPrefix(ctx, granteeKey)
|
||||
if err != nil {
|
||||
log.Error("fail to load grant privilege entities", zap.String("key", granteeKey), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to load grant privilege entities", zap.String("key", granteeKey), zap.Error(err))
|
||||
return entities, err
|
||||
}
|
||||
for i, key := range keys {
|
||||
grantInfos := typeutil.AfterN(key, granteeKey+"/", "/")
|
||||
if len(grantInfos) != 2 {
|
||||
log.Warn("invalid grantee key", zap.String("string", key), zap.String("sub_string", granteeKey))
|
||||
log.Ctx(ctx).Warn("invalid grantee key", zap.String("string", key), zap.String("sub_string", granteeKey))
|
||||
continue
|
||||
}
|
||||
err = appendGrantEntity(values[i], grantInfos[0], grantInfos[1])
|
||||
|
@ -1339,7 +1340,7 @@ func (kc *Catalog) DeleteGrant(ctx context.Context, tenant string, role *milvusp
|
|||
// the values are the grantee id list
|
||||
_, values, err := kc.Txn.LoadWithPrefix(ctx, k)
|
||||
if err != nil {
|
||||
log.Warn("fail to load grant privilege entities", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to load grant privilege entities", zap.String("key", k), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
for _, v := range values {
|
||||
|
@ -1348,7 +1349,7 @@ func (kc *Catalog) DeleteGrant(ctx context.Context, tenant string, role *milvusp
|
|||
}
|
||||
|
||||
if err = kc.Txn.MultiSaveAndRemoveWithPrefix(ctx, nil, removeKeys); err != nil {
|
||||
log.Error("fail to remove with the prefix", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to remove with the prefix", zap.String("key", k), zap.Error(err))
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
@ -1358,26 +1359,26 @@ func (kc *Catalog) ListPolicy(ctx context.Context, tenant string) ([]string, err
|
|||
granteeKey := funcutil.HandleTenantForEtcdKey(GranteePrefix, tenant, "")
|
||||
keys, values, err := kc.Txn.LoadWithPrefix(ctx, granteeKey)
|
||||
if err != nil {
|
||||
log.Error("fail to load all grant privilege entities", zap.String("key", granteeKey), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to load all grant privilege entities", zap.String("key", granteeKey), zap.Error(err))
|
||||
return []string{}, err
|
||||
}
|
||||
|
||||
for i, key := range keys {
|
||||
grantInfos := typeutil.AfterN(key, granteeKey+"/", "/")
|
||||
if len(grantInfos) != 3 {
|
||||
log.Warn("invalid grantee key", zap.String("string", key), zap.String("sub_string", granteeKey))
|
||||
log.Ctx(ctx).Warn("invalid grantee key", zap.String("string", key), zap.String("sub_string", granteeKey))
|
||||
continue
|
||||
}
|
||||
granteeIDKey := funcutil.HandleTenantForEtcdKey(GranteeIDPrefix, tenant, values[i])
|
||||
idKeys, _, err := kc.Txn.LoadWithPrefix(ctx, granteeIDKey)
|
||||
if err != nil {
|
||||
log.Error("fail to load the grantee ids", zap.String("key", granteeIDKey), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to load the grantee ids", zap.String("key", granteeIDKey), zap.Error(err))
|
||||
return []string{}, err
|
||||
}
|
||||
for _, idKey := range idKeys {
|
||||
granteeIDInfos := typeutil.AfterN(idKey, granteeIDKey+"/", "/")
|
||||
if len(granteeIDInfos) != 1 {
|
||||
log.Warn("invalid grantee id", zap.String("string", idKey), zap.String("sub_string", granteeIDKey))
|
||||
log.Ctx(ctx).Warn("invalid grantee id", zap.String("string", idKey), zap.String("sub_string", granteeIDKey))
|
||||
continue
|
||||
}
|
||||
dbName, objectName := funcutil.SplitObjectName(grantInfos[2])
|
||||
|
@ -1393,14 +1394,14 @@ func (kc *Catalog) ListUserRole(ctx context.Context, tenant string) ([]string, e
|
|||
k := funcutil.HandleTenantForEtcdKey(RoleMappingPrefix, tenant, "")
|
||||
keys, _, err := kc.Txn.LoadWithPrefix(ctx, k)
|
||||
if err != nil {
|
||||
log.Error("fail to load all user-role mappings", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Error("fail to load all user-role mappings", zap.String("key", k), zap.Error(err))
|
||||
return []string{}, err
|
||||
}
|
||||
|
||||
for _, key := range keys {
|
||||
userRolesInfos := typeutil.AfterN(key, k+"/", "/")
|
||||
if len(userRolesInfos) != 2 {
|
||||
log.Warn("invalid user-role key", zap.String("string", key), zap.String("sub_string", k))
|
||||
log.Ctx(ctx).Warn("invalid user-role key", zap.String("string", key), zap.String("sub_string", k))
|
||||
continue
|
||||
}
|
||||
userRoles = append(userRoles, funcutil.EncodeUserRoleCache(userRolesInfos[0], userRolesInfos[1]))
|
||||
|
@ -1478,12 +1479,12 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp
|
|||
needRollbackPrivilegeGroups := make([]*milvuspb.PrivilegeGroupInfo, 0)
|
||||
defer func() {
|
||||
if err != nil {
|
||||
log.Warn("failed to restore rbac, try to rollback", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("failed to restore rbac, try to rollback", zap.Error(err))
|
||||
// roll back role
|
||||
for _, role := range needRollbackRole {
|
||||
err = kc.DropRole(ctx, tenant, role.Name)
|
||||
if err != nil {
|
||||
log.Warn("failed to rollback roles after restore failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("failed to rollback roles after restore failed", zap.Error(err))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1491,7 +1492,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp
|
|||
for _, grant := range needRollbackGrants {
|
||||
err = kc.AlterGrant(ctx, tenant, grant, milvuspb.OperatePrivilegeType_Revoke)
|
||||
if err != nil {
|
||||
log.Warn("failed to rollback grants after restore failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("failed to rollback grants after restore failed", zap.Error(err))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1499,7 +1500,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp
|
|||
// roll back user
|
||||
err = kc.DropCredential(ctx, user.User)
|
||||
if err != nil {
|
||||
log.Warn("failed to rollback users after restore failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("failed to rollback users after restore failed", zap.Error(err))
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1507,7 +1508,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp
|
|||
for _, group := range needRollbackPrivilegeGroups {
|
||||
err = kc.DropPrivilegeGroup(ctx, group.GroupName)
|
||||
if err != nil {
|
||||
log.Warn("failed to rollback privilege groups after restore failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("failed to rollback privilege groups after restore failed", zap.Error(err))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -1521,7 +1522,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp
|
|||
existRoleMap := lo.SliceToMap(existRoles, func(entity *milvuspb.RoleResult) (string, struct{}) { return entity.GetRole().GetName(), struct{}{} })
|
||||
for _, role := range meta.Roles {
|
||||
if _, ok := existRoleMap[role.GetName()]; ok {
|
||||
log.Warn("failed to restore, role already exists", zap.String("role", role.GetName()))
|
||||
log.Ctx(ctx).Warn("failed to restore, role already exists", zap.String("role", role.GetName()))
|
||||
err = errors.Newf("role [%s] already exists", role.GetName())
|
||||
return err
|
||||
}
|
||||
|
@ -1540,7 +1541,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp
|
|||
existPrivGroupMap := lo.SliceToMap(existPrivGroups, func(entity *milvuspb.PrivilegeGroupInfo) (string, struct{}) { return entity.GroupName, struct{}{} })
|
||||
for _, group := range meta.PrivilegeGroups {
|
||||
if _, ok := existPrivGroupMap[group.GroupName]; ok {
|
||||
log.Warn("failed to restore, privilege group already exists", zap.String("group", group.GroupName))
|
||||
log.Ctx(ctx).Warn("failed to restore, privilege group already exists", zap.String("group", group.GroupName))
|
||||
err = errors.Newf("privilege group [%s] already exists", group.GroupName)
|
||||
return err
|
||||
}
|
||||
|
@ -1564,7 +1565,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp
|
|||
} else if _, ok := existPrivGroupMap[privName]; ok {
|
||||
grant.Grantor.Privilege.Name = util.PrivilegeGroupNameForMetastore(privName)
|
||||
} else {
|
||||
log.Warn("failed to restore, privilege group does not exist", zap.String("group", privName))
|
||||
log.Ctx(ctx).Warn("failed to restore, privilege group does not exist", zap.String("group", privName))
|
||||
err = errors.Newf("privilege group [%s] does not exist", privName)
|
||||
return err
|
||||
}
|
||||
|
@ -1583,7 +1584,7 @@ func (kc *Catalog) RestoreRBAC(ctx context.Context, tenant string, meta *milvusp
|
|||
existUserMap := lo.SliceToMap(existUser, func(entity *milvuspb.UserResult) (string, struct{}) { return entity.GetUser().GetName(), struct{}{} })
|
||||
for _, user := range meta.Users {
|
||||
if _, ok := existUserMap[user.GetUser()]; ok {
|
||||
log.Info("failed to restore, user already exists", zap.String("user", user.GetUser()))
|
||||
log.Ctx(ctx).Info("failed to restore, user already exists", zap.String("user", user.GetUser()))
|
||||
err = errors.Newf("user [%s] already exists", user.GetUser())
|
||||
return err
|
||||
}
|
||||
|
@ -1619,13 +1620,13 @@ func (kc *Catalog) GetPrivilegeGroup(ctx context.Context, groupName string) (*mi
|
|||
if errors.Is(err, merr.ErrIoKeyNotFound) {
|
||||
return nil, fmt.Errorf("privilege group [%s] does not exist", groupName)
|
||||
}
|
||||
log.Error("failed to load privilege group", zap.String("group", groupName), zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to load privilege group", zap.String("group", groupName), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
privGroupInfo := &milvuspb.PrivilegeGroupInfo{}
|
||||
err = proto.Unmarshal([]byte(val), privGroupInfo)
|
||||
if err != nil {
|
||||
log.Error("failed to unmarshal privilege group info", zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to unmarshal privilege group info", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
return privGroupInfo, nil
|
||||
|
@ -1635,7 +1636,7 @@ func (kc *Catalog) DropPrivilegeGroup(ctx context.Context, groupName string) err
|
|||
k := BuildPrivilegeGroupkey(groupName)
|
||||
err := kc.Txn.Remove(ctx, k)
|
||||
if err != nil {
|
||||
log.Warn("fail to drop privilege group", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to drop privilege group", zap.String("key", k), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
|
@ -1649,11 +1650,11 @@ func (kc *Catalog) SavePrivilegeGroup(ctx context.Context, data *milvuspb.Privil
|
|||
}
|
||||
v, err := proto.Marshal(groupInfo)
|
||||
if err != nil {
|
||||
log.Error("failed to marshal privilege group info", zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to marshal privilege group info", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
if err = kc.Txn.Save(ctx, k, string(v)); err != nil {
|
||||
log.Warn("fail to put privilege group", zap.String("key", k), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("fail to put privilege group", zap.String("key", k), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
return nil
|
||||
|
@ -1662,7 +1663,7 @@ func (kc *Catalog) SavePrivilegeGroup(ctx context.Context, data *milvuspb.Privil
|
|||
func (kc *Catalog) ListPrivilegeGroups(ctx context.Context) ([]*milvuspb.PrivilegeGroupInfo, error) {
|
||||
_, vals, err := kc.Txn.LoadWithPrefix(ctx, PrivilegeGroupPrefix)
|
||||
if err != nil {
|
||||
log.Error("failed to list privilege groups", zap.String("prefix", PrivilegeGroupPrefix), zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to list privilege groups", zap.String("prefix", PrivilegeGroupPrefix), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
privGroups := make([]*milvuspb.PrivilegeGroupInfo, 0, len(vals))
|
||||
|
@ -1670,7 +1671,7 @@ func (kc *Catalog) ListPrivilegeGroups(ctx context.Context) ([]*milvuspb.Privile
|
|||
privGroupInfo := &milvuspb.PrivilegeGroupInfo{}
|
||||
err = proto.Unmarshal([]byte(val), privGroupInfo)
|
||||
if err != nil {
|
||||
log.Error("failed to unmarshal privilege group info", zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to unmarshal privilege group info", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
privGroups = append(privGroups, privGroupInfo)
|
||||
|
|
|
@ -1664,7 +1664,7 @@ func TestRBAC_Role(t *testing.T) {
|
|||
}
|
||||
for _, test := range tests {
|
||||
t.Run(test.description, func(t *testing.T) {
|
||||
err := c.remove(context.TODO(), test.key)
|
||||
err := c.remove(ctx, test.key)
|
||||
if test.isValid {
|
||||
assert.NoError(t, err)
|
||||
} else {
|
||||
|
@ -1710,7 +1710,7 @@ func TestRBAC_Role(t *testing.T) {
|
|||
|
||||
for _, test := range tests {
|
||||
t.Run(test.description, func(t *testing.T) {
|
||||
err := c.save(context.TODO(), test.key)
|
||||
err := c.save(ctx, test.key)
|
||||
if test.isValid {
|
||||
assert.NoError(t, err)
|
||||
} else {
|
||||
|
@ -2081,7 +2081,7 @@ func TestRBAC_Role(t *testing.T) {
|
|||
|
||||
for _, test := range tests {
|
||||
t.Run(test.description, func(t *testing.T) {
|
||||
res, err := c.getUserResult(context.TODO(), tenant, test.user, test.includeRoleInfo)
|
||||
res, err := c.getUserResult(ctx, tenant, test.user, test.includeRoleInfo)
|
||||
|
||||
assert.Equal(t, test.user, res.GetUser().GetName())
|
||||
|
||||
|
|
|
@ -607,6 +607,7 @@ func (ss *SuffixSnapshot) Close() {
|
|||
|
||||
// startBackgroundGC the data will clean up if key ts!=0 and expired
|
||||
func (ss *SuffixSnapshot) startBackgroundGC(ctx context.Context) {
|
||||
log := log.Ctx(ctx)
|
||||
log.Debug("suffix snapshot GC goroutine start!")
|
||||
ticker := time.NewTicker(60 * time.Minute)
|
||||
defer ticker.Stop()
|
||||
|
@ -659,6 +660,7 @@ func (ss *SuffixSnapshot) batchRemoveExpiredKvs(ctx context.Context, keyGroup []
|
|||
// It walks through all keys with the snapshot prefix, groups them by original key,
|
||||
// and removes expired versions or all versions if the original key has been deleted
|
||||
func (ss *SuffixSnapshot) removeExpiredKvs(ctx context.Context, now time.Time) error {
|
||||
log := log.Ctx(ctx)
|
||||
ttlTime := paramtable.Get().ServiceParam.MetaStoreCfg.SnapshotTTLSeconds.GetAsDuration(time.Second)
|
||||
reserveTime := paramtable.Get().ServiceParam.MetaStoreCfg.SnapshotReserveTimeSeconds.GetAsDuration(time.Second)
|
||||
|
||||
|
|
|
@ -456,6 +456,7 @@ func SaveBinLog(ctx context.Context,
|
|||
schema *schemapb.CollectionSchema,
|
||||
chunkManager storage.ChunkManager,
|
||||
) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, error) {
|
||||
log := log.Ctx(ctx)
|
||||
binLogs, statsLogs, err := genStorageBlob(collectionID,
|
||||
partitionID,
|
||||
segmentID,
|
||||
|
@ -660,6 +661,7 @@ func SaveDeltaLog(collectionID int64,
|
|||
segmentID int64,
|
||||
cm storage.ChunkManager,
|
||||
) ([]*datapb.FieldBinlog, error) {
|
||||
log := log.Ctx(context.TODO())
|
||||
binlogWriter := storage.NewDeleteBinlogWriter(schemapb.DataType_String, collectionID, partitionID, segmentID)
|
||||
eventWriter, _ := binlogWriter.NextDeleteEventWriter()
|
||||
dData := &storage.DeleteData{
|
||||
|
|
|
@ -78,7 +78,7 @@ func (node *Proxy) GetComponentStates(ctx context.Context, req *milvuspb.GetComp
|
|||
Status: merr.Success(),
|
||||
}
|
||||
code := node.GetStateCode()
|
||||
log.Debug("Proxy current state", zap.String("StateCode", code.String()))
|
||||
log.Ctx(ctx).Debug("Proxy current state", zap.String("StateCode", code.String()))
|
||||
nodeID := common.NotRegisteredID
|
||||
if node.session != nil && node.session.Registered() {
|
||||
nodeID = node.session.ServerID
|
||||
|
@ -257,8 +257,7 @@ func (node *Proxy) CreateDatabase(ctx context.Context, request *milvuspb.CreateD
|
|||
replicateMsgStream: node.replicateMsgStream,
|
||||
}
|
||||
|
||||
log := log.With(
|
||||
zap.String("traceID", sp.SpanContext().TraceID().String()),
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.String("role", typeutil.ProxyRole),
|
||||
zap.String("dbName", request.DbName),
|
||||
)
|
||||
|
@ -326,8 +325,7 @@ func (node *Proxy) DropDatabase(ctx context.Context, request *milvuspb.DropDatab
|
|||
replicateMsgStream: node.replicateMsgStream,
|
||||
}
|
||||
|
||||
log := log.With(
|
||||
zap.String("traceID", sp.SpanContext().TraceID().String()),
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.String("role", typeutil.ProxyRole),
|
||||
zap.String("dbName", request.DbName),
|
||||
)
|
||||
|
@ -394,8 +392,7 @@ func (node *Proxy) ListDatabases(ctx context.Context, request *milvuspb.ListData
|
|||
rootCoord: node.rootCoord,
|
||||
}
|
||||
|
||||
log := log.With(
|
||||
zap.String("traceID", sp.SpanContext().TraceID().String()),
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.String("role", typeutil.ProxyRole),
|
||||
)
|
||||
|
||||
|
@ -3770,7 +3767,7 @@ func (node *Proxy) Query(ctx context.Context, request *milvuspb.QueryRequest) (*
|
|||
return res, err
|
||||
}
|
||||
|
||||
log.Debug(rpcDone(method))
|
||||
log.Ctx(ctx).Debug(rpcDone(method))
|
||||
|
||||
metrics.ProxyFunctionCall.WithLabelValues(
|
||||
strconv.FormatInt(paramtable.GetNodeID(), 10),
|
||||
|
@ -5998,14 +5995,14 @@ func (node *Proxy) TransferNode(ctx context.Context, request *milvuspb.TransferN
|
|||
|
||||
method := "TransferNode"
|
||||
if err := ValidateResourceGroupName(request.GetSourceResourceGroup()); err != nil {
|
||||
log.Warn("TransferNode failed",
|
||||
log.Ctx(ctx).Warn("TransferNode failed",
|
||||
zap.Error(err),
|
||||
)
|
||||
return getErrResponse(err, method, "", ""), nil
|
||||
}
|
||||
|
||||
if err := ValidateResourceGroupName(request.GetTargetResourceGroup()); err != nil {
|
||||
log.Warn("TransferNode failed",
|
||||
log.Ctx(ctx).Warn("TransferNode failed",
|
||||
zap.Error(err),
|
||||
)
|
||||
return getErrResponse(err, method, "", ""), nil
|
||||
|
@ -6065,14 +6062,14 @@ func (node *Proxy) TransferReplica(ctx context.Context, request *milvuspb.Transf
|
|||
|
||||
method := "TransferReplica"
|
||||
if err := ValidateResourceGroupName(request.GetSourceResourceGroup()); err != nil {
|
||||
log.Warn("TransferReplica failed",
|
||||
log.Ctx(ctx).Warn("TransferReplica failed",
|
||||
zap.Error(err),
|
||||
)
|
||||
return getErrResponse(err, method, request.GetDbName(), request.GetCollectionName()), nil
|
||||
}
|
||||
|
||||
if err := ValidateResourceGroupName(request.GetTargetResourceGroup()); err != nil {
|
||||
log.Warn("TransferReplica failed",
|
||||
log.Ctx(ctx).Warn("TransferReplica failed",
|
||||
zap.Error(err),
|
||||
)
|
||||
return getErrResponse(err, method, request.GetDbName(), request.GetCollectionName()), nil
|
||||
|
@ -6448,6 +6445,9 @@ func (node *Proxy) AllocTimestamp(ctx context.Context, req *milvuspb.AllocTimest
|
|||
return &milvuspb.AllocTimestampResponse{Status: merr.Status(err)}, nil
|
||||
}
|
||||
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.String("role", typeutil.ProxyRole),
|
||||
)
|
||||
log.Info("AllocTimestamp request receive")
|
||||
ts, err := node.tsoAllocator.AllocOne(ctx)
|
||||
if err != nil {
|
||||
|
@ -6477,10 +6477,10 @@ func (node *Proxy) ImportV2(ctx context.Context, req *internalpb.ImportRequest)
|
|||
return &internalpb.ImportResponse{Status: merr.Status(err)}, nil
|
||||
}
|
||||
log := log.Ctx(ctx).With(
|
||||
zap.String("role", typeutil.ProxyRole),
|
||||
zap.String("collectionName", req.GetCollectionName()),
|
||||
zap.String("partition name", req.GetPartitionName()),
|
||||
zap.Any("files", req.GetFiles()),
|
||||
zap.String("role", typeutil.ProxyRole),
|
||||
zap.Any("options", req.GetOptions()),
|
||||
)
|
||||
|
||||
|
|
|
@ -625,7 +625,7 @@ func (m *MetaCache) GetCollectionSchema(ctx context.Context, database, collectio
|
|||
return nil, err
|
||||
}
|
||||
metrics.ProxyUpdateCacheLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), method).Observe(float64(tr.ElapseSpan().Milliseconds()))
|
||||
log.Debug("Reload collection from root coordinator ",
|
||||
log.Ctx(ctx).Debug("Reload collection from root coordinator ",
|
||||
zap.String("collectionName", collectionName),
|
||||
zap.Int64("time (milliseconds) take ", tr.ElapseSpan().Milliseconds()))
|
||||
return collInfo.schema, nil
|
||||
|
@ -844,7 +844,7 @@ func (m *MetaCache) RemoveCollection(ctx context.Context, database, collectionNa
|
|||
if dbOk {
|
||||
delete(m.collInfo[database], collectionName)
|
||||
}
|
||||
log.Debug("remove collection", zap.String("db", database), zap.String("collection", collectionName))
|
||||
log.Ctx(ctx).Debug("remove collection", zap.String("db", database), zap.String("collection", collectionName))
|
||||
}
|
||||
|
||||
func (m *MetaCache) RemoveCollectionsByID(ctx context.Context, collectionID UniqueID, version uint64, removeVersion bool) []string {
|
||||
|
@ -868,7 +868,7 @@ func (m *MetaCache) RemoveCollectionsByID(ctx context.Context, collectionID Uniq
|
|||
} else if version != 0 {
|
||||
m.collectionCacheVersion[collectionID] = version
|
||||
}
|
||||
log.Debug("remove collection by id", zap.Int64("id", collectionID),
|
||||
log.Ctx(ctx).Debug("remove collection by id", zap.Int64("id", collectionID),
|
||||
zap.Strings("collection", collNames), zap.Uint64("currentVersion", curVersion),
|
||||
zap.Uint64("version", version), zap.Bool("removeVersion", removeVersion))
|
||||
return collNames
|
||||
|
@ -1191,7 +1191,7 @@ func (m *MetaCache) RefreshPolicyInfo(op typeutil.CacheOp) (err error) {
|
|||
}
|
||||
|
||||
func (m *MetaCache) RemoveDatabase(ctx context.Context, database string) {
|
||||
log.Debug("remove database", zap.String("name", database))
|
||||
log.Ctx(ctx).Debug("remove database", zap.String("name", database))
|
||||
m.mu.Lock()
|
||||
delete(m.collInfo, database)
|
||||
delete(m.dbInfo, database)
|
||||
|
|
|
@ -129,7 +129,7 @@ func repackInsertDataByPartition(ctx context.Context,
|
|||
assignedSegmentInfos, err := segIDAssigner.GetSegmentID(insertMsg.CollectionID, partitionID, channelName, uint32(len(rowOffsets)), maxTs)
|
||||
metrics.ProxyAssignSegmentIDLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(float64(time.Since(beforeAssign).Milliseconds()))
|
||||
if err != nil {
|
||||
log.Error("allocate segmentID for insert data failed",
|
||||
log.Ctx(ctx).Error("allocate segmentID for insert data failed",
|
||||
zap.String("collectionName", insertMsg.CollectionName),
|
||||
zap.String("channelName", channelName),
|
||||
zap.Int("allocate count", len(rowOffsets)),
|
||||
|
@ -142,7 +142,7 @@ func repackInsertDataByPartition(ctx context.Context,
|
|||
subRowOffsets := rowOffsets[startPos : startPos+int(count)]
|
||||
msgs, err := genInsertMsgsByPartition(ctx, segmentID, partitionID, partitionName, subRowOffsets, channelName, insertMsg)
|
||||
if err != nil {
|
||||
log.Warn("repack insert data to insert msgs failed",
|
||||
log.Ctx(ctx).Warn("repack insert data to insert msgs failed",
|
||||
zap.String("collectionName", insertMsg.CollectionName),
|
||||
zap.Int64("partitionID", partitionID),
|
||||
zap.Error(err))
|
||||
|
@ -167,7 +167,7 @@ func setMsgID(ctx context.Context,
|
|||
return err
|
||||
})
|
||||
if err != nil {
|
||||
log.Error("failed to allocate msg id", zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to allocate msg id", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -195,7 +195,7 @@ func repackInsertData(ctx context.Context,
|
|||
partitionName := insertMsg.PartitionName
|
||||
msgs, err := repackInsertDataByPartition(ctx, partitionName, rowOffsets, channel, insertMsg, segIDAssigner)
|
||||
if err != nil {
|
||||
log.Warn("repack insert data to msg pack failed",
|
||||
log.Ctx(ctx).Warn("repack insert data to msg pack failed",
|
||||
zap.String("collectionName", insertMsg.CollectionName),
|
||||
zap.String("partition name", partitionName),
|
||||
zap.Error(err))
|
||||
|
@ -207,7 +207,7 @@ func repackInsertData(ctx context.Context,
|
|||
|
||||
err := setMsgID(ctx, msgPack.Msgs, idAllocator)
|
||||
if err != nil {
|
||||
log.Error("failed to set msgID when repack insert data",
|
||||
log.Ctx(ctx).Error("failed to set msgID when repack insert data",
|
||||
zap.String("collectionName", insertMsg.CollectionName),
|
||||
zap.String("partition name", insertMsg.PartitionName),
|
||||
zap.Error(err))
|
||||
|
@ -233,14 +233,14 @@ func repackInsertDataWithPartitionKey(ctx context.Context,
|
|||
channel2RowOffsets := assignChannelsByPK(result.IDs, channelNames, insertMsg)
|
||||
partitionNames, err := getDefaultPartitionsInPartitionKeyMode(ctx, insertMsg.GetDbName(), insertMsg.CollectionName)
|
||||
if err != nil {
|
||||
log.Warn("get default partition names failed in partition key mode",
|
||||
log.Ctx(ctx).Warn("get default partition names failed in partition key mode",
|
||||
zap.String("collectionName", insertMsg.CollectionName),
|
||||
zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
hashValues, err := typeutil.HashKey2Partitions(partitionKeys, partitionNames)
|
||||
if err != nil {
|
||||
log.Warn("has partition keys to partitions failed",
|
||||
log.Ctx(ctx).Warn("has partition keys to partitions failed",
|
||||
zap.String("collectionName", insertMsg.CollectionName),
|
||||
zap.Error(err))
|
||||
return nil, err
|
||||
|
@ -274,7 +274,7 @@ func repackInsertDataWithPartitionKey(ctx context.Context,
|
|||
|
||||
err = errGroup.Wait()
|
||||
if err != nil {
|
||||
log.Warn("repack insert data into insert msg pack failed",
|
||||
log.Ctx(ctx).Warn("repack insert data into insert msg pack failed",
|
||||
zap.String("collectionName", insertMsg.CollectionName),
|
||||
zap.String("channelName", channel),
|
||||
zap.Error(err))
|
||||
|
@ -289,7 +289,7 @@ func repackInsertDataWithPartitionKey(ctx context.Context,
|
|||
|
||||
err = setMsgID(ctx, msgPack.Msgs, idAllocator)
|
||||
if err != nil {
|
||||
log.Error("failed to set msgID when repack insert data",
|
||||
log.Ctx(ctx).Error("failed to set msgID when repack insert data",
|
||||
zap.String("collectionName", insertMsg.CollectionName),
|
||||
zap.Error(err))
|
||||
return nil, err
|
||||
|
|
|
@ -216,6 +216,7 @@ func (node *Proxy) initRateCollector() error {
|
|||
|
||||
// Init initialize proxy.
|
||||
func (node *Proxy) Init() error {
|
||||
log := log.Ctx(node.ctx)
|
||||
log.Info("init session for Proxy")
|
||||
if err := node.initSession(); err != nil {
|
||||
log.Warn("failed to init Proxy's session", zap.Error(err))
|
||||
|
@ -309,6 +310,7 @@ func (node *Proxy) Init() error {
|
|||
|
||||
// sendChannelsTimeTickLoop starts a goroutine that synchronizes the time tick information.
|
||||
func (node *Proxy) sendChannelsTimeTickLoop() {
|
||||
log := log.Ctx(node.ctx)
|
||||
node.wg.Add(1)
|
||||
go func() {
|
||||
defer node.wg.Done()
|
||||
|
@ -392,6 +394,7 @@ func (node *Proxy) sendChannelsTimeTickLoop() {
|
|||
|
||||
// Start starts a proxy node.
|
||||
func (node *Proxy) Start() error {
|
||||
log := log.Ctx(node.ctx)
|
||||
if err := node.sched.Start(); err != nil {
|
||||
log.Warn("failed to start task scheduler", zap.String("role", typeutil.ProxyRole), zap.Error(err))
|
||||
return err
|
||||
|
@ -441,6 +444,7 @@ func (node *Proxy) Start() error {
|
|||
|
||||
// Stop stops a proxy node.
|
||||
func (node *Proxy) Stop() error {
|
||||
log := log.Ctx(node.ctx)
|
||||
if node.rowIDAllocator != nil {
|
||||
node.rowIDAllocator.Close()
|
||||
log.Info("close id allocator", zap.String("role", typeutil.ProxyRole))
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package proxy
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"math"
|
||||
"reflect"
|
||||
|
@ -124,11 +125,12 @@ func (ws *weightedScorer) scorerType() rankType {
|
|||
return weightedRankType
|
||||
}
|
||||
|
||||
func NewReScorers(reqCnt int, rankParams []*commonpb.KeyValuePair) ([]reScorer, error) {
|
||||
func NewReScorers(ctx context.Context, reqCnt int, rankParams []*commonpb.KeyValuePair) ([]reScorer, error) {
|
||||
if reqCnt == 0 {
|
||||
return []reScorer{}, nil
|
||||
}
|
||||
|
||||
log := log.Ctx(ctx)
|
||||
res := make([]reScorer, reqCnt)
|
||||
rankTypeStr, err := funcutil.GetAttrByKeyFromRepeatedKV(RankTypeKey, rankParams)
|
||||
if err != nil {
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package proxy
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/stretchr/testify/assert"
|
||||
|
@ -11,7 +12,7 @@ import (
|
|||
|
||||
func TestRescorer(t *testing.T) {
|
||||
t.Run("default scorer", func(t *testing.T) {
|
||||
rescorers, err := NewReScorers(2, nil)
|
||||
rescorers, err := NewReScorers(context.TODO(), 2, nil)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 2, len(rescorers))
|
||||
assert.Equal(t, rrfRankType, rescorers[0].scorerType())
|
||||
|
@ -26,7 +27,7 @@ func TestRescorer(t *testing.T) {
|
|||
{Key: RankParamsKey, Value: string(b)},
|
||||
}
|
||||
|
||||
_, err = NewReScorers(2, rankParams)
|
||||
_, err = NewReScorers(context.TODO(), 2, rankParams)
|
||||
assert.Error(t, err)
|
||||
assert.Contains(t, err.Error(), "k not found in rank_params")
|
||||
})
|
||||
|
@ -41,7 +42,7 @@ func TestRescorer(t *testing.T) {
|
|||
{Key: RankParamsKey, Value: string(b)},
|
||||
}
|
||||
|
||||
_, err = NewReScorers(2, rankParams)
|
||||
_, err = NewReScorers(context.TODO(), 2, rankParams)
|
||||
assert.Error(t, err)
|
||||
|
||||
params[RRFParamsKey] = maxRRFParamsValue + 1
|
||||
|
@ -52,7 +53,7 @@ func TestRescorer(t *testing.T) {
|
|||
{Key: RankParamsKey, Value: string(b)},
|
||||
}
|
||||
|
||||
_, err = NewReScorers(2, rankParams)
|
||||
_, err = NewReScorers(context.TODO(), 2, rankParams)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
|
@ -66,7 +67,7 @@ func TestRescorer(t *testing.T) {
|
|||
{Key: RankParamsKey, Value: string(b)},
|
||||
}
|
||||
|
||||
rescorers, err := NewReScorers(2, rankParams)
|
||||
rescorers, err := NewReScorers(context.TODO(), 2, rankParams)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 2, len(rescorers))
|
||||
assert.Equal(t, rrfRankType, rescorers[0].scorerType())
|
||||
|
@ -82,7 +83,7 @@ func TestRescorer(t *testing.T) {
|
|||
{Key: RankParamsKey, Value: string(b)},
|
||||
}
|
||||
|
||||
_, err = NewReScorers(2, rankParams)
|
||||
_, err = NewReScorers(context.TODO(), 2, rankParams)
|
||||
assert.Error(t, err)
|
||||
assert.Contains(t, err.Error(), "not found in rank_params")
|
||||
})
|
||||
|
@ -98,7 +99,7 @@ func TestRescorer(t *testing.T) {
|
|||
{Key: RankParamsKey, Value: string(b)},
|
||||
}
|
||||
|
||||
_, err = NewReScorers(2, rankParams)
|
||||
_, err = NewReScorers(context.TODO(), 2, rankParams)
|
||||
assert.Error(t, err)
|
||||
assert.Contains(t, err.Error(), "rank param weight should be in range [0, 1]")
|
||||
})
|
||||
|
@ -114,7 +115,7 @@ func TestRescorer(t *testing.T) {
|
|||
{Key: RankParamsKey, Value: string(b)},
|
||||
}
|
||||
|
||||
rescorers, err := NewReScorers(2, rankParams)
|
||||
rescorers, err := NewReScorers(context.TODO(), 2, rankParams)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 2, len(rescorers))
|
||||
assert.Equal(t, weightedRankType, rescorers[0].scorerType())
|
||||
|
|
|
@ -237,7 +237,7 @@ func reduceSearchResultDataWithGroupBy(ctx context.Context, subSearchResultData
|
|||
)
|
||||
|
||||
for j = 0; j < groupBound; {
|
||||
subSearchIdx, resultDataIdx := selectHighestScoreIndex(subSearchResultData, subSearchNqOffset, cursors, i)
|
||||
subSearchIdx, resultDataIdx := selectHighestScoreIndex(ctx, subSearchResultData, subSearchNqOffset, cursors, i)
|
||||
if subSearchIdx == -1 {
|
||||
break
|
||||
}
|
||||
|
@ -382,7 +382,7 @@ func reduceSearchResultDataNoGroupBy(ctx context.Context, subSearchResultData []
|
|||
|
||||
// skip offset results
|
||||
for k := int64(0); k < offset; k++ {
|
||||
subSearchIdx, _ := selectHighestScoreIndex(subSearchResultData, subSearchNqOffset, cursors, i)
|
||||
subSearchIdx, _ := selectHighestScoreIndex(ctx, subSearchResultData, subSearchNqOffset, cursors, i)
|
||||
if subSearchIdx == -1 {
|
||||
break
|
||||
}
|
||||
|
@ -395,7 +395,7 @@ func reduceSearchResultDataNoGroupBy(ctx context.Context, subSearchResultData []
|
|||
// From all the sub-query result sets of the i-th query vector,
|
||||
// find the sub-query result set index of the score j-th data,
|
||||
// and the index of the data in schemapb.SearchResultData
|
||||
subSearchIdx, resultDataIdx := selectHighestScoreIndex(subSearchResultData, subSearchNqOffset, cursors, i)
|
||||
subSearchIdx, resultDataIdx := selectHighestScoreIndex(ctx, subSearchResultData, subSearchNqOffset, cursors, i)
|
||||
if subSearchIdx == -1 {
|
||||
break
|
||||
}
|
||||
|
|
|
@ -82,7 +82,7 @@ func (info *segInfo) Capacity(ts Timestamp) uint32 {
|
|||
|
||||
func (info *segInfo) Assign(ts Timestamp, count uint32) uint32 {
|
||||
if info.IsExpired(ts) {
|
||||
log.Debug("segInfo Assign IsExpired", zap.Uint64("ts", ts),
|
||||
log.Ctx(context.TODO()).Debug("segInfo Assign IsExpired", zap.Uint64("ts", ts),
|
||||
zap.Uint32("count", count))
|
||||
return 0
|
||||
}
|
||||
|
@ -228,7 +228,7 @@ func (sa *segIDAssigner) pickCanDoFunc() {
|
|||
sa.CanDoReqs = append(sa.CanDoReqs, req)
|
||||
}
|
||||
}
|
||||
log.Debug("Proxy segIDAssigner pickCanDoFunc", zap.Any("records", records),
|
||||
log.Ctx(context.TODO()).Debug("Proxy segIDAssigner pickCanDoFunc", zap.Any("records", records),
|
||||
zap.Int("len(newTodoReqs)", len(newTodoReqs)),
|
||||
zap.Int("len(CanDoReqs)", len(sa.CanDoReqs)))
|
||||
sa.ToDoReqs = newTodoReqs
|
||||
|
@ -268,7 +268,7 @@ func (sa *segIDAssigner) checkSegReqEqual(req1, req2 *datapb.SegmentIDRequest) b
|
|||
}
|
||||
|
||||
func (sa *segIDAssigner) reduceSegReqs() {
|
||||
log.Debug("Proxy segIDAssigner reduceSegReqs", zap.Int("len(segReqs)", len(sa.segReqs)))
|
||||
log.Ctx(context.TODO()).Debug("Proxy segIDAssigner reduceSegReqs", zap.Int("len(segReqs)", len(sa.segReqs)))
|
||||
if len(sa.segReqs) == 0 {
|
||||
return
|
||||
}
|
||||
|
@ -276,7 +276,7 @@ func (sa *segIDAssigner) reduceSegReqs() {
|
|||
var newSegReqs []*datapb.SegmentIDRequest
|
||||
for _, req1 := range sa.segReqs {
|
||||
if req1.Count == 0 {
|
||||
log.Debug("Proxy segIDAssigner reduceSegReqs hit perCount == 0")
|
||||
log.Ctx(context.TODO()).Debug("Proxy segIDAssigner reduceSegReqs hit perCount == 0")
|
||||
req1.Count = sa.countPerRPC
|
||||
}
|
||||
beforeCnt += req1.Count
|
||||
|
@ -298,7 +298,7 @@ func (sa *segIDAssigner) reduceSegReqs() {
|
|||
afterCnt += req.Count
|
||||
}
|
||||
sa.segReqs = newSegReqs
|
||||
log.Debug("Proxy segIDAssigner reduceSegReqs after reduce", zap.Int("len(segReqs)", len(sa.segReqs)),
|
||||
log.Ctx(context.TODO()).Debug("Proxy segIDAssigner reduceSegReqs after reduce", zap.Int("len(segReqs)", len(sa.segReqs)),
|
||||
zap.Uint32("BeforeCnt", beforeCnt),
|
||||
zap.Uint32("AfterCnt", afterCnt))
|
||||
}
|
||||
|
@ -317,7 +317,7 @@ func (sa *segIDAssigner) syncSegments() (bool, error) {
|
|||
strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(float64(len(sa.segReqs)))
|
||||
sa.segReqs = nil
|
||||
|
||||
log.Debug("syncSegments call dataCoord.AssignSegmentID", zap.Stringer("request", req))
|
||||
log.Ctx(context.TODO()).Debug("syncSegments call dataCoord.AssignSegmentID", zap.Stringer("request", req))
|
||||
|
||||
resp, err := sa.dataCoord.AssignSegmentID(context.Background(), req)
|
||||
if err != nil {
|
||||
|
@ -333,7 +333,7 @@ func (sa *segIDAssigner) syncSegments() (bool, error) {
|
|||
success := true
|
||||
for _, segAssign := range resp.SegIDAssignments {
|
||||
if segAssign.Status.GetErrorCode() != commonpb.ErrorCode_Success {
|
||||
log.Warn("proxy", zap.String("SyncSegment Error", segAssign.GetStatus().GetReason()))
|
||||
log.Ctx(context.TODO()).Warn("proxy", zap.String("SyncSegment Error", segAssign.GetStatus().GetReason()))
|
||||
errMsg += segAssign.GetStatus().GetReason()
|
||||
errMsg += "\n"
|
||||
success = false
|
||||
|
|
|
@ -232,7 +232,7 @@ func initLimiter(source string, rln *rlinternal.RateLimiterNode, rateLimiterConf
|
|||
updated = true
|
||||
}
|
||||
if updated {
|
||||
log.Debug("RateLimiter register for rateType",
|
||||
log.Ctx(context.TODO()).Debug("RateLimiter register for rateType",
|
||||
zap.String("source", source),
|
||||
zap.String("rateType", internalpb.RateType_name[(int32(rt))]),
|
||||
zap.String("rateLimit", newLimit.String()),
|
||||
|
|
|
@ -210,7 +210,7 @@ func (t *createCollectionTask) OnEnqueue() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (t *createCollectionTask) validatePartitionKey() error {
|
||||
func (t *createCollectionTask) validatePartitionKey(ctx context.Context) error {
|
||||
idx := -1
|
||||
for i, field := range t.schema.Fields {
|
||||
if field.GetIsPartitionKey() {
|
||||
|
@ -265,7 +265,7 @@ func (t *createCollectionTask) validatePartitionKey() error {
|
|||
return fmt.Errorf("num_partitions should only be specified with partition key field enabled")
|
||||
}
|
||||
} else {
|
||||
log.Info("create collection with partition key mode",
|
||||
log.Ctx(ctx).Info("create collection with partition key mode",
|
||||
zap.String("collectionName", t.CollectionName),
|
||||
zap.Int64("numDefaultPartitions", t.GetNumPartitions()))
|
||||
}
|
||||
|
@ -273,7 +273,7 @@ func (t *createCollectionTask) validatePartitionKey() error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (t *createCollectionTask) validateClusteringKey() error {
|
||||
func (t *createCollectionTask) validateClusteringKey(ctx context.Context) error {
|
||||
idx := -1
|
||||
for i, field := range t.schema.Fields {
|
||||
if field.GetIsClusteringKey() {
|
||||
|
@ -290,7 +290,7 @@ func (t *createCollectionTask) validateClusteringKey() error {
|
|||
}
|
||||
|
||||
if idx != -1 {
|
||||
log.Info("create collection with clustering key",
|
||||
log.Ctx(ctx).Info("create collection with clustering key",
|
||||
zap.String("collectionName", t.CollectionName),
|
||||
zap.String("clusteringKeyField", t.schema.Fields[idx].Name))
|
||||
}
|
||||
|
@ -360,17 +360,17 @@ func (t *createCollectionTask) PreExecute(ctx context.Context) error {
|
|||
}
|
||||
|
||||
// validate partition key mode
|
||||
if err := t.validatePartitionKey(); err != nil {
|
||||
if err := t.validatePartitionKey(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
hasPartitionKey := hasParitionKeyModeField(t.schema)
|
||||
if _, err := validatePartitionKeyIsolation(t.CollectionName, hasPartitionKey, t.GetProperties()...); err != nil {
|
||||
if _, err := validatePartitionKeyIsolation(ctx, t.CollectionName, hasPartitionKey, t.GetProperties()...); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// validate clustering key
|
||||
if err := t.validateClusteringKey(); err != nil {
|
||||
if err := t.validateClusteringKey(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -810,7 +810,7 @@ func (t *showCollectionsTask) Execute(ctx context.Context) error {
|
|||
for _, collectionName := range t.CollectionNames {
|
||||
collectionID, err := globalMetaCache.GetCollectionID(ctx, t.GetDbName(), collectionName)
|
||||
if err != nil {
|
||||
log.Debug("Failed to get collection id.", zap.String("collectionName", collectionName),
|
||||
log.Ctx(ctx).Debug("Failed to get collection id.", zap.String("collectionName", collectionName),
|
||||
zap.Int64("requestID", t.Base.MsgID), zap.String("requestType", "showCollections"))
|
||||
return err
|
||||
}
|
||||
|
@ -856,14 +856,14 @@ func (t *showCollectionsTask) Execute(ctx context.Context) error {
|
|||
for offset, id := range resp.CollectionIDs {
|
||||
collectionName, ok := IDs2Names[id]
|
||||
if !ok {
|
||||
log.Debug("Failed to get collection info. This collection may be not released",
|
||||
log.Ctx(ctx).Debug("Failed to get collection info. This collection may be not released",
|
||||
zap.Int64("collectionID", id),
|
||||
zap.Int64("requestID", t.Base.MsgID), zap.String("requestType", "showCollections"))
|
||||
continue
|
||||
}
|
||||
collectionInfo, err := globalMetaCache.GetCollectionInfo(ctx, t.GetDbName(), collectionName, id)
|
||||
if err != nil {
|
||||
log.Debug("Failed to get collection info.", zap.String("collectionName", collectionName),
|
||||
log.Ctx(ctx).Debug("Failed to get collection info.", zap.String("collectionName", collectionName),
|
||||
zap.Int64("requestID", t.Base.MsgID), zap.String("requestType", "showCollections"))
|
||||
return err
|
||||
}
|
||||
|
@ -964,7 +964,7 @@ func hasPropInDeletekeys(keys []string) string {
|
|||
return ""
|
||||
}
|
||||
|
||||
func validatePartitionKeyIsolation(colName string, isPartitionKeyEnabled bool, props ...*commonpb.KeyValuePair) (bool, error) {
|
||||
func validatePartitionKeyIsolation(ctx context.Context, colName string, isPartitionKeyEnabled bool, props ...*commonpb.KeyValuePair) (bool, error) {
|
||||
iso, err := common.IsPartitionKeyIsolationKvEnabled(props...)
|
||||
if err != nil {
|
||||
return false, err
|
||||
|
@ -985,7 +985,7 @@ func validatePartitionKeyIsolation(colName string, isPartitionKeyEnabled bool, p
|
|||
"partition key isolation mode is enabled but current Milvus does not support it. Please contact us")
|
||||
}
|
||||
|
||||
log.Info("validated with partition key isolation", zap.String("collectionName", colName))
|
||||
log.Ctx(ctx).Info("validated with partition key isolation", zap.String("collectionName", colName))
|
||||
|
||||
return true, nil
|
||||
}
|
||||
|
@ -1030,7 +1030,7 @@ func (t *alterCollectionTask) PreExecute(ctx context.Context) error {
|
|||
return err
|
||||
}
|
||||
// check if the new partition key isolation is valid to use
|
||||
newIsoValue, err := validatePartitionKeyIsolation(t.CollectionName, isPartitionKeyMode, t.Properties...)
|
||||
newIsoValue, err := validatePartitionKeyIsolation(ctx, t.CollectionName, isPartitionKeyMode, t.Properties...)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -1040,7 +1040,7 @@ func (t *alterCollectionTask) PreExecute(ctx context.Context) error {
|
|||
}
|
||||
oldIsoValue := collBasicInfo.partitionKeyIsolation
|
||||
|
||||
log.Info("alter collection pre check with partition key isolation",
|
||||
log.Ctx(ctx).Info("alter collection pre check with partition key isolation",
|
||||
zap.String("collectionName", t.CollectionName),
|
||||
zap.Bool("isPartitionKeyMode", isPartitionKeyMode),
|
||||
zap.Bool("newIsoValue", newIsoValue),
|
||||
|
@ -1607,7 +1607,7 @@ func (t *showPartitionsTask) Execute(ctx context.Context) error {
|
|||
collectionName := t.CollectionName
|
||||
collectionID, err := globalMetaCache.GetCollectionID(ctx, t.GetDbName(), collectionName)
|
||||
if err != nil {
|
||||
log.Debug("Failed to get collection id.", zap.String("collectionName", collectionName),
|
||||
log.Ctx(ctx).Debug("Failed to get collection id.", zap.String("collectionName", collectionName),
|
||||
zap.Int64("requestID", t.Base.MsgID), zap.String("requestType", "showPartitions"))
|
||||
return err
|
||||
}
|
||||
|
@ -1620,7 +1620,7 @@ func (t *showPartitionsTask) Execute(ctx context.Context) error {
|
|||
for _, partitionName := range t.PartitionNames {
|
||||
partitionID, err := globalMetaCache.GetPartitionID(ctx, t.GetDbName(), collectionName, partitionName)
|
||||
if err != nil {
|
||||
log.Debug("Failed to get partition id.", zap.String("partitionName", partitionName),
|
||||
log.Ctx(ctx).Debug("Failed to get partition id.", zap.String("partitionName", partitionName),
|
||||
zap.Int64("requestID", t.Base.MsgID), zap.String("requestType", "showPartitions"))
|
||||
return err
|
||||
}
|
||||
|
@ -1651,13 +1651,13 @@ func (t *showPartitionsTask) Execute(ctx context.Context) error {
|
|||
for offset, id := range resp.PartitionIDs {
|
||||
partitionName, ok := IDs2Names[id]
|
||||
if !ok {
|
||||
log.Debug("Failed to get partition id.", zap.String("partitionName", partitionName),
|
||||
log.Ctx(ctx).Debug("Failed to get partition id.", zap.String("partitionName", partitionName),
|
||||
zap.Int64("requestID", t.Base.MsgID), zap.String("requestType", "showPartitions"))
|
||||
return errors.New("failed to show partitions")
|
||||
}
|
||||
partitionInfo, err := globalMetaCache.GetPartitionInfo(ctx, t.GetDbName(), collectionName, partitionName)
|
||||
if err != nil {
|
||||
log.Debug("Failed to get partition id.", zap.String("partitionName", partitionName),
|
||||
log.Ctx(ctx).Debug("Failed to get partition id.", zap.String("partitionName", partitionName),
|
||||
zap.Int64("requestID", t.Base.MsgID), zap.String("requestType", "showPartitions"))
|
||||
return err
|
||||
}
|
||||
|
@ -2051,7 +2051,7 @@ func (t *loadPartitionsTask) Execute(ctx context.Context) error {
|
|||
|
||||
if len(unindexedVecFields) != 0 {
|
||||
errMsg := fmt.Sprintf("there is no vector index on field: %v, please create index firstly", unindexedVecFields)
|
||||
log.Debug(errMsg)
|
||||
log.Ctx(ctx).Debug(errMsg)
|
||||
return errors.New(errMsg)
|
||||
}
|
||||
|
||||
|
@ -2462,7 +2462,7 @@ func (t *DescribeResourceGroupTask) Execute(ctx context.Context) error {
|
|||
for key, value := range collections {
|
||||
name, err := globalMetaCache.GetCollectionName(ctx, "", key)
|
||||
if err != nil {
|
||||
log.Warn("failed to get collection name",
|
||||
log.Ctx(ctx).Warn("failed to get collection name",
|
||||
zap.Int64("collectionID", key),
|
||||
zap.Error(err))
|
||||
|
||||
|
|
|
@ -364,12 +364,12 @@ func (t *describeDatabaseTask) Execute(ctx context.Context) error {
|
|||
}
|
||||
ret, err := t.rootCoord.DescribeDatabase(ctx, req)
|
||||
if err != nil {
|
||||
log.Warn("DescribeDatabase failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("DescribeDatabase failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
if err := merr.CheckRPCCall(ret, err); err != nil {
|
||||
log.Warn("DescribeDatabase failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("DescribeDatabase failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
|
|
@ -171,7 +171,7 @@ func (dt *deleteTask) Execute(ctx context.Context) (err error) {
|
|||
}
|
||||
}
|
||||
|
||||
log.Debug("send delete request to virtual channels",
|
||||
log.Ctx(ctx).Debug("send delete request to virtual channels",
|
||||
zap.String("collectionName", dt.req.GetCollectionName()),
|
||||
zap.Int64("collectionID", dt.collectionID),
|
||||
zap.Strings("virtual_channels", dt.vChannels),
|
||||
|
@ -397,7 +397,7 @@ func (dr *deleteRunner) Run(ctx context.Context) error {
|
|||
// need query from querynode before delete
|
||||
err = dr.complexDelete(ctx, plan)
|
||||
if err != nil {
|
||||
log.Warn("complex delete failed,but delete some data", zap.Int64("count", dr.result.DeleteCnt), zap.String("expr", dr.req.GetExpr()))
|
||||
log.Ctx(ctx).Warn("complex delete failed,but delete some data", zap.Int64("count", dr.result.DeleteCnt), zap.String("expr", dr.req.GetExpr()))
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
@ -425,7 +425,7 @@ func (dr *deleteRunner) produce(ctx context.Context, primaryKeys *schemapb.IDs)
|
|||
}
|
||||
|
||||
if err := dr.queue.Enqueue(enqueuedTask); err != nil {
|
||||
log.Error("Failed to enqueue delete task: " + err.Error())
|
||||
log.Ctx(ctx).Error("Failed to enqueue delete task: " + err.Error())
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -539,7 +539,7 @@ func (dr *deleteRunner) receiveQueryResult(ctx context.Context, client querypb.Q
|
|||
result, err := client.Recv()
|
||||
if err != nil {
|
||||
if err == io.EOF {
|
||||
log.Debug("query stream for delete finished", zap.Int64("msgID", dr.msgID))
|
||||
log.Ctx(ctx).Debug("query stream for delete finished", zap.Int64("msgID", dr.msgID))
|
||||
return nil
|
||||
}
|
||||
return err
|
||||
|
@ -547,21 +547,21 @@ func (dr *deleteRunner) receiveQueryResult(ctx context.Context, client querypb.Q
|
|||
|
||||
err = merr.Error(result.GetStatus())
|
||||
if err != nil {
|
||||
log.Warn("query stream for delete get error status", zap.Int64("msgID", dr.msgID), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("query stream for delete get error status", zap.Int64("msgID", dr.msgID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
if dr.limiter != nil {
|
||||
err := dr.limiter.Alloc(ctx, dr.dbID, map[int64][]int64{dr.collectionID: partitionIDs}, internalpb.RateType_DMLDelete, proto.Size(result.GetIds()))
|
||||
if err != nil {
|
||||
log.Warn("query stream for delete failed because rate limiter", zap.Int64("msgID", dr.msgID), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("query stream for delete failed because rate limiter", zap.Int64("msgID", dr.msgID), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
||||
task, err := dr.produce(ctx, result.GetIds())
|
||||
if err != nil {
|
||||
log.Warn("produce delete task failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("produce delete task failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
task.allQueryCnt = result.GetAllRetrieveCount()
|
||||
|
@ -594,26 +594,26 @@ func (dr *deleteRunner) complexDelete(ctx context.Context, plan *planpb.PlanNode
|
|||
dr.result.DeleteCnt = dr.count.Load()
|
||||
dr.result.Timestamp = dr.sessionTS.Load()
|
||||
if err != nil {
|
||||
log.Warn("fail to execute complex delete",
|
||||
log.Ctx(ctx).Warn("fail to execute complex delete",
|
||||
zap.Int64("deleteCnt", dr.result.GetDeleteCnt()),
|
||||
zap.Duration("interval", rc.ElapseSpan()),
|
||||
zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
log.Info("complex delete finished", zap.Int64("deleteCnt", dr.result.GetDeleteCnt()), zap.Duration("interval", rc.ElapseSpan()))
|
||||
log.Ctx(ctx).Info("complex delete finished", zap.Int64("deleteCnt", dr.result.GetDeleteCnt()), zap.Duration("interval", rc.ElapseSpan()))
|
||||
return nil
|
||||
}
|
||||
|
||||
func (dr *deleteRunner) simpleDelete(ctx context.Context, pk *schemapb.IDs, numRow int64) error {
|
||||
log.Debug("get primary keys from expr",
|
||||
log.Ctx(ctx).Debug("get primary keys from expr",
|
||||
zap.Int64("len of primary keys", numRow),
|
||||
zap.Int64("collectionID", dr.collectionID),
|
||||
zap.Int64("partitionID", dr.partitionID))
|
||||
|
||||
task, err := dr.produce(ctx, pk)
|
||||
if err != nil {
|
||||
log.Warn("produce delete task failed")
|
||||
log.Ctx(ctx).Warn("produce delete task failed")
|
||||
return err
|
||||
}
|
||||
|
||||
|
|
|
@ -60,7 +60,7 @@ func (dt *deleteTaskByStreamingService) Execute(ctx context.Context) (err error)
|
|||
}
|
||||
}
|
||||
|
||||
log.Debug("send delete request to virtual channels",
|
||||
log.Ctx(ctx).Debug("send delete request to virtual channels",
|
||||
zap.String("collectionName", dt.req.GetCollectionName()),
|
||||
zap.Int64("collectionID", dt.collectionID),
|
||||
zap.Strings("virtual_channels", dt.vChannels),
|
||||
|
@ -69,7 +69,7 @@ func (dt *deleteTaskByStreamingService) Execute(ctx context.Context) (err error)
|
|||
|
||||
resp := streaming.WAL().AppendMessages(ctx, msgs...)
|
||||
if resp.UnwrapFirstError(); err != nil {
|
||||
log.Warn("append messages to wal failed", zap.Error(err))
|
||||
log.Ctx(ctx).Warn("append messages to wal failed", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
dt.sessionTS = resp.MaxTimeTick()
|
||||
|
|
|
@ -20,7 +20,6 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
|
||||
"github.com/pingcap/log"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
|
@ -29,6 +28,7 @@ import (
|
|||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/distributed/streaming"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/pkg/log"
|
||||
"github.com/milvus-io/milvus/pkg/streaming/util/message"
|
||||
"github.com/milvus-io/milvus/pkg/util/commonpbutil"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
|
@ -48,7 +48,7 @@ func (t *flushTaskByStreamingService) Execute(ctx context.Context) error {
|
|||
channelCps := make(map[string]*msgpb.MsgPosition)
|
||||
|
||||
flushTs := t.BeginTs()
|
||||
log.Info("flushTaskByStreamingService.Execute", zap.Int("collectionNum", len(t.CollectionNames)), zap.Uint64("flushTs", flushTs))
|
||||
log.Ctx(ctx).Info("flushTaskByStreamingService.Execute", zap.Int("collectionNum", len(t.CollectionNames)), zap.Uint64("flushTs", flushTs))
|
||||
timeOfSeal, _ := tsoutil.ParseTS(flushTs)
|
||||
for _, collName := range t.CollectionNames {
|
||||
collID, err := globalMetaCache.GetCollectionID(t.ctx, t.DbName, collName)
|
||||
|
@ -115,7 +115,7 @@ func (t *flushTaskByStreamingService) Execute(ctx context.Context) error {
|
|||
|
||||
// sendManualFlushToWAL sends a manual flush message to WAL.
|
||||
func (t *flushTaskByStreamingService) sendManualFlushToWAL(ctx context.Context, collID int64, vchannel string, flushTs uint64) ([]int64, error) {
|
||||
logger := log.With(zap.Int64("collectionID", collID), zap.String("vchannel", vchannel))
|
||||
logger := log.Ctx(ctx).With(zap.Int64("collectionID", collID), zap.String("vchannel", vchannel))
|
||||
flushMsg, err := message.NewManualFlushMessageBuilderV2().
|
||||
WithVChannel(vchannel).
|
||||
WithHeader(&message.ManualFlushMessageHeader{
|
||||
|
|
|
@ -167,7 +167,7 @@ func (cit *createIndexTask) parseFunctionParamsToIndex(indexParamsMap map[string
|
|||
return nil
|
||||
}
|
||||
|
||||
func (cit *createIndexTask) parseIndexParams() error {
|
||||
func (cit *createIndexTask) parseIndexParams(ctx context.Context) error {
|
||||
cit.newExtraParams = cit.req.GetExtraParams()
|
||||
|
||||
isVecIndex := typeutil.IsVectorType(cit.fieldSchema.DataType)
|
||||
|
@ -194,13 +194,13 @@ func (cit *createIndexTask) parseIndexParams() error {
|
|||
specifyIndexType, exist := indexParamsMap[common.IndexTypeKey]
|
||||
if exist && specifyIndexType != "" {
|
||||
if err := indexparamcheck.ValidateMmapIndexParams(specifyIndexType, indexParamsMap); err != nil {
|
||||
log.Ctx(cit.ctx).Warn("Invalid mmap type params", zap.String(common.IndexTypeKey, specifyIndexType), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("Invalid mmap type params", zap.String(common.IndexTypeKey, specifyIndexType), zap.Error(err))
|
||||
return merr.WrapErrParameterInvalidMsg("invalid mmap type params", err.Error())
|
||||
}
|
||||
checker, err := indexparamcheck.GetIndexCheckerMgrInstance().GetChecker(specifyIndexType)
|
||||
// not enable hybrid index for user, used in milvus internally
|
||||
if err != nil || indexparamcheck.IsHYBRIDChecker(checker) {
|
||||
log.Ctx(cit.ctx).Warn("Failed to get index checker", zap.String(common.IndexTypeKey, specifyIndexType))
|
||||
log.Ctx(ctx).Warn("Failed to get index checker", zap.String(common.IndexTypeKey, specifyIndexType))
|
||||
return merr.WrapErrParameterInvalid("valid index", fmt.Sprintf("invalid index type: %s", specifyIndexType))
|
||||
}
|
||||
}
|
||||
|
@ -240,7 +240,7 @@ func (cit *createIndexTask) parseIndexParams() error {
|
|||
} else {
|
||||
specifyIndexType, exist := indexParamsMap[common.IndexTypeKey]
|
||||
if Params.AutoIndexConfig.Enable.GetAsBool() { // `enable` only for cloud instance.
|
||||
log.Info("create index trigger AutoIndex",
|
||||
log.Ctx(ctx).Info("create index trigger AutoIndex",
|
||||
zap.String("original type", specifyIndexType),
|
||||
zap.String("final type", Params.AutoIndexConfig.AutoIndexTypeName.GetValue()))
|
||||
|
||||
|
@ -275,7 +275,7 @@ func (cit *createIndexTask) parseIndexParams() error {
|
|||
indexParamsMap[k] = v
|
||||
fields = append(fields, zap.String(k, v))
|
||||
}
|
||||
log.Ctx(cit.ctx).Info("AutoIndex triggered", fields...)
|
||||
log.Ctx(ctx).Info("AutoIndex triggered", fields...)
|
||||
}
|
||||
|
||||
handle := func(numberParams int, autoIndexConfig map[string]string) error {
|
||||
|
@ -378,7 +378,7 @@ func (cit *createIndexTask) parseIndexParams() error {
|
|||
}
|
||||
}
|
||||
|
||||
err := checkTrain(cit.fieldSchema, indexParamsMap)
|
||||
err := checkTrain(ctx, cit.fieldSchema, indexParamsMap)
|
||||
if err != nil {
|
||||
return merr.WrapErrParameterInvalid("valid index params", "invalid index params", err.Error())
|
||||
}
|
||||
|
@ -411,20 +411,20 @@ func (cit *createIndexTask) parseIndexParams() error {
|
|||
func (cit *createIndexTask) getIndexedFieldAndFunction(ctx context.Context) error {
|
||||
schema, err := globalMetaCache.GetCollectionSchema(ctx, cit.req.GetDbName(), cit.req.GetCollectionName())
|
||||
if err != nil {
|
||||
log.Error("failed to get collection schema", zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to get collection schema", zap.Error(err))
|
||||
return fmt.Errorf("failed to get collection schema: %s", err)
|
||||
}
|
||||
|
||||
field, err := schema.schemaHelper.GetFieldFromName(cit.req.GetFieldName())
|
||||
if err != nil {
|
||||
log.Error("create index on non-exist field", zap.Error(err))
|
||||
log.Ctx(ctx).Error("create index on non-exist field", zap.Error(err))
|
||||
return fmt.Errorf("cannot create index on non-exist field: %s", cit.req.GetFieldName())
|
||||
}
|
||||
|
||||
if field.IsFunctionOutput {
|
||||
function, err := schema.schemaHelper.GetFunctionByOutputField(field)
|
||||
if err != nil {
|
||||
log.Error("create index failed, cannot find function of function output field", zap.Error(err))
|
||||
log.Ctx(ctx).Error("create index failed, cannot find function of function output field", zap.Error(err))
|
||||
return fmt.Errorf("create index failed, cannot find function of function output field: %s", cit.req.GetFieldName())
|
||||
}
|
||||
cit.functionSchema = function
|
||||
|
@ -455,7 +455,7 @@ func fillDimension(field *schemapb.FieldSchema, indexParams map[string]string) e
|
|||
return nil
|
||||
}
|
||||
|
||||
func checkTrain(field *schemapb.FieldSchema, indexParams map[string]string) error {
|
||||
func checkTrain(ctx context.Context, field *schemapb.FieldSchema, indexParams map[string]string) error {
|
||||
indexType := indexParams[common.IndexTypeKey]
|
||||
|
||||
if indexType == indexparamcheck.IndexHybrid {
|
||||
|
@ -466,7 +466,7 @@ func checkTrain(field *schemapb.FieldSchema, indexParams map[string]string) erro
|
|||
}
|
||||
checker, err := indexparamcheck.GetIndexCheckerMgrInstance().GetChecker(indexType)
|
||||
if err != nil {
|
||||
log.Warn("Failed to get index checker", zap.String(common.IndexTypeKey, indexType))
|
||||
log.Ctx(ctx).Warn("Failed to get index checker", zap.String(common.IndexTypeKey, indexType))
|
||||
return fmt.Errorf("invalid index type: %s", indexType)
|
||||
}
|
||||
|
||||
|
@ -486,12 +486,12 @@ func checkTrain(field *schemapb.FieldSchema, indexParams map[string]string) erro
|
|||
}
|
||||
|
||||
if err := checker.CheckValidDataType(indexType, field); err != nil {
|
||||
log.Info("create index with invalid data type", zap.Error(err), zap.String("data_type", field.GetDataType().String()))
|
||||
log.Ctx(ctx).Info("create index with invalid data type", zap.Error(err), zap.String("data_type", field.GetDataType().String()))
|
||||
return err
|
||||
}
|
||||
|
||||
if err := checker.CheckTrain(field.DataType, indexParams); err != nil {
|
||||
log.Info("create index with invalid parameters", zap.Error(err))
|
||||
log.Ctx(ctx).Info("create index with invalid parameters", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -517,7 +517,7 @@ func (cit *createIndexTask) PreExecute(ctx context.Context) error {
|
|||
}
|
||||
|
||||
// check index param, not accurate, only some static rules
|
||||
err = cit.parseIndexParams()
|
||||
err = cit.parseIndexParams(ctx)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -758,7 +758,7 @@ func (dit *describeIndexTask) PreExecute(ctx context.Context) error {
|
|||
func (dit *describeIndexTask) Execute(ctx context.Context) error {
|
||||
schema, err := globalMetaCache.GetCollectionSchema(ctx, dit.GetDbName(), dit.GetCollectionName())
|
||||
if err != nil {
|
||||
log.Error("failed to get collection schema", zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to get collection schema", zap.Error(err))
|
||||
return fmt.Errorf("failed to get collection schema: %s", err)
|
||||
}
|
||||
|
||||
|
@ -780,7 +780,7 @@ func (dit *describeIndexTask) Execute(ctx context.Context) error {
|
|||
for _, indexInfo := range resp.IndexInfos {
|
||||
field, err := schema.schemaHelper.GetFieldFromID(indexInfo.FieldID)
|
||||
if err != nil {
|
||||
log.Error("failed to get collection field", zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to get collection field", zap.Error(err))
|
||||
return fmt.Errorf("failed to get collection field: %d", indexInfo.FieldID)
|
||||
}
|
||||
params := indexInfo.GetUserIndexParams()
|
||||
|
@ -877,7 +877,7 @@ func (dit *getIndexStatisticsTask) PreExecute(ctx context.Context) error {
|
|||
func (dit *getIndexStatisticsTask) Execute(ctx context.Context) error {
|
||||
schema, err := globalMetaCache.GetCollectionSchema(ctx, dit.GetDbName(), dit.GetCollectionName())
|
||||
if err != nil {
|
||||
log.Error("failed to get collection schema", zap.String("collection_name", dit.GetCollectionName()), zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to get collection schema", zap.String("collection_name", dit.GetCollectionName()), zap.Error(err))
|
||||
return fmt.Errorf("failed to get collection schema: %s", dit.GetCollectionName())
|
||||
}
|
||||
schemaHelper := schema.schemaHelper
|
||||
|
@ -893,7 +893,7 @@ func (dit *getIndexStatisticsTask) Execute(ctx context.Context) error {
|
|||
for _, indexInfo := range resp.IndexInfos {
|
||||
field, err := schemaHelper.GetFieldFromID(indexInfo.FieldID)
|
||||
if err != nil {
|
||||
log.Error("failed to get collection field", zap.Int64("field_id", indexInfo.FieldID), zap.Error(err))
|
||||
log.Ctx(ctx).Error("failed to get collection field", zap.Int64("field_id", indexInfo.FieldID), zap.Error(err))
|
||||
return fmt.Errorf("failed to get collection field: %d", indexInfo.FieldID)
|
||||
}
|
||||
params := indexInfo.GetUserIndexParams()
|
||||
|
|
|
@ -374,7 +374,7 @@ func Test_sparse_parseIndexParams(t *testing.T) {
|
|||
}
|
||||
|
||||
t.Run("parse index params", func(t *testing.T) {
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
|
||||
assert.ElementsMatch(t,
|
||||
|
@ -452,7 +452,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
}
|
||||
|
||||
t.Run("parse index params", func(t *testing.T) {
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
|
||||
assert.ElementsMatch(t,
|
||||
|
@ -545,7 +545,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
indexParamsStr, err := json.Marshal(indexParams)
|
||||
assert.NoError(t, err)
|
||||
Params.Save(Params.AutoIndexConfig.IndexParams.Key, string(indexParamsStr))
|
||||
err = cit2.parseIndexParams()
|
||||
err = cit2.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
|
||||
assert.ElementsMatch(t,
|
||||
|
@ -622,7 +622,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
DataType: schemapb.DataType_JSON,
|
||||
},
|
||||
}
|
||||
err := cit3.parseIndexParams()
|
||||
err := cit3.parseIndexParams(context.TODO())
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
|
@ -644,7 +644,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
DataType: schemapb.DataType_VarChar,
|
||||
},
|
||||
}
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
|
@ -661,7 +661,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
DataType: schemapb.DataType_VarChar,
|
||||
},
|
||||
}
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
sortKeyValuePairs(cit.newIndexParams)
|
||||
assert.Equal(t, cit.newIndexParams, []*commonpb.KeyValuePair{
|
||||
|
@ -687,7 +687,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
}
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
|
@ -704,7 +704,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
}
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
sortKeyValuePairs(cit.newIndexParams)
|
||||
assert.Equal(t, cit.newIndexParams, []*commonpb.KeyValuePair{
|
||||
|
@ -731,7 +731,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
DataType: schemapb.DataType_VarChar,
|
||||
},
|
||||
}
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
|
@ -754,7 +754,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
}
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
})
|
||||
|
||||
|
@ -776,7 +776,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
}
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
|
@ -812,7 +812,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
ElementType: schemapb.DataType_Int64,
|
||||
},
|
||||
}
|
||||
err := cit3.parseIndexParams()
|
||||
err := cit3.parseIndexParams(context.TODO())
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
|
@ -859,7 +859,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
DataType: schemapb.DataType_VarChar,
|
||||
},
|
||||
}
|
||||
err := cit4.parseIndexParams()
|
||||
err := cit4.parseIndexParams(context.TODO())
|
||||
assert.Error(t, err)
|
||||
|
||||
cit5 := &createIndexTask{
|
||||
|
@ -904,7 +904,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
}
|
||||
err = cit5.parseIndexParams()
|
||||
err = cit5.parseIndexParams(context.TODO())
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
|
@ -932,7 +932,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
},
|
||||
}
|
||||
|
||||
err = cit.parseIndexParams()
|
||||
err = cit.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
sortKeyValuePairs(cit.newIndexParams)
|
||||
assert.Equal(t, cit.newIndexParams, []*commonpb.KeyValuePair{
|
||||
|
@ -961,7 +961,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
},
|
||||
}
|
||||
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
sortKeyValuePairs(cit.newIndexParams)
|
||||
assert.Equal(t, cit.newIndexParams, []*commonpb.KeyValuePair{
|
||||
|
@ -990,7 +990,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
},
|
||||
}
|
||||
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
sortKeyValuePairs(cit.newIndexParams)
|
||||
assert.Equal(t, cit.newIndexParams, []*commonpb.KeyValuePair{
|
||||
|
@ -1019,7 +1019,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
},
|
||||
}
|
||||
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
|
@ -1051,7 +1051,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
},
|
||||
}
|
||||
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
|
@ -1087,7 +1087,7 @@ func Test_parseIndexParams(t *testing.T) {
|
|||
},
|
||||
},
|
||||
}
|
||||
err := cit.parseIndexParams()
|
||||
err := cit.parseIndexParams(context.TODO())
|
||||
// Out of range in json: param 'M' (3000) should be in range [2, 2048]
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
@ -1142,7 +1142,7 @@ func Test_parseIndexParams_AutoIndex_WithType(t *testing.T) {
|
|||
},
|
||||
},
|
||||
}
|
||||
err := task.parseIndexParams()
|
||||
err := task.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
assert.True(t, task.userAutoIndexMetricTypeSpecified)
|
||||
assert.ElementsMatch(t, []*commonpb.KeyValuePair{
|
||||
|
@ -1162,7 +1162,7 @@ func Test_parseIndexParams_AutoIndex_WithType(t *testing.T) {
|
|||
},
|
||||
},
|
||||
}
|
||||
err := task.parseIndexParams()
|
||||
err := task.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
assert.True(t, task.userAutoIndexMetricTypeSpecified)
|
||||
assert.ElementsMatch(t, []*commonpb.KeyValuePair{
|
||||
|
@ -1181,7 +1181,7 @@ func Test_parseIndexParams_AutoIndex_WithType(t *testing.T) {
|
|||
},
|
||||
},
|
||||
}
|
||||
err := task.parseIndexParams()
|
||||
err := task.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
assert.True(t, task.userAutoIndexMetricTypeSpecified)
|
||||
assert.ElementsMatch(t, []*commonpb.KeyValuePair{
|
||||
|
@ -1232,7 +1232,7 @@ func Test_parseIndexParams_AutoIndex(t *testing.T) {
|
|||
ExtraParams: make([]*commonpb.KeyValuePair, 0),
|
||||
},
|
||||
}
|
||||
err := task.parseIndexParams()
|
||||
err := task.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
assert.False(t, task.userAutoIndexMetricTypeSpecified)
|
||||
assert.ElementsMatch(t, []*commonpb.KeyValuePair{
|
||||
|
@ -1248,7 +1248,7 @@ func Test_parseIndexParams_AutoIndex(t *testing.T) {
|
|||
ExtraParams: make([]*commonpb.KeyValuePair, 0),
|
||||
},
|
||||
}
|
||||
err := task.parseIndexParams()
|
||||
err := task.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
assert.False(t, task.userAutoIndexMetricTypeSpecified)
|
||||
assert.ElementsMatch(t, []*commonpb.KeyValuePair{
|
||||
|
@ -1264,7 +1264,7 @@ func Test_parseIndexParams_AutoIndex(t *testing.T) {
|
|||
ExtraParams: make([]*commonpb.KeyValuePair, 0),
|
||||
},
|
||||
}
|
||||
err := task.parseIndexParams()
|
||||
err := task.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
assert.False(t, task.userAutoIndexMetricTypeSpecified)
|
||||
assert.ElementsMatch(t, []*commonpb.KeyValuePair{
|
||||
|
@ -1282,7 +1282,7 @@ func Test_parseIndexParams_AutoIndex(t *testing.T) {
|
|||
},
|
||||
},
|
||||
}
|
||||
err := task.parseIndexParams()
|
||||
err := task.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
assert.True(t, task.userAutoIndexMetricTypeSpecified)
|
||||
assert.ElementsMatch(t, []*commonpb.KeyValuePair{
|
||||
|
@ -1301,7 +1301,7 @@ func Test_parseIndexParams_AutoIndex(t *testing.T) {
|
|||
},
|
||||
},
|
||||
}
|
||||
err := task.parseIndexParams()
|
||||
err := task.parseIndexParams(context.TODO())
|
||||
assert.NoError(t, err)
|
||||
assert.ElementsMatch(t, []*commonpb.KeyValuePair{
|
||||
{Key: common.IndexTypeKey, Value: AutoIndexName},
|
||||
|
@ -1318,7 +1318,7 @@ func Test_parseIndexParams_AutoIndex(t *testing.T) {
|
|||
},
|
||||
},
|
||||
}
|
||||
err := task.parseIndexParams()
|
||||
err := task.parseIndexParams(context.TODO())
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
|
@ -1332,7 +1332,7 @@ func Test_parseIndexParams_AutoIndex(t *testing.T) {
|
|||
},
|
||||
},
|
||||
}
|
||||
err := task.parseIndexParams()
|
||||
err := task.parseIndexParams(context.TODO())
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
|
@ -1347,7 +1347,7 @@ func Test_parseIndexParams_AutoIndex(t *testing.T) {
|
|||
},
|
||||
},
|
||||
}
|
||||
err := task.parseIndexParams()
|
||||
err := task.parseIndexParams(context.TODO())
|
||||
assert.Error(t, err)
|
||||
})
|
||||
}
|
||||
|
|
|
@ -114,20 +114,20 @@ func (it *insertTask) PreExecute(ctx context.Context) error {
|
|||
|
||||
collectionName := it.insertMsg.CollectionName
|
||||
if err := validateCollectionName(collectionName); err != nil {
|
||||
log.Warn("valid collection name failed", zap.String("collectionName", collectionName), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("valid collection name failed", zap.String("collectionName", collectionName), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
maxInsertSize := Params.QuotaConfig.MaxInsertSize.GetAsInt()
|
||||
if maxInsertSize != -1 && it.insertMsg.Size() > maxInsertSize {
|
||||
log.Warn("insert request size exceeds maxInsertSize",
|
||||
log.Ctx(ctx).Warn("insert request size exceeds maxInsertSize",
|
||||
zap.Int("request size", it.insertMsg.Size()), zap.Int("maxInsertSize", maxInsertSize))
|
||||
return merr.WrapErrAsInputError(merr.WrapErrParameterTooLarge("insert request size exceeds maxInsertSize"))
|
||||
}
|
||||
|
||||
schema, err := globalMetaCache.GetCollectionSchema(ctx, it.insertMsg.GetDbName(), collectionName)
|
||||
if err != nil {
|
||||
log.Warn("get collection schema from global meta cache failed", zap.String("collectionName", collectionName), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("get collection schema from global meta cache failed", zap.String("collectionName", collectionName), zap.Error(err))
|
||||
return merr.WrapErrAsInputErrorWhen(err, merr.ErrCollectionNotFound, merr.ErrDatabaseNotFound)
|
||||
}
|
||||
it.schema = schema.CollectionSchema
|
||||
|
|
|
@ -133,7 +133,7 @@ func repackInsertDataWithPartitionKeyForStreamingService(
|
|||
channel2RowOffsets := assignChannelsByPK(result.IDs, channelNames, insertMsg)
|
||||
partitionNames, err := getDefaultPartitionsInPartitionKeyMode(ctx, insertMsg.GetDbName(), insertMsg.CollectionName)
|
||||
if err != nil {
|
||||
log.Warn("get default partition names failed in partition key mode",
|
||||
log.Ctx(ctx).Warn("get default partition names failed in partition key mode",
|
||||
zap.String("collectionName", insertMsg.CollectionName),
|
||||
zap.Error(err))
|
||||
return nil, err
|
||||
|
@ -144,7 +144,7 @@ func repackInsertDataWithPartitionKeyForStreamingService(
|
|||
for _, partitionName := range partitionNames {
|
||||
partitionID, err := globalMetaCache.GetPartitionID(ctx, insertMsg.GetDbName(), insertMsg.CollectionName, partitionName)
|
||||
if err != nil {
|
||||
log.Warn("get partition id failed",
|
||||
log.Ctx(ctx).Warn("get partition id failed",
|
||||
zap.String("collectionName", insertMsg.CollectionName),
|
||||
zap.String("partitionName", partitionName),
|
||||
zap.Error(err))
|
||||
|
@ -155,7 +155,7 @@ func repackInsertDataWithPartitionKeyForStreamingService(
|
|||
|
||||
hashValues, err := typeutil.HashKey2Partitions(partitionKeys, partitionNames)
|
||||
if err != nil {
|
||||
log.Warn("has partition keys to partitions failed",
|
||||
log.Ctx(ctx).Warn("has partition keys to partitions failed",
|
||||
zap.String("collectionName", insertMsg.CollectionName),
|
||||
zap.Error(err))
|
||||
return nil, err
|
||||
|
|
|
@ -35,13 +35,13 @@ func RoundRobinPolicy(
|
|||
for _, target := range leaders {
|
||||
qn, err := mgr.GetClient(ctx, target)
|
||||
if err != nil {
|
||||
log.Warn("query channel failed, node not available", zap.String("channel", channel), zap.Int64("nodeID", target.nodeID), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("query channel failed, node not available", zap.String("channel", channel), zap.Int64("nodeID", target.nodeID), zap.Error(err))
|
||||
combineErr = merr.Combine(combineErr, err)
|
||||
continue
|
||||
}
|
||||
err = query(ctx, target.nodeID, qn, channel)
|
||||
if err != nil {
|
||||
log.Warn("query channel failed", zap.String("channel", channel), zap.Int64("nodeID", target.nodeID), zap.Error(err))
|
||||
log.Ctx(ctx).Warn("query channel failed", zap.String("channel", channel), zap.Int64("nodeID", target.nodeID), zap.Error(err))
|
||||
combineErr = merr.Combine(combineErr, err)
|
||||
continue
|
||||
}
|
||||
|
|
|
@ -284,14 +284,14 @@ func (t *queryTask) CanSkipAllocTimestamp() bool {
|
|||
} else {
|
||||
collID, err := globalMetaCache.GetCollectionID(context.Background(), t.request.GetDbName(), t.request.GetCollectionName())
|
||||
if err != nil { // err is not nil if collection not exists
|
||||
log.Warn("query task get collectionID failed, can't skip alloc timestamp",
|
||||
log.Ctx(t.ctx).Warn("query task get collectionID failed, can't skip alloc timestamp",
|
||||
zap.String("collectionName", t.request.GetCollectionName()), zap.Error(err))
|
||||
return false
|
||||
}
|
||||
|
||||
collectionInfo, err2 := globalMetaCache.GetCollectionInfo(context.Background(), t.request.GetDbName(), t.request.GetCollectionName(), collID)
|
||||
if err2 != nil {
|
||||
log.Warn("query task get collection info failed, can't skip alloc timestamp",
|
||||
log.Ctx(t.ctx).Warn("query task get collection info failed, can't skip alloc timestamp",
|
||||
zap.String("collectionName", t.request.GetCollectionName()), zap.Error(err))
|
||||
return false
|
||||
}
|
||||
|
|
|
@ -127,7 +127,7 @@ func (queue *baseTaskQueue) AddActiveTask(t task) {
|
|||
tID := t.ID()
|
||||
_, ok := queue.activeTasks[tID]
|
||||
if ok {
|
||||
log.Warn("Proxy task with tID already in active task list!", zap.Int64("ID", tID))
|
||||
log.Ctx(t.TraceCtx()).Warn("Proxy task with tID already in active task list!", zap.Int64("ID", tID))
|
||||
}
|
||||
|
||||
queue.activeTasks[tID] = t
|
||||
|
@ -142,7 +142,7 @@ func (queue *baseTaskQueue) PopActiveTask(taskID UniqueID) task {
|
|||
return t
|
||||
}
|
||||
|
||||
log.Warn("Proxy task not in active task list! ts", zap.Int64("taskID", taskID))
|
||||
log.Ctx(context.TODO()).Warn("Proxy task not in active task list! ts", zap.Int64("taskID", taskID))
|
||||
return t
|
||||
}
|
||||
|
||||
|
@ -250,7 +250,7 @@ func (queue *dmTaskQueue) Enqueue(t task) error {
|
|||
dmt := t.(dmlTask)
|
||||
err := dmt.setChannels()
|
||||
if err != nil {
|
||||
log.Warn("setChannels failed when Enqueue", zap.Int64("taskID", t.ID()), zap.Error(err))
|
||||
log.Ctx(t.TraceCtx()).Warn("setChannels failed when Enqueue", zap.Int64("taskID", t.ID()), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
@ -279,10 +279,10 @@ func (queue *dmTaskQueue) PopActiveTask(taskID UniqueID) task {
|
|||
defer queue.statsLock.Unlock()
|
||||
|
||||
delete(queue.activeTasks, taskID)
|
||||
log.Debug("Proxy dmTaskQueue popPChanStats", zap.Int64("taskID", t.ID()))
|
||||
log.Ctx(t.TraceCtx()).Debug("Proxy dmTaskQueue popPChanStats", zap.Int64("taskID", t.ID()))
|
||||
queue.popPChanStats(t)
|
||||
} else {
|
||||
log.Warn("Proxy task not in active task list!", zap.Int64("taskID", taskID))
|
||||
log.Ctx(context.TODO()).Warn("Proxy task not in active task list!", zap.Int64("taskID", taskID))
|
||||
}
|
||||
return t
|
||||
}
|
||||
|
@ -567,7 +567,7 @@ func (sched *taskScheduler) queryLoop() {
|
|||
return struct{}{}, nil
|
||||
})
|
||||
} else {
|
||||
log.Debug("query queue is empty ...")
|
||||
log.Ctx(context.TODO()).Debug("query queue is empty ...")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -100,14 +100,14 @@ func (t *searchTask) CanSkipAllocTimestamp() bool {
|
|||
} else {
|
||||
collID, err := globalMetaCache.GetCollectionID(context.Background(), t.request.GetDbName(), t.request.GetCollectionName())
|
||||
if err != nil { // err is not nil if collection not exists
|
||||
log.Warn("search task get collectionID failed, can't skip alloc timestamp",
|
||||
log.Ctx(t.ctx).Warn("search task get collectionID failed, can't skip alloc timestamp",
|
||||
zap.String("collectionName", t.request.GetCollectionName()), zap.Error(err))
|
||||
return false
|
||||
}
|
||||
|
||||
collectionInfo, err2 := globalMetaCache.GetCollectionInfo(context.Background(), t.request.GetDbName(), t.request.GetCollectionName(), collID)
|
||||
if err2 != nil {
|
||||
log.Warn("search task get collection info failed, can't skip alloc timestamp",
|
||||
log.Ctx(t.ctx).Warn("search task get collection info failed, can't skip alloc timestamp",
|
||||
zap.String("collectionName", t.request.GetCollectionName()), zap.Error(err))
|
||||
return false
|
||||
}
|
||||
|
@ -321,20 +321,20 @@ func setQueryInfoIfMvEnable(queryInfo *planpb.QueryInfo, t *searchTask, plan *pl
|
|||
if t.enableMaterializedView {
|
||||
partitionKeyFieldSchema, err := typeutil.GetPartitionKeyFieldSchema(t.schema.CollectionSchema)
|
||||
if err != nil {
|
||||
log.Warn("failed to get partition key field schema", zap.Error(err))
|
||||
log.Ctx(t.ctx).Warn("failed to get partition key field schema", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
if typeutil.IsFieldDataTypeSupportMaterializedView(partitionKeyFieldSchema) {
|
||||
collInfo, colErr := globalMetaCache.GetCollectionInfo(t.ctx, t.request.GetDbName(), t.collectionName, t.CollectionID)
|
||||
if colErr != nil {
|
||||
log.Warn("failed to get collection info", zap.Error(colErr))
|
||||
log.Ctx(t.ctx).Warn("failed to get collection info", zap.Error(colErr))
|
||||
return err
|
||||
}
|
||||
|
||||
if collInfo.partitionKeyIsolation {
|
||||
expr, err := exprutil.ParseExprFromPlan(plan)
|
||||
if err != nil {
|
||||
log.Warn("failed to parse expr from plan during MV", zap.Error(err))
|
||||
log.Ctx(t.ctx).Warn("failed to parse expr from plan during MV", zap.Error(err))
|
||||
return err
|
||||
}
|
||||
err = exprutil.ValidatePartitionKeyIsolation(expr)
|
||||
|
@ -426,7 +426,7 @@ func (t *searchTask) initAdvancedSearchRequest(ctx context.Context) error {
|
|||
t.SearchRequest.PartitionIDs = t.partitionIDsSet.Collect()
|
||||
}
|
||||
var err error
|
||||
t.reScorers, err = NewReScorers(len(t.request.GetSubReqs()), t.request.GetSearchParams())
|
||||
t.reScorers, err = NewReScorers(ctx, len(t.request.GetSubReqs()), t.request.GetSearchParams())
|
||||
if err != nil {
|
||||
log.Info("generate reScorer failed", zap.Any("params", t.request.GetSearchParams()), zap.Error(err))
|
||||
return err
|
||||
|
@ -528,12 +528,12 @@ func (t *searchTask) tryGeneratePlan(params []*commonpb.KeyValuePair, dsl string
|
|||
searchInfo.planInfo.QueryFieldId = annField.GetFieldID()
|
||||
plan, planErr := planparserv2.CreateSearchPlan(t.schema.schemaHelper, dsl, annsFieldName, searchInfo.planInfo, exprTemplateValues)
|
||||
if planErr != nil {
|
||||
log.Warn("failed to create query plan", zap.Error(planErr),
|
||||
log.Ctx(t.ctx).Warn("failed to create query plan", zap.Error(planErr),
|
||||
zap.String("dsl", dsl), // may be very large if large term passed.
|
||||
zap.String("anns field", annsFieldName), zap.Any("query info", searchInfo.planInfo))
|
||||
return nil, nil, 0, false, merr.WrapErrParameterInvalidMsg("failed to create query plan: %v", planErr)
|
||||
}
|
||||
log.Debug("create query plan",
|
||||
log.Ctx(t.ctx).Debug("create query plan",
|
||||
zap.String("dsl", t.request.Dsl), // may be very large if large term passed.
|
||||
zap.String("anns field", annsFieldName), zap.Any("query info", searchInfo.planInfo))
|
||||
return plan, searchInfo.planInfo, searchInfo.offset, searchInfo.isIterator, nil
|
||||
|
@ -542,13 +542,13 @@ func (t *searchTask) tryGeneratePlan(params []*commonpb.KeyValuePair, dsl string
|
|||
func (t *searchTask) tryParsePartitionIDsFromPlan(plan *planpb.PlanNode) ([]int64, error) {
|
||||
expr, err := exprutil.ParseExprFromPlan(plan)
|
||||
if err != nil {
|
||||
log.Warn("failed to parse expr", zap.Error(err))
|
||||
log.Ctx(t.ctx).Warn("failed to parse expr", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
partitionKeys := exprutil.ParseKeys(expr, exprutil.PartitionKey)
|
||||
hashedPartitionNames, err := assignPartitionKeys(t.ctx, t.request.GetDbName(), t.collectionName, partitionKeys)
|
||||
if err != nil {
|
||||
log.Warn("failed to assign partition keys", zap.Error(err))
|
||||
log.Ctx(t.ctx).Warn("failed to assign partition keys", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
|
@ -556,7 +556,7 @@ func (t *searchTask) tryParsePartitionIDsFromPlan(plan *planpb.PlanNode) ([]int6
|
|||
// translate partition name to partition ids. Use regex-pattern to match partition name.
|
||||
PartitionIDs, err2 := getPartitionIDs(t.ctx, t.request.GetDbName(), t.collectionName, hashedPartitionNames)
|
||||
if err2 != nil {
|
||||
log.Warn("failed to get partition ids", zap.Error(err2))
|
||||
log.Ctx(t.ctx).Warn("failed to get partition ids", zap.Error(err2))
|
||||
return nil, err2
|
||||
}
|
||||
return PartitionIDs, nil
|
||||
|
@ -599,6 +599,7 @@ func (t *searchTask) reduceResults(ctx context.Context, toReduceResults []*inter
|
|||
ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "reduceResults")
|
||||
defer sp.End()
|
||||
|
||||
log := log.Ctx(ctx)
|
||||
// Decode all search results
|
||||
validSearchResults, err := decodeSearchResults(ctx, toReduceResults)
|
||||
if err != nil {
|
||||
|
@ -992,7 +993,7 @@ func checkSearchResultData(data *schemapb.SearchResultData, nq int64, topk int64
|
|||
return nil
|
||||
}
|
||||
|
||||
func selectHighestScoreIndex(subSearchResultData []*schemapb.SearchResultData, subSearchNqOffset [][]int64, cursors []int64, qi int64) (int, int64) {
|
||||
func selectHighestScoreIndex(ctx context.Context, subSearchResultData []*schemapb.SearchResultData, subSearchNqOffset [][]int64, cursors []int64, qi int64) (int, int64) {
|
||||
var (
|
||||
subSearchIdx = -1
|
||||
resultDataIdx int64 = -1
|
||||
|
@ -1014,7 +1015,7 @@ func selectHighestScoreIndex(subSearchResultData []*schemapb.SearchResultData, s
|
|||
if subSearchIdx == -1 {
|
||||
// A bad case happens where Knowhere returns distance/score == +/-maxFloat32
|
||||
// by mistake.
|
||||
log.Error("a bad score is returned, something is wrong here!", zap.Float32("score", sScore))
|
||||
log.Ctx(ctx).Error("a bad score is returned, something is wrong here!", zap.Float32("score", sScore))
|
||||
} else if typeutil.ComparePK(
|
||||
typeutil.GetPK(subSearchResultData[i].GetIds(), sIdx),
|
||||
typeutil.GetPK(subSearchResultData[subSearchIdx].GetIds(), resultDataIdx)) {
|
||||
|
|
|
@ -1369,7 +1369,7 @@ func TestTaskSearch_selectHighestScoreIndex(t *testing.T) {
|
|||
for _, test := range tests {
|
||||
t.Run(test.description, func(t *testing.T) {
|
||||
for nqNum := int64(0); nqNum < test.args.nq; nqNum++ {
|
||||
idx, dataIdx := selectHighestScoreIndex(test.args.subSearchResultData, test.args.subSearchNqOffset, test.args.cursors, nqNum)
|
||||
idx, dataIdx := selectHighestScoreIndex(context.TODO(), test.args.subSearchResultData, test.args.subSearchNqOffset, test.args.cursors, nqNum)
|
||||
assert.Equal(t, test.expectedIdx[nqNum], idx)
|
||||
assert.Equal(t, test.expectedDataIdx[nqNum], int(dataIdx))
|
||||
}
|
||||
|
@ -1493,7 +1493,7 @@ func TestTaskSearch_selectHighestScoreIndex(t *testing.T) {
|
|||
for _, test := range tests {
|
||||
t.Run(test.description, func(t *testing.T) {
|
||||
for nqNum := int64(0); nqNum < test.args.nq; nqNum++ {
|
||||
idx, dataIdx := selectHighestScoreIndex(test.args.subSearchResultData, test.args.subSearchNqOffset, test.args.cursors, nqNum)
|
||||
idx, dataIdx := selectHighestScoreIndex(context.TODO(), test.args.subSearchResultData, test.args.subSearchNqOffset, test.args.cursors, nqNum)
|
||||
assert.Equal(t, test.expectedIdx[nqNum], idx)
|
||||
assert.Equal(t, test.expectedDataIdx[nqNum], int(dataIdx))
|
||||
}
|
||||
|
|
|
@ -178,7 +178,7 @@ func (g *getStatisticsTask) Execute(ctx context.Context) error {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
log.Debug("get collection statistics from DataCoord execute done")
|
||||
log.Ctx(ctx).Debug("get collection statistics from DataCoord execute done")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -194,13 +194,13 @@ func (g *getStatisticsTask) PostExecute(ctx context.Context) error {
|
|||
toReduceResults := make([]*internalpb.GetStatisticsResponse, 0)
|
||||
select {
|
||||
case <-g.TraceCtx().Done():
|
||||
log.Debug("wait to finish timeout!")
|
||||
log.Ctx(ctx).Debug("wait to finish timeout!")
|
||||
return nil
|
||||
default:
|
||||
log.Debug("all get statistics are finished or canceled")
|
||||
log.Ctx(ctx).Debug("all get statistics are finished or canceled")
|
||||
g.resultBuf.Range(func(res *internalpb.GetStatisticsResponse) bool {
|
||||
toReduceResults = append(toReduceResults, res)
|
||||
log.Debug("proxy receives one get statistic response",
|
||||
log.Ctx(ctx).Debug("proxy receives one get statistic response",
|
||||
zap.Int64("sourceID", res.GetBase().GetSourceID()))
|
||||
return true
|
||||
})
|
||||
|
@ -220,7 +220,7 @@ func (g *getStatisticsTask) PostExecute(ctx context.Context) error {
|
|||
Stats: result,
|
||||
}
|
||||
|
||||
log.Debug("get statistics post execute done", zap.Any("result", result))
|
||||
log.Ctx(ctx).Debug("get statistics post execute done", zap.Any("result", result))
|
||||
return nil
|
||||
}
|
||||
|
||||
|
@ -283,7 +283,7 @@ func (g *getStatisticsTask) getStatisticsShard(ctx context.Context, nodeID int64
|
|||
}
|
||||
result, err := qn.GetStatistics(ctx, req)
|
||||
if err != nil {
|
||||
log.Warn("QueryNode statistic return error",
|
||||
log.Ctx(ctx).Warn("QueryNode statistic return error",
|
||||
zap.Int64("nodeID", nodeID),
|
||||
zap.String("channel", channel),
|
||||
zap.Error(err))
|
||||
|
@ -291,14 +291,14 @@ func (g *getStatisticsTask) getStatisticsShard(ctx context.Context, nodeID int64
|
|||
return err
|
||||
}
|
||||
if result.GetStatus().GetErrorCode() == commonpb.ErrorCode_NotShardLeader {
|
||||
log.Warn("QueryNode is not shardLeader",
|
||||
log.Ctx(ctx).Warn("QueryNode is not shardLeader",
|
||||
zap.Int64("nodeID", nodeID),
|
||||
zap.String("channel", channel))
|
||||
globalMetaCache.DeprecateShardCache(g.request.GetDbName(), g.collectionName)
|
||||
return errInvalidShardLeaders
|
||||
}
|
||||
if result.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
|
||||
log.Warn("QueryNode statistic result error",
|
||||
log.Ctx(ctx).Warn("QueryNode statistic result error",
|
||||
zap.Int64("nodeID", nodeID),
|
||||
zap.String("reason", result.GetStatus().GetReason()))
|
||||
return errors.Wrapf(merr.Error(result.GetStatus()), "fail to get statistic on QueryNode ID=%d", nodeID)
|
||||
|
|
|
@ -2373,7 +2373,7 @@ func Test_checkTrain(t *testing.T) {
|
|||
"nlist": "1024",
|
||||
common.MetricTypeKey: "L2",
|
||||
}
|
||||
assert.NoError(t, checkTrain(f, m))
|
||||
assert.NoError(t, checkTrain(context.TODO(), f, m))
|
||||
})
|
||||
|
||||
t.Run("scalar", func(t *testing.T) {
|
||||
|
@ -2383,7 +2383,7 @@ func Test_checkTrain(t *testing.T) {
|
|||
m := map[string]string{
|
||||
common.IndexTypeKey: "scalar",
|
||||
}
|
||||
assert.Error(t, checkTrain(f, m))
|
||||
assert.Error(t, checkTrain(context.TODO(), f, m))
|
||||
})
|
||||
|
||||
t.Run("dimension mismatch", func(t *testing.T) {
|
||||
|
@ -2402,7 +2402,7 @@ func Test_checkTrain(t *testing.T) {
|
|||
common.MetricTypeKey: "L2",
|
||||
common.DimKey: "8",
|
||||
}
|
||||
assert.Error(t, checkTrain(f, m))
|
||||
assert.Error(t, checkTrain(context.TODO(), f, m))
|
||||
})
|
||||
|
||||
t.Run("nlist test", func(t *testing.T) {
|
||||
|
@ -2419,7 +2419,7 @@ func Test_checkTrain(t *testing.T) {
|
|||
common.IndexTypeKey: "IVF_FLAT",
|
||||
common.MetricTypeKey: "L2",
|
||||
}
|
||||
assert.NoError(t, checkTrain(f, m))
|
||||
assert.NoError(t, checkTrain(context.TODO(), f, m))
|
||||
})
|
||||
}
|
||||
|
||||
|
|
|
@ -148,7 +148,7 @@ func (it *upsertTask) OnEnqueue() error {
|
|||
func (it *upsertTask) insertPreExecute(ctx context.Context) error {
|
||||
collectionName := it.upsertMsg.InsertMsg.CollectionName
|
||||
if err := validateCollectionName(collectionName); err != nil {
|
||||
log.Error("valid collection name failed", zap.String("collectionName", collectionName), zap.Error(err))
|
||||
log.Ctx(ctx).Error("valid collection name failed", zap.String("collectionName", collectionName), zap.Error(err))
|
||||
return err
|
||||
}
|
||||
|
||||
|
|
|
@ -1219,7 +1219,7 @@ func passwordVerify(ctx context.Context, username, rawPwd string, globalMetaCach
|
|||
// meanwhile, generating Sha256Password depends on raw password and encrypted password will not cache.
|
||||
credInfo, err := globalMetaCache.GetCredentialInfo(ctx, username)
|
||||
if err != nil {
|
||||
log.Error("found no credential", zap.String("username", username), zap.Error(err))
|
||||
log.Ctx(ctx).Error("found no credential", zap.String("username", username), zap.Error(err))
|
||||
return false
|
||||
}
|
||||
|
||||
|
@ -1231,13 +1231,13 @@ func passwordVerify(ctx context.Context, username, rawPwd string, globalMetaCach
|
|||
|
||||
// miss cache, verify against encrypted password from etcd
|
||||
if err := bcrypt.CompareHashAndPassword([]byte(credInfo.EncryptedPassword), []byte(rawPwd)); err != nil {
|
||||
log.Error("Verify password failed", zap.Error(err))
|
||||
log.Ctx(ctx).Error("Verify password failed", zap.Error(err))
|
||||
return false
|
||||
}
|
||||
|
||||
// update cache after miss cache
|
||||
credInfo.Sha256Password = sha256Pwd
|
||||
log.Debug("get credential miss cache, update cache with", zap.Any("credential", credInfo))
|
||||
log.Ctx(ctx).Debug("get credential miss cache, update cache with", zap.Any("credential", credInfo))
|
||||
globalMetaCache.UpdateCredential(credInfo)
|
||||
return true
|
||||
}
|
||||
|
@ -1712,7 +1712,7 @@ func getCollectionProgress(
|
|||
CollectionIDs: []int64{collectionID},
|
||||
})
|
||||
if err != nil {
|
||||
log.Warn("fail to show collections",
|
||||
log.Ctx(ctx).Warn("fail to show collections",
|
||||
zap.Int64("collectionID", collectionID),
|
||||
zap.Error(err),
|
||||
)
|
||||
|
@ -1721,7 +1721,7 @@ func getCollectionProgress(
|
|||
|
||||
err = merr.Error(resp.GetStatus())
|
||||
if err != nil {
|
||||
log.Warn("fail to show collections",
|
||||
log.Ctx(ctx).Warn("fail to show collections",
|
||||
zap.Int64("collectionID", collectionID),
|
||||
zap.Error(err))
|
||||
return
|
||||
|
@ -1766,7 +1766,7 @@ func getPartitionProgress(
|
|||
PartitionIDs: partitionIDs,
|
||||
})
|
||||
if err != nil {
|
||||
log.Warn("fail to show partitions", zap.Int64("collection_id", collectionID),
|
||||
log.Ctx(ctx).Warn("fail to show partitions", zap.Int64("collection_id", collectionID),
|
||||
zap.String("collection_name", collectionName),
|
||||
zap.Strings("partition_names", partitionNames),
|
||||
zap.Error(err))
|
||||
|
@ -1776,7 +1776,7 @@ func getPartitionProgress(
|
|||
err = merr.Error(resp.GetStatus())
|
||||
if err != nil {
|
||||
err = merr.Error(resp.GetStatus())
|
||||
log.Warn("fail to show partitions",
|
||||
log.Ctx(ctx).Warn("fail to show partitions",
|
||||
zap.String("collectionName", collectionName),
|
||||
zap.Strings("partitionNames", partitionNames),
|
||||
zap.Error(err))
|
||||
|
@ -1931,7 +1931,7 @@ func checkDynamicFieldData(schema *schemapb.CollectionSchema, insertMsg *msgstre
|
|||
|
||||
func SendReplicateMessagePack(ctx context.Context, replicateMsgStream msgstream.MsgStream, request interface{ GetBase() *commonpb.MsgBase }) {
|
||||
if replicateMsgStream == nil || request == nil {
|
||||
log.Warn("replicate msg stream or request is nil", zap.Any("request", request))
|
||||
log.Ctx(ctx).Warn("replicate msg stream or request is nil", zap.Any("request", request))
|
||||
return
|
||||
}
|
||||
msgBase := request.GetBase()
|
||||
|
|
|
@ -122,7 +122,7 @@ func (c *LeaderChecker) findNeedSyncPartitionStats(ctx context.Context, replica
|
|||
if psVersionInLView < psVersionInTarget {
|
||||
partStatsToUpdate[partID] = psVersionInTarget
|
||||
} else {
|
||||
log.RatedDebug(60, "no need to update part stats for partition",
|
||||
log.Ctx(ctx).RatedDebug(60, "no need to update part stats for partition",
|
||||
zap.Int64("partitionID", partID),
|
||||
zap.Int64("psVersionInLView", psVersionInLView),
|
||||
zap.Int64("psVersionInTarget", psVersionInTarget))
|
||||
|
@ -144,7 +144,7 @@ func (c *LeaderChecker) findNeedSyncPartitionStats(ctx context.Context, replica
|
|||
t.SetPriority(task.TaskPriorityLow)
|
||||
t.SetReason("sync partition stats versions")
|
||||
ret = append(ret, t)
|
||||
log.Debug("Created leader actions for partitionStats",
|
||||
log.Ctx(ctx).Debug("Created leader actions for partitionStats",
|
||||
zap.Int64("collectionID", leaderView.CollectionID),
|
||||
zap.Any("action", action.String()))
|
||||
}
|
||||
|
|
|
@ -127,7 +127,7 @@ func (s *Server) balanceSegments(ctx context.Context,
|
|||
actions = append(actions, releaseAction)
|
||||
}
|
||||
|
||||
t, err := task.NewSegmentTask(s.ctx,
|
||||
t, err := task.NewSegmentTask(ctx,
|
||||
Params.QueryCoordCfg.SegmentTaskTimeout.GetAsDuration(time.Millisecond),
|
||||
utils.ManualBalance,
|
||||
collectionID,
|
||||
|
|
|
@ -113,6 +113,7 @@ func (broker *CoordinatorBroker) GetCollectionLoadInfo(ctx context.Context, coll
|
|||
return nil, 0, err
|
||||
}
|
||||
|
||||
log := log.Ctx(ctx)
|
||||
replicaNum, err := common.CollectionLevelReplicaNumber(collectionInfo.GetProperties())
|
||||
if err != nil {
|
||||
log.Debug("failed to get collection level load info", zap.Int64("collectionID", collectionID), zap.Error(err))
|
||||
|
|
|
@ -140,6 +140,7 @@ func (mgr *TargetManager) UpdateCollectionCurrentTarget(ctx context.Context, col
|
|||
// WARN: DO NOT call this method for an existing collection as target observer running, or it will lead to a double-update,
|
||||
// which may make the current target not available
|
||||
func (mgr *TargetManager) UpdateCollectionNextTarget(ctx context.Context, collectionID int64) error {
|
||||
log := log.Ctx(ctx)
|
||||
var vChannelInfos []*datapb.VchannelInfo
|
||||
var segmentInfos []*datapb.SegmentInfo
|
||||
err := retry.Handle(ctx, func() (bool, error) {
|
||||
|
|
|
@ -134,7 +134,7 @@ func (node *MockQueryNode) Start() error {
|
|||
node.session.Init(typeutil.QueryNodeRole, node.addr, false, true)
|
||||
node.session.ServerID = node.ID
|
||||
node.session.Register()
|
||||
log.Debug("mock QueryNode started",
|
||||
log.Ctx(context.TODO()).Debug("mock QueryNode started",
|
||||
zap.Int64("nodeID", node.ID),
|
||||
zap.String("nodeAddr", node.addr))
|
||||
|
||||
|
|
|
@ -93,6 +93,7 @@ func (ob *ResourceObserver) checkAndRecoverResourceGroup(ctx context.Context) {
|
|||
manager := ob.meta.ResourceManager
|
||||
rgNames := manager.ListResourceGroups(ctx)
|
||||
enableRGAutoRecover := params.Params.QueryCoordCfg.EnableRGAutoRecover.GetAsBool()
|
||||
log := log.Ctx(ctx)
|
||||
log.Debug("start to check resource group", zap.Bool("enableRGAutoRecover", enableRGAutoRecover), zap.Int("resourceGroupNum", len(rgNames)))
|
||||
|
||||
// Check if there is any incoming node.
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue