From 216149c4d91ea418fb583f75c1a7d3d61ef6d18e Mon Sep 17 00:00:00 2001 From: tinswzy Date: Tue, 3 Dec 2024 20:32:09 +0800 Subject: [PATCH] refine exists log print with ctx Signed-off-by: tinswzy --- cmd/components/data_coord.go | 4 +- cmd/components/data_node.go | 4 +- cmd/components/index_coord.go | 4 +- cmd/components/index_node.go | 9 +- cmd/components/proxy.go | 4 +- cmd/components/query_coord.go | 4 +- cmd/components/query_node.go | 4 +- cmd/components/root_coord.go | 4 +- cmd/components/streaming_node.go | 4 +- cmd/milvus/mck.go | 12 +- cmd/milvus/util.go | 3 +- cmd/tools/config/generate.go | 11 +- cmd/tools/migration/mmap/tool/main.go | 2 +- internal/coordinator/coordclient/registry.go | 6 +- internal/datacoord/channel_manager.go | 24 +-- internal/datacoord/channel_store.go | 9 +- internal/datacoord/cluster.go | 4 +- internal/datacoord/compaction.go | 11 +- .../datacoord/compaction_task_clustering.go | 14 +- internal/datacoord/compaction_trigger.go | 3 +- internal/datacoord/compaction_trigger_v2.go | 10 +- internal/datacoord/garbage_collector.go | 12 +- internal/datacoord/import_scheduler.go | 2 +- internal/datacoord/index_meta.go | 40 ++--- internal/datacoord/index_service.go | 7 +- internal/datacoord/meta.go | 97 ++++++------ internal/datacoord/server.go | 39 +++-- internal/datacoord/services.go | 10 +- .../datacoord/session/indexnode_manager.go | 11 +- .../compaction/clustering_compactor.go | 4 + .../datanode/compaction/compactor_common.go | 1 + internal/datanode/data_node.go | 11 +- internal/datanode/services.go | 16 +- internal/distributed/connection_manager.go | 26 ++-- .../distributed/connection_manager_test.go | 4 +- .../distributed/datacoord/client/client.go | 8 +- internal/distributed/datacoord/service.go | 23 +-- .../distributed/datanode/client/client.go | 8 +- internal/distributed/datanode/service.go | 29 ++-- .../distributed/indexnode/client/client.go | 4 +- internal/distributed/indexnode/service.go | 18 ++- internal/distributed/proxy/client/client.go | 4 +- .../distributed/proxy/listener_manager.go | 10 +- internal/distributed/proxy/service.go | 30 ++-- .../distributed/querycoord/client/client.go | 13 +- internal/distributed/querycoord/service.go | 15 +- .../distributed/querynode/client/client.go | 7 +- internal/distributed/querynode/service.go | 21 +-- .../distributed/rootcoord/client/client.go | 8 +- internal/distributed/rootcoord/service.go | 16 +- internal/distributed/streamingnode/service.go | 76 +++++---- internal/distributed/utils/util.go | 5 + internal/flushcommon/io/binlog_io.go | 8 +- .../pipeline/flow_graph_dd_node.go | 1 + .../pipeline/flow_graph_time_tick_node.go | 3 +- .../flushcommon/pipeline/testutils_test.go | 2 +- internal/indexnode/indexnode.go | 4 +- internal/indexnode/task_scheduler.go | 6 +- internal/indexnode/task_stats.go | 30 ++-- internal/indexnode/taskinfo_ops.go | 2 +- internal/kv/etcd/etcd_kv.go | 2 +- internal/metastore/kv/datacoord/kv_catalog.go | 38 ++--- .../metastore/kv/querycoord/kv_catalog.go | 2 +- internal/metastore/kv/rootcoord/kv_catalog.go | 147 +++++++++--------- .../metastore/kv/rootcoord/kv_catalog_test.go | 6 +- .../metastore/kv/rootcoord/suffix_snapshot.go | 2 + internal/mocks/util/mock_segcore/mock_data.go | 2 + internal/proxy/impl.go | 26 ++-- internal/proxy/meta_cache.go | 8 +- internal/proxy/msg_pack.go | 18 +-- internal/proxy/proxy.go | 4 + internal/proxy/reScorer.go | 4 +- internal/proxy/reScorer_test.go | 17 +- internal/proxy/search_reduce_util.go | 6 +- internal/proxy/segment.go | 14 +- internal/proxy/simple_rate_limiter.go | 2 +- internal/proxy/task.go | 40 ++--- internal/proxy/task_database.go | 4 +- internal/proxy/task_delete.go | 22 +-- internal/proxy/task_delete_streaming.go | 4 +- internal/proxy/task_flush_streaming.go | 6 +- internal/proxy/task_index.go | 36 ++--- internal/proxy/task_index_test.go | 62 ++++---- internal/proxy/task_insert.go | 6 +- internal/proxy/task_insert_streaming.go | 6 +- internal/proxy/task_policies.go | 4 +- internal/proxy/task_query.go | 4 +- internal/proxy/task_scheduler.go | 12 +- internal/proxy/task_search.go | 27 ++-- internal/proxy/task_search_test.go | 4 +- internal/proxy/task_statistic.go | 16 +- internal/proxy/task_test.go | 8 +- internal/proxy/task_upsert.go | 2 +- internal/proxy/util.go | 16 +- .../querycoordv2/checkers/leader_checker.go | 4 +- internal/querycoordv2/handlers.go | 4 +- .../querycoordv2/meta/coordinator_broker.go | 1 + internal/querycoordv2/meta/target_manager.go | 1 + internal/querycoordv2/mocks/querynode.go | 2 +- .../observers/resource_observer.go | 1 + internal/querycoordv2/ops_services.go | 8 +- internal/querycoordv2/server.go | 22 ++- internal/querycoordv2/services.go | 15 +- internal/querycoordv2/task/scheduler.go | 34 ++-- .../querynodev2/delegator/exclude_info.go | 3 +- internal/querynodev2/delegator/idf_oracle.go | 3 +- .../querynodev2/delegator/segment_pruner.go | 2 +- internal/querynodev2/handlers.go | 6 +- .../querynodev2/pipeline/embedding_node.go | 3 +- internal/querynodev2/pipeline/filter_node.go | 2 + internal/querynodev2/pipeline/insert_node.go | 3 +- internal/querynodev2/segments/result.go | 6 +- internal/querynodev2/segments/retrieve.go | 1 + internal/querynodev2/server.go | 13 +- internal/querynodev2/services.go | 14 +- internal/querynodev2/services_test.go | 2 +- internal/rootcoord/alter_collection_task.go | 6 +- internal/rootcoord/alter_database_task.go | 4 +- internal/rootcoord/broker.go | 4 +- internal/rootcoord/create_collection_task.go | 52 +++---- .../rootcoord/create_collection_task_test.go | 70 ++++----- internal/rootcoord/create_partition_task.go | 2 +- internal/rootcoord/dml_channels.go | 18 +-- internal/rootcoord/drop_collection_task.go | 2 +- internal/rootcoord/drop_partition_task.go | 2 +- internal/rootcoord/list_db_task.go | 2 +- internal/rootcoord/meta_table.go | 56 +++---- internal/rootcoord/meta_table_test.go | 6 +- internal/rootcoord/root_coord.go | 64 ++++---- internal/rootcoord/show_collection_task.go | 2 +- internal/rootcoord/step.go | 4 + internal/rootcoord/step_executor.go | 4 +- internal/rootcoord/timeticksync.go | 2 +- internal/rootcoord/undo.go | 2 +- .../client/manager/manager_client_impl.go | 1 + internal/util/flowgraph/node.go | 3 +- internal/util/grpcclient/client.go | 2 +- internal/util/metrics/thread.go | 2 + internal/util/pipeline/stream_pipeline.go | 5 +- internal/util/proxyutil/proxy_watcher.go | 4 +- internal/util/sessionutil/session_util.go | 20 ++- pkg/config/etcd_source.go | 6 +- pkg/config/refresher.go | 2 + pkg/log/global.go | 10 ++ pkg/mq/mqimpl/rocksmq/client/client_impl.go | 3 +- pkg/mq/mqimpl/rocksmq/server/global_rmq.go | 5 +- pkg/mq/mqimpl/rocksmq/server/rocksmq_impl.go | 39 +++-- .../rocksmq/server/rocksmq_retention.go | 5 +- pkg/mq/msgstream/mq_msgstream.go | 30 ++-- .../msgstream/mqwrapper/kafka/kafka_client.go | 1 + .../mqwrapper/kafka/kafka_producer.go | 1 + pkg/mq/msgstream/mqwrapper/nmq/nmq_server.go | 3 +- .../mqwrapper/pulsar/pulsar_consumer.go | 1 + pkg/util/funcutil/parallel.go | 4 +- pkg/util/funcutil/policy.go | 2 + pkg/util/gc/gc_tuner.go | 5 +- tests/integration/minicluster_v2.go | 4 +- 157 files changed, 1094 insertions(+), 874 deletions(-) diff --git a/cmd/components/data_coord.go b/cmd/components/data_coord.go index 6a031174d829d..8f291d0daa49b 100644 --- a/cmd/components/data_coord.go +++ b/cmd/components/data_coord.go @@ -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 } diff --git a/cmd/components/data_node.go b/cmd/components/data_node.go index f2830bcbf9549..f410bbd0b5984 100644 --- a/cmd/components/data_node.go +++ b/cmd/components/data_node.go @@ -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 } diff --git a/cmd/components/index_coord.go b/cmd/components/index_coord.go index 2d03c1d08fadf..bcc2c8d458986 100644 --- a/cmd/components/index_coord.go +++ b/cmd/components/index_coord.go @@ -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 } diff --git a/cmd/components/index_node.go b/cmd/components/index_node.go index 0d4f6496f76a7..c0f4e6c45b055 100644 --- a/cmd/components/index_node.go +++ b/cmd/components/index_node.go @@ -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 } diff --git a/cmd/components/proxy.go b/cmd/components/proxy.go index 96ee18ed6a574..60baf102958f9 100644 --- a/cmd/components/proxy.go +++ b/cmd/components/proxy.go @@ -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 } diff --git a/cmd/components/query_coord.go b/cmd/components/query_coord.go index f796e7c9e3523..5b07d4d1148fe 100644 --- a/cmd/components/query_coord.go +++ b/cmd/components/query_coord.go @@ -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 } diff --git a/cmd/components/query_node.go b/cmd/components/query_node.go index 325fd77c72a71..fa048e466ab32 100644 --- a/cmd/components/query_node.go +++ b/cmd/components/query_node.go @@ -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 } diff --git a/cmd/components/root_coord.go b/cmd/components/root_coord.go index 24040bc94a8ca..ab81b5818b61d 100644 --- a/cmd/components/root_coord.go +++ b/cmd/components/root_coord.go @@ -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 } diff --git a/cmd/components/streaming_node.go b/cmd/components/streaming_node.go index ab46f0b901a36..2b0d63bdc1249 100644 --- a/cmd/components/streaming_node.go +++ b/cmd/components/streaming_node.go @@ -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 } diff --git a/cmd/milvus/mck.go b/cmd/milvus/mck.go index 6581356c252b6..039232f08c881 100644 --- a/cmd/milvus/mck.go +++ b/cmd/milvus/mck.go @@ -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(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(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(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 { diff --git a/cmd/milvus/util.go b/cmd/milvus/util.go index e7dcb2035343c..35040094aeb5f 100644 --- a/cmd/milvus/util.go +++ b/cmd/milvus/util.go @@ -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{}{} }) diff --git a/cmd/tools/config/generate.go b/cmd/tools/config/generate.go index f09925c573206..f323e0b913e59 100644 --- a/cmd/tools/config/generate.go +++ b/cmd/tools/config/generate.go @@ -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] diff --git a/cmd/tools/migration/mmap/tool/main.go b/cmd/tools/migration/mmap/tool/main.go index 8975ffe59ef54..0c659e6185518 100644 --- a/cmd/tools/migration/mmap/tool/main.go +++ b/cmd/tools/migration/mmap/tool/main.go @@ -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 diff --git a/internal/coordinator/coordclient/registry.go b/internal/coordinator/coordclient/registry.go index 8ba4a721cd5ec..a7a9e65a02a26 100644 --- a/internal/coordinator/coordclient/registry.go +++ b/internal/coordinator/coordclient/registry.go @@ -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 diff --git a/internal/datacoord/channel_manager.go b/internal/datacoord/channel_manager.go index c144d35c62a74..28bab70ac18a2 100644 --- a/internal/datacoord/channel_manager.go +++ b/internal/datacoord/channel_manager.go @@ -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()), diff --git a/internal/datacoord/channel_store.go b/internal/datacoord/channel_store.go index d3b6a2b3914ac..278e6ccda567a 100644 --- a/internal/datacoord/channel_store.go +++ b/internal/datacoord/channel_store.go @@ -17,6 +17,7 @@ package datacoord import ( + "context" "fmt" "math" "strconv" @@ -516,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 } @@ -549,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 @@ -571,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 diff --git a/internal/datacoord/cluster.go b/internal/datacoord/cluster.go index aeabfbbfc2de9..6927648ed0593 100644 --- a/internal/datacoord/cluster.go +++ b/internal/datacoord/cluster.go @@ -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 } diff --git a/internal/datacoord/compaction.go b/internal/datacoord/compaction.go index 7368c36b56188..ea56f29349dac 100644 --- a/internal/datacoord/compaction.go +++ b/internal/datacoord/compaction.go @@ -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 } diff --git a/internal/datacoord/compaction_task_clustering.go b/internal/datacoord/compaction_task_clustering.go index fe0e37d1abcfa..2df1e5c651cd7 100644 --- a/internal/datacoord/compaction_task_clustering.go +++ b/internal/datacoord/compaction_task_clustering.go @@ -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()), ) diff --git a/internal/datacoord/compaction_trigger.go b/internal/datacoord/compaction_trigger.go index bb2aa10d94225..a86aaccbfc057 100644 --- a/internal/datacoord/compaction_trigger.go +++ b/internal/datacoord/compaction_trigger.go @@ -662,7 +662,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), @@ -674,6 +674,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) { diff --git a/internal/datacoord/compaction_trigger_v2.go b/internal/datacoord/compaction_trigger_v2.go index 25ffae444b3b0..e7ee53f172087 100644 --- a/internal/datacoord/compaction_trigger_v2.go +++ b/internal/datacoord/compaction_trigger_v2.go @@ -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) diff --git a/internal/datacoord/garbage_collector.go b/internal/datacoord/garbage_collector.go index 8dcabc15ddef0..f24a67cbbcc4b 100644 --- a/internal/datacoord/garbage_collector.go +++ b/internal/datacoord/garbage_collector.go @@ -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))) }() diff --git a/internal/datacoord/import_scheduler.go b/internal/datacoord/import_scheduler.go index 428cb26ae7b35..e5b7585e2fb82 100644 --- a/internal/datacoord/import_scheduler.go +++ b/internal/datacoord/import_scheduler.go @@ -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 } diff --git a/internal/datacoord/index_meta.go b/internal/datacoord/index_meta.go index 3eb3210d13b76..b9052afb8b476 100644 --- a/internal/datacoord/index_meta.go +++ b/internal/datacoord/index_meta.go @@ -365,7 +365,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 @@ -375,20 +375,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 } @@ -415,19 +415,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() @@ -562,7 +562,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() @@ -587,14 +587,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 } @@ -759,7 +759,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) @@ -780,7 +780,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 { @@ -796,7 +796,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()), ) @@ -811,7 +811,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 } @@ -824,7 +824,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 } @@ -844,7 +844,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 @@ -852,7 +852,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() @@ -930,10 +930,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 } @@ -946,7 +946,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 } diff --git a/internal/datacoord/index_service.go b/internal/datacoord/index_service.go index 136850c4149c5..b0c9b70417b94 100644 --- a/internal/datacoord/index_service.go +++ b/internal/datacoord/index_service.go @@ -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() diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index 20f72ec454562..60529aa8d2dea 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -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())) diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index eb6108a77f972..e219b5e757eff 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -247,6 +247,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() @@ -300,6 +301,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) @@ -335,6 +337,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 { @@ -431,6 +434,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") @@ -566,6 +570,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 { @@ -649,6 +654,7 @@ func (s *Server) initSegmentManager() error { } func (s *Server) initMeta(chunkManager storage.ChunkManager) error { + log := log.Ctx(s.ctx) if s.meta != nil { return nil } @@ -766,6 +772,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 { @@ -795,6 +802,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) @@ -820,7 +828,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 { @@ -831,6 +839,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 { @@ -986,11 +995,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)) @@ -1074,12 +1083,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...") @@ -1094,16 +1104,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() @@ -1114,14 +1124,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("") err2 := s.watchClient.RemoveWithPrefix("") if err2 != nil { @@ -1189,7 +1199,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. @@ -1246,7 +1256,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: @@ -1260,6 +1270,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 { diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index dc19a9abab03c..8b8d042b2a65f 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -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), diff --git a/internal/datacoord/session/indexnode_manager.go b/internal/datacoord/session/indexnode_manager.go index d0a2f7477c5e4..5a77f611698d8 100644 --- a/internal/datacoord/session/indexnode_manager.go +++ b/internal/datacoord/session/indexnode_manager.go @@ -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 { diff --git a/internal/datanode/compaction/clustering_compactor.go b/internal/datanode/compaction/clustering_compactor.go index 2a09d58436402..57087a319fa21 100644 --- a/internal/datanode/compaction/clustering_compactor.go +++ b/internal/datanode/compaction/clustering_compactor.go @@ -363,6 +363,7 @@ func (t *clusteringCompactionTask) getScalarAnalyzeResult(ctx context.Context) e 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) @@ -417,6 +418,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) @@ -881,6 +883,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) @@ -1309,6 +1312,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), diff --git a/internal/datanode/compaction/compactor_common.go b/internal/datanode/compaction/compactor_common.go index 560e761adfa5c..8fd26ebf6815a 100644 --- a/internal/datanode/compaction/compactor_common.go +++ b/internal/datanode/compaction/compactor_common.go @@ -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 diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index df4d671e461b7..9067cb1088c75 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -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() } diff --git a/internal/datanode/services.go b/internal/datanode/services.go index fa14b7d4182a5..825f63a14c1ad 100644 --- a/internal/datanode/services.go +++ b/internal/datanode/services.go @@ -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 diff --git a/internal/distributed/connection_manager.go b/internal/distributed/connection_manager.go index 6f7591c18013f..25b5b00d6901c 100644 --- a/internal/distributed/connection_manager.go +++ b/internal/distributed/connection_manager.go @@ -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 } diff --git a/internal/distributed/connection_manager_test.go b/internal/distributed/connection_manager_test.go index ae054f81a99e9..68569588c264f 100644 --- a/internal/distributed/connection_manager_test.go +++ b/internal/distributed/connection_manager_test.go @@ -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 { diff --git a/internal/distributed/datacoord/client/client.go b/internal/distributed/datacoord/client/client.go index bb095cdae30b0..8c5372244248b 100644 --- a/internal/distributed/datacoord/client/client.go +++ b/internal/distributed/datacoord/client/client.go @@ -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,14 +59,16 @@ 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 } + clientCtx := log.WithFields(ctx, zap.String("module", "DataCoordClient")) config := &Params.DataCoordGrpcClientCfg client := &Client{ grpcClient: grpcclient.NewClientBase[datapb.DataCoordClient](config, "milvus.proto.data.DataCoord"), sess: sess, + ctx: clientCtx, } client.grpcClient.SetRole(typeutil.DataCoordRole) client.grpcClient.SetGetAddrFunc(client.getDataCoordAddr) @@ -76,7 +79,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) @@ -90,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)) diff --git a/internal/distributed/datacoord/service.go b/internal/distributed/datacoord/service.go index 40a43841cd034..13dee4098e484 100644 --- a/internal/distributed/datacoord/service.go +++ b/internal/distributed/datacoord/service.go @@ -53,6 +53,7 @@ import ( "github.com/milvus-io/milvus/pkg/util/netutil" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tikv" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) // Server is the grpc server of datacoord @@ -75,7 +76,8 @@ type Server struct { // NewServer new data service grpc server func NewServer(ctx context.Context, factory dependency.Factory, opts ...datacoord.Option) (*Server, error) { - ctx1, cancel := context.WithCancel(ctx) + srvCtx := log.WithFields(ctx, zap.String("role", typeutil.DataCoordRole)) + ctx1, cancel := context.WithCancel(srvCtx) s := &Server{ ctx: ctx1, cancel: cancel, @@ -93,10 +95,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 +106,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 +226,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 +241,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 +264,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 +281,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 } diff --git a/internal/distributed/datanode/client/client.go b/internal/distributed/datanode/client/client.go index 5859e7ee33e5c..3001bf483ff2b 100644 --- a/internal/distributed/datanode/client/client.go +++ b/internal/distributed/datanode/client/client.go @@ -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,18 @@ 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 } + + clientCtx := log.WithFields(ctx, zap.String("module", "DataNodeClient")) config := &Params.DataNodeGrpcClientCfg client := &Client{ addr: addr, grpcClient: grpcclient.NewClientBase[datapb.DataNodeClient](config, "milvus.proto.data.DataNode"), sess: sess, serverID: serverID, + ctx: clientCtx, } // node shall specify node id client.grpcClient.SetRole(fmt.Sprintf("%s-%d", typeutil.DataNodeRole, serverID)) @@ -77,7 +81,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) diff --git a/internal/distributed/datanode/service.go b/internal/distributed/datanode/service.go index 2c08267b7bfe2..7d615c8a1d03d 100644 --- a/internal/distributed/datanode/service.go +++ b/internal/distributed/datanode/service.go @@ -50,6 +50,7 @@ import ( "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/netutil" "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) type Server struct { @@ -71,7 +72,8 @@ type Server struct { // NewServer new DataNode grpc server func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) { - ctx1, cancel := context.WithCancel(ctx) + srvCtx := log.WithFields(ctx, zap.String("role", typeutil.DataNodeRole)) + ctx1, cancel := context.WithCancel(srvCtx) s := &Server{ ctx: ctx1, cancel: cancel, @@ -96,10 +98,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 +168,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 +191,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 +222,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 +236,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 +257,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 +272,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 +291,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 +318,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 diff --git a/internal/distributed/indexnode/client/client.go b/internal/distributed/indexnode/client/client.go index 7387bdb1385e3..343a17416f390 100644 --- a/internal/distributed/indexnode/client/client.go +++ b/internal/distributed/indexnode/client/client.go @@ -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) diff --git a/internal/distributed/indexnode/service.go b/internal/distributed/indexnode/service.go index 8108615450441..a174979e52161 100644 --- a/internal/distributed/indexnode/service.go +++ b/internal/distributed/indexnode/service.go @@ -46,6 +46,7 @@ import ( "github.com/milvus-io/milvus/pkg/util/logutil" "github.com/milvus-io/milvus/pkg/util/netutil" "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) // Server is the grpc wrapper of IndexNode. @@ -71,11 +72,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 +88,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 +156,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 +206,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 +222,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() { @@ -315,7 +318,8 @@ func (s *Server) DropJobsV2(ctx context.Context, request *workerpb.DropJobsV2Req // NewServer create a new IndexNode grpc server. func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) { - ctx1, cancel := context.WithCancel(ctx) + srvCtx := log.WithFields(ctx, zap.String("role", typeutil.IndexNodeRole)) + ctx1, cancel := context.WithCancel(srvCtx) node := indexnode.NewIndexNode(ctx1, factory) return &Server{ diff --git a/internal/distributed/proxy/client/client.go b/internal/distributed/proxy/client/client.go index ffbc91ab20ca3..0daf804a7ea20 100644 --- a/internal/distributed/proxy/client/client.go +++ b/internal/distributed/proxy/client/client.go @@ -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) diff --git a/internal/distributed/proxy/listener_manager.go b/internal/distributed/proxy/listener_manager.go index dabe1e5935886..8d79d546a5794 100644 --- a/internal/distributed/proxy/listener_manager.go +++ b/internal/distributed/proxy/listener_manager.go @@ -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") diff --git a/internal/distributed/proxy/service.go b/internal/distributed/proxy/service.go index 33c21e505f2b0..5c9bd3f283fc2 100644 --- a/internal/distributed/proxy/service.go +++ b/internal/distributed/proxy/service.go @@ -73,6 +73,7 @@ import ( "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) var ( @@ -113,8 +114,9 @@ type Server struct { // NewServer create a Proxy server. func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) { + srvCtx := log.WithFields(ctx, zap.String("role", typeutil.ProxyRole)) server := &Server{ - ctx: ctx, + ctx: srvCtx, } var err error @@ -129,7 +131,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 } @@ -141,7 +143,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)) } c.AbortWithStatusJSON(http.StatusUnauthorized, gin.H{mhttp.HTTPReturnCode: merr.Code(merr.ErrNeedAuthenticate), mhttp.HTTPReturnMessage: merr.ErrNeedAuthenticate.Error()}) } @@ -193,11 +195,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) { @@ -223,6 +225,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)) @@ -376,6 +380,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)) @@ -389,7 +394,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 } @@ -399,6 +404,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)) @@ -418,6 +424,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") @@ -551,6 +558,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 @@ -577,7 +585,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()), @@ -601,17 +609,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) } @@ -626,7 +634,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 } diff --git a/internal/distributed/querycoord/client/client.go b/internal/distributed/querycoord/client/client.go index 867d73e7f6a87..fd385790b85e2 100644 --- a/internal/distributed/querycoord/client/client.go +++ b/internal/distributed/querycoord/client/client.go @@ -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) @@ -80,16 +83,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), ) diff --git a/internal/distributed/querycoord/service.go b/internal/distributed/querycoord/service.go index 3273e6de281a3..0d28ac22f780a 100644 --- a/internal/distributed/querycoord/service.go +++ b/internal/distributed/querycoord/service.go @@ -50,6 +50,7 @@ import ( "github.com/milvus-io/milvus/pkg/util/netutil" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tikv" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) // Server is the grpc server of QueryCoord. @@ -77,7 +78,8 @@ type Server struct { // NewServer create a new QueryCoord grpc server. func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) { - ctx1, cancel := context.WithCancel(ctx) + srvCtx := log.WithFields(ctx, zap.String("role", typeutil.QueryCoordRole)) + ctx1, cancel := context.WithCancel(srvCtx) svr, err := qc.NewQueryCoord(ctx1) if err != nil { cancel() @@ -99,10 +101,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 } @@ -112,12 +114,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 } @@ -125,6 +127,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 @@ -275,7 +278,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())) } diff --git a/internal/distributed/querynode/client/client.go b/internal/distributed/querynode/client/client.go index 7dfe4dc8be62a..abc7c839dd46e 100644 --- a/internal/distributed/querynode/client/client.go +++ b/internal/distributed/querynode/client/client.go @@ -46,6 +46,7 @@ type Client struct { addr string sess *sessionutil.Session nodeID int64 + ctx context.Context } // NewClient creates a new QueryNode client. @@ -56,15 +57,17 @@ 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 } + clientCtx := log.WithFields(ctx, zap.String("module", "QueryNodeClient")) config := ¶mtable.Get().QueryNodeGrpcClientCfg client := &Client{ addr: addr, grpcClient: grpcclient.NewClientBase[querypb.QueryNodeClient](config, "milvus.proto.query.QueryNode"), sess: sess, nodeID: nodeID, + ctx: clientCtx, } // node shall specify node id client.grpcClient.SetRole(fmt.Sprintf("%s-%d", typeutil.QueryNodeRole, nodeID)) @@ -77,7 +80,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) diff --git a/internal/distributed/querynode/service.go b/internal/distributed/querynode/service.go index 00e898de70b97..b0b8cbb4f333a 100644 --- a/internal/distributed/querynode/service.go +++ b/internal/distributed/querynode/service.go @@ -78,7 +78,8 @@ func (s *Server) GetQueryNode() types.QueryNodeComponent { // NewServer create a new QueryNode grpc server. func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) { - ctx1, cancel := context.WithCancel(ctx) + srvCtx := log.WithFields(ctx, zap.String("role", typeutil.QueryNodeRole)) + ctx1, cancel := context.WithCancel(srvCtx) s := &Server{ ctx: ctx1, @@ -95,11 +96,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 +110,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 +153,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 +218,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 +228,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 +251,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 { diff --git a/internal/distributed/rootcoord/client/client.go b/internal/distributed/rootcoord/client/client.go index 6d0c871042366..beb9c5a35be18 100644 --- a/internal/distributed/rootcoord/client/client.go +++ b/internal/distributed/rootcoord/client/client.go @@ -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,15 @@ 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 } + clientCtx := log.WithFields(ctx, zap.String("module", "RootCoordClient")) config := &Params.RootCoordGrpcClientCfg client := &Client{ grpcClient: grpcclient.NewClientBase[rootcoordpb.RootCoordClient](config, "milvus.proto.rootcoord.RootCoord"), sess: sess, + ctx: clientCtx, } client.grpcClient.SetRole(typeutil.RootCoordRole) client.grpcClient.SetGetAddrFunc(client.getRootCoordAddr) @@ -75,7 +78,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) @@ -89,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 { diff --git a/internal/distributed/rootcoord/service.go b/internal/distributed/rootcoord/service.go index c1e0241a5eec5..1ad42e8a41820 100644 --- a/internal/distributed/rootcoord/service.go +++ b/internal/distributed/rootcoord/service.go @@ -50,6 +50,7 @@ import ( "github.com/milvus-io/milvus/pkg/util/netutil" "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/tikv" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) // Server grpc wrapper @@ -126,7 +127,8 @@ func (s *Server) ListAliases(ctx context.Context, request *milvuspb.ListAliasesR // NewServer create a new RootCoord grpc server. func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) { - ctx1, cancel := context.WithCancel(ctx) + srvCtx := log.WithFields(ctx, zap.String("role", typeutil.RootCoordRole)) + ctx1, cancel := context.WithCancel(srvCtx) s := &Server{ ctx: ctx1, cancel: cancel, @@ -142,6 +144,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 +168,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 +182,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 +263,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 +309,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 +325,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() { diff --git a/internal/distributed/streamingnode/service.go b/internal/distributed/streamingnode/service.go index 3f17c1aeb4c9e..5a34ebfed76b8 100644 --- a/internal/distributed/streamingnode/service.go +++ b/internal/distributed/streamingnode/service.go @@ -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,16 @@ 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) { + srvCtx := log.WithFields(ctx, zap.String("role", typeutil.StreamingNodeRole)) + ctx1, cancel := context.WithCancel(srvCtx) return &Server{ stopOnce: sync.Once{}, factory: f, grpcServerChan: make(chan struct{}), componentState: componentutil.NewComponentStateService(typeutil.StreamingNodeRole), + ctx: ctx1, + cancel: cancel, }, nil } @@ -102,11 +109,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 +122,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 +143,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 +183,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 +195,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 +211,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 +236,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 +256,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 +266,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 +301,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 +345,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 +406,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) }) } diff --git a/internal/distributed/utils/util.go b/internal/distributed/utils/util.go index a61b6ff2b0905..8717c1d145f8a 100644 --- a/internal/distributed/utils/util.go +++ b/internal/distributed/utils/util.go @@ -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() diff --git a/internal/flushcommon/io/binlog_io.go b/internal/flushcommon/io/binlog_io.go index e1fed730b3061..73eb866bb5b50 100644 --- a/internal/flushcommon/io/binlog_io.go +++ b/internal/flushcommon/io/binlog_io.go @@ -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) diff --git a/internal/flushcommon/pipeline/flow_graph_dd_node.go b/internal/flushcommon/pipeline/flow_graph_dd_node.go index ddfb5ca81fe0e..fd5a0b8a78a48 100644 --- a/internal/flushcommon/pipeline/flow_graph_dd_node.go +++ b/internal/flushcommon/pipeline/flow_graph_dd_node.go @@ -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())) diff --git a/internal/flushcommon/pipeline/flow_graph_time_tick_node.go b/internal/flushcommon/pipeline/flow_graph_time_tick_node.go index e1cb4f64dbfc4..72f7d4b84ba36 100644 --- a/internal/flushcommon/pipeline/flow_graph_time_tick_node.go +++ b/internal/flushcommon/pipeline/flow_graph_time_tick_node.go @@ -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)) diff --git a/internal/flushcommon/pipeline/testutils_test.go b/internal/flushcommon/pipeline/testutils_test.go index 7606bee2152e8..7a460763f71e7 100644 --- a/internal/flushcommon/pipeline/testutils_test.go +++ b/internal/flushcommon/pipeline/testutils_test.go @@ -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 } diff --git a/internal/indexnode/indexnode.go b/internal/indexnode/indexnode.go index abaf49a83d111..31617d18276ed 100644 --- a/internal/indexnode/indexnode.go +++ b/internal/indexnode/indexnode.go @@ -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 diff --git a/internal/indexnode/task_scheduler.go b/internal/indexnode/task_scheduler.go index 3f5c986149cad..cccc85595e052 100644 --- a/internal/indexnode/task_scheduler.go +++ b/internal/indexnode/task_scheduler.go @@ -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 { diff --git a/internal/indexnode/task_stats.go b/internal/indexnode/task_stats.go index 336e6354a6511..0ace1334a62a6 100644 --- a/internal/indexnode/task_stats.go +++ b/internal/indexnode/task_stats.go @@ -127,13 +127,13 @@ func (st *statsTask) PreExecute(ctx context.Context) error { if err := binlog.DecompressBinLog(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.DecompressBinLog(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 } diff --git a/internal/indexnode/taskinfo_ops.go b/internal/indexnode/taskinfo_ops.go index 23c71983fe50c..8a0431f76bf42 100644 --- a/internal/indexnode/taskinfo_ops.go +++ b/internal/indexnode/taskinfo_ops.go @@ -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 diff --git a/internal/kv/etcd/etcd_kv.go b/internal/kv/etcd/etcd_kv.go index 5003d0fae4b1f..9259ad1488464 100644 --- a/internal/kv/etcd/etcd_kv.go +++ b/internal/kv/etcd/etcd_kv.go @@ -63,7 +63,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. diff --git a/internal/metastore/kv/datacoord/kv_catalog.go b/internal/metastore/kv/datacoord/kv_catalog.go index 2233029ee30f6..2429a4b331254 100644 --- a/internal/metastore/kv/datacoord/kv_catalog.go +++ b/internal/metastore/kv/datacoord/kv_catalog.go @@ -271,14 +271,14 @@ func (kc *Catalog) AddSegment(ctx context.Context, segment *datapb.SegmentInfo) func (kc *Catalog) LoadFromSegmentPath(colID, partID, segID typeutil.UniqueID) (*datapb.SegmentInfo, error) { v, err := kc.MetaKv.Load(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 } @@ -327,7 +327,7 @@ func (kc *Catalog) AlterSegments(ctx context.Context, segments []*datapb.Segment maps.Copy(kvs, binlogKvs) } - return kc.SaveByBatch(kvs) + return kc.SaveByBatch(ctx, kvs) } func (kc *Catalog) handleDroppedSegment(segment *datapb.SegmentInfo) (kvs map[string]string, err error) { @@ -346,13 +346,13 @@ func (kc *Catalog) handleDroppedSegment(segment *datapb.SegmentInfo) (kvs map[st return } -func (kc *Catalog) SaveByBatch(kvs map[string]string) error { +func (kc *Catalog) SaveByBatch(ctx context.Context, kvs map[string]string) error { saveFn := func(partialKvs map[string]string) error { return kc.MetaKv.MultiSave(partialKvs) } 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 @@ -428,10 +428,10 @@ func (kc *Catalog) MarkChannelAdded(ctx context.Context, channel string) error { key := buildChannelRemovePath(channel) err := kc.MetaKv.Save(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 } @@ -439,10 +439,10 @@ func (kc *Catalog) MarkChannelDeleted(ctx context.Context, channel string) error key := buildChannelRemovePath(channel) err := kc.MetaKv.Save(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 } @@ -464,7 +464,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(key) } @@ -474,7 +474,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), "/") @@ -510,7 +510,7 @@ func (kc *Catalog) SaveChannelCheckpoints(ctx context.Context, positions []*msgp } kvs[k] = string(v) } - return kc.SaveByBatch(kvs) + return kc.SaveByBatch(ctx, kvs) } func (kc *Catalog) DropChannelCheckpoint(ctx context.Context, vChannel string) error { @@ -560,7 +560,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 } @@ -606,7 +606,7 @@ func (kc *Catalog) DropIndex(ctx context.Context, collID typeutil.UniqueID, drop err := kc.MetaKv.Remove(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 } @@ -623,7 +623,7 @@ func (kc *Catalog) CreateSegmentIndex(ctx context.Context, segIdx *model.Segment } err = kc.MetaKv.Save(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 } @@ -636,7 +636,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 } @@ -670,7 +670,7 @@ func (kc *Catalog) DropSegmentIndex(ctx context.Context, collID, partID, segID, err := kc.MetaKv.Remove(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 } @@ -824,7 +824,7 @@ func (kc *Catalog) SaveCompactionTask(ctx context.Context, coll *datapb.Compacti } kvs := make(map[string]string) kvs[k] = v - return kc.SaveByBatch(kvs) + return kc.SaveByBatch(ctx, kvs) } func (kc *Catalog) DropCompactionTask(ctx context.Context, task *datapb.CompactionTask) error { @@ -903,7 +903,7 @@ func (kc *Catalog) SavePartitionStatsInfo(ctx context.Context, coll *datapb.Part } kvs := make(map[string]string) kvs[k] = v - return kc.SaveByBatch(kvs) + return kc.SaveByBatch(ctx, kvs) } func (kc *Catalog) DropPartitionStatsInfo(ctx context.Context, info *datapb.PartitionStatsInfo) error { diff --git a/internal/metastore/kv/querycoord/kv_catalog.go b/internal/metastore/kv/querycoord/kv_catalog.go index f121462e47848..12c1e62816a11 100644 --- a/internal/metastore/kv/querycoord/kv_catalog.go +++ b/internal/metastore/kv/querycoord/kv_catalog.go @@ -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" ) diff --git a/internal/metastore/kv/rootcoord/kv_catalog.go b/internal/metastore/kv/rootcoord/kv_catalog.go index 7bc1e4e170496..b26b2d47fb99a 100644 --- a/internal/metastore/kv/rootcoord/kv_catalog.go +++ b/internal/metastore/kv/rootcoord/kv_catalog.go @@ -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(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 } @@ -561,9 +561,9 @@ func (kc *Catalog) GetCredential(ctx context.Context, username string) (*model.C v, err := kc.Txn.Load(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 } @@ -732,7 +732,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) @@ -747,7 +747,7 @@ func (kc *Catalog) DropCredential(ctx context.Context, username string) error { } err = kc.Txn.MultiRemove(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 } @@ -765,7 +765,7 @@ func (kc *Catalog) GetCollectionByName(ctx context.Context, dbID int64, collecti prefix := getDatabasePrefix(dbID) _, vals, err := kc.Snapshot.LoadWithPrefix(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 } @@ -773,7 +773,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 { @@ -789,7 +789,7 @@ func (kc *Catalog) ListCollections(ctx context.Context, dbID int64, ts typeutil. prefix := getDatabasePrefix(dbID) _, vals, err := kc.Snapshot.LoadWithPrefix(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)) @@ -802,17 +802,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 } @@ -919,7 +919,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(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 } @@ -927,13 +927,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 @@ -942,24 +942,25 @@ func (kc *Catalog) ListCredentialsWithPasswd(ctx context.Context) (map[string]st return users, nil } -func (kc *Catalog) save(k string) error { +func (kc *Catalog) save(ctx context.Context, k string) error { var err error if _, err = kc.Txn.Load(k); err != nil && !errors.Is(err, merr.ErrIoKeyNotFound) { 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(k, "") } -func (kc *Catalog) remove(k string) error { +func (kc *Catalog) remove(ctx context.Context, k string) error { var err error if _, err = kc.Txn.Load(k); err != nil && !errors.Is(err, merr.ErrIoKeyNotFound) { 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(k) @@ -967,9 +968,9 @@ func (kc *Catalog) remove(k string) error { func (kc *Catalog) CreateRole(ctx context.Context, tenant string, entity *milvuspb.RoleEntity) error { k := funcutil.HandleTenantForEtcdKey(RolePrefix, tenant, entity.Name) - err := kc.save(k) + 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 } @@ -978,7 +979,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 } @@ -995,7 +996,7 @@ func (kc *Catalog) DropRole(ctx context.Context, tenant string, roleName string) err = kc.Txn.MultiRemove(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 @@ -1005,14 +1006,14 @@ func (kc *Catalog) AlterUserRole(ctx context.Context, tenant string, userEntity k := funcutil.HandleTenantForEtcdKey(RoleMappingPrefix, tenant, fmt.Sprintf("%s/%s", userEntity.Name, roleEntity.Name)) var err error if operateType == milvuspb.OperateUserRoleType_AddUserToRole { - err = kc.save(k) + 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(k) + 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) @@ -1028,14 +1029,14 @@ func (kc *Catalog) ListRole(ctx context.Context, tenant string, entity *milvuspb roleMappingKey := funcutil.HandleTenantForEtcdKey(RoleMappingPrefix, tenant, "") keys, _, err := kc.Txn.LoadWithPrefix(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] @@ -1059,13 +1060,13 @@ func (kc *Catalog) ListRole(ctx context.Context, tenant string, entity *milvuspb roleKey := funcutil.HandleTenantForEtcdKey(RolePrefix, tenant, "") keys, _, err := kc.Txn.LoadWithPrefix(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]) @@ -1077,7 +1078,7 @@ func (kc *Catalog) ListRole(ctx context.Context, tenant string, entity *milvuspb roleKey := funcutil.HandleTenantForEtcdKey(RolePrefix, tenant, entity.Name) _, err := kc.Txn.Load(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) @@ -1086,18 +1087,18 @@ func (kc *Catalog) ListRole(ctx context.Context, tenant string, entity *milvuspb return results, nil } -func (kc *Catalog) getRolesByUsername(tenant string, username string) ([]string, error) { +func (kc *Catalog) getRolesByUsername(ctx context.Context, tenant string, username string) ([]string, error) { var roles []string k := funcutil.HandleTenantForEtcdKey(RoleMappingPrefix, tenant, username) keys, _, err := kc.Txn.LoadWithPrefix(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]) @@ -1106,14 +1107,14 @@ func (kc *Catalog) getRolesByUsername(tenant string, username string) ([]string, } // getUserResult get the user result by the username. And never return the error because the error means the user isn't added to a role. -func (kc *Catalog) getUserResult(tenant string, username string, includeRoleInfo bool) (*milvuspb.UserResult, error) { +func (kc *Catalog) getUserResult(ctx context.Context, tenant string, username string, includeRoleInfo bool) (*milvuspb.UserResult, error) { result := &milvuspb.UserResult{User: &milvuspb.UserEntity{Name: username}} if !includeRoleInfo { return result, nil } - roleNames, err := kc.getRolesByUsername(tenant, username) + 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 @@ -1132,7 +1133,7 @@ func (kc *Catalog) ListUser(ctx context.Context, tenant string, entity *milvuspb ) appendUserResult := func(username string) error { - result, err := kc.getUserResult(tenant, username, includeRoleInfo) + result, err := kc.getUserResult(ctx, tenant, username, includeRoleInfo) if err != nil { return err } @@ -1184,7 +1185,7 @@ func (kc *Catalog) AlterGrant(ctx context.Context, tenant string, entity *milvus if v, err = kc.Txn.Load(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)) @@ -1198,7 +1199,7 @@ func (kc *Catalog) AlterGrant(ctx context.Context, tenant string, entity *milvus idStr = crypto.MD5(k) err = kc.Txn.Save(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 } } @@ -1206,18 +1207,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(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(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 } @@ -1225,7 +1226,7 @@ func (kc *Catalog) AlterGrant(ctx context.Context, tenant string, entity *milvus } if funcutil.IsRevoke(operateType) { if err = kc.Txn.Remove(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 @@ -1246,13 +1247,13 @@ func (kc *Catalog) ListGrant(ctx context.Context, tenant string, entity *milvusp granteeIDKey := funcutil.HandleTenantForEtcdKey(GranteeIDPrefix, tenant, v) keys, values, err := kc.Txn.LoadWithPrefix(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]) @@ -1296,7 +1297,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(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)) @@ -1307,13 +1308,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(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]) @@ -1338,7 +1339,7 @@ func (kc *Catalog) DeleteGrant(ctx context.Context, tenant string, role *milvusp // the values are the grantee id list _, values, err := kc.Txn.LoadWithPrefix(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 { @@ -1347,7 +1348,7 @@ func (kc *Catalog) DeleteGrant(ctx context.Context, tenant string, role *milvusp } if err = kc.Txn.MultiSaveAndRemoveWithPrefix(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 } @@ -1357,26 +1358,26 @@ func (kc *Catalog) ListPolicy(ctx context.Context, tenant string) ([]string, err granteeKey := funcutil.HandleTenantForEtcdKey(GranteePrefix, tenant, "") keys, values, err := kc.Txn.LoadWithPrefix(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(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]) @@ -1392,14 +1393,14 @@ func (kc *Catalog) ListUserRole(ctx context.Context, tenant string) ([]string, e k := funcutil.HandleTenantForEtcdKey(RoleMappingPrefix, tenant, "") keys, _, err := kc.Txn.LoadWithPrefix(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])) @@ -1477,12 +1478,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)) } } @@ -1490,7 +1491,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)) } } @@ -1498,7 +1499,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)) } } @@ -1506,7 +1507,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)) } } } @@ -1520,7 +1521,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 } @@ -1539,7 +1540,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 } @@ -1563,7 +1564,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 } @@ -1582,7 +1583,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 } @@ -1618,13 +1619,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 @@ -1634,7 +1635,7 @@ func (kc *Catalog) DropPrivilegeGroup(ctx context.Context, groupName string) err k := BuildPrivilegeGroupkey(groupName) err := kc.Txn.Remove(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 @@ -1648,11 +1649,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(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 @@ -1661,7 +1662,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(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)) @@ -1669,7 +1670,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) diff --git a/internal/metastore/kv/rootcoord/kv_catalog_test.go b/internal/metastore/kv/rootcoord/kv_catalog_test.go index c0dcc6368b880..71b0c96624f3b 100644 --- a/internal/metastore/kv/rootcoord/kv_catalog_test.go +++ b/internal/metastore/kv/rootcoord/kv_catalog_test.go @@ -1657,7 +1657,7 @@ func TestRBAC_Role(t *testing.T) { } for _, test := range tests { t.Run(test.description, func(t *testing.T) { - err := c.remove(test.key) + err := c.remove(ctx, test.key) if test.isValid { assert.NoError(t, err) } else { @@ -1703,7 +1703,7 @@ func TestRBAC_Role(t *testing.T) { for _, test := range tests { t.Run(test.description, func(t *testing.T) { - err := c.save(test.key) + err := c.save(ctx, test.key) if test.isValid { assert.NoError(t, err) } else { @@ -2074,7 +2074,7 @@ func TestRBAC_Role(t *testing.T) { for _, test := range tests { t.Run(test.description, func(t *testing.T) { - res, err := c.getUserResult(tenant, test.user, test.includeRoleInfo) + res, err := c.getUserResult(ctx, tenant, test.user, test.includeRoleInfo) assert.Equal(t, test.user, res.GetUser().GetName()) diff --git a/internal/metastore/kv/rootcoord/suffix_snapshot.go b/internal/metastore/kv/rootcoord/suffix_snapshot.go index 5a625d1347450..875469e509eef 100644 --- a/internal/metastore/kv/rootcoord/suffix_snapshot.go +++ b/internal/metastore/kv/rootcoord/suffix_snapshot.go @@ -18,6 +18,7 @@ package rootcoord import ( "bytes" + "context" "fmt" "path" "sort" @@ -604,6 +605,7 @@ func (ss *SuffixSnapshot) Close() { // startBackgroundGC the data will clean up if key ts!=0 and expired func (ss *SuffixSnapshot) startBackgroundGC() { + log := log.Ctx(context.TODO()) log.Debug("suffix snapshot GC goroutine start!") ticker := time.NewTicker(60 * time.Minute) defer ticker.Stop() diff --git a/internal/mocks/util/mock_segcore/mock_data.go b/internal/mocks/util/mock_segcore/mock_data.go index 95257bc987cca..0d7ed4b137626 100644 --- a/internal/mocks/util/mock_segcore/mock_data.go +++ b/internal/mocks/util/mock_segcore/mock_data.go @@ -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{ diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index da06cf4d4abdc..5c56d09afbed2 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -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 @@ -250,8 +250,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), ) @@ -319,8 +318,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), ) @@ -387,8 +385,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), ) @@ -3681,7 +3678,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), @@ -5909,14 +5906,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 @@ -5976,14 +5973,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 @@ -6359,6 +6356,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 { @@ -6388,10 +6388,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()), ) diff --git a/internal/proxy/meta_cache.go b/internal/proxy/meta_cache.go index 57543a2abd64a..b286a834715f5 100644 --- a/internal/proxy/meta_cache.go +++ b/internal/proxy/meta_cache.go @@ -600,7 +600,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 @@ -819,7 +819,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) []string { @@ -834,7 +834,7 @@ func (m *MetaCache) RemoveCollectionsByID(ctx context.Context, collectionID Uniq } } } - log.Debug("remove collection by id", zap.Int64("id", collectionID), zap.Strings("collection", collNames)) + log.Ctx(ctx).Debug("remove collection by id", zap.Int64("id", collectionID), zap.Strings("collection", collNames)) return collNames } @@ -1155,7 +1155,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) diff --git a/internal/proxy/msg_pack.go b/internal/proxy/msg_pack.go index 96fe24eb5ead3..fb079a08d8b0d 100644 --- a/internal/proxy/msg_pack.go +++ b/internal/proxy/msg_pack.go @@ -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 diff --git a/internal/proxy/proxy.go b/internal/proxy/proxy.go index 8486526dd620e..29b600a457152 100644 --- a/internal/proxy/proxy.go +++ b/internal/proxy/proxy.go @@ -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)) diff --git a/internal/proxy/reScorer.go b/internal/proxy/reScorer.go index fc6776f4bc796..53d16aef9b588 100644 --- a/internal/proxy/reScorer.go +++ b/internal/proxy/reScorer.go @@ -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 { diff --git a/internal/proxy/reScorer_test.go b/internal/proxy/reScorer_test.go index e824099acaecd..7478a67693893 100644 --- a/internal/proxy/reScorer_test.go +++ b/internal/proxy/reScorer_test.go @@ -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()) diff --git a/internal/proxy/search_reduce_util.go b/internal/proxy/search_reduce_util.go index 15029f1e01256..d398a2dd24689 100644 --- a/internal/proxy/search_reduce_util.go +++ b/internal/proxy/search_reduce_util.go @@ -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 } diff --git a/internal/proxy/segment.go b/internal/proxy/segment.go index 3c1d8321d2af2..d6cdcb97e0c9a 100644 --- a/internal/proxy/segment.go +++ b/internal/proxy/segment.go @@ -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 diff --git a/internal/proxy/simple_rate_limiter.go b/internal/proxy/simple_rate_limiter.go index 10ca38dcf3f71..4881c2a83004f 100644 --- a/internal/proxy/simple_rate_limiter.go +++ b/internal/proxy/simple_rate_limiter.go @@ -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()), diff --git a/internal/proxy/task.go b/internal/proxy/task.go index 07b6cf6f864e1..c81aa47f969e9 100644 --- a/internal/proxy/task.go +++ b/internal/proxy/task.go @@ -208,7 +208,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() { @@ -263,7 +263,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())) } @@ -271,7 +271,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() { @@ -288,7 +288,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)) } @@ -358,17 +358,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 } @@ -808,7 +808,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 } @@ -854,14 +854,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 } @@ -953,7 +953,7 @@ func hasLazyLoadProp(props ...*commonpb.KeyValuePair) bool { return false } -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 @@ -974,7 +974,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 } @@ -1001,7 +1001,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 } @@ -1011,7 +1011,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), @@ -1403,7 +1403,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 } @@ -1416,7 +1416,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 } @@ -1447,13 +1447,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 } @@ -1847,7 +1847,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) } @@ -2258,7 +2258,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)) diff --git a/internal/proxy/task_database.go b/internal/proxy/task_database.go index 5ae997aab2d87..b44678eca7e14 100644 --- a/internal/proxy/task_database.go +++ b/internal/proxy/task_database.go @@ -363,12 +363,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 } diff --git a/internal/proxy/task_delete.go b/internal/proxy/task_delete.go index 1d9df1c59f8f6..13c3ad1b387c5 100644 --- a/internal/proxy/task_delete.go +++ b/internal/proxy/task_delete.go @@ -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), @@ -393,7 +393,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 } } @@ -421,7 +421,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 } @@ -535,7 +535,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 @@ -543,21 +543,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() @@ -590,26 +590,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 } diff --git a/internal/proxy/task_delete_streaming.go b/internal/proxy/task_delete_streaming.go index 25facf4d71c43..a46a02e64896b 100644 --- a/internal/proxy/task_delete_streaming.go +++ b/internal/proxy/task_delete_streaming.go @@ -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() diff --git a/internal/proxy/task_flush_streaming.go b/internal/proxy/task_flush_streaming.go index e0cc8625adeb1..8f122ee2e4fc9 100644 --- a/internal/proxy/task_flush_streaming.go +++ b/internal/proxy/task_flush_streaming.go @@ -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{ diff --git a/internal/proxy/task_index.go b/internal/proxy/task_index.go index 914b752807980..57777814b8e89 100644 --- a/internal/proxy/task_index.go +++ b/internal/proxy/task_index.go @@ -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 } @@ -744,7 +744,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) } @@ -766,7 +766,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() @@ -863,7 +863,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 @@ -879,7 +879,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() diff --git a/internal/proxy/task_index_test.go b/internal/proxy/task_index_test.go index 554856e9871f6..780dea5041f4f 100644 --- a/internal/proxy/task_index_test.go +++ b/internal/proxy/task_index_test.go @@ -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) }) } diff --git a/internal/proxy/task_insert.go b/internal/proxy/task_insert.go index 620b469d2fa96..2ce037d018079 100644 --- a/internal/proxy/task_insert.go +++ b/internal/proxy/task_insert.go @@ -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 diff --git a/internal/proxy/task_insert_streaming.go b/internal/proxy/task_insert_streaming.go index 0a99d9816844e..6116839b0e00f 100644 --- a/internal/proxy/task_insert_streaming.go +++ b/internal/proxy/task_insert_streaming.go @@ -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 diff --git a/internal/proxy/task_policies.go b/internal/proxy/task_policies.go index f3a047bd38385..1c49a08ce621b 100644 --- a/internal/proxy/task_policies.go +++ b/internal/proxy/task_policies.go @@ -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 } diff --git a/internal/proxy/task_query.go b/internal/proxy/task_query.go index 3bbdaffd9ae29..910227e796135 100644 --- a/internal/proxy/task_query.go +++ b/internal/proxy/task_query.go @@ -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 } diff --git a/internal/proxy/task_scheduler.go b/internal/proxy/task_scheduler.go index 878363c94e161..da1a2ededbdba 100644 --- a/internal/proxy/task_scheduler.go +++ b/internal/proxy/task_scheduler.go @@ -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 ...") } } } diff --git a/internal/proxy/task_search.go b/internal/proxy/task_search.go index 3dc48cfe9503c..550bc393c335c 100644 --- a/internal/proxy/task_search.go +++ b/internal/proxy/task_search.go @@ -99,14 +99,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 } @@ -320,20 +320,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) @@ -425,7 +425,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 @@ -526,12 +526,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 @@ -540,13 +540,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 } @@ -554,7 +554,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 @@ -597,6 +597,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 { @@ -985,7 +986,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 @@ -1007,7 +1008,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)) { diff --git a/internal/proxy/task_search_test.go b/internal/proxy/task_search_test.go index bab8ee02bd436..a0c5042376054 100644 --- a/internal/proxy/task_search_test.go +++ b/internal/proxy/task_search_test.go @@ -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)) } diff --git a/internal/proxy/task_statistic.go b/internal/proxy/task_statistic.go index 09e43b64df506..49d176d002702 100644 --- a/internal/proxy/task_statistic.go +++ b/internal/proxy/task_statistic.go @@ -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) diff --git a/internal/proxy/task_test.go b/internal/proxy/task_test.go index 1f74590c16bcc..0f0c0b6827964 100644 --- a/internal/proxy/task_test.go +++ b/internal/proxy/task_test.go @@ -2356,7 +2356,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) { @@ -2366,7 +2366,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) { @@ -2385,7 +2385,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) { @@ -2402,7 +2402,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)) }) } diff --git a/internal/proxy/task_upsert.go b/internal/proxy/task_upsert.go index 3ca4853fa9f53..26c7b6df163ce 100644 --- a/internal/proxy/task_upsert.go +++ b/internal/proxy/task_upsert.go @@ -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 } diff --git a/internal/proxy/util.go b/internal/proxy/util.go index 1bf7d0fbb546d..ff50806427afe 100644 --- a/internal/proxy/util.go +++ b/internal/proxy/util.go @@ -1211,7 +1211,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 } @@ -1223,13 +1223,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 } @@ -1641,7 +1641,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), ) @@ -1650,7 +1650,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 @@ -1695,7 +1695,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)) @@ -1705,7 +1705,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)) @@ -1860,7 +1860,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() diff --git a/internal/querycoordv2/checkers/leader_checker.go b/internal/querycoordv2/checkers/leader_checker.go index f95c376c3cd68..320ac743e8073 100644 --- a/internal/querycoordv2/checkers/leader_checker.go +++ b/internal/querycoordv2/checkers/leader_checker.go @@ -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())) } diff --git a/internal/querycoordv2/handlers.go b/internal/querycoordv2/handlers.go index d770edc1e6fef..a3f12f281381b 100644 --- a/internal/querycoordv2/handlers.go +++ b/internal/querycoordv2/handlers.go @@ -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, @@ -206,7 +206,7 @@ func (s *Server) balanceChannels(ctx context.Context, releaseAction := task.NewChannelAction(plan.From, task.ActionTypeReduce, plan.Channel.GetChannelName()) actions = append(actions, releaseAction) } - t, err := task.NewChannelTask(s.ctx, + t, err := task.NewChannelTask(ctx, Params.QueryCoordCfg.ChannelTaskTimeout.GetAsDuration(time.Millisecond), utils.ManualBalance, collectionID, diff --git a/internal/querycoordv2/meta/coordinator_broker.go b/internal/querycoordv2/meta/coordinator_broker.go index e6ab9a104f821..cc84db836bbf3 100644 --- a/internal/querycoordv2/meta/coordinator_broker.go +++ b/internal/querycoordv2/meta/coordinator_broker.go @@ -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)) diff --git a/internal/querycoordv2/meta/target_manager.go b/internal/querycoordv2/meta/target_manager.go index 68f5b8cf7ba55..eb40e9281f95f 100644 --- a/internal/querycoordv2/meta/target_manager.go +++ b/internal/querycoordv2/meta/target_manager.go @@ -138,6 +138,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(context.TODO(), func() (bool, error) { diff --git a/internal/querycoordv2/mocks/querynode.go b/internal/querycoordv2/mocks/querynode.go index c8fc5e835fc11..1f05aa4b5fd41 100644 --- a/internal/querycoordv2/mocks/querynode.go +++ b/internal/querycoordv2/mocks/querynode.go @@ -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)) diff --git a/internal/querycoordv2/observers/resource_observer.go b/internal/querycoordv2/observers/resource_observer.go index ae701dc52ef00..c06abde495160 100644 --- a/internal/querycoordv2/observers/resource_observer.go +++ b/internal/querycoordv2/observers/resource_observer.go @@ -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. diff --git a/internal/querycoordv2/ops_services.go b/internal/querycoordv2/ops_services.go index 9051f50bd93e3..d55f734d85e16 100644 --- a/internal/querycoordv2/ops_services.go +++ b/internal/querycoordv2/ops_services.go @@ -257,7 +257,7 @@ func (s *Server) TransferSegment(ctx context.Context, req *querypb.TransferSegme // check whether srcNode is healthy srcNode := req.GetSourceNodeID() - if err := s.isStoppingNode(srcNode); err != nil { + if err := s.isStoppingNode(ctx, srcNode); err != nil { err := merr.WrapErrNodeNotAvailable(srcNode, "the source node is invalid") return merr.Status(err), nil } @@ -270,7 +270,7 @@ func (s *Server) TransferSegment(ctx context.Context, req *querypb.TransferSegme dstNodeSet.Insert(replica.GetRWNodes()...) } else { // check whether dstNode is healthy - if err := s.isStoppingNode(req.GetTargetNodeID()); err != nil { + if err := s.isStoppingNode(ctx, req.GetTargetNodeID()); err != nil { err := merr.WrapErrNodeNotAvailable(srcNode, "the target node is invalid") return merr.Status(err), nil } @@ -329,7 +329,7 @@ func (s *Server) TransferChannel(ctx context.Context, req *querypb.TransferChann // check whether srcNode is healthy srcNode := req.GetSourceNodeID() - if err := s.isStoppingNode(srcNode); err != nil { + if err := s.isStoppingNode(ctx, srcNode); err != nil { err := merr.WrapErrNodeNotAvailable(srcNode, "the source node is invalid") return merr.Status(err), nil } @@ -342,7 +342,7 @@ func (s *Server) TransferChannel(ctx context.Context, req *querypb.TransferChann dstNodeSet.Insert(replica.GetRWNodes()...) } else { // check whether dstNode is healthy - if err := s.isStoppingNode(req.GetTargetNodeID()); err != nil { + if err := s.isStoppingNode(ctx, req.GetTargetNodeID()); err != nil { err := merr.WrapErrNodeNotAvailable(srcNode, "the target node is invalid") return merr.Status(err), nil } diff --git a/internal/querycoordv2/server.go b/internal/querycoordv2/server.go index 8549f3d1a2852..c4675fc08f61a 100644 --- a/internal/querycoordv2/server.go +++ b/internal/querycoordv2/server.go @@ -156,6 +156,7 @@ func NewQueryCoord(ctx context.Context) (*Server, error) { } func (s *Server) Register() error { + log := log.Ctx(s.ctx) s.session.Register() afterRegister := func() { metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.QueryCoordRole).Inc() @@ -239,10 +240,11 @@ func (s *Server) registerMetricsRequest() { // register actions that requests are processed in querynode s.metricsRequest.RegisterMetricsRequest(metricsinfo.SegmentKey, QuerySegmentsAction) s.metricsRequest.RegisterMetricsRequest(metricsinfo.ChannelKey, QueryChannelsAction) - log.Info("register metrics actions finished") + log.Ctx(s.ctx).Info("register metrics actions finished") } func (s *Server) Init() error { + log := log.Ctx(s.ctx) log.Info("QueryCoord start init", zap.String("meta-root-path", Params.EtcdCfg.MetaRootPath.GetValue()), zap.String("address", s.address)) @@ -276,6 +278,7 @@ func (s *Server) Init() error { } func (s *Server) initQueryCoord() error { + log := log.Ctx(s.ctx) s.UpdateStateCode(commonpb.StateCode_Initializing) log.Info("start init querycoord", zap.Any("State", commonpb.StateCode_Initializing)) // Init KV and ID allocator @@ -409,6 +412,7 @@ func (s *Server) initQueryCoord() error { } func (s *Server) initMeta() error { + log := log.Ctx(s.ctx) record := timerecord.NewTimeRecorder("querycoord") log.Info("init meta") @@ -462,7 +466,7 @@ func (s *Server) initMeta() error { } func (s *Server) initObserver() { - log.Info("init observers") + log.Ctx(s.ctx).Info("init observers") s.targetObserver = observers.NewTargetObserver( s.meta, s.targetMgr, @@ -500,13 +504,13 @@ func (s *Server) Start() error { if err := s.startQueryCoord(); err != nil { return err } - log.Info("QueryCoord started") + log.Ctx(s.ctx).Info("QueryCoord started") } return nil } func (s *Server) startQueryCoord() error { - log.Info("start watcher...") + log.Ctx(s.ctx).Info("start watcher...") sessions, revision, err := s.session.GetSessions(typeutil.QueryNodeRole) if err != nil { return err @@ -541,7 +545,7 @@ func (s *Server) startQueryCoord() error { s.updateBalanceConfigLoop(s.ctx) if err := s.proxyWatcher.WatchProxy(s.ctx); err != nil { - log.Warn("querycoord failed to watch proxy", zap.Error(err)) + log.Ctx(s.ctx).Warn("querycoord failed to watch proxy", zap.Error(err)) } s.startServerLoop() @@ -552,6 +556,7 @@ func (s *Server) startQueryCoord() error { } func (s *Server) startServerLoop() { + log := log.Ctx(s.ctx) // leader cache observer shall be started before `SyncAll` call s.leaderCacheObserver.Start(s.ctx) // Recover dist, to avoid generate too much task when dist not ready after restart @@ -579,6 +584,7 @@ func (s *Server) startServerLoop() { } func (s *Server) Stop() error { + log := log.Ctx(s.ctx) // FOLLOW the dependence graph: // job scheduler -> checker controller -> task scheduler -> dist controller -> cluster -> session // observers -> dist controller @@ -652,7 +658,7 @@ func (s *Server) State() commonpb.StateCode { } func (s *Server) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) { - log.Debug("QueryCoord current state", zap.String("StateCode", s.State().String())) + log.Ctx(ctx).Debug("QueryCoord current state", zap.String("StateCode", s.State().String())) nodeID := common.NotRegisteredID if s.session != nil && s.session.Registered() { nodeID = s.session.GetServerID() @@ -721,6 +727,7 @@ func (s *Server) SetQueryNodeCreator(f func(ctx context.Context, addr string, no } func (s *Server) watchNodes(revision int64) { + log := log.Ctx(s.ctx) defer s.wg.Done() eventChan := s.session.WatchServices(typeutil.QueryNodeRole, revision+1, nil) @@ -787,6 +794,7 @@ func (s *Server) watchNodes(revision int64) { } func (s *Server) handleNodeUpLoop() { + log := log.Ctx(s.ctx) defer s.wg.Done() ticker := time.NewTicker(Params.QueryCoordCfg.CheckHealthInterval.GetAsDuration(time.Millisecond)) defer ticker.Stop() @@ -866,6 +874,7 @@ func (s *Server) checkNodeStateInRG() { } func (s *Server) updateBalanceConfigLoop(ctx context.Context) { + log := log.Ctx(s.ctx) success := s.updateBalanceConfig() if success { return @@ -914,6 +923,7 @@ func (s *Server) updateBalanceConfig() bool { } func (s *Server) watchLoadConfigChanges() { + log := log.Ctx(s.ctx) replicaNumHandler := config.NewHandler("watchReplicaNumberChanges", func(e *config.Event) { log.Info("watch load config changes", zap.String("key", e.Key), zap.String("value", e.Value), zap.String("type", e.EventType)) diff --git a/internal/querycoordv2/services.go b/internal/querycoordv2/services.go index d764dcaddba01..ff94adc961e4f 100644 --- a/internal/querycoordv2/services.go +++ b/internal/querycoordv2/services.go @@ -56,8 +56,9 @@ var ( ) func (s *Server) ShowCollections(ctx context.Context, req *querypb.ShowCollectionsRequest) (*querypb.ShowCollectionsResponse, error) { - log.Ctx(ctx).Info("show collections request received", zap.Int64s("collections", req.GetCollectionIDs())) + log := log.Ctx(ctx).With(zap.Int64s("collections", req.GetCollectionIDs())) + log.Info("show collections request received") if err := merr.CheckHealthy(s.State()); err != nil { msg := "failed to show collections" log.Warn(msg, zap.Error(err)) @@ -687,15 +688,15 @@ func (s *Server) refreshCollection(ctx context.Context, collectionID int64) erro // } // } -func (s *Server) isStoppingNode(nodeID int64) error { +func (s *Server) isStoppingNode(ctx context.Context, nodeID int64) error { isStopping, err := s.nodeMgr.IsStoppingNode(nodeID) if err != nil { - log.Warn("fail to check whether the node is stopping", zap.Int64("node_id", nodeID), zap.Error(err)) + log.Ctx(ctx).Warn("fail to check whether the node is stopping", zap.Int64("node_id", nodeID), zap.Error(err)) return err } if isStopping { msg := fmt.Sprintf("failed to balance due to the source/destination node[%d] is stopping", nodeID) - log.Warn(msg) + log.Ctx(ctx).Warn(msg) return errors.New(msg) } return nil @@ -738,7 +739,7 @@ func (s *Server) LoadBalance(ctx context.Context, req *querypb.LoadBalanceReques log.Warn(msg) return merr.Status(err), nil } - if err := s.isStoppingNode(srcNode); err != nil { + if err := s.isStoppingNode(ctx, srcNode); err != nil { return merr.Status(errors.Wrap(err, fmt.Sprintf("can't balance, because the source node[%d] is invalid", srcNode))), nil } @@ -760,7 +761,7 @@ func (s *Server) LoadBalance(ctx context.Context, req *querypb.LoadBalanceReques // check whether dstNode is healthy for dstNode := range dstNodeSet { - if err := s.isStoppingNode(dstNode); err != nil { + if err := s.isStoppingNode(ctx, dstNode); err != nil { return merr.Status(errors.Wrap(err, fmt.Sprintf("can't balance, because the destination node[%d] is invalid", dstNode))), nil } @@ -924,7 +925,7 @@ func (s *Server) CheckHealth(ctx context.Context, req *milvuspb.CheckHealthReque } if err := utils.CheckCollectionsQueryable(ctx, s.meta, s.targetMgr, s.dist, s.nodeMgr); err != nil { - log.Warn("some collection is not queryable during health check", zap.Error(err)) + log.Ctx(ctx).Warn("some collection is not queryable during health check", zap.Error(err)) } return componentutil.CheckHealthRespWithErr(nil), nil diff --git a/internal/querycoordv2/task/scheduler.go b/internal/querycoordv2/task/scheduler.go index cde0c3d43df34..d34a4eeaeb848 100644 --- a/internal/querycoordv2/task/scheduler.go +++ b/internal/querycoordv2/task/scheduler.go @@ -242,7 +242,7 @@ func (scheduler *taskScheduler) AddExecutor(nodeID int64) { scheduler.executors[nodeID] = executor executor.Start(scheduler.ctx) - log.Info("add executor for new QueryNode", zap.Int64("nodeID", nodeID)) + log.Ctx(scheduler.ctx).Info("add executor for new QueryNode", zap.Int64("nodeID", nodeID)) } func (scheduler *taskScheduler) RemoveExecutor(nodeID int64) { @@ -253,7 +253,7 @@ func (scheduler *taskScheduler) RemoveExecutor(nodeID int64) { if ok { executor.Stop() delete(scheduler.executors, nodeID) - log.Info("remove executor of offline QueryNode", zap.Int64("nodeID", nodeID)) + log.Ctx(scheduler.ctx).Info("remove executor of offline QueryNode", zap.Int64("nodeID", nodeID)) } } @@ -334,7 +334,7 @@ func (scheduler *taskScheduler) preAdd(task Task) error { index := NewReplicaSegmentIndex(task) if old, ok := scheduler.segmentTasks[index]; ok { if task.Priority() > old.Priority() { - log.Info("replace old task, the new one with higher priority", + log.Ctx(scheduler.ctx).Info("replace old task, the new one with higher priority", zap.Int64("oldID", old.ID()), zap.String("oldPriority", old.Priority().String()), zap.Int64("newID", task.ID()), @@ -367,7 +367,7 @@ func (scheduler *taskScheduler) preAdd(task Task) error { index := replicaChannelIndex{task.ReplicaID(), task.Channel()} if old, ok := scheduler.channelTasks[index]; ok { if task.Priority() > old.Priority() { - log.Info("replace old task, the new one with higher priority", + log.Ctx(scheduler.ctx).Info("replace old task, the new one with higher priority", zap.Int64("oldID", old.ID()), zap.String("oldPriority", old.Priority().String()), zap.Int64("newID", task.ID()), @@ -400,7 +400,7 @@ func (scheduler *taskScheduler) preAdd(task Task) error { index := NewReplicaLeaderIndex(task) if old, ok := scheduler.segmentTasks[index]; ok { if task.Priority() > old.Priority() { - log.Info("replace old task, the new one with higher priority", + log.Ctx(scheduler.ctx).Info("replace old task, the new one with higher priority", zap.Int64("oldID", old.ID()), zap.String("oldPriority", old.Priority().String()), zap.Int64("newID", task.ID()), @@ -428,7 +428,7 @@ func (scheduler *taskScheduler) tryPromoteAll() { if err != nil { task.Cancel(err) toRemove = append(toRemove, task) - log.Warn("failed to promote task", + log.Ctx(scheduler.ctx).Warn("failed to promote task", zap.Int64("taskID", task.ID()), zap.Error(err), ) @@ -447,14 +447,14 @@ func (scheduler *taskScheduler) tryPromoteAll() { } if len(toPromote) > 0 || len(toRemove) > 0 { - log.Debug("promoted tasks", + log.Ctx(scheduler.ctx).Debug("promoted tasks", zap.Int("promotedNum", len(toPromote)), zap.Int("toRemoveNum", len(toRemove))) } } func (scheduler *taskScheduler) promote(task Task) error { - log := log.With( + log := log.Ctx(scheduler.ctx).With( zap.Int64("taskID", task.ID()), zap.Int64("collectionID", task.CollectionID()), zap.Int64("replicaID", task.ReplicaID()), @@ -474,7 +474,7 @@ func (scheduler *taskScheduler) promote(task Task) error { func (scheduler *taskScheduler) Dispatch(node int64) { select { case <-scheduler.ctx.Done(): - log.Info("scheduler stopped") + log.Ctx(scheduler.ctx).Info("scheduler stopped") default: scheduler.rwmutex.Lock() @@ -629,7 +629,7 @@ func (scheduler *taskScheduler) GetTasksJSON() string { tasks := scheduler.taskStats.Values() ret, err := json.Marshal(tasks) if err != nil { - log.Warn("marshal tasks fail", zap.Error(err)) + log.Ctx(scheduler.ctx).Warn("marshal tasks fail", zap.Error(err)) return "" } return string(ret) @@ -644,7 +644,7 @@ func (scheduler *taskScheduler) schedule(node int64) { return } - log := log.With( + log := log.Ctx(scheduler.ctx).With( zap.Int64("nodeID", node), ) @@ -784,7 +784,7 @@ func (scheduler *taskScheduler) preProcess(task Task) bool { // process processes the given task, // return true if the task is started and succeeds to commit the current action func (scheduler *taskScheduler) process(task Task) bool { - log := log.With( + log := log.Ctx(scheduler.ctx).With( zap.Int64("taskID", task.ID()), zap.Int64("collectionID", task.CollectionID()), zap.Int64("replicaID", task.ReplicaID()), @@ -830,7 +830,7 @@ func (scheduler *taskScheduler) RemoveByNode(node int64) { } func (scheduler *taskScheduler) recordSegmentTaskError(task *SegmentTask) { - log.Warn("task scheduler recordSegmentTaskError", + log.Ctx(scheduler.ctx).Warn("task scheduler recordSegmentTaskError", zap.Int64("taskID", task.ID()), zap.Int64("collectionID", task.CollectionID()), zap.Int64("replicaID", task.ReplicaID()), @@ -928,7 +928,7 @@ func (scheduler *taskScheduler) getTaskMetricsLabel(task Task) string { } func (scheduler *taskScheduler) checkStale(task Task) error { - log := log.With( + log := log.Ctx(task.Context()).With( zap.Int64("taskID", task.ID()), zap.Int64("collectionID", task.CollectionID()), zap.Int64("replicaID", task.ReplicaID()), @@ -970,7 +970,7 @@ func (scheduler *taskScheduler) checkStale(task Task) error { } func (scheduler *taskScheduler) checkSegmentTaskStale(task *SegmentTask) error { - log := log.With( + log := log.Ctx(task.Context()).With( zap.Int64("taskID", task.ID()), zap.Int64("collectionID", task.CollectionID()), zap.Int64("replicaID", task.ReplicaID()), @@ -1013,7 +1013,7 @@ func (scheduler *taskScheduler) checkSegmentTaskStale(task *SegmentTask) error { } func (scheduler *taskScheduler) checkChannelTaskStale(task *ChannelTask) error { - log := log.With( + log := log.Ctx(task.Context()).With( zap.Int64("taskID", task.ID()), zap.Int64("collectionID", task.CollectionID()), zap.Int64("replicaID", task.ReplicaID()), @@ -1041,7 +1041,7 @@ func (scheduler *taskScheduler) checkChannelTaskStale(task *ChannelTask) error { } func (scheduler *taskScheduler) checkLeaderTaskStale(task *LeaderTask) error { - log := log.With( + log := log.Ctx(task.Context()).With( zap.Int64("taskID", task.ID()), zap.Int64("collectionID", task.CollectionID()), zap.Int64("replicaID", task.ReplicaID()), diff --git a/internal/querynodev2/delegator/exclude_info.go b/internal/querynodev2/delegator/exclude_info.go index 72d0354e3417e..ad116f8872807 100644 --- a/internal/querynodev2/delegator/exclude_info.go +++ b/internal/querynodev2/delegator/exclude_info.go @@ -17,6 +17,7 @@ package delegator import ( + "context" "sync" "time" @@ -45,7 +46,7 @@ func (s *ExcludedSegments) Insert(excludeInfo map[int64]uint64) { defer s.mu.Unlock() for segmentID, ts := range excludeInfo { - log.Debug("add exclude info", + log.Ctx(context.TODO()).Debug("add exclude info", zap.Int64("segmentID", segmentID), zap.Uint64("ts", ts), ) diff --git a/internal/querynodev2/delegator/idf_oracle.go b/internal/querynodev2/delegator/idf_oracle.go index 2f16f8352bd20..3e6c487ddc1a8 100644 --- a/internal/querynodev2/delegator/idf_oracle.go +++ b/internal/querynodev2/delegator/idf_oracle.go @@ -17,6 +17,7 @@ package delegator import ( + "context" "fmt" "sync" @@ -239,7 +240,7 @@ func (o *idfOracle) SyncDistribution(snapshot *snapshot) { } } - log.Debug("sync distribution finished", zap.Int64("version", o.targetVersion), zap.Int64("numrow", o.current.NumRow())) + log.Ctx(context.TODO()).Debug("sync distribution finished", zap.Int64("version", o.targetVersion), zap.Int64("numrow", o.current.NumRow())) } func (o *idfOracle) BuildIDF(fieldID int64, tfs *schemapb.SparseFloatArray) ([][]byte, float64, error) { diff --git a/internal/querynodev2/delegator/segment_pruner.go b/internal/querynodev2/delegator/segment_pruner.go index 24cb690a16dad..35503e00c8926 100644 --- a/internal/querynodev2/delegator/segment_pruner.go +++ b/internal/querynodev2/delegator/segment_pruner.go @@ -269,7 +269,7 @@ func FilterSegmentsByVector(partitionStats *storage.PartitionStatsSnapshot, segmentCount := len(segmentsToSearch) targetSegNum := int(math.Sqrt(float64(segmentCount)) * filterRatio) if targetSegNum > segmentCount { - log.Debug("Warn! targetSegNum is larger or equal than segmentCount, no prune effect at all", + log.Ctx(context.TODO()).Debug("Warn! targetSegNum is larger or equal than segmentCount, no prune effect at all", zap.Int("targetSegNum", targetSegNum), zap.Int("segmentCount", segmentCount), zap.Float64("filterRatio", filterRatio)) diff --git a/internal/querynodev2/handlers.go b/internal/querynodev2/handlers.go index 5b16778a49424..9c3cc113de236 100644 --- a/internal/querynodev2/handlers.go +++ b/internal/querynodev2/handlers.go @@ -310,7 +310,7 @@ func (node *QueryNode) queryChannelStream(ctx context.Context, req *querypb.Quer } func (node *QueryNode) queryStreamSegments(ctx context.Context, req *querypb.QueryRequest, srv streamrpc.QueryStreamServer) error { - log.Debug("received query stream request", + log.Ctx(ctx).Debug("received query stream request", zap.Int64s("outputFields", req.GetReq().GetOutputFieldsId()), zap.Int64s("segmentIDs", req.GetSegmentIDs()), zap.Uint64("guaranteeTimestamp", req.GetReq().GetGuaranteeTimestamp()), @@ -328,13 +328,13 @@ func (node *QueryNode) queryStreamSegments(ctx context.Context, req *querypb.Que paramtable.Get().QueryNodeCfg.QueryStreamBatchSize.GetAsInt(), paramtable.Get().QueryNodeCfg.QueryStreamMaxBatchSize.GetAsInt()) if err := node.scheduler.Add(task); err != nil { - log.Warn("failed to add query task into scheduler", zap.Error(err)) + log.Ctx(ctx).Warn("failed to add query task into scheduler", zap.Error(err)) return err } err := task.Wait() if err != nil { - log.Warn("failed to execute task by node scheduler", zap.Error(err)) + log.Ctx(ctx).Warn("failed to execute task by node scheduler", zap.Error(err)) return err } diff --git a/internal/querynodev2/pipeline/embedding_node.go b/internal/querynodev2/pipeline/embedding_node.go index d3a8298214ead..da75099f0b8af 100644 --- a/internal/querynodev2/pipeline/embedding_node.go +++ b/internal/querynodev2/pipeline/embedding_node.go @@ -17,6 +17,7 @@ package pipeline import ( + "context" "fmt" "go.uber.org/zap" @@ -125,7 +126,7 @@ func (eNode *embeddingNode) addInsertData(insertDatas map[UniqueID]*delegator.In iData.PrimaryKeys = append(iData.PrimaryKeys, pks...) iData.RowIDs = append(iData.RowIDs, msg.RowIDs...) iData.Timestamps = append(iData.Timestamps, msg.Timestamps...) - log.Debug("pipeline embedding insert msg", + log.Ctx(context.TODO()).Debug("pipeline embedding insert msg", zap.Int64("collectionID", eNode.collectionID), zap.Int64("segmentID", msg.SegmentID), zap.Int("insertRowNum", len(pks)), diff --git a/internal/querynodev2/pipeline/filter_node.go b/internal/querynodev2/pipeline/filter_node.go index d13e2bc5a0812..fc88e559b7a2d 100644 --- a/internal/querynodev2/pipeline/filter_node.go +++ b/internal/querynodev2/pipeline/filter_node.go @@ -17,6 +17,7 @@ package pipeline import ( + "context" "fmt" "reflect" @@ -46,6 +47,7 @@ type filterNode struct { } func (fNode *filterNode) Operate(in Msg) Msg { + log := log.Ctx(context.TODO()) if in == nil { log.Debug("type assertion failed for Msg in filterNode because it's nil", zap.String("name", fNode.Name())) diff --git a/internal/querynodev2/pipeline/insert_node.go b/internal/querynodev2/pipeline/insert_node.go index f3fd1bb26455c..1e5c238f198ac 100644 --- a/internal/querynodev2/pipeline/insert_node.go +++ b/internal/querynodev2/pipeline/insert_node.go @@ -17,6 +17,7 @@ package pipeline import ( + "context" "fmt" "sort" @@ -77,7 +78,7 @@ func (iNode *insertNode) addInsertData(insertDatas map[UniqueID]*delegator.Inser iData.PrimaryKeys = append(iData.PrimaryKeys, pks...) iData.RowIDs = append(iData.RowIDs, msg.RowIDs...) iData.Timestamps = append(iData.Timestamps, msg.Timestamps...) - log.Debug("pipeline fetch insert msg", + log.Ctx(context.TODO()).Debug("pipeline fetch insert msg", zap.Int64("collectionID", iNode.collectionID), zap.Int64("segmentID", msg.SegmentID), zap.Int("insertRowNum", len(pks)), diff --git a/internal/querynodev2/segments/result.go b/internal/querynodev2/segments/result.go index 6eaaab1ad6715..9f78edece3b5a 100644 --- a/internal/querynodev2/segments/result.go +++ b/internal/querynodev2/segments/result.go @@ -261,7 +261,8 @@ func EncodeSearchResultData(ctx context.Context, searchResultData *schemapb.Sear } func MergeInternalRetrieveResult(ctx context.Context, retrieveResults []*internalpb.RetrieveResults, param *mergeParam) (*internalpb.RetrieveResults, error) { - log.Ctx(ctx).Debug("mergeInternelRetrieveResults", + log := log.Ctx(ctx) + log.Debug("mergeInternelRetrieveResults", zap.Int64("limit", param.limit), zap.Int("resultNum", len(retrieveResults)), ) @@ -386,7 +387,8 @@ func MergeSegcoreRetrieveResults(ctx context.Context, retrieveResults []*segcore ctx, span := otel.Tracer(typeutil.QueryNodeRole).Start(ctx, "MergeSegcoreResults") defer span.End() - log.Ctx(ctx).Debug("mergeSegcoreRetrieveResults", + log := log.Ctx(ctx) + log.Debug("mergeSegcoreRetrieveResults", zap.Int64("limit", param.limit), zap.Int("resultNum", len(retrieveResults)), ) diff --git a/internal/querynodev2/segments/retrieve.go b/internal/querynodev2/segments/retrieve.go index f7c30bfa4ed62..c615dcf79f8e0 100644 --- a/internal/querynodev2/segments/retrieve.go +++ b/internal/querynodev2/segments/retrieve.go @@ -152,6 +152,7 @@ func Retrieve(ctx context.Context, manager *Manager, plan *RetrievePlan, req *qu segIDs := req.GetSegmentIDs() collID := req.Req.GetCollectionID() + log := log.Ctx(ctx) log.Debug("retrieve on segments", zap.Int64s("segmentIDs", segIDs), zap.Int64("collectionID", collID)) if req.GetScope() == querypb.DataScope_Historical { diff --git a/internal/querynodev2/server.go b/internal/querynodev2/server.go index 9bc9bf3de2a06..e326d5fb8338e 100644 --- a/internal/querynodev2/server.go +++ b/internal/querynodev2/server.go @@ -168,7 +168,7 @@ func (node *QueryNode) initSession() error { sessionutil.SaveServerInfo(typeutil.QueryNodeRole, node.session.ServerID) paramtable.SetNodeID(node.session.ServerID) node.serverID = node.session.ServerID - log.Info("QueryNode init session", zap.Int64("nodeID", node.GetNodeID()), zap.String("node address", node.session.Address)) + log.Ctx(node.ctx).Info("QueryNode init session", zap.Int64("nodeID", node.GetNodeID()), zap.String("node address", node.session.Address)) return nil } @@ -178,7 +178,7 @@ func (node *QueryNode) Register() error { // start liveness check metrics.NumNodes.WithLabelValues(fmt.Sprint(node.GetNodeID()), typeutil.QueryNodeRole).Inc() node.session.LivenessCheck(node.ctx, func() { - log.Error("Query Node disconnected from etcd, process will exit", zap.Int64("Server Id", paramtable.GetNodeID())) + log.Ctx(node.ctx).Error("Query Node disconnected from etcd, process will exit", zap.Int64("Server Id", paramtable.GetNodeID())) os.Exit(1) }) return nil @@ -235,7 +235,7 @@ func (node *QueryNode) InitSegcore() error { if knowhereBuildPoolSize < uint32(1) { knowhereBuildPoolSize = uint32(1) } - log.Info("set up knowhere build pool size", zap.Uint32("pool_size", knowhereBuildPoolSize)) + log.Ctx(node.ctx).Info("set up knowhere build pool size", zap.Uint32("pool_size", knowhereBuildPoolSize)) cKnowhereBuildPoolSize := C.uint32_t(knowhereBuildPoolSize) C.SegcoreSetKnowhereBuildThreadPoolNum(cKnowhereBuildPoolSize) @@ -294,11 +294,12 @@ func (node *QueryNode) registerMetricsRequest() { func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { return getChannelJSON(node), nil }) - log.Info("register metrics actions finished") + log.Ctx(node.ctx).Info("register metrics actions finished") } // Init function init historical and streaming module to manage segments func (node *QueryNode) Init() error { + log := log.Ctx(node.ctx) var initError error node.initOnce.Do(func() { node.registerMetricsRequest() @@ -395,6 +396,7 @@ func (node *QueryNode) Init() error { // Start mainly start QueryNode's query service. func (node *QueryNode) Start() error { + log := log.Ctx(node.ctx) node.startOnce.Do(func() { node.scheduler.Start() @@ -429,6 +431,7 @@ func (node *QueryNode) Start() error { // Stop mainly stop QueryNode's query service, historical loop and streaming loop. func (node *QueryNode) Stop() error { + log := log.Ctx(node.ctx) node.stopOnce.Do(func() { log.Info("Query node stop...") err := node.session.GoingStop() @@ -535,6 +538,7 @@ func (node *QueryNode) SetAddress(address string) { // initHook initializes parameter tuning hook. func (node *QueryNode) initHook() error { + log := log.Ctx(node.ctx) path := paramtable.Get().QueryNodeCfg.SoPath.GetValue() if path == "" { return fmt.Errorf("fail to set the plugin path") @@ -570,6 +574,7 @@ func (node *QueryNode) initHook() error { } func (node *QueryNode) handleQueryHookEvent() { + log := log.Ctx(node.ctx) onEvent := func(event *config.Event) { if node.queryHook != nil { if err := node.queryHook.Init(event.Value); err != nil { diff --git a/internal/querynodev2/services.go b/internal/querynodev2/services.go index 734d8e4d717c6..f3c65d3fe5422 100644 --- a/internal/querynodev2/services.go +++ b/internal/querynodev2/services.go @@ -70,7 +70,7 @@ func (node *QueryNode) GetComponentStates(ctx context.Context, req *milvuspb.Get if node.session != nil && node.session.Registered() { nodeID = node.GetNodeID() } - log.Debug("QueryNode current state", zap.Int64("NodeID", nodeID), zap.String("StateCode", code.String())) + log.Ctx(ctx).Debug("QueryNode current state", zap.Int64("NodeID", nodeID), zap.String("StateCode", code.String())) info := &milvuspb.ComponentInfo{ NodeID: nodeID, @@ -162,7 +162,7 @@ func (node *QueryNode) GetStatistics(ctx context.Context, req *querypb.GetStatis return ret, nil } -func (node *QueryNode) composeIndexMeta(indexInfos []*indexpb.IndexInfo, schema *schemapb.CollectionSchema) *segcorepb.CollectionIndexMeta { +func (node *QueryNode) composeIndexMeta(ctx context.Context, indexInfos []*indexpb.IndexInfo, schema *schemapb.CollectionSchema) *segcorepb.CollectionIndexMeta { fieldIndexMetas := make([]*segcorepb.FieldIndexMeta, 0) for _, info := range indexInfos { fieldIndexMetas = append(fieldIndexMetas, &segcorepb.FieldIndexMeta{ @@ -178,7 +178,7 @@ func (node *QueryNode) composeIndexMeta(indexInfos []*indexpb.IndexInfo, schema sizePerRecord, err := typeutil.EstimateSizePerRecord(schema) maxIndexRecordPerSegment := int64(0) if err != nil || sizePerRecord == 0 { - log.Warn("failed to transfer segment size to collection, because failed to estimate size per record", zap.Error(err)) + log.Ctx(ctx).Warn("failed to transfer segment size to collection, because failed to estimate size per record", zap.Error(err)) } else { threshold := paramtable.Get().DataCoordCfg.SegmentMaxSize.GetAsFloat() * 1024 * 1024 proportion := paramtable.Get().DataCoordCfg.SegmentSealProportion.GetAsFloat() @@ -239,7 +239,7 @@ func (node *QueryNode) WatchDmChannels(ctx context.Context, req *querypb.WatchDm } node.manager.Collection.PutOrRef(req.GetCollectionID(), req.GetSchema(), - node.composeIndexMeta(req.GetIndexInfoList(), req.Schema), req.GetLoadMeta()) + node.composeIndexMeta(ctx, req.GetIndexInfoList(), req.Schema), req.GetLoadMeta()) defer func() { if !merr.Ok(status) { node.manager.Collection.Unref(req.GetCollectionID(), 1) @@ -476,7 +476,7 @@ func (node *QueryNode) LoadSegments(ctx context.Context, req *querypb.LoadSegmen } node.manager.Collection.PutOrRef(req.GetCollectionID(), req.GetSchema(), - node.composeIndexMeta(req.GetIndexInfoList(), req.GetSchema()), req.GetLoadMeta()) + node.composeIndexMeta(ctx, req.GetIndexInfoList(), req.GetSchema()), req.GetLoadMeta()) defer node.manager.Collection.Unref(req.GetCollectionID(), 1) if req.GetLoadScope() == querypb.LoadScope_Delta { @@ -1068,7 +1068,7 @@ func (node *QueryNode) SyncReplicaSegments(ctx context.Context, req *querypb.Syn // ShowConfigurations returns the configurations of queryNode matching req.Pattern func (node *QueryNode) ShowConfigurations(ctx context.Context, req *internalpb.ShowConfigurationsRequest) (*internalpb.ShowConfigurationsResponse, error) { if err := node.lifetime.Add(merr.IsHealthy); err != nil { - log.Warn("QueryNode.ShowConfigurations failed", + log.Ctx(ctx).Warn("QueryNode.ShowConfigurations failed", zap.Int64("nodeId", node.GetNodeID()), zap.String("req", req.Pattern), zap.Error(err)) @@ -1098,7 +1098,7 @@ func (node *QueryNode) ShowConfigurations(ctx context.Context, req *internalpb.S // GetMetrics return system infos of the query node, such as total memory, memory usage, cpu usage ... func (node *QueryNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest) (*milvuspb.GetMetricsResponse, error) { if err := node.lifetime.Add(merr.IsHealthy); err != nil { - log.Warn("QueryNode.GetMetrics failed", + log.Ctx(ctx).Warn("QueryNode.GetMetrics failed", zap.Int64("nodeId", node.GetNodeID()), zap.String("req", req.Request), zap.Error(err)) diff --git a/internal/querynodev2/services_test.go b/internal/querynodev2/services_test.go index 075290bf0543c..4b02290fda840 100644 --- a/internal/querynodev2/services_test.go +++ b/internal/querynodev2/services_test.go @@ -1268,7 +1268,7 @@ func (suite *ServiceSuite) TestSearch_Failed() { CollectionID: suite.collectionID, PartitionIDs: suite.partitionIDs, } - indexMeta := suite.node.composeIndexMeta(mock_segcore.GenTestIndexInfoList(suite.collectionID, schema), schema) + indexMeta := suite.node.composeIndexMeta(ctx, mock_segcore.GenTestIndexInfoList(suite.collectionID, schema), schema) suite.node.manager.Collection.PutOrRef(suite.collectionID, schema, indexMeta, LoadMeta) // Delegator not found diff --git a/internal/rootcoord/alter_collection_task.go b/internal/rootcoord/alter_collection_task.go index f0887f2cf82d5..7a08b00bbf6b2 100644 --- a/internal/rootcoord/alter_collection_task.go +++ b/internal/rootcoord/alter_collection_task.go @@ -54,13 +54,13 @@ func (a *alterCollectionTask) Execute(ctx context.Context) error { oldColl, err := a.core.meta.GetCollectionByName(ctx, a.Req.GetDbName(), a.Req.GetCollectionName(), a.ts) if err != nil { - log.Warn("get collection failed during changing collection state", + log.Ctx(ctx).Warn("get collection failed during changing collection state", zap.String("collectionName", a.Req.GetCollectionName()), zap.Uint64("ts", a.ts)) return err } if ContainsKeyPairArray(a.Req.GetProperties(), oldColl.Properties) { - log.Info("skip to alter collection due to no changes were detected in the properties", zap.Int64("collectionID", oldColl.CollectionID)) + log.Ctx(ctx).Info("skip to alter collection due to no changes were detected in the properties", zap.Int64("collectionID", oldColl.CollectionID)) return nil } @@ -115,7 +115,7 @@ func (a *alterCollectionTask) Execute(ctx context.Context) error { ResourceGroups: newResourceGroups, }) if err := merr.CheckRPCCall(resp, err); err != nil { - log.Warn("failed to trigger update load config for collection", zap.Int64("collectionID", newColl.CollectionID), zap.Error(err)) + log.Ctx(ctx).Warn("failed to trigger update load config for collection", zap.Int64("collectionID", newColl.CollectionID), zap.Error(err)) return nil, err } return nil, nil diff --git a/internal/rootcoord/alter_database_task.go b/internal/rootcoord/alter_database_task.go index f87ec79d177dc..d9ab2d9e0cf3c 100644 --- a/internal/rootcoord/alter_database_task.go +++ b/internal/rootcoord/alter_database_task.go @@ -95,7 +95,7 @@ func (a *alterDatabaseTask) Execute(ctx context.Context) error { redoTask.AddAsyncStep(NewSimpleStep("", func(ctx context.Context) ([]nestedStep, error) { colls, err := a.core.meta.ListCollections(ctx, oldDB.Name, a.ts, true) if err != nil { - log.Warn("failed to trigger update load config for database", zap.Int64("dbID", oldDB.ID), zap.Error(err)) + log.Ctx(ctx).Warn("failed to trigger update load config for database", zap.Int64("dbID", oldDB.ID), zap.Error(err)) return nil, err } if len(colls) == 0 { @@ -108,7 +108,7 @@ func (a *alterDatabaseTask) Execute(ctx context.Context) error { ResourceGroups: newResourceGroups, }) if err := merr.CheckRPCCall(resp, err); err != nil { - log.Warn("failed to trigger update load config for database", zap.Int64("dbID", oldDB.ID), zap.Error(err)) + log.Ctx(ctx).Warn("failed to trigger update load config for database", zap.Int64("dbID", oldDB.ID), zap.Error(err)) return nil, err } return nil, nil diff --git a/internal/rootcoord/broker.go b/internal/rootcoord/broker.go index c4811b3550a9a..563e41a78af66 100644 --- a/internal/rootcoord/broker.go +++ b/internal/rootcoord/broker.go @@ -225,7 +225,7 @@ func (b *ServerBroker) GetSegmentIndexState(ctx context.Context, collID UniqueID } func (b *ServerBroker) BroadcastAlteredCollection(ctx context.Context, req *milvuspb.AlterCollectionRequest) error { - log.Info("broadcasting request to alter collection", zap.String("collectionName", req.GetCollectionName()), zap.Int64("collectionID", req.GetCollectionID()), zap.Any("props", req.GetProperties())) + log.Ctx(ctx).Info("broadcasting request to alter collection", zap.String("collectionName", req.GetCollectionName()), zap.Int64("collectionID", req.GetCollectionID()), zap.Any("props", req.GetProperties())) colMeta, err := b.s.meta.GetCollectionByID(ctx, req.GetDbName(), req.GetCollectionID(), typeutil.MaxTimestamp, false) if err != nil { @@ -265,7 +265,7 @@ func (b *ServerBroker) BroadcastAlteredCollection(ctx context.Context, req *milv if resp.ErrorCode != commonpb.ErrorCode_Success { return errors.New(resp.Reason) } - log.Info("done to broadcast request to alter collection", zap.String("collectionName", req.GetCollectionName()), zap.Int64("collectionID", req.GetCollectionID()), zap.Any("props", req.GetProperties())) + log.Ctx(ctx).Info("done to broadcast request to alter collection", zap.String("collectionName", req.GetCollectionName()), zap.Int64("collectionID", req.GetCollectionID()), zap.Any("props", req.GetProperties())) return nil } diff --git a/internal/rootcoord/create_collection_task.go b/internal/rootcoord/create_collection_task.go index 00e7235267ffe..ee835558ae9fe 100644 --- a/internal/rootcoord/create_collection_task.go +++ b/internal/rootcoord/create_collection_task.go @@ -63,7 +63,7 @@ type createCollectionTask struct { partitionNames []string } -func (t *createCollectionTask) validate() error { +func (t *createCollectionTask) validate(ctx context.Context) error { if t.Req == nil { return errors.New("empty requests") } @@ -86,7 +86,7 @@ func (t *createCollectionTask) validate() error { // 2. check db-collection capacity db2CollIDs := t.core.meta.ListAllAvailCollections(t.ctx) - if err := t.checkMaxCollectionsPerDB(db2CollIDs); err != nil { + if err := t.checkMaxCollectionsPerDB(ctx, db2CollIDs); err != nil { return err } @@ -98,7 +98,7 @@ func (t *createCollectionTask) validate() error { maxCollectionNum := Params.QuotaConfig.MaxCollectionNum.GetAsInt() if totalCollections >= maxCollectionNum { - log.Warn("unable to create collection because the number of collection has reached the limit", zap.Int("max_collection_num", maxCollectionNum)) + log.Ctx(ctx).Warn("unable to create collection because the number of collection has reached the limit", zap.Int("max_collection_num", maxCollectionNum)) return merr.WrapErrCollectionNumLimitExceeded(t.Req.GetDbName(), maxCollectionNum) } @@ -111,22 +111,22 @@ func (t *createCollectionTask) validate() error { } // checkMaxCollectionsPerDB DB properties take precedence over quota configurations for max collections. -func (t *createCollectionTask) checkMaxCollectionsPerDB(db2CollIDs map[int64][]int64) error { +func (t *createCollectionTask) checkMaxCollectionsPerDB(ctx context.Context, db2CollIDs map[int64][]int64) error { collIDs, ok := db2CollIDs[t.dbID] if !ok { - log.Warn("can not found DB ID", zap.String("collection", t.Req.GetCollectionName()), zap.String("dbName", t.Req.GetDbName())) + log.Ctx(ctx).Warn("can not found DB ID", zap.String("collection", t.Req.GetCollectionName()), zap.String("dbName", t.Req.GetDbName())) return merr.WrapErrDatabaseNotFound(t.Req.GetDbName(), "failed to create collection") } db, err := t.core.meta.GetDatabaseByName(t.ctx, t.Req.GetDbName(), typeutil.MaxTimestamp) if err != nil { - log.Warn("can not found DB ID", zap.String("collection", t.Req.GetCollectionName()), zap.String("dbName", t.Req.GetDbName())) + log.Ctx(ctx).Warn("can not found DB ID", zap.String("collection", t.Req.GetCollectionName()), zap.String("dbName", t.Req.GetDbName())) return merr.WrapErrDatabaseNotFound(t.Req.GetDbName(), "failed to create collection") } check := func(maxColNumPerDB int) error { if len(collIDs) >= maxColNumPerDB { - log.Warn("unable to create collection because the number of collection has reached the limit in DB", zap.Int("maxCollectionNumPerDB", maxColNumPerDB)) + log.Ctx(ctx).Warn("unable to create collection because the number of collection has reached the limit in DB", zap.Int("maxCollectionNumPerDB", maxColNumPerDB)) return merr.WrapErrCollectionNumLimitExceeded(t.Req.GetDbName(), maxColNumPerDB) } return nil @@ -136,7 +136,7 @@ func (t *createCollectionTask) checkMaxCollectionsPerDB(db2CollIDs map[int64][]i if maxColNumPerDBStr != "" { maxColNumPerDB, err := strconv.Atoi(maxColNumPerDBStr) if err != nil { - log.Warn("parse value of property fail", zap.String("key", common.DatabaseMaxCollectionsKey), + log.Ctx(ctx).Warn("parse value of property fail", zap.String("key", common.DatabaseMaxCollectionsKey), zap.String("value", maxColNumPerDBStr), zap.Error(err)) return fmt.Errorf(fmt.Sprintf("parse value of property fail, key:%s, value:%s", common.DatabaseMaxCollectionsKey, maxColNumPerDBStr)) } @@ -243,10 +243,10 @@ func validateFieldDataType(schema *schemapb.CollectionSchema) error { return nil } -func (t *createCollectionTask) validateSchema(schema *schemapb.CollectionSchema) error { - log.With(zap.String("CollectionName", t.Req.CollectionName)) +func (t *createCollectionTask) validateSchema(ctx context.Context, schema *schemapb.CollectionSchema) error { + log.Ctx(ctx).With(zap.String("CollectionName", t.Req.CollectionName)) if t.Req.GetCollectionName() != schema.GetName() { - log.Error("collection name not matches schema name", zap.String("SchemaName", schema.Name)) + log.Ctx(ctx).Error("collection name not matches schema name", zap.String("SchemaName", schema.Name)) msg := fmt.Sprintf("collection name = %s, schema.Name=%s", t.Req.GetCollectionName(), schema.Name) return merr.WrapErrParameterInvalid("collection name matches schema name", "don't match", msg) } @@ -256,7 +256,7 @@ func (t *createCollectionTask) validateSchema(schema *schemapb.CollectionSchema) } if hasSystemFields(schema, []string{RowIDFieldName, TimeStampFieldName, MetaFieldName}) { - log.Error("schema contains system field", + log.Ctx(ctx).Error("schema contains system field", zap.String("RowIDFieldName", RowIDFieldName), zap.String("TimeStampFieldName", TimeStampFieldName), zap.String("MetaFieldName", MetaFieldName)) @@ -296,7 +296,7 @@ func (t *createCollectionTask) assignFieldAndFunctionID(schema *schemapb.Collect return nil } -func (t *createCollectionTask) appendDynamicField(schema *schemapb.CollectionSchema) { +func (t *createCollectionTask) appendDynamicField(ctx context.Context, schema *schemapb.CollectionSchema) { if schema.EnableDynamicField { schema.Fields = append(schema.Fields, &schemapb.FieldSchema{ Name: MetaFieldName, @@ -304,7 +304,7 @@ func (t *createCollectionTask) appendDynamicField(schema *schemapb.CollectionSch DataType: schemapb.DataType_JSON, IsDynamic: true, }) - log.Info("append dynamic field", zap.String("collection", schema.Name)) + log.Ctx(ctx).Info("append dynamic field", zap.String("collection", schema.Name)) } } @@ -325,15 +325,15 @@ func (t *createCollectionTask) appendSysFields(schema *schemapb.CollectionSchema }) } -func (t *createCollectionTask) prepareSchema() error { +func (t *createCollectionTask) prepareSchema(ctx context.Context) error { var schema schemapb.CollectionSchema if err := proto.Unmarshal(t.Req.GetSchema(), &schema); err != nil { return err } - if err := t.validateSchema(&schema); err != nil { + if err := t.validateSchema(ctx, &schema); err != nil { return err } - t.appendDynamicField(&schema) + t.appendDynamicField(ctx, &schema) if err := t.assignFieldAndFunctionID(&schema); err != nil { return err @@ -356,7 +356,7 @@ func (t *createCollectionTask) assignCollectionID() error { return err } -func (t *createCollectionTask) assignPartitionIDs() error { +func (t *createCollectionTask) assignPartitionIDs(ctx context.Context) error { t.partitionNames = make([]string, 0) defaultPartitionName := Params.CommonCfg.DefaultPartitionName.GetValue() @@ -391,7 +391,7 @@ func (t *createCollectionTask) assignPartitionIDs() error { for i := start; i < end; i++ { t.partIDs[i-start] = i } - log.Info("assign partitions when create collection", + log.Ctx(ctx).Info("assign partitions when create collection", zap.String("collectionName", t.Req.GetCollectionName()), zap.Strings("partitionNames", t.partitionNames)) @@ -425,11 +425,11 @@ func (t *createCollectionTask) Prepare(ctx context.Context) error { } t.dbID = db.ID - if err := t.validate(); err != nil { + if err := t.validate(ctx); err != nil { return err } - if err := t.prepareSchema(); err != nil { + if err := t.prepareSchema(ctx); err != nil { return err } @@ -439,7 +439,7 @@ func (t *createCollectionTask) Prepare(ctx context.Context) error { return err } - if err := t.assignPartitionIDs(); err != nil { + if err := t.assignPartitionIDs(ctx); err != nil { return err } @@ -527,13 +527,13 @@ func (t *createCollectionTask) broadcastCreateCollectionMsgIntoStreamingService( return startPositions, nil } -func (t *createCollectionTask) getCreateTs() (uint64, error) { +func (t *createCollectionTask) getCreateTs(ctx context.Context) (uint64, error) { replicateInfo := t.Req.GetBase().GetReplicateInfo() if !replicateInfo.GetIsReplicate() { return t.GetTs(), nil } if replicateInfo.GetMsgTimestamp() == 0 { - log.Warn("the cdc timestamp is not set in the request for the backup instance") + log.Ctx(ctx).Warn("the cdc timestamp is not set in the request for the backup instance") return 0, merr.WrapErrParameterInvalidMsg("the cdc timestamp is not set in the request for the backup instance") } return replicateInfo.GetMsgTimestamp(), nil @@ -542,7 +542,7 @@ func (t *createCollectionTask) getCreateTs() (uint64, error) { func (t *createCollectionTask) Execute(ctx context.Context) error { collID := t.collID partIDs := t.partIDs - ts, err := t.getCreateTs() + ts, err := t.getCreateTs(ctx) if err != nil { return err } @@ -592,7 +592,7 @@ func (t *createCollectionTask) Execute(ctx context.Context) error { return fmt.Errorf("create duplicate collection with different parameters, collection: %s", t.Req.GetCollectionName()) } // make creating collection idempotent. - log.Warn("add duplicate collection", zap.String("collection", t.Req.GetCollectionName()), zap.Uint64("ts", ts)) + log.Ctx(ctx).Warn("add duplicate collection", zap.String("collection", t.Req.GetCollectionName()), zap.Uint64("ts", ts)) return nil } diff --git a/internal/rootcoord/create_collection_task_test.go b/internal/rootcoord/create_collection_task_test.go index e7d7cc2ce9c88..188fdc1d7947a 100644 --- a/internal/rootcoord/create_collection_task_test.go +++ b/internal/rootcoord/create_collection_task_test.go @@ -47,7 +47,7 @@ func Test_createCollectionTask_validate(t *testing.T) { task := createCollectionTask{ Req: nil, } - err := task.validate() + err := task.validate(context.TODO()) assert.Error(t, err) }) @@ -57,7 +57,7 @@ func Test_createCollectionTask_validate(t *testing.T) { } { task.SetTs(1000) - ts, err := task.getCreateTs() + ts, err := task.getCreateTs(context.TODO()) assert.NoError(t, err) assert.EqualValues(t, 1000, ts) } @@ -72,14 +72,14 @@ func Test_createCollectionTask_validate(t *testing.T) { } { task.SetTs(1000) - _, err := task.getCreateTs() + _, err := task.getCreateTs(context.TODO()) assert.Error(t, err) err = task.Execute(context.Background()) assert.Error(t, err) } { task.Req.Base.ReplicateInfo.MsgTimestamp = 2000 - ts, err := task.getCreateTs() + ts, err := task.getCreateTs(context.TODO()) assert.NoError(t, err) assert.EqualValues(t, 2000, ts) } @@ -91,7 +91,7 @@ func Test_createCollectionTask_validate(t *testing.T) { Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_DropCollection}, }, } - err := task.validate() + err := task.validate(context.TODO()) assert.Error(t, err) }) @@ -104,7 +104,7 @@ func Test_createCollectionTask_validate(t *testing.T) { ShardsNum: cfgMaxShardNum + 1, }, } - err := task.validate() + err := task.validate(context.TODO()) assert.Error(t, err) }) @@ -117,7 +117,7 @@ func Test_createCollectionTask_validate(t *testing.T) { ShardsNum: cfgShardLimit + 1, }, } - err := task.validate() + err := task.validate(context.TODO()) assert.Error(t, err) }) @@ -140,7 +140,7 @@ func Test_createCollectionTask_validate(t *testing.T) { Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection}, }, } - err := task.validate() + err := task.validate(context.TODO()) assert.Error(t, err) task = createCollectionTask{ @@ -150,7 +150,7 @@ func Test_createCollectionTask_validate(t *testing.T) { }, dbID: util.DefaultDBID, } - err = task.validate() + err = task.validate(context.TODO()) assert.Error(t, err) }) @@ -181,7 +181,7 @@ func Test_createCollectionTask_validate(t *testing.T) { }, dbID: util.DefaultDBID, } - err := task.validate() + err := task.validate(context.TODO()) assert.Error(t, err) // invalid properties @@ -204,7 +204,7 @@ func Test_createCollectionTask_validate(t *testing.T) { dbID: util.DefaultDBID, } - err = task.validate() + err = task.validate(context.TODO()) assert.Error(t, err) }) @@ -224,7 +224,7 @@ func Test_createCollectionTask_validate(t *testing.T) { Base: &commonpb.MsgBase{MsgType: commonpb.MsgType_CreateCollection}, }, } - err := task.validate() + err := task.validate(context.TODO()) assert.Error(t, err) task = createCollectionTask{ @@ -234,7 +234,7 @@ func Test_createCollectionTask_validate(t *testing.T) { }, dbID: util.DefaultDBID, } - err = task.validate() + err = task.validate(context.TODO()) assert.Error(t, err) }) @@ -259,7 +259,7 @@ func Test_createCollectionTask_validate(t *testing.T) { }, dbID: util.DefaultDBID, } - err := task.validate() + err := task.validate(context.TODO()) assert.ErrorIs(t, err, merr.ErrGeneralCapacityExceeded) }) @@ -295,7 +295,7 @@ func Test_createCollectionTask_validate(t *testing.T) { paramtable.Get().Save(Params.QuotaConfig.MaxCollectionNum.Key, strconv.Itoa(math.MaxInt64)) defer paramtable.Get().Reset(Params.QuotaConfig.MaxCollectionNum.Key) - err := task.validate() + err := task.validate(context.TODO()) assert.NoError(t, err) }) } @@ -313,7 +313,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { schema := &schemapb.CollectionSchema{ Name: otherName, } - err := task.validateSchema(schema) + err := task.validateSchema(context.TODO(), schema) assert.Error(t, err) }) @@ -335,7 +335,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err := task.validateSchema(schema) + err := task.validateSchema(context.TODO(), schema) assert.Error(t, err) }) @@ -361,7 +361,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err := task.validateSchema(schema) + err := task.validateSchema(context.TODO(), schema) assert.Error(t, err) }) @@ -379,7 +379,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { {Name: RowIDFieldName}, }, } - err := task.validateSchema(schema) + err := task.validateSchema(context.TODO(), schema) assert.Error(t, err) }) @@ -404,7 +404,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err := task.validateSchema(schema) + err := task.validateSchema(context.TODO(), schema) assert.ErrorIs(t, err, merr.ErrParameterInvalid) schema1 := &schemapb.CollectionSchema{ @@ -420,7 +420,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err1 := task.validateSchema(schema1) + err1 := task.validateSchema(context.TODO(), schema1) assert.ErrorIs(t, err1, merr.ErrParameterInvalid) schema2 := &schemapb.CollectionSchema{ @@ -436,7 +436,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err2 := task.validateSchema(schema2) + err2 := task.validateSchema(context.TODO(), schema2) assert.ErrorIs(t, err2, merr.ErrParameterInvalid) schema3 := &schemapb.CollectionSchema{ @@ -452,7 +452,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err3 := task.validateSchema(schema3) + err3 := task.validateSchema(context.TODO(), schema3) assert.ErrorIs(t, err3, merr.ErrParameterInvalid) schema4 := &schemapb.CollectionSchema{ @@ -468,7 +468,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err4 := task.validateSchema(schema4) + err4 := task.validateSchema(context.TODO(), schema4) assert.ErrorIs(t, err4, merr.ErrParameterInvalid) schema5 := &schemapb.CollectionSchema{ @@ -484,7 +484,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err5 := task.validateSchema(schema5) + err5 := task.validateSchema(context.TODO(), schema5) assert.ErrorIs(t, err5, merr.ErrParameterInvalid) schema6 := &schemapb.CollectionSchema{ @@ -500,7 +500,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err6 := task.validateSchema(schema6) + err6 := task.validateSchema(context.TODO(), schema6) assert.ErrorIs(t, err6, merr.ErrParameterInvalid) schema7 := &schemapb.CollectionSchema{ @@ -516,7 +516,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err7 := task.validateSchema(schema7) + err7 := task.validateSchema(context.TODO(), schema7) assert.ErrorIs(t, err7, merr.ErrParameterInvalid) schema8 := &schemapb.CollectionSchema{ @@ -532,7 +532,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err8 := task.validateSchema(schema8) + err8 := task.validateSchema(context.TODO(), schema8) assert.ErrorIs(t, err8, merr.ErrParameterInvalid) }) @@ -563,7 +563,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { }, }, } - err := task.validateSchema(schema) + err := task.validateSchema(context.TODO(), schema) assert.ErrorIs(t, err, merr.ErrParameterInvalid) }) @@ -579,7 +579,7 @@ func Test_createCollectionTask_validateSchema(t *testing.T) { Name: collectionName, Fields: []*schemapb.FieldSchema{}, } - err := task.validateSchema(schema) + err := task.validateSchema(context.TODO(), schema) assert.NoError(t, err) }) } @@ -594,7 +594,7 @@ func Test_createCollectionTask_prepareSchema(t *testing.T) { Schema: []byte("invalid schema"), }, } - err := task.prepareSchema() + err := task.prepareSchema(context.TODO()) assert.Error(t, err) }) @@ -617,7 +617,7 @@ func Test_createCollectionTask_prepareSchema(t *testing.T) { Schema: marshaledSchema, }, } - err = task.prepareSchema() + err = task.prepareSchema(context.TODO()) assert.Error(t, err) }) @@ -644,7 +644,7 @@ func Test_createCollectionTask_prepareSchema(t *testing.T) { Schema: marshaledSchema, }, } - err = task.prepareSchema() + err = task.prepareSchema(context.TODO()) assert.NoError(t, err) }) @@ -671,7 +671,7 @@ func Test_createCollectionTask_prepareSchema(t *testing.T) { Schema: marshaledSchema, }, } - err = task.prepareSchema() + err = task.prepareSchema(context.TODO()) assert.Error(t, err) }) @@ -699,7 +699,7 @@ func Test_createCollectionTask_prepareSchema(t *testing.T) { Schema: marshaledSchema, }, } - err = task.prepareSchema() + err = task.prepareSchema(context.TODO()) assert.Error(t, err) }) } diff --git a/internal/rootcoord/create_partition_task.go b/internal/rootcoord/create_partition_task.go index a2a20a55b5fc8..5090c9f63bcdf 100644 --- a/internal/rootcoord/create_partition_task.go +++ b/internal/rootcoord/create_partition_task.go @@ -52,7 +52,7 @@ func (t *createPartitionTask) Prepare(ctx context.Context) error { func (t *createPartitionTask) Execute(ctx context.Context) error { for _, partition := range t.collMeta.Partitions { if partition.PartitionName == t.Req.GetPartitionName() { - log.Warn("add duplicate partition", zap.String("collection", t.Req.GetCollectionName()), zap.String("partition", t.Req.GetPartitionName()), zap.Uint64("ts", t.GetTs())) + log.Ctx(ctx).Warn("add duplicate partition", zap.String("collection", t.Req.GetCollectionName()), zap.String("partition", t.Req.GetPartitionName()), zap.Uint64("ts", t.GetTs())) return nil } } diff --git a/internal/rootcoord/dml_channels.go b/internal/rootcoord/dml_channels.go index 8e9719614f4b3..a5e3bbb69e679 100644 --- a/internal/rootcoord/dml_channels.go +++ b/internal/rootcoord/dml_channels.go @@ -178,7 +178,7 @@ func newDmlChannels(ctx context.Context, factory msgstream.Factory, chanNamePref for i, name := range names { ms, err := factory.NewMsgStream(ctx) if err != nil { - log.Error("Failed to add msgstream", + log.Ctx(ctx).Error("Failed to add msgstream", zap.String("name", name), zap.Error(err)) panic("Failed to add msgstream") @@ -202,7 +202,7 @@ func newDmlChannels(ctx context.Context, factory msgstream.Factory, chanNamePref heap.Init(&d.channelsHeap) - log.Info("init dml channels", zap.String("prefix", chanNamePrefix), zap.Int64("num", chanNum)) + log.Ctx(ctx).Info("init dml channels", zap.String("prefix", chanNamePrefix), zap.Int64("num", chanNum)) metrics.RootCoordNumOfDMLChannel.Add(float64(chanNum)) metrics.RootCoordNumOfMsgStream.Add(float64(chanNum)) @@ -276,7 +276,7 @@ func (d *dmlChannels) getChannelNum() int { func (d *dmlChannels) getMsgStreamByName(chanName string) (*dmlMsgStream, error) { dms, ok := d.pool.Get(chanName) if !ok { - log.Error("invalid channelName", zap.String("chanName", chanName)) + log.Ctx(d.ctx).Error("invalid channelName", zap.String("chanName", chanName)) return nil, errors.Newf("invalid channel name: %s", chanName) } return dms, nil @@ -292,7 +292,7 @@ func (d *dmlChannels) broadcast(chanNames []string, pack *msgstream.MsgPack) err dms.mutex.RLock() if dms.refcnt > 0 { if _, err := dms.ms.Broadcast(pack); err != nil { - log.Error("Broadcast failed", zap.Error(err), zap.String("chanName", chanName)) + log.Ctx(d.ctx).Error("Broadcast failed", zap.Error(err), zap.String("chanName", chanName)) dms.mutex.RUnlock() return err } @@ -314,7 +314,7 @@ func (d *dmlChannels) broadcastMark(chanNames []string, pack *msgstream.MsgPack) if dms.refcnt > 0 { ids, err := dms.ms.Broadcast(pack) if err != nil { - log.Error("BroadcastMark failed", zap.Error(err), zap.String("chanName", chanName)) + log.Ctx(d.ctx).Error("BroadcastMark failed", zap.Error(err), zap.String("chanName", chanName)) dms.mutex.RUnlock() return result, err } @@ -378,12 +378,12 @@ func genChannelNames(prefix string, num int64) []string { func parseChannelNameIndex(channelName string) int { index := strings.LastIndex(channelName, "_") if index < 0 { - log.Error("invalid channelName", zap.String("chanName", channelName)) + log.Ctx(context.TODO()).Error("invalid channelName", zap.String("chanName", channelName)) panic("invalid channel name: " + channelName) } index, err := strconv.Atoi(channelName[index+1:]) if err != nil { - log.Error("invalid channelName", zap.String("chanName", channelName), zap.Error(err)) + log.Ctx(context.TODO()).Error("invalid channelName", zap.String("chanName", channelName), zap.Error(err)) panic("invalid channel name: " + channelName) } return index @@ -407,7 +407,7 @@ func getNeedChanNum(setNum int, chanMap map[typeutil.UniqueID][]string) int { panic("topic were empty") } if chanNameSet.Contain(topic) { - log.Error("duplicate topics are pre-created", zap.String("topic", topic)) + log.Ctx(context.TODO()).Error("duplicate topics are pre-created", zap.String("topic", topic)) panic("duplicate topic: " + topic) } chanNameSet.Insert(topic) @@ -416,7 +416,7 @@ func getNeedChanNum(setNum int, chanMap map[typeutil.UniqueID][]string) int { for _, chanNames := range chanMap { for _, chanName := range chanNames { if !chanNameSet.Contain(chanName) { - log.Error("invalid channel that is not in the list when pre-created topic", zap.String("chanName", chanName)) + log.Ctx(context.TODO()).Error("invalid channel that is not in the list when pre-created topic", zap.String("chanName", chanName)) panic("invalid chanName: " + chanName) } } diff --git a/internal/rootcoord/drop_collection_task.go b/internal/rootcoord/drop_collection_task.go index 06affe5e5c050..3ae2eca075888 100644 --- a/internal/rootcoord/drop_collection_task.go +++ b/internal/rootcoord/drop_collection_task.go @@ -59,7 +59,7 @@ func (t *dropCollectionTask) Execute(ctx context.Context) error { collMeta, err := t.core.meta.GetCollectionByName(ctx, t.Req.GetDbName(), t.Req.GetCollectionName(), typeutil.MaxTimestamp) if errors.Is(err, merr.ErrCollectionNotFound) || errors.Is(err, merr.ErrDatabaseNotFound) { // make dropping collection idempotent. - log.Warn("drop non-existent collection", zap.String("collection", t.Req.GetCollectionName()), zap.String("database", t.Req.GetDbName())) + log.Ctx(ctx).Warn("drop non-existent collection", zap.String("collection", t.Req.GetCollectionName()), zap.String("database", t.Req.GetDbName())) return nil } diff --git a/internal/rootcoord/drop_partition_task.go b/internal/rootcoord/drop_partition_task.go index ef8f107adb2e1..d25265f05da55 100644 --- a/internal/rootcoord/drop_partition_task.go +++ b/internal/rootcoord/drop_partition_task.go @@ -62,7 +62,7 @@ func (t *dropPartitionTask) Execute(ctx context.Context) error { } } if partID == common.InvalidPartitionID { - log.Warn("drop an non-existent partition", zap.String("collection", t.Req.GetCollectionName()), zap.String("partition", t.Req.GetPartitionName())) + log.Ctx(ctx).Warn("drop an non-existent partition", zap.String("collection", t.Req.GetCollectionName()), zap.String("partition", t.Req.GetPartitionName())) // make dropping partition idempotent. return nil } diff --git a/internal/rootcoord/list_db_task.go b/internal/rootcoord/list_db_task.go index dbc5fa061a2d5..a4e7891428f34 100644 --- a/internal/rootcoord/list_db_task.go +++ b/internal/rootcoord/list_db_task.go @@ -53,7 +53,7 @@ func (t *listDatabaseTask) Execute(ctx context.Context) error { // it will fail if the inner node server use the list database API if err != nil || curUser == util.UserRoot { if err != nil { - log.Warn("get current user from context failed", zap.Error(err)) + log.Ctx(ctx).Warn("get current user from context failed", zap.Error(err)) } privilegeDBs.Insert(util.AnyWord) return privilegeDBs, nil diff --git a/internal/rootcoord/meta_table.go b/internal/rootcoord/meta_table.go index 0901a792c5401..af77f6959a98e 100644 --- a/internal/rootcoord/meta_table.go +++ b/internal/rootcoord/meta_table.go @@ -159,7 +159,7 @@ func (mt *MetaTable) reload() error { return err } - log.Info("recover databases", zap.Int("num of dbs", len(dbs))) + log.Ctx(mt.ctx).Info("recover databases", zap.Int("num of dbs", len(dbs))) for _, db := range dbs { mt.dbName2Meta[db.Name] = db } @@ -203,7 +203,7 @@ func (mt *MetaTable) reload() error { metrics.RootCoordNumOfDatabases.Inc() metrics.RootCoordNumOfCollections.WithLabelValues(dbName).Add(float64(collectionNum)) metrics.RootCoordNumOfPartitions.WithLabelValues().Add(float64(partitionNum)) - log.Info("collections recovered from db", zap.String("db_name", dbName), + log.Ctx(mt.ctx).Info("collections recovered from db", zap.String("db_name", dbName), zap.Int64("collection_num", collectionNum), zap.Int64("partition_num", partitionNum)) } @@ -219,7 +219,7 @@ func (mt *MetaTable) reload() error { mt.aliases.insert(dbName, alias.Name, alias.CollectionID) } } - log.Info("RootCoord meta table reload done", zap.Duration("duration", record.ElapseSpan())) + log.Ctx(mt.ctx).Info("RootCoord meta table reload done", zap.Duration("duration", record.ElapseSpan())) return nil } @@ -242,7 +242,7 @@ func (mt *MetaTable) reloadWithNonDatabase() error { } if collectionNum > 0 { - log.Info("recover collections without db", zap.Int64("collection_num", collectionNum), zap.Int64("partition_num", partitionNum)) + log.Ctx(mt.ctx).Info("recover collections without db", zap.Int64("collection_num", collectionNum), zap.Int64("partition_num", partitionNum)) } aliases, err := mt.catalog.ListAliases(mt.ctx, util.NonDBID, typeutil.MaxTimestamp) @@ -315,7 +315,7 @@ func (mt *MetaTable) AlterDatabase(ctx context.Context, oldDB *model.Database, n return err } mt.dbName2Meta[oldDB.Name] = newDB - log.Info("alter database finished", zap.String("dbName", oldDB.Name), zap.Uint64("ts", ts)) + log.Ctx(ctx).Info("alter database finished", zap.String("dbName", oldDB.Name), zap.Uint64("ts", ts)) return nil } @@ -329,11 +329,11 @@ func (mt *MetaTable) DropDatabase(ctx context.Context, dbName string, ts typeuti db, err := mt.getDatabaseByNameInternal(ctx, dbName, typeutil.MaxTimestamp) if err != nil { - log.Warn("not found database", zap.String("db", dbName)) + log.Ctx(ctx).Warn("not found database", zap.String("db", dbName)) return nil } - colls, err := mt.listCollectionFromCache(dbName, true) + colls, err := mt.listCollectionFromCache(ctx, dbName, true) if err != nil { return err } @@ -382,10 +382,10 @@ func (mt *MetaTable) GetDatabaseByName(ctx context.Context, dbName string, ts Ti return mt.getDatabaseByNameInternal(ctx, dbName, ts) } -func (mt *MetaTable) getDatabaseByNameInternal(_ context.Context, dbName string, _ Timestamp) (*model.Database, error) { +func (mt *MetaTable) getDatabaseByNameInternal(ctx context.Context, dbName string, _ Timestamp) (*model.Database, error) { // backward compatibility for rolling upgrade if dbName == "" { - log.Warn("db name is empty") + log.Ctx(ctx).Warn("db name is empty") dbName = util.DefaultDBName } @@ -503,7 +503,7 @@ func (mt *MetaTable) RemoveCollection(ctx context.Context, collectionID UniqueID // which is bigger than `ts1`. So we assume that ts should always be the latest. coll, ok := mt.collID2Meta[collectionID] if !ok { - log.Warn("not found collection, skip remove", zap.Int64("collectionID", collectionID)) + log.Ctx(ctx).Warn("not found collection, skip remove", zap.Int64("collectionID", collectionID)) return nil } @@ -613,7 +613,7 @@ func (mt *MetaTable) GetCollectionByName(ctx context.Context, dbName string, col func (mt *MetaTable) getCollectionByNameInternal(ctx context.Context, dbName string, collectionName string, ts Timestamp) (*model.Collection, error) { // backward compatibility for rolling upgrade if dbName == "" { - log.Warn("db name is empty", zap.String("collectionName", collectionName), zap.Uint64("ts", ts)) + log.Ctx(ctx).Warn("db name is empty", zap.String("collectionName", collectionName), zap.Uint64("ts", ts)) dbName = util.DefaultDBName } @@ -690,7 +690,7 @@ func (mt *MetaTable) ListCollections(ctx context.Context, dbName string, ts Time defer mt.ddLock.RUnlock() if isMaxTs(ts) { - return mt.listCollectionFromCache(dbName, onlyAvail) + return mt.listCollectionFromCache(ctx, dbName, onlyAvail) } db, err := mt.getDatabaseByNameInternal(ctx, dbName, typeutil.MaxTimestamp) @@ -714,10 +714,10 @@ func (mt *MetaTable) ListCollections(ctx context.Context, dbName string, ts Time return onlineCollections, nil } -func (mt *MetaTable) listCollectionFromCache(dbName string, onlyAvail bool) ([]*model.Collection, error) { +func (mt *MetaTable) listCollectionFromCache(ctx context.Context, dbName string, onlyAvail bool) ([]*model.Collection, error) { // backward compatibility for rolling upgrade if dbName == "" { - log.Warn("db name is empty") + log.Ctx(ctx).Warn("db name is empty") dbName = util.DefaultDBName } @@ -763,7 +763,7 @@ func (mt *MetaTable) AlterCollection(ctx context.Context, oldColl *model.Collect return err } mt.collID2Meta[oldColl.CollectionID] = newColl - log.Info("alter collection finished", zap.Int64("collectionID", oldColl.CollectionID), zap.Uint64("ts", ts)) + log.Ctx(ctx).Info("alter collection finished", zap.Int64("collectionID", oldColl.CollectionID), zap.Uint64("ts", ts)) return nil } @@ -973,7 +973,7 @@ func (mt *MetaTable) RemovePartition(ctx context.Context, dbID int64, collection coll.Partitions = append(coll.Partitions[:loc], coll.Partitions[loc+1:]...) mt.generalCnt -= int(coll.ShardsNum) // 1 partition * shardNum } - log.Info("remove partition", zap.Int64("collection", collectionID), zap.Int64("partition", partitionID), zap.Uint64("ts", ts)) + log.Ctx(ctx).Info("remove partition", zap.Int64("collection", collectionID), zap.Int64("partition", partitionID), zap.Uint64("ts", ts)) return nil } @@ -982,7 +982,7 @@ func (mt *MetaTable) CreateAlias(ctx context.Context, dbName string, alias strin defer mt.ddLock.Unlock() // backward compatibility for rolling upgrade if dbName == "" { - log.Warn("db name is empty", zap.String("alias", alias), zap.String("collection", collectionName)) + log.Ctx(ctx).Warn("db name is empty", zap.String("alias", alias), zap.String("collection", collectionName)) dbName = util.DefaultDBName } @@ -1013,7 +1013,7 @@ func (mt *MetaTable) CreateAlias(ctx context.Context, dbName string, alias strin // check if alias exists. aliasedCollectionID, ok := mt.aliases.get(dbName, alias) if ok && aliasedCollectionID == collectionID { - log.Warn("add duplicate alias", zap.String("alias", alias), zap.String("collection", collectionName), zap.Uint64("ts", ts)) + log.Ctx(ctx).Warn("add duplicate alias", zap.String("alias", alias), zap.String("collection", collectionName), zap.Uint64("ts", ts)) return nil } else if ok { // TODO: better to check if aliasedCollectionID exist or is available, though not very possible. @@ -1058,7 +1058,7 @@ func (mt *MetaTable) DropAlias(ctx context.Context, dbName string, alias string, defer mt.ddLock.Unlock() // backward compatibility for rolling upgrade if dbName == "" { - log.Warn("db name is empty", zap.String("alias", alias), zap.Uint64("ts", ts)) + log.Ctx(ctx).Warn("db name is empty", zap.String("alias", alias), zap.Uint64("ts", ts)) dbName = util.DefaultDBName } @@ -1087,7 +1087,7 @@ func (mt *MetaTable) AlterAlias(ctx context.Context, dbName string, alias string defer mt.ddLock.Unlock() // backward compatibility for rolling upgrade if dbName == "" { - log.Warn("db name is empty", zap.String("alias", alias), zap.String("collection", collectionName)) + log.Ctx(ctx).Warn("db name is empty", zap.String("alias", alias), zap.String("collection", collectionName)) dbName = util.DefaultDBName } @@ -1154,7 +1154,7 @@ func (mt *MetaTable) DescribeAlias(ctx context.Context, dbName string, alias str defer mt.ddLock.Unlock() if dbName == "" { - log.Warn("db name is empty", zap.String("alias", alias)) + log.Ctx(ctx).Warn("db name is empty", zap.String("alias", alias)) dbName = util.DefaultDBName } @@ -1184,7 +1184,7 @@ func (mt *MetaTable) ListAliases(ctx context.Context, dbName string, collectionN defer mt.ddLock.Unlock() if dbName == "" { - log.Warn("db name is empty", zap.String("collection", collectionName)) + log.Ctx(ctx).Warn("db name is empty", zap.String("collection", collectionName)) dbName = util.DefaultDBName } @@ -1262,7 +1262,7 @@ func (mt *MetaTable) AddCredential(ctx context.Context, credInfo *internalpb.Cre } if len(usernames) >= Params.ProxyCfg.MaxUserNum.GetAsInt() { errMsg := "unable to add user because the number of users has reached the limit" - log.Error(errMsg, zap.Int("max_user_num", Params.ProxyCfg.MaxUserNum.GetAsInt())) + log.Ctx(ctx).Error(errMsg, zap.Int("max_user_num", Params.ProxyCfg.MaxUserNum.GetAsInt())) return errors.New(errMsg) } @@ -1332,18 +1332,18 @@ func (mt *MetaTable) CreateRole(ctx context.Context, tenant string, entity *milv results, err := mt.catalog.ListRole(ctx, tenant, nil, false) if err != nil { - log.Warn("fail to list roles", zap.Error(err)) + log.Ctx(ctx).Warn("fail to list roles", zap.Error(err)) return err } for _, result := range results { if result.GetRole().GetName() == entity.Name { - log.Info("role already exists", zap.String("role", entity.Name)) + log.Ctx(ctx).Info("role already exists", zap.String("role", entity.Name)) return common.NewIgnorableError(errors.Newf("role [%s] already exists", entity)) } } if len(results) >= Params.ProxyCfg.MaxRoleNum.GetAsInt() { errMsg := "unable to create role because the number of roles has reached the limit" - log.Warn(errMsg, zap.Int("max_role_num", Params.ProxyCfg.MaxRoleNum.GetAsInt())) + log.Ctx(ctx).Warn(errMsg, zap.Int("max_role_num", Params.ProxyCfg.MaxRoleNum.GetAsInt())) return errors.New(errMsg) } @@ -1610,7 +1610,7 @@ func (mt *MetaTable) OperatePrivilegeGroup(ctx context.Context, groupName string // merge with current privileges group, err := mt.catalog.GetPrivilegeGroup(ctx, groupName) if err != nil { - log.Warn("fail to get privilege group", zap.String("privilege_group", groupName), zap.Error(err)) + log.Ctx(ctx).Warn("fail to get privilege group", zap.String("privilege_group", groupName), zap.Error(err)) return err } privSet := lo.SliceToMap(group.Privileges, func(p *milvuspb.PrivilegeEntity) (string, struct{}) { @@ -1626,7 +1626,7 @@ func (mt *MetaTable) OperatePrivilegeGroup(ctx context.Context, groupName string delete(privSet, p.Name) } default: - log.Warn("unsupported operate type", zap.Any("operate_type", operateType)) + log.Ctx(ctx).Warn("unsupported operate type", zap.Any("operate_type", operateType)) return fmt.Errorf("unsupported operate type: %v", operateType) } diff --git a/internal/rootcoord/meta_table_test.go b/internal/rootcoord/meta_table_test.go index 3503b2f27de3d..a143cbcf97431 100644 --- a/internal/rootcoord/meta_table_test.go +++ b/internal/rootcoord/meta_table_test.go @@ -1947,17 +1947,17 @@ func TestMetaTable_EmtpyDatabaseName(t *testing.T) { }, } - ret, err := mt.listCollectionFromCache("none", false) + ret, err := mt.listCollectionFromCache(context.TODO(), "none", false) assert.Error(t, err) assert.Nil(t, ret) - ret, err = mt.listCollectionFromCache("", false) + ret, err = mt.listCollectionFromCache(context.TODO(), "", false) assert.NoError(t, err) assert.Equal(t, 2, len(ret)) assert.Contains(t, []int64{1, 2}, ret[0].CollectionID) assert.Contains(t, []int64{1, 2}, ret[1].CollectionID) - ret, err = mt.listCollectionFromCache("db2", false) + ret, err = mt.listCollectionFromCache(context.TODO(), "db2", false) assert.NoError(t, err) assert.Equal(t, 1, len(ret)) assert.Equal(t, int64(3), ret[0].CollectionID) diff --git a/internal/rootcoord/root_coord.go b/internal/rootcoord/root_coord.go index 13bff93517145..7b50e1446f5cc 100644 --- a/internal/rootcoord/root_coord.go +++ b/internal/rootcoord/root_coord.go @@ -63,7 +63,6 @@ import ( "github.com/milvus-io/milvus/pkg/util/crypto" "github.com/milvus-io/milvus/pkg/util/expr" "github.com/milvus-io/milvus/pkg/util/funcutil" - "github.com/milvus-io/milvus/pkg/util/logutil" "github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/metricsinfo" "github.com/milvus-io/milvus/pkg/util/paramtable" @@ -157,7 +156,7 @@ func NewCore(c context.Context, factory dependency.Factory) (*Core, error) { // UpdateStateCode update state code func (c *Core) UpdateStateCode(code commonpb.StateCode) { c.stateCode.Store(int32(code)) - log.Info("update rootcoord state", zap.String("state", code.String())) + log.Ctx(c.ctx).Info("update rootcoord state", zap.String("state", code.String())) } func (c *Core) GetStateCode() commonpb.StateCode { @@ -187,6 +186,7 @@ func (c *Core) sendMinDdlTsAsTt() { if !paramtable.Get().CommonCfg.TTMsgEnabled.GetAsBool() { return } + log := log.Ctx(c.ctx) code := c.GetStateCode() if code != commonpb.StateCode_Healthy { log.Warn("rootCoord is not healthy, skip send timetick") @@ -214,6 +214,7 @@ func (c *Core) sendMinDdlTsAsTt() { } func (c *Core) startTimeTickLoop() { + log := log.Ctx(c.ctx) defer c.wg.Done() ticker := time.NewTicker(Params.ProxyCfg.TimeTickInterval.GetAsDuration(time.Millisecond)) defer ticker.Stop() @@ -234,6 +235,7 @@ func (c *Core) tsLoop() { defer tsoTicker.Stop() ctx, cancel := context.WithCancel(c.ctx) defer cancel() + log := log.Ctx(c.ctx) for { select { case <-tsoTicker.C: @@ -273,6 +275,7 @@ func (c *Core) SetQueryCoordClient(s types.QueryCoordClient) error { // Register register rootcoord at etcd func (c *Core) Register() error { + log := log.Ctx(c.ctx) c.session.Register() afterRegister := func() { metrics.NumNodes.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), typeutil.RootCoordRole).Inc() @@ -338,6 +341,7 @@ func (c *Core) initKVCreator() { } func (c *Core) initMetaTable() error { + log := log.Ctx(c.ctx) fn := func() error { var catalog metastore.RootCoordCatalog var err error @@ -401,7 +405,7 @@ func (c *Core) initIDAllocator() error { } c.idAllocator = idAllocator - log.Info("id allocator initialized", + log.Ctx(c.ctx).Info("id allocator initialized", zap.String("root_path", kvPath), zap.String("sub_path", globalIDAllocatorSubPath), zap.String("key", globalIDAllocatorKey)) @@ -425,7 +429,7 @@ func (c *Core) initTSOAllocator() error { } c.tsoAllocator = tsoAllocator - log.Info("tso allocator initialized", + log.Ctx(c.ctx).Info("tso allocator initialized", zap.String("root_path", kvPath), zap.String("sub_path", globalIDAllocatorSubPath), zap.String("key", globalIDAllocatorKey)) @@ -434,6 +438,7 @@ func (c *Core) initTSOAllocator() error { } func (c *Core) initInternal() error { + log := log.Ctx(c.ctx) c.UpdateStateCode(commonpb.StateCode_Initializing) c.initKVCreator() @@ -504,11 +509,12 @@ func (c *Core) registerMetricsRequest() { func(ctx context.Context, req *milvuspb.GetMetricsRequest, jsonReq gjson.Result) (string, error) { return c.getSystemInfoMetrics(ctx, req) }) - log.Info("register metrics actions finished") + log.Ctx(c.ctx).Info("register metrics actions finished") } // Init initialize routine func (c *Core) Init() error { + log := log.Ctx(c.ctx) var initError error c.registerMetricsRequest() c.factory.Init(Params) @@ -549,6 +555,7 @@ func (c *Core) Init() error { } func (c *Core) initCredentials() error { + log := log.Ctx(c.ctx) credInfo, _ := c.meta.GetCredential(c.ctx, util.UserRoot) if credInfo == nil { encryptedRootPassword, err := crypto.PasswordEncrypt(Params.CommonCfg.DefaultRootPassword.GetValue()) @@ -674,6 +681,7 @@ func (c *Core) initBuiltinPrivilegeGroups() []*milvuspb.PrivilegeGroupInfo { } func (c *Core) initBuiltinRoles() error { + log := log.Ctx(c.ctx) rolePrivilegesMap := Params.RoleCfg.Roles.GetAsRoleDetails() for role, privilegesJSON := range rolePrivilegesMap { err := c.meta.CreateRole(c.ctx, util.DefaultTenant, &milvuspb.RoleEntity{Name: role}) @@ -752,6 +760,7 @@ func (c *Core) restore(ctx context.Context) error { } func (c *Core) startInternal() error { + log := log.Ctx(c.ctx) if err := c.proxyWatcher.WatchProxy(c.ctx); err != nil { log.Fatal("rootcoord failed to watch proxy", zap.Error(err)) // you can not just stuck here, @@ -786,7 +795,7 @@ func (c *Core) startInternal() error { c.startServerLoop() c.UpdateStateCode(commonpb.StateCode_Healthy) sessionutil.SaveServerInfo(typeutil.RootCoordRole, c.session.ServerID) - logutil.Logger(c.ctx).Info("rootcoord startup successfully") + log.Info("rootcoord startup successfully") return nil } @@ -816,21 +825,21 @@ func (c *Core) Start() error { func (c *Core) stopExecutor() { if c.stepExecutor != nil { c.stepExecutor.Stop() - log.Info("stop rootcoord executor") + log.Ctx(c.ctx).Info("stop rootcoord executor") } } func (c *Core) stopScheduler() { if c.scheduler != nil { c.scheduler.Stop() - log.Info("stop rootcoord scheduler") + log.Ctx(c.ctx).Info("stop rootcoord scheduler") } } func (c *Core) cancelIfNotNil() { if c.cancel != nil { c.cancel() - log.Info("cancel rootcoord goroutines") + log.Ctx(c.ctx).Info("cancel rootcoord goroutines") } } @@ -838,7 +847,7 @@ func (c *Core) revokeSession() { if c.session != nil { // wait at most one second to revoke c.session.Stop() - log.Info("rootcoord session stop") + log.Ctx(c.ctx).Info("rootcoord session stop") } } @@ -863,7 +872,7 @@ func (c *Core) Stop() error { // GetComponentStates get states of components func (c *Core) GetComponentStates(ctx context.Context, req *milvuspb.GetComponentStatesRequest) (*milvuspb.ComponentStates, error) { code := c.GetStateCode() - log.Debug("RootCoord current state", zap.String("StateCode", code.String())) + log.Ctx(ctx).Debug("RootCoord current state", zap.String("StateCode", code.String())) nodeID := common.NotRegisteredID if c.session != nil && c.session.Registered() { @@ -2098,8 +2107,7 @@ func (c *Core) CreateCredential(ctx context.Context, credInfo *internalpb.Creden ctxLog.Warn("CreateCredential add cache failed", zap.Error(err)) metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc() } - log.Debug("CreateCredential success", zap.String("role", typeutil.RootCoordRole), - zap.String("username", credInfo.Username)) + ctxLog.Debug("CreateCredential success") metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc() metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds())) @@ -2161,7 +2169,7 @@ func (c *Core) UpdateCredential(ctx context.Context, credInfo *internalpb.Creden metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.FailLabel).Inc() return merr.StatusWithErrorCode(err, commonpb.ErrorCode_UpdateCredentialFailure), nil } - log.Debug("UpdateCredential success") + ctxLog.Debug("UpdateCredential success") metrics.RootCoordDDLReqCounter.WithLabelValues(method, metrics.SuccessLabel).Inc() metrics.RootCoordDDLReqLatency.WithLabelValues(method).Observe(float64(tr.ElapseSpan().Milliseconds())) @@ -2401,7 +2409,7 @@ func (c *Core) OperateUserRole(ctx context.Context, in *milvuspb.OperateUserRole redoTask.AddSyncStep(NewSimpleStep("operate user role meta data", func(ctx context.Context) ([]nestedStep, error) { err := c.meta.OperateUserRole(ctx, util.DefaultTenant, &milvuspb.UserEntity{Name: in.Username}, &milvuspb.RoleEntity{Name: in.RoleName}, in.Type) if err != nil && !common.IsIgnorableError(err) { - log.Warn("operate user role mata data failed", zap.Error(err)) + ctxLog.Warn("operate user role mata data failed", zap.Error(err)) return nil, err } return nil, nil @@ -2415,14 +2423,14 @@ func (c *Core) OperateUserRole(ctx context.Context, in *milvuspb.OperateUserRole opType = int32(typeutil.CacheRemoveUserFromRole) default: errMsg := "invalid operate type for the OperateUserRole api" - log.Warn(errMsg, zap.Any("in", in)) + ctxLog.Warn(errMsg, zap.Any("in", in)) return nil, nil } if err := c.proxyClientManager.RefreshPolicyInfoCache(ctx, &proxypb.RefreshPolicyInfoCacheRequest{ OpType: opType, OpKey: funcutil.EncodeUserRoleCache(in.Username, in.RoleName), }); err != nil { - log.Warn("fail to refresh policy info cache", zap.Any("in", in), zap.Error(err)) + ctxLog.Warn("fail to refresh policy info cache", zap.Any("in", in), zap.Error(err)) return nil, err } return nil, nil @@ -2430,7 +2438,7 @@ func (c *Core) OperateUserRole(ctx context.Context, in *milvuspb.OperateUserRole err := redoTask.Execute(ctx) if err != nil { errMsg := "fail to execute task when operate the user and role" - log.Warn(errMsg, zap.Error(err)) + ctxLog.Warn(errMsg, zap.Error(err)) return merr.StatusWithErrorCode(errors.New(errMsg), commonpb.ErrorCode_OperateUserRoleFailure), nil } @@ -2657,7 +2665,7 @@ func (c *Core) OperatePrivilege(ctx context.Context, in *milvuspb.OperatePrivile err := c.meta.OperatePrivilege(ctx, util.DefaultTenant, in.Entity, in.Type) if err != nil && !common.IsIgnorableError(err) { - log.Warn("fail to operate the privilege", zap.Any("in", in), zap.Error(err)) + ctxLog.Warn("fail to operate the privilege", zap.Any("in", in), zap.Error(err)) return nil, err } return nil, nil @@ -2702,7 +2710,7 @@ func (c *Core) OperatePrivilege(ctx context.Context, in *milvuspb.OperatePrivile err := redoTask.Execute(ctx) if err != nil { errMsg := "fail to execute task when operating the privilege" - log.Warn(errMsg, zap.Error(err)) + ctxLog.Warn(errMsg, zap.Error(err)) return merr.StatusWithErrorCode(err, commonpb.ErrorCode_OperatePrivilegeFailure), nil } @@ -2738,7 +2746,7 @@ func (c *Core) operatePrivilegeCommonCheck(ctx context.Context, in *milvuspb.Ope return errors.New("the user entity in the grantor entity is nil or empty") } if _, err := c.meta.SelectUser(ctx, util.DefaultTenant, &milvuspb.UserEntity{Name: entity.User.Name}, false); err != nil { - log.Warn("fail to select the user", zap.String("username", entity.User.Name), zap.Error(err)) + log.Ctx(ctx).Warn("fail to select the user", zap.String("username", entity.User.Name), zap.Error(err)) return errors.New("not found the user, maybe the user isn't existed or internal system error") } if entity.Privilege == nil { @@ -2919,7 +2927,7 @@ func (c *Core) RestoreRBAC(ctx context.Context, in *milvuspb.RestoreRBACMetaRequ redoTask := newBaseRedoTask(c.stepExecutor) redoTask.AddSyncStep(NewSimpleStep("restore rbac meta data", func(ctx context.Context) ([]nestedStep, error) { if err := c.meta.RestoreRBAC(ctx, util.DefaultTenant, in.RBACMeta); err != nil { - log.Warn("fail to restore rbac meta data", zap.Any("in", in), zap.Error(err)) + ctxLog.Warn("fail to restore rbac meta data", zap.Any("in", in), zap.Error(err)) return nil, err } return nil, nil @@ -2928,7 +2936,7 @@ func (c *Core) RestoreRBAC(ctx context.Context, in *milvuspb.RestoreRBACMetaRequ if err := c.proxyClientManager.RefreshPolicyInfoCache(c.ctx, &proxypb.RefreshPolicyInfoCacheRequest{ OpType: int32(typeutil.CacheRefresh), }); err != nil { - log.Warn("fail to refresh policy info cache", zap.Any("in", in), zap.Error(err)) + ctxLog.Warn("fail to refresh policy info cache", zap.Any("in", in), zap.Error(err)) return nil, err } return nil, nil @@ -2937,7 +2945,7 @@ func (c *Core) RestoreRBAC(ctx context.Context, in *milvuspb.RestoreRBACMetaRequ err := redoTask.Execute(ctx) if err != nil { errMsg := "fail to execute task when restore rbac meta data" - log.Warn(errMsg, zap.Error(err)) + ctxLog.Warn(errMsg, zap.Error(err)) return merr.StatusWithErrorCode(err, commonpb.ErrorCode_OperatePrivilegeFailure), nil } @@ -3176,7 +3184,7 @@ func (c *Core) OperatePrivilegeGroup(ctx context.Context, in *milvuspb.OperatePr redoTask.AddSyncStep(NewSimpleStep("operate privilege group", func(ctx context.Context) ([]nestedStep, error) { groups, err := c.meta.ListPrivilegeGroups(ctx) if err != nil && !common.IsIgnorableError(err) { - log.Warn("fail to list privilege groups", zap.Error(err)) + ctxLog.Warn("fail to list privilege groups", zap.Error(err)) return nil, err } currGroups := lo.SliceToMap(groups, func(group *milvuspb.PrivilegeGroupInfo) (string, []*milvuspb.PrivilegeEntity) { @@ -3266,7 +3274,7 @@ func (c *Core) OperatePrivilegeGroup(ctx context.Context, in *milvuspb.OperatePr OpType: opType, OpKey: funcutil.PolicyForPrivileges(rolesToRevoke), }); err != nil { - log.Warn("fail to refresh policy info cache for revoke privileges in operate privilege group", zap.Any("in", in), zap.Error(err)) + ctxLog.Warn("fail to refresh policy info cache for revoke privileges in operate privilege group", zap.Any("in", in), zap.Error(err)) return nil, err } } @@ -3277,7 +3285,7 @@ func (c *Core) OperatePrivilegeGroup(ctx context.Context, in *milvuspb.OperatePr OpType: opType, OpKey: funcutil.PolicyForPrivileges(rolesToGrant), }); err != nil { - log.Warn("fail to refresh policy info cache for grants privilege in operate privilege group", zap.Any("in", in), zap.Error(err)) + ctxLog.Warn("fail to refresh policy info cache for grants privilege in operate privilege group", zap.Any("in", in), zap.Error(err)) return nil, err } } @@ -3287,7 +3295,7 @@ func (c *Core) OperatePrivilegeGroup(ctx context.Context, in *milvuspb.OperatePr redoTask.AddSyncStep(NewSimpleStep("operate privilege group meta data", func(ctx context.Context) ([]nestedStep, error) { err := c.meta.OperatePrivilegeGroup(ctx, in.GroupName, in.Privileges, in.Type) if err != nil && !common.IsIgnorableError(err) { - log.Warn("fail to operate privilege group", zap.Error(err)) + ctxLog.Warn("fail to operate privilege group", zap.Error(err)) } return nil, err })) diff --git a/internal/rootcoord/show_collection_task.go b/internal/rootcoord/show_collection_task.go index f5a3fa5eeb725..b0f40be928d61 100644 --- a/internal/rootcoord/show_collection_task.go +++ b/internal/rootcoord/show_collection_task.go @@ -60,7 +60,7 @@ func (t *showCollectionTask) Execute(ctx context.Context) error { curUser, err := contextutil.GetCurUserFromContext(ctx) if err != nil || curUser == util.UserRoot { if err != nil { - log.Warn("get current user from context failed", zap.Error(err)) + log.Ctx(ctx).Warn("get current user from context failed", zap.Error(err)) } privilegeColls.Insert(util.AnyWord) return privilegeColls, nil diff --git a/internal/rootcoord/step.go b/internal/rootcoord/step.go index 9b18df3a3c38e..bf5c9f7acf166 100644 --- a/internal/rootcoord/step.go +++ b/internal/rootcoord/step.go @@ -21,6 +21,8 @@ import ( "fmt" "time" + "go.uber.org/zap" + "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" "github.com/milvus-io/milvus-proto/go-api/v2/msgpb" @@ -28,6 +30,7 @@ import ( "github.com/milvus-io/milvus/internal/metastore/model" pb "github.com/milvus-io/milvus/internal/proto/etcdpb" "github.com/milvus-io/milvus/internal/util/proxyutil" + "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/streaming/util/message" "github.com/milvus-io/milvus/pkg/util/commonpbutil" ) @@ -297,6 +300,7 @@ type releaseCollectionStep struct { func (s *releaseCollectionStep) Execute(ctx context.Context) ([]nestedStep, error) { err := s.core.broker.ReleaseCollection(ctx, s.collectionID) + log.Ctx(ctx).Info("release collection done", zap.Int64("collectionID", s.collectionID)) return nil, err } diff --git a/internal/rootcoord/step_executor.go b/internal/rootcoord/step_executor.go index 5e63faadd86b5..bbadf6252eee8 100644 --- a/internal/rootcoord/step_executor.go +++ b/internal/rootcoord/step_executor.go @@ -65,14 +65,14 @@ func (s *stepStack) Execute(ctx context.Context) *stepStack { if !retry.IsRecoverable(err) { if !skipLog { - log.Warn("failed to execute step, not able to reschedule", zap.Error(err), zap.String("step", todo.Desc())) + log.Ctx(ctx).Warn("failed to execute step, not able to reschedule", zap.Error(err), zap.String("step", todo.Desc())) } return nil } if err != nil { s.steps = nil // let's can be collected. if !skipLog { - log.Warn("failed to execute step, wait for reschedule", zap.Error(err), zap.String("step", todo.Desc())) + log.Ctx(ctx).Warn("failed to execute step, wait for reschedule", zap.Error(err), zap.String("step", todo.Desc())) } return &stepStack{steps: steps} } diff --git a/internal/rootcoord/timeticksync.go b/internal/rootcoord/timeticksync.go index 7dc0f81fc77e3..1b4ea74539d13 100644 --- a/internal/rootcoord/timeticksync.go +++ b/internal/rootcoord/timeticksync.go @@ -124,7 +124,7 @@ func newTimeTickSync(ctx context.Context, sourceID int64, factory msgstream.Fact // recover physical channels for all collections for collID, chanNames := range chanMap { dmlChannels.addChannels(chanNames...) - log.Info("recover physical channels", zap.Int64("collectionID", collID), zap.Strings("physical channels", chanNames)) + log.Ctx(ctx).Info("recover physical channels", zap.Int64("collectionID", collID), zap.Strings("physical channels", chanNames)) } return &timetickSync{ diff --git a/internal/rootcoord/undo.go b/internal/rootcoord/undo.go index 29e9b7dfac05b..f41e00394b292 100644 --- a/internal/rootcoord/undo.go +++ b/internal/rootcoord/undo.go @@ -52,7 +52,7 @@ func (b *baseUndoTask) Execute(ctx context.Context) error { todoStep := b.todoStep[i] // no children step in normal case. if _, err := todoStep.Execute(ctx); err != nil { - log.Warn("failed to execute step, trying to undo", zap.Error(err), zap.String("desc", todoStep.Desc())) + log.Ctx(ctx).Warn("failed to execute step, trying to undo", zap.Error(err), zap.String("desc", todoStep.Desc())) undoSteps := b.undoStep[:i] b.undoStep = nil // let baseUndoTask can be collected. go b.stepExecutor.AddSteps(&stepStack{undoSteps}) diff --git a/internal/streamingnode/client/manager/manager_client_impl.go b/internal/streamingnode/client/manager/manager_client_impl.go index a886165c78f51..d07b2b4ce1d30 100644 --- a/internal/streamingnode/client/manager/manager_client_impl.go +++ b/internal/streamingnode/client/manager/manager_client_impl.go @@ -87,6 +87,7 @@ func (c *managerClientImpl) CollectAllStatus(ctx context.Context) (map[int64]*ty } func (c *managerClientImpl) getAllStreamingNodeStatus(ctx context.Context, state discoverer.VersionedState) (map[int64]*types.StreamingNodeStatus, error) { + log := log.Ctx(ctx) // wait for manager service ready. manager, err := c.service.GetService(ctx) if err != nil { diff --git a/internal/util/flowgraph/node.go b/internal/util/flowgraph/node.go index 4922fcae16a3b..9cb65609ae0ce 100644 --- a/internal/util/flowgraph/node.go +++ b/internal/util/flowgraph/node.go @@ -17,6 +17,7 @@ package flowgraph import ( + "context" "fmt" "sync" "time" @@ -201,7 +202,7 @@ func (nodeCtx *nodeCtx) Close() { if nodeCtx.checker != nil { nodeCtx.checker.Close() } - log.Debug("flow graph node closed", zap.String("nodeName", nodeCtx.node.Name())) + log.Ctx(context.TODO()).Debug("flow graph node closed", zap.String("nodeName", nodeCtx.node.Name())) nodeCtx = nodeCtx.downstream } } diff --git a/internal/util/grpcclient/client.go b/internal/util/grpcclient/client.go index 8927b44d207b1..c03bb5572d01e 100644 --- a/internal/util/grpcclient/client.go +++ b/internal/util/grpcclient/client.go @@ -264,7 +264,7 @@ func (c *ClientBase[T]) connect(ctx context.Context) error { compress = Zstd } if c.encryption { - log.Debug("Running in internalTLS mode with encryption enabled") + log.Ctx(ctx).Debug("Running in internalTLS mode with encryption enabled") conn, err = grpc.DialContext( dialContext, addr, diff --git a/internal/util/metrics/thread.go b/internal/util/metrics/thread.go index dea0fda8ae5d8..ae4af3605081b 100644 --- a/internal/util/metrics/thread.go +++ b/internal/util/metrics/thread.go @@ -17,6 +17,7 @@ package metrics import ( + "context" "os" "sync" "time" @@ -55,6 +56,7 @@ func (thw *threadWatcher) Start() { } func (thw *threadWatcher) watchThreadNum() { + log := log.Ctx(context.TODO()) ticker := time.NewTicker(time.Second * 30) defer ticker.Stop() pid := os.Getpid() diff --git a/internal/util/pipeline/stream_pipeline.go b/internal/util/pipeline/stream_pipeline.go index d340fb3829399..46f086cb97992 100644 --- a/internal/util/pipeline/stream_pipeline.go +++ b/internal/util/pipeline/stream_pipeline.go @@ -64,11 +64,11 @@ func (p *streamPipeline) work() { for { select { case <-p.closeCh: - log.Debug("stream pipeline input closed") + log.Ctx(context.TODO()).Debug("stream pipeline input closed") return case msg := <-p.input: p.lastAccessTime.Store(time.Now()) - log.RatedDebug(10, "stream pipeline fetch msg", zap.Int("sum", len(msg.Msgs))) + log.Ctx(context.TODO()).RatedDebug(10, "stream pipeline fetch msg", zap.Int("sum", len(msg.Msgs))) p.pipeline.inputChannel <- msg p.pipeline.process() } @@ -86,6 +86,7 @@ func (p *streamPipeline) Status() string { } func (p *streamPipeline) ConsumeMsgStream(ctx context.Context, position *msgpb.MsgPosition) error { + log := log.Ctx(ctx) var err error if position == nil { log.Error("seek stream to nil position") diff --git a/internal/util/proxyutil/proxy_watcher.go b/internal/util/proxyutil/proxy_watcher.go index decc1e6420b1b..529024736e655 100644 --- a/internal/util/proxyutil/proxy_watcher.go +++ b/internal/util/proxyutil/proxy_watcher.go @@ -166,7 +166,7 @@ func (p *ProxyWatcher) handlePutEvent(e *clientv3.Event) error { if err != nil { return err } - log.Debug("received proxy put event with session", zap.Any("session", session)) + log.Ctx(context.TODO()).Debug("received proxy put event with session", zap.Any("session", session)) for _, f := range p.addSessionsFunc { f(session) } @@ -178,7 +178,7 @@ func (p *ProxyWatcher) handleDeleteEvent(e *clientv3.Event) error { if err != nil { return err } - log.Debug("received proxy delete event with session", zap.Any("session", session)) + log.Ctx(context.TODO()).Debug("received proxy delete event with session", zap.Any("session", session)) for _, f := range p.delSessionsFunc { f(session) } diff --git a/internal/util/sessionutil/session_util.go b/internal/util/sessionutil/session_util.go index 5358fc6955db4..3ed57a9245907 100644 --- a/internal/util/sessionutil/session_util.go +++ b/internal/util/sessionutil/session_util.go @@ -231,7 +231,7 @@ func NewSession(ctx context.Context, opts ...SessionOption) *Session { func NewSessionWithEtcd(ctx context.Context, metaRoot string, client *clientv3.Client, opts ...SessionOption) *Session { hostName, hostNameErr := os.Hostname() if hostNameErr != nil { - log.Error("get host name fail", zap.Error(hostNameErr)) + log.Ctx(ctx).Error("get host name fail", zap.Error(hostNameErr)) } session := &Session{ @@ -260,7 +260,7 @@ func NewSessionWithEtcd(ctx context.Context, metaRoot string, client *clientv3.C session.UpdateRegistered(false) connectEtcdFn := func() error { - log.Debug("Session try to connect to etcd") + log.Ctx(ctx).Debug("Session try to connect to etcd") ctx2, cancel2 := context.WithTimeout(session.ctx, 5*time.Second) defer cancel2() if _, err := client.Get(ctx2, "health"); err != nil { @@ -271,11 +271,11 @@ func NewSessionWithEtcd(ctx context.Context, metaRoot string, client *clientv3.C } err := retry.Do(ctx, connectEtcdFn, retry.Attempts(100)) if err != nil { - log.Warn("failed to initialize session", + log.Ctx(ctx).Warn("failed to initialize session", zap.Error(err)) return nil } - log.Debug("Session connect to etcd success") + log.Ctx(ctx).Debug("Session connect to etcd success") return session } @@ -319,7 +319,7 @@ func (s *Session) getServerID() (int64, error) { serverIDMu.Lock() defer serverIDMu.Unlock() - log.Debug("getServerID", zap.Bool("reuse", s.reuseNodeID)) + log.Ctx(s.ctx).Debug("getServerID", zap.Bool("reuse", s.reuseNodeID)) if s.reuseNodeID { // Notice, For standalone, all process share the same nodeID. if nodeID := paramtable.GetNodeID(); nodeID != 0 { @@ -365,6 +365,7 @@ func (s *Session) checkIDExist() { } func (s *Session) getServerIDWithKey(key string) (int64, error) { + log := log.Ctx(s.ctx) for { getResp, err := s.etcdCli.Get(s.ctx, path.Join(s.metaRoot, DefaultServiceRoot, key)) if err != nil { @@ -459,6 +460,7 @@ func (s *Session) registerService() (<-chan *clientv3.LeaseKeepAliveResponse, er } completeKey := s.getCompleteKey() var ch <-chan *clientv3.LeaseKeepAliveResponse + log := log.Ctx(s.ctx) log.Debug("service begin to register to etcd", zap.String("serverName", s.ServerName), zap.Int64("ServerID", s.ServerID)) registerFn := func() error { @@ -617,7 +619,7 @@ func fnWithTimeout(fn func() error, d time.Duration) error { select { case <-resultChan: - log.Debug("retry func success") + log.Ctx(context.TODO()).Debug("retry func success") case <-time.After(d): return fmt.Errorf("func timed out") } @@ -643,7 +645,7 @@ func (s *Session) GetSessions(prefix string) (map[string]*Session, int64, error) return nil, 0, err } _, mapKey := path.Split(string(kv.Key)) - log.Debug("SessionUtil GetSessions", + log.Ctx(s.ctx).Debug("SessionUtil GetSessions", zap.String("prefix", prefix), zap.String("key", mapKey), zap.String("address", session.Address)) @@ -655,6 +657,7 @@ func (s *Session) GetSessions(prefix string) (map[string]*Session, int64, error) // GetSessionsWithVersionRange will get all sessions with provided prefix and version range in etcd. // Revision is returned for WatchServices to prevent missing events. func (s *Session) GetSessionsWithVersionRange(prefix string, r semver.Range) (map[string]*Session, int64, error) { + log := log.Ctx(s.ctx) res := make(map[string]*Session) key := path.Join(s.metaRoot, DefaultServiceRoot, prefix) resp, err := s.etcdCli.Get(s.ctx, key, clientv3.WithPrefix(), @@ -789,6 +792,7 @@ func (s *Session) WatchServicesWithVersionRange(prefix string, r semver.Range, r } func (w *sessionWatcher) handleWatchResponse(wresp clientv3.WatchResponse) { + log := log.Ctx(context.TODO()) if wresp.Err() != nil { err := w.handleWatchErr(wresp.Err()) if err != nil { @@ -1048,7 +1052,7 @@ func (s *Session) safeCloseLiveCh() { // activateFunc is the function to re-active the service. func (s *Session) ProcessActiveStandBy(activateFunc func() error) error { s.activeKey = path.Join(s.metaRoot, DefaultServiceRoot, s.ServerName) - + log := log.Ctx(s.ctx) // try to register to the active_key. // return // 1. doRegistered: if registered the active_key by this session or by other session diff --git a/pkg/config/etcd_source.go b/pkg/config/etcd_source.go index 9885ae746a4be..c7b84e65427a6 100644 --- a/pkg/config/etcd_source.go +++ b/pkg/config/etcd_source.go @@ -49,7 +49,7 @@ type EtcdSource struct { } func NewEtcdSource(etcdInfo *EtcdInfo) (*EtcdSource, error) { - log.Debug("init etcd source", zap.Any("etcdInfo", etcdInfo)) + log.Ctx(context.TODO()).Debug("init etcd source", zap.Any("etcdInfo", etcdInfo)) etcdCli, err := etcd.CreateEtcdClient( etcdInfo.UseEmbed, etcdInfo.EnableAuth, @@ -144,7 +144,7 @@ func (es *EtcdSource) UpdateOptions(opts Options) { } func (es *EtcdSource) refreshConfigurations() error { - log := log.Ctx(context.TODO()).WithRateGroup("config.etcdSource", 1, 60) + log := log.Ctx(es.ctx).WithRateGroup("config.etcdSource", 1, 60) es.RLock() prefix := path.Join(es.keyPrefix, "config") es.RUnlock() @@ -176,7 +176,7 @@ func (es *EtcdSource) update(configs map[string]string) error { events, err := PopulateEvents(es.GetSourceName(), es.currentConfigs, configs) if err != nil { es.Unlock() - log.Warn("generating event error", zap.Error(err)) + log.Ctx(es.ctx).Warn("generating event error", zap.Error(err)) return err } es.currentConfigs = configs diff --git a/pkg/config/refresher.go b/pkg/config/refresher.go index 2a403f5ed5156..64ea2b2714ed1 100644 --- a/pkg/config/refresher.go +++ b/pkg/config/refresher.go @@ -16,6 +16,7 @@ package config import ( + "context" "sync" "time" @@ -64,6 +65,7 @@ func (r *refresher) refreshPeriodically(name string) { defer r.wg.Done() ticker := time.NewTicker(r.refreshInterval) defer ticker.Stop() + log := log.Ctx(context.TODO()) log.Debug("start refreshing configurations", zap.String("source", name)) for { select { diff --git a/pkg/log/global.go b/pkg/log/global.go index 879da86bddb94..6a00a8d4d25ba 100644 --- a/pkg/log/global.go +++ b/pkg/log/global.go @@ -26,24 +26,28 @@ var CtxLogKey = ctxLogKeyType{} // Debug logs a message at DebugLevel. The message includes any fields passed // at the log site, as well as any fields accumulated on the logger. +// Deprecated: Use Ctx(ctx).Debug instead. func Debug(msg string, fields ...zap.Field) { L().Debug(msg, fields...) } // Info logs a message at InfoLevel. The message includes any fields passed // at the log site, as well as any fields accumulated on the logger. +// Deprecated: Use Ctx(ctx).Info instead. func Info(msg string, fields ...zap.Field) { L().Info(msg, fields...) } // Warn logs a message at WarnLevel. The message includes any fields passed // at the log site, as well as any fields accumulated on the logger. +// Deprecated: Use Ctx(ctx).Warn instead. func Warn(msg string, fields ...zap.Field) { L().Warn(msg, fields...) } // Error logs a message at ErrorLevel. The message includes any fields passed // at the log site, as well as any fields accumulated on the logger. +// Deprecated: Use Ctx(ctx).Error instead. func Error(msg string, fields ...zap.Field) { L().Error(msg, fields...) } @@ -52,6 +56,7 @@ func Error(msg string, fields ...zap.Field) { // at the log site, as well as any fields accumulated on the logger. // // The logger then panics, even if logging at PanicLevel is disabled. +// Deprecated: Use Ctx(ctx).Panic instead. func Panic(msg string, fields ...zap.Field) { L().Panic(msg, fields...) } @@ -61,6 +66,7 @@ func Panic(msg string, fields ...zap.Field) { // // The logger then calls os.Exit(1), even if logging at FatalLevel is // disabled. +// Deprecated: Use Ctx(ctx).Fatal instead. func Fatal(msg string, fields ...zap.Field) { L().Fatal(msg, fields...) } @@ -68,6 +74,7 @@ func Fatal(msg string, fields ...zap.Field) { // RatedDebug print logs at debug level // it limit log print to avoid too many logs // return true if log successfully +// Deprecated: Use Ctx(ctx).RatedDebug instead. func RatedDebug(cost float64, msg string, fields ...zap.Field) bool { if R().CheckCredit(cost) { L().Debug(msg, fields...) @@ -79,6 +86,7 @@ func RatedDebug(cost float64, msg string, fields ...zap.Field) bool { // RatedInfo print logs at info level // it limit log print to avoid too many logs // return true if log successfully +// Deprecated: Use Ctx(ctx).RatedInfo instead. func RatedInfo(cost float64, msg string, fields ...zap.Field) bool { if R().CheckCredit(cost) { L().Info(msg, fields...) @@ -90,6 +98,7 @@ func RatedInfo(cost float64, msg string, fields ...zap.Field) bool { // RatedWarn print logs at warn level // it limit log print to avoid too many logs // return true if log successfully +// Deprecated: Use Ctx(ctx).RatedWarn instead. func RatedWarn(cost float64, msg string, fields ...zap.Field) bool { if R().CheckCredit(cost) { L().Warn(msg, fields...) @@ -100,6 +109,7 @@ func RatedWarn(cost float64, msg string, fields ...zap.Field) bool { // With creates a child logger and adds structured context to it. // Fields added to the child don't affect the parent, and vice versa. +// Deprecated: Use Ctx(ctx).With instead. func With(fields ...zap.Field) *MLogger { return &MLogger{ Logger: L().WithOptions(zap.WrapCore(func(core zapcore.Core) zapcore.Core { diff --git a/pkg/mq/mqimpl/rocksmq/client/client_impl.go b/pkg/mq/mqimpl/rocksmq/client/client_impl.go index 3334f8a5532b2..d550468fedf67 100644 --- a/pkg/mq/mqimpl/rocksmq/client/client_impl.go +++ b/pkg/mq/mqimpl/rocksmq/client/client_impl.go @@ -12,6 +12,7 @@ package client import ( + "context" "reflect" "sync" @@ -79,7 +80,7 @@ func (c *client) Subscribe(options ConsumerOptions) (Consumer, error) { return nil, err } if exist { - log.Debug("ConsumerGroup already existed", zap.Any("topic", options.Topic), zap.String("SubscriptionName", options.SubscriptionName)) + log.Ctx(context.TODO()).Debug("ConsumerGroup already existed", zap.Any("topic", options.Topic), zap.String("SubscriptionName", options.SubscriptionName)) consumer, err := getExistedConsumer(c, options, con.MsgMutex) if err != nil { return nil, err diff --git a/pkg/mq/mqimpl/rocksmq/server/global_rmq.go b/pkg/mq/mqimpl/rocksmq/server/global_rmq.go index bccebb56bc870..20291ea281090 100644 --- a/pkg/mq/mqimpl/rocksmq/server/global_rmq.go +++ b/pkg/mq/mqimpl/rocksmq/server/global_rmq.go @@ -17,6 +17,7 @@ package server import ( + "context" "os" "sync" @@ -36,7 +37,7 @@ var once sync.Once func InitRocksMQ(path string) error { var finalErr error once.Do(func() { - log.Debug("initializing global rmq", zap.String("path", path)) + log.Ctx(context.TODO()).Debug("initializing global rmq", zap.String("path", path)) var fi os.FileInfo fi, finalErr = os.Stat(path) if os.IsNotExist(finalErr) { @@ -58,7 +59,7 @@ func InitRocksMQ(path string) error { // CloseRocksMQ is used to close global rocksmq func CloseRocksMQ() { - log.Debug("Close Rocksmq!") + log.Ctx(context.TODO()).Debug("Close Rocksmq!") if Rmq != nil && Rmq.store != nil { Rmq.Close() } diff --git a/pkg/mq/mqimpl/rocksmq/server/rocksmq_impl.go b/pkg/mq/mqimpl/rocksmq/server/rocksmq_impl.go index 195604b5eeb7f..ca83565109532 100644 --- a/pkg/mq/mqimpl/rocksmq/server/rocksmq_impl.go +++ b/pkg/mq/mqimpl/rocksmq/server/rocksmq_impl.go @@ -12,6 +12,7 @@ package server import ( + "context" "fmt" "path" "strconv" @@ -124,6 +125,7 @@ type rocksmq struct { readers sync.Map state RmqState topicName2LatestMsgID sync.Map + ctx context.Context } func parseCompressionType(params *paramtable.ComponentParam) ([]gorocksdb.CompressionType, error) { @@ -173,7 +175,7 @@ func NewRocksMQ(name string) (*rocksmq, error) { rocksDBLRUCacheCapacity = calculatedCapacity } } - log.Debug("Start rocksmq", zap.Int("max proc", maxProcs), + log.Ctx(context.TODO()).Debug("Start rocksmq", zap.Int("max proc", maxProcs), zap.Int("parallism", parallelism), zap.Uint64("lru cache", rocksDBLRUCacheCapacity)) bbto := gorocksdb.NewDefaultBlockBasedTableOptions() bbto.SetBlockSize(64 << 10) @@ -229,6 +231,7 @@ func NewRocksMQ(name string) (*rocksmq, error) { return nil, err } + ctx := log.WithFields(context.Background(), zap.String("module", "rocksmq")) rmq := &rocksmq{ store: db, cfh: cfHandles, @@ -237,6 +240,7 @@ func NewRocksMQ(name string) (*rocksmq, error) { consumers: sync.Map{}, readers: sync.Map{}, topicName2LatestMsgID: sync.Map{}, + ctx: ctx, } ri, err := initRetentionInfo(kv, db) @@ -254,7 +258,7 @@ func NewRocksMQ(name string) (*rocksmq, error) { for { time.Sleep(10 * time.Minute) - log.Info("Rocksmq stats", + log.Ctx(ctx).Info("Rocksmq stats", zap.String("cache", kv.DB.GetProperty("rocksdb.block-cache-usage")), zap.String("rockskv memtable ", kv.DB.GetProperty("rocksdb.size-all-mem-tables")), zap.String("rockskv table readers", kv.DB.GetProperty("rocksdb.estimate-table-readers-mem")), @@ -297,9 +301,9 @@ func (rmq *rocksmq) allocMsgID(topicName string, delta int) (UniqueID, UniqueID, if msgID == DefaultMessageID { // initialize a new message id if not found the latest msg in the topic msgID = UniqueID(tsoutil.ComposeTSByTime(time.Now(), 0)) - log.Warn("init new message id", zap.String("topicName", topicName), zap.Error(err)) + log.Ctx(rmq.ctx).Warn("init new message id", zap.String("topicName", topicName), zap.Error(err)) } - log.Info("init the latest message id done", zap.String("topicName", topicName), zap.Int64("msgID", msgID)) + log.Ctx(rmq.ctx).Info("init the latest message id done", zap.String("topicName", topicName), zap.Int64("msgID", msgID)) } else { msgID = v.(int64) } @@ -322,7 +326,7 @@ func (rmq *rocksmq) Close() { for _, consumer := range v.([]*Consumer) { err := rmq.destroyConsumerGroupInternal(consumer.Topic, consumer.GroupName) if err != nil { - log.Warn("Failed to destroy consumer group in rocksmq!", zap.String("topic", consumer.Topic), zap.String("groupName", consumer.GroupName), zap.Error(err)) + log.Ctx(rmq.ctx).Warn("Failed to destroy consumer group in rocksmq!", zap.String("topic", consumer.Topic), zap.String("groupName", consumer.GroupName), zap.Error(err)) } } return true @@ -331,11 +335,12 @@ func (rmq *rocksmq) Close() { defer rmq.storeMu.Unlock() rmq.kv.Close() rmq.store.Close() - log.Info("Successfully close rocksmq") + log.Ctx(rmq.ctx).Info("Successfully close rocksmq") } // print rmq consumer Info func (rmq *rocksmq) Info() bool { + log := log.Ctx(rmq.ctx) rtn := true rmq.consumers.Range(func(key, vals interface{}) bool { topic, _ := key.(string) @@ -397,6 +402,7 @@ func (rmq *rocksmq) CreateTopic(topicName string) error { } start := time.Now() + log := log.Ctx(rmq.ctx) // Check if topicName contains "/" if strings.Contains(topicName, "/") { log.Warn("rocksmq failed to create topic for topic name contains \"/\"", zap.String("topic", topicName)) @@ -501,7 +507,7 @@ func (rmq *rocksmq) DestroyTopic(topicName string) error { topicMu.Delete(topicName) rmq.retentionInfo.topicRetetionTime.GetAndRemove(topicName) - log.Debug("Rocksmq destroy topic successfully ", zap.String("topic", topicName), zap.Int64("elapsed", time.Since(start).Milliseconds())) + log.Ctx(rmq.ctx).Debug("Rocksmq destroy topic successfully ", zap.String("topic", topicName), zap.Int64("elapsed", time.Since(start).Milliseconds())) return nil } @@ -533,7 +539,7 @@ func (rmq *rocksmq) CreateConsumerGroup(topicName, groupName string) error { return fmt.Errorf("RMQ CreateConsumerGroup key already exists, key = %s", key) } rmq.consumersID.Store(key, DefaultMessageID) - log.Debug("Rocksmq create consumer group successfully ", zap.String("topic", topicName), + log.Ctx(rmq.ctx).Debug("Rocksmq create consumer group successfully ", zap.String("topic", topicName), zap.String("group", groupName), zap.Int64("elapsed", time.Since(start).Milliseconds())) return nil @@ -559,7 +565,7 @@ func (rmq *rocksmq) RegisterConsumer(consumer *Consumer) error { consumers[0] = consumer rmq.consumers.Store(consumer.Topic, consumers) } - log.Debug("Rocksmq register consumer successfully ", zap.String("topic", consumer.Topic), zap.Int64("elapsed", time.Since(start).Milliseconds())) + log.Ctx(rmq.ctx).Debug("Rocksmq register consumer successfully ", zap.String("topic", consumer.Topic), zap.Int64("elapsed", time.Since(start).Milliseconds())) return nil } @@ -610,7 +616,7 @@ func (rmq *rocksmq) destroyConsumerGroupInternal(topicName, groupName string) er } } } - log.Debug("Rocksmq destroy consumer group successfully ", zap.String("topic", topicName), + log.Ctx(rmq.ctx).Debug("Rocksmq destroy consumer group successfully ", zap.String("topic", topicName), zap.String("group", groupName), zap.Int64("elapsed", time.Since(start).Milliseconds())) return nil @@ -687,7 +693,7 @@ func (rmq *rocksmq) Produce(topicName string, messages []ProducerMessage) ([]Uni // TODO add this to monitor metrics getProduceTime := time.Since(start).Milliseconds() if getProduceTime > 200 { - log.Warn("rocksmq produce too slowly", zap.String("topic", topicName), + log.Ctx(rmq.ctx).Warn("rocksmq produce too slowly", zap.String("topic", topicName), zap.Int64("get lock elapse", getLockTime), zap.Int64("alloc elapse", allocTime-getLockTime), zap.Int64("write elapse", writeTime-allocTime), @@ -852,7 +858,7 @@ func (rmq *rocksmq) Consume(topicName string, groupName string, n int) ([]Consum // TODO add this to monitor metrics getConsumeTime := time.Since(start).Milliseconds() if getConsumeTime > 200 { - log.Warn("rocksmq consume too slowly", zap.String("topic", topicName), + log.Ctx(rmq.ctx).Warn("rocksmq consume too slowly", zap.String("topic", topicName), zap.Int64("get lock elapse", getLockTime), zap.Int64("iterator elapse", iterTime-getLockTime), zap.Int64("moveConsumePosTime elapse", moveConsumePosTime-iterTime), @@ -880,7 +886,7 @@ func (rmq *rocksmq) seek(topicName string, groupName string, msgID UniqueID) err } defer val.Free() if !val.Exists() { - log.Warn("RocksMQ: trying to seek to no exist position, reset current id", + log.Ctx(rmq.ctx).Warn("RocksMQ: trying to seek to no exist position, reset current id", zap.String("topic", topicName), zap.String("group", groupName), zap.Int64("msgId", msgID)) err := rmq.moveConsumePos(topicName, groupName, DefaultMessageID) // skip seek if key is not found, this is the behavior as pulsar @@ -897,6 +903,7 @@ func (rmq *rocksmq) moveConsumePos(topicName string, groupName string, msgID Uni return errors.New("move unknown consumer") } + log := log.Ctx(rmq.ctx) if msgID < oldPos { log.Warn("RocksMQ: trying to move Consume position backward", zap.String("topic", topicName), zap.String("group", groupName), zap.Int64("oldPos", oldPos), zap.Int64("newPos", msgID)) @@ -936,12 +943,13 @@ func (rmq *rocksmq) Seek(topicName string, groupName string, msgID UniqueID) err if err != nil { return err } - log.Debug("successfully seek", zap.String("topic", topicName), zap.String("group", groupName), zap.Uint64("msgId", uint64(msgID))) + log.Ctx(rmq.ctx).Debug("successfully seek", zap.String("topic", topicName), zap.String("group", groupName), zap.Uint64("msgId", uint64(msgID))) return nil } // Only for test func (rmq *rocksmq) ForceSeek(topicName string, groupName string, msgID UniqueID) error { + log := log.Ctx(rmq.ctx) log.Warn("Use method ForceSeek that only for test") if rmq.isClosed() { return errors.New(RmqNotServingErrMsg) @@ -998,7 +1006,7 @@ func (rmq *rocksmq) SeekToLatest(topicName, groupName string) error { return err } - log.Debug("successfully seek to latest", zap.String("topic", topicName), + log.Ctx(rmq.ctx).Debug("successfully seek to latest", zap.String("topic", topicName), zap.String("group", groupName), zap.Uint64("latest", uint64(msgID+1))) return nil } @@ -1060,6 +1068,7 @@ func (rmq *rocksmq) Notify(topicName, groupName string) { // updateAckedInfo update acked informations for retention after consume func (rmq *rocksmq) updateAckedInfo(topicName, groupName string, firstID UniqueID, lastID UniqueID) error { + log := log.Ctx(rmq.ctx) // 1. Try to get the page id between first ID and last ID of ids pageMsgPrefix := constructKey(PageMsgSizeTitle, topicName) + "/" readOpts := gorocksdb.NewDefaultReadOptions() diff --git a/pkg/mq/mqimpl/rocksmq/server/rocksmq_retention.go b/pkg/mq/mqimpl/rocksmq/server/rocksmq_retention.go index 98ce3d03bcc22..6f6190697a3e7 100644 --- a/pkg/mq/mqimpl/rocksmq/server/rocksmq_retention.go +++ b/pkg/mq/mqimpl/rocksmq/server/rocksmq_retention.go @@ -12,6 +12,7 @@ package server import ( + "context" "fmt" "path" "strconv" @@ -77,6 +78,7 @@ func (ri *retentionInfo) startRetentionInfo() { // retention do time ticker and trigger retention check and operation for each topic func (ri *retentionInfo) retention() error { + log := log.Ctx(context.TODO()) log.Debug("Rocksmq retention goroutine start!") params := paramtable.Get() // Do retention check every 10 mins @@ -142,6 +144,7 @@ func (ri *retentionInfo) expiredCleanUp(topic string) error { return err } // Quick Path, No page to check + log := log.Ctx(context.TODO()) if totalAckedSize == 0 { log.Debug("All messages are not expired, skip retention because no ack", zap.String("topic", topic), zap.Int64("time taken", time.Since(start).Milliseconds())) @@ -350,7 +353,7 @@ func DeleteMessages(db *gorocksdb.DB, topic string, startID, endID UniqueID) err if err != nil { return err } - log.Debug("Delete message for topic", zap.String("topic", topic), zap.Int64("startID", startID), zap.Int64("endID", endID)) + log.Ctx(context.TODO()).Debug("Delete message for topic", zap.String("topic", topic), zap.Int64("startID", startID), zap.Int64("endID", endID)) return nil } diff --git a/pkg/mq/msgstream/mq_msgstream.go b/pkg/mq/msgstream/mq_msgstream.go index 09d7121985d25..3a80875031c2e 100644 --- a/pkg/mq/msgstream/mq_msgstream.go +++ b/pkg/mq/msgstream/mq_msgstream.go @@ -124,7 +124,7 @@ func NewMqMsgStream(ctx context.Context, func (ms *mqMsgStream) AsProducer(channels []string) { for _, channel := range channels { if len(channel) == 0 { - log.Error("MsgStream asProducer's channel is an empty string") + log.Ctx(ms.ctx).Error("MsgStream asProducer's channel is an empty string") break } @@ -205,7 +205,7 @@ func (ms *mqMsgStream) AsConsumer(ctx context.Context, channels []string, subNam panic(fmt.Sprintf("%s, errors = %s", errMsg, err.Error())) } - log.Info("Successfully create consumer", zap.String("channel", channel), zap.String("subname", subName)) + log.Ctx(ms.ctx).Info("Successfully create consumer", zap.String("channel", channel), zap.String("subname", subName)) } return nil } @@ -215,7 +215,7 @@ func (ms *mqMsgStream) SetRepackFunc(repackFunc RepackFunc) { } func (ms *mqMsgStream) Close() { - log.Info("start to close mq msg stream", + log.Ctx(ms.ctx).Info("start to close mq msg stream", zap.Int("producer num", len(ms.producers)), zap.Int("consumer num", len(ms.consumers))) ms.streamCancel() @@ -275,11 +275,11 @@ func (ms *mqMsgStream) isEnabledProduce() bool { func (ms *mqMsgStream) Produce(msgPack *MsgPack) error { if !ms.isEnabledProduce() { - log.Warn("can't produce the msg in the backup instance", zap.Stack("stack")) + log.Ctx(ms.ctx).Warn("can't produce the msg in the backup instance", zap.Stack("stack")) return merr.ErrDenyProduceMsg } if msgPack == nil || len(msgPack.Msgs) <= 0 { - log.Debug("Warning: Receive empty msgPack") + log.Ctx(ms.ctx).Debug("Warning: Receive empty msgPack") return nil } if len(ms.producers) <= 0 { @@ -356,7 +356,7 @@ func (ms *mqMsgStream) Broadcast(msgPack *MsgPack) (map[string][]MessageID, erro isCreateCollectionMsg := len(msgPack.Msgs) == 1 && msgPack.Msgs[0].Type() == commonpb.MsgType_CreateCollection if !ms.isEnabledProduce() && !isCreateCollectionMsg { - log.Warn("can't broadcast the msg in the backup instance", zap.Stack("stack")) + log.Ctx(ms.ctx).Warn("can't broadcast the msg in the backup instance", zap.Stack("stack")) return ids, merr.ErrDenyProduceMsg } for _, v := range msgPack.Msgs { @@ -432,14 +432,14 @@ func (ms *mqMsgStream) receiveMsg(consumer mqwrapper.Consumer) { } consumer.Ack(msg) if msg.Payload() == nil { - log.Warn("MqMsgStream get msg whose payload is nil") + log.Ctx(ms.ctx).Warn("MqMsgStream get msg whose payload is nil") continue } // not need to check the preCreatedTopic is empty, related issue: https://github.com/milvus-io/milvus/issues/27295 // if the message not belong to the topic, will skip it tsMsg, err := ms.getTsMsgFromConsumerMsg(msg) if err != nil { - log.Warn("Failed to getTsMsgFromConsumerMsg", zap.Error(err)) + log.Ctx(ms.ctx).Warn("Failed to getTsMsgFromConsumerMsg", zap.Error(err)) continue } pos := tsMsg.Position() @@ -501,13 +501,13 @@ func (ms *mqMsgStream) Seek(ctx context.Context, msgPositions []*MsgPosition, in } } - log.Info("MsgStream seek begin", zap.String("channel", mp.ChannelName), zap.Any("MessageID", mp.MsgID), zap.Bool("includeCurrentMsg", includeCurrentMsg)) + log.Ctx(ctx).Info("MsgStream seek begin", zap.String("channel", mp.ChannelName), zap.Any("MessageID", mp.MsgID), zap.Bool("includeCurrentMsg", includeCurrentMsg)) err = consumer.Seek(messageID, includeCurrentMsg) if err != nil { - log.Warn("Failed to seek", zap.String("channel", mp.ChannelName), zap.Error(err)) + log.Ctx(ctx).Warn("Failed to seek", zap.String("channel", mp.ChannelName), zap.Error(err)) return err } - log.Info("MsgStream seek finished", zap.String("channel", mp.ChannelName)) + log.Ctx(ctx).Info("MsgStream seek finished", zap.String("channel", mp.ChannelName)) } return nil } @@ -662,7 +662,7 @@ func (ms *MqTtMsgStream) bufMsgPackToChannel() { // block here until addConsumer if _, ok := <-ms.syncConsumer; !ok { - log.Warn("consumer closed!") + log.Ctx(ms.ctx).Warn("consumer closed!") return } @@ -761,7 +761,7 @@ func (ms *MqTtMsgStream) bufMsgPackToChannel() { uniqueMsgs := make([]TsMsg, 0, len(timeTickBuf)) for _, msg := range timeTickBuf { if isDMLMsg(msg) && idset.Contain(msg.ID()) { - log.Warn("mqTtMsgStream, found duplicated msg", zap.Int64("msgID", msg.ID())) + log.Ctx(ms.ctx).Warn("mqTtMsgStream, found duplicated msg", zap.Int64("msgID", msg.ID())) continue } idset.Insert(msg.ID()) @@ -791,6 +791,7 @@ func (ms *MqTtMsgStream) bufMsgPackToChannel() { // Save all msgs into chanMsgBuf[] till receive one ttMsg func (ms *MqTtMsgStream) consumeToTtMsg(consumer mqwrapper.Consumer) { + log := log.Ctx(ms.ctx) defer ms.chanWaitGroup.Done() for { select { @@ -862,6 +863,7 @@ func (ms *MqTtMsgStream) Seek(ctx context.Context, msgPositions []*MsgPosition, var consumer mqwrapper.Consumer var mp *MsgPosition var err error + log := log.Ctx(ctx) fn := func() (bool, error) { var ok bool consumer, ok = ms.consumers[mp.ChannelName] @@ -879,7 +881,7 @@ func (ms *MqTtMsgStream) Seek(ctx context.Context, msgPositions []*MsgPosition, // try to use latest message ID first seekMsgID, err = consumer.GetLatestMsgID() if err != nil { - log.Ctx(ctx).Warn("Ignoring bad message id", zap.Error(err)) + log.Warn("Ignoring bad message id", zap.Error(err)) return false, nil } } else { diff --git a/pkg/mq/msgstream/mqwrapper/kafka/kafka_client.go b/pkg/mq/msgstream/mqwrapper/kafka/kafka_client.go index 9950d6e164c54..f490f0c3ab566 100644 --- a/pkg/mq/msgstream/mqwrapper/kafka/kafka_client.go +++ b/pkg/mq/msgstream/mqwrapper/kafka/kafka_client.go @@ -141,6 +141,7 @@ func (kc *kafkaClient) getKafkaProducer() (*kafka.Producer, error) { if p := producer.Load(); p != nil { return p, nil } + log := log.Ctx(context.TODO()) p, err, _ := sf.Do("kafka_producer", func() (*kafka.Producer, error) { if p := producer.Load(); p != nil { return p, nil diff --git a/pkg/mq/msgstream/mqwrapper/kafka/kafka_producer.go b/pkg/mq/msgstream/mqwrapper/kafka/kafka_producer.go index 2ce921157ebf4..0d17b441da864 100644 --- a/pkg/mq/msgstream/mqwrapper/kafka/kafka_producer.go +++ b/pkg/mq/msgstream/mqwrapper/kafka/kafka_producer.go @@ -79,6 +79,7 @@ func (kp *kafkaProducer) Send(ctx context.Context, message *mqcommon.ProducerMes } func (kp *kafkaProducer) Close() { + log := log.Ctx(context.TODO()) kp.closeOnce.Do(func() { kp.isClosed = true diff --git a/pkg/mq/msgstream/mqwrapper/nmq/nmq_server.go b/pkg/mq/msgstream/mqwrapper/nmq/nmq_server.go index bbd963139d340..7fab92658a8a4 100644 --- a/pkg/mq/msgstream/mqwrapper/nmq/nmq_server.go +++ b/pkg/mq/msgstream/mqwrapper/nmq/nmq_server.go @@ -17,6 +17,7 @@ package nmq import ( + "context" "os" "path" "sync" @@ -103,7 +104,7 @@ func ParseServerOption(params *paramtable.ComponentParam) *NatsMQConfig { // CloseNatsMQ is used to close global natsmq func CloseNatsMQ() { - log.Debug("Closing Natsmq!") + log.Ctx(context.TODO()).Debug("Closing Natsmq!") if Nmq != nil { // Shut down the server. Nmq.Shutdown() diff --git a/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_consumer.go b/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_consumer.go index 9ab566476f299..5350c84d16991 100644 --- a/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_consumer.go +++ b/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_consumer.go @@ -51,6 +51,7 @@ func (pc *Consumer) Subscription() string { // Chan returns a message channel func (pc *Consumer) Chan() <-chan common.Message { + log := log.Ctx(context.TODO()) if pc.msgChannel == nil { pc.once.Do(func() { pc.msgChannel = make(chan common.Message, 256) diff --git a/pkg/util/funcutil/parallel.go b/pkg/util/funcutil/parallel.go index 6e679c7cbbb47..d8214ddcf9bcd 100644 --- a/pkg/util/funcutil/parallel.go +++ b/pkg/util/funcutil/parallel.go @@ -17,6 +17,7 @@ package funcutil import ( + "context" "reflect" "runtime" "sync" @@ -51,7 +52,7 @@ func ProcessFuncParallel(total, maxParallel int, f ProcessFunc, fname string) er t := time.Now() defer func() { - log.Debug(fname, zap.Int("total", total), zap.Any("time cost", time.Since(t))) + log.Ctx(context.TODO()).Debug(fname, zap.Int("total", total), zap.Any("time cost", time.Since(t))) }() nPerBatch := (total + maxParallel - 1) / maxParallel @@ -134,6 +135,7 @@ func ProcessTaskParallel(maxParallel int, fname string, tasks ...TaskFunc) error // for _, opt := range opts { // opt(&option) // } + log := log.Ctx(context.TODO()) if maxParallel <= 0 { maxParallel = 1 diff --git a/pkg/util/funcutil/policy.go b/pkg/util/funcutil/policy.go index bdf2650a51dab..25d5174c2e102 100644 --- a/pkg/util/funcutil/policy.go +++ b/pkg/util/funcutil/policy.go @@ -1,6 +1,7 @@ package funcutil import ( + "context" "fmt" "strings" @@ -17,6 +18,7 @@ import ( ) func GetVersion(m interface{}) (string, error) { + log := log.Ctx(context.TODO()) pbMsg, ok := m.(proto.Message) if !ok { err := fmt.Errorf("MessageDescriptorProto result is nil") diff --git a/pkg/util/gc/gc_tuner.go b/pkg/util/gc/gc_tuner.go index 26e1f74c335b7..ff50aeab31081 100644 --- a/pkg/util/gc/gc_tuner.go +++ b/pkg/util/gc/gc_tuner.go @@ -17,6 +17,7 @@ package gc import ( + "context" "math" "os" "runtime" @@ -87,7 +88,7 @@ func optimizeGOGC() { // currently we assume 20 ms as long gc pause if (m.PauseNs[(m.NumGC+255)%256] / uint64(time.Millisecond)) < 20 { - log.Debug("GC Tune done", zap.Uint32("previous GOGC", previousGOGC), + log.Ctx(context.TODO()).Debug("GC Tune done", zap.Uint32("previous GOGC", previousGOGC), zap.Uint64("heapuse ", toMB(heapuse)), zap.Uint64("total memory", toMB(totaluse)), zap.Uint64("next GC", toMB(m.NextGC)), @@ -96,7 +97,7 @@ func optimizeGOGC() { zap.Uint64("gc-pause-end", m.PauseEnd[(m.NumGC+255)%256]), ) } else { - log.Warn("GC Tune done, and the gc is slow", zap.Uint32("previous GOGC", previousGOGC), + log.Ctx(context.TODO()).Warn("GC Tune done, and the gc is slow", zap.Uint32("previous GOGC", previousGOGC), zap.Uint64("heapuse ", toMB(heapuse)), zap.Uint64("total memory", toMB(totaluse)), zap.Uint64("next GC", toMB(m.NextGC)), diff --git a/tests/integration/minicluster_v2.go b/tests/integration/minicluster_v2.go index 9b0f1d445240e..cd4a81e3a06fc 100644 --- a/tests/integration/minicluster_v2.go +++ b/tests/integration/minicluster_v2.go @@ -256,7 +256,7 @@ func StartMiniClusterV2(ctx context.Context, opts ...OptionV2) (*MiniClusterV2, return nil, err } if streamingutil.IsStreamingServiceEnabled() { - cluster.StreamingNode, err = streamingnode.NewServer(cluster.factory) + cluster.StreamingNode, err = streamingnode.NewServer(ctx, cluster.factory) if err != nil { return nil, err } @@ -334,7 +334,7 @@ func (cluster *MiniClusterV2) AddStreamingNode() { cluster.ptmu.Lock() defer cluster.ptmu.Unlock() - node, err := streamingnode.NewServer(cluster.factory) + node, err := streamingnode.NewServer(context.TODO(), cluster.factory) if err != nil { panic(err) }