From 4c2d9be6727cb923f74a7afd71245ab7ca94d2d8 Mon Sep 17 00:00:00 2001 From: tinswzy Date: Fri, 13 Dec 2024 16:20:02 +0800 Subject: [PATCH] enhance watch chan async trace Signed-off-by: tinswzy --- internal/datacoord/channel.go | 13 +- internal/datacoord/channel_manager.go | 119 ++++----- internal/datacoord/channel_manager_test.go | 64 ++--- internal/datacoord/channel_store.go | 35 +-- internal/datacoord/channel_store_test.go | 10 +- internal/datacoord/cluster.go | 8 +- internal/datacoord/cluster_test.go | 16 +- internal/datacoord/meta.go | 4 +- internal/datacoord/metrics_info.go | 2 +- internal/datacoord/metrics_info_test.go | 4 +- internal/datacoord/mock_channel_store.go | 27 ++- internal/datacoord/mock_channelmanager.go | 226 +++++++++--------- internal/datacoord/policy.go | 9 +- internal/datacoord/policy_test.go | 22 +- internal/datacoord/server_test.go | 14 +- internal/datacoord/services.go | 22 +- internal/datacoord/services_test.go | 32 +-- internal/datacoord/sync_segments_scheduler.go | 2 +- .../datacoord/sync_segments_scheduler_test.go | 6 +- internal/datacoord/util.go | 2 +- internal/datanode/channel/channel_manager.go | 45 ++-- .../datanode/channel/channel_manager_test.go | 20 +- .../datanode/channel/mock_channelmanager.go | 33 ++- internal/datanode/services.go | 2 +- internal/flushcommon/broker/datacoord.go | 6 +- .../flushcommon/pipeline/data_sync_service.go | 22 +- .../flow_graph_dmstream_input_node.go | 3 +- internal/flushcommon/syncmgr/meta_writer.go | 2 +- internal/kv/etcd/etcd_kv.go | 13 + internal/metastore/kv/datacoord/kv_catalog.go | 3 + .../metastore/kv/querycoord/kv_catalog.go | 5 + internal/proxy/channels_mgr.go | 6 + internal/proxy/impl.go | 10 +- internal/proxy/task_insert.go | 4 + internal/proxy/task_search.go | 6 +- internal/querycoordv2/job/job_release.go | 14 +- internal/querycoordv2/services.go | 5 + .../querynodev2/delegator/delegator_data.go | 2 +- internal/querynodev2/services.go | 4 +- internal/rootcoord/create_collection_task.go | 5 +- internal/rootcoord/dml_channels.go | 9 +- internal/rootcoord/dml_channels_test.go | 4 +- internal/rootcoord/drop_collection_task.go | 3 + internal/rootcoord/meta_table.go | 7 + internal/rootcoord/redo.go | 11 +- internal/rootcoord/redo_test.go | 4 +- internal/rootcoord/show_collection_task.go | 12 + internal/rootcoord/step_executor.go | 7 +- internal/rootcoord/step_executor_test.go | 30 +-- internal/rootcoord/task.go | 2 +- internal/rootcoord/timeticksync.go | 4 +- internal/rootcoord/undo.go | 2 +- .../util/proxyutil/proxy_client_manager.go | 6 +- pkg/config/etcd_source.go | 1 + pkg/mq/msgdispatcher/client.go | 5 +- pkg/mq/msgdispatcher/dispatcher.go | 2 +- pkg/mq/msgdispatcher/manager.go | 7 +- pkg/mq/msgdispatcher/manager_test.go | 12 +- pkg/mq/msgstream/mq_msgstream.go | 49 +++- .../msgstream/mqwrapper/kafka/kafka_client.go | 3 + .../mqwrapper/pulsar/pulsar_client.go | 7 + .../mqwrapper/pulsar/pulsar_producer.go | 6 + pkg/mq/msgstream/mqwrapper/rmq/rmq_client.go | 3 + 63 files changed, 619 insertions(+), 424 deletions(-) diff --git a/internal/datacoord/channel.go b/internal/datacoord/channel.go index c13309ba537ef..29a7b027b5a32 100644 --- a/internal/datacoord/channel.go +++ b/internal/datacoord/channel.go @@ -17,6 +17,7 @@ package datacoord import ( + "context" "fmt" "go.uber.org/zap" @@ -43,7 +44,8 @@ type RWChannel interface { UpdateWatchInfo(info *datapb.ChannelWatchInfo) } -func NewRWChannel(name string, +func NewRWChannel(ctx context.Context, + name string, collectionID int64, startPos []*commonpb.KeyDataPair, schema *schemapb.CollectionSchema, @@ -55,6 +57,7 @@ func NewRWChannel(name string, StartPositions: startPos, Schema: schema, CreateTimestamp: createTs, + currentCtx: ctx, } } @@ -131,11 +134,12 @@ type StateChannel struct { currentState ChannelState assignedNode int64 + currentCtx context.Context } var _ RWChannel = (*StateChannel)(nil) -func NewStateChannel(ch RWChannel) *StateChannel { +func NewStateChannel(ctx context.Context, ch RWChannel) *StateChannel { c := &StateChannel{ Name: ch.GetName(), CollectionID: ch.GetCollectionID(), @@ -145,19 +149,21 @@ func NewStateChannel(ch RWChannel) *StateChannel { Info: ch.GetWatchInfo(), assignedNode: bufferID, + currentCtx: ctx, } c.setState(Standby) return c } -func NewStateChannelByWatchInfo(nodeID int64, info *datapb.ChannelWatchInfo) *StateChannel { +func NewStateChannelByWatchInfo(ctx context.Context, nodeID int64, info *datapb.ChannelWatchInfo) *StateChannel { c := &StateChannel{ Name: info.GetVchan().GetChannelName(), CollectionID: info.GetVchan().GetCollectionID(), Schema: info.GetSchema(), Info: info, assignedNode: nodeID, + currentCtx: ctx, } switch info.GetState() { @@ -219,6 +225,7 @@ func (c *StateChannel) Clone() *StateChannel { currentState: c.currentState, assignedNode: c.assignedNode, + currentCtx: c.currentCtx, } } diff --git a/internal/datacoord/channel_manager.go b/internal/datacoord/channel_manager.go index c144d35c62a74..7eee5dafadbff 100644 --- a/internal/datacoord/channel_manager.go +++ b/internal/datacoord/channel_manager.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/errors" "github.com/samber/lo" + "go.opentelemetry.io/otel" "go.uber.org/zap" "google.golang.org/protobuf/proto" @@ -43,20 +44,20 @@ type ChannelManager interface { Startup(ctx context.Context, legacyNodes, allNodes []int64) error Close() - AddNode(nodeID UniqueID) error - DeleteNode(nodeID UniqueID) error + AddNode(ctx context.Context, nodeID UniqueID) error + DeleteNode(ctx context.Context, nodeID UniqueID) error Watch(ctx context.Context, ch RWChannel) error - Release(nodeID UniqueID, channelName string) error + Release(ctx context.Context, nodeID UniqueID, channelName string) error - Match(nodeID UniqueID, channel string) bool - FindWatcher(channel string) (UniqueID, error) + Match(ctx context.Context, nodeID UniqueID, channel string) bool + FindWatcher(ctx context.Context, channel string) (UniqueID, error) - GetChannel(nodeID int64, channel string) (RWChannel, bool) - GetNodeChannelsByCollectionID(collectionID int64) map[int64][]string - GetChannelsByCollectionID(collectionID int64) []RWChannel - GetChannelNamesByCollectionID(collectionID int64) []string + GetChannel(ctx context.Context, nodeID int64, channel string) (RWChannel, bool) + GetNodeChannelsByCollectionID(ctx context.Context, collectionID int64) map[int64][]string + GetChannelsByCollectionID(ctx context.Context, collectionID int64) []RWChannel + GetChannelNamesByCollectionID(ctx context.Context, collectionID int64) []string - GetChannelWatchInfos() map[int64]map[string]*datapb.ChannelWatchInfo + GetChannelWatchInfos(ctx context.Context) map[int64]map[string]*datapb.ChannelWatchInfo } // An interface sessionManager implments @@ -142,13 +143,13 @@ func (m *ChannelManagerImpl) Startup(ctx context.Context, legacyNodes, allNodes offLines, newOnLines := lo.Difference(oNodes, allNodes) // Delete offlines from the cluster for _, nodeID := range offLines { - if err := m.DeleteNode(nodeID); err != nil { + if err := m.DeleteNode(ctx, nodeID); err != nil { return err } } // Add new online nodes to the cluster. for _, nodeID := range newOnLines { - if err := m.AddNode(nodeID); err != nil { + if err := m.AddNode(ctx, nodeID); err != nil { return err } } @@ -190,21 +191,21 @@ func (m *ChannelManagerImpl) Close() { } } -func (m *ChannelManagerImpl) AddNode(nodeID UniqueID) error { +func (m *ChannelManagerImpl) AddNode(ctx context.Context, nodeID UniqueID) error { m.mu.Lock() defer m.mu.Unlock() - - log.Info("register node", zap.Int64("registered node", nodeID)) + log := log.Ctx(ctx).With(zap.Int64("registered node", nodeID)) + log.Info("register node") m.store.AddNode(nodeID) - updates := m.assignPolicy(m.store.GetNodesChannels(), m.store.GetBufferChannelInfo(), m.legacyNodes.Collect()) + updates := m.assignPolicy(context.TODO(), m.store.GetNodesChannels(), m.store.GetBufferChannelInfo(), m.legacyNodes.Collect()) if updates == nil { - log.Info("register node with no reassignment", zap.Int64("registered node", nodeID)) + log.Info("register node with no reassignment") return nil } - err := m.execute(updates) + err := m.execute(ctx, updates) if err != nil { log.Warn("fail to update channel operation updates into meta", zap.Error(err)) } @@ -212,8 +213,8 @@ func (m *ChannelManagerImpl) AddNode(nodeID UniqueID) error { } // Release writes ToRelease channel watch states for a channel -func (m *ChannelManagerImpl) Release(nodeID UniqueID, channelName string) error { - log := log.With( +func (m *ChannelManagerImpl) Release(ctx context.Context, nodeID UniqueID, channelName string) error { + log := log.Ctx(ctx).With( zap.Int64("nodeID", nodeID), zap.String("channel", channelName), ) @@ -224,7 +225,7 @@ func (m *ChannelManagerImpl) Release(nodeID UniqueID, channelName string) error } log.Info("Releasing channel from watched node") - ch, found := m.GetChannel(nodeID, channelName) + ch, found := m.GetChannel(ctx, nodeID, channelName) if !found { return fmt.Errorf("fail to find matching nodeID: %d with channelName: %s", nodeID, channelName) } @@ -232,17 +233,19 @@ func (m *ChannelManagerImpl) Release(nodeID UniqueID, channelName string) error m.mu.Lock() defer m.mu.Unlock() updates := NewChannelOpSet(NewChannelOp(nodeID, Release, ch)) - return m.execute(updates) + return m.execute(ctx, updates) } func (m *ChannelManagerImpl) Watch(ctx context.Context, ch RWChannel) error { + ctx, sp := otel.Tracer("ChannelManager").Start(ctx, "Watch") + defer sp.End() log := log.Ctx(ctx).With(zap.String("channel", ch.GetName())) m.mu.Lock() defer m.mu.Unlock() log.Info("Add channel") updates := NewChannelOpSet(NewChannelOp(bufferID, Watch, ch)) - err := m.execute(updates) + err := m.execute(ctx, updates) if err != nil { log.Warn("fail to update new channel updates into meta", zap.Array("updates", updates), zap.Error(err)) @@ -250,12 +253,12 @@ func (m *ChannelManagerImpl) Watch(ctx context.Context, ch RWChannel) error { // channel already written into meta, try to assign it to the cluster // not error is returned if failed, the assignment will retry later - updates = m.assignPolicy(m.store.GetNodesChannels(), m.store.GetBufferChannelInfo(), m.legacyNodes.Collect()) + updates = m.assignPolicy(ctx, m.store.GetNodesChannels(), m.store.GetBufferChannelInfo(), m.legacyNodes.Collect()) if updates == nil { return nil } - if err := m.execute(updates); err != nil { + if err := m.execute(ctx, updates); err != nil { log.Warn("fail to assign channel, will retry later", zap.Array("updates", updates), zap.Error(err)) return nil } @@ -264,7 +267,7 @@ func (m *ChannelManagerImpl) Watch(ctx context.Context, ch RWChannel) error { return nil } -func (m *ChannelManagerImpl) DeleteNode(nodeID UniqueID) error { +func (m *ChannelManagerImpl) DeleteNode(ctx context.Context, nodeID UniqueID) error { m.mu.Lock() defer m.mu.Unlock() @@ -281,9 +284,10 @@ func (m *ChannelManagerImpl) DeleteNode(nodeID UniqueID) error { NewChannelOp(info.NodeID, Delete, lo.Values(info.Channels)...), NewChannelOp(bufferID, Watch, lo.Values(info.Channels)...), ) - log.Info("deregister node", zap.Int64("nodeID", nodeID), zap.Array("updates", updates)) + log := log.Ctx(ctx).With(zap.Int64("nodeID", nodeID), zap.Array("updates", updates)) + log.Info("deregister node") - err := m.execute(updates) + err := m.execute(ctx, updates) if err != nil { log.Warn("fail to update channel operation updates into meta", zap.Error(err)) return err @@ -296,13 +300,13 @@ func (m *ChannelManagerImpl) DeleteNode(nodeID UniqueID) error { } // reassign reassigns a channel to another DataNode. -func (m *ChannelManagerImpl) reassign(original *NodeChannelInfo) error { +func (m *ChannelManagerImpl) reassign(ctx context.Context, original *NodeChannelInfo) error { m.mu.Lock() defer m.mu.Unlock() - updates := m.assignPolicy(m.store.GetNodesChannels(), original, m.legacyNodes.Collect()) + updates := m.assignPolicy(context.TODO(), m.store.GetNodesChannels(), original, m.legacyNodes.Collect()) if updates != nil { - return m.execute(updates) + return m.execute(ctx, updates) } if original.NodeID != bufferID { @@ -311,7 +315,7 @@ func (m *ChannelManagerImpl) reassign(original *NodeChannelInfo) error { zap.Strings("channels", lo.Keys(original.Channels)), ) updates := NewChannelOpSet(NewChannelOp(original.NodeID, Watch, lo.Values(original.Channels)...)) - return m.execute(updates) + return m.execute(ctx, updates) } return nil @@ -328,12 +332,12 @@ func (m *ChannelManagerImpl) Balance() { } log.Info("Channel balancer got new reAllocations:", zap.Array("assignment", updates)) - if err := m.execute(updates); err != nil { + if err := m.execute(context.TODO(), updates); err != nil { log.Warn("Channel balancer fail to execute", zap.Array("assignment", updates), zap.Error(err)) } } -func (m *ChannelManagerImpl) Match(nodeID UniqueID, channel string) bool { +func (m *ChannelManagerImpl) Match(ctx context.Context, nodeID UniqueID, channel string) bool { m.mu.RLock() defer m.mu.RUnlock() @@ -351,7 +355,7 @@ func (m *ChannelManagerImpl) Match(nodeID UniqueID, channel string) bool { return ok } -func (m *ChannelManagerImpl) GetChannel(nodeID int64, channelName string) (RWChannel, bool) { +func (m *ChannelManagerImpl) GetChannel(ctx context.Context, nodeID int64, channelName string) (RWChannel, bool) { m.mu.RLock() defer m.mu.RUnlock() @@ -363,13 +367,13 @@ func (m *ChannelManagerImpl) GetChannel(nodeID int64, channelName string) (RWCha return nil, false } -func (m *ChannelManagerImpl) GetNodeChannelsByCollectionID(collectionID int64) map[int64][]string { +func (m *ChannelManagerImpl) GetNodeChannelsByCollectionID(ctx context.Context, collectionID int64) map[int64][]string { m.mu.RLock() defer m.mu.RUnlock() return m.store.GetNodeChannelsByCollectionID(collectionID) } -func (m *ChannelManagerImpl) GetChannelsByCollectionID(collectionID int64) []RWChannel { +func (m *ChannelManagerImpl) GetChannelsByCollectionID(ctx context.Context, collectionID int64) []RWChannel { m.mu.RLock() defer m.mu.RUnlock() channels := []RWChannel{} @@ -383,14 +387,14 @@ func (m *ChannelManagerImpl) GetChannelsByCollectionID(collectionID int64) []RWC return channels } -func (m *ChannelManagerImpl) GetChannelNamesByCollectionID(collectionID int64) []string { - channels := m.GetChannelsByCollectionID(collectionID) +func (m *ChannelManagerImpl) GetChannelNamesByCollectionID(ctx context.Context, collectionID int64) []string { + channels := m.GetChannelsByCollectionID(ctx, collectionID) return lo.Map(channels, func(ch RWChannel, _ int) string { return ch.GetName() }) } -func (m *ChannelManagerImpl) FindWatcher(channel string) (UniqueID, error) { +func (m *ChannelManagerImpl) FindWatcher(ctx context.Context, channel string) (UniqueID, error) { m.mu.RLock() defer m.mu.RUnlock() @@ -420,7 +424,7 @@ func (m *ChannelManagerImpl) removeChannel(nodeID int64, ch RWChannel) error { zap.String("channel", ch.GetName()), zap.Int64("assignment", nodeID), zap.Int64("collectionID", ch.GetCollectionID())) - return m.store.Update(op) + return m.store.Update(context.TODO(), op) } func (m *ChannelManagerImpl) CheckLoop(ctx context.Context) { @@ -431,7 +435,7 @@ func (m *ChannelManagerImpl) CheckLoop(ctx context.Context) { for { select { case <-ctx.Done(): - log.Info("background checking channels loop quit") + log.Ctx(ctx).Info("background checking channels loop quit") return case <-balanceTicker.C: // balance @@ -444,7 +448,7 @@ func (m *ChannelManagerImpl) CheckLoop(ctx context.Context) { } } -func (m *ChannelManagerImpl) AdvanceChannelState(ctx context.Context) { +func (m *ChannelManagerImpl) AdvanceChannelState(loopCheckCtx context.Context) { m.mu.RLock() standbys := m.store.GetNodeChannelsBy(WithAllNodes(), WithChannelStates(Standby)) toNotifies := m.store.GetNodeChannelsBy(WithoutBufferNode(), WithChannelStates(ToWatch, ToRelease)) @@ -454,8 +458,8 @@ func (m *ChannelManagerImpl) AdvanceChannelState(ctx context.Context) { // Processing standby channels updatedStandbys := false - updatedStandbys = m.advanceStandbys(ctx, standbys) - updatedToCheckes := m.advanceToChecks(ctx, toChecks) + updatedStandbys = m.advanceStandbys(loopCheckCtx, standbys) + updatedToCheckes := m.advanceToChecks(loopCheckCtx, toChecks) var ( updatedToNotifies bool @@ -465,7 +469,7 @@ func (m *ChannelManagerImpl) AdvanceChannelState(ctx context.Context) { if toNotifyNum > 0 { toNotifies = lo.Slice(toNotifies, 0, toNotifyNum) - updatedToNotifies = m.advanceToNotifies(ctx, toNotifies) + updatedToNotifies = m.advanceToNotifies(loopCheckCtx, toNotifies) } if updatedStandbys || updatedToCheckes || updatedToNotifies { @@ -489,7 +493,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) @@ -515,7 +519,7 @@ func (m *ChannelManagerImpl) advanceStandbys(_ context.Context, standbys []*Node } chNames := lo.Keys(validChannels) - if err := m.reassign(nodeAssign); err != nil { + if err := m.reassign(ctx, nodeAssign); err != nil { log.Warn("Reassign channels fail", zap.Int64("nodeID", nodeAssign.NodeID), zap.Strings("channels", chNames), @@ -534,7 +538,7 @@ func (m *ChannelManagerImpl) advanceStandbys(_ context.Context, standbys []*Node return advanced } -func (m *ChannelManagerImpl) advanceToNotifies(ctx context.Context, toNotifies []*NodeChannelInfo) bool { +func (m *ChannelManagerImpl) advanceToNotifies(loopCheckCtx context.Context, toNotifies []*NodeChannelInfo) bool { var advanced bool = false for _, nodeAssign := range toNotifies { channelCount := len(nodeAssign.Channels) @@ -550,7 +554,7 @@ func (m *ChannelManagerImpl) advanceToNotifies(ctx context.Context, toNotifies [ ) chNames := lo.Keys(nodeAssign.Channels) - log.Info("Notify channel operations to datanode", + log.Ctx(loopCheckCtx).Info("Notify channel operations to datanode", zap.Int64("assignment", nodeAssign.NodeID), zap.Int("total operation count", len(nodeAssign.Channels)), zap.Strings("channel names", chNames), @@ -561,7 +565,12 @@ func (m *ChannelManagerImpl) advanceToNotifies(ctx context.Context, toNotifies [ tmpWatchInfo.Vchan = m.h.GetDataVChanPositions(innerCh, allPartitionID) future := getOrCreateIOPool().Submit(func() (any, error) { - err := m.Notify(ctx, nodeID, tmpWatchInfo) + if stateChan, isStateChannel := ch.(*StateChannel); isStateChannel { + renewCtx := context.WithoutCancel(stateChan.currentCtx) + err := m.Notify(renewCtx, nodeID, tmpWatchInfo) + return innerCh, err + } + err := m.Notify(loopCheckCtx, nodeID, tmpWatchInfo) return innerCh, err }) futures = append(futures, future) @@ -577,7 +586,7 @@ func (m *ChannelManagerImpl) advanceToNotifies(ctx context.Context, toNotifies [ } } - log.Info("Finish to notify channel operations to datanode", + log.Ctx(loopCheckCtx).Info("Finish to notify channel operations to datanode", zap.Int64("assignment", nodeAssign.NodeID), zap.Int("operation count", channelCount), zap.Int("success count", len(succeededChannels)), @@ -708,7 +717,7 @@ func (m *ChannelManagerImpl) Check(ctx context.Context, nodeID int64, info *data return false, false } -func (m *ChannelManagerImpl) execute(updates *ChannelOpSet) error { +func (m *ChannelManagerImpl) execute(ctx context.Context, updates *ChannelOpSet) error { for _, op := range updates.ops { if op.Type != Delete { if err := m.fillChannelWatchInfo(op); err != nil { @@ -718,7 +727,7 @@ func (m *ChannelManagerImpl) execute(updates *ChannelOpSet) error { } } - return m.store.Update(updates) + return m.store.Update(ctx, updates) } // fillChannelWatchInfoWithState updates the channel op by filling in channel watch info. @@ -754,7 +763,7 @@ func (m *ChannelManagerImpl) fillChannelWatchInfo(op *ChannelOp) error { return nil } -func (m *ChannelManagerImpl) GetChannelWatchInfos() map[int64]map[string]*datapb.ChannelWatchInfo { +func (m *ChannelManagerImpl) GetChannelWatchInfos(ctx context.Context) map[int64]map[string]*datapb.ChannelWatchInfo { m.mu.RLock() defer m.mu.RUnlock() infos := make(map[int64]map[string]*datapb.ChannelWatchInfo) diff --git a/internal/datacoord/channel_manager_test.go b/internal/datacoord/channel_manager_test.go index 99a0d953cd369..2c1825906ca07 100644 --- a/internal/datacoord/channel_manager_test.go +++ b/internal/datacoord/channel_manager_test.go @@ -69,7 +69,7 @@ func (s *ChannelManagerSuite) prepareMeta(chNodes map[string]int64, state datapb } func (s *ChannelManagerSuite) checkAssignment(m *ChannelManagerImpl, nodeID int64, channel string, state ChannelState) { - rwChannel, found := m.GetChannel(nodeID, channel) + rwChannel, found := m.GetChannel(context.TODO(), nodeID, channel) s.True(found) s.NotNil(rwChannel) s.Equal(channel, rwChannel.GetName()) @@ -77,20 +77,20 @@ func (s *ChannelManagerSuite) checkAssignment(m *ChannelManagerImpl, nodeID int6 s.True(ok) s.Equal(state, sChannel.currentState) s.EqualValues(nodeID, sChannel.assignedNode) - s.True(m.Match(nodeID, channel)) + s.True(m.Match(context.TODO(), nodeID, channel)) if nodeID != bufferID { - gotNode, err := m.FindWatcher(channel) + gotNode, err := m.FindWatcher(context.TODO(), channel) s.NoError(err) s.EqualValues(gotNode, nodeID) } } func (s *ChannelManagerSuite) checkNoAssignment(m *ChannelManagerImpl, nodeID int64, channel string) { - rwChannel, found := m.GetChannel(nodeID, channel) + rwChannel, found := m.GetChannel(context.TODO(), nodeID, channel) s.False(found) s.Nil(rwChannel) - s.False(m.Match(nodeID, channel)) + s.False(m.Match(context.TODO(), nodeID, channel)) } func (s *ChannelManagerSuite) SetupTest() { @@ -122,7 +122,7 @@ func (s *ChannelManagerSuite) TestAddNode() { s.Require().NoError(err) var testNode int64 = 1 - err = m.AddNode(testNode) + err = m.AddNode(context.TODO(), testNode) s.NoError(err) info := m.store.GetNode(testNode) @@ -147,7 +147,7 @@ func (s *ChannelManagerSuite) TestAddNode() { s.checkAssignment(m, bufferID, ch, Standby) }) - err = m.AddNode(testNodeID) + err = m.AddNode(context.TODO(), testNodeID) s.NoError(err) lo.ForEach(testChannels, func(ch string, _ int) { @@ -169,7 +169,7 @@ func (s *ChannelManagerSuite) TestAddNode() { s.checkAssignment(m, storedNodeID, testChannel, Watched) - err = m.AddNode(testNodeID) + err = m.AddNode(context.TODO(), testNodeID) s.NoError(err) s.ElementsMatch([]int64{100, 1}, m.store.GetNodes()) s.checkNoAssignment(m, testNodeID, testChannel) @@ -178,7 +178,7 @@ func (s *ChannelManagerSuite) TestAddNode() { paramtable.Get().Save(paramtable.Get().DataCoordCfg.AutoBalance.Key, "true") defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.AutoBalance.Key) - err = m.AddNode(testNodeID) + err = m.AddNode(context.TODO(), testNodeID) s.NoError(err) s.ElementsMatch([]int64{100, 101, 1}, m.store.GetNodes()) s.checkNoAssignment(m, testNodeID, testChannel) @@ -198,7 +198,7 @@ func (s *ChannelManagerSuite) TestAddNode() { paramtable.Get().Save(paramtable.Get().DataCoordCfg.AutoBalance.Key, "true") defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.AutoBalance.Key) - err = m.AddNode(testNodeID) + err = m.AddNode(context.TODO(), testNodeID) s.NoError(err) s.ElementsMatch([]int64{testNodeID, 1}, m.store.GetNodes()) }) @@ -226,7 +226,7 @@ func (s *ChannelManagerSuite) TestWatch() { testCh string = "ch1" testNodeID int64 = 1 ) - err = m.AddNode(testNodeID) + err = m.AddNode(context.TODO(), testNodeID) s.NoError(err) s.checkNoAssignment(m, testNodeID, testCh) @@ -243,12 +243,12 @@ func (s *ChannelManagerSuite) TestRelease() { m, err := NewChannelManager(s.mockKv, s.mockHandler, s.mockCluster, s.mockAlloc) s.Require().NoError(err) - err = m.Release(1, "ch1") + err = m.Release(context.TODO(), 1, "ch1") s.Error(err) log.Info("error", zap.String("msg", err.Error())) - m.AddNode(1) - err = m.Release(1, "ch1") + m.AddNode(context.TODO(), 1) + err = m.Release(context.TODO(), 1, "ch1") s.Error(err) log.Info("error", zap.String("msg", err.Error())) }) @@ -261,7 +261,7 @@ func (s *ChannelManagerSuite) TestRelease() { m.Watch(context.TODO(), getChannel("ch1", 1)) s.checkAssignment(m, bufferID, "ch1", Standby) - err = m.Release(bufferID, "ch1") + err = m.Release(context.TODO(), bufferID, "ch1") s.NoError(err) s.checkAssignment(m, bufferID, "ch1", Standby) }) @@ -275,7 +275,7 @@ func (s *ChannelManagerSuite) TestDeleteNode() { info := m.store.GetNode(1) s.Require().Nil(info) - err = m.DeleteNode(1) + err = m.DeleteNode(context.TODO(), 1) s.NoError(err) }) s.Run("delete bufferID", func() { @@ -285,7 +285,7 @@ func (s *ChannelManagerSuite) TestDeleteNode() { info := m.store.GetNode(bufferID) s.Require().NotNil(info) - err = m.DeleteNode(bufferID) + err = m.DeleteNode(context.TODO(), bufferID) s.NoError(err) }) @@ -294,12 +294,12 @@ func (s *ChannelManagerSuite) TestDeleteNode() { m, err := NewChannelManager(s.mockKv, s.mockHandler, s.mockCluster, s.mockAlloc) s.Require().NoError(err) - err = m.AddNode(1) + err = m.AddNode(context.TODO(), 1) s.NoError(err) info := m.store.GetNode(bufferID) s.Require().NotNil(info) - err = m.DeleteNode(1) + err = m.DeleteNode(context.TODO(), 1) s.NoError(err) info = m.store.GetNode(1) s.Nil(info) @@ -317,10 +317,10 @@ func (s *ChannelManagerSuite) TestDeleteNode() { s.checkAssignment(m, 1, "ch2", Watched) s.checkAssignment(m, 1, "ch3", Watched) - err = m.AddNode(2) + err = m.AddNode(context.TODO(), 2) s.NoError(err) - err = m.DeleteNode(1) + err = m.DeleteNode(context.TODO(), 1) s.NoError(err) info := m.store.GetNode(bufferID) s.NotNil(info) @@ -363,7 +363,7 @@ func (s *ChannelManagerSuite) TestFindWatcher() { for _, test := range tests { s.Run(test.description, func() { - gotID, gotErr := m.FindWatcher(test.testCh) + gotID, gotErr := m.FindWatcher(context.TODO(), test.testCh) s.EqualValues(test.outNodeID, gotID) if test.outError { s.Error(gotErr) @@ -695,13 +695,13 @@ func (s *ChannelManagerSuite) TestStartup() { s.checkAssignment(m, 1, "ch2", Legacy) s.checkAssignment(m, bufferID, "ch3", Standby) - err = m.DeleteNode(1) + err = m.DeleteNode(context.TODO(), 1) s.NoError(err) s.checkAssignment(m, bufferID, "ch1", Standby) s.checkAssignment(m, bufferID, "ch2", Standby) - err = m.AddNode(2) + err = m.AddNode(context.TODO(), 2) s.NoError(err) s.checkAssignment(m, 2, "ch1", ToWatch) s.checkAssignment(m, 2, "ch2", ToWatch) @@ -731,7 +731,7 @@ func (s *ChannelManagerSuite) TestStartupNilSchema() { s.Require().NoError(err) for ch, node := range chNodes { - channel, got := m.GetChannel(node, ch) + channel, got := m.GetChannel(context.TODO(), node, ch) s.Require().True(got) s.Nil(channel.GetSchema()) s.Equal(ch, channel.GetName()) @@ -743,10 +743,10 @@ func (s *ChannelManagerSuite) TestStartupNilSchema() { nil, ) - err = m.DeleteNode(1) + err = m.DeleteNode(context.TODO(), 1) s.Require().NoError(err) - err = m.DeleteNode(3) + err = m.DeleteNode(context.TODO(), 3) s.Require().NoError(err) s.checkAssignment(m, bufferID, "ch1", Standby) @@ -754,7 +754,7 @@ func (s *ChannelManagerSuite) TestStartupNilSchema() { s.checkAssignment(m, bufferID, "ch3", Standby) for ch := range chNodes { - channel, got := m.GetChannel(bufferID, ch) + channel, got := m.GetChannel(context.TODO(), bufferID, ch) s.Require().True(got) s.NotNil(channel.GetSchema()) s.Equal(ch, channel.GetName()) @@ -764,14 +764,14 @@ func (s *ChannelManagerSuite) TestStartupNilSchema() { log.Info("Recovered non-nil schema channel", zap.Any("channel", channel)) } - err = m.AddNode(7) + err = m.AddNode(context.TODO(), 7) s.Require().NoError(err) s.checkAssignment(m, 7, "ch1", ToWatch) s.checkAssignment(m, 7, "ch2", ToWatch) s.checkAssignment(m, 7, "ch3", ToWatch) for ch := range chNodes { - channel, got := m.GetChannel(7, ch) + channel, got := m.GetChannel(context.TODO(), 7, ch) s.Require().True(got) s.NotNil(channel.GetSchema()) s.Equal(ch, channel.GetName()) @@ -842,7 +842,7 @@ func (s *ChannelManagerSuite) TestGetChannelWatchInfos() { }) cm := &ChannelManagerImpl{store: store} - infos := cm.GetChannelWatchInfos() + infos := cm.GetChannelWatchInfos(context.TODO()) s.Equal(2, len(infos)) s.Equal("ch1", infos[1]["ch1"].GetVchan().ChannelName) s.Equal("ch2", infos[2]["ch2"].GetVchan().ChannelName) @@ -850,6 +850,6 @@ func (s *ChannelManagerSuite) TestGetChannelWatchInfos() { // test empty value store.EXPECT().GetNodesChannels().Unset() store.EXPECT().GetNodesChannels().Return([]*NodeChannelInfo{}) - infos = cm.GetChannelWatchInfos() + infos = cm.GetChannelWatchInfos(context.TODO()) s.Equal(0, len(infos)) } diff --git a/internal/datacoord/channel_store.go b/internal/datacoord/channel_store.go index 822a4779fad30..afbad3b036072 100644 --- a/internal/datacoord/channel_store.go +++ b/internal/datacoord/channel_store.go @@ -19,6 +19,7 @@ package datacoord import ( "context" "fmt" + "go.opentelemetry.io/otel" "math" "strconv" "strings" @@ -69,7 +70,7 @@ type RWChannelStore interface { // Delete removes nodeID and returns its channels. RemoveNode(nodeID int64) // Update applies the operations in ChannelOpSet. - Update(op *ChannelOpSet) error + Update(ctx context.Context, op *ChannelOpSet) error // UpdateState is used by StateChannelStore only UpdateState(isSuccessful bool, channels ...RWChannel) @@ -339,6 +340,7 @@ func (c *StateChannelStore) Reload() error { if err != nil { return err } + ctx, _ := otel.Tracer(typeutil.DataCoordRole).Start(context.Background(), "channel store reload") for i := 0; i < len(keys); i++ { k := keys[i] v := values[i] @@ -355,13 +357,13 @@ func (c *StateChannelStore) Reload() error { c.AddNode(nodeID) - channel := NewStateChannelByWatchInfo(nodeID, info) + channel := NewStateChannelByWatchInfo(ctx, nodeID, info) c.channelsInfo[nodeID].AddChannel(channel) - log.Info("channel store reload channel", + log.Ctx(ctx).Info("channel store reload channel", zap.Int64("nodeID", nodeID), zap.String("channel", channel.Name)) metrics.DataCoordDmlChannelNum.WithLabelValues(strconv.FormatInt(nodeID, 10)).Set(float64(len(c.channelsInfo[nodeID].Channels))) } - log.Info("channel store reload done", zap.Duration("duration", record.ElapseSpan())) + log.Ctx(ctx).Info("channel store reload done", zap.Duration("duration", record.ElapseSpan())) return nil } @@ -399,7 +401,7 @@ func (c *StateChannelStore) SetLegacyChannelByNode(nodeIDs ...int64) { }) } -func (c *StateChannelStore) Update(opSet *ChannelOpSet) error { +func (c *StateChannelStore) Update(ctx context.Context, opSet *ChannelOpSet) error { // Split opset into multiple txn. Operations on the same channel must be executed in one txn. perChOps := opSet.SplitByChannel() @@ -408,16 +410,16 @@ func (c *StateChannelStore) Update(opSet *ChannelOpSet) error { operations := make([]*ChannelOp, 0, maxOperationsPerTxn) for _, opset := range perChOps { if !c.sanityCheckPerChannelOpSet(opset) { - log.Error("unsupported ChannelOpSet", zap.Any("OpSet", opset)) + log.Ctx(ctx).Error("unsupported ChannelOpSet", zap.Any("OpSet", opset)) continue } if opset.Len() > maxOperationsPerTxn { - log.Error("Operations for one channel exceeds maxOperationsPerTxn", + log.Ctx(ctx).Error("Operations for one channel exceeds maxOperationsPerTxn", zap.Any("opset size", opset.Len()), zap.Int("limit", maxOperationsPerTxn)) } if count+opset.Len() > maxOperationsPerTxn { - if err := c.updateMeta(NewChannelOpSet(operations...)); err != nil { + if err := c.updateMeta(ctx, NewChannelOpSet(operations...)); err != nil { return err } count = 0 @@ -430,7 +432,7 @@ func (c *StateChannelStore) Update(opSet *ChannelOpSet) error { return nil } - return c.updateMeta(NewChannelOpSet(operations...)) + return c.updateMeta(ctx, NewChannelOpSet(operations...)) } // remove from the assignments @@ -473,7 +475,7 @@ func (c *StateChannelStore) sanityCheckPerChannelOpSet(opSet *ChannelOpSet) bool } // DELETE + WATCH -func (c *StateChannelStore) updateMetaMemoryForPairOp(chName string, opSet *ChannelOpSet) error { +func (c *StateChannelStore) updateMetaMemoryForPairOp(ctx context.Context, chName string, opSet *ChannelOpSet) error { if !c.sanityCheckPerChannelOpSet(opSet) { return errUnknownOpType } @@ -508,6 +510,7 @@ func (c *StateChannelStore) updateMetaMemoryForPairOp(chName string, opSet *Chan } else { ch.setState(ToWatch) } + ch.currentCtx = ctx } return nil } @@ -524,18 +527,19 @@ func (c *StateChannelStore) getChannel(nodeID int64, channelName string) *StateC return nil } -func (c *StateChannelStore) updateMetaMemoryForSingleOp(op *ChannelOp) error { +func (c *StateChannelStore) updateMetaMemoryForSingleOp(ctx context.Context, op *ChannelOp) error { lo.ForEach(op.Channels, func(ch RWChannel, _ int) { switch op.Type { case Release: // release an already exsits storedChannel-node pair if channel := c.getChannel(op.NodeID, ch.GetName()); channel != nil { channel.setState(ToRelease) + channel.currentCtx = ctx } case Watch: storedChannel := c.getChannel(op.NodeID, ch.GetName()) if storedChannel == nil { // New Channel // set the correct assigment and state for NEW stateChannel - newChannel := NewStateChannel(ch) + newChannel := NewStateChannel(ctx, ch) newChannel.Assign(op.NodeID) if op.NodeID != bufferID { @@ -546,6 +550,7 @@ func (c *StateChannelStore) updateMetaMemoryForSingleOp(op *ChannelOp) error { c.addAssignment(op.NodeID, newChannel) } else { // assign to the original nodes storedChannel.setState(ToWatch) + storedChannel.currentCtx = ctx } case Delete: // Remove Channel c.removeAssignment(op.NodeID, ch.GetName()) @@ -556,7 +561,7 @@ func (c *StateChannelStore) updateMetaMemoryForSingleOp(op *ChannelOp) error { return nil } -func (c *StateChannelStore) updateMeta(opSet *ChannelOpSet) error { +func (c *StateChannelStore) updateMeta(ctx context.Context, opSet *ChannelOpSet) error { // Update ChannelStore's kv store. if err := c.txn(opSet); err != nil { return err @@ -567,10 +572,10 @@ func (c *StateChannelStore) updateMeta(opSet *ChannelOpSet) error { for chName, ops := range chOpSet { // DELETE + WATCH if ops.Len() == 2 { - c.updateMetaMemoryForPairOp(chName, ops) + c.updateMetaMemoryForPairOp(ctx, chName, ops) // RELEASE, DELETE, WATCH } else if ops.Len() == 1 { - c.updateMetaMemoryForSingleOp(ops.Collect()[0]) + c.updateMetaMemoryForSingleOp(ctx, ops.Collect()[0]) } else { log.Error("unsupported ChannelOpSet", zap.Any("OpSet", ops)) } diff --git a/internal/datacoord/channel_store_test.go b/internal/datacoord/channel_store_test.go index d15414c717d13..38787bbb53d70 100644 --- a/internal/datacoord/channel_store_test.go +++ b/internal/datacoord/channel_store_test.go @@ -255,7 +255,7 @@ func (s *StateChannelStoreSuite) TestUpdateWithTxnLimit() { // Get operations ops := genChannelOperations(1, Watch, test.inOpCount) - err := store.Update(ops) + err := store.Update(context.TODO(), ops) s.NoError(err) }) } @@ -287,7 +287,7 @@ func (s *StateChannelStoreSuite) TestUpdateMeta2000kSegs() { store.addAssignment(bufferID, ch) s.Require().Equal(1, store.GetNodeChannelCount(bufferID)) - err := store.updateMeta(opSet) + err := store.updateMeta(context.TODO(), opSet) s.NoError(err) got := store.GetNodeChannelsBy(WithNodeIDs(100)) @@ -403,7 +403,7 @@ func (s *StateChannelStoreSuite) TestUpdateMeta() { s.Require().Equal(1, store.GetNodeChannelCount(nodeID)) } - err := store.updateMeta(test.opSet) + err := store.updateMeta(context.TODO(), test.opSet) s.NoError(err) for nodeID, channels := range test.outAssignments { @@ -434,7 +434,7 @@ func (s *StateChannelStoreSuite) TestUpdateState() { store := NewStateChannelStore(s.mockTxn) ch := "ch-1" - channel := NewStateChannel(getChannel(ch, 1)) + channel := NewStateChannel(context.TODO(), getChannel(ch, 1)) channel.setState(test.inChannelState) store.channelsInfo[1] = &NodeChannelInfo{ NodeID: bufferID, @@ -513,7 +513,7 @@ func genChannelOperations(nodeID int64, opType ChannelOpType, num int) *ChannelO channels := make([]RWChannel, 0, num) for i := 0; i < num; i++ { name := fmt.Sprintf("ch%d", i) - channel := NewStateChannel(getChannel(name, 1)) + channel := NewStateChannel(context.TODO(), getChannel(name, 1)) channel.Info = generateWatchInfo(name, datapb.ChannelWatchState_ToWatch) channels = append(channels, channel) } diff --git a/internal/datacoord/cluster.go b/internal/datacoord/cluster.go index aeabfbbfc2de9..0e6c20b1c7fa9 100644 --- a/internal/datacoord/cluster.go +++ b/internal/datacoord/cluster.go @@ -92,13 +92,13 @@ func (c *ClusterImpl) Startup(ctx context.Context, nodes []*session.NodeInfo) er // Register registers a new node in cluster func (c *ClusterImpl) Register(node *session.NodeInfo) error { c.sessionManager.AddSession(node) - return c.channelManager.AddNode(node.NodeID) + return c.channelManager.AddNode(context.TODO(), node.NodeID) } // UnRegister removes a node from cluster func (c *ClusterImpl) UnRegister(node *session.NodeInfo) error { c.sessionManager.DeleteSession(node) - return c.channelManager.DeleteNode(node.NodeID) + return c.channelManager.DeleteNode(context.TODO(), node.NodeID) } // Watch tries to add a channel in datanode cluster @@ -109,7 +109,7 @@ func (c *ClusterImpl) Watch(ctx context.Context, ch RWChannel) error { // Flush sends async FlushSegments requests to dataNodes // which also according to channels where segments are assigned to. func (c *ClusterImpl) Flush(ctx context.Context, nodeID int64, channel string, segments []*datapb.SegmentInfo) error { - ch, founded := c.channelManager.GetChannel(nodeID, channel) + ch, founded := c.channelManager.GetChannel(ctx, nodeID, channel) if !founded { log.Warn("node is not matched with channel", zap.String("channel", channel), @@ -143,7 +143,7 @@ func (c *ClusterImpl) FlushChannels(ctx context.Context, nodeID int64, flushTs T } for _, channel := range channels { - if !c.channelManager.Match(nodeID, channel) { + if !c.channelManager.Match(ctx, nodeID, channel) { return fmt.Errorf("channel %s is not watched on node %d", channel, nodeID) } } diff --git a/internal/datacoord/cluster_test.go b/internal/datacoord/cluster_test.go index 0e3c95aa7fea0..0ca0083789bcd 100644 --- a/internal/datacoord/cluster_test.go +++ b/internal/datacoord/cluster_test.go @@ -85,8 +85,8 @@ func (suite *ClusterSuite) TestRegister() { info := &session.NodeInfo{NodeID: 1, Address: "addr1"} suite.mockSession.EXPECT().AddSession(mock.Anything).Return().Once() - suite.mockChManager.EXPECT().AddNode(mock.Anything). - RunAndReturn(func(nodeID int64) error { + suite.mockChManager.EXPECT().AddNode(mock.Anything, mock.Anything). + RunAndReturn(func(ctx context.Context, nodeID int64) error { suite.EqualValues(info.NodeID, nodeID) return nil }).Once() @@ -100,8 +100,8 @@ func (suite *ClusterSuite) TestUnregister() { info := &session.NodeInfo{NodeID: 1, Address: "addr1"} suite.mockSession.EXPECT().DeleteSession(mock.Anything).Return().Once() - suite.mockChManager.EXPECT().DeleteNode(mock.Anything). - RunAndReturn(func(nodeID int64) error { + suite.mockChManager.EXPECT().DeleteNode(mock.Anything, mock.Anything). + RunAndReturn(func(ctx context.Context, nodeID int64) error { suite.EqualValues(info.NodeID, nodeID) return nil }).Once() @@ -130,8 +130,8 @@ func (suite *ClusterSuite) TestWatch() { } func (suite *ClusterSuite) TestFlush() { - suite.mockChManager.EXPECT().GetChannel(mock.Anything, mock.Anything). - RunAndReturn(func(nodeID int64, channel string) (RWChannel, bool) { + suite.mockChManager.EXPECT().GetChannel(mock.Anything, mock.Anything, mock.Anything). + RunAndReturn(func(ctx context.Context, nodeID int64, channel string) (RWChannel, bool) { if nodeID == 1 { return nil, false } @@ -161,7 +161,7 @@ func (suite *ClusterSuite) TestFlushChannels() { suite.Run("channel not match with node", func() { suite.SetupTest() - suite.mockChManager.EXPECT().Match(mock.Anything, mock.Anything).Return(false).Once() + suite.mockChManager.EXPECT().Match(mock.Anything, mock.Anything, mock.Anything).Return(false).Once() cluster := NewClusterImpl(suite.mockSession, suite.mockChManager) err := cluster.FlushChannels(context.Background(), 1, 0, []string{"ch-1", "ch-2"}) suite.Error(err) @@ -171,7 +171,7 @@ func (suite *ClusterSuite) TestFlushChannels() { suite.SetupTest() channels := []string{"ch-1", "ch-2"} - suite.mockChManager.EXPECT().Match(mock.Anything, mock.Anything).Return(true).Times(len(channels)) + suite.mockChManager.EXPECT().Match(mock.Anything, mock.Anything, mock.Anything).Return(true).Times(len(channels)) suite.mockSession.EXPECT().FlushChannels(mock.Anything, mock.Anything, mock.Anything).Return(nil).Once() cluster := NewClusterImpl(suite.mockSession, suite.mockChManager) err := cluster.FlushChannels(context.Background(), 1, 0, channels) diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index 20f72ec454562..34cef6e424a0c 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -20,6 +20,7 @@ package datacoord import ( "context" "fmt" + "go.opentelemetry.io/otel" "math" "path" "strconv" @@ -1693,7 +1694,8 @@ func (m *meta) UpdateChannelCheckpoint(ctx context.Context, vChannel string, pos if pos == nil || pos.GetMsgID() == nil { return fmt.Errorf("channelCP is nil, vChannel=%s", vChannel) } - + ctx, sp := otel.Tracer("Meta").Start(ctx, "UpdateChannelCheckpoint") + defer sp.End() m.channelCPs.Lock() defer m.channelCPs.Unlock() diff --git a/internal/datacoord/metrics_info.go b/internal/datacoord/metrics_info.go index 0dd5c8cd720d8..cd84526fc836c 100644 --- a/internal/datacoord/metrics_info.go +++ b/internal/datacoord/metrics_info.go @@ -174,7 +174,7 @@ func (s *Server) getSegmentsJSON(ctx context.Context, req *milvuspb.GetMetricsRe func (s *Server) getDistJSON(ctx context.Context, req *milvuspb.GetMetricsRequest) string { segments := s.meta.getSegmentsMetrics(-1) var channels []*metricsinfo.DmChannel - for nodeID, ch := range s.channelManager.GetChannelWatchInfos() { + for nodeID, ch := range s.channelManager.GetChannelWatchInfos(ctx) { for _, chInfo := range ch { dmChannel := metrics.NewDMChannelFrom(chInfo.GetVchan()) dmChannel.NodeID = nodeID diff --git a/internal/datacoord/metrics_info_test.go b/internal/datacoord/metrics_info_test.go index c5d7c5250ca60..5b557243e4197 100644 --- a/internal/datacoord/metrics_info_test.go +++ b/internal/datacoord/metrics_info_test.go @@ -629,7 +629,7 @@ func TestGetDistJSON(t *testing.T) { } cm := NewMockChannelManager(t) - cm.EXPECT().GetChannelWatchInfos().Return(map[int64]map[string]*datapb.ChannelWatchInfo{ + cm.EXPECT().GetChannelWatchInfos(ctx).Return(map[int64]map[string]*datapb.ChannelWatchInfo{ 1: { "channel1": { State: datapb.ChannelWatchState_ToWatch, @@ -678,7 +678,7 @@ func TestGetDistJSON(t *testing.T) { svr.meta = &meta{segments: &SegmentsInfo{segments: map[int64]*SegmentInfo{}}} cm := NewMockChannelManager(t) - cm.EXPECT().GetChannelWatchInfos().Return(map[int64]map[string]*datapb.ChannelWatchInfo{}) + cm.EXPECT().GetChannelWatchInfos(ctx).Return(map[int64]map[string]*datapb.ChannelWatchInfo{}) svr.channelManager = cm expectedJSON := "{}" diff --git a/internal/datacoord/mock_channel_store.go b/internal/datacoord/mock_channel_store.go index a416692c7db09..f8558887a909f 100644 --- a/internal/datacoord/mock_channel_store.go +++ b/internal/datacoord/mock_channel_store.go @@ -2,7 +2,11 @@ package datacoord -import mock "github.com/stretchr/testify/mock" +import ( + context "context" + + mock "github.com/stretchr/testify/mock" +) // MockRWChannelStore is an autogenerated mock type for the RWChannelStore type type MockRWChannelStore struct { @@ -520,17 +524,17 @@ func (_c *MockRWChannelStore_SetLegacyChannelByNode_Call) RunAndReturn(run func( return _c } -// Update provides a mock function with given fields: op -func (_m *MockRWChannelStore) Update(op *ChannelOpSet) error { - ret := _m.Called(op) +// Update provides a mock function with given fields: ctx, op +func (_m *MockRWChannelStore) Update(ctx context.Context, op *ChannelOpSet) error { + ret := _m.Called(ctx, op) if len(ret) == 0 { panic("no return value specified for Update") } var r0 error - if rf, ok := ret.Get(0).(func(*ChannelOpSet) error); ok { - r0 = rf(op) + if rf, ok := ret.Get(0).(func(context.Context, *ChannelOpSet) error); ok { + r0 = rf(ctx, op) } else { r0 = ret.Error(0) } @@ -544,14 +548,15 @@ type MockRWChannelStore_Update_Call struct { } // Update is a helper method to define mock.On call +// - ctx context.Context // - op *ChannelOpSet -func (_e *MockRWChannelStore_Expecter) Update(op interface{}) *MockRWChannelStore_Update_Call { - return &MockRWChannelStore_Update_Call{Call: _e.mock.On("Update", op)} +func (_e *MockRWChannelStore_Expecter) Update(ctx interface{}, op interface{}) *MockRWChannelStore_Update_Call { + return &MockRWChannelStore_Update_Call{Call: _e.mock.On("Update", ctx, op)} } -func (_c *MockRWChannelStore_Update_Call) Run(run func(op *ChannelOpSet)) *MockRWChannelStore_Update_Call { +func (_c *MockRWChannelStore_Update_Call) Run(run func(ctx context.Context, op *ChannelOpSet)) *MockRWChannelStore_Update_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(*ChannelOpSet)) + run(args[0].(context.Context), args[1].(*ChannelOpSet)) }) return _c } @@ -561,7 +566,7 @@ func (_c *MockRWChannelStore_Update_Call) Return(_a0 error) *MockRWChannelStore_ return _c } -func (_c *MockRWChannelStore_Update_Call) RunAndReturn(run func(*ChannelOpSet) error) *MockRWChannelStore_Update_Call { +func (_c *MockRWChannelStore_Update_Call) RunAndReturn(run func(context.Context, *ChannelOpSet) error) *MockRWChannelStore_Update_Call { _c.Call.Return(run) return _c } diff --git a/internal/datacoord/mock_channelmanager.go b/internal/datacoord/mock_channelmanager.go index fccecee419de2..354c384824119 100644 --- a/internal/datacoord/mock_channelmanager.go +++ b/internal/datacoord/mock_channelmanager.go @@ -22,17 +22,17 @@ func (_m *MockChannelManager) EXPECT() *MockChannelManager_Expecter { return &MockChannelManager_Expecter{mock: &_m.Mock} } -// AddNode provides a mock function with given fields: nodeID -func (_m *MockChannelManager) AddNode(nodeID int64) error { - ret := _m.Called(nodeID) +// AddNode provides a mock function with given fields: ctx, nodeID +func (_m *MockChannelManager) AddNode(ctx context.Context, nodeID int64) error { + ret := _m.Called(ctx, nodeID) if len(ret) == 0 { panic("no return value specified for AddNode") } var r0 error - if rf, ok := ret.Get(0).(func(int64) error); ok { - r0 = rf(nodeID) + if rf, ok := ret.Get(0).(func(context.Context, int64) error); ok { + r0 = rf(ctx, nodeID) } else { r0 = ret.Error(0) } @@ -46,14 +46,15 @@ type MockChannelManager_AddNode_Call struct { } // AddNode is a helper method to define mock.On call +// - ctx context.Context // - nodeID int64 -func (_e *MockChannelManager_Expecter) AddNode(nodeID interface{}) *MockChannelManager_AddNode_Call { - return &MockChannelManager_AddNode_Call{Call: _e.mock.On("AddNode", nodeID)} +func (_e *MockChannelManager_Expecter) AddNode(ctx interface{}, nodeID interface{}) *MockChannelManager_AddNode_Call { + return &MockChannelManager_AddNode_Call{Call: _e.mock.On("AddNode", ctx, nodeID)} } -func (_c *MockChannelManager_AddNode_Call) Run(run func(nodeID int64)) *MockChannelManager_AddNode_Call { +func (_c *MockChannelManager_AddNode_Call) Run(run func(ctx context.Context, nodeID int64)) *MockChannelManager_AddNode_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(int64)) + run(args[0].(context.Context), args[1].(int64)) }) return _c } @@ -63,7 +64,7 @@ func (_c *MockChannelManager_AddNode_Call) Return(_a0 error) *MockChannelManager return _c } -func (_c *MockChannelManager_AddNode_Call) RunAndReturn(run func(int64) error) *MockChannelManager_AddNode_Call { +func (_c *MockChannelManager_AddNode_Call) RunAndReturn(run func(context.Context, int64) error) *MockChannelManager_AddNode_Call { _c.Call.Return(run) return _c } @@ -100,17 +101,17 @@ func (_c *MockChannelManager_Close_Call) RunAndReturn(run func()) *MockChannelMa return _c } -// DeleteNode provides a mock function with given fields: nodeID -func (_m *MockChannelManager) DeleteNode(nodeID int64) error { - ret := _m.Called(nodeID) +// DeleteNode provides a mock function with given fields: ctx, nodeID +func (_m *MockChannelManager) DeleteNode(ctx context.Context, nodeID int64) error { + ret := _m.Called(ctx, nodeID) if len(ret) == 0 { panic("no return value specified for DeleteNode") } var r0 error - if rf, ok := ret.Get(0).(func(int64) error); ok { - r0 = rf(nodeID) + if rf, ok := ret.Get(0).(func(context.Context, int64) error); ok { + r0 = rf(ctx, nodeID) } else { r0 = ret.Error(0) } @@ -124,14 +125,15 @@ type MockChannelManager_DeleteNode_Call struct { } // DeleteNode is a helper method to define mock.On call +// - ctx context.Context // - nodeID int64 -func (_e *MockChannelManager_Expecter) DeleteNode(nodeID interface{}) *MockChannelManager_DeleteNode_Call { - return &MockChannelManager_DeleteNode_Call{Call: _e.mock.On("DeleteNode", nodeID)} +func (_e *MockChannelManager_Expecter) DeleteNode(ctx interface{}, nodeID interface{}) *MockChannelManager_DeleteNode_Call { + return &MockChannelManager_DeleteNode_Call{Call: _e.mock.On("DeleteNode", ctx, nodeID)} } -func (_c *MockChannelManager_DeleteNode_Call) Run(run func(nodeID int64)) *MockChannelManager_DeleteNode_Call { +func (_c *MockChannelManager_DeleteNode_Call) Run(run func(ctx context.Context, nodeID int64)) *MockChannelManager_DeleteNode_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(int64)) + run(args[0].(context.Context), args[1].(int64)) }) return _c } @@ -141,14 +143,14 @@ func (_c *MockChannelManager_DeleteNode_Call) Return(_a0 error) *MockChannelMana return _c } -func (_c *MockChannelManager_DeleteNode_Call) RunAndReturn(run func(int64) error) *MockChannelManager_DeleteNode_Call { +func (_c *MockChannelManager_DeleteNode_Call) RunAndReturn(run func(context.Context, int64) error) *MockChannelManager_DeleteNode_Call { _c.Call.Return(run) return _c } -// FindWatcher provides a mock function with given fields: channel -func (_m *MockChannelManager) FindWatcher(channel string) (int64, error) { - ret := _m.Called(channel) +// FindWatcher provides a mock function with given fields: ctx, channel +func (_m *MockChannelManager) FindWatcher(ctx context.Context, channel string) (int64, error) { + ret := _m.Called(ctx, channel) if len(ret) == 0 { panic("no return value specified for FindWatcher") @@ -156,17 +158,17 @@ func (_m *MockChannelManager) FindWatcher(channel string) (int64, error) { var r0 int64 var r1 error - if rf, ok := ret.Get(0).(func(string) (int64, error)); ok { - return rf(channel) + if rf, ok := ret.Get(0).(func(context.Context, string) (int64, error)); ok { + return rf(ctx, channel) } - if rf, ok := ret.Get(0).(func(string) int64); ok { - r0 = rf(channel) + if rf, ok := ret.Get(0).(func(context.Context, string) int64); ok { + r0 = rf(ctx, channel) } else { r0 = ret.Get(0).(int64) } - if rf, ok := ret.Get(1).(func(string) error); ok { - r1 = rf(channel) + if rf, ok := ret.Get(1).(func(context.Context, string) error); ok { + r1 = rf(ctx, channel) } else { r1 = ret.Error(1) } @@ -180,14 +182,15 @@ type MockChannelManager_FindWatcher_Call struct { } // FindWatcher is a helper method to define mock.On call +// - ctx context.Context // - channel string -func (_e *MockChannelManager_Expecter) FindWatcher(channel interface{}) *MockChannelManager_FindWatcher_Call { - return &MockChannelManager_FindWatcher_Call{Call: _e.mock.On("FindWatcher", channel)} +func (_e *MockChannelManager_Expecter) FindWatcher(ctx interface{}, channel interface{}) *MockChannelManager_FindWatcher_Call { + return &MockChannelManager_FindWatcher_Call{Call: _e.mock.On("FindWatcher", ctx, channel)} } -func (_c *MockChannelManager_FindWatcher_Call) Run(run func(channel string)) *MockChannelManager_FindWatcher_Call { +func (_c *MockChannelManager_FindWatcher_Call) Run(run func(ctx context.Context, channel string)) *MockChannelManager_FindWatcher_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(string)) + run(args[0].(context.Context), args[1].(string)) }) return _c } @@ -197,14 +200,14 @@ func (_c *MockChannelManager_FindWatcher_Call) Return(_a0 int64, _a1 error) *Moc return _c } -func (_c *MockChannelManager_FindWatcher_Call) RunAndReturn(run func(string) (int64, error)) *MockChannelManager_FindWatcher_Call { +func (_c *MockChannelManager_FindWatcher_Call) RunAndReturn(run func(context.Context, string) (int64, error)) *MockChannelManager_FindWatcher_Call { _c.Call.Return(run) return _c } -// GetChannel provides a mock function with given fields: nodeID, channel -func (_m *MockChannelManager) GetChannel(nodeID int64, channel string) (RWChannel, bool) { - ret := _m.Called(nodeID, channel) +// GetChannel provides a mock function with given fields: ctx, nodeID, channel +func (_m *MockChannelManager) GetChannel(ctx context.Context, nodeID int64, channel string) (RWChannel, bool) { + ret := _m.Called(ctx, nodeID, channel) if len(ret) == 0 { panic("no return value specified for GetChannel") @@ -212,19 +215,19 @@ func (_m *MockChannelManager) GetChannel(nodeID int64, channel string) (RWChanne var r0 RWChannel var r1 bool - if rf, ok := ret.Get(0).(func(int64, string) (RWChannel, bool)); ok { - return rf(nodeID, channel) + if rf, ok := ret.Get(0).(func(context.Context, int64, string) (RWChannel, bool)); ok { + return rf(ctx, nodeID, channel) } - if rf, ok := ret.Get(0).(func(int64, string) RWChannel); ok { - r0 = rf(nodeID, channel) + if rf, ok := ret.Get(0).(func(context.Context, int64, string) RWChannel); ok { + r0 = rf(ctx, nodeID, channel) } else { if ret.Get(0) != nil { r0 = ret.Get(0).(RWChannel) } } - if rf, ok := ret.Get(1).(func(int64, string) bool); ok { - r1 = rf(nodeID, channel) + if rf, ok := ret.Get(1).(func(context.Context, int64, string) bool); ok { + r1 = rf(ctx, nodeID, channel) } else { r1 = ret.Get(1).(bool) } @@ -238,15 +241,16 @@ type MockChannelManager_GetChannel_Call struct { } // GetChannel is a helper method to define mock.On call +// - ctx context.Context // - nodeID int64 // - channel string -func (_e *MockChannelManager_Expecter) GetChannel(nodeID interface{}, channel interface{}) *MockChannelManager_GetChannel_Call { - return &MockChannelManager_GetChannel_Call{Call: _e.mock.On("GetChannel", nodeID, channel)} +func (_e *MockChannelManager_Expecter) GetChannel(ctx interface{}, nodeID interface{}, channel interface{}) *MockChannelManager_GetChannel_Call { + return &MockChannelManager_GetChannel_Call{Call: _e.mock.On("GetChannel", ctx, nodeID, channel)} } -func (_c *MockChannelManager_GetChannel_Call) Run(run func(nodeID int64, channel string)) *MockChannelManager_GetChannel_Call { +func (_c *MockChannelManager_GetChannel_Call) Run(run func(ctx context.Context, nodeID int64, channel string)) *MockChannelManager_GetChannel_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(int64), args[1].(string)) + run(args[0].(context.Context), args[1].(int64), args[2].(string)) }) return _c } @@ -256,22 +260,22 @@ func (_c *MockChannelManager_GetChannel_Call) Return(_a0 RWChannel, _a1 bool) *M return _c } -func (_c *MockChannelManager_GetChannel_Call) RunAndReturn(run func(int64, string) (RWChannel, bool)) *MockChannelManager_GetChannel_Call { +func (_c *MockChannelManager_GetChannel_Call) RunAndReturn(run func(context.Context, int64, string) (RWChannel, bool)) *MockChannelManager_GetChannel_Call { _c.Call.Return(run) return _c } -// GetChannelNamesByCollectionID provides a mock function with given fields: collectionID -func (_m *MockChannelManager) GetChannelNamesByCollectionID(collectionID int64) []string { - ret := _m.Called(collectionID) +// GetChannelNamesByCollectionID provides a mock function with given fields: ctx, collectionID +func (_m *MockChannelManager) GetChannelNamesByCollectionID(ctx context.Context, collectionID int64) []string { + ret := _m.Called(ctx, collectionID) if len(ret) == 0 { panic("no return value specified for GetChannelNamesByCollectionID") } var r0 []string - if rf, ok := ret.Get(0).(func(int64) []string); ok { - r0 = rf(collectionID) + if rf, ok := ret.Get(0).(func(context.Context, int64) []string); ok { + r0 = rf(ctx, collectionID) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]string) @@ -287,14 +291,15 @@ type MockChannelManager_GetChannelNamesByCollectionID_Call struct { } // GetChannelNamesByCollectionID is a helper method to define mock.On call +// - ctx context.Context // - collectionID int64 -func (_e *MockChannelManager_Expecter) GetChannelNamesByCollectionID(collectionID interface{}) *MockChannelManager_GetChannelNamesByCollectionID_Call { - return &MockChannelManager_GetChannelNamesByCollectionID_Call{Call: _e.mock.On("GetChannelNamesByCollectionID", collectionID)} +func (_e *MockChannelManager_Expecter) GetChannelNamesByCollectionID(ctx interface{}, collectionID interface{}) *MockChannelManager_GetChannelNamesByCollectionID_Call { + return &MockChannelManager_GetChannelNamesByCollectionID_Call{Call: _e.mock.On("GetChannelNamesByCollectionID", ctx, collectionID)} } -func (_c *MockChannelManager_GetChannelNamesByCollectionID_Call) Run(run func(collectionID int64)) *MockChannelManager_GetChannelNamesByCollectionID_Call { +func (_c *MockChannelManager_GetChannelNamesByCollectionID_Call) Run(run func(ctx context.Context, collectionID int64)) *MockChannelManager_GetChannelNamesByCollectionID_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(int64)) + run(args[0].(context.Context), args[1].(int64)) }) return _c } @@ -304,22 +309,22 @@ func (_c *MockChannelManager_GetChannelNamesByCollectionID_Call) Return(_a0 []st return _c } -func (_c *MockChannelManager_GetChannelNamesByCollectionID_Call) RunAndReturn(run func(int64) []string) *MockChannelManager_GetChannelNamesByCollectionID_Call { +func (_c *MockChannelManager_GetChannelNamesByCollectionID_Call) RunAndReturn(run func(context.Context, int64) []string) *MockChannelManager_GetChannelNamesByCollectionID_Call { _c.Call.Return(run) return _c } -// GetChannelWatchInfos provides a mock function with given fields: -func (_m *MockChannelManager) GetChannelWatchInfos() map[int64]map[string]*datapb.ChannelWatchInfo { - ret := _m.Called() +// GetChannelWatchInfos provides a mock function with given fields: ctx +func (_m *MockChannelManager) GetChannelWatchInfos(ctx context.Context) map[int64]map[string]*datapb.ChannelWatchInfo { + ret := _m.Called(ctx) if len(ret) == 0 { panic("no return value specified for GetChannelWatchInfos") } var r0 map[int64]map[string]*datapb.ChannelWatchInfo - if rf, ok := ret.Get(0).(func() map[int64]map[string]*datapb.ChannelWatchInfo); ok { - r0 = rf() + if rf, ok := ret.Get(0).(func(context.Context) map[int64]map[string]*datapb.ChannelWatchInfo); ok { + r0 = rf(ctx) } else { if ret.Get(0) != nil { r0 = ret.Get(0).(map[int64]map[string]*datapb.ChannelWatchInfo) @@ -335,13 +340,14 @@ type MockChannelManager_GetChannelWatchInfos_Call struct { } // GetChannelWatchInfos is a helper method to define mock.On call -func (_e *MockChannelManager_Expecter) GetChannelWatchInfos() *MockChannelManager_GetChannelWatchInfos_Call { - return &MockChannelManager_GetChannelWatchInfos_Call{Call: _e.mock.On("GetChannelWatchInfos")} +// - ctx context.Context +func (_e *MockChannelManager_Expecter) GetChannelWatchInfos(ctx interface{}) *MockChannelManager_GetChannelWatchInfos_Call { + return &MockChannelManager_GetChannelWatchInfos_Call{Call: _e.mock.On("GetChannelWatchInfos", ctx)} } -func (_c *MockChannelManager_GetChannelWatchInfos_Call) Run(run func()) *MockChannelManager_GetChannelWatchInfos_Call { +func (_c *MockChannelManager_GetChannelWatchInfos_Call) Run(run func(ctx context.Context)) *MockChannelManager_GetChannelWatchInfos_Call { _c.Call.Run(func(args mock.Arguments) { - run() + run(args[0].(context.Context)) }) return _c } @@ -351,22 +357,22 @@ func (_c *MockChannelManager_GetChannelWatchInfos_Call) Return(_a0 map[int64]map return _c } -func (_c *MockChannelManager_GetChannelWatchInfos_Call) RunAndReturn(run func() map[int64]map[string]*datapb.ChannelWatchInfo) *MockChannelManager_GetChannelWatchInfos_Call { +func (_c *MockChannelManager_GetChannelWatchInfos_Call) RunAndReturn(run func(context.Context) map[int64]map[string]*datapb.ChannelWatchInfo) *MockChannelManager_GetChannelWatchInfos_Call { _c.Call.Return(run) return _c } -// GetChannelsByCollectionID provides a mock function with given fields: collectionID -func (_m *MockChannelManager) GetChannelsByCollectionID(collectionID int64) []RWChannel { - ret := _m.Called(collectionID) +// GetChannelsByCollectionID provides a mock function with given fields: ctx, collectionID +func (_m *MockChannelManager) GetChannelsByCollectionID(ctx context.Context, collectionID int64) []RWChannel { + ret := _m.Called(ctx, collectionID) if len(ret) == 0 { panic("no return value specified for GetChannelsByCollectionID") } var r0 []RWChannel - if rf, ok := ret.Get(0).(func(int64) []RWChannel); ok { - r0 = rf(collectionID) + if rf, ok := ret.Get(0).(func(context.Context, int64) []RWChannel); ok { + r0 = rf(ctx, collectionID) } else { if ret.Get(0) != nil { r0 = ret.Get(0).([]RWChannel) @@ -382,14 +388,15 @@ type MockChannelManager_GetChannelsByCollectionID_Call struct { } // GetChannelsByCollectionID is a helper method to define mock.On call +// - ctx context.Context // - collectionID int64 -func (_e *MockChannelManager_Expecter) GetChannelsByCollectionID(collectionID interface{}) *MockChannelManager_GetChannelsByCollectionID_Call { - return &MockChannelManager_GetChannelsByCollectionID_Call{Call: _e.mock.On("GetChannelsByCollectionID", collectionID)} +func (_e *MockChannelManager_Expecter) GetChannelsByCollectionID(ctx interface{}, collectionID interface{}) *MockChannelManager_GetChannelsByCollectionID_Call { + return &MockChannelManager_GetChannelsByCollectionID_Call{Call: _e.mock.On("GetChannelsByCollectionID", ctx, collectionID)} } -func (_c *MockChannelManager_GetChannelsByCollectionID_Call) Run(run func(collectionID int64)) *MockChannelManager_GetChannelsByCollectionID_Call { +func (_c *MockChannelManager_GetChannelsByCollectionID_Call) Run(run func(ctx context.Context, collectionID int64)) *MockChannelManager_GetChannelsByCollectionID_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(int64)) + run(args[0].(context.Context), args[1].(int64)) }) return _c } @@ -399,22 +406,22 @@ func (_c *MockChannelManager_GetChannelsByCollectionID_Call) Return(_a0 []RWChan return _c } -func (_c *MockChannelManager_GetChannelsByCollectionID_Call) RunAndReturn(run func(int64) []RWChannel) *MockChannelManager_GetChannelsByCollectionID_Call { +func (_c *MockChannelManager_GetChannelsByCollectionID_Call) RunAndReturn(run func(context.Context, int64) []RWChannel) *MockChannelManager_GetChannelsByCollectionID_Call { _c.Call.Return(run) return _c } -// GetNodeChannelsByCollectionID provides a mock function with given fields: collectionID -func (_m *MockChannelManager) GetNodeChannelsByCollectionID(collectionID int64) map[int64][]string { - ret := _m.Called(collectionID) +// GetNodeChannelsByCollectionID provides a mock function with given fields: ctx, collectionID +func (_m *MockChannelManager) GetNodeChannelsByCollectionID(ctx context.Context, collectionID int64) map[int64][]string { + ret := _m.Called(ctx, collectionID) if len(ret) == 0 { panic("no return value specified for GetNodeChannelsByCollectionID") } var r0 map[int64][]string - if rf, ok := ret.Get(0).(func(int64) map[int64][]string); ok { - r0 = rf(collectionID) + if rf, ok := ret.Get(0).(func(context.Context, int64) map[int64][]string); ok { + r0 = rf(ctx, collectionID) } else { if ret.Get(0) != nil { r0 = ret.Get(0).(map[int64][]string) @@ -430,14 +437,15 @@ type MockChannelManager_GetNodeChannelsByCollectionID_Call struct { } // GetNodeChannelsByCollectionID is a helper method to define mock.On call +// - ctx context.Context // - collectionID int64 -func (_e *MockChannelManager_Expecter) GetNodeChannelsByCollectionID(collectionID interface{}) *MockChannelManager_GetNodeChannelsByCollectionID_Call { - return &MockChannelManager_GetNodeChannelsByCollectionID_Call{Call: _e.mock.On("GetNodeChannelsByCollectionID", collectionID)} +func (_e *MockChannelManager_Expecter) GetNodeChannelsByCollectionID(ctx interface{}, collectionID interface{}) *MockChannelManager_GetNodeChannelsByCollectionID_Call { + return &MockChannelManager_GetNodeChannelsByCollectionID_Call{Call: _e.mock.On("GetNodeChannelsByCollectionID", ctx, collectionID)} } -func (_c *MockChannelManager_GetNodeChannelsByCollectionID_Call) Run(run func(collectionID int64)) *MockChannelManager_GetNodeChannelsByCollectionID_Call { +func (_c *MockChannelManager_GetNodeChannelsByCollectionID_Call) Run(run func(ctx context.Context, collectionID int64)) *MockChannelManager_GetNodeChannelsByCollectionID_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(int64)) + run(args[0].(context.Context), args[1].(int64)) }) return _c } @@ -447,22 +455,22 @@ func (_c *MockChannelManager_GetNodeChannelsByCollectionID_Call) Return(_a0 map[ return _c } -func (_c *MockChannelManager_GetNodeChannelsByCollectionID_Call) RunAndReturn(run func(int64) map[int64][]string) *MockChannelManager_GetNodeChannelsByCollectionID_Call { +func (_c *MockChannelManager_GetNodeChannelsByCollectionID_Call) RunAndReturn(run func(context.Context, int64) map[int64][]string) *MockChannelManager_GetNodeChannelsByCollectionID_Call { _c.Call.Return(run) return _c } -// Match provides a mock function with given fields: nodeID, channel -func (_m *MockChannelManager) Match(nodeID int64, channel string) bool { - ret := _m.Called(nodeID, channel) +// Match provides a mock function with given fields: ctx, nodeID, channel +func (_m *MockChannelManager) Match(ctx context.Context, nodeID int64, channel string) bool { + ret := _m.Called(ctx, nodeID, channel) if len(ret) == 0 { panic("no return value specified for Match") } var r0 bool - if rf, ok := ret.Get(0).(func(int64, string) bool); ok { - r0 = rf(nodeID, channel) + if rf, ok := ret.Get(0).(func(context.Context, int64, string) bool); ok { + r0 = rf(ctx, nodeID, channel) } else { r0 = ret.Get(0).(bool) } @@ -476,15 +484,16 @@ type MockChannelManager_Match_Call struct { } // Match is a helper method to define mock.On call +// - ctx context.Context // - nodeID int64 // - channel string -func (_e *MockChannelManager_Expecter) Match(nodeID interface{}, channel interface{}) *MockChannelManager_Match_Call { - return &MockChannelManager_Match_Call{Call: _e.mock.On("Match", nodeID, channel)} +func (_e *MockChannelManager_Expecter) Match(ctx interface{}, nodeID interface{}, channel interface{}) *MockChannelManager_Match_Call { + return &MockChannelManager_Match_Call{Call: _e.mock.On("Match", ctx, nodeID, channel)} } -func (_c *MockChannelManager_Match_Call) Run(run func(nodeID int64, channel string)) *MockChannelManager_Match_Call { +func (_c *MockChannelManager_Match_Call) Run(run func(ctx context.Context, nodeID int64, channel string)) *MockChannelManager_Match_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(int64), args[1].(string)) + run(args[0].(context.Context), args[1].(int64), args[2].(string)) }) return _c } @@ -494,22 +503,22 @@ func (_c *MockChannelManager_Match_Call) Return(_a0 bool) *MockChannelManager_Ma return _c } -func (_c *MockChannelManager_Match_Call) RunAndReturn(run func(int64, string) bool) *MockChannelManager_Match_Call { +func (_c *MockChannelManager_Match_Call) RunAndReturn(run func(context.Context, int64, string) bool) *MockChannelManager_Match_Call { _c.Call.Return(run) return _c } -// Release provides a mock function with given fields: nodeID, channelName -func (_m *MockChannelManager) Release(nodeID int64, channelName string) error { - ret := _m.Called(nodeID, channelName) +// Release provides a mock function with given fields: ctx, nodeID, channelName +func (_m *MockChannelManager) Release(ctx context.Context, nodeID int64, channelName string) error { + ret := _m.Called(ctx, nodeID, channelName) if len(ret) == 0 { panic("no return value specified for Release") } var r0 error - if rf, ok := ret.Get(0).(func(int64, string) error); ok { - r0 = rf(nodeID, channelName) + if rf, ok := ret.Get(0).(func(context.Context, int64, string) error); ok { + r0 = rf(ctx, nodeID, channelName) } else { r0 = ret.Error(0) } @@ -523,15 +532,16 @@ type MockChannelManager_Release_Call struct { } // Release is a helper method to define mock.On call +// - ctx context.Context // - nodeID int64 // - channelName string -func (_e *MockChannelManager_Expecter) Release(nodeID interface{}, channelName interface{}) *MockChannelManager_Release_Call { - return &MockChannelManager_Release_Call{Call: _e.mock.On("Release", nodeID, channelName)} +func (_e *MockChannelManager_Expecter) Release(ctx interface{}, nodeID interface{}, channelName interface{}) *MockChannelManager_Release_Call { + return &MockChannelManager_Release_Call{Call: _e.mock.On("Release", ctx, nodeID, channelName)} } -func (_c *MockChannelManager_Release_Call) Run(run func(nodeID int64, channelName string)) *MockChannelManager_Release_Call { +func (_c *MockChannelManager_Release_Call) Run(run func(ctx context.Context, nodeID int64, channelName string)) *MockChannelManager_Release_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(int64), args[1].(string)) + run(args[0].(context.Context), args[1].(int64), args[2].(string)) }) return _c } @@ -541,7 +551,7 @@ func (_c *MockChannelManager_Release_Call) Return(_a0 error) *MockChannelManager return _c } -func (_c *MockChannelManager_Release_Call) RunAndReturn(run func(int64, string) error) *MockChannelManager_Release_Call { +func (_c *MockChannelManager_Release_Call) RunAndReturn(run func(context.Context, int64, string) error) *MockChannelManager_Release_Call { _c.Call.Return(run) return _c } diff --git a/internal/datacoord/policy.go b/internal/datacoord/policy.go index 43c256807bfa3..f8910f42ab165 100644 --- a/internal/datacoord/policy.go +++ b/internal/datacoord/policy.go @@ -17,6 +17,7 @@ package datacoord import ( + "context" "sort" "github.com/samber/lo" @@ -143,13 +144,13 @@ func AvgBalanceChannelPolicy(cluster Assignments) *ChannelOpSet { // if empty, this policy will balance the currentCluster // // ExclusiveNodes means donot assign channels to these nodes. -type AssignPolicy func(currentCluster Assignments, toAssign *NodeChannelInfo, exclusiveNodes []int64) *ChannelOpSet +type AssignPolicy func(ctx context.Context, currentCluster Assignments, toAssign *NodeChannelInfo, exclusiveNodes []int64) *ChannelOpSet -func EmptyAssignPolicy(currentCluster Assignments, toAssign *NodeChannelInfo, execlusiveNodes []int64) *ChannelOpSet { +func EmptyAssignPolicy(ctx context.Context, currentCluster Assignments, toAssign *NodeChannelInfo, execlusiveNodes []int64) *ChannelOpSet { return nil } -func AvgAssignByCountPolicy(currentCluster Assignments, toAssign *NodeChannelInfo, execlusiveNodes []int64) *ChannelOpSet { +func AvgAssignByCountPolicy(ctx context.Context, currentCluster Assignments, toAssign *NodeChannelInfo, execlusiveNodes []int64) *ChannelOpSet { var ( toCluster Assignments fromCluster Assignments @@ -210,7 +211,7 @@ func AvgAssignByCountPolicy(currentCluster Assignments, toAssign *NodeChannelInf opSet.Delete(toAssign.NodeID, chs...) } - log.Info("Assign channels to nodes by channel count", + log.Ctx(ctx).Info("Assign channels to nodes by channel count", zap.Int("toAssign channel count", len(toAssign.Channels)), zap.Any("original nodeID", toAssign.NodeID), zap.Int64s("exclusive nodes", execlusiveNodes), diff --git a/internal/datacoord/policy_test.go b/internal/datacoord/policy_test.go index 422e12ccfa527..f2f9836f8768f 100644 --- a/internal/datacoord/policy_test.go +++ b/internal/datacoord/policy_test.go @@ -17,6 +17,7 @@ package datacoord import ( + "context" "fmt" "testing" @@ -40,7 +41,8 @@ func getChannel(name string, collID int64) *StateChannel { Info: &datapb.ChannelWatchInfo{ Vchan: &datapb.VchannelInfo{ChannelName: name, CollectionID: collID}, }, - Schema: &schemapb.CollectionSchema{Name: "coll1"}, + Schema: &schemapb.CollectionSchema{Name: "coll1"}, + currentCtx: context.TODO(), } } @@ -126,7 +128,7 @@ func (s *AssignByCountPolicySuite) SetupSubTest() { func (s *AssignByCountPolicySuite) TestWithoutUnassignedChannels() { s.Run("balance without exclusive", func() { - opSet := AvgAssignByCountPolicy(s.curCluster, nil, nil) + opSet := AvgAssignByCountPolicy(context.TODO(), s.curCluster, nil, nil) s.NotNil(opSet) s.Equal(2, opSet.GetChannelNumber()) @@ -136,7 +138,7 @@ func (s *AssignByCountPolicySuite) TestWithoutUnassignedChannels() { }) s.Run("balance with exclusive", func() { execlusiveNodes := []int64{1, 2} - opSet := AvgAssignByCountPolicy(s.curCluster, nil, execlusiveNodes) + opSet := AvgAssignByCountPolicy(context.TODO(), s.curCluster, nil, execlusiveNodes) s.NotNil(opSet) s.Equal(2, opSet.GetChannelNumber()) @@ -163,7 +165,7 @@ func (s *AssignByCountPolicySuite) TestWithoutUnassignedChannels() { } execlusiveNodes := []int64{4, 5} - opSet := AvgAssignByCountPolicy(s.curCluster, nil, execlusiveNodes) + opSet := AvgAssignByCountPolicy(context.TODO(), s.curCluster, nil, execlusiveNodes) s.NotNil(opSet) }) } @@ -172,7 +174,7 @@ func (s *AssignByCountPolicySuite) TestWithUnassignedChannels() { s.Run("one unassigned channel", func() { unassigned := NewNodeChannelInfo(bufferID, getChannel("new-ch-1", 1)) - opSet := AvgAssignByCountPolicy(s.curCluster, unassigned, nil) + opSet := AvgAssignByCountPolicy(context.TODO(), s.curCluster, unassigned, nil) s.NotNil(opSet) s.Equal(1, opSet.GetChannelNumber()) @@ -192,7 +194,7 @@ func (s *AssignByCountPolicySuite) TestWithUnassignedChannels() { getChannel("new-ch-3", 1), ) - opSet := AvgAssignByCountPolicy(s.curCluster, unassigned, nil) + opSet := AvgAssignByCountPolicy(context.TODO(), s.curCluster, unassigned, nil) s.NotNil(opSet) s.Equal(3, opSet.GetChannelNumber()) @@ -216,7 +218,7 @@ func (s *AssignByCountPolicySuite) TestWithUnassignedChannels() { getChannel("new-ch-3", 1), ) - opSet := AvgAssignByCountPolicy(s.curCluster, unassigned, []int64{1, 2}) + opSet := AvgAssignByCountPolicy(context.TODO(), s.curCluster, unassigned, []int64{1, 2}) s.NotNil(opSet) s.Equal(3, opSet.GetChannelNumber()) @@ -248,7 +250,7 @@ func (s *AssignByCountPolicySuite) TestWithUnassignedChannels() { } unassigned := NewNodeChannelInfo(bufferID, unassignedChannels...) - opSet := AvgAssignByCountPolicy(s.curCluster, unassigned, nil) + opSet := AvgAssignByCountPolicy(context.TODO(), s.curCluster, unassigned, nil) s.NotNil(opSet) s.Equal(67, opSet.GetChannelNumber()) @@ -274,7 +276,7 @@ func (s *AssignByCountPolicySuite) TestWithUnassignedChannels() { } s.Require().NotNil(unassigned) - opSet := AvgAssignByCountPolicy(s.curCluster, unassigned, []int64{1, 2}) + opSet := AvgAssignByCountPolicy(context.TODO(), s.curCluster, unassigned, []int64{1, 2}) s.NotNil(opSet) s.Equal(3, opSet.GetChannelNumber()) @@ -302,7 +304,7 @@ func (s *AssignByCountPolicySuite) TestWithUnassignedChannels() { getChannel("new-ch-3", 1), ) - opSet := AvgAssignByCountPolicy(curCluster, unassigned, nil) + opSet := AvgAssignByCountPolicy(context.TODO(), curCluster, unassigned, nil) s.NotNil(opSet) s.Equal(3, opSet.GetChannelNumber()) diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index acfb03d8af543..119531445bab9 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -928,8 +928,8 @@ func TestDropVirtualChannel(t *testing.T) { ctx := context.Background() chanName := "ch1" mockChManager := NewMockChannelManager(t) - mockChManager.EXPECT().Match(mock.Anything, mock.Anything).Return(true).Twice() - mockChManager.EXPECT().Release(mock.Anything, chanName).Return(nil).Twice() + mockChManager.EXPECT().Match(mock.Anything, mock.Anything, mock.Anything).Return(true).Twice() + mockChManager.EXPECT().Release(mock.Anything, mock.Anything, chanName).Return(nil).Twice() svr.channelManager = mockChManager req := &datapb.DropVirtualChannelRequest{ @@ -1011,7 +1011,7 @@ func TestDropVirtualChannel(t *testing.T) { svr := newTestServer(t) defer closeTestServer(t, svr) mockChManager := NewMockChannelManager(t) - mockChManager.EXPECT().Match(mock.Anything, mock.Anything).Return(false).Once() + mockChManager.EXPECT().Match(mock.Anything, mock.Anything, mock.Anything).Return(false).Once() svr.channelManager = mockChManager resp, err := svr.DropVirtualChannel(context.Background(), &datapb.DropVirtualChannelRequest{ @@ -2313,7 +2313,7 @@ func TestDataCoordServer_UpdateChannelCheckpoint(t *testing.T) { datanodeID := int64(1) channelManager := NewMockChannelManager(t) - channelManager.EXPECT().Match(datanodeID, mockVChannel).Return(true) + channelManager.EXPECT().Match(mock.Anything, datanodeID, mockVChannel).Return(true) svr.channelManager = channelManager req := &datapb.UpdateChannelCheckpointRequest{ @@ -2359,7 +2359,7 @@ func TestDataCoordServer_UpdateChannelCheckpoint(t *testing.T) { datanodeID := int64(1) channelManager := NewMockChannelManager(t) - channelManager.EXPECT().Match(datanodeID, mockVChannel).Return(false) + channelManager.EXPECT().Match(mock.Anything, datanodeID, mockVChannel).Return(false) svr.channelManager = channelManager req := &datapb.UpdateChannelCheckpointRequest{ @@ -2512,9 +2512,9 @@ func Test_CheckHealth(t *testing.T) { getChannelManager := func(t *testing.T, findWatcherOk bool) ChannelManager { channelManager := NewMockChannelManager(t) if findWatcherOk { - channelManager.EXPECT().FindWatcher(mock.Anything).Return(0, nil) + channelManager.EXPECT().FindWatcher(mock.Anything, mock.Anything).Return(0, nil) } else { - channelManager.EXPECT().FindWatcher(mock.Anything).Return(0, errors.New("error")) + channelManager.EXPECT().FindWatcher(mock.Anything, mock.Anything).Return(0, errors.New("error")) } return channelManager } diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index dc19a9abab03c..842931d08ff29 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -142,7 +142,7 @@ func (s *Server) Flush(ctx context.Context, req *datapb.FlushRequest) (*datapb.F if !streamingutil.IsStreamingServiceEnabled() { var isUnimplemented bool err = retry.Do(ctx, func() error { - nodeChannels := s.channelManager.GetNodeChannelsByCollectionID(req.GetCollectionID()) + nodeChannels := s.channelManager.GetNodeChannelsByCollectionID(ctx, req.GetCollectionID()) for nodeID, channelNames := range nodeChannels { err = s.cluster.FlushChannels(ctx, nodeID, ts, channelNames) @@ -499,7 +499,7 @@ func (s *Server) SaveBinlogPaths(ctx context.Context, req *datapb.SaveBinlogPath // for compatibility issue , if len(channelName) not exist, skip the check // Also avoid to handle segment not found error if not the owner of shard if len(channelName) != 0 { - if !s.channelManager.Match(nodeID, channelName) { + if !s.channelManager.Match(ctx, nodeID, channelName) { err := merr.WrapErrChannelNotFound(channelName, fmt.Sprintf("for node %d", nodeID)) log.Warn("node is not matched with channel", zap.String("channel", channelName), zap.Error(err)) return merr.Status(err), nil @@ -608,7 +608,7 @@ func (s *Server) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtual // validate nodeID := req.GetBase().GetSourceID() - if !s.channelManager.Match(nodeID, channel) { + if !s.channelManager.Match(ctx, nodeID, channel) { err := merr.WrapErrChannelNotFound(channel, fmt.Sprintf("for node %d", nodeID)) resp.Status = merr.Status(err) log.Warn("node is not matched with channel", zap.String("channel", channel), zap.Int64("nodeID", nodeID)) @@ -643,7 +643,7 @@ func (s *Server) DropVirtualChannel(ctx context.Context, req *datapb.DropVirtual // if streaming service is enabled, the channel manager will never manage the channel. // so we don't need to release the channel anymore. log.Info("DropVChannel plan to remove", zap.String("channel", channel)) - err = s.channelManager.Release(nodeID, channel) + err = s.channelManager.Release(ctx, nodeID, channel) if err != nil { log.Warn("DropVChannel failed to ReleaseAndRemove", zap.String("channel", channel), zap.Error(err)) } @@ -863,7 +863,7 @@ func (s *Server) GetRecoveryInfoV2(ctx context.Context, req *datapb.GetRecoveryI Status: merr.Status(err), }, nil } - channels := s.channelManager.GetChannelsByCollectionID(collectionID) + channels := s.channelManager.GetChannelsByCollectionID(ctx, collectionID) channelInfos := make([]*datapb.VchannelInfo, 0, len(channels)) flushedIDs := make(typeutil.UniqueSet) for _, ch := range channels { @@ -951,7 +951,7 @@ func (s *Server) GetChannelRecoveryInfo(ctx context.Context, req *datapb.GetChan return resp, nil } - channel := NewRWChannel(req.GetVchannel(), collectionID, nil, collection.Schema, 0) // TODO: remove RWChannel, just use vchannel + collectionID + channel := NewRWChannel(ctx, req.GetVchannel(), collectionID, nil, collection.Schema, 0) // TODO: remove RWChannel, just use vchannel + collectionID channelInfo := s.handler.GetDataVChanPositions(channel, allPartitionID) if channelInfo.SeekPosition == nil { log.Warn("channel recovery start position is not found, may collection is on creating") @@ -1242,7 +1242,7 @@ func (s *Server) WatchChannels(ctx context.Context, req *datapb.WatchChannelsReq }, nil } for _, channelName := range req.GetChannelNames() { - ch := NewRWChannel(channelName, req.GetCollectionID(), req.GetStartPositions(), req.GetSchema(), req.GetCreateTimestamp()) + ch := NewRWChannel(ctx, channelName, req.GetCollectionID(), req.GetStartPositions(), req.GetSchema(), req.GetCreateTimestamp()) err := s.channelManager.Watch(ctx, ch) if err != nil { log.Warn("fail to watch channelName", zap.Error(err)) @@ -1302,7 +1302,7 @@ func (s *Server) GetFlushState(ctx context.Context, req *datapb.GetFlushStateReq } } - channels := s.channelManager.GetChannelsByCollectionID(req.GetCollectionID()) + channels := s.channelManager.GetChannelsByCollectionID(ctx, req.GetCollectionID()) if len(channels) == 0 { // For compatibility with old client resp.Flushed = true @@ -1405,7 +1405,7 @@ func (s *Server) UpdateChannelCheckpoint(ctx context.Context, req *datapb.Update // For compatibility with old client if req.GetVChannel() != "" && req.GetPosition() != nil { channel := req.GetVChannel() - if !s.channelManager.Match(nodeID, channel) { + if !s.channelManager.Match(ctx, nodeID, channel) { log.Warn("node is not matched with channel", zap.String("channel", channel), zap.Int64("nodeID", nodeID)) return merr.Status(merr.WrapErrChannelNotFound(channel, fmt.Sprintf("from node %d", nodeID))), nil } @@ -1419,7 +1419,7 @@ func (s *Server) UpdateChannelCheckpoint(ctx context.Context, req *datapb.Update checkpoints := lo.Filter(req.GetChannelCheckpoints(), func(cp *msgpb.MsgPosition, _ int) bool { channel := cp.GetChannelName() - matched := s.channelManager.Match(nodeID, channel) + matched := s.channelManager.Match(ctx, nodeID, channel) if !matched { log.Warn("node is not matched with channel", zap.String("channel", channel), zap.Int64("nodeID", nodeID)) } @@ -1479,7 +1479,7 @@ func (s *Server) handleDataNodeTtMsg(ctx context.Context, ttMsg *msgpb.DataNodeT log.RatedWarn(60.0, "time tick lag behind for more than 1 minutes") } // ignore report from a different node - if !s.channelManager.Match(sourceID, channel) { + if !s.channelManager.Match(ctx, sourceID, channel) { log.Warn("node is not matched with channel") return nil } diff --git a/internal/datacoord/services_test.go b/internal/datacoord/services_test.go index 3a92456b5e55d..91809fb5cbfed 100644 --- a/internal/datacoord/services_test.go +++ b/internal/datacoord/services_test.go @@ -92,10 +92,10 @@ func genMsg(msgType commonpb.MsgType, ch string, t Timestamp, sourceID int64) *m } func (s *ServerSuite) TestGetFlushState_ByFlushTs() { - s.mockChMgr.EXPECT().GetChannelsByCollectionID(int64(0)). + s.mockChMgr.EXPECT().GetChannelsByCollectionID(context.TODO(), int64(0)). Return([]RWChannel{&channelMeta{Name: "ch1", CollectionID: 0}}).Times(3) - s.mockChMgr.EXPECT().GetChannelsByCollectionID(int64(1)).Return(nil).Times(1) + s.mockChMgr.EXPECT().GetChannelsByCollectionID(context.TODO(), int64(1)).Return(nil).Times(1) tests := []struct { description string inTs Timestamp @@ -132,7 +132,7 @@ func (s *ServerSuite) TestGetFlushState_ByFlushTs() { } func (s *ServerSuite) TestGetFlushState_BySegment() { - s.mockChMgr.EXPECT().GetChannelsByCollectionID(mock.Anything). + s.mockChMgr.EXPECT().GetChannelsByCollectionID(mock.Anything, mock.Anything). Return([]RWChannel{&channelMeta{Name: "ch1", CollectionID: 0}}).Times(3) tests := []struct { @@ -185,7 +185,7 @@ func (s *ServerSuite) TestSaveBinlogPath_ClosedServer() { } func (s *ServerSuite) TestSaveBinlogPath_ChannelNotMatch() { - s.mockChMgr.EXPECT().Match(mock.Anything, mock.Anything).Return(false) + s.mockChMgr.EXPECT().Match(mock.Anything, mock.Anything, mock.Anything).Return(false) resp, err := s.testServer.SaveBinlogPaths(context.Background(), &datapb.SaveBinlogPathsRequest{ SegmentID: 1, Channel: "test", @@ -195,7 +195,7 @@ func (s *ServerSuite) TestSaveBinlogPath_ChannelNotMatch() { } func (s *ServerSuite) TestSaveBinlogPath_SaveUnhealthySegment() { - s.mockChMgr.EXPECT().Match(int64(0), "ch1").Return(true) + s.mockChMgr.EXPECT().Match(context.TODO(), int64(0), "ch1").Return(true) s.testServer.meta.AddCollection(&collectionInfo{ID: 0}) segments := map[int64]commonpb.SegmentState{ @@ -240,7 +240,7 @@ func (s *ServerSuite) TestSaveBinlogPath_SaveUnhealthySegment() { } func (s *ServerSuite) TestSaveBinlogPath_SaveDroppedSegment() { - s.mockChMgr.EXPECT().Match(int64(0), "ch1").Return(true) + s.mockChMgr.EXPECT().Match(mock.Anything, int64(0), "ch1").Return(true) s.testServer.meta.AddCollection(&collectionInfo{ID: 0}) segments := map[int64]commonpb.SegmentState{ @@ -312,7 +312,7 @@ func (s *ServerSuite) TestSaveBinlogPath_SaveDroppedSegment() { } func (s *ServerSuite) TestSaveBinlogPath_L0Segment() { - s.mockChMgr.EXPECT().Match(int64(0), "ch1").Return(true) + s.mockChMgr.EXPECT().Match(mock.Anything, int64(0), "ch1").Return(true) s.testServer.meta.AddCollection(&collectionInfo{ID: 0}) segment := s.testServer.meta.GetHealthySegment(context.TODO(), 1) @@ -365,7 +365,7 @@ func (s *ServerSuite) TestSaveBinlogPath_L0Segment() { } func (s *ServerSuite) TestSaveBinlogPath_NormalCase() { - s.mockChMgr.EXPECT().Match(int64(0), "ch1").Return(true) + s.mockChMgr.EXPECT().Match(mock.Anything, int64(0), "ch1").Return(true) s.testServer.meta.AddCollection(&collectionInfo{ID: 0}) segments := map[int64]int64{ @@ -488,7 +488,7 @@ func (s *ServerSuite) TestFlush_NormalCase() { CollectionID: 0, } - s.mockChMgr.EXPECT().GetNodeChannelsByCollectionID(mock.Anything).Return(map[int64][]string{ + s.mockChMgr.EXPECT().GetNodeChannelsByCollectionID(mock.Anything, mock.Anything).Return(map[int64][]string{ 1: {"channel-1"}, }) @@ -581,7 +581,7 @@ func (s *ServerSuite) TestFlush_RollingUpgrade() { mockCluster.EXPECT().Close().Maybe() s.testServer.cluster = mockCluster s.testServer.meta.AddCollection(&collectionInfo{ID: 0}) - s.mockChMgr.EXPECT().GetNodeChannelsByCollectionID(mock.Anything).Return(map[int64][]string{ + s.mockChMgr.EXPECT().GetNodeChannelsByCollectionID(mock.Anything, mock.Anything).Return(map[int64][]string{ 1: {"channel-1"}, }).Once() @@ -923,7 +923,7 @@ func TestGetRecoveryInfoV2(t *testing.T) { assert.NoError(t, err) ch := &channelMeta{Name: "vchan1", CollectionID: 0} - svr.channelManager.AddNode(0) + svr.channelManager.AddNode(context.TODO(), 0) svr.channelManager.Watch(context.Background(), ch) req := &datapb.GetRecoveryInfoRequestV2{ @@ -1003,7 +1003,7 @@ func TestGetRecoveryInfoV2(t *testing.T) { assert.NoError(t, err) ch := &channelMeta{Name: "vchan1", CollectionID: 0} - svr.channelManager.AddNode(0) + svr.channelManager.AddNode(context.TODO(), 0) svr.channelManager.Watch(context.Background(), ch) req := &datapb.GetRecoveryInfoRequestV2{ @@ -1096,7 +1096,7 @@ func TestGetRecoveryInfoV2(t *testing.T) { }) assert.NoError(t, err) - err = svr.channelManager.AddNode(0) + err = svr.channelManager.AddNode(context.TODO(), 0) assert.NoError(t, err) err = svr.channelManager.Watch(context.Background(), &channelMeta{Name: "vchan1", CollectionID: 0}) assert.NoError(t, err) @@ -1145,7 +1145,7 @@ func TestGetRecoveryInfoV2(t *testing.T) { assert.NoError(t, err) ch := &channelMeta{Name: "vchan1", CollectionID: 0} - svr.channelManager.AddNode(0) + svr.channelManager.AddNode(context.TODO(), 0) svr.channelManager.Watch(context.Background(), ch) req := &datapb.GetRecoveryInfoRequestV2{ @@ -1191,7 +1191,7 @@ func TestGetRecoveryInfoV2(t *testing.T) { assert.NoError(t, err) ch := &channelMeta{Name: "vchan1", CollectionID: 0} - svr.channelManager.AddNode(0) + svr.channelManager.AddNode(context.TODO(), 0) svr.channelManager.Watch(context.Background(), ch) req := &datapb.GetRecoveryInfoRequestV2{ @@ -1275,7 +1275,7 @@ func TestGetRecoveryInfoV2(t *testing.T) { }) ch := &channelMeta{Name: "vchan1", CollectionID: 0} - svr.channelManager.AddNode(0) + svr.channelManager.AddNode(context.TODO(), 0) svr.channelManager.Watch(context.Background(), ch) req := &datapb.GetRecoveryInfoRequestV2{ diff --git a/internal/datacoord/sync_segments_scheduler.go b/internal/datacoord/sync_segments_scheduler.go index 849b41a4460cb..2842723d156d4 100644 --- a/internal/datacoord/sync_segments_scheduler.go +++ b/internal/datacoord/sync_segments_scheduler.go @@ -99,7 +99,7 @@ func (sss *SyncSegmentsScheduler) SyncSegmentsForCollections(ctx context.Context continue } for _, channelName := range collInfo.VChannelNames { - nodeID, err := sss.channelManager.FindWatcher(channelName) + nodeID, err := sss.channelManager.FindWatcher(ctx, channelName) if err != nil { log.Warn("find watcher for channel failed", zap.Int64("collectionID", collID), zap.String("channelName", channelName), zap.Error(err)) diff --git a/internal/datacoord/sync_segments_scheduler_test.go b/internal/datacoord/sync_segments_scheduler_test.go index d9a0043bd2522..df03fe2a042ec 100644 --- a/internal/datacoord/sync_segments_scheduler_test.go +++ b/internal/datacoord/sync_segments_scheduler_test.go @@ -321,7 +321,7 @@ func (s *SyncSegmentsSchedulerSuite) SetupTest() { func (s *SyncSegmentsSchedulerSuite) Test_newSyncSegmentsScheduler() { cm := NewMockChannelManager(s.T()) - cm.EXPECT().FindWatcher(mock.Anything).Return(100, nil) + cm.EXPECT().FindWatcher(mock.Anything, mock.Anything).Return(100, nil) sm := session.NewMockDataNodeManager(s.T()) sm.EXPECT().SyncSegments(mock.Anything, mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, i int64, request *datapb.SyncSegmentsRequest) error { @@ -362,12 +362,12 @@ func (s *SyncSegmentsSchedulerSuite) Test_SyncSegmentsFail() { }) s.Run("find watcher failed", func() { - cm.EXPECT().FindWatcher(mock.Anything).Return(0, errors.New("mock error")).Twice() + cm.EXPECT().FindWatcher(mock.Anything, mock.Anything).Return(0, errors.New("mock error")).Twice() sss.SyncSegmentsForCollections(ctx) }) s.Run("sync segment failed", func() { - cm.EXPECT().FindWatcher(mock.Anything).Return(100, nil) + cm.EXPECT().FindWatcher(mock.Anything, mock.Anything).Return(100, nil) sm.EXPECT().SyncSegments(mock.Anything, mock.Anything, mock.Anything).Return(errors.New("mock error")) sss.SyncSegmentsForCollections(ctx) }) diff --git a/internal/datacoord/util.go b/internal/datacoord/util.go index 19b4fa2f68949..ccfbdbbfd5644 100644 --- a/internal/datacoord/util.go +++ b/internal/datacoord/util.go @@ -315,7 +315,7 @@ func CheckAllChannelsWatched(meta *meta, channelManager ChannelManager) error { } for _, channelName := range collInfo.VChannelNames { - _, err := channelManager.FindWatcher(channelName) + _, err := channelManager.FindWatcher(context.TODO(), channelName) if err != nil { log.Warn("find watcher for channel failed", zap.Int64("collectionID", collID), zap.String("channelName", channelName), zap.Error(err)) diff --git a/internal/datanode/channel/channel_manager.go b/internal/datanode/channel/channel_manager.go index d43de2ce7cabd..fc252142e1b63 100644 --- a/internal/datanode/channel/channel_manager.go +++ b/internal/datanode/channel/channel_manager.go @@ -18,10 +18,12 @@ package channel import ( "context" + "fmt" "sync" "time" "github.com/cockroachdb/errors" + "go.opentelemetry.io/otel" "go.uber.org/zap" "github.com/milvus-io/milvus/internal/flushcommon/pipeline" @@ -40,7 +42,7 @@ type ( ) type ChannelManager interface { - Submit(info *datapb.ChannelWatchInfo) error + Submit(ctx context.Context, info *datapb.ChannelWatchInfo) error GetProgress(info *datapb.ChannelWatchInfo) *datapb.ChannelOperationProgressResponse Close() Start() @@ -88,30 +90,31 @@ func NewChannelManager(pipelineParams *util.PipelineParams, fgManager pipeline.F return &cm } -func (m *ChannelManagerImpl) Submit(info *datapb.ChannelWatchInfo) error { +func (m *ChannelManagerImpl) Submit(ctx context.Context, info *datapb.ChannelWatchInfo) error { channel := info.GetVchan().GetChannelName() + log := log.Ctx(ctx).With(zap.Int64("opID", info.GetOpID()), zap.String("channel", channel)) // skip enqueue the same operation resubmmited by datacoord if runner, ok := m.opRunners.Get(channel); ok { if _, exists := runner.Exist(info.GetOpID()); exists { - log.Warn("op already exist, skip", zap.Int64("opID", info.GetOpID()), zap.String("channel", channel)) + log.Warn("op already exist, skip") return nil } } if info.GetState() == datapb.ChannelWatchState_ToWatch && m.fgManager.HasFlowgraphWithOpID(channel, info.GetOpID()) { - log.Warn("Watch op already finished, skip", zap.Int64("opID", info.GetOpID()), zap.String("channel", channel)) + log.Warn("Watch op already finished, skip") return nil } if info.GetState() == datapb.ChannelWatchState_ToRelease && !m.fgManager.HasFlowgraph(channel) { - log.Warn("Release op already finished, skip", zap.Int64("opID", info.GetOpID()), zap.String("channel", channel)) + log.Warn("Release op already finished, skip") return nil } - runner := m.getOrCreateRunner(channel) + runner := m.getOrCreateRunner(ctx, channel) return runner.Enqueue(info) } @@ -224,8 +227,8 @@ func (m *ChannelManagerImpl) handleOpState(opState *opState) { m.finishOp(opState.opID, opState.channel) } -func (m *ChannelManagerImpl) getOrCreateRunner(channel string) *opRunner { - runner, loaded := m.opRunners.GetOrInsert(channel, NewOpRunner(channel, m.pipelineParams, m.releaseFunc, executeWatch, m.communicateCh)) +func (m *ChannelManagerImpl) getOrCreateRunner(ctx context.Context, channel string) *opRunner { + runner, loaded := m.opRunners.GetOrInsert(channel, NewOpRunner(ctx, channel, m.pipelineParams, m.releaseFunc, executeWatch, m.communicateCh)) if !loaded { runner.Start() } @@ -256,9 +259,11 @@ type opRunner struct { closeCh lifetime.SafeChan closeWg sync.WaitGroup + + opCtx context.Context } -func NewOpRunner(channel string, pipelineParams *util.PipelineParams, releaseF releaseFunc, watchF watchFunc, resultCh chan *opState) *opRunner { +func NewOpRunner(ctx context.Context, channel string, pipelineParams *util.PipelineParams, releaseF releaseFunc, watchF watchFunc, resultCh chan *opState) *opRunner { return &opRunner{ channel: channel, pipelineParams: pipelineParams, @@ -268,6 +273,7 @@ func NewOpRunner(channel string, pipelineParams *util.PipelineParams, releaseF r allOps: make(map[typeutil.UniqueID]*opInfo), resultCh: resultCh, closeCh: lifetime.NewSafeChan(), + opCtx: context.WithoutCancel(ctx), } } @@ -328,7 +334,7 @@ func (r *opRunner) UnfinishedOpSize() int { // Execute excutes channel operations, channel state is validated during enqueue func (r *opRunner) Execute(info *datapb.ChannelWatchInfo) *opState { - log.Info("Start to execute channel operation", + log.Ctx(r.opCtx).Info("Start to execute channel operation", zap.String("channel", info.GetVchan().GetChannelName()), zap.Int64("opID", info.GetOpID()), zap.String("state", info.GetState().String()), @@ -358,7 +364,10 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState { channel: info.GetVchan().GetChannelName(), opID: info.GetOpID(), } - log := log.With(zap.String("channel", opState.channel), zap.Int64("opID", opState.opID)) + ctx, sp := otel.Tracer(typeutil.DataNodeRole).Start(r.opCtx, "watchWithTimer") + defer sp.End() + sp.AddEvent(fmt.Sprintf("channel=%s", opState.channel)) + log := log.Ctx(ctx).With(zap.String("channel", opState.channel), zap.Int64("opID", opState.opID)) tickler := util.NewTickler() ok := r.updateTickler(info.GetOpID(), tickler) @@ -373,7 +382,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState { ) watchTimeout := paramtable.Get().DataCoordCfg.WatchTimeoutInterval.GetAsDuration(time.Second) - ctx, cancel := context.WithCancel(context.Background()) + ctx, cancel := context.WithCancel(ctx) defer cancel() startTimer := func(finishWg *sync.WaitGroup) { @@ -421,7 +430,7 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState { finishWaiter.Add(2) go startTimer(&finishWaiter) - go func() { + watchFuncExec := func() { defer finishWaiter.Done() fg, err := r.watchFunc(ctx, r.pipelineParams, info, tickler) if err != nil { @@ -432,7 +441,8 @@ func (r *opRunner) watchWithTimer(info *datapb.ChannelWatchInfo) *opState { opState.fg = fg successSig <- struct{}{} } - }() + } + go watchFuncExec() finishWaiter.Wait() return opState @@ -449,7 +459,10 @@ func (r *opRunner) releaseWithTimer(releaseFunc releaseFunc, channel string, opI finishWaiter sync.WaitGroup ) - log := log.With(zap.Int64("opID", opID), zap.String("channel", channel)) + ctx, sp := otel.Tracer(typeutil.DataNodeRole).Start(r.opCtx, "releaseWithTimer") + defer sp.End() + sp.AddEvent(fmt.Sprintf("channel=%s", channel)) + log := log.Ctx(ctx).With(zap.Int64("opID", opID), zap.String("channel", channel)) startTimer := func(finishWaiter *sync.WaitGroup) { defer finishWaiter.Done() @@ -517,6 +530,8 @@ type opState struct { // executeWatch will always return, won't be stuck, either success or fail. func executeWatch(ctx context.Context, pipelineParams *util.PipelineParams, info *datapb.ChannelWatchInfo, tickler *util.Tickler) (*pipeline.DataSyncService, error) { + ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "executeWatch") + defer span.End() dataSyncService, err := pipeline.NewDataSyncService(ctx, pipelineParams, info, tickler) if err != nil { return nil, err diff --git a/internal/datanode/channel/channel_manager_test.go b/internal/datanode/channel/channel_manager_test.go index 307b5be0261b3..12e6b2f8108c2 100644 --- a/internal/datanode/channel/channel_manager_test.go +++ b/internal/datanode/channel/channel_manager_test.go @@ -96,7 +96,7 @@ func (s *OpRunnerSuite) TestWatchWithTimer() { log.Info("mock release func") } - runner := NewOpRunner(channel, s.pipelineParams, mockReleaseFunc, executeWatch, commuCh) + runner := NewOpRunner(context.TODO(), channel, s.pipelineParams, mockReleaseFunc, executeWatch, commuCh) err := runner.Enqueue(info) s.Require().NoError(err) @@ -123,7 +123,7 @@ func (s *OpRunnerSuite) TestWatchTimeout() { return nil, errors.New("timeout") } - runner := NewOpRunner(channel, s.pipelineParams, mockReleaseFunc, mockWatchFunc, commuCh) + runner := NewOpRunner(context.TODO(), channel, s.pipelineParams, mockReleaseFunc, mockWatchFunc, commuCh) runner.Start() defer runner.Close() err := runner.Enqueue(info) @@ -188,7 +188,7 @@ func (s *ChannelManagerSuite) TestReleaseStuck() { info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) s.Require().Equal(0, s.manager.opRunners.Len()) - err := s.manager.Submit(info) + err := s.manager.Submit(context.TODO(), info) s.Require().NoError(err) opState := <-s.manager.communicateCh @@ -200,7 +200,7 @@ func (s *ChannelManagerSuite) TestReleaseStuck() { paramtable.Get().Save(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key, "0.1") defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.WatchTimeoutInterval.Key) - err = s.manager.Submit(releaseInfo) + err = s.manager.Submit(context.TODO(), releaseInfo) s.NoError(err) opState = <-s.manager.communicateCh @@ -224,7 +224,7 @@ func (s *ChannelManagerSuite) TestSubmitIdempotent() { s.Require().Equal(0, s.manager.opRunners.Len()) for i := 0; i < 10; i++ { - err := s.manager.Submit(info) + err := s.manager.Submit(context.TODO(), info) s.NoError(err) } @@ -242,7 +242,7 @@ func (s *ChannelManagerSuite) TestSubmitSkip() { info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) s.Require().Equal(0, s.manager.opRunners.Len()) - err := s.manager.Submit(info) + err := s.manager.Submit(context.TODO(), info) s.NoError(err) s.Equal(1, s.manager.opRunners.Len()) @@ -254,7 +254,7 @@ func (s *ChannelManagerSuite) TestSubmitSkip() { s.Equal(info.GetOpID(), opState.fg.GetOpID()) s.manager.handleOpState(opState) - err = s.manager.Submit(info) + err = s.manager.Submit(context.TODO(), info) s.NoError(err) runner, ok := s.manager.opRunners.Get(channel) @@ -267,7 +267,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() { // watch info := GetWatchInfoByOpID(100, channel, datapb.ChannelWatchState_ToWatch) - err := s.manager.Submit(info) + err := s.manager.Submit(context.TODO(), info) s.NoError(err) // wait for result @@ -292,7 +292,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() { // release info = GetWatchInfoByOpID(101, channel, datapb.ChannelWatchState_ToRelease) - err = s.manager.Submit(info) + err = s.manager.Submit(context.TODO(), info) s.NoError(err) // wait for result @@ -309,7 +309,7 @@ func (s *ChannelManagerSuite) TestSubmitWatchAndRelease() { s.False(s.manager.opRunners.Contain(info.GetVchan().GetChannelName())) s.Equal(0, s.manager.opRunners.Len()) - err = s.manager.Submit(info) + err = s.manager.Submit(context.TODO(), info) s.NoError(err) runner, ok := s.manager.opRunners.Get(channel) s.False(ok) diff --git a/internal/datanode/channel/mock_channelmanager.go b/internal/datanode/channel/mock_channelmanager.go index f94f890280b0c..1d1deb0a6591a 100644 --- a/internal/datanode/channel/mock_channelmanager.go +++ b/internal/datanode/channel/mock_channelmanager.go @@ -1,8 +1,10 @@ -// Code generated by mockery v2.32.4. DO NOT EDIT. +// Code generated by mockery v2.46.0. DO NOT EDIT. package channel import ( + context "context" + datapb "github.com/milvus-io/milvus/internal/proto/datapb" mock "github.com/stretchr/testify/mock" ) @@ -56,6 +58,10 @@ func (_c *MockChannelManager_Close_Call) RunAndReturn(run func()) *MockChannelMa func (_m *MockChannelManager) GetProgress(info *datapb.ChannelWatchInfo) *datapb.ChannelOperationProgressResponse { ret := _m.Called(info) + if len(ret) == 0 { + panic("no return value specified for GetProgress") + } + var r0 *datapb.ChannelOperationProgressResponse if rf, ok := ret.Get(0).(func(*datapb.ChannelWatchInfo) *datapb.ChannelOperationProgressResponse); ok { r0 = rf(info) @@ -128,13 +134,17 @@ func (_c *MockChannelManager_Start_Call) RunAndReturn(run func()) *MockChannelMa return _c } -// Submit provides a mock function with given fields: info -func (_m *MockChannelManager) Submit(info *datapb.ChannelWatchInfo) error { - ret := _m.Called(info) +// Submit provides a mock function with given fields: ctx, info +func (_m *MockChannelManager) Submit(ctx context.Context, info *datapb.ChannelWatchInfo) error { + ret := _m.Called(ctx, info) + + if len(ret) == 0 { + panic("no return value specified for Submit") + } var r0 error - if rf, ok := ret.Get(0).(func(*datapb.ChannelWatchInfo) error); ok { - r0 = rf(info) + if rf, ok := ret.Get(0).(func(context.Context, *datapb.ChannelWatchInfo) error); ok { + r0 = rf(ctx, info) } else { r0 = ret.Error(0) } @@ -148,14 +158,15 @@ type MockChannelManager_Submit_Call struct { } // Submit is a helper method to define mock.On call +// - ctx context.Context // - info *datapb.ChannelWatchInfo -func (_e *MockChannelManager_Expecter) Submit(info interface{}) *MockChannelManager_Submit_Call { - return &MockChannelManager_Submit_Call{Call: _e.mock.On("Submit", info)} +func (_e *MockChannelManager_Expecter) Submit(ctx interface{}, info interface{}) *MockChannelManager_Submit_Call { + return &MockChannelManager_Submit_Call{Call: _e.mock.On("Submit", ctx, info)} } -func (_c *MockChannelManager_Submit_Call) Run(run func(info *datapb.ChannelWatchInfo)) *MockChannelManager_Submit_Call { +func (_c *MockChannelManager_Submit_Call) Run(run func(ctx context.Context, info *datapb.ChannelWatchInfo)) *MockChannelManager_Submit_Call { _c.Call.Run(func(args mock.Arguments) { - run(args[0].(*datapb.ChannelWatchInfo)) + run(args[0].(context.Context), args[1].(*datapb.ChannelWatchInfo)) }) return _c } @@ -165,7 +176,7 @@ func (_c *MockChannelManager_Submit_Call) Return(_a0 error) *MockChannelManager_ return _c } -func (_c *MockChannelManager_Submit_Call) RunAndReturn(run func(*datapb.ChannelWatchInfo) error) *MockChannelManager_Submit_Call { +func (_c *MockChannelManager_Submit_Call) RunAndReturn(run func(context.Context, *datapb.ChannelWatchInfo) error) *MockChannelManager_Submit_Call { _c.Call.Return(run) return _c } diff --git a/internal/datanode/services.go b/internal/datanode/services.go index fa14b7d4182a5..21a7c555527ec 100644 --- a/internal/datanode/services.go +++ b/internal/datanode/services.go @@ -365,7 +365,7 @@ func (node *DataNode) NotifyChannelOperation(ctx context.Context, req *datapb.Ch } for _, info := range req.GetInfos() { - err := node.channelManager.Submit(info) + err := node.channelManager.Submit(ctx, info) if err != nil { log.Warn("Submit error", zap.Error(err)) return merr.Status(err), nil diff --git a/internal/flushcommon/broker/datacoord.go b/internal/flushcommon/broker/datacoord.go index 453e3eb4a0fe1..bb24a3415fa10 100644 --- a/internal/flushcommon/broker/datacoord.go +++ b/internal/flushcommon/broker/datacoord.go @@ -78,13 +78,13 @@ func (dc *dataCoordBroker) GetSegmentInfo(ctx context.Context, ids []int64) ([]* IncludeUnHealthy: true, }) if err := merr.CheckRPCCall(infoResp, err); err != nil { - log.Warn("Fail to get SegmentInfo by ids from datacoord", zap.Int64s("segments", ids), zap.Error(err)) + log.Ctx(ctx).Warn("Fail to get SegmentInfo by ids from datacoord", zap.Int64s("segments", ids), zap.Error(err)) return nil, err } err = binlog.DecompressMultiBinLogs(infoResp.GetInfos()) if err != nil { - log.Warn("Fail to DecompressMultiBinLogs", zap.Int64s("segments", ids), zap.Error(err)) + log.Ctx(ctx).Warn("Fail to DecompressMultiBinLogs", zap.Int64s("segments", ids), zap.Error(err)) return nil, err } return infoResp, nil @@ -98,7 +98,7 @@ func (dc *dataCoordBroker) GetSegmentInfo(ctx context.Context, ids []int64) ([]* resp, err := getSegmentInfo(ids[startIdx:endIdx]) if err != nil { - log.Warn("Fail to get SegmentInfo", zap.Int("total segment num", len(ids)), zap.Int("returned num", startIdx)) + log.Ctx(ctx).Warn("Fail to get SegmentInfo", zap.Int("total segment num", len(ids)), zap.Int("returned num", startIdx)) return nil, err } ret = append(ret, resp.GetInfos()...) diff --git a/internal/flushcommon/pipeline/data_sync_service.go b/internal/flushcommon/pipeline/data_sync_service.go index 3e69ad016d433..2ae09fdca51f5 100644 --- a/internal/flushcommon/pipeline/data_sync_service.go +++ b/internal/flushcommon/pipeline/data_sync_service.go @@ -20,6 +20,7 @@ import ( "context" "sync" + "go.opentelemetry.io/otel" "go.uber.org/zap" "github.com/milvus-io/milvus/internal/datanode/compaction" @@ -151,7 +152,7 @@ func initMetaCache(initCtx context.Context, chunkManager storage.ChunkManager, i loadSegmentStats := func(segType string, segments []*datapb.SegmentInfo) { for _, item := range segments { - log.Info("recover segments from checkpoints", + log.Ctx(initCtx).Info("recover segments from checkpoints", zap.String("vChannelName", item.GetInsertChannel()), zap.Int64("segmentID", item.GetID()), zap.Int64("numRows", item.GetNumOfRows()), @@ -314,7 +315,7 @@ func getServiceWithChannel(initCtx context.Context, params *util.PipelineParams, writebuffer.WithIDAllocator(params.Allocator), writebuffer.WithTaskObserverCallback(wbTaskObserverCallback)) if err != nil { - log.Warn("failed to register channel buffer", zap.String("channel", channelName), zap.Error(err)) + log.Ctx(initCtx).Warn("failed to register channel buffer", zap.String("channel", channelName), zap.Error(err)) return nil, err } @@ -332,31 +333,38 @@ func NewDataSyncService(initCtx context.Context, pipelineParams *util.PipelinePa unflushedSegmentInfos []*datapb.SegmentInfo flushedSegmentInfos []*datapb.SegmentInfo ) + ctx, sp := otel.Tracer(typeutil.DataNodeRole).Start(initCtx, "NewDataSyncService") + defer sp.End() + log.Ctx(ctx).Info("NewDataSyncService init") if len(info.GetVchan().GetUnflushedSegmentIds()) > 0 { - unflushedSegmentInfos, err = pipelineParams.Broker.GetSegmentInfo(initCtx, info.GetVchan().GetUnflushedSegmentIds()) + log.Ctx(ctx).Debug("get unflushed segmentInfos", zap.Int("count", len(info.GetVchan().GetUnflushedSegmentIds()))) + unflushedSegmentInfos, err = pipelineParams.Broker.GetSegmentInfo(ctx, info.GetVchan().GetUnflushedSegmentIds()) if err != nil { return nil, err } } if len(info.GetVchan().GetFlushedSegmentIds()) > 0 { - flushedSegmentInfos, err = pipelineParams.Broker.GetSegmentInfo(initCtx, info.GetVchan().GetFlushedSegmentIds()) + log.Ctx(ctx).Debug("get flushed segmentInfos", zap.Int("count", len(info.GetVchan().GetFlushedSegmentIds()))) + flushedSegmentInfos, err = pipelineParams.Broker.GetSegmentInfo(ctx, info.GetVchan().GetFlushedSegmentIds()) if err != nil { return nil, err } } // init metaCache meta - if metaCache, err = getMetaCacheWithTickler(initCtx, pipelineParams, info, tickler, unflushedSegmentInfos, flushedSegmentInfos); err != nil { + if metaCache, err = getMetaCacheWithTickler(ctx, pipelineParams, info, tickler, unflushedSegmentInfos, flushedSegmentInfos); err != nil { return nil, err } - input, err := createNewInputFromDispatcher(initCtx, pipelineParams.DispClient, info.GetVchan().GetChannelName(), info.GetVchan().GetSeekPosition()) + log.Ctx(ctx).Info("createNewInputFromDispatcher") + input, err := createNewInputFromDispatcher(ctx, pipelineParams.DispClient, info.GetVchan().GetChannelName(), info.GetVchan().GetSeekPosition()) if err != nil { return nil, err } - ds, err := getServiceWithChannel(initCtx, pipelineParams, info, metaCache, unflushedSegmentInfos, flushedSegmentInfos, input, nil, nil) + ds, err := getServiceWithChannel(ctx, pipelineParams, info, metaCache, unflushedSegmentInfos, flushedSegmentInfos, input, nil, nil) if err != nil { // deregister channel if failed to init flowgraph to avoid resource leak. + log.Ctx(initCtx).Warn("failed to init flowgraph, deregister channel", zap.String("channel", info.GetVchan().GetChannelName()), zap.Error(err)) pipelineParams.DispClient.Deregister(info.GetVchan().GetChannelName()) return nil, err } diff --git a/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go b/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go index cb62f585f63a4..043136eee8450 100644 --- a/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go +++ b/internal/flushcommon/pipeline/flow_graph_dmstream_input_node.go @@ -58,8 +58,9 @@ func newDmInputNode(dmNodeConfig *nodeConfig, input <-chan *msgstream.MsgPack) * } func createNewInputFromDispatcher(initCtx context.Context, dispatcherClient msgdispatcher.Client, vchannel string, seekPos *msgpb.MsgPosition) (<-chan *msgstream.MsgPack, error) { - log := log.With(zap.Int64("nodeID", paramtable.GetNodeID()), + log := log.Ctx(initCtx).With(zap.Int64("nodeID", paramtable.GetNodeID()), zap.String("vchannel", vchannel)) + log.Info("create new input from msgDispatcher") if seekPos != nil && len(seekPos.MsgID) != 0 { input, err := dispatcherClient.Register(initCtx, vchannel, seekPos, common.SubscriptionPositionUnknown) if err != nil { diff --git a/internal/flushcommon/syncmgr/meta_writer.go b/internal/flushcommon/syncmgr/meta_writer.go index 3f6773c37a831..f7f278c0ee528 100644 --- a/internal/flushcommon/syncmgr/meta_writer.go +++ b/internal/flushcommon/syncmgr/meta_writer.go @@ -158,7 +158,7 @@ func (b *brokerMetaWriter) DropChannel(ctx context.Context, channelName string) return false, nil }, b.opts...) if err != nil { - log.Warn("failed to DropChannel", + log.Ctx(ctx).Warn("failed to DropChannel", zap.String("channel", channelName), zap.Error(err)) } diff --git a/internal/kv/etcd/etcd_kv.go b/internal/kv/etcd/etcd_kv.go index b6be89d890739..c177fa18db1a0 100644 --- a/internal/kv/etcd/etcd_kv.go +++ b/internal/kv/etcd/etcd_kv.go @@ -24,6 +24,7 @@ import ( "time" clientv3 "go.etcd.io/etcd/client/v3" + "go.opentelemetry.io/otel" "go.uber.org/zap" "github.com/milvus-io/milvus/pkg/kv" @@ -339,6 +340,9 @@ func (kv *etcdKV) LoadBytesWithRevision(ctx context.Context, key string) ([]stri func (kv *etcdKV) Save(ctx context.Context, key, value string) error { start := time.Now() key = path.Join(kv.rootPath, key) + ctx, sp := otel.Tracer("etcdKV").Start(ctx, "Save") + defer sp.End() + sp.AddEvent(fmt.Sprintf("key=%s", key)) ctx, cancel := context.WithTimeout(ctx, kv.requestTimeout) defer cancel() CheckValueSizeAndWarn(key, value) @@ -381,6 +385,9 @@ func (kv *etcdKV) MultiSave(ctx context.Context, kvs map[string]string) error { ops = append(ops, clientv3.OpPut(path.Join(kv.rootPath, key), value)) } + ctx, sp := otel.Tracer("etcdKV").Start(ctx, "MultiSave") + defer sp.End() + sp.AddEvent(fmt.Sprintf("key count=%d", len(keys))) ctx, cancel := context.WithTimeout(ctx, kv.requestTimeout) defer cancel() @@ -417,10 +424,13 @@ func (kv *etcdKV) MultiSaveBytes(ctx context.Context, kvs map[string][]byte) err // RemoveWithPrefix removes the keys with given prefix. func (kv *etcdKV) RemoveWithPrefix(ctx context.Context, prefix string) error { + ctx, sp := otel.Tracer("etcdKV").Start(ctx, "RemoveWithPrefix") + defer sp.End() start := time.Now() key := path.Join(kv.rootPath, prefix) ctx, cancel := context.WithTimeout(ctx, kv.requestTimeout) defer cancel() + sp.AddEvent(fmt.Sprintf("prefix=%s", key)) _, err := kv.removeEtcdMeta(ctx, key, clientv3.WithPrefix()) CheckElapseAndWarn(start, "Slow etcd operation remove with prefix", zap.String("prefix", prefix)) @@ -429,10 +439,13 @@ func (kv *etcdKV) RemoveWithPrefix(ctx context.Context, prefix string) error { // Remove removes the key. func (kv *etcdKV) Remove(ctx context.Context, key string) error { + ctx, sp := otel.Tracer("etcdKV").Start(ctx, "Remove") + defer sp.End() start := time.Now() key = path.Join(kv.rootPath, key) ctx, cancel := context.WithTimeout(ctx, kv.requestTimeout) defer cancel() + sp.AddEvent(fmt.Sprintf("key=%s", key)) _, err := kv.removeEtcdMeta(ctx, key) CheckElapseAndWarn(start, "Slow etcd operation remove", zap.String("key", key)) diff --git a/internal/metastore/kv/datacoord/kv_catalog.go b/internal/metastore/kv/datacoord/kv_catalog.go index 989fb3e2e0d04..293f479ba3d21 100644 --- a/internal/metastore/kv/datacoord/kv_catalog.go +++ b/internal/metastore/kv/datacoord/kv_catalog.go @@ -24,6 +24,7 @@ import ( "strings" "github.com/cockroachdb/errors" + "go.opentelemetry.io/otel" "go.uber.org/zap" "golang.org/x/exp/maps" "golang.org/x/sync/errgroup" @@ -431,6 +432,8 @@ func (kc *Catalog) DropSegment(ctx context.Context, segment *datapb.SegmentInfo) } func (kc *Catalog) MarkChannelAdded(ctx context.Context, channel string) error { + ctx, sp := otel.Tracer("Catalog").Start(ctx, "MarkChannelAdded") + defer sp.End() key := buildChannelRemovePath(channel) err := kc.MetaKv.Save(ctx, key, NonRemoveFlagTomestone) if err != nil { diff --git a/internal/metastore/kv/querycoord/kv_catalog.go b/internal/metastore/kv/querycoord/kv_catalog.go index 3531d8e1e43dc..badd208641f45 100644 --- a/internal/metastore/kv/querycoord/kv_catalog.go +++ b/internal/metastore/kv/querycoord/kv_catalog.go @@ -10,6 +10,7 @@ import ( "github.com/klauspost/compress/zstd" "github.com/pingcap/log" "github.com/samber/lo" + "go.opentelemetry.io/otel" "go.uber.org/zap" "google.golang.org/protobuf/proto" @@ -213,6 +214,8 @@ func (s Catalog) GetResourceGroups(ctx context.Context) ([]*querypb.ResourceGrou } func (s Catalog) ReleaseCollection(ctx context.Context, collection int64) error { + ctx, span := otel.Tracer("Catalog").Start(ctx, "ReleaseCollection") + defer span.End() // remove collection and obtained partitions collectionKey := EncodeCollectionLoadInfoKey(collection) err := s.cli.Remove(ctx, collectionKey) @@ -246,6 +249,8 @@ func (s Catalog) ReleasePartition(ctx context.Context, collection int64, partiti } func (s Catalog) ReleaseReplicas(ctx context.Context, collectionID int64) error { + ctx, span := otel.Tracer("Catalog").Start(ctx, "ReleaseReplicas") + defer span.End() key := encodeCollectionReplicaKey(collectionID) return s.cli.RemoveWithPrefix(ctx, key) } diff --git a/internal/proxy/channels_mgr.go b/internal/proxy/channels_mgr.go index 375649e5b6ea9..12a1375981aaa 100644 --- a/internal/proxy/channels_mgr.go +++ b/internal/proxy/channels_mgr.go @@ -22,6 +22,8 @@ import ( "strconv" "sync" + "github.com/milvus-io/milvus/pkg/util/typeutil" + "go.opentelemetry.io/otel" "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" @@ -173,6 +175,8 @@ func (mgr *singleTypeChannelsMgr) streamExistPrivate(collectionID UniqueID) bool } func createStream(ctx context.Context, factory msgstream.Factory, pchans []pChan, repack repackFuncType) (msgstream.MsgStream, error) { + ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "createStream") + defer sp.End() var stream msgstream.MsgStream var err error @@ -254,6 +258,8 @@ func (mgr *singleTypeChannelsMgr) lockGetStream(collectionID UniqueID) (msgstrea // getOrCreateStream get message stream of specified collection. // If stream doesn't exist, call createMsgStream to create for it. func (mgr *singleTypeChannelsMgr) getOrCreateStream(ctx context.Context, collectionID UniqueID) (msgstream.MsgStream, error) { + ctx, sp := otel.Tracer(typeutil.ProxyRole).Start(ctx, "getOrCreateStream") + defer sp.End() if stream, err := mgr.lockGetStream(collectionID); err == nil { return stream, nil } diff --git a/internal/proxy/impl.go b/internal/proxy/impl.go index b51180301bc59..ba5d90da70540 100644 --- a/internal/proxy/impl.go +++ b/internal/proxy/impl.go @@ -846,7 +846,7 @@ func (node *Proxy) LoadCollection(ctx context.Context, request *milvuspb.LoadCol zap.Bool("refreshMode", request.Refresh), ) - log.Debug("LoadCollection received") + log.Info("LoadCollection received") if err := node.sched.ddQueue.Enqueue(lct); err != nil { log.Warn("LoadCollection failed to enqueue", @@ -857,7 +857,7 @@ func (node *Proxy) LoadCollection(ctx context.Context, request *milvuspb.LoadCol return merr.Status(err), nil } - log.Debug( + log.Info( "LoadCollection enqueued", zap.Uint64("BeginTS", lct.BeginTs()), zap.Uint64("EndTS", lct.EndTs()), @@ -873,7 +873,7 @@ func (node *Proxy) LoadCollection(ctx context.Context, request *milvuspb.LoadCol return merr.Status(err), nil } - log.Debug( + log.Info( "LoadCollection done", zap.Uint64("BeginTS", lct.BeginTs()), zap.Uint64("EndTS", lct.EndTs()), @@ -3145,7 +3145,7 @@ func (node *Proxy) search(ctx context.Context, request *milvuspb.SearchRequest, } }() - log.Debug(rpcReceived(method)) + log.Info(rpcReceived(method)) if err := node.sched.dqQueue.Enqueue(qt); err != nil { log.Warn( @@ -3167,7 +3167,7 @@ func (node *Proxy) search(ctx context.Context, request *milvuspb.SearchRequest, } tr.CtxRecord(ctx, "search request enqueue") - log.Debug( + log.Info( rpcEnqueued(method), zap.Uint64("timestamp", qt.Base.Timestamp), ) diff --git a/internal/proxy/task_insert.go b/internal/proxy/task_insert.go index 444d317c7999c..6dac78389d2d3 100644 --- a/internal/proxy/task_insert.go +++ b/internal/proxy/task_insert.go @@ -264,11 +264,14 @@ func (it *insertTask) Execute(ctx context.Context) error { zap.Duration("get cache duration", getCacheDur), zap.Duration("get msgStream duration", getMsgStreamDur)) + _, sp1 := otel.Tracer(typeutil.ProxyRole).Start(ctx, "repack Data") // assign segmentID for insert data and repack data by segmentID var msgPack *msgstream.MsgPack if it.partitionKeys == nil { + sp1.AddEvent("repackInsertData") msgPack, err = repackInsertData(it.TraceCtx(), channelNames, it.insertMsg, it.result, it.idAllocator, it.segIDAssigner) } else { + sp1.AddEvent("repackInsertDataWithPartitionKey") msgPack, err = repackInsertDataWithPartitionKey(it.TraceCtx(), channelNames, it.partitionKeys, it.insertMsg, it.result, it.idAllocator, it.segIDAssigner) } if err != nil { @@ -276,6 +279,7 @@ func (it *insertTask) Execute(ctx context.Context) error { it.result.Status = merr.Status(err) return err } + sp1.End() assignSegmentIDDur := tr.RecordSpan() log.Debug("assign segmentID for insert data success", diff --git a/internal/proxy/task_search.go b/internal/proxy/task_search.go index 50a366413ff5d..53a0e96f35044 100644 --- a/internal/proxy/task_search.go +++ b/internal/proxy/task_search.go @@ -700,7 +700,7 @@ func (t *searchTask) PostExecute(ctx context.Context) error { if len(internalResults) >= 1 { metricType = internalResults[0].GetMetricType() } - result, err := t.reduceResults(t.ctx, internalResults, subReq.GetNq(), subReq.GetTopk(), subReq.GetOffset(), t.queryInfos[index], true) + result, err := t.reduceResults(ctx, internalResults, subReq.GetNq(), subReq.GetTopk(), subReq.GetOffset(), t.queryInfos[index], true) if err != nil { return err } @@ -720,7 +720,7 @@ func (t *searchTask) PostExecute(ctx context.Context) error { return err } } else { - t.result, err = t.reduceResults(t.ctx, toReduceResults, t.SearchRequest.GetNq(), t.SearchRequest.GetTopk(), t.SearchRequest.GetOffset(), t.queryInfos[0], false) + t.result, err = t.reduceResults(ctx, toReduceResults, t.SearchRequest.GetNq(), t.SearchRequest.GetTopk(), t.SearchRequest.GetOffset(), t.queryInfos[0], false) if err != nil { return err } @@ -757,7 +757,7 @@ func (t *searchTask) PostExecute(ctx context.Context) error { metrics.ProxyReduceResultLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), metrics.SearchLabel).Observe(float64(tr.RecordSpan().Milliseconds())) - log.Debug("Search post execute done", + log.Info("Search post execute done", zap.Int64("collection", t.GetCollectionID()), zap.Int64s("partitionIDs", t.GetPartitionIDs())) return nil diff --git a/internal/querycoordv2/job/job_release.go b/internal/querycoordv2/job/job_release.go index 61c95284835fd..ca50558b0f828 100644 --- a/internal/querycoordv2/job/job_release.go +++ b/internal/querycoordv2/job/job_release.go @@ -18,6 +18,7 @@ package job import ( "context" + "go.opentelemetry.io/otel" "github.com/cockroachdb/errors" "github.com/samber/lo" @@ -33,6 +34,7 @@ import ( "github.com/milvus-io/milvus/internal/util/proxyutil" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) type ReleaseCollectionJob struct { @@ -72,22 +74,24 @@ func NewReleaseCollectionJob(ctx context.Context, } func (job *ReleaseCollectionJob) Execute() error { + ctx, span := otel.Tracer(typeutil.QueryCoordRole).Start(job.ctx, "ReleaseCollectionJob-Execute") + defer span.End() req := job.req - log := log.Ctx(job.ctx).With(zap.Int64("collectionID", req.GetCollectionID())) + log := log.Ctx(ctx).With(zap.Int64("collectionID", req.GetCollectionID())) - if !job.meta.CollectionManager.Exist(job.ctx, req.GetCollectionID()) { + if !job.meta.CollectionManager.Exist(ctx, req.GetCollectionID()) { log.Info("release collection end, the collection has not been loaded into QueryNode") return nil } - err := job.meta.CollectionManager.RemoveCollection(job.ctx, req.GetCollectionID()) + err := job.meta.CollectionManager.RemoveCollection(ctx, req.GetCollectionID()) if err != nil { msg := "failed to remove collection" log.Warn(msg, zap.Error(err)) return errors.Wrap(err, msg) } - err = job.meta.ReplicaManager.RemoveCollection(job.ctx, req.GetCollectionID()) + err = job.meta.ReplicaManager.RemoveCollection(ctx, req.GetCollectionID()) if err != nil { msg := "failed to remove replicas" log.Warn(msg, zap.Error(err)) @@ -97,7 +101,7 @@ func (job *ReleaseCollectionJob) Execute() error { // try best discard cache // shall not affect releasing if failed - job.proxyManager.InvalidateCollectionMetaCache(job.ctx, + job.proxyManager.InvalidateCollectionMetaCache(ctx, &proxypb.InvalidateCollMetaCacheRequest{ CollectionID: req.GetCollectionID(), }, diff --git a/internal/querycoordv2/services.go b/internal/querycoordv2/services.go index 456711e326315..a513bfc8b72ab 100644 --- a/internal/querycoordv2/services.go +++ b/internal/querycoordv2/services.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/errors" "github.com/samber/lo" + "go.opentelemetry.io/otel" "go.uber.org/zap" "golang.org/x/sync/errgroup" "google.golang.org/protobuf/proto" @@ -308,6 +309,8 @@ func (s *Server) LoadCollection(ctx context.Context, req *querypb.LoadCollection } func (s *Server) ReleaseCollection(ctx context.Context, req *querypb.ReleaseCollectionRequest) (*commonpb.Status, error) { + ctx, span := otel.Tracer(typeutil.QueryCoordRole).Start(ctx, "ReleaseCollection") + defer span.End() log := log.Ctx(ctx).With( zap.Int64("collectionID", req.GetCollectionID()), ) @@ -333,9 +336,11 @@ func (s *Server) ReleaseCollection(ctx context.Context, req *querypb.ReleaseColl s.proxyClientManager, ) s.jobScheduler.Add(releaseJob) + span.AddEvent("enqueue release coll job") err := releaseJob.Wait() if err != nil { msg := "failed to release collection" + span.AddEvent(msg) log.Warn(msg, zap.Error(err)) metrics.QueryCoordReleaseCount.WithLabelValues(metrics.FailLabel).Inc() return merr.Status(errors.Wrap(err, msg)), nil diff --git a/internal/querynodev2/delegator/delegator_data.go b/internal/querynodev2/delegator/delegator_data.go index e0c3040fee2da..e4ea4935f7dda 100644 --- a/internal/querynodev2/delegator/delegator_data.go +++ b/internal/querynodev2/delegator/delegator_data.go @@ -722,7 +722,7 @@ func (sd *shardDelegator) createStreamFromMsgStream(ctx context.Context, positio // Random the subname in case we trying to load same delta at the same time subName := fmt.Sprintf("querynode-delta-loader-%d-%d-%d", paramtable.GetNodeID(), sd.collectionID, rand.Int()) - log.Info("from dml check point load delete", zap.Any("position", position), zap.String("vChannel", vchannelName), zap.String("subName", subName), zap.Time("positionTs", ts)) + log.Ctx(ctx).Info("from dml check point load delete", zap.Any("position", position), zap.String("vChannel", vchannelName), zap.String("subName", subName), zap.Time("positionTs", ts)) err = stream.AsConsumer(context.TODO(), []string{pChannelName}, subName, mqcommon.SubscriptionPositionUnknown) if err != nil { return nil, stream.Close, err diff --git a/internal/querynodev2/services.go b/internal/querynodev2/services.go index ce9f7779a3753..989aaa2e52b60 100644 --- a/internal/querynodev2/services.go +++ b/internal/querynodev2/services.go @@ -744,7 +744,7 @@ func (node *QueryNode) Search(ctx context.Context, req *querypb.SearchRequest) ( zap.Int64("nq", req.GetReq().GetNq()), ) - log.Debug("Received SearchRequest", + log.Info("Received SearchRequest", zap.Int64s("segmentIDs", req.GetSegmentIDs()), zap.Uint64("guaranteeTimestamp", req.GetReq().GetGuaranteeTimestamp()), zap.Uint64("mvccTimestamp", req.GetReq().GetMvccTimestamp())) @@ -801,6 +801,8 @@ func (node *QueryNode) Search(ctx context.Context, req *querypb.SearchRequest) ( if ret.GetCostAggregation() != nil { ret.GetCostAggregation().ResponseTime = tr.ElapseSpan().Milliseconds() } + + log.Info("Search Done", zap.Int64("ResponseTime", tr.ElapseSpan().Milliseconds())) return ret, nil } diff --git a/internal/rootcoord/create_collection_task.go b/internal/rootcoord/create_collection_task.go index 60a46ec9baac4..f4be75e5e0cb9 100644 --- a/internal/rootcoord/create_collection_task.go +++ b/internal/rootcoord/create_collection_task.go @@ -23,6 +23,7 @@ import ( "strconv" "github.com/cockroachdb/errors" + "go.opentelemetry.io/otel" "go.uber.org/zap" "google.golang.org/protobuf/proto" @@ -482,12 +483,14 @@ func (t *createCollectionTask) genCreateCollectionRequest() *msgpb.CreateCollect } func (t *createCollectionTask) addChannelsAndGetStartPositions(ctx context.Context, ts uint64) (map[string][]byte, error) { + ctx, sp := otel.Tracer("createCollectionTask").Start(ctx, "addChannelsAndGetStartPositions") + defer sp.End() t.core.chanTimeTick.addDmlChannels(t.channels.physicalChannels...) if streamingutil.IsStreamingServiceEnabled() { return t.broadcastCreateCollectionMsgIntoStreamingService(ctx, ts) } msg := t.genCreateCollectionMsg(ctx, ts) - return t.core.chanTimeTick.broadcastMarkDmlChannels(t.channels.physicalChannels, msg) + return t.core.chanTimeTick.broadcastMarkDmlChannels(ctx, t.channels.physicalChannels, msg) } func (t *createCollectionTask) broadcastCreateCollectionMsgIntoStreamingService(ctx context.Context, ts uint64) (map[string][]byte, error) { diff --git a/internal/rootcoord/dml_channels.go b/internal/rootcoord/dml_channels.go index 8c40f99807c06..0b8920d99dc40 100644 --- a/internal/rootcoord/dml_channels.go +++ b/internal/rootcoord/dml_channels.go @@ -25,6 +25,7 @@ import ( "sync" "github.com/cockroachdb/errors" + "go.opentelemetry.io/otel" "go.uber.org/zap" "github.com/milvus-io/milvus/pkg/log" @@ -302,7 +303,9 @@ func (d *dmlChannels) broadcast(chanNames []string, pack *msgstream.MsgPack) err return nil } -func (d *dmlChannels) broadcastMark(chanNames []string, pack *msgstream.MsgPack) (map[string][]byte, error) { +func (d *dmlChannels) broadcastMark(ctx context.Context, chanNames []string, pack *msgstream.MsgPack) (map[string][]byte, error) { + ctx, sp := otel.Tracer("dmlChannels").Start(ctx, "addChannelsAndGetStartPositions") + defer sp.End() result := make(map[string][]byte) for _, chanName := range chanNames { dms, err := d.getMsgStreamByName(chanName) @@ -312,9 +315,9 @@ func (d *dmlChannels) broadcastMark(chanNames []string, pack *msgstream.MsgPack) dms.mutex.RLock() if dms.refcnt > 0 { - ids, err := dms.ms.Broadcast(d.ctx, pack) + ids, err := dms.ms.Broadcast(ctx, pack) if err != nil { - log.Error("BroadcastMark failed", zap.Error(err), zap.String("chanName", chanName)) + log.Ctx(ctx).Error("BroadcastMark failed", zap.Error(err), zap.String("chanName", chanName)) dms.mutex.RUnlock() return result, err } diff --git a/internal/rootcoord/dml_channels_test.go b/internal/rootcoord/dml_channels_test.go index 7d7f3284835f2..f011618fe4876 100644 --- a/internal/rootcoord/dml_channels_test.go +++ b/internal/rootcoord/dml_channels_test.go @@ -138,7 +138,7 @@ func TestDmlChannels(t *testing.T) { dml.addChannels(randStr) assert.Error(t, dml.broadcast([]string{randStr}, nil)) { - _, err := dml.broadcastMark([]string{randStr}, nil) + _, err := dml.broadcastMark(context.TODO(), []string{randStr}, nil) assert.Error(t, err) } dml.removeChannels(randStr) @@ -189,7 +189,7 @@ func TestDmChannelsFailure(t *testing.T) { err := dml.broadcast([]string{chanName0}, nil) assert.Error(t, err) - v, err := dml.broadcastMark([]string{chanName0}, nil) + v, err := dml.broadcastMark(context.TODO(), []string{chanName0}, nil) assert.Empty(t, v) assert.Error(t, err) }) diff --git a/internal/rootcoord/drop_collection_task.go b/internal/rootcoord/drop_collection_task.go index 06affe5e5c050..65612bbb8d581 100644 --- a/internal/rootcoord/drop_collection_task.go +++ b/internal/rootcoord/drop_collection_task.go @@ -21,6 +21,7 @@ import ( "fmt" "github.com/cockroachdb/errors" + "go.opentelemetry.io/otel" "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" @@ -52,6 +53,8 @@ func (t *dropCollectionTask) Prepare(ctx context.Context) error { } func (t *dropCollectionTask) Execute(ctx context.Context) error { + ctx, sp := otel.Tracer(typeutil.RootCoordRole).Start(ctx, "dropCollectionTask.Execute") + defer sp.End() // use max ts to check if latest collection exists. // we cannot handle case that // dropping collection with `ts1` but a collection exists in catalog with newer ts which is bigger than `ts1`. diff --git a/internal/rootcoord/meta_table.go b/internal/rootcoord/meta_table.go index 14d6e5a135262..ead85f320a5ae 100644 --- a/internal/rootcoord/meta_table.go +++ b/internal/rootcoord/meta_table.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/errors" "github.com/samber/lo" + "go.opentelemetry.io/otel" "go.uber.org/zap" "golang.org/x/exp/maps" @@ -402,6 +403,9 @@ func (mt *MetaTable) getDatabaseByNameInternal(_ context.Context, dbName string, } func (mt *MetaTable) AddCollection(ctx context.Context, coll *model.Collection) error { + ctx, sp := otel.Tracer("MetaTable").Start(ctx, "AddCollection") + defer sp.End() + sp.AddEvent(fmt.Sprintf("dbID=%d, collectionName=%s", coll.DBID, coll.Name)) mt.ddLock.Lock() defer mt.ddLock.Unlock() @@ -437,6 +441,9 @@ func (mt *MetaTable) AddCollection(ctx context.Context, coll *model.Collection) } func (mt *MetaTable) ChangeCollectionState(ctx context.Context, collectionID UniqueID, state pb.CollectionState, ts Timestamp) error { + ctx, sp := otel.Tracer("MetaTable").Start(ctx, "ChangeCollectionState") + defer sp.End() + sp.AddEvent(fmt.Sprintf("state=%s", state.String())) mt.ddLock.Lock() defer mt.ddLock.Unlock() diff --git a/internal/rootcoord/redo.go b/internal/rootcoord/redo.go index 72406ee29604d..4de090db65d39 100644 --- a/internal/rootcoord/redo.go +++ b/internal/rootcoord/redo.go @@ -46,14 +46,17 @@ func (b *baseRedoTask) AddAsyncStep(step nestedStep) { b.asyncTodoStep = append(b.asyncTodoStep, step) } -func (b *baseRedoTask) redoAsyncSteps() { +func (b *baseRedoTask) redoAsyncSteps(ctx context.Context) { l := len(b.asyncTodoStep) steps := make([]nestedStep, 0, l) for i := l - 1; i >= 0; i-- { steps = append(steps, b.asyncTodoStep[i]) } b.asyncTodoStep = nil // make baseRedoTask can be collected. - b.stepExecutor.AddSteps(&stepStack{steps: steps}) + b.stepExecutor.AddSteps(&stepStack{ + steps: steps, + stepsCtx: context.WithoutCancel(ctx), + }) } func (b *baseRedoTask) Execute(ctx context.Context) error { @@ -61,10 +64,10 @@ func (b *baseRedoTask) Execute(ctx context.Context) error { todo := b.syncTodoStep[i] // no children step in sync steps. if _, err := todo.Execute(ctx); err != nil { - log.Error("failed to execute step", zap.Error(err), zap.String("desc", todo.Desc())) + log.Ctx(ctx).Error("failed to execute step", zap.Error(err), zap.String("desc", todo.Desc())) return err } } - go b.redoAsyncSteps() + go b.redoAsyncSteps(ctx) return nil } diff --git a/internal/rootcoord/redo_test.go b/internal/rootcoord/redo_test.go index a01e897fcd598..67b00b11d15c4 100644 --- a/internal/rootcoord/redo_test.go +++ b/internal/rootcoord/redo_test.go @@ -77,7 +77,7 @@ func Test_baseRedoTask_redoAsyncSteps(t *testing.T) { for _, step := range steps { redo.AddAsyncStep(step) } - redo.redoAsyncSteps() + redo.redoAsyncSteps(context.TODO()) assert.True(t, steps[0].(*mockNormalStep).called) assert.False(t, steps[2].(*mockNormalStep).called) }) @@ -92,7 +92,7 @@ func Test_baseRedoTask_redoAsyncSteps(t *testing.T) { for _, step := range steps { redo.AddAsyncStep(step) } - redo.redoAsyncSteps() + redo.redoAsyncSteps(context.TODO()) for _, step := range steps { assert.True(t, step.(*mockNormalStep).called) } diff --git a/internal/rootcoord/show_collection_task.go b/internal/rootcoord/show_collection_task.go index f5a3fa5eeb725..a63558ab2db87 100644 --- a/internal/rootcoord/show_collection_task.go +++ b/internal/rootcoord/show_collection_task.go @@ -18,8 +18,10 @@ package rootcoord import ( "context" + "fmt" "github.com/samber/lo" + "go.opentelemetry.io/otel" "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" @@ -48,6 +50,13 @@ func (t *showCollectionTask) Prepare(ctx context.Context) error { // Execute task execution func (t *showCollectionTask) Execute(ctx context.Context) error { + taskName := "ShowCollectionTask" + ctx, span := otel.Tracer(typeutil.RootCoordRole).Start(ctx, taskName) + defer span.End() + span.AddEvent(fmt.Sprintf("taskID=%d", t.GetID())) + log := log.Ctx(ctx).With(zap.String("taskName", taskName), zap.Int64("taskID", t.GetID())) + log.Info("showCollectionTask Execute Start") + t.Rsp.Status = merr.Success() getVisibleCollections := func() (typeutil.Set[string], error) { @@ -115,6 +124,7 @@ func (t *showCollectionTask) Execute(ctx context.Context) error { visibleCollections, err := getVisibleCollections() if err != nil { + log.Warn("get visible collections failed", zap.Error(err)) t.Rsp.Status = merr.Status(err) return err } @@ -129,6 +139,7 @@ func (t *showCollectionTask) Execute(ctx context.Context) error { colls, err := t.core.meta.ListCollections(ctx, t.Req.GetDbName(), ts, true) if err != nil { t.Rsp.Status = merr.Status(err) + log.Warn("list collections from meta failed", zap.Error(err)) return err } for _, coll := range colls { @@ -145,6 +156,7 @@ func (t *showCollectionTask) Execute(ctx context.Context) error { physical, _ := tsoutil.ParseHybridTs(coll.CreateTime) t.Rsp.CreatedUtcTimestamps = append(t.Rsp.CreatedUtcTimestamps, uint64(physical)) } + log.Info("showCollectionTask Execute Finish") return nil } diff --git a/internal/rootcoord/step_executor.go b/internal/rootcoord/step_executor.go index 5e63faadd86b5..959a6dfe1845d 100644 --- a/internal/rootcoord/step_executor.go +++ b/internal/rootcoord/step_executor.go @@ -40,7 +40,8 @@ type StepExecutor interface { } type stepStack struct { - steps []nestedStep + steps []nestedStep + stepsCtx context.Context } func (s *stepStack) totalPriority() int { @@ -74,7 +75,7 @@ func (s *stepStack) Execute(ctx context.Context) *stepStack { if !skipLog { log.Warn("failed to execute step, wait for reschedule", zap.Error(err), zap.String("step", todo.Desc())) } - return &stepStack{steps: steps} + return &stepStack{steps: steps, stepsCtx: ctx} } // this step is done. steps = steps[:l-1] @@ -198,7 +199,7 @@ func (bg *bgStepExecutor) process(steps []*stepStack) { wg.Add(1) go func() { defer wg.Done() - child := s.Execute(bg.ctx) + child := s.Execute(s.stepsCtx) if child != nil { // don't notify, wait for reschedule. bg.addStepsInternal(child) diff --git a/internal/rootcoord/step_executor_test.go b/internal/rootcoord/step_executor_test.go index 29d3d4784853a..44bf63778ffcc 100644 --- a/internal/rootcoord/step_executor_test.go +++ b/internal/rootcoord/step_executor_test.go @@ -70,7 +70,7 @@ func Test_stepStack_Execute(t *testing.T) { newMockStepWithChild(), newMockChildStep(), } - s := &stepStack{steps: steps} + s := &stepStack{steps: steps, stepsCtx: context.TODO()} unfinished := s.Execute(context.Background()) assert.Nil(t, unfinished) }) @@ -82,7 +82,7 @@ func Test_stepStack_Execute(t *testing.T) { newMockFailStep(), newMockNormalStep(), } - s := &stepStack{steps: steps} + s := &stepStack{steps: steps, stepsCtx: context.TODO()} unfinished := s.Execute(context.Background()) assert.Equal(t, 3, len(unfinished.steps)) }) @@ -93,32 +93,32 @@ func Test_stepStack_Execute(t *testing.T) { steps := []nestedStep{ failStep, } - s := &stepStack{steps: steps} + s := &stepStack{steps: steps, stepsCtx: context.TODO()} unfinished := s.Execute(context.Background()) assert.Nil(t, unfinished) }) } func Test_randomSelect(t *testing.T) { - s0 := &stepStack{steps: []nestedStep{}} + s0 := &stepStack{steps: []nestedStep{}, stepsCtx: context.TODO()} s1 := &stepStack{steps: []nestedStep{ newMockNormalStep(), - }} + }, stepsCtx: context.TODO()} s2 := &stepStack{steps: []nestedStep{ newMockNormalStep(), newMockNormalStep(), - }} + }, stepsCtx: context.TODO()} s3 := &stepStack{steps: []nestedStep{ newMockNormalStep(), newMockNormalStep(), newMockNormalStep(), - }} + }, stepsCtx: context.TODO()} s4 := &stepStack{steps: []nestedStep{ newMockNormalStep(), newMockNormalStep(), newMockNormalStep(), newMockNormalStep(), - }} + }, stepsCtx: context.TODO()} m := map[*stepStack]struct{}{ s0: {}, s1: {}, @@ -162,13 +162,13 @@ func Test_bgStepExecutor_scheduleLoop(t *testing.T) { newMockNormalStep(), failStep, newMockNormalStep(), - }} + }, stepsCtx: context.TODO()} case 2: s = &stepStack{steps: []nestedStep{ newMockNormalStep(), newMockNormalStep(), newMockNormalStep(), - }} + }, stepsCtx: context.TODO()} default: } steps = append(steps, s) @@ -208,11 +208,11 @@ func Test_selectByPriorityPolicy(t *testing.T) { &releaseCollectionStep{}, &releaseCollectionStep{}, } - s1 := &stepStack{steps: steps[0:1]} - s2 := &stepStack{steps: steps[0:2]} - s3 := &stepStack{steps: steps[0:3]} - s4 := &stepStack{steps: steps[0:4]} - s5 := &stepStack{steps: steps[0:5]} + s1 := &stepStack{steps: steps[0:1], stepsCtx: context.TODO()} + s2 := &stepStack{steps: steps[0:2], stepsCtx: context.TODO()} + s3 := &stepStack{steps: steps[0:3], stepsCtx: context.TODO()} + s4 := &stepStack{steps: steps[0:4], stepsCtx: context.TODO()} + s5 := &stepStack{steps: steps[0:5], stepsCtx: context.TODO()} m := map[*stepStack]struct{}{ s1: {}, s2: {}, diff --git a/internal/rootcoord/task.go b/internal/rootcoord/task.go index 4b0927b3bf77f..42caa9ac31be6 100644 --- a/internal/rootcoord/task.go +++ b/internal/rootcoord/task.go @@ -72,8 +72,8 @@ func newBaseTask(ctx context.Context, core *Core) baseTask { core: core, done: make(chan error, 1), tr: timerecord.NewTimeRecorderWithTrace(ctx, "new task"), + ctx: ctx, } - b.SetCtx(ctx) return b } diff --git a/internal/rootcoord/timeticksync.go b/internal/rootcoord/timeticksync.go index 7dc0f81fc77e3..a77348ee511a2 100644 --- a/internal/rootcoord/timeticksync.go +++ b/internal/rootcoord/timeticksync.go @@ -396,8 +396,8 @@ func (t *timetickSync) broadcastDmlChannels(chanNames []string, pack *msgstream. } // BroadcastMarkDmlChannels broadcasts msg pack into dml channels -func (t *timetickSync) broadcastMarkDmlChannels(chanNames []string, pack *msgstream.MsgPack) (map[string][]byte, error) { - return t.dmlChannels.broadcastMark(chanNames, pack) +func (t *timetickSync) broadcastMarkDmlChannels(ctx context.Context, chanNames []string, pack *msgstream.MsgPack) (map[string][]byte, error) { + return t.dmlChannels.broadcastMark(ctx, chanNames, pack) } func (t *timetickSync) getSyncedTimeTick(channel string) Timestamp { diff --git a/internal/rootcoord/undo.go b/internal/rootcoord/undo.go index 29e9b7dfac05b..6cfd9e19a98ac 100644 --- a/internal/rootcoord/undo.go +++ b/internal/rootcoord/undo.go @@ -55,7 +55,7 @@ func (b *baseUndoTask) Execute(ctx context.Context) error { log.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}) + go b.stepExecutor.AddSteps(&stepStack{steps: undoSteps, stepsCtx: context.WithoutCancel(ctx)}) return err } } diff --git a/internal/util/proxyutil/proxy_client_manager.go b/internal/util/proxyutil/proxy_client_manager.go index 76018dda5b2d4..15a022723f79b 100644 --- a/internal/util/proxyutil/proxy_client_manager.go +++ b/internal/util/proxyutil/proxy_client_manager.go @@ -175,7 +175,7 @@ func (p *ProxyClientManager) InvalidateCollectionMetaCache(ctx context.Context, c.Apply(request) if p.proxyClient.Len() == 0 { - log.Warn("proxy client is empty, InvalidateCollectionMetaCache will not send to any client") + log.Ctx(ctx).Warn("proxy client is empty, InvalidateCollectionMetaCache will not send to any client") return nil } @@ -186,7 +186,7 @@ func (p *ProxyClientManager) InvalidateCollectionMetaCache(ctx context.Context, sta, err := v.InvalidateCollectionMetaCache(ctx, request) if err != nil { if errors.Is(err, merr.ErrNodeNotFound) { - log.Warn("InvalidateCollectionMetaCache failed due to proxy service not found", zap.Error(err)) + log.Ctx(ctx).Warn("InvalidateCollectionMetaCache failed due to proxy service not found", zap.Error(err)) return nil } @@ -209,7 +209,7 @@ func (p *ProxyClientManager) InvalidateCollectionMetaCache(ctx context.Context, // InvalidateCredentialCache TODO: too many codes similar to InvalidateCollectionMetaCache. func (p *ProxyClientManager) InvalidateCredentialCache(ctx context.Context, request *proxypb.InvalidateCredCacheRequest) error { if p.proxyClient.Len() == 0 { - log.Warn("proxy client is empty, InvalidateCredentialCache will not send to any client") + log.Ctx(ctx).Warn("proxy client is empty, InvalidateCredentialCache will not send to any client") return nil } diff --git a/pkg/config/etcd_source.go b/pkg/config/etcd_source.go index 9885ae746a4be..289e327474dc7 100644 --- a/pkg/config/etcd_source.go +++ b/pkg/config/etcd_source.go @@ -154,6 +154,7 @@ func (es *EtcdSource) refreshConfigurations() error { log.RatedDebug(10, "etcd refreshConfigurations", zap.String("prefix", prefix), zap.Any("endpoints", es.etcdCli.Endpoints())) response, err := es.etcdCli.Get(ctx, prefix, clientv3.WithPrefix(), clientv3.WithSerializable()) if err != nil { + log.Error("etcd refreshConfigurations failed", zap.Error(err)) return err } newConfig := make(map[string]string, len(response.Kvs)) diff --git a/pkg/mq/msgdispatcher/client.go b/pkg/mq/msgdispatcher/client.go index 95e5ff1730811..7409ca5e24c36 100644 --- a/pkg/mq/msgdispatcher/client.go +++ b/pkg/mq/msgdispatcher/client.go @@ -63,15 +63,16 @@ func NewClient(factory msgstream.Factory, role string, nodeID int64) Client { } func (c *client) Register(ctx context.Context, vchannel string, pos *Pos, subPos SubPos) (<-chan *MsgPack, error) { - log := log.With(zap.String("role", c.role), + log := log.Ctx(ctx).With(zap.String("role", c.role), zap.Int64("nodeID", c.nodeID), zap.String("vchannel", vchannel)) + log.Info("register vchannel") pchannel := funcutil.ToPhysicalChannel(vchannel) c.managerMut.Lock(pchannel) defer c.managerMut.Unlock(pchannel) var manager DispatcherManager manager, ok := c.managers.Get(pchannel) if !ok { - manager = NewDispatcherManager(pchannel, c.role, c.nodeID, c.factory) + manager = NewDispatcherManager(ctx, pchannel, c.role, c.nodeID, c.factory) c.managers.Insert(pchannel, manager) go manager.Run() } diff --git a/pkg/mq/msgdispatcher/dispatcher.go b/pkg/mq/msgdispatcher/dispatcher.go index c41aa7be83311..6c75c8bd36938 100644 --- a/pkg/mq/msgdispatcher/dispatcher.go +++ b/pkg/mq/msgdispatcher/dispatcher.go @@ -88,7 +88,7 @@ func NewDispatcher(ctx context.Context, lagTargets *typeutil.ConcurrentMap[string, *target], includeCurrentMsg bool, ) (*Dispatcher, error) { - log := log.With(zap.String("pchannel", pchannel), + log := log.Ctx(ctx).With(zap.String("pchannel", pchannel), zap.String("subName", subName), zap.Bool("isMain", isMain)) log.Info("creating dispatcher...") diff --git a/pkg/mq/msgdispatcher/manager.go b/pkg/mq/msgdispatcher/manager.go index 6fd9a22f20354..402f3b8729ba5 100644 --- a/pkg/mq/msgdispatcher/manager.go +++ b/pkg/mq/msgdispatcher/manager.go @@ -62,8 +62,8 @@ type dispatcherManager struct { closeOnce sync.Once } -func NewDispatcherManager(pchannel string, role string, nodeID int64, factory msgstream.Factory) DispatcherManager { - log.Info("create new dispatcherManager", zap.String("role", role), +func NewDispatcherManager(ctx context.Context, pchannel string, role string, nodeID int64, factory msgstream.Factory) DispatcherManager { + log.Ctx(ctx).Info("create new dispatcherManager", zap.String("role", role), zap.Int64("nodeID", nodeID), zap.String("pchannel", pchannel)) c := &dispatcherManager{ role: role, @@ -83,8 +83,9 @@ func (c *dispatcherManager) constructSubName(vchannel string, isMain bool) strin } func (c *dispatcherManager) Add(ctx context.Context, vchannel string, pos *Pos, subPos SubPos) (<-chan *MsgPack, error) { - log := log.With(zap.String("role", c.role), + log := log.Ctx(ctx).With(zap.String("role", c.role), zap.Int64("nodeID", c.nodeID), zap.String("vchannel", vchannel)) + log.Info("add new vchannel dispatcher") c.mu.Lock() defer c.mu.Unlock() diff --git a/pkg/mq/msgdispatcher/manager_test.go b/pkg/mq/msgdispatcher/manager_test.go index feb55799651be..982c1707d6247 100644 --- a/pkg/mq/msgdispatcher/manager_test.go +++ b/pkg/mq/msgdispatcher/manager_test.go @@ -37,7 +37,7 @@ import ( func TestManager(t *testing.T) { t.Run("test add and remove dispatcher", func(t *testing.T) { - c := NewDispatcherManager("mock_pchannel_0", typeutil.ProxyRole, 1, newMockFactory()) + c := NewDispatcherManager(context.TODO(), "mock_pchannel_0", typeutil.ProxyRole, 1, newMockFactory()) assert.NotNil(t, c) assert.Equal(t, 0, c.Num()) @@ -65,7 +65,7 @@ func TestManager(t *testing.T) { t.Run("test merge and split", func(t *testing.T) { prefix := fmt.Sprintf("mock%d", time.Now().UnixNano()) ctx := context.Background() - c := NewDispatcherManager(prefix+"_pchannel_0", typeutil.ProxyRole, 1, newMockFactory()) + c := NewDispatcherManager(context.TODO(), prefix+"_pchannel_0", typeutil.ProxyRole, 1, newMockFactory()) assert.NotNil(t, c) _, err := c.Add(ctx, "mock_vchannel_0", nil, common.SubscriptionPositionUnknown) assert.NoError(t, err) @@ -96,7 +96,7 @@ func TestManager(t *testing.T) { t.Run("test run and close", func(t *testing.T) { prefix := fmt.Sprintf("mock%d", time.Now().UnixNano()) ctx := context.Background() - c := NewDispatcherManager(prefix+"_pchannel_0", typeutil.ProxyRole, 1, newMockFactory()) + c := NewDispatcherManager(context.TODO(), prefix+"_pchannel_0", typeutil.ProxyRole, 1, newMockFactory()) assert.NotNil(t, c) _, err := c.Add(ctx, "mock_vchannel_0", nil, common.SubscriptionPositionUnknown) assert.NoError(t, err) @@ -131,7 +131,7 @@ func TestManager(t *testing.T) { ctx, cancel := context.WithTimeout(ctx, time.Millisecond*2) defer cancel() time.Sleep(time.Millisecond * 2) - c := NewDispatcherManager(prefix+"_pchannel_0", typeutil.ProxyRole, 1, newMockFactory()) + c := NewDispatcherManager(context.TODO(), prefix+"_pchannel_0", typeutil.ProxyRole, 1, newMockFactory()) go c.Run() assert.NotNil(t, c) _, err := c.Add(ctx, "mock_vchannel_0", nil, common.SubscriptionPositionUnknown) @@ -149,7 +149,7 @@ func TestManager(t *testing.T) { t.Run("test_repeated_vchannel", func(t *testing.T) { prefix := fmt.Sprintf("mock%d", time.Now().UnixNano()) - c := NewDispatcherManager(prefix+"_pchannel_0", typeutil.ProxyRole, 1, newMockFactory()) + c := NewDispatcherManager(context.TODO(), prefix+"_pchannel_0", typeutil.ProxyRole, 1, newMockFactory()) go c.Run() assert.NotNil(t, c) ctx := context.Background() @@ -210,7 +210,7 @@ func (suite *SimulationSuite) SetupTest() { assert.NoError(suite.T(), err) suite.producer = producer - suite.manager = NewDispatcherManager(suite.pchannel, typeutil.DataNodeRole, 0, suite.factory) + suite.manager = NewDispatcherManager(context.TODO(), suite.pchannel, typeutil.DataNodeRole, 0, suite.factory) go suite.manager.Run() } diff --git a/pkg/mq/msgstream/mq_msgstream.go b/pkg/mq/msgstream/mq_msgstream.go index 808b4da318865..af4ef642a497c 100644 --- a/pkg/mq/msgstream/mq_msgstream.go +++ b/pkg/mq/msgstream/mq_msgstream.go @@ -19,6 +19,7 @@ package msgstream import ( "context" "fmt" + "go.opentelemetry.io/otel" "path/filepath" "strconv" "sync" @@ -123,9 +124,11 @@ func NewMqMsgStream(ctx context.Context, // AsProducer create producer to send message to channels func (ms *mqMsgStream) AsProducer(ctx context.Context, channels []string) { + ctx, sp := otel.Tracer("MqMsgStream").Start(ctx, "AsProducer") + defer sp.End() for _, channel := range channels { if len(channel) == 0 { - log.Error("MsgStream asProducer's channel is an empty string") + log.Ctx(ctx).Error("MsgStream asProducer's channel is an empty string") break } @@ -137,6 +140,8 @@ func (ms *mqMsgStream) AsProducer(ctx context.Context, channels []string) { if pp == nil { return errors.New("Producer is nil") } + sp.AddEvent(fmt.Sprintf("CreateProducer:%s", channel)) + log.Ctx(ctx).Debug("Producer created", zap.String("channel", channel)) ms.producerLock.Lock() defer ms.producerLock.Unlock() @@ -144,7 +149,8 @@ func (ms *mqMsgStream) AsProducer(ctx context.Context, channels []string) { ms.producerChannels = append(ms.producerChannels, channel) return nil } - err := retry.Do(context.TODO(), fn, retry.Attempts(20), retry.Sleep(time.Millisecond*200), retry.MaxSleepTime(5*time.Second)) + err := retry.Do(log.WithFields(ctx, zap.String("action", "CreateProducer"), zap.String("channel", channel)), + fn, retry.Attempts(20), retry.Sleep(time.Millisecond*200), retry.MaxSleepTime(5*time.Second)) if err != nil { errMsg := "Failed to create producer " + channel + ", error = " + err.Error() panic(errMsg) @@ -172,6 +178,8 @@ func (ms *mqMsgStream) CheckTopicValid(channel string) error { // AsConsumerWithPosition Create consumer to receive message from channels, with initial position // if initial position is set to latest, last message in the channel is exclusive func (ms *mqMsgStream) AsConsumer(ctx context.Context, channels []string, subName string, position common.SubscriptionInitialPosition) error { + ctx, sp := otel.Tracer("MqMsgStream").Start(ctx, "AsConsumer") + defer sp.End() for _, channel := range channels { if _, ok := ms.consumers[channel]; ok { continue @@ -189,6 +197,8 @@ func (ms *mqMsgStream) AsConsumer(ctx context.Context, channels []string, subNam if pc == nil { return errors.New("Consumer is nil") } + sp.AddEvent(fmt.Sprintf("Subscribe %s at Position:%d", channel, position)) + log.Ctx(ctx).Debug("Subscribed at channel", zap.String("channel", channel)) ms.consumerLock.Lock() defer ms.consumerLock.Unlock() @@ -197,7 +207,8 @@ func (ms *mqMsgStream) AsConsumer(ctx context.Context, channels []string, subNam return nil } - err := retry.Do(ctx, fn, retry.Attempts(20), retry.Sleep(time.Millisecond*200), retry.MaxSleepTime(5*time.Second)) + err := retry.Do(log.WithFields(ctx, zap.String("action", "Subscribe"), zap.String("channel", channel)), + fn, retry.Attempts(20), retry.Sleep(time.Millisecond*200), retry.MaxSleepTime(5*time.Second)) if err != nil { errMsg := fmt.Sprintf("Failed to create consumer %s", channel) if merr.IsCanceledOrTimeout(err) { @@ -206,7 +217,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(ctx).Info("Successfully create consumer", zap.String("channel", channel), zap.String("subname", subName)) } return nil } @@ -275,6 +286,8 @@ func (ms *mqMsgStream) isEnabledProduce() bool { } func (ms *mqMsgStream) Produce(ctx context.Context, msgPack *MsgPack) error { + ctx, sp := otel.Tracer("mqMsgStream").Start(ctx, "Produce") + defer sp.End() if !ms.isEnabledProduce() { log.Warn("can't produce the msg in the backup instance", zap.Stack("stack")) return merr.ErrDenyProduceMsg @@ -283,6 +296,7 @@ func (ms *mqMsgStream) Produce(ctx context.Context, msgPack *MsgPack) error { log.Debug("Warning: Receive empty msgPack") return nil } + sp.AddEvent(fmt.Sprintf("%d msgs", len(msgPack.Msgs))) if len(ms.producers) <= 0 { return errors.New("nil producer in msg stream") } @@ -320,8 +334,8 @@ func (ms *mqMsgStream) Produce(ctx context.Context, msgPack *MsgPack) error { } for i := 0; i < len(v.Msgs); i++ { - spanCtx, sp := MsgSpanFromCtx(v.Msgs[i].TraceCtx(), v.Msgs[i]) - defer sp.End() + spanCtx, _ := MsgSpanFromCtx(v.Msgs[i].TraceCtx(), v.Msgs[i]) + //defer sp.End() mb, err := v.Msgs[i].Marshal(v.Msgs[i]) if err != nil { @@ -338,7 +352,8 @@ func (ms *mqMsgStream) Produce(ctx context.Context, msgPack *MsgPack) error { }} InjectCtx(spanCtx, msg.Properties) - if _, err := producer.Send(spanCtx, msg); err != nil { + if _, err := producer.Send(ctx, msg); err != nil { + //if _, err := producer.Send(spanCtx, msg); err != nil { sp.RecordError(err) return err } @@ -352,10 +367,13 @@ func (ms *mqMsgStream) Produce(ctx context.Context, msgPack *MsgPack) error { // BroadcastMark broadcast msg pack to all producers and returns corresponding msg id // the returned message id serves as marking func (ms *mqMsgStream) Broadcast(ctx context.Context, msgPack *MsgPack) (map[string][]MessageID, error) { + ctx1, span := otel.Tracer("mqMsgStream").Start(ctx, "Broadcast") + defer span.End() ids := make(map[string][]MessageID) if msgPack == nil || len(msgPack.Msgs) <= 0 { return ids, errors.New("empty msgs") } + span.AddEvent(fmt.Sprintf("%d msgs", len(msgPack.Msgs))) // Only allow to create collection msg in backup instance // However, there may be a problem of ts disorder here, but because the start position of the collection only uses offsets, not time, there is no problem for the time being isCreateCollectionMsg := len(msgPack.Msgs) == 1 && msgPack.Msgs[0].Type() == commonpb.MsgType_CreateCollection @@ -365,7 +383,7 @@ func (ms *mqMsgStream) Broadcast(ctx context.Context, msgPack *MsgPack) (map[str return ids, merr.ErrDenyProduceMsg } for _, v := range msgPack.Msgs { - spanCtx, sp := MsgSpanFromCtx(v.TraceCtx(), v) + spanCtx, _ := MsgSpanFromCtx(v.TraceCtx(), v) mb, err := v.Marshal(v) if err != nil { @@ -386,15 +404,15 @@ func (ms *mqMsgStream) Broadcast(ctx context.Context, msgPack *MsgPack) (map[str ms.producerLock.RUnlock() for channel, producer := range producers { - id, err := producer.Send(spanCtx, msg) + id, err := producer.Send(ctx1, msg) if err != nil { - sp.RecordError(err) - sp.End() + //sp.RecordError(err) + //sp.End() return ids, err } ids[channel] = append(ids[channel], id) } - sp.End() + //sp.End() } return ids, nil } @@ -583,6 +601,8 @@ func (ms *MqTtMsgStream) addConsumer(consumer mqwrapper.Consumer, channel string // AsConsumerWithPosition subscribes channels as consumer for a MsgStream and seeks to a certain position. func (ms *MqTtMsgStream) AsConsumer(ctx context.Context, channels []string, subName string, position common.SubscriptionInitialPosition) error { + ctx, sp := otel.Tracer("MqTtMsgStream").Start(ctx, "AsConsumer") + defer sp.End() for _, channel := range channels { if _, ok := ms.consumers[channel]; ok { continue @@ -600,6 +620,8 @@ func (ms *MqTtMsgStream) AsConsumer(ctx context.Context, channels []string, subN if pc == nil { return errors.New("Consumer is nil") } + sp.AddEvent(fmt.Sprintf("Subscribe %s at Position:%d", channel, position)) + log.Ctx(ctx).Debug("Subscribed at channel", zap.String("channel", channel)) ms.consumerLock.Lock() defer ms.consumerLock.Unlock() @@ -607,7 +629,8 @@ func (ms *MqTtMsgStream) AsConsumer(ctx context.Context, channels []string, subN return nil } - err := retry.Do(ctx, fn, retry.Attempts(20), retry.Sleep(time.Millisecond*200), retry.MaxSleepTime(5*time.Second)) + err := retry.Do(log.WithFields(ctx, zap.String("action", "Subscribe"), zap.String("channel", channel)), + fn, retry.Attempts(20), retry.Sleep(time.Millisecond*200), retry.MaxSleepTime(5*time.Second)) if err != nil { errMsg := fmt.Sprintf("Failed to create consumer %s", channel) if merr.IsCanceledOrTimeout(err) { diff --git a/pkg/mq/msgstream/mqwrapper/kafka/kafka_client.go b/pkg/mq/msgstream/mqwrapper/kafka/kafka_client.go index 91b9753ce009e..5f588077b7a48 100644 --- a/pkg/mq/msgstream/mqwrapper/kafka/kafka_client.go +++ b/pkg/mq/msgstream/mqwrapper/kafka/kafka_client.go @@ -3,6 +3,7 @@ package kafka import ( "context" "fmt" + "go.opentelemetry.io/otel" "strconv" "sync" "time" @@ -228,6 +229,8 @@ func (kc *kafkaClient) Subscribe(ctx context.Context, options mqwrapper.Consumer start := timerecord.NewTimeRecorder("create consumer") metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.TotalLabel).Inc() + ctx, sp := otel.Tracer("KafkaClient").Start(ctx, "NewConsumer") + defer sp.End() config := kc.newConsumerConfig(options.SubscriptionName, options.SubscriptionInitialPosition) consumer, err := newKafkaConsumer(config, options.BufSize, options.Topic, options.SubscriptionName, options.SubscriptionInitialPosition) if err != nil { diff --git a/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_client.go b/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_client.go index c33f0df93846d..9476c356e87ce 100644 --- a/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_client.go +++ b/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_client.go @@ -26,6 +26,7 @@ import ( "github.com/cockroachdb/errors" pulsarctl "github.com/streamnative/pulsarctl/pkg/pulsar" "github.com/streamnative/pulsarctl/pkg/pulsar/common" + "go.opentelemetry.io/otel" "go.uber.org/zap" "github.com/milvus-io/milvus/pkg/log" @@ -68,6 +69,8 @@ func NewClient(tenant string, namespace string, opts pulsar.ClientOptions) (*pul // CreateProducer create a pulsar producer from options func (pc *pulsarClient) CreateProducer(ctx context.Context, options mqcommon.ProducerOptions) (mqwrapper.Producer, error) { + ctx, sp := otel.Tracer("PulsarClient").Start(ctx, "CreateProducer") + defer sp.End() start := timerecord.NewTimeRecorder("create producer") metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateProducerLabel, metrics.TotalLabel).Inc() @@ -104,6 +107,8 @@ func (pc *pulsarClient) CreateProducer(ctx context.Context, options mqcommon.Pro // Subscribe creates a pulsar consumer instance and subscribe a topic func (pc *pulsarClient) Subscribe(ctx context.Context, options mqwrapper.ConsumerOptions) (mqwrapper.Consumer, error) { + ctx, sp := otel.Tracer("PulsarClient").Start(ctx, "Subscribe") + defer sp.End() start := timerecord.NewTimeRecorder("create consumer") metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.TotalLabel).Inc() @@ -113,6 +118,8 @@ func (pc *pulsarClient) Subscribe(ctx context.Context, options mqwrapper.Consume metrics.MsgStreamOpCounter.WithLabelValues(metrics.CreateConsumerLabel, metrics.FailLabel).Inc() return nil, err } + sp.AddEvent(fmt.Sprintf("subscribing to topic=%s", fullTopicName)) + sp.AddEvent(fmt.Sprintf("subscriptionName=%s", options.SubscriptionName)) consumer, err := pc.client.Subscribe(pulsar.ConsumerOptions{ Topic: fullTopicName, SubscriptionName: options.SubscriptionName, diff --git a/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_producer.go b/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_producer.go index fcd95ca16354d..3a78abba81437 100644 --- a/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_producer.go +++ b/pkg/mq/msgstream/mqwrapper/pulsar/pulsar_producer.go @@ -18,8 +18,10 @@ package pulsar import ( "context" + "fmt" "github.com/apache/pulsar-client-go/pulsar" + "go.opentelemetry.io/otel" "github.com/milvus-io/milvus/pkg/metrics" "github.com/milvus-io/milvus/pkg/mq/common" @@ -40,6 +42,10 @@ func (pp *pulsarProducer) Topic() string { } func (pp *pulsarProducer) Send(ctx context.Context, message *common.ProducerMessage) (common.MessageID, error) { + ctx, sp := otel.Tracer("pulsarProducer").Start(ctx, "PulsarProducer.Send") + defer sp.End() + sp.AddEvent(fmt.Sprintf("Topic=%s", pp.p.Topic())) + sp.AddEvent(fmt.Sprintf("PayloadSize=%d", len(message.Payload))) start := timerecord.NewTimeRecorder("send msg to stream") metrics.MsgStreamOpCounter.WithLabelValues(metrics.SendMsgLabel, metrics.TotalLabel).Inc() diff --git a/pkg/mq/msgstream/mqwrapper/rmq/rmq_client.go b/pkg/mq/msgstream/mqwrapper/rmq/rmq_client.go index b40192fe85fdc..d745a8f923047 100644 --- a/pkg/mq/msgstream/mqwrapper/rmq/rmq_client.go +++ b/pkg/mq/msgstream/mqwrapper/rmq/rmq_client.go @@ -21,6 +21,7 @@ import ( "strconv" "github.com/cockroachdb/errors" + "go.opentelemetry.io/otel" "go.uber.org/zap" "github.com/milvus-io/milvus/pkg/log" @@ -89,6 +90,8 @@ func (rc *rmqClient) Subscribe(ctx context.Context, options mqwrapper.ConsumerOp } receiveChannel := make(chan common.Message, options.BufSize) + ctx, sp := otel.Tracer("RmqClient").Start(ctx, "Subscribe") + defer sp.End() cli, err := rc.client.Subscribe(client.ConsumerOptions{ Topic: options.Topic, SubscriptionName: options.SubscriptionName,