mirror of https://github.com/milvus-io/milvus.git
Fix Release tasks may end with incorrect status (#20263)
If the resource has been removed, the release tasks should end with Succeeded status, not stale Signed-off-by: yah01 <yang.cen@zilliz.com> Signed-off-by: yah01 <yang.cen@zilliz.com>pull/20285/head
parent
f4a9761e6b
commit
c08f11420f
|
@ -24,6 +24,7 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/internal/proto/querypb"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
. "github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
)
|
||||
|
||||
|
@ -106,15 +107,27 @@ func (action *SegmentAction) IsFinished(distMgr *meta.DistributionManager) bool
|
|||
if action.Type() == ActionTypeGrow {
|
||||
nodes := distMgr.LeaderViewManager.GetSealedSegmentDist(action.SegmentID())
|
||||
return lo.Contains(nodes, action.Node())
|
||||
} else if action.Type() == ActionTypeReduce {
|
||||
// FIXME: Now shard leader's segment view is a map of segment ID to node ID,
|
||||
// loading segment replaces the node ID with the new one,
|
||||
// which confuses the condition of finishing,
|
||||
// the leader should return a map of segment ID to list of nodes,
|
||||
// now, we just always commit the release task to executor once.
|
||||
// NOTE: DO NOT create a task containing release action and the action is not the last action
|
||||
sealed := distMgr.SegmentDistManager.GetByNode(action.Node())
|
||||
growing := distMgr.LeaderViewManager.GetSegmentByNode(action.Node())
|
||||
segments := make([]int64, 0, len(sealed)+len(growing))
|
||||
for _, segment := range sealed {
|
||||
segments = append(segments, segment.GetID())
|
||||
}
|
||||
segments = append(segments, growing...)
|
||||
if !funcutil.SliceContain(segments, action.SegmentID()) {
|
||||
return true
|
||||
}
|
||||
return action.isReleaseCommitted.Load()
|
||||
}
|
||||
// FIXME: Now shard leader's segment view is a map of segment ID to node ID,
|
||||
// loading segment replaces the node ID with the new one,
|
||||
// which confuses the condition of finishing,
|
||||
// the leader should return a map of segment ID to list of nodes,
|
||||
// now, we just always commit the release task to executor once.
|
||||
// NOTE: DO NOT create a task containing release action and the action is not the last action
|
||||
|
||||
return action.isReleaseCommitted.Load()
|
||||
return true
|
||||
}
|
||||
|
||||
type ChannelAction struct {
|
||||
|
|
|
@ -26,7 +26,6 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/session"
|
||||
"github.com/milvus-io/milvus/internal/querycoordv2/utils"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
. "github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
@ -645,19 +644,8 @@ func (scheduler *taskScheduler) checkSegmentTaskStale(task *SegmentTask) bool {
|
|||
}
|
||||
|
||||
case ActionTypeReduce:
|
||||
// Do nothing here,
|
||||
// the task should succeeded if the segment not exists
|
||||
sealed := scheduler.distMgr.SegmentDistManager.GetByNode(action.Node())
|
||||
growing := scheduler.distMgr.LeaderViewManager.GetSegmentByNode(action.Node())
|
||||
segments := make([]int64, 0, len(sealed)+len(growing))
|
||||
for _, segment := range sealed {
|
||||
segments = append(segments, segment.GetID())
|
||||
}
|
||||
segments = append(segments, growing...)
|
||||
if !funcutil.SliceContain(segments, task.SegmentID()) {
|
||||
log.Warn("the task is stale, the segment to release not exists in dist",
|
||||
zap.Int64("segment", task.segmentID))
|
||||
return true
|
||||
}
|
||||
}
|
||||
}
|
||||
return false
|
||||
|
@ -679,20 +667,8 @@ func (scheduler *taskScheduler) checkChannelTaskStale(task *ChannelTask) bool {
|
|||
}
|
||||
|
||||
case ActionTypeReduce:
|
||||
// Do nothing here,
|
||||
// the task should succeeded if the channel not exists
|
||||
hasChannel := false
|
||||
views := scheduler.distMgr.LeaderViewManager.GetLeaderView(action.Node())
|
||||
for _, view := range views {
|
||||
if view.Channel == task.Channel() {
|
||||
hasChannel = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !hasChannel {
|
||||
log.Warn("the task is stale, the channel to unsubscribe not exists in dist",
|
||||
zap.String("channel", task.Channel()))
|
||||
return true
|
||||
}
|
||||
}
|
||||
}
|
||||
return false
|
||||
|
|
|
@ -144,6 +144,7 @@ func (suite *TaskSuite) BeforeTest(suiteName, testName string) {
|
|||
"TestLoadSegmentTask",
|
||||
"TestLoadSegmentTaskFailed",
|
||||
"TestSegmentTaskStale",
|
||||
"TestTaskCanceled",
|
||||
"TestMoveSegmentTask":
|
||||
suite.meta.PutCollection(&meta.Collection{
|
||||
CollectionLoadInfo: &querypb.CollectionLoadInfo{
|
||||
|
@ -297,14 +298,9 @@ func (suite *TaskSuite) TestUnsubscribeChannelTask() {
|
|||
suite.dispatchAndWait(targetNode)
|
||||
suite.AssertTaskNum(0, 0, 0, 0)
|
||||
|
||||
for i, task := range tasks {
|
||||
if i == 0 {
|
||||
suite.Equal(TaskStatusSucceeded, task.Status())
|
||||
suite.NoError(task.Err())
|
||||
} else {
|
||||
suite.Equal(TaskStatusStale, task.Status())
|
||||
suite.Error(task.Err())
|
||||
}
|
||||
for _, task := range tasks {
|
||||
suite.Equal(TaskStatusSucceeded, task.Status())
|
||||
suite.NoError(task.Err())
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -586,14 +582,9 @@ func (suite *TaskSuite) TestReleaseGrowingSegmentTask() {
|
|||
suite.dispatchAndWait(targetNode)
|
||||
suite.AssertTaskNum(0, 0, 0, 0)
|
||||
|
||||
for i, task := range tasks {
|
||||
if i == 0 {
|
||||
suite.Equal(TaskStatusStale, task.Status())
|
||||
suite.Error(task.Err())
|
||||
} else {
|
||||
suite.Equal(TaskStatusSucceeded, task.Status())
|
||||
suite.NoError(task.Err())
|
||||
}
|
||||
for _, task := range tasks {
|
||||
suite.Equal(TaskStatusSucceeded, task.Status())
|
||||
suite.NoError(task.Err())
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -702,60 +693,81 @@ func (suite *TaskSuite) TestMoveSegmentTask() {
|
|||
func (suite *TaskSuite) TestTaskCanceled() {
|
||||
ctx := context.Background()
|
||||
timeout := 10 * time.Second
|
||||
targetNode := int64(1)
|
||||
targetNode := int64(3)
|
||||
partition := int64(100)
|
||||
channel := &datapb.VchannelInfo{
|
||||
CollectionID: suite.collection,
|
||||
ChannelName: Params.CommonCfg.RootCoordDml + "-test",
|
||||
}
|
||||
|
||||
// Expect
|
||||
suite.cluster.EXPECT().UnsubDmChannel(mock.Anything, targetNode, mock.Anything).Return(utils.WrapStatus(commonpb.ErrorCode_Success, ""), nil)
|
||||
suite.broker.EXPECT().GetCollectionSchema(mock.Anything, suite.collection).Return(&schemapb.CollectionSchema{
|
||||
Name: "TestSubscribeChannelTask",
|
||||
}, nil)
|
||||
suite.broker.EXPECT().GetPartitions(mock.Anything, suite.collection).Return([]int64{100, 101}, nil)
|
||||
for _, segment := range suite.loadSegments {
|
||||
suite.broker.EXPECT().GetSegmentInfo(mock.Anything, segment).Return([]*datapb.SegmentInfo{
|
||||
{
|
||||
ID: segment,
|
||||
CollectionID: suite.collection,
|
||||
PartitionID: partition,
|
||||
InsertChannel: channel.ChannelName,
|
||||
},
|
||||
}, nil)
|
||||
suite.broker.EXPECT().GetIndexInfo(mock.Anything, suite.collection, segment).Return(nil, nil)
|
||||
}
|
||||
// suite.broker.EXPECT().GetRecoveryInfo(mock.Anything, suite.collection, partition).
|
||||
// Return([]*datapb.VchannelInfo{channel}, nil, nil)
|
||||
suite.cluster.EXPECT().LoadSegments(mock.Anything, targetNode, mock.Anything).Return(utils.WrapStatus(commonpb.ErrorCode_Success, ""), nil)
|
||||
|
||||
// Test unsubscribe channel task
|
||||
// Test load segment task
|
||||
suite.dist.ChannelDistManager.Update(targetNode, meta.DmChannelFromVChannel(&datapb.VchannelInfo{
|
||||
CollectionID: suite.collection,
|
||||
ChannelName: channel.ChannelName,
|
||||
}))
|
||||
tasks := []Task{}
|
||||
for _, channel := range suite.unsubChannels {
|
||||
suite.target.AddDmChannel(meta.DmChannelFromVChannel(&datapb.VchannelInfo{
|
||||
CollectionID: suite.collection,
|
||||
ChannelName: channel,
|
||||
}))
|
||||
task, err := NewChannelTask(
|
||||
for _, segment := range suite.loadSegments {
|
||||
suite.target.AddSegment(&datapb.SegmentInfo{
|
||||
ID: segment,
|
||||
CollectionID: suite.collection,
|
||||
PartitionID: partition,
|
||||
InsertChannel: channel.ChannelName,
|
||||
})
|
||||
task, err := NewSegmentTask(
|
||||
ctx,
|
||||
timeout,
|
||||
0,
|
||||
suite.collection,
|
||||
-1,
|
||||
NewChannelAction(targetNode, ActionTypeReduce, channel),
|
||||
suite.replica,
|
||||
NewSegmentAction(targetNode, ActionTypeGrow, channel.GetChannelName(), segment),
|
||||
)
|
||||
suite.NoError(err)
|
||||
tasks = append(tasks, task)
|
||||
err = suite.scheduler.Add(task)
|
||||
suite.NoError(err)
|
||||
}
|
||||
// Only first channel exists
|
||||
suite.dist.LeaderViewManager.Update(targetNode, &meta.LeaderView{
|
||||
ID: targetNode,
|
||||
CollectionID: suite.collection,
|
||||
Channel: suite.unsubChannels[0],
|
||||
})
|
||||
suite.AssertTaskNum(0, len(suite.unsubChannels), len(suite.unsubChannels), 0)
|
||||
segmentsNum := len(suite.loadSegments)
|
||||
suite.AssertTaskNum(0, segmentsNum, 0, segmentsNum)
|
||||
|
||||
// ProcessTasks
|
||||
// Process tasks
|
||||
suite.dispatchAndWait(targetNode)
|
||||
suite.AssertTaskNum(1, 0, 1, 0)
|
||||
suite.AssertTaskNum(segmentsNum, 0, 0, segmentsNum)
|
||||
|
||||
// Other nodes' HB can't trigger the procedure of tasks
|
||||
suite.dispatchAndWait(targetNode + 1)
|
||||
suite.AssertTaskNum(1, 0, 1, 0)
|
||||
suite.AssertTaskNum(segmentsNum, 0, 0, segmentsNum)
|
||||
|
||||
// Cancel all tasks
|
||||
for _, task := range tasks {
|
||||
task.Cancel()
|
||||
}
|
||||
|
||||
// Cancel first task
|
||||
tasks[0].Cancel()
|
||||
suite.dispatchAndWait(targetNode)
|
||||
suite.AssertTaskNum(0, 0, 0, 0)
|
||||
|
||||
for i, task := range tasks {
|
||||
if i == 0 {
|
||||
suite.Equal(TaskStatusCanceled, task.Status())
|
||||
suite.ErrorIs(task.Err(), ErrTaskCanceled)
|
||||
} else {
|
||||
suite.Equal(TaskStatusStale, task.Status())
|
||||
suite.Error(task.Err())
|
||||
}
|
||||
for _, task := range tasks {
|
||||
suite.Equal(TaskStatusCanceled, task.Status())
|
||||
suite.Error(task.Err())
|
||||
}
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue