Skip to content

Commit

Permalink
enhance: Decouple shard client manager from shard cache (#37371) (#37753
Browse files Browse the repository at this point in the history
)

issue: #37115
pr: #37371 #37646 #37729
the old implementation update shard cache and shard client manager at
same time, which causes lots of conor case due to concurrent issue
without lock.

This PR decouple shard client manager from shard cache, so only shard
cache will be updated if delegator changes. and make sure shard client
manager will always return the right client, and create a new client if
not exist. in case of client leak, shard client manager will purge
client in async for every 10 minutes.

---------

---------

Signed-off-by: Wei Liu <[email protected]>
Signed-off-by: Congqi Xia <[email protected]>
Co-authored-by: congqixia <[email protected]>
  • Loading branch information
weiliu1031 and congqixia authored Nov 25, 2024
1 parent e928e15 commit b245101
Show file tree
Hide file tree
Showing 18 changed files with 522 additions and 359 deletions.
1 change: 1 addition & 0 deletions internal/proxy/lb_balancer.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
)

type LBBalancer interface {
RegisterNodeInfo(nodeInfos []nodeInfo)
SelectNode(ctx context.Context, availableNodes []int64, nq int64) (int64, error)
CancelWorkload(node int64, nq int64)
UpdateCostMetrics(node int64, cost *internalpb.CostAggregation)
Expand Down
62 changes: 37 additions & 25 deletions internal/proxy/lb_policy.go
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ type ChannelWorkload struct {
collectionName string
collectionID int64
channel string
shardLeaders []int64
shardLeaders []nodeInfo
nq int64
exec executeFunc
retryTimes uint
Expand Down Expand Up @@ -115,9 +115,20 @@ func (lb *LBPolicyImpl) GetShardLeaders(ctx context.Context, dbName string, coll
}

// try to select the best node from the available nodes
func (lb *LBPolicyImpl) selectNode(ctx context.Context, balancer LBBalancer, workload ChannelWorkload, excludeNodes typeutil.UniqueSet) (int64, error) {
availableNodes := lo.FilterMap(workload.shardLeaders, func(node int64, _ int) (int64, bool) { return node, !excludeNodes.Contain(node) })
targetNode, err := balancer.SelectNode(ctx, availableNodes, workload.nq)
func (lb *LBPolicyImpl) selectNode(ctx context.Context, balancer LBBalancer, workload ChannelWorkload, excludeNodes typeutil.UniqueSet) (nodeInfo, error) {
filterDelegator := func(nodes []nodeInfo) map[int64]nodeInfo {
ret := make(map[int64]nodeInfo)
for _, node := range nodes {
if !excludeNodes.Contain(node.nodeID) {
ret[node.nodeID] = node
}
}
return ret
}

availableNodes := filterDelegator(workload.shardLeaders)
balancer.RegisterNodeInfo(lo.Values(availableNodes))
targetNode, err := balancer.SelectNode(ctx, lo.Keys(availableNodes), workload.nq)
if err != nil {
log := log.Ctx(ctx)
globalMetaCache.DeprecateShardCache(workload.db, workload.collectionName)
Expand All @@ -127,32 +138,33 @@ func (lb *LBPolicyImpl) selectNode(ctx context.Context, balancer LBBalancer, wor
zap.Int64("collectionID", workload.collectionID),
zap.String("channelName", workload.channel),
zap.Error(err))
return -1, err
return nodeInfo{}, err
}

availableNodes := lo.FilterMap(shardLeaders[workload.channel], func(node nodeInfo, _ int) (int64, bool) { return node.nodeID, !excludeNodes.Contain(node.nodeID) })
availableNodes = filterDelegator(shardLeaders[workload.channel])
if len(availableNodes) == 0 {
nodes := lo.Map(shardLeaders[workload.channel], func(node nodeInfo, _ int) int64 { return node.nodeID })
log.Warn("no available shard delegator found",
zap.Int64("collectionID", workload.collectionID),
zap.String("channelName", workload.channel),
zap.Int64s("nodes", nodes),
zap.Int64s("availableNodes", lo.Keys(availableNodes)),
zap.Int64s("excluded", excludeNodes.Collect()))
return -1, merr.WrapErrChannelNotAvailable("no available shard delegator found")
return nodeInfo{}, merr.WrapErrChannelNotAvailable("no available shard delegator found")
}

targetNode, err = balancer.SelectNode(ctx, availableNodes, workload.nq)
balancer.RegisterNodeInfo(lo.Values(availableNodes))
targetNode, err = balancer.SelectNode(ctx, lo.Keys(availableNodes), workload.nq)
if err != nil {
log.Warn("failed to select shard",
zap.Int64("collectionID", workload.collectionID),
zap.String("channelName", workload.channel),
zap.Int64s("availableNodes", availableNodes),
zap.Int64s("availableNodes", lo.Keys(availableNodes)),
zap.Int64s("excluded", excludeNodes.Collect()),
zap.Error(err))
return -1, err
return nodeInfo{}, err
}
}

return targetNode, nil
return availableNodes[targetNode], nil
}

// ExecuteWithRetry will choose a qn to execute the workload, and retry if failed, until reach the max retryTimes.
Expand All @@ -167,7 +179,7 @@ func (lb *LBPolicyImpl) ExecuteWithRetry(ctx context.Context, workload ChannelWo
log.Warn("failed to select node for shard",
zap.Int64("collectionID", workload.collectionID),
zap.String("channelName", workload.channel),
zap.Int64("nodeID", targetNode),
zap.Int64("nodeID", targetNode.nodeID),
zap.Error(err),
)
if lastErr != nil {
Expand All @@ -176,30 +188,30 @@ func (lb *LBPolicyImpl) ExecuteWithRetry(ctx context.Context, workload ChannelWo
return err
}
// cancel work load which assign to the target node
defer balancer.CancelWorkload(targetNode, workload.nq)
defer balancer.CancelWorkload(targetNode.nodeID, workload.nq)

client, err := lb.clientMgr.GetClient(ctx, targetNode)
if err != nil {
log.Warn("search/query channel failed, node not available",
zap.Int64("collectionID", workload.collectionID),
zap.String("channelName", workload.channel),
zap.Int64("nodeID", targetNode),
zap.Int64("nodeID", targetNode.nodeID),
zap.Error(err))
excludeNodes.Insert(targetNode)
excludeNodes.Insert(targetNode.nodeID)

lastErr = errors.Wrapf(err, "failed to get delegator %d for channel %s", targetNode, workload.channel)
lastErr = errors.Wrapf(err, "failed to get delegator %d for channel %s", targetNode.nodeID, workload.channel)
return lastErr
}

err = workload.exec(ctx, targetNode, client, workload.channel)
err = workload.exec(ctx, targetNode.nodeID, client, workload.channel)
if err != nil {
log.Warn("search/query channel failed",
zap.Int64("collectionID", workload.collectionID),
zap.String("channelName", workload.channel),
zap.Int64("nodeID", targetNode),
zap.Int64("nodeID", targetNode.nodeID),
zap.Error(err))
excludeNodes.Insert(targetNode)
lastErr = errors.Wrapf(err, "failed to search/query delegator %d for channel %s", targetNode, workload.channel)
excludeNodes.Insert(targetNode.nodeID)
lastErr = errors.Wrapf(err, "failed to search/query delegator %d for channel %s", targetNode.nodeID, workload.channel)
return lastErr
}

Expand All @@ -220,9 +232,9 @@ func (lb *LBPolicyImpl) Execute(ctx context.Context, workload CollectionWorkLoad
// let every request could retry at least twice, which could retry after update shard leader cache
retryTimes := Params.ProxyCfg.RetryTimesOnReplica.GetAsInt()
wg, ctx := errgroup.WithContext(ctx)
for channel, nodes := range dml2leaders {
channel := channel
nodes := lo.Map(nodes, func(node nodeInfo, _ int) int64 { return node.nodeID })
for k, v := range dml2leaders {
channel := k
nodes := v
channelRetryTimes := retryTimes
if len(nodes) > 0 {
channelRetryTimes *= len(nodes)
Expand Down
41 changes: 28 additions & 13 deletions internal/proxy/lb_policy_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,8 @@ type LBPolicySuite struct {
lbBalancer *MockLBBalancer
lbPolicy *LBPolicyImpl

nodes []int64
nodeIDs []int64
nodes []nodeInfo
channels []string
qnList []*mocks.MockQueryNode

Expand All @@ -65,7 +66,14 @@ func (s *LBPolicySuite) SetupSuite() {
}

func (s *LBPolicySuite) SetupTest() {
s.nodes = []int64{1, 2, 3, 4, 5}
s.nodeIDs = make([]int64, 0)
for i := 1; i <= 5; i++ {
s.nodeIDs = append(s.nodeIDs, int64(i))
s.nodes = append(s.nodes, nodeInfo{
nodeID: int64(i),
address: "localhost",
})
}
s.channels = []string{"channel1", "channel2"}
successStatus := commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}
qc := mocks.NewMockQueryCoordClient(s.T())
Expand All @@ -77,12 +85,12 @@ func (s *LBPolicySuite) SetupTest() {
Shards: []*querypb.ShardLeadersList{
{
ChannelName: s.channels[0],
NodeIds: s.nodes,
NodeIds: s.nodeIDs,
NodeAddrs: []string{"localhost:9000", "localhost:9001", "localhost:9002", "localhost:9003", "localhost:9004"},
},
{
ChannelName: s.channels[1],
NodeIds: s.nodes,
NodeIds: s.nodeIDs,
NodeAddrs: []string{"localhost:9000", "localhost:9001", "localhost:9002", "localhost:9003", "localhost:9004"},
},
},
Expand All @@ -99,7 +107,6 @@ func (s *LBPolicySuite) SetupTest() {
s.qn.EXPECT().GetComponentStates(mock.Anything, mock.Anything).Return(nil, nil).Maybe()

s.mgr = NewMockShardClientManager(s.T())
s.mgr.EXPECT().UpdateShardLeaders(mock.Anything, mock.Anything).Return(nil).Maybe()
s.lbBalancer = NewMockLBBalancer(s.T())
s.lbBalancer.EXPECT().Start(context.Background()).Maybe()
s.lbPolicy = NewLBPolicyImpl(s.mgr)
Expand Down Expand Up @@ -167,6 +174,7 @@ func (s *LBPolicySuite) loadCollection() {

func (s *LBPolicySuite) TestSelectNode() {
ctx := context.Background()
s.lbBalancer.EXPECT().RegisterNodeInfo(mock.Anything)
s.lbBalancer.EXPECT().SelectNode(mock.Anything, mock.Anything, mock.Anything).Return(5, nil)
targetNode, err := s.lbPolicy.selectNode(ctx, s.lbBalancer, ChannelWorkload{
db: dbName,
Expand All @@ -177,39 +185,41 @@ func (s *LBPolicySuite) TestSelectNode() {
nq: 1,
}, typeutil.NewUniqueSet())
s.NoError(err)
s.Equal(int64(5), targetNode)
s.Equal(int64(5), targetNode.nodeID)

// test select node failed, then update shard leader cache and retry, expect success
s.lbBalancer.ExpectedCalls = nil
s.lbBalancer.EXPECT().RegisterNodeInfo(mock.Anything)
s.lbBalancer.EXPECT().SelectNode(mock.Anything, mock.Anything, mock.Anything).Return(-1, errors.New("fake err")).Times(1)
s.lbBalancer.EXPECT().SelectNode(mock.Anything, mock.Anything, mock.Anything).Return(3, nil)
targetNode, err = s.lbPolicy.selectNode(ctx, s.lbBalancer, ChannelWorkload{
db: dbName,
collectionName: s.collectionName,
collectionID: s.collectionID,
channel: s.channels[0],
shardLeaders: []int64{},
shardLeaders: []nodeInfo{},
nq: 1,
}, typeutil.NewUniqueSet())
s.NoError(err)
s.Equal(int64(3), targetNode)
s.Equal(int64(3), targetNode.nodeID)

// test select node always fails, expected failure
s.lbBalancer.ExpectedCalls = nil
s.lbBalancer.EXPECT().RegisterNodeInfo(mock.Anything)
s.lbBalancer.EXPECT().SelectNode(mock.Anything, mock.Anything, mock.Anything).Return(-1, merr.ErrNodeNotAvailable)
targetNode, err = s.lbPolicy.selectNode(ctx, s.lbBalancer, ChannelWorkload{
db: dbName,
collectionName: s.collectionName,
collectionID: s.collectionID,
channel: s.channels[0],
shardLeaders: []int64{},
shardLeaders: []nodeInfo{},
nq: 1,
}, typeutil.NewUniqueSet())
s.ErrorIs(err, merr.ErrNodeNotAvailable)
s.Equal(int64(-1), targetNode)

// test all nodes has been excluded, expected failure
s.lbBalancer.ExpectedCalls = nil
s.lbBalancer.EXPECT().RegisterNodeInfo(mock.Anything)
s.lbBalancer.EXPECT().SelectNode(mock.Anything, mock.Anything, mock.Anything).Return(-1, merr.ErrNodeNotAvailable)
targetNode, err = s.lbPolicy.selectNode(ctx, s.lbBalancer, ChannelWorkload{
db: dbName,
Expand All @@ -218,12 +228,12 @@ func (s *LBPolicySuite) TestSelectNode() {
channel: s.channels[0],
shardLeaders: s.nodes,
nq: 1,
}, typeutil.NewUniqueSet(s.nodes...))
}, typeutil.NewUniqueSet(s.nodeIDs...))
s.ErrorIs(err, merr.ErrChannelNotAvailable)
s.Equal(int64(-1), targetNode)

// test get shard leaders failed, retry to select node failed
s.lbBalancer.ExpectedCalls = nil
s.lbBalancer.EXPECT().RegisterNodeInfo(mock.Anything)
s.lbBalancer.EXPECT().SelectNode(mock.Anything, mock.Anything, mock.Anything).Return(-1, merr.ErrNodeNotAvailable)
s.qc.ExpectedCalls = nil
s.qc.EXPECT().GetShardLeaders(mock.Anything, mock.Anything).Return(nil, merr.ErrServiceUnavailable)
Expand All @@ -236,7 +246,6 @@ func (s *LBPolicySuite) TestSelectNode() {
nq: 1,
}, typeutil.NewUniqueSet())
s.ErrorIs(err, merr.ErrServiceUnavailable)
s.Equal(int64(-1), targetNode)
}

func (s *LBPolicySuite) TestExecuteWithRetry() {
Expand All @@ -245,6 +254,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {
// test execute success
s.lbBalancer.ExpectedCalls = nil
s.mgr.EXPECT().GetClient(mock.Anything, mock.Anything).Return(s.qn, nil)
s.lbBalancer.EXPECT().RegisterNodeInfo(mock.Anything)
s.lbBalancer.EXPECT().SelectNode(mock.Anything, mock.Anything, mock.Anything).Return(1, nil)
s.lbBalancer.EXPECT().CancelWorkload(mock.Anything, mock.Anything)
err := s.lbPolicy.ExecuteWithRetry(ctx, ChannelWorkload{
Expand All @@ -263,6 +273,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {

// test select node failed, expected error
s.lbBalancer.ExpectedCalls = nil
s.lbBalancer.EXPECT().RegisterNodeInfo(mock.Anything)
s.lbBalancer.EXPECT().SelectNode(mock.Anything, mock.Anything, mock.Anything).Return(-1, merr.ErrNodeNotAvailable)
err = s.lbPolicy.ExecuteWithRetry(ctx, ChannelWorkload{
db: dbName,
Expand All @@ -282,6 +293,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {
s.mgr.ExpectedCalls = nil
s.mgr.EXPECT().GetClient(mock.Anything, mock.Anything).Return(nil, errors.New("fake error")).Times(1)
s.lbBalancer.ExpectedCalls = nil
s.lbBalancer.EXPECT().RegisterNodeInfo(mock.Anything)
s.lbBalancer.EXPECT().SelectNode(mock.Anything, mock.Anything, mock.Anything).Return(1, nil)
s.lbBalancer.EXPECT().CancelWorkload(mock.Anything, mock.Anything)
err = s.lbPolicy.ExecuteWithRetry(ctx, ChannelWorkload{
Expand All @@ -301,6 +313,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {
s.mgr.ExpectedCalls = nil
s.mgr.EXPECT().GetClient(mock.Anything, mock.Anything).Return(nil, errors.New("fake error")).Times(1)
s.mgr.EXPECT().GetClient(mock.Anything, mock.Anything).Return(s.qn, nil)
s.lbBalancer.EXPECT().RegisterNodeInfo(mock.Anything)
s.lbBalancer.EXPECT().CancelWorkload(mock.Anything, mock.Anything)
err = s.lbPolicy.ExecuteWithRetry(ctx, ChannelWorkload{
db: dbName,
Expand All @@ -320,6 +333,7 @@ func (s *LBPolicySuite) TestExecuteWithRetry() {
s.mgr.ExpectedCalls = nil
s.mgr.EXPECT().GetClient(mock.Anything, mock.Anything).Return(s.qn, nil)
s.lbBalancer.ExpectedCalls = nil
s.lbBalancer.EXPECT().RegisterNodeInfo(mock.Anything)
s.lbBalancer.EXPECT().SelectNode(mock.Anything, mock.Anything, mock.Anything).Return(1, nil)
s.lbBalancer.EXPECT().CancelWorkload(mock.Anything, mock.Anything)
counter := 0
Expand Down Expand Up @@ -369,6 +383,7 @@ func (s *LBPolicySuite) TestExecute() {
mockErr := errors.New("mock error")
// test all channel success
s.mgr.EXPECT().GetClient(mock.Anything, mock.Anything).Return(s.qn, nil)
s.lbBalancer.EXPECT().RegisterNodeInfo(mock.Anything)
s.lbBalancer.EXPECT().SelectNode(mock.Anything, mock.Anything, mock.Anything).Return(1, nil)
s.lbBalancer.EXPECT().CancelWorkload(mock.Anything, mock.Anything)
err := s.lbPolicy.Execute(ctx, CollectionWorkLoad{
Expand Down
18 changes: 14 additions & 4 deletions internal/proxy/look_aside_balancer.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,8 @@ type CostMetrics struct {
type LookAsideBalancer struct {
clientMgr shardClientMgr

metricsMap *typeutil.ConcurrentMap[int64, *CostMetrics]
knownNodeInfos *typeutil.ConcurrentMap[int64, nodeInfo]
metricsMap *typeutil.ConcurrentMap[int64, *CostMetrics]
// query node id -> number of consecutive heartbeat failures
failedHeartBeatCounter *typeutil.ConcurrentMap[int64, *atomic.Int64]

Expand All @@ -64,6 +65,7 @@ type LookAsideBalancer struct {
func NewLookAsideBalancer(clientMgr shardClientMgr) *LookAsideBalancer {
balancer := &LookAsideBalancer{
clientMgr: clientMgr,
knownNodeInfos: typeutil.NewConcurrentMap[int64, nodeInfo](),
metricsMap: typeutil.NewConcurrentMap[int64, *CostMetrics](),
failedHeartBeatCounter: typeutil.NewConcurrentMap[int64, *atomic.Int64](),
closeCh: make(chan struct{}),
Expand All @@ -88,6 +90,12 @@ func (b *LookAsideBalancer) Close() {
})
}

func (b *LookAsideBalancer) RegisterNodeInfo(nodeInfos []nodeInfo) {
for _, node := range nodeInfos {
b.knownNodeInfos.Insert(node.nodeID, node)
}
}

func (b *LookAsideBalancer) SelectNode(ctx context.Context, availableNodes []int64, nq int64) (int64, error) {
targetNode := int64(-1)
defer func() {
Expand Down Expand Up @@ -233,9 +241,10 @@ func (b *LookAsideBalancer) checkQueryNodeHealthLoop(ctx context.Context) {
case <-ticker.C:
var futures []*conc.Future[any]
now := time.Now()
b.metricsMap.Range(func(node int64, metrics *CostMetrics) bool {
b.knownNodeInfos.Range(func(node int64, info nodeInfo) bool {
futures = append(futures, pool.Submit(func() (any, error) {
if now.UnixMilli()-metrics.ts.Load() > checkHealthInterval.Milliseconds() {
metrics, ok := b.metricsMap.Get(node)
if !ok || now.UnixMilli()-metrics.ts.Load() > checkHealthInterval.Milliseconds() {
checkTimeout := Params.ProxyCfg.HealthCheckTimeout.GetAsDuration(time.Millisecond)
ctx, cancel := context.WithTimeout(context.Background(), checkTimeout)
defer cancel()
Expand All @@ -244,7 +253,7 @@ func (b *LookAsideBalancer) checkQueryNodeHealthLoop(ctx context.Context) {
panic("let it panic")
}

qn, err := b.clientMgr.GetClient(ctx, node)
qn, err := b.clientMgr.GetClient(ctx, info)
if err != nil {
// get client from clientMgr failed, which means this qn isn't a shard leader anymore, skip it's health check
b.trySetQueryNodeUnReachable(node, err)
Expand Down Expand Up @@ -304,6 +313,7 @@ func (b *LookAsideBalancer) trySetQueryNodeUnReachable(node int64, err error) {
zap.Int64("nodeID", node))
// stop the heartbeat
b.metricsMap.Remove(node)
b.knownNodeInfos.Remove(node)
return
}

Expand Down
Loading

0 comments on commit b245101

Please sign in to comment.