2022-10-11 11:39:22 +08:00
|
|
|
// Licensed to the LF AI & Data foundation under one
|
|
|
|
// or more contributor license agreements. See the NOTICE file
|
|
|
|
// distributed with this work for additional information
|
|
|
|
// regarding copyright ownership. The ASF licenses this file
|
|
|
|
// to you under the Apache License, Version 2.0 (the
|
|
|
|
// "License"); you may not use this file except in compliance
|
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
//
|
|
|
|
// Unless required by applicable law or agreed to in writing, software
|
|
|
|
// distributed under the License is distributed on an "AS IS" BASIS,
|
|
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
|
|
// See the License for the specific language governing permissions and
|
|
|
|
// limitations under the License.
|
|
|
|
|
2022-09-15 18:48:32 +08:00
|
|
|
package observers
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
|
|
|
"sync"
|
|
|
|
"time"
|
|
|
|
|
2022-10-16 20:49:27 +08:00
|
|
|
"github.com/milvus-io/milvus-proto/go-api/commonpb"
|
2022-09-15 18:48:32 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/querypb"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/meta"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/session"
|
|
|
|
"github.com/milvus-io/milvus/internal/querycoordv2/utils"
|
|
|
|
"go.uber.org/zap"
|
|
|
|
)
|
|
|
|
|
|
|
|
const interval = 1 * time.Second
|
|
|
|
|
|
|
|
// LeaderObserver is to sync the distribution with leader
|
|
|
|
type LeaderObserver struct {
|
|
|
|
wg sync.WaitGroup
|
|
|
|
closeCh chan struct{}
|
|
|
|
dist *meta.DistributionManager
|
|
|
|
meta *meta.Meta
|
|
|
|
target *meta.TargetManager
|
|
|
|
cluster session.Cluster
|
|
|
|
}
|
|
|
|
|
|
|
|
func (o *LeaderObserver) Start(ctx context.Context) {
|
|
|
|
o.wg.Add(1)
|
|
|
|
go func() {
|
|
|
|
defer o.wg.Done()
|
|
|
|
ticker := time.NewTicker(interval)
|
|
|
|
for {
|
|
|
|
select {
|
|
|
|
case <-o.closeCh:
|
|
|
|
log.Info("stop leader observer")
|
|
|
|
return
|
|
|
|
case <-ctx.Done():
|
|
|
|
log.Info("stop leader observer due to ctx done")
|
|
|
|
return
|
|
|
|
case <-ticker.C:
|
|
|
|
o.observe(ctx)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (o *LeaderObserver) Stop() {
|
|
|
|
close(o.closeCh)
|
|
|
|
o.wg.Wait()
|
|
|
|
}
|
|
|
|
|
|
|
|
func (o *LeaderObserver) observe(ctx context.Context) {
|
|
|
|
o.observeSegmentsDist(ctx)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (o *LeaderObserver) observeSegmentsDist(ctx context.Context) {
|
|
|
|
collectionIDs := o.meta.CollectionManager.GetAll()
|
|
|
|
for _, cid := range collectionIDs {
|
|
|
|
o.observeCollection(ctx, cid)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (o *LeaderObserver) observeCollection(ctx context.Context, collection int64) {
|
|
|
|
replicas := o.meta.ReplicaManager.GetByCollection(collection)
|
|
|
|
for _, replica := range replicas {
|
|
|
|
leaders := o.dist.ChannelDistManager.GetShardLeadersByReplica(replica)
|
|
|
|
for ch, leaderID := range leaders {
|
|
|
|
leaderView := o.dist.LeaderViewManager.GetLeaderShardView(leaderID, ch)
|
|
|
|
if leaderView == nil {
|
|
|
|
continue
|
|
|
|
}
|
2022-09-23 15:16:51 +08:00
|
|
|
dists := o.dist.SegmentDistManager.GetByShardWithReplica(ch, replica)
|
2022-09-15 18:48:32 +08:00
|
|
|
needLoaded, needRemoved := o.findNeedLoadedSegments(leaderView, dists),
|
|
|
|
o.findNeedRemovedSegments(leaderView, dists)
|
|
|
|
o.sync(ctx, leaderView, append(needLoaded, needRemoved...))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (o *LeaderObserver) findNeedLoadedSegments(leaderView *meta.LeaderView, dists []*meta.Segment) []*querypb.SyncAction {
|
|
|
|
ret := make([]*querypb.SyncAction, 0)
|
|
|
|
dists = utils.FindMaxVersionSegments(dists)
|
|
|
|
for _, s := range dists {
|
2022-09-28 12:10:54 +08:00
|
|
|
version, ok := leaderView.Segments[s.GetID()]
|
|
|
|
if ok && version.GetVersion() >= s.Version ||
|
|
|
|
!o.target.ContainSegment(s.GetID()) {
|
2022-09-15 18:48:32 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
ret = append(ret, &querypb.SyncAction{
|
|
|
|
Type: querypb.SyncType_Set,
|
|
|
|
PartitionID: s.GetPartitionID(),
|
|
|
|
SegmentID: s.GetID(),
|
|
|
|
NodeID: s.Node,
|
2022-09-28 12:10:54 +08:00
|
|
|
Version: s.Version,
|
2022-09-15 18:48:32 +08:00
|
|
|
})
|
|
|
|
}
|
|
|
|
return ret
|
|
|
|
}
|
|
|
|
|
|
|
|
func (o *LeaderObserver) findNeedRemovedSegments(leaderView *meta.LeaderView, dists []*meta.Segment) []*querypb.SyncAction {
|
|
|
|
ret := make([]*querypb.SyncAction, 0)
|
|
|
|
distMap := make(map[int64]struct{})
|
|
|
|
for _, s := range dists {
|
|
|
|
distMap[s.GetID()] = struct{}{}
|
|
|
|
}
|
|
|
|
for sid := range leaderView.Segments {
|
|
|
|
_, ok := distMap[sid]
|
|
|
|
if ok || o.target.ContainSegment(sid) {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
ret = append(ret, &querypb.SyncAction{
|
|
|
|
Type: querypb.SyncType_Remove,
|
|
|
|
SegmentID: sid,
|
|
|
|
})
|
|
|
|
}
|
|
|
|
return ret
|
|
|
|
}
|
|
|
|
|
|
|
|
func (o *LeaderObserver) sync(ctx context.Context, leaderView *meta.LeaderView, diffs []*querypb.SyncAction) {
|
2022-09-28 12:10:54 +08:00
|
|
|
if len(diffs) == 0 {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2022-09-15 18:48:32 +08:00
|
|
|
log := log.With(
|
|
|
|
zap.Int64("leaderID", leaderView.ID),
|
|
|
|
zap.Int64("collectionID", leaderView.CollectionID),
|
|
|
|
zap.String("channel", leaderView.Channel),
|
|
|
|
)
|
|
|
|
req := &querypb.SyncDistributionRequest{
|
|
|
|
Base: &commonpb.MsgBase{
|
|
|
|
MsgType: commonpb.MsgType_SyncDistribution,
|
|
|
|
},
|
|
|
|
CollectionID: leaderView.CollectionID,
|
|
|
|
Channel: leaderView.Channel,
|
|
|
|
Actions: diffs,
|
|
|
|
}
|
|
|
|
resp, err := o.cluster.SyncDistribution(ctx, leaderView.ID, req)
|
|
|
|
if err != nil {
|
|
|
|
log.Error("failed to sync distribution", zap.Error(err))
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
if resp.ErrorCode != commonpb.ErrorCode_Success {
|
|
|
|
log.Error("failed to sync distribution", zap.String("reason", resp.GetReason()))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func NewLeaderObserver(
|
|
|
|
dist *meta.DistributionManager,
|
|
|
|
meta *meta.Meta,
|
|
|
|
targetMgr *meta.TargetManager,
|
|
|
|
cluster session.Cluster,
|
|
|
|
) *LeaderObserver {
|
|
|
|
return &LeaderObserver{
|
|
|
|
closeCh: make(chan struct{}),
|
|
|
|
dist: dist,
|
|
|
|
meta: meta,
|
|
|
|
target: targetMgr,
|
|
|
|
cluster: cluster,
|
|
|
|
}
|
|
|
|
}
|