fix: Fix some replicas don't participate in the query after the failure recovery (#35850)

issue: #35846
querycoord will notify proxy to update shard leader cache after
delegator location changes, but during querynode's failure recovery,
some delegator may become unserviceable due to lacking of segments, and
back to serviceable after segment loaded, so we also need to notify
proxy to invalidate shard leader cache when delegator serviceable state
changes.

This PR will maintain querynode's serviceable state during heartbeat,
and notify proxy to invalidate shard leader cache if serviceable state
changes.

Signed-off-by: Wei Liu <wei.liu@zilliz.com>
This commit is contained in:
wei liu 2024-09-03 15:39:03 +08:00 committed by GitHub
parent 6130a85444
commit c84ea5465c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 136 additions and 59 deletions

View File

@ -171,7 +171,6 @@ func (c *ChannelChecker) findRepeatedChannels(ctx context.Context, replicaID int
} }
dist := c.dist.ChannelDistManager.GetByCollectionAndFilter(replica.GetCollectionID(), meta.WithReplica2Channel(replica)) dist := c.dist.ChannelDistManager.GetByCollectionAndFilter(replica.GetCollectionID(), meta.WithReplica2Channel(replica))
targets := c.targetMgr.GetSealedSegmentsByCollection(replica.GetCollectionID(), meta.CurrentTarget)
versionsMap := make(map[string]*meta.DmChannel) versionsMap := make(map[string]*meta.DmChannel)
for _, ch := range dist { for _, ch := range dist {
leaderView := c.dist.LeaderViewManager.GetLeaderShardView(ch.Node, ch.GetChannelName()) leaderView := c.dist.LeaderViewManager.GetLeaderShardView(ch.Node, ch.GetChannelName())
@ -184,13 +183,13 @@ func (c *ChannelChecker) findRepeatedChannels(ctx context.Context, replicaID int
continue continue
} }
if err := utils.CheckLeaderAvailable(c.nodeMgr, leaderView, targets); err != nil { if leaderView.UnServiceableError != nil {
log.RatedInfo(10, "replica has unavailable shard leader", log.RatedInfo(10, "replica has unavailable shard leader",
zap.Int64("collectionID", replica.GetCollectionID()), zap.Int64("collectionID", replica.GetCollectionID()),
zap.Int64("replicaID", replicaID), zap.Int64("replicaID", replicaID),
zap.Int64("leaderID", ch.Node), zap.Int64("leaderID", ch.Node),
zap.String("channel", ch.GetChannelName()), zap.String("channel", ch.GetChannelName()),
zap.Error(err)) zap.Error(leaderView.UnServiceableError))
continue continue
} }

View File

@ -32,6 +32,7 @@ import (
. "github.com/milvus-io/milvus/internal/querycoordv2/params" . "github.com/milvus-io/milvus/internal/querycoordv2/params"
"github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/session"
"github.com/milvus-io/milvus/internal/querycoordv2/task" "github.com/milvus-io/milvus/internal/querycoordv2/task"
"github.com/milvus-io/milvus/internal/querycoordv2/utils"
"github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/commonpbutil" "github.com/milvus-io/milvus/pkg/util/commonpbutil"
"github.com/milvus-io/milvus/pkg/util/merr" "github.com/milvus-io/milvus/pkg/util/merr"
@ -220,6 +221,11 @@ func (dh *distHandler) updateLeaderView(resp *querypb.GetDataDistributionRespons
NumOfGrowingRows: lview.GetNumOfGrowingRows(), NumOfGrowingRows: lview.GetNumOfGrowingRows(),
PartitionStatsVersions: lview.PartitionStatsVersions, PartitionStatsVersions: lview.PartitionStatsVersions,
} }
// check leader serviceable
// todo by weiliu1031: serviceable status should be maintained by delegator, to avoid heavy check here
if err := utils.CheckLeaderAvailable(dh.nodeManager, dh.target, view); err != nil {
view.UnServiceableError = err
}
updates = append(updates, view) updates = append(updates, view)
} }

View File

@ -25,6 +25,7 @@ import (
"github.com/stretchr/testify/mock" "github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite" "github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/querycoordv2/meta" "github.com/milvus-io/milvus/internal/querycoordv2/meta"
"github.com/milvus-io/milvus/internal/querycoordv2/session" "github.com/milvus-io/milvus/internal/querycoordv2/session"
@ -74,6 +75,8 @@ func (suite *DistHandlerSuite) TestBasic() {
suite.dispatchMockCall.Unset() suite.dispatchMockCall.Unset()
suite.dispatchMockCall = nil suite.dispatchMockCall = nil
} }
suite.target.EXPECT().GetSealedSegmentsByChannel(mock.Anything, mock.Anything, mock.Anything).Return(map[int64]*datapb.SegmentInfo{})
suite.dispatchMockCall = suite.scheduler.EXPECT().Dispatch(mock.Anything).Maybe() suite.dispatchMockCall = suite.scheduler.EXPECT().Dispatch(mock.Anything).Maybe()
suite.nodeManager.Add(session.NewNodeInfo(session.ImmutableNodeInfo{ suite.nodeManager.Add(session.NewNodeInfo(session.ImmutableNodeInfo{
NodeID: 1, NodeID: 1,
@ -120,6 +123,7 @@ func (suite *DistHandlerSuite) TestGetDistributionFailed() {
suite.dispatchMockCall.Unset() suite.dispatchMockCall.Unset()
suite.dispatchMockCall = nil suite.dispatchMockCall = nil
} }
suite.target.EXPECT().GetSealedSegmentsByChannel(mock.Anything, mock.Anything, mock.Anything).Return(map[int64]*datapb.SegmentInfo{}).Maybe()
suite.dispatchMockCall = suite.scheduler.EXPECT().Dispatch(mock.Anything).Maybe() suite.dispatchMockCall = suite.scheduler.EXPECT().Dispatch(mock.Anything).Maybe()
suite.nodeManager.Add(session.NewNodeInfo(session.ImmutableNodeInfo{ suite.nodeManager.Add(session.NewNodeInfo(session.ImmutableNodeInfo{
NodeID: 1, NodeID: 1,
@ -140,6 +144,8 @@ func (suite *DistHandlerSuite) TestForcePullDist() {
suite.dispatchMockCall = nil suite.dispatchMockCall = nil
} }
suite.target.EXPECT().GetSealedSegmentsByChannel(mock.Anything, mock.Anything, mock.Anything).Return(map[int64]*datapb.SegmentInfo{}).Maybe()
suite.nodeManager.Add(session.NewNodeInfo(session.ImmutableNodeInfo{ suite.nodeManager.Add(session.NewNodeInfo(session.ImmutableNodeInfo{
NodeID: 1, NodeID: 1,
Address: "localhost", Address: "localhost",

View File

@ -119,6 +119,7 @@ type LeaderView struct {
TargetVersion int64 TargetVersion int64
NumOfGrowingRows int64 NumOfGrowingRows int64
PartitionStatsVersions map[int64]int64 PartitionStatsVersions map[int64]int64
UnServiceableError error
} }
func (view *LeaderView) Clone() *LeaderView { func (view *LeaderView) Clone() *LeaderView {
@ -231,6 +232,9 @@ func (mgr *LeaderViewManager) Update(leaderID int64, views ...*LeaderView) {
mgr.views[leaderID] = composeNodeViews(views...) mgr.views[leaderID] = composeNodeViews(views...)
// compute leader location change, find it's correspond collection // compute leader location change, find it's correspond collection
// 1. leader has been released from node
// 2. leader has been loaded to node
// 3. leader serviceable status changed
if mgr.notifyFunc != nil { if mgr.notifyFunc != nil {
viewChanges := typeutil.NewUniqueSet() viewChanges := typeutil.NewUniqueSet()
for channel, oldView := range oldViews { for channel, oldView := range oldViews {
@ -240,9 +244,17 @@ func (mgr *LeaderViewManager) Update(leaderID int64, views ...*LeaderView) {
} }
} }
serviceableChange := func(old, new *LeaderView) bool {
if old == nil || new == nil {
return true
}
return (old.UnServiceableError == nil) != (new.UnServiceableError == nil)
}
for channel, newView := range newViews { for channel, newView := range newViews {
// if channel loaded to current node // if channel loaded to current node
if _, ok := oldViews[channel]; !ok { if oldView, ok := oldViews[channel]; !ok || serviceableChange(oldView, newView) {
viewChanges.Insert(newView.CollectionID) viewChanges.Insert(newView.CollectionID)
} }
} }

View File

@ -19,10 +19,12 @@ package meta
import ( import (
"testing" "testing"
"github.com/cockroachdb/errors"
"github.com/samber/lo" "github.com/samber/lo"
"github.com/stretchr/testify/suite" "github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus/internal/proto/querypb" "github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/pkg/util/typeutil"
) )
type LeaderViewManagerSuite struct { type LeaderViewManagerSuite struct {
@ -248,16 +250,68 @@ func (suite *LeaderViewManagerSuite) TestNotifyDelegatorChanges() {
}, },
} }
updateCollections := make([]int64, 0) retSet := typeutil.NewUniqueSet()
mgr.SetNotifyFunc(func(collectionIDs ...int64) { mgr.SetNotifyFunc(func(collectionIDs ...int64) {
updateCollections = append(updateCollections, collectionIDs...) retSet.Insert(collectionIDs...)
}) })
mgr.Update(1, newViews...) mgr.Update(1, newViews...)
suite.Equal(2, retSet.Len())
suite.True(retSet.Contain(100))
suite.True(retSet.Contain(103))
suite.Equal(2, len(updateCollections)) newViews1 := []*LeaderView{
suite.Contains(updateCollections, int64(100)) {
suite.Contains(updateCollections, int64(103)) ID: 1,
CollectionID: 101,
Channel: "test-channel-2",
UnServiceableError: errors.New("test error"),
},
{
ID: 1,
CollectionID: 102,
Channel: "test-channel-3",
UnServiceableError: errors.New("test error"),
},
{
ID: 1,
CollectionID: 103,
Channel: "test-channel-4",
UnServiceableError: errors.New("test error"),
},
}
retSet.Clear()
mgr.Update(1, newViews1...)
suite.Equal(3, len(retSet))
suite.True(retSet.Contain(101))
suite.True(retSet.Contain(102))
suite.True(retSet.Contain(103))
newViews2 := []*LeaderView{
{
ID: 1,
CollectionID: 101,
Channel: "test-channel-2",
UnServiceableError: errors.New("test error"),
},
{
ID: 1,
CollectionID: 102,
Channel: "test-channel-3",
},
{
ID: 1,
CollectionID: 103,
Channel: "test-channel-4",
},
}
retSet.Clear()
mgr.Update(1, newViews2...)
suite.Equal(2, len(retSet))
suite.True(retSet.Contain(102))
suite.True(retSet.Contain(103))
} }
func TestLeaderViewManager(t *testing.T) { func TestLeaderViewManager(t *testing.T) {

View File

@ -2038,6 +2038,7 @@ func (suite *ServiceSuite) updateChannelDistWithoutSegment(collection int64) {
ID: node, ID: node,
CollectionID: collection, CollectionID: collection,
Channel: channels[i], Channel: channels[i],
UnServiceableError: merr.ErrSegmentLack,
}) })
i++ i++
if i >= len(channels) { if i >= len(channels) {

View File

@ -699,25 +699,6 @@ func (scheduler *taskScheduler) preProcess(task Task) bool {
return false return false
} }
// check if new delegator is ready to release old delegator
checkLeaderView := func(collectionID int64, channel string, node int64) bool {
segmentsInTarget := scheduler.targetMgr.GetSealedSegmentsByChannel(collectionID, channel, meta.CurrentTarget)
leader := scheduler.distMgr.LeaderViewManager.GetLeaderShardView(node, channel)
if leader == nil {
return false
}
for segmentID, s := range segmentsInTarget {
_, exist := leader.Segments[segmentID]
l0WithWrongLocation := exist && s.GetLevel() == datapb.SegmentLevel_L0 && leader.Segments[segmentID].GetNodeID() != leader.ID
if !exist || l0WithWrongLocation {
return false
}
}
return true
}
actions, step := task.Actions(), task.Step() actions, step := task.Actions(), task.Step()
for step < len(actions) && actions[step].IsFinished(scheduler.distMgr) { for step < len(actions) && actions[step].IsFinished(scheduler.distMgr) {
if GetTaskType(task) == TaskTypeMove && actions[step].Type() == ActionTypeGrow { if GetTaskType(task) == TaskTypeMove && actions[step].Type() == ActionTypeGrow {
@ -729,7 +710,8 @@ func (scheduler *taskScheduler) preProcess(task Task) bool {
// causes a few time to load delta log, if reduce the old delegator in advance, // causes a few time to load delta log, if reduce the old delegator in advance,
// new delegator can't service search and query, will got no available channel error // new delegator can't service search and query, will got no available channel error
channelAction := actions[step].(*ChannelAction) channelAction := actions[step].(*ChannelAction)
ready = checkLeaderView(task.CollectionID(), channelAction.Shard(), channelAction.Node()) leader := scheduler.distMgr.LeaderViewManager.GetLeaderShardView(channelAction.Node(), channelAction.Shard())
ready = leader.UnServiceableError == nil
default: default:
ready = true ready = true
} }

View File

@ -1669,6 +1669,7 @@ func (suite *TaskSuite) TestBalanceChannelTask() {
ID: 1, ID: 1,
CollectionID: collectionID, CollectionID: collectionID,
Channel: channel, Channel: channel,
UnServiceableError: merr.ErrSegmentLack,
}) })
task, err := NewChannelTask(context.Background(), task, err := NewChannelTask(context.Background(),
10*time.Second, 10*time.Second,
@ -1763,6 +1764,7 @@ func (suite *TaskSuite) TestBalanceChannelWithL0SegmentTask() {
2: {NodeID: 2}, 2: {NodeID: 2},
3: {NodeID: 2}, 3: {NodeID: 2},
}, },
UnServiceableError: merr.ErrSegmentLack,
}) })
task, err := NewChannelTask(context.Background(), task, err := NewChannelTask(context.Background(),

View File

@ -45,7 +45,7 @@ func CheckNodeAvailable(nodeID int64, info *session.NodeInfo) error {
// 2. All QueryNodes in the distribution are online // 2. All QueryNodes in the distribution are online
// 3. The last heartbeat response time is within HeartbeatAvailableInterval for all QueryNodes(include leader) in the distribution // 3. The last heartbeat response time is within HeartbeatAvailableInterval for all QueryNodes(include leader) in the distribution
// 4. All segments of the shard in target should be in the distribution // 4. All segments of the shard in target should be in the distribution
func CheckLeaderAvailable(nodeMgr *session.NodeManager, leader *meta.LeaderView, currentTargets map[int64]*datapb.SegmentInfo) error { func CheckLeaderAvailable(nodeMgr *session.NodeManager, targetMgr meta.TargetManagerInterface, leader *meta.LeaderView) error {
log := log.Ctx(context.TODO()). log := log.Ctx(context.TODO()).
WithRateGroup("utils.CheckLeaderAvailable", 1, 60). WithRateGroup("utils.CheckLeaderAvailable", 1, 60).
With(zap.Int64("leaderID", leader.ID)) With(zap.Int64("leaderID", leader.ID))
@ -68,18 +68,20 @@ func CheckLeaderAvailable(nodeMgr *session.NodeManager, leader *meta.LeaderView,
return err return err
} }
} }
segmentDist := targetMgr.GetSealedSegmentsByChannel(leader.CollectionID, leader.Channel, meta.CurrentTarget)
// Check whether segments are fully loaded // Check whether segments are fully loaded
for segmentID, info := range currentTargets { for segmentID, info := range segmentDist {
if info.GetInsertChannel() != leader.Channel {
continue
}
_, exist := leader.Segments[segmentID] _, exist := leader.Segments[segmentID]
if !exist { if !exist {
log.RatedInfo(10, "leader is not available due to lack of segment", zap.Int64("segmentID", segmentID)) log.RatedInfo(10, "leader is not available due to lack of segment", zap.Int64("segmentID", segmentID))
return merr.WrapErrSegmentLack(segmentID) return merr.WrapErrSegmentLack(segmentID)
} }
l0WithWrongLocation := info.GetLevel() == datapb.SegmentLevel_L0 && leader.Segments[segmentID].GetNodeID() != leader.ID
if l0WithWrongLocation {
log.RatedInfo(10, "leader is not available due to lack of L0 segment", zap.Int64("segmentID", segmentID))
return merr.WrapErrSegmentLack(segmentID)
}
} }
return nil return nil
} }
@ -110,7 +112,6 @@ func GetShardLeadersWithChannels(m *meta.Meta, targetMgr meta.TargetManagerInter
nodeMgr *session.NodeManager, collectionID int64, channels map[string]*meta.DmChannel, nodeMgr *session.NodeManager, collectionID int64, channels map[string]*meta.DmChannel,
) ([]*querypb.ShardLeadersList, error) { ) ([]*querypb.ShardLeadersList, error) {
ret := make([]*querypb.ShardLeadersList, 0) ret := make([]*querypb.ShardLeadersList, 0)
currentTargets := targetMgr.GetSealedSegmentsByCollection(collectionID, meta.CurrentTarget)
for _, channel := range channels { for _, channel := range channels {
log := log.With(zap.String("channel", channel.GetChannelName())) log := log.With(zap.String("channel", channel.GetChannelName()))
@ -122,8 +123,8 @@ func GetShardLeadersWithChannels(m *meta.Meta, targetMgr meta.TargetManagerInter
readableLeaders := make(map[int64]*meta.LeaderView) readableLeaders := make(map[int64]*meta.LeaderView)
for _, leader := range leaders { for _, leader := range leaders {
if err := CheckLeaderAvailable(nodeMgr, leader, currentTargets); err != nil { if leader.UnServiceableError != nil {
multierr.AppendInto(&channelErr, err) multierr.AppendInto(&channelErr, leader.UnServiceableError)
continue continue
} }
readableLeaders[leader.ID] = leader readableLeaders[leader.ID] = leader
@ -149,6 +150,9 @@ func GetShardLeadersWithChannels(m *meta.Meta, targetMgr meta.TargetManagerInter
// to avoid node down during GetShardLeaders // to avoid node down during GetShardLeaders
if len(ids) == 0 { if len(ids) == 0 {
if channelErr == nil {
channelErr = merr.WrapErrChannelNotAvailable(channel.GetChannelName())
}
msg := fmt.Sprintf("channel %s is not available in any replica", channel.GetChannelName()) msg := fmt.Sprintf("channel %s is not available in any replica", channel.GetChannelName())
log.Warn(msg, zap.Error(channelErr)) log.Warn(msg, zap.Error(channelErr))
err := merr.WrapErrChannelNotAvailable(channel.GetChannelName(), channelErr.Error()) err := merr.WrapErrChannelNotAvailable(channel.GetChannelName(), channelErr.Error())

View File

@ -20,6 +20,7 @@ import (
"testing" "testing"
"time" "time"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite" "github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/datapb"
@ -56,13 +57,16 @@ func (suite *UtilTestSuite) TestCheckLeaderAvaliable() {
Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2}}, Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2}},
} }
suite.setNodeAvailable(1, 2) mockTargetManager := meta.NewMockTargetManager(suite.T())
err := CheckLeaderAvailable(suite.nodeMgr, leadview, map[int64]*datapb.SegmentInfo{ mockTargetManager.EXPECT().GetSealedSegmentsByChannel(mock.Anything, mock.Anything, mock.Anything).Return(map[int64]*datapb.SegmentInfo{
2: { 2: {
ID: 2, ID: 2,
InsertChannel: "test", InsertChannel: "test",
}, },
}) }).Maybe()
suite.setNodeAvailable(1, 2)
err := CheckLeaderAvailable(suite.nodeMgr, mockTargetManager, leadview)
suite.NoError(err) suite.NoError(err)
} }
@ -73,14 +77,16 @@ func (suite *UtilTestSuite) TestCheckLeaderAvaliableFailed() {
Channel: "test", Channel: "test",
Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2}}, Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2}},
} }
// leader nodeID=1 not available mockTargetManager := meta.NewMockTargetManager(suite.T())
suite.setNodeAvailable(2) mockTargetManager.EXPECT().GetSealedSegmentsByChannel(mock.Anything, mock.Anything, mock.Anything).Return(map[int64]*datapb.SegmentInfo{
err := CheckLeaderAvailable(suite.nodeMgr, leadview, map[int64]*datapb.SegmentInfo{
2: { 2: {
ID: 2, ID: 2,
InsertChannel: "test", InsertChannel: "test",
}, },
}) }).Maybe()
// leader nodeID=1 not available
suite.setNodeAvailable(2)
err := CheckLeaderAvailable(suite.nodeMgr, mockTargetManager, leadview)
suite.Error(err) suite.Error(err)
suite.nodeMgr = session.NewNodeManager() suite.nodeMgr = session.NewNodeManager()
}) })
@ -91,14 +97,17 @@ func (suite *UtilTestSuite) TestCheckLeaderAvaliableFailed() {
Channel: "test", Channel: "test",
Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2}}, Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2}},
} }
// leader nodeID=2 not available
suite.setNodeAvailable(1) mockTargetManager := meta.NewMockTargetManager(suite.T())
err := CheckLeaderAvailable(suite.nodeMgr, leadview, map[int64]*datapb.SegmentInfo{ mockTargetManager.EXPECT().GetSealedSegmentsByChannel(mock.Anything, mock.Anything, mock.Anything).Return(map[int64]*datapb.SegmentInfo{
2: { 2: {
ID: 2, ID: 2,
InsertChannel: "test", InsertChannel: "test",
}, },
}) }).Maybe()
// leader nodeID=2 not available
suite.setNodeAvailable(1)
err := CheckLeaderAvailable(suite.nodeMgr, mockTargetManager, leadview)
suite.Error(err) suite.Error(err)
suite.nodeMgr = session.NewNodeManager() suite.nodeMgr = session.NewNodeManager()
}) })
@ -109,14 +118,16 @@ func (suite *UtilTestSuite) TestCheckLeaderAvaliableFailed() {
Channel: "test", Channel: "test",
Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2}}, Segments: map[int64]*querypb.SegmentDist{2: {NodeID: 2}},
} }
suite.setNodeAvailable(1, 2) mockTargetManager := meta.NewMockTargetManager(suite.T())
err := CheckLeaderAvailable(suite.nodeMgr, leadview, map[int64]*datapb.SegmentInfo{ mockTargetManager.EXPECT().GetSealedSegmentsByChannel(mock.Anything, mock.Anything, mock.Anything).Return(map[int64]*datapb.SegmentInfo{
// target segmentID=1 not in leadView // target segmentID=1 not in leadView
1: { 1: {
ID: 1, ID: 1,
InsertChannel: "test", InsertChannel: "test",
}, },
}) }).Maybe()
suite.setNodeAvailable(1, 2)
err := CheckLeaderAvailable(suite.nodeMgr, mockTargetManager, leadview)
suite.Error(err) suite.Error(err)
suite.nodeMgr = session.NewNodeManager() suite.nodeMgr = session.NewNodeManager()
}) })