Skip to content

Commit

Permalink
fix: Fix dynamic release partition may fail search/query request
Browse files Browse the repository at this point in the history
cause concurrent issue may occur between remove parition in target
manager and sync segment list to delegator. when it happens, some
segment may be released in delegator, and those segment may also be
synced to delegator, which cause delegator become unserviceable due to
lack of necessary segments, then search/query fails.

this PR make sure that all write access to target_manager will be
executed in serial to avoid the concurrent issues.

Signed-off-by: Wei Liu <[email protected]>
  • Loading branch information
weiliu1031 committed Sep 3, 2024
1 parent 9d80137 commit 7ee41a8
Show file tree
Hide file tree
Showing 5 changed files with 95 additions and 30 deletions.
4 changes: 1 addition & 3 deletions internal/querycoordv2/job/job_release.go
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,6 @@ func (job *ReleaseCollectionJob) Execute() error {
log.Warn(msg, zap.Error(err))
}

job.targetMgr.RemoveCollection(req.GetCollectionID())
job.targetObserver.ReleaseCollection(req.GetCollectionID())
waitCollectionReleased(job.dist, job.checkerController, req.GetCollectionID())
metrics.QueryCoordNumCollections.WithLabelValues().Dec()
Expand Down Expand Up @@ -178,7 +177,6 @@ func (job *ReleasePartitionJob) Execute() error {
if err != nil {
log.Warn("failed to remove replicas", zap.Error(err))
}
job.targetMgr.RemoveCollection(req.GetCollectionID())
job.targetObserver.ReleaseCollection(req.GetCollectionID())
metrics.QueryCoordNumCollections.WithLabelValues().Dec()
waitCollectionReleased(job.dist, job.checkerController, req.GetCollectionID())
Expand All @@ -189,7 +187,7 @@ func (job *ReleasePartitionJob) Execute() error {
log.Warn(msg, zap.Error(err))
return errors.Wrap(err, msg)
}
job.targetMgr.RemovePartition(req.GetCollectionID(), toRelease...)
job.targetObserver.ReleasePartition(req.GetCollectionID(), toRelease...)
waitCollectionReleased(job.dist, job.checkerController, req.GetCollectionID(), toRelease...)
}
metrics.QueryCoordNumPartitions.WithLabelValues().Sub(float64(len(toRelease)))
Expand Down
3 changes: 1 addition & 2 deletions internal/querycoordv2/job/undo.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,10 +78,9 @@ func (u *UndoList) RollBack() {

if u.IsTargetUpdated {
if u.IsNewCollection {
u.targetMgr.RemoveCollection(u.CollectionID)
u.targetObserver.ReleaseCollection(u.CollectionID)
} else {
u.targetMgr.RemovePartition(u.CollectionID, u.LackPartitions...)
u.targetObserver.ReleasePartition(u.CollectionID, u.LackPartitions...)
}
}
}
6 changes: 3 additions & 3 deletions internal/querycoordv2/observers/collection_observer.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ func (ob *CollectionObserver) observeTimeout() {
zap.Duration("loadTime", time.Since(collection.CreatedAt)))
ob.meta.CollectionManager.RemoveCollection(collection.GetCollectionID())
ob.meta.ReplicaManager.RemoveCollection(collection.GetCollectionID())
ob.targetMgr.RemoveCollection(collection.GetCollectionID())
ob.targetObserver.ReleaseCollection(collection.GetCollectionID())
ob.loadTasks.Remove(traceID)
}
case querypb.LoadType_LoadPartition:
Expand Down Expand Up @@ -214,7 +214,7 @@ func (ob *CollectionObserver) observeTimeout() {
zap.Int64s("partitionIDs", task.PartitionIDs))
for _, partition := range partitions {
ob.meta.CollectionManager.RemovePartition(partition.CollectionID, partition.GetPartitionID())
ob.targetMgr.RemovePartition(partition.GetCollectionID(), partition.GetPartitionID())
ob.targetObserver.ReleasePartition(partition.GetCollectionID(), partition.GetPartitionID())
}

// all partition timeout, remove collection
Expand All @@ -223,7 +223,7 @@ func (ob *CollectionObserver) observeTimeout() {

ob.meta.CollectionManager.RemoveCollection(task.CollectionID)
ob.meta.ReplicaManager.RemoveCollection(task.CollectionID)
ob.targetMgr.RemoveCollection(task.CollectionID)
ob.targetObserver.ReleaseCollection(task.CollectionID)
}
}
}
Expand Down
97 changes: 75 additions & 22 deletions internal/querycoordv2/observers/target_observer.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,15 +38,33 @@ import (
"github.com/milvus-io/milvus/pkg/util/typeutil"
)

type checkRequest struct {
CollectionID int64
Notifier chan bool
type targetOp int

func (op *targetOp) String() string {
switch *op {
case UpdateCollection:
return "UpdateCollection"
case ReleaseCollection:
return "ReleaseCollection"
case ReleasePartition:
return "ReleasePartition"
default:
return "Unknown"
}
}

const (
UpdateCollection targetOp = iota + 1
ReleaseCollection
ReleasePartition
)

type targetUpdateRequest struct {
CollectionID int64
PartitionIDs []int64
Notifier chan error
ReadyNotifier chan struct{}
opType targetOp
}

type initRequest struct{}
Expand All @@ -60,8 +78,7 @@ type TargetObserver struct {
broker meta.Broker
cluster session.Cluster

initChan chan initRequest
manualCheck chan checkRequest
initChan chan initRequest
// nextTargetLastUpdate map[int64]time.Time
nextTargetLastUpdate *typeutil.ConcurrentMap[int64, time.Time]
updateChan chan targetUpdateRequest
Expand All @@ -88,7 +105,6 @@ func NewTargetObserver(
distMgr: distMgr,
broker: broker,
cluster: cluster,
manualCheck: make(chan checkRequest, 10),
nextTargetLastUpdate: typeutil.NewConcurrentMap[int64, time.Time](),
updateChan: make(chan targetUpdateRequest),
readyNotifiers: make(map[int64][]chan struct{}),
Expand Down Expand Up @@ -152,23 +168,37 @@ func (ob *TargetObserver) schedule(ctx context.Context) {
ob.dispatcher.AddTask(ob.meta.GetAll()...)

case req := <-ob.updateChan:
log := log.With(zap.Int64("collectionID", req.CollectionID))
log.Info("manually trigger update next target")
ob.keylocks.Lock(req.CollectionID)
err := ob.updateNextTarget(req.CollectionID)
ob.keylocks.Unlock(req.CollectionID)
if err != nil {
log.Warn("failed to manually update next target", zap.Error(err))
close(req.ReadyNotifier)
} else {
ob.mut.Lock()
ob.readyNotifiers[req.CollectionID] = append(ob.readyNotifiers[req.CollectionID], req.ReadyNotifier)
ob.mut.Unlock()
log.Info("manually trigger update target",
zap.Int64("collectionID", req.CollectionID),
zap.String("opType", req.opType.String()),
)
switch req.opType {
case UpdateCollection:
ob.keylocks.Lock(req.CollectionID)
err := ob.updateNextTarget(req.CollectionID)
ob.keylocks.Unlock(req.CollectionID)
if err != nil {
log.Warn("failed to manually update next target",
zap.Int64("collectionID", req.CollectionID),
zap.String("opType", req.opType.String()),
zap.Error(err))
close(req.ReadyNotifier)
} else {
ob.mut.Lock()
ob.readyNotifiers[req.CollectionID] = append(ob.readyNotifiers[req.CollectionID], req.ReadyNotifier)
ob.mut.Unlock()
}
req.Notifier <- err
case ReleaseCollection:
ob.targetMgr.RemoveCollection(req.CollectionID)
req.Notifier <- nil
case ReleasePartition:
ob.targetMgr.RemovePartition(req.CollectionID, req.PartitionIDs...)
req.Notifier <- nil
}

log.Info("manually trigger update target done")
req.Notifier <- err
log.Info("notify manually trigger update target done")
log.Info("manually trigger update target done",
zap.Int64("collectionID", req.CollectionID),
zap.String("opType", req.opType.String()))
}
}
}
Expand Down Expand Up @@ -229,6 +259,7 @@ func (ob *TargetObserver) UpdateNextTarget(collectionID int64) (chan struct{}, e

ob.updateChan <- targetUpdateRequest{
CollectionID: collectionID,
opType: UpdateCollection,
Notifier: notifier,
ReadyNotifier: readyCh,
}
Expand All @@ -242,6 +273,28 @@ func (ob *TargetObserver) ReleaseCollection(collectionID int64) {
close(notifier)
}
delete(ob.readyNotifiers, collectionID)

notifier := make(chan error)
ob.updateChan <- targetUpdateRequest{
CollectionID: collectionID,
opType: ReleaseCollection,
Notifier: notifier,
}
<-notifier
}

func (ob *TargetObserver) ReleasePartition(collectionID int64, partitionID ...int64) {
ob.mut.Lock()
defer ob.mut.Unlock()

notifier := make(chan error)
ob.updateChan <- targetUpdateRequest{
CollectionID: collectionID,
PartitionIDs: partitionID,
opType: ReleasePartition,
Notifier: notifier,
}
<-notifier
}

func (ob *TargetObserver) clean() {
Expand Down
15 changes: 15 additions & 0 deletions internal/querycoordv2/observers/target_observer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"testing"
"time"

"github.com/samber/lo"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"

Expand Down Expand Up @@ -214,6 +215,20 @@ func (suite *TargetObserverSuite) TestTriggerUpdateTarget() {
}, 7*time.Second, 1*time.Second)
}

func (suite *TargetObserverSuite) TestTriggerRelease() {
// Manually update next target
_, err := suite.observer.UpdateNextTarget(suite.collectionID)
suite.NoError(err)

// manually release partition
partitions := suite.meta.CollectionManager.GetPartitionsByCollection(suite.collectionID)
partitionIDs := lo.Map(partitions, func(partition *meta.Partition, _ int) int64 { return partition.PartitionID })
suite.observer.ReleasePartition(suite.collectionID, partitionIDs[0])

// manually release collection
suite.observer.ReleaseCollection(suite.collectionID)
}

func (suite *TargetObserverSuite) TearDownTest() {
suite.kv.Close()
suite.observer.Stop()
Expand Down

0 comments on commit 7ee41a8

Please sign in to comment.