enhance: [cherry-pick] Decouple compaction from shard (#34157)

This PR cherry-picks the following commits:

- Implement task limit control logic in datanode.
https://github.com/milvus-io/milvus/pull/32881
- Load bf from storage instead of memory during L0 compaction.
https://github.com/milvus-io/milvus/pull/32913
- Remove dependencies on shards (e.g. SyncSegments, injection).
https://github.com/milvus-io/milvus/pull/33138
- Rename Compaction interface to CompactionV2.
https://github.com/milvus-io/milvus/pull/33858
- Remove the unused residual compaction logic.
https://github.com/milvus-io/milvus/pull/33932

issue: https://github.com/milvus-io/milvus/issues/32809

pr: https://github.com/milvus-io/milvus/pull/32881,
https://github.com/milvus-io/milvus/pull/32913,
https://github.com/milvus-io/milvus/pull/33138,
https://github.com/milvus-io/milvus/pull/33858,
https://github.com/milvus-io/milvus/pull/33932

---------

Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
pull/34165/head
yihao.dai 2024-06-25 20:22:03 +08:00 committed by GitHub
parent 2ecfe1599c
commit b1e74dc7cb
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
64 changed files with 1718 additions and 1588 deletions

View File

@ -538,6 +538,8 @@ dataNode:
serverMaxRecvSize: 268435456
clientMaxSendSize: 268435456
clientMaxRecvSize: 536870912
slot:
slotCap: 2 # The maximum number of tasks(e.g. compaction, importing) allowed to run concurrently on a datanode.
# Configures the system log output.
log:

2
go.sum
View File

@ -290,6 +290,7 @@ github.com/go-latex/latex v0.0.0-20210118124228-b3d85cf34e07/go.mod h1:CO1AlKB2C
github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE=
github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk=
github.com/go-logfmt/logfmt v0.5.0/go.mod h1:wCYkCAKZfumFQihp8CzCvQ3paCTfi41vtzG1KdI/P7A=
github.com/go-logfmt/logfmt v0.5.1/go.mod h1:WYhtIu8zTZfxdn5+rREduYbwxfcBr/Vr6KEVveWlfTs=
github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A=
github.com/go-logr/logr v1.3.0 h1:2y3SDp0ZXuc6/cjLSZ+Q3ir+QB9T/iG5yYRXqsagWSY=
github.com/go-logr/logr v1.3.0/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY=
@ -516,6 +517,7 @@ github.com/kataras/iris/v12 v12.1.8/go.mod h1:LMYy4VlP67TQ3Zgriz8RE2h2kMZV2SgMYb
github.com/kataras/neffos v0.0.14/go.mod h1:8lqADm8PnbeFfL7CLXh1WHw53dG27MC3pgi2R1rmoTE=
github.com/kataras/pio v0.0.2/go.mod h1:hAoW0t9UmXi4R5Oyq5Z4irTbaTsOemSrDGUtaTl7Dro=
github.com/kataras/sitemap v0.0.5/go.mod h1:KY2eugMKiPwsJgx7+U103YZehfvNGOXURubcGyk0Bz8=
github.com/keybase/go-keychain v0.0.0-20190712205309-48d3d31d256d/go.mod h1:JJNrCn9otv/2QP4D7SMJBgaleKpOf66PnW6F5WGNRIc=
github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q=
github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00=
github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8=

View File

@ -19,6 +19,7 @@ package datacoord
import (
"context"
"fmt"
"sync"
"github.com/samber/lo"
"go.uber.org/zap"
@ -31,6 +32,8 @@ import (
)
// Cluster provides interfaces to interact with datanode cluster
//
//go:generate mockery --name=Cluster --structname=MockCluster --output=./ --filename=mock_cluster.go --with-expecter --inpackage
type Cluster interface {
Startup(ctx context.Context, nodes []*NodeInfo) error
Register(node *NodeInfo) error
@ -43,6 +46,7 @@ type Cluster interface {
QueryPreImport(nodeID int64, in *datapb.QueryPreImportRequest) (*datapb.QueryPreImportResponse, error)
QueryImport(nodeID int64, in *datapb.QueryImportRequest) (*datapb.QueryImportResponse, error)
DropImport(nodeID int64, in *datapb.DropImportRequest) error
QuerySlots() map[int64]int64
GetSessions() []*Session
Close()
}
@ -175,6 +179,30 @@ func (c *ClusterImpl) DropImport(nodeID int64, in *datapb.DropImportRequest) err
return c.sessionManager.DropImport(nodeID, in)
}
func (c *ClusterImpl) QuerySlots() map[int64]int64 {
nodeIDs := c.sessionManager.GetSessionIDs()
nodeSlots := make(map[int64]int64)
mu := &sync.Mutex{}
wg := &sync.WaitGroup{}
for _, nodeID := range nodeIDs {
wg.Add(1)
go func(nodeID int64) {
defer wg.Done()
resp, err := c.sessionManager.QuerySlot(nodeID)
if err != nil {
log.Warn("query slot failed", zap.Int64("nodeID", nodeID), zap.Error(err))
return
}
mu.Lock()
defer mu.Unlock()
nodeSlots[nodeID] = resp.GetNumSlots()
}(nodeID)
}
wg.Wait()
log.Debug("query slot done", zap.Any("nodeSlots", nodeSlots))
return nodeSlots
}
// GetSessions returns all sessions
func (c *ClusterImpl) GetSessions() []*Session {
return c.sessionManager.GetSessions()

View File

@ -20,6 +20,7 @@ import (
"context"
"testing"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/require"
@ -28,6 +29,7 @@ import (
"github.com/milvus-io/milvus/internal/kv"
etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
"github.com/milvus-io/milvus/internal/kv/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/util/testutils"
)
@ -175,3 +177,29 @@ func (suite *ClusterSuite) TestFlushChannels() {
suite.NoError(err)
})
}
func (suite *ClusterSuite) TestQuerySlot() {
suite.Run("query slot failed", func() {
suite.SetupTest()
suite.mockSession.EXPECT().GetSessionIDs().Return([]int64{1}).Once()
suite.mockSession.EXPECT().QuerySlot(int64(1)).Return(nil, errors.New("mock err")).Once()
cluster := NewClusterImpl(suite.mockSession, suite.mockChManager)
nodeSlots := cluster.QuerySlots()
suite.Equal(0, len(nodeSlots))
})
suite.Run("normal", func() {
suite.SetupTest()
suite.mockSession.EXPECT().GetSessionIDs().Return([]int64{1, 2, 3, 4}).Once()
suite.mockSession.EXPECT().QuerySlot(int64(1)).Return(&datapb.QuerySlotResponse{NumSlots: 1}, nil).Once()
suite.mockSession.EXPECT().QuerySlot(int64(2)).Return(&datapb.QuerySlotResponse{NumSlots: 2}, nil).Once()
suite.mockSession.EXPECT().QuerySlot(int64(3)).Return(&datapb.QuerySlotResponse{NumSlots: 3}, nil).Once()
suite.mockSession.EXPECT().QuerySlot(int64(4)).Return(&datapb.QuerySlotResponse{NumSlots: 4}, nil).Once()
cluster := NewClusterImpl(suite.mockSession, suite.mockChManager)
nodeSlots := cluster.QuerySlots()
suite.Equal(int64(1), nodeSlots[1])
suite.Equal(int64(2), nodeSlots[2])
suite.Equal(int64(3), nodeSlots[3])
suite.Equal(int64(4), nodeSlots[4])
})
}

View File

@ -44,11 +44,12 @@ const (
tsTimeout = uint64(1)
)
//go:generate mockery --name=compactionPlanContext --structname=MockCompactionPlanContext --output=./ --filename=mock_compaction_plan_context.go --with-expecter --inpackage
type compactionPlanContext interface {
start()
stop()
// execCompactionPlan start to execute plan and return immediately
execCompactionPlan(signal *compactionSignal, plan *datapb.CompactionPlan) error
execCompactionPlan(signal *compactionSignal, plan *datapb.CompactionPlan)
// getCompaction return compaction task. If planId does not exist, return nil.
getCompaction(planID int64) *compactionTask
// updateCompaction set the compaction state to timeout or completed
@ -126,7 +127,7 @@ type compactionPlanHandler struct {
stopWg sync.WaitGroup
}
func newCompactionPlanHandler(sessions SessionManager, cm ChannelManager, meta CompactionMeta, allocator allocator,
func newCompactionPlanHandler(cluster Cluster, sessions SessionManager, cm ChannelManager, meta CompactionMeta, allocator allocator,
) *compactionPlanHandler {
return &compactionPlanHandler{
plans: make(map[int64]*compactionTask),
@ -134,7 +135,7 @@ func newCompactionPlanHandler(sessions SessionManager, cm ChannelManager, meta C
meta: meta,
sessions: sessions,
allocator: allocator,
scheduler: NewCompactionScheduler(),
scheduler: NewCompactionScheduler(cluster),
}
}
@ -198,7 +199,7 @@ func (c *compactionPlanHandler) start() {
// influence the schedule
go func() {
defer c.stopWg.Done()
scheduleTicker := time.NewTicker(200 * time.Millisecond)
scheduleTicker := time.NewTicker(2 * time.Second)
defer scheduleTicker.Stop()
log.Info("compaction handler start schedule")
for {
@ -276,14 +277,8 @@ func (c *compactionPlanHandler) updateTask(planID int64, opts ...compactionTaskO
}
}
func (c *compactionPlanHandler) enqueuePlan(signal *compactionSignal, plan *datapb.CompactionPlan) error {
nodeID, err := c.chManager.FindWatcher(plan.GetChannel())
if err != nil {
log.Error("failed to find watcher", zap.Int64("planID", plan.GetPlanID()), zap.Error(err))
return err
}
log := log.With(zap.Int64("planID", plan.GetPlanID()), zap.Int64("nodeID", nodeID))
func (c *compactionPlanHandler) enqueuePlan(signal *compactionSignal, plan *datapb.CompactionPlan) {
log := log.With(zap.Int64("planID", plan.GetPlanID()))
c.setSegmentsCompacting(plan, true)
_, span := otel.Tracer(typeutil.DataCoordRole).Start(context.Background(), fmt.Sprintf("Compaction-%s", plan.GetType()))
@ -292,7 +287,6 @@ func (c *compactionPlanHandler) enqueuePlan(signal *compactionSignal, plan *data
triggerInfo: signal,
plan: plan,
state: pipelining,
dataNodeID: nodeID,
span: span,
}
c.mu.Lock()
@ -300,8 +294,7 @@ func (c *compactionPlanHandler) enqueuePlan(signal *compactionSignal, plan *data
c.mu.Unlock()
c.scheduler.Submit(task)
log.Info("Compaction plan submited")
return nil
log.Info("Compaction plan submitted")
}
func (c *compactionPlanHandler) RefreshPlan(task *compactionTask) error {
@ -337,10 +330,14 @@ func (c *compactionPlanHandler) RefreshPlan(task *compactionTask) error {
sealedSegBinlogs := lo.Map(sealedSegments, func(info *SegmentInfo, _ int) *datapb.CompactionSegmentBinlogs {
return &datapb.CompactionSegmentBinlogs{
SegmentID: info.GetID(),
Level: datapb.SegmentLevel_L1,
CollectionID: info.GetCollectionID(),
PartitionID: info.GetPartitionID(),
SegmentID: info.GetID(),
FieldBinlogs: nil,
Field2StatslogPaths: info.GetStatslogs(),
Deltalogs: nil,
InsertChannel: info.GetInsertChannel(),
Level: info.GetLevel(),
CollectionID: info.GetCollectionID(),
PartitionID: info.GetPartitionID(),
}
})
@ -407,8 +404,8 @@ func (c *compactionPlanHandler) notifyTasks(tasks []*compactionTask) {
}
// execCompactionPlan start to execute plan and return immediately
func (c *compactionPlanHandler) execCompactionPlan(signal *compactionSignal, plan *datapb.CompactionPlan) error {
return c.enqueuePlan(signal, plan)
func (c *compactionPlanHandler) execCompactionPlan(signal *compactionSignal, plan *datapb.CompactionPlan) {
c.enqueuePlan(signal, plan)
}
func (c *compactionPlanHandler) setSegmentsCompacting(plan *datapb.CompactionPlan, compacting bool) {
@ -483,25 +480,17 @@ func (c *compactionPlanHandler) handleMergeCompactionResult(plan *datapb.Compact
log.Info("meta has already been changed, skip meta change and retry sync segments")
} else {
// Also prepare metric updates.
newSegments, metricMutation, err := c.meta.CompleteCompactionMutation(plan, result)
_, metricMutation, err := c.meta.CompleteCompactionMutation(plan, result)
if err != nil {
return err
}
// Apply metrics after successful meta update.
metricMutation.commit()
newSegmentInfo = newSegments[0]
}
nodeID := c.plans[plan.GetPlanID()].dataNodeID
req := &datapb.SyncSegmentsRequest{
PlanID: plan.PlanID,
CompactedTo: newSegmentInfo.GetID(),
CompactedFrom: newSegmentInfo.GetCompactionFrom(),
NumOfRows: newSegmentInfo.GetNumOfRows(),
StatsLogs: newSegmentInfo.GetStatslogs(),
ChannelName: plan.GetChannel(),
PartitionId: newSegmentInfo.GetPartitionID(),
CollectionId: newSegmentInfo.GetCollectionID(),
PlanID: plan.PlanID,
}
log.Info("handleCompactionResult: syncing segments with node", zap.Int64("nodeID", nodeID))
@ -632,8 +621,7 @@ func (c *compactionPlanHandler) updateCompaction(ts Timestamp) error {
// without changing the meta
log.Info("compaction syncing unknown plan with node")
if err := c.sessions.SyncSegments(nodeID, &datapb.SyncSegmentsRequest{
PlanID: planID,
ChannelName: plan.GetChannel(),
PlanID: planID,
}); err != nil {
log.Warn("compaction failed to sync segments with node", zap.Error(err))
return err

View File

@ -35,15 +35,17 @@ type CompactionScheduler struct {
taskGuard sync.RWMutex
planHandler *compactionPlanHandler
cluster Cluster
}
var _ Scheduler = (*CompactionScheduler)(nil)
func NewCompactionScheduler() *CompactionScheduler {
func NewCompactionScheduler(cluster Cluster) *CompactionScheduler {
return &CompactionScheduler{
taskNumber: atomic.NewInt32(0),
queuingTasks: make([]*compactionTask, 0),
parallelTasks: make(map[int64][]*compactionTask),
cluster: cluster,
}
}
@ -62,71 +64,64 @@ func (s *CompactionScheduler) Submit(tasks ...*compactionTask) {
// Schedule pick 1 or 0 tasks for 1 node
func (s *CompactionScheduler) Schedule() []*compactionTask {
nodeTasks := make(map[int64][]*compactionTask) // nodeID
s.taskGuard.RLock()
if len(s.queuingTasks) == 0 {
s.taskGuard.RUnlock()
return nil // To mitigate the need for frequent slot querying
}
s.taskGuard.RUnlock()
nodeSlots := s.cluster.QuerySlots()
l0ChannelExcludes := typeutil.NewSet[string]()
mixChannelExcludes := typeutil.NewSet[string]()
for _, tasks := range s.parallelTasks {
for _, t := range tasks {
switch t.plan.GetType() {
case datapb.CompactionType_Level0DeleteCompaction:
l0ChannelExcludes.Insert(t.plan.GetChannel())
case datapb.CompactionType_MixCompaction:
mixChannelExcludes.Insert(t.plan.GetChannel())
}
}
}
s.taskGuard.Lock()
defer s.taskGuard.Unlock()
for _, task := range s.queuingTasks {
if _, ok := nodeTasks[task.dataNodeID]; !ok {
nodeTasks[task.dataNodeID] = make([]*compactionTask, 0)
}
nodeTasks[task.dataNodeID] = append(nodeTasks[task.dataNodeID], task)
}
executable := make(map[int64]*compactionTask)
pickPriorPolicy := func(tasks []*compactionTask, exclusiveChannels []string, executing []string) *compactionTask {
for _, task := range tasks {
if lo.Contains(exclusiveChannels, task.plan.GetChannel()) {
continue
}
if task.plan.GetType() == datapb.CompactionType_Level0DeleteCompaction {
// Channel of LevelZeroCompaction task with no executing compactions
if !lo.Contains(executing, task.plan.GetChannel()) {
return task
}
// Don't schedule any tasks for channel with LevelZeroCompaction task
// when there're executing compactions
exclusiveChannels = append(exclusiveChannels, task.plan.GetChannel())
continue
}
return task
}
return nil
}
// pick 1 or 0 task for 1 node
for node, tasks := range nodeTasks {
parallel := s.parallelTasks[node]
if len(parallel) >= calculateParallel() {
log.Info("Compaction parallel in DataNode reaches the limit", zap.Int64("nodeID", node), zap.Int("parallel", len(parallel)))
picked := make([]*compactionTask, 0)
for _, t := range s.queuingTasks {
nodeID := s.pickAnyNode(nodeSlots)
if nodeID == NullNodeID {
log.Warn("cannot find datanode for compaction task",
zap.Int64("planID", t.plan.PlanID), zap.String("vchannel", t.plan.Channel))
continue
}
var (
executing = typeutil.NewSet[string]()
channelsExecPrior = typeutil.NewSet[string]()
)
for _, t := range parallel {
executing.Insert(t.plan.GetChannel())
if t.plan.GetType() == datapb.CompactionType_Level0DeleteCompaction {
channelsExecPrior.Insert(t.plan.GetChannel())
switch t.plan.GetType() {
case datapb.CompactionType_Level0DeleteCompaction:
if l0ChannelExcludes.Contain(t.plan.GetChannel()) ||
mixChannelExcludes.Contain(t.plan.GetChannel()) {
continue
}
}
picked := pickPriorPolicy(tasks, channelsExecPrior.Collect(), executing.Collect())
if picked != nil {
executable[node] = picked
t.dataNodeID = nodeID
picked = append(picked, t)
l0ChannelExcludes.Insert(t.plan.GetChannel())
nodeSlots[nodeID]--
case datapb.CompactionType_MixCompaction:
if l0ChannelExcludes.Contain(t.plan.GetChannel()) {
continue
}
t.dataNodeID = nodeID
picked = append(picked, t)
mixChannelExcludes.Insert(t.plan.GetChannel())
nodeSlots[nodeID]--
}
}
var pickPlans []int64
for node, task := range executable {
for _, task := range picked {
node := task.dataNodeID
pickPlans = append(pickPlans, task.plan.PlanID)
if _, ok := s.parallelTasks[node]; !ok {
s.parallelTasks[node] = []*compactionTask{task}
@ -150,7 +145,7 @@ func (s *CompactionScheduler) Schedule() []*compactionTask {
}
}
return lo.Values(executable)
return picked
}
func (s *CompactionScheduler) Finish(nodeID UniqueID, plan *datapb.CompactionPlan) {
@ -211,3 +206,24 @@ func (s *CompactionScheduler) LogStatus() {
func (s *CompactionScheduler) GetTaskCount() int {
return int(s.taskNumber.Load())
}
func (s *CompactionScheduler) pickAnyNode(nodeSlots map[int64]int64) int64 {
var (
nodeID int64 = NullNodeID
maxSlots int64 = -1
)
for id, slots := range nodeSlots {
if slots > 0 && slots > maxSlots {
nodeID = id
maxSlots = slots
}
}
return nodeID
}
func (s *CompactionScheduler) pickShardNode(nodeID int64, nodeSlots map[int64]int64) int64 {
if nodeSlots[nodeID] > 0 {
return nodeID
}
return NullNodeID
}

View File

@ -22,7 +22,8 @@ type SchedulerSuite struct {
}
func (s *SchedulerSuite) SetupTest() {
s.scheduler = NewCompactionScheduler()
cluster := NewMockCluster(s.T())
s.scheduler = NewCompactionScheduler(cluster)
s.scheduler.parallelTasks = map[int64][]*compactionTask{
100: {
{dataNodeID: 100, plan: &datapb.CompactionPlan{PlanID: 1, Channel: "ch-1", Type: datapb.CompactionType_MixCompaction}},
@ -39,7 +40,8 @@ func (s *SchedulerSuite) SetupTest() {
}
func (s *SchedulerSuite) TestScheduleEmpty() {
emptySch := NewCompactionScheduler()
cluster := NewMockCluster(s.T())
emptySch := NewCompactionScheduler(cluster)
tasks := emptySch.Schedule()
s.Empty(tasks)
@ -58,11 +60,11 @@ func (s *SchedulerSuite) TestScheduleParallelTaskFull() {
}{
{"with L0 tasks", []*compactionTask{
{dataNodeID: 100, plan: &datapb.CompactionPlan{PlanID: 10, Channel: "ch-10", Type: datapb.CompactionType_Level0DeleteCompaction}},
{dataNodeID: 100, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MinorCompaction}},
{dataNodeID: 100, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MixCompaction}},
}, []UniqueID{}},
{"without L0 tasks", []*compactionTask{
{dataNodeID: 100, plan: &datapb.CompactionPlan{PlanID: 10, Channel: "ch-10", Type: datapb.CompactionType_MinorCompaction}},
{dataNodeID: 100, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MinorCompaction}},
{dataNodeID: 100, plan: &datapb.CompactionPlan{PlanID: 10, Channel: "ch-10", Type: datapb.CompactionType_MixCompaction}},
{dataNodeID: 100, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MixCompaction}},
}, []UniqueID{}},
{"empty tasks", []*compactionTask{}, []UniqueID{}},
}
@ -72,6 +74,12 @@ func (s *SchedulerSuite) TestScheduleParallelTaskFull() {
s.SetupTest()
s.Require().Equal(4, s.scheduler.GetTaskCount())
if len(test.tasks) > 0 {
cluster := NewMockCluster(s.T())
cluster.EXPECT().QuerySlots().Return(map[int64]int64{100: 0})
s.scheduler.cluster = cluster
}
// submit the testing tasks
s.scheduler.Submit(test.tasks...)
s.Equal(4+len(test.tasks), s.scheduler.GetTaskCount())
@ -93,16 +101,16 @@ func (s *SchedulerSuite) TestScheduleNodeWith1ParallelTask() {
}{
{"with L0 tasks diff channel", []*compactionTask{
{dataNodeID: 101, plan: &datapb.CompactionPlan{PlanID: 10, Channel: "ch-10", Type: datapb.CompactionType_Level0DeleteCompaction}},
{dataNodeID: 101, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MinorCompaction}},
}, []UniqueID{10}},
{dataNodeID: 101, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MixCompaction}},
}, []UniqueID{10, 11}},
{"with L0 tasks same channel", []*compactionTask{
{dataNodeID: 101, plan: &datapb.CompactionPlan{PlanID: 10, Channel: "ch-2", Type: datapb.CompactionType_Level0DeleteCompaction}},
{dataNodeID: 101, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MinorCompaction}},
{dataNodeID: 101, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MixCompaction}},
}, []UniqueID{11}},
{"without L0 tasks", []*compactionTask{
{dataNodeID: 101, plan: &datapb.CompactionPlan{PlanID: 14, Channel: "ch-2", Type: datapb.CompactionType_MinorCompaction}},
{dataNodeID: 101, plan: &datapb.CompactionPlan{PlanID: 13, Channel: "ch-11", Type: datapb.CompactionType_MinorCompaction}},
}, []UniqueID{14}},
{dataNodeID: 101, plan: &datapb.CompactionPlan{PlanID: 14, Channel: "ch-2", Type: datapb.CompactionType_MixCompaction}},
{dataNodeID: 101, plan: &datapb.CompactionPlan{PlanID: 13, Channel: "ch-11", Type: datapb.CompactionType_MixCompaction}},
}, []UniqueID{14, 13}},
{"empty tasks", []*compactionTask{}, []UniqueID{}},
}
@ -111,6 +119,12 @@ func (s *SchedulerSuite) TestScheduleNodeWith1ParallelTask() {
s.SetupTest()
s.Require().Equal(4, s.scheduler.GetTaskCount())
if len(test.tasks) > 0 {
cluster := NewMockCluster(s.T())
cluster.EXPECT().QuerySlots().Return(map[int64]int64{101: 2})
s.scheduler.cluster = cluster
}
// submit the testing tasks
s.scheduler.Submit(test.tasks...)
s.Equal(4+len(test.tasks), s.scheduler.GetTaskCount())
@ -120,10 +134,6 @@ func (s *SchedulerSuite) TestScheduleNodeWith1ParallelTask() {
return t.plan.PlanID
}))
// the second schedule returns empty for full paralleTasks
gotTasks = s.scheduler.Schedule()
s.Empty(gotTasks)
s.Equal(4+len(test.tasks), s.scheduler.GetTaskCount())
})
}
@ -139,16 +149,16 @@ func (s *SchedulerSuite) TestScheduleNodeWithL0Executing() {
}{
{"with L0 tasks diff channel", []*compactionTask{
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 10, Channel: "ch-10", Type: datapb.CompactionType_Level0DeleteCompaction}},
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MinorCompaction}},
}, []UniqueID{10}},
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MixCompaction}},
}, []UniqueID{10, 11}},
{"with L0 tasks same channel", []*compactionTask{
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 10, Channel: "ch-3", Type: datapb.CompactionType_Level0DeleteCompaction}},
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MinorCompaction}},
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 13, Channel: "ch-3", Type: datapb.CompactionType_MinorCompaction}},
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 11, Channel: "ch-11", Type: datapb.CompactionType_MixCompaction}},
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 13, Channel: "ch-3", Type: datapb.CompactionType_MixCompaction}},
}, []UniqueID{11}},
{"without L0 tasks", []*compactionTask{
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 14, Channel: "ch-3", Type: datapb.CompactionType_MinorCompaction}},
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 13, Channel: "ch-11", Type: datapb.CompactionType_MinorCompaction}},
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 14, Channel: "ch-3", Type: datapb.CompactionType_MixCompaction}},
{dataNodeID: 102, plan: &datapb.CompactionPlan{PlanID: 13, Channel: "ch-11", Type: datapb.CompactionType_MixCompaction}},
}, []UniqueID{13}},
{"empty tasks", []*compactionTask{}, []UniqueID{}},
}
@ -158,6 +168,12 @@ func (s *SchedulerSuite) TestScheduleNodeWithL0Executing() {
s.SetupTest()
s.Require().Equal(4, s.scheduler.GetTaskCount())
if len(test.tasks) > 0 {
cluster := NewMockCluster(s.T())
cluster.EXPECT().QuerySlots().Return(map[int64]int64{102: 2})
s.scheduler.cluster = cluster
}
// submit the testing tasks
s.scheduler.Submit(test.tasks...)
s.Equal(4+len(test.tasks), s.scheduler.GetTaskCount())
@ -167,12 +183,6 @@ func (s *SchedulerSuite) TestScheduleNodeWithL0Executing() {
return t.plan.PlanID
}))
// the second schedule returns empty for full paralleTasks
if len(gotTasks) > 0 {
gotTasks = s.scheduler.Schedule()
s.Empty(gotTasks)
}
s.Equal(4+len(test.tasks), s.scheduler.GetTaskCount())
})
}
@ -215,3 +225,17 @@ func (s *SchedulerSuite) TestFinish() {
s.MetricsEqual(taskNum, 1)
})
}
func (s *SchedulerSuite) TestPickNode() {
s.Run("test pickAnyNode", func() {
nodeSlots := map[int64]int64{
100: 2,
101: 6,
}
node := s.scheduler.pickAnyNode(nodeSlots)
s.Equal(int64(101), node)
node = s.scheduler.pickAnyNode(map[int64]int64{})
s.Equal(int64(NullNodeID), node)
})
}

View File

@ -58,7 +58,7 @@ func (s *CompactionPlanHandlerSuite) SetupTest() {
func (s *CompactionPlanHandlerSuite) TestRemoveTasksByChannel() {
s.mockSch.EXPECT().Finish(mock.Anything, mock.Anything).Return().Once()
handler := newCompactionPlanHandler(nil, nil, nil, nil)
handler := newCompactionPlanHandler(nil, nil, nil, nil, nil)
handler.scheduler = s.mockSch
var ch string = "ch1"
@ -88,13 +88,13 @@ func (s *CompactionPlanHandlerSuite) TestCheckResult() {
s.mockSessMgr.EXPECT().SyncSegments(int64(100), mock.Anything).Return(nil).Once()
{
s.mockAlloc.EXPECT().allocTimestamp(mock.Anything).Return(0, errors.New("mock")).Once()
handler := newCompactionPlanHandler(s.mockSessMgr, nil, nil, s.mockAlloc)
handler := newCompactionPlanHandler(nil, s.mockSessMgr, nil, nil, s.mockAlloc)
handler.checkResult()
}
{
s.mockAlloc.EXPECT().allocTimestamp(mock.Anything).Return(19530, nil).Once()
handler := newCompactionPlanHandler(s.mockSessMgr, nil, nil, s.mockAlloc)
handler := newCompactionPlanHandler(nil, s.mockSessMgr, nil, nil, s.mockAlloc)
handler.checkResult()
}
}
@ -196,7 +196,7 @@ func (s *CompactionPlanHandlerSuite) TestHandleL0CompactionResults() {
},
}
handler := newCompactionPlanHandler(nil, nil, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, nil, nil, s.mockMeta, s.mockAlloc)
err := handler.handleL0CompactionResult(plan, result)
s.NoError(err)
}
@ -259,7 +259,7 @@ func (s *CompactionPlanHandlerSuite) TestRefreshL0Plan() {
dataNodeID: 1,
}
handler := newCompactionPlanHandler(nil, nil, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, nil, nil, s.mockMeta, s.mockAlloc)
err := handler.RefreshPlan(task)
s.Require().NoError(err)
@ -294,7 +294,7 @@ func (s *CompactionPlanHandlerSuite) TestRefreshL0Plan() {
dataNodeID: 1,
}
handler := newCompactionPlanHandler(nil, nil, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, nil, nil, s.mockMeta, s.mockAlloc)
err := handler.RefreshPlan(task)
s.Error(err)
s.ErrorIs(err, merr.ErrSegmentNotFound)
@ -338,7 +338,7 @@ func (s *CompactionPlanHandlerSuite) TestRefreshL0Plan() {
dataNodeID: 1,
}
handler := newCompactionPlanHandler(nil, nil, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, nil, nil, s.mockMeta, s.mockAlloc)
err := handler.RefreshPlan(task)
s.Error(err)
})
@ -383,7 +383,7 @@ func (s *CompactionPlanHandlerSuite) TestRefreshPlanMixCompaction() {
dataNodeID: 1,
}
handler := newCompactionPlanHandler(nil, nil, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, nil, nil, s.mockMeta, s.mockAlloc)
err := handler.RefreshPlan(task)
s.Require().NoError(err)
@ -424,7 +424,7 @@ func (s *CompactionPlanHandlerSuite) TestRefreshPlanMixCompaction() {
dataNodeID: 1,
}
handler := newCompactionPlanHandler(nil, nil, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, nil, nil, s.mockMeta, s.mockAlloc)
err := handler.RefreshPlan(task)
s.Error(err)
s.ErrorIs(err, merr.ErrSegmentNotFound)
@ -432,43 +432,22 @@ func (s *CompactionPlanHandlerSuite) TestRefreshPlanMixCompaction() {
}
func (s *CompactionPlanHandlerSuite) TestExecCompactionPlan() {
s.mockCm.EXPECT().FindWatcher(mock.Anything).RunAndReturn(func(channel string) (int64, error) {
if channel == "ch-1" {
return 0, errors.Errorf("mock error for ch-1")
}
return 1, nil
}).Twice()
s.mockSch.EXPECT().Submit(mock.Anything).Return().Once()
tests := []struct {
description string
channel string
hasError bool
}{
{"channel with error", "ch-1", true},
{"channel with no error", "ch-2", false},
}
handler := newCompactionPlanHandler(s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
handler.scheduler = s.mockSch
for idx, test := range tests {
sig := &compactionSignal{id: int64(idx)}
plan := &datapb.CompactionPlan{
PlanID: int64(idx),
}
s.Run(test.description, func() {
plan.Channel = test.channel
err := handler.execCompactionPlan(sig, plan)
if test.hasError {
s.Error(err)
} else {
s.NoError(err)
}
})
sig := &compactionSignal{id: int64(1)}
plan := &datapb.CompactionPlan{
PlanID: int64(1),
}
plan.Channel = "ch-1"
handler.execCompactionPlan(sig, plan)
handler.mu.RLock()
defer handler.mu.RUnlock()
_, ok := handler.plans[int64(1)]
s.True(ok)
}
func (s *CompactionPlanHandlerSuite) TestHandleMergeCompactionResult() {
@ -483,7 +462,7 @@ func (s *CompactionPlanHandlerSuite) TestHandleMergeCompactionResult() {
s.Run("illegal nil result", func() {
s.SetupTest()
handler := newCompactionPlanHandler(s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
err := handler.handleMergeCompactionResult(nil, nil)
s.Error(err)
})
@ -499,7 +478,7 @@ func (s *CompactionPlanHandlerSuite) TestHandleMergeCompactionResult() {
}).Once()
s.mockSessMgr.EXPECT().SyncSegments(mock.Anything, mock.Anything).Return(nil).Once()
handler := newCompactionPlanHandler(s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
handler.plans[plan.PlanID] = &compactionTask{dataNodeID: 111, plan: plan}
compactionResult := &datapb.CompactionPlanResult{
@ -519,7 +498,7 @@ func (s *CompactionPlanHandlerSuite) TestHandleMergeCompactionResult() {
s.mockMeta.EXPECT().CompleteCompactionMutation(mock.Anything, mock.Anything).Return(
nil, nil, errors.New("mock error")).Once()
handler := newCompactionPlanHandler(s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
handler.plans[plan.PlanID] = &compactionTask{dataNodeID: 111, plan: plan}
compactionResult := &datapb.CompactionPlanResult{
PlanID: plan.PlanID,
@ -541,7 +520,7 @@ func (s *CompactionPlanHandlerSuite) TestHandleMergeCompactionResult() {
&segMetricMutation{}, nil).Once()
s.mockSessMgr.EXPECT().SyncSegments(mock.Anything, mock.Anything).Return(errors.New("mock error")).Once()
handler := newCompactionPlanHandler(s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
handler.plans[plan.PlanID] = &compactionTask{dataNodeID: 111, plan: plan}
compactionResult := &datapb.CompactionPlanResult{
PlanID: plan.PlanID,
@ -557,7 +536,7 @@ func (s *CompactionPlanHandlerSuite) TestHandleMergeCompactionResult() {
func (s *CompactionPlanHandlerSuite) TestCompleteCompaction() {
s.Run("test not exists compaction task", func() {
handler := newCompactionPlanHandler(nil, nil, nil, nil)
handler := newCompactionPlanHandler(nil, nil, nil, nil, nil)
err := handler.completeCompaction(&datapb.CompactionPlanResult{PlanID: 2})
s.Error(err)
})
@ -637,7 +616,7 @@ func (s *CompactionPlanHandlerSuite) TestCompleteCompaction() {
},
}
c := newCompactionPlanHandler(s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
c := newCompactionPlanHandler(nil, s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
c.scheduler = s.mockSch
c.plans = plans
@ -735,7 +714,7 @@ func (s *CompactionPlanHandlerSuite) TestUpdateCompaction() {
s.mockCm.EXPECT().Match(int64(111), "ch-1").Return(true)
s.mockCm.EXPECT().Match(int64(111), "ch-2").Return(false).Once()
handler := newCompactionPlanHandler(s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
handler := newCompactionPlanHandler(nil, s.mockSessMgr, s.mockCm, s.mockMeta, s.mockAlloc)
handler.plans = inPlans
_, ok := handler.plans[5]

View File

@ -429,23 +429,14 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
break
}
start := time.Now()
if err := fillOriginPlan(t.allocator, plan); err != nil {
if err := fillOriginPlan(coll.Schema, t.allocator, plan); err != nil {
log.Warn("failed to fill plan",
zap.Int64("collectionID", signal.collectionID),
zap.Int64s("segmentIDs", segIDs),
zap.Error(err))
continue
}
err := t.compactionHandler.execCompactionPlan(signal, plan)
if err != nil {
log.Warn("failed to execute compaction plan",
zap.Int64("collectionID", signal.collectionID),
zap.Int64("planID", plan.PlanID),
zap.Int64s("segmentIDs", segIDs),
zap.Error(err))
continue
}
t.compactionHandler.execCompactionPlan(signal, plan)
log.Info("time cost of generating global compaction",
zap.Int64("planID", plan.PlanID),
zap.Int64("time cost", time.Since(start).Milliseconds()),
@ -529,18 +520,11 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
break
}
start := time.Now()
if err := fillOriginPlan(t.allocator, plan); err != nil {
if err := fillOriginPlan(coll.Schema, t.allocator, plan); err != nil {
log.Warn("failed to fill plan", zap.Error(err))
continue
}
if err := t.compactionHandler.execCompactionPlan(signal, plan); err != nil {
log.Warn("failed to execute compaction plan",
zap.Int64("collection", signal.collectionID),
zap.Int64("planID", plan.PlanID),
zap.Int64s("segmentIDs", fetchSegIDs(plan.GetSegmentBinlogs())),
zap.Error(err))
continue
}
t.compactionHandler.execCompactionPlan(signal, plan)
log.Info("time cost of generating compaction",
zap.Int64("planID", plan.PlanID),
zap.Int64("time cost", time.Since(start).Milliseconds()),
@ -712,6 +696,7 @@ func segmentsToPlan(segments []*SegmentInfo, compactTime *compactTime) *datapb.C
}
log.Info("generate a plan for priority candidates", zap.Any("plan", plan),
zap.Int("len(segments)", len(plan.GetSegmentBinlogs())),
zap.Int64("target segment row", plan.TotalRows), zap.Int64("target segment size", size))
return plan
}

View File

@ -51,9 +51,8 @@ var _ compactionPlanContext = (*spyCompactionHandler)(nil)
func (h *spyCompactionHandler) removeTasksByChannel(channel string) {}
// execCompactionPlan start to execute plan and return immediately
func (h *spyCompactionHandler) execCompactionPlan(signal *compactionSignal, plan *datapb.CompactionPlan) error {
func (h *spyCompactionHandler) execCompactionPlan(signal *compactionSignal, plan *datapb.CompactionPlan) {
h.spyChan <- plan
return nil
}
// completeCompaction record the result of a compaction
@ -106,6 +105,22 @@ func Test_compactionTrigger_force(t *testing.T) {
vecFieldID := int64(201)
indexID := int64(1001)
schema := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
FieldID: vecFieldID,
DataType: schemapb.DataType_FloatVector,
TypeParams: []*commonpb.KeyValuePair{
{
Key: common.DimKey,
Value: "128",
},
},
},
},
}
tests := []struct {
name string
fields fields
@ -292,21 +307,8 @@ func Test_compactionTrigger_force(t *testing.T) {
},
collections: map[int64]*collectionInfo{
2: {
ID: 2,
Schema: &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
FieldID: vecFieldID,
DataType: schemapb.DataType_FloatVector,
TypeParams: []*commonpb.KeyValuePair{
{
Key: common.DimKey,
Value: "128",
},
},
},
},
},
ID: 2,
Schema: schema,
Properties: map[string]string{
common.CollectionTTLConfigKey: "0",
},
@ -469,6 +471,7 @@ func Test_compactionTrigger_force(t *testing.T) {
Type: datapb.CompactionType_MixCompaction,
Channel: "ch1",
TotalRows: 200,
Schema: schema,
},
},
},
@ -2000,12 +2003,12 @@ func Test_compactionTrigger_new(t *testing.T) {
}
func Test_compactionTrigger_allocTs(t *testing.T) {
got := newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{scheduler: NewCompactionScheduler()}, newMockAllocator(), newMockHandler(), newMockVersionManager())
got := newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{scheduler: NewCompactionScheduler(nil)}, newMockAllocator(), newMockHandler(), newMockVersionManager())
ts, err := got.allocTs()
assert.NoError(t, err)
assert.True(t, ts > 0)
got = newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{scheduler: NewCompactionScheduler()}, &FailsAllocator{}, newMockHandler(), newMockVersionManager())
got = newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{scheduler: NewCompactionScheduler(nil)}, &FailsAllocator{}, newMockHandler(), newMockVersionManager())
ts, err = got.allocTs()
assert.Error(t, err)
assert.Equal(t, uint64(0), ts)
@ -2032,7 +2035,7 @@ func Test_compactionTrigger_getCompactTime(t *testing.T) {
}
m := &meta{segments: NewSegmentsInfo(), collections: collections}
got := newCompactionTrigger(m, &compactionPlanHandler{scheduler: NewCompactionScheduler()}, newMockAllocator(),
got := newCompactionTrigger(m, &compactionPlanHandler{scheduler: NewCompactionScheduler(nil)}, newMockAllocator(),
&ServerHandler{
&Server{
meta: m,
@ -2386,7 +2389,7 @@ func (s *CompactionTriggerSuite) TestHandleSignal() {
},
},
}, nil)
s.compactionHandler.EXPECT().execCompactionPlan(mock.Anything, mock.Anything).Return(nil)
s.compactionHandler.EXPECT().execCompactionPlan(mock.Anything, mock.Anything).Return()
tr.handleSignal(&compactionSignal{
segmentID: 1,
collectionID: s.collectionID,
@ -2517,7 +2520,7 @@ func (s *CompactionTriggerSuite) TestHandleGlobalSignal() {
common.CollectionAutoCompactionKey: "false",
},
}, nil)
s.compactionHandler.EXPECT().execCompactionPlan(mock.Anything, mock.Anything).Return(nil)
s.compactionHandler.EXPECT().execCompactionPlan(mock.Anything, mock.Anything).Return()
tr.handleGlobalSignal(&compactionSignal{
segmentID: 1,
collectionID: s.collectionID,

View File

@ -2,10 +2,12 @@ package datacoord
import (
"context"
"time"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/log"
)
@ -33,16 +35,18 @@ type TriggerManager interface {
// 2. SystemIDLE & schedulerIDLE
// 3. Manual Compaction
type CompactionTriggerManager struct {
scheduler Scheduler
handler compactionPlanContext // TODO replace with scheduler
scheduler Scheduler
handler Handler
compactionHandler compactionPlanContext // TODO replace with scheduler
allocator allocator
}
func NewCompactionTriggerManager(alloc allocator, handler compactionPlanContext) *CompactionTriggerManager {
func NewCompactionTriggerManager(alloc allocator, handler Handler, compactionHandler compactionPlanContext) *CompactionTriggerManager {
m := &CompactionTriggerManager{
allocator: alloc,
handler: handler,
allocator: alloc,
handler: handler,
compactionHandler: compactionHandler,
}
return m
@ -51,7 +55,7 @@ func NewCompactionTriggerManager(alloc allocator, handler compactionPlanContext)
func (m *CompactionTriggerManager) Notify(taskID UniqueID, eventType CompactionTriggerType, views []CompactionView) {
log := log.With(zap.Int64("taskID", taskID))
for _, view := range views {
if m.handler.isFull() {
if m.compactionHandler.isFull() {
log.RatedInfo(1.0, "Skip trigger compaction for scheduler is full")
return
}
@ -103,7 +107,7 @@ func (m *CompactionTriggerManager) SubmitL0ViewToScheduler(taskID int64, outView
// TODO, remove handler, use scheduler
// m.scheduler.Submit(plan)
m.handler.execCompactionPlan(signal, plan)
m.compactionHandler.execCompactionPlan(signal, plan)
log.Info("Finish to submit a LevelZeroCompaction plan",
zap.Int64("taskID", taskID),
zap.Int64("planID", plan.GetPlanID()),
@ -130,7 +134,14 @@ func (m *CompactionTriggerManager) buildL0CompactionPlan(view CompactionView) *d
Channel: view.GetGroupLabel().Channel,
}
if err := fillOriginPlan(m.allocator, plan); err != nil {
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
collection, err := m.handler.GetCollection(ctx, view.GetGroupLabel().CollectionID)
if err != nil {
return nil
}
if err := fillOriginPlan(collection.Schema, m.allocator, plan); err != nil {
return nil
}
@ -145,14 +156,16 @@ type chanPartSegments struct {
segments []*SegmentInfo
}
func fillOriginPlan(alloc allocator, plan *datapb.CompactionPlan) error {
// TODO context
id, err := alloc.allocID(context.TODO())
func fillOriginPlan(schema *schemapb.CollectionSchema, alloc allocator, plan *datapb.CompactionPlan) error {
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
defer cancel()
id, err := alloc.allocID(ctx)
if err != nil {
return err
}
plan.PlanID = id
plan.TimeoutInSeconds = Params.DataCoordCfg.CompactionTimeoutInSeconds.GetAsInt32()
plan.Schema = schema
return nil
}

View File

@ -20,6 +20,7 @@ type CompactionTriggerManagerSuite struct {
suite.Suite
mockAlloc *NMockAllocator
handler Handler
mockPlanContext *MockCompactionPlanContext
testLabel *CompactionGroupLabel
meta *meta
@ -29,6 +30,7 @@ type CompactionTriggerManagerSuite struct {
func (s *CompactionTriggerManagerSuite) SetupTest() {
s.mockAlloc = NewNMockAllocator(s.T())
s.handler = NewNMockHandler(s.T())
s.mockPlanContext = NewMockCompactionPlanContext(s.T())
s.testLabel = &CompactionGroupLabel{
@ -42,7 +44,7 @@ func (s *CompactionTriggerManagerSuite) SetupTest() {
s.meta.segments.SetSegment(id, segment)
}
s.m = NewCompactionTriggerManager(s.mockAlloc, s.mockPlanContext)
s.m = NewCompactionTriggerManager(s.mockAlloc, s.handler, s.mockPlanContext)
}
func (s *CompactionTriggerManagerSuite) TestNotifyToFullScheduler() {
@ -73,6 +75,10 @@ func (s *CompactionTriggerManagerSuite) TestNotifyToFullScheduler() {
}
func (s *CompactionTriggerManagerSuite) TestNotifyByViewIDLE() {
handler := NewNMockHandler(s.T())
handler.EXPECT().GetCollection(mock.Anything, mock.Anything).Return(&collectionInfo{}, nil)
s.m.handler = handler
viewManager := NewCompactionViewManager(s.meta, s.m, s.m.allocator)
collSegs := s.meta.GetCompactableSegmentGroupByCollection()
@ -120,12 +126,16 @@ func (s *CompactionTriggerManagerSuite) TestNotifyByViewIDLE() {
s.ElementsMatch(expectedSegs, gotSegs)
log.Info("generated plan", zap.Any("plan", plan))
}).Return(nil).Once()
}).Return().Once()
s.m.Notify(19530, TriggerTypeLevelZeroViewIDLE, levelZeroView)
}
func (s *CompactionTriggerManagerSuite) TestNotifyByViewChange() {
handler := NewNMockHandler(s.T())
handler.EXPECT().GetCollection(mock.Anything, mock.Anything).Return(&collectionInfo{}, nil)
s.m.handler = handler
viewManager := NewCompactionViewManager(s.meta, s.m, s.m.allocator)
collSegs := s.meta.GetCompactableSegmentGroupByCollection()
@ -168,7 +178,7 @@ func (s *CompactionTriggerManagerSuite) TestNotifyByViewChange() {
s.ElementsMatch(expectedSegs, gotSegs)
log.Info("generated plan", zap.Any("plan", plan))
}).Return(nil).Once()
}).Return().Once()
s.m.Notify(19530, TriggerTypeLevelZeroViewChange, levelZeroView)
}

View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
// Code generated by mockery v2.30.1. DO NOT EDIT.
package datacoord
@ -74,8 +74,8 @@ type MockCluster_DropImport_Call struct {
}
// DropImport is a helper method to define mock.On call
// - nodeID int64
// - in *datapb.DropImportRequest
// - nodeID int64
// - in *datapb.DropImportRequest
func (_e *MockCluster_Expecter) DropImport(nodeID interface{}, in interface{}) *MockCluster_DropImport_Call {
return &MockCluster_DropImport_Call{Call: _e.mock.On("DropImport", nodeID, in)}
}
@ -117,10 +117,10 @@ type MockCluster_Flush_Call struct {
}
// Flush is a helper method to define mock.On call
// - ctx context.Context
// - nodeID int64
// - channel string
// - segments []*datapb.SegmentInfo
// - ctx context.Context
// - nodeID int64
// - channel string
// - segments []*datapb.SegmentInfo
func (_e *MockCluster_Expecter) Flush(ctx interface{}, nodeID interface{}, channel interface{}, segments interface{}) *MockCluster_Flush_Call {
return &MockCluster_Flush_Call{Call: _e.mock.On("Flush", ctx, nodeID, channel, segments)}
}
@ -162,10 +162,10 @@ type MockCluster_FlushChannels_Call struct {
}
// FlushChannels is a helper method to define mock.On call
// - ctx context.Context
// - nodeID int64
// - flushTs uint64
// - channels []string
// - ctx context.Context
// - nodeID int64
// - flushTs uint64
// - channels []string
func (_e *MockCluster_Expecter) FlushChannels(ctx interface{}, nodeID interface{}, flushTs interface{}, channels interface{}) *MockCluster_FlushChannels_Call {
return &MockCluster_FlushChannels_Call{Call: _e.mock.On("FlushChannels", ctx, nodeID, flushTs, channels)}
}
@ -250,8 +250,8 @@ type MockCluster_ImportV2_Call struct {
}
// ImportV2 is a helper method to define mock.On call
// - nodeID int64
// - in *datapb.ImportRequest
// - nodeID int64
// - in *datapb.ImportRequest
func (_e *MockCluster_Expecter) ImportV2(nodeID interface{}, in interface{}) *MockCluster_ImportV2_Call {
return &MockCluster_ImportV2_Call{Call: _e.mock.On("ImportV2", nodeID, in)}
}
@ -293,8 +293,8 @@ type MockCluster_PreImport_Call struct {
}
// PreImport is a helper method to define mock.On call
// - nodeID int64
// - in *datapb.PreImportRequest
// - nodeID int64
// - in *datapb.PreImportRequest
func (_e *MockCluster_Expecter) PreImport(nodeID interface{}, in interface{}) *MockCluster_PreImport_Call {
return &MockCluster_PreImport_Call{Call: _e.mock.On("PreImport", nodeID, in)}
}
@ -348,8 +348,8 @@ type MockCluster_QueryImport_Call struct {
}
// QueryImport is a helper method to define mock.On call
// - nodeID int64
// - in *datapb.QueryImportRequest
// - nodeID int64
// - in *datapb.QueryImportRequest
func (_e *MockCluster_Expecter) QueryImport(nodeID interface{}, in interface{}) *MockCluster_QueryImport_Call {
return &MockCluster_QueryImport_Call{Call: _e.mock.On("QueryImport", nodeID, in)}
}
@ -403,8 +403,8 @@ type MockCluster_QueryPreImport_Call struct {
}
// QueryPreImport is a helper method to define mock.On call
// - nodeID int64
// - in *datapb.QueryPreImportRequest
// - nodeID int64
// - in *datapb.QueryPreImportRequest
func (_e *MockCluster_Expecter) QueryPreImport(nodeID interface{}, in interface{}) *MockCluster_QueryPreImport_Call {
return &MockCluster_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", nodeID, in)}
}
@ -426,6 +426,49 @@ func (_c *MockCluster_QueryPreImport_Call) RunAndReturn(run func(int64, *datapb.
return _c
}
// QuerySlots provides a mock function with given fields:
func (_m *MockCluster) QuerySlots() map[int64]int64 {
ret := _m.Called()
var r0 map[int64]int64
if rf, ok := ret.Get(0).(func() map[int64]int64); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(map[int64]int64)
}
}
return r0
}
// MockCluster_QuerySlots_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'QuerySlots'
type MockCluster_QuerySlots_Call struct {
*mock.Call
}
// QuerySlots is a helper method to define mock.On call
func (_e *MockCluster_Expecter) QuerySlots() *MockCluster_QuerySlots_Call {
return &MockCluster_QuerySlots_Call{Call: _e.mock.On("QuerySlots")}
}
func (_c *MockCluster_QuerySlots_Call) Run(run func()) *MockCluster_QuerySlots_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockCluster_QuerySlots_Call) Return(_a0 map[int64]int64) *MockCluster_QuerySlots_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockCluster_QuerySlots_Call) RunAndReturn(run func() map[int64]int64) *MockCluster_QuerySlots_Call {
_c.Call.Return(run)
return _c
}
// Register provides a mock function with given fields: node
func (_m *MockCluster) Register(node *NodeInfo) error {
ret := _m.Called(node)
@ -446,7 +489,7 @@ type MockCluster_Register_Call struct {
}
// Register is a helper method to define mock.On call
// - node *NodeInfo
// - node *NodeInfo
func (_e *MockCluster_Expecter) Register(node interface{}) *MockCluster_Register_Call {
return &MockCluster_Register_Call{Call: _e.mock.On("Register", node)}
}
@ -488,8 +531,8 @@ type MockCluster_Startup_Call struct {
}
// Startup is a helper method to define mock.On call
// - ctx context.Context
// - nodes []*NodeInfo
// - ctx context.Context
// - nodes []*NodeInfo
func (_e *MockCluster_Expecter) Startup(ctx interface{}, nodes interface{}) *MockCluster_Startup_Call {
return &MockCluster_Startup_Call{Call: _e.mock.On("Startup", ctx, nodes)}
}
@ -531,7 +574,7 @@ type MockCluster_UnRegister_Call struct {
}
// UnRegister is a helper method to define mock.On call
// - node *NodeInfo
// - node *NodeInfo
func (_e *MockCluster_Expecter) UnRegister(node interface{}) *MockCluster_UnRegister_Call {
return &MockCluster_UnRegister_Call{Call: _e.mock.On("UnRegister", node)}
}
@ -573,8 +616,8 @@ type MockCluster_Watch_Call struct {
}
// Watch is a helper method to define mock.On call
// - ctx context.Context
// - ch RWChannel
// - ctx context.Context
// - ch RWChannel
func (_e *MockCluster_Expecter) Watch(ctx interface{}, ch interface{}) *MockCluster_Watch_Call {
return &MockCluster_Watch_Call{Call: _e.mock.On("Watch", ctx, ch)}
}

View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
// Code generated by mockery v2.30.1. DO NOT EDIT.
package datacoord
@ -21,17 +21,8 @@ func (_m *MockCompactionPlanContext) EXPECT() *MockCompactionPlanContext_Expecte
}
// execCompactionPlan provides a mock function with given fields: signal, plan
func (_m *MockCompactionPlanContext) execCompactionPlan(signal *compactionSignal, plan *datapb.CompactionPlan) error {
ret := _m.Called(signal, plan)
var r0 error
if rf, ok := ret.Get(0).(func(*compactionSignal, *datapb.CompactionPlan) error); ok {
r0 = rf(signal, plan)
} else {
r0 = ret.Error(0)
}
return r0
func (_m *MockCompactionPlanContext) execCompactionPlan(signal *compactionSignal, plan *datapb.CompactionPlan) {
_m.Called(signal, plan)
}
// MockCompactionPlanContext_execCompactionPlan_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'execCompactionPlan'
@ -40,8 +31,8 @@ type MockCompactionPlanContext_execCompactionPlan_Call struct {
}
// execCompactionPlan is a helper method to define mock.On call
// - signal *compactionSignal
// - plan *datapb.CompactionPlan
// - signal *compactionSignal
// - plan *datapb.CompactionPlan
func (_e *MockCompactionPlanContext_Expecter) execCompactionPlan(signal interface{}, plan interface{}) *MockCompactionPlanContext_execCompactionPlan_Call {
return &MockCompactionPlanContext_execCompactionPlan_Call{Call: _e.mock.On("execCompactionPlan", signal, plan)}
}
@ -53,12 +44,12 @@ func (_c *MockCompactionPlanContext_execCompactionPlan_Call) Run(run func(signal
return _c
}
func (_c *MockCompactionPlanContext_execCompactionPlan_Call) Return(_a0 error) *MockCompactionPlanContext_execCompactionPlan_Call {
_c.Call.Return(_a0)
func (_c *MockCompactionPlanContext_execCompactionPlan_Call) Return() *MockCompactionPlanContext_execCompactionPlan_Call {
_c.Call.Return()
return _c
}
func (_c *MockCompactionPlanContext_execCompactionPlan_Call) RunAndReturn(run func(*compactionSignal, *datapb.CompactionPlan) error) *MockCompactionPlanContext_execCompactionPlan_Call {
func (_c *MockCompactionPlanContext_execCompactionPlan_Call) RunAndReturn(run func(*compactionSignal, *datapb.CompactionPlan)) *MockCompactionPlanContext_execCompactionPlan_Call {
_c.Call.Return(run)
return _c
}
@ -85,7 +76,7 @@ type MockCompactionPlanContext_getCompaction_Call struct {
}
// getCompaction is a helper method to define mock.On call
// - planID int64
// - planID int64
func (_e *MockCompactionPlanContext_Expecter) getCompaction(planID interface{}) *MockCompactionPlanContext_getCompaction_Call {
return &MockCompactionPlanContext_getCompaction_Call{Call: _e.mock.On("getCompaction", planID)}
}
@ -129,7 +120,7 @@ type MockCompactionPlanContext_getCompactionTasksBySignalID_Call struct {
}
// getCompactionTasksBySignalID is a helper method to define mock.On call
// - signalID int64
// - signalID int64
func (_e *MockCompactionPlanContext_Expecter) getCompactionTasksBySignalID(signalID interface{}) *MockCompactionPlanContext_getCompactionTasksBySignalID_Call {
return &MockCompactionPlanContext_getCompactionTasksBySignalID_Call{Call: _e.mock.On("getCompactionTasksBySignalID", signalID)}
}
@ -203,7 +194,7 @@ type MockCompactionPlanContext_removeTasksByChannel_Call struct {
}
// removeTasksByChannel is a helper method to define mock.On call
// - channel string
// - channel string
func (_e *MockCompactionPlanContext_Expecter) removeTasksByChannel(channel interface{}) *MockCompactionPlanContext_removeTasksByChannel_Call {
return &MockCompactionPlanContext_removeTasksByChannel_Call{Call: _e.mock.On("removeTasksByChannel", channel)}
}
@ -309,7 +300,7 @@ type MockCompactionPlanContext_updateCompaction_Call struct {
}
// updateCompaction is a helper method to define mock.On call
// - ts uint64
// - ts uint64
func (_e *MockCompactionPlanContext_Expecter) updateCompaction(ts interface{}) *MockCompactionPlanContext_updateCompaction_Call {
return &MockCompactionPlanContext_updateCompaction_Call{Call: _e.mock.On("updateCompaction", ts)}
}

View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
// Code generated by mockery v2.30.1. DO NOT EDIT.
package datacoord
@ -35,7 +35,7 @@ type MockSessionManager_AddSession_Call struct {
}
// AddSession is a helper method to define mock.On call
// - node *NodeInfo
// - node *NodeInfo
func (_e *MockSessionManager_Expecter) AddSession(node interface{}) *MockSessionManager_AddSession_Call {
return &MockSessionManager_AddSession_Call{Call: _e.mock.On("AddSession", node)}
}
@ -89,9 +89,9 @@ type MockSessionManager_CheckChannelOperationProgress_Call struct {
}
// CheckChannelOperationProgress is a helper method to define mock.On call
// - ctx context.Context
// - nodeID int64
// - info *datapb.ChannelWatchInfo
// - ctx context.Context
// - nodeID int64
// - info *datapb.ChannelWatchInfo
func (_e *MockSessionManager_Expecter) CheckChannelOperationProgress(ctx interface{}, nodeID interface{}, info interface{}) *MockSessionManager_CheckChannelOperationProgress_Call {
return &MockSessionManager_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress", ctx, nodeID, info)}
}
@ -133,7 +133,7 @@ type MockSessionManager_CheckHealth_Call struct {
}
// CheckHealth is a helper method to define mock.On call
// - ctx context.Context
// - ctx context.Context
func (_e *MockSessionManager_Expecter) CheckHealth(ctx interface{}) *MockSessionManager_CheckHealth_Call {
return &MockSessionManager_CheckHealth_Call{Call: _e.mock.On("CheckHealth", ctx)}
}
@ -207,9 +207,9 @@ type MockSessionManager_Compaction_Call struct {
}
// Compaction is a helper method to define mock.On call
// - ctx context.Context
// - nodeID int64
// - plan *datapb.CompactionPlan
// - ctx context.Context
// - nodeID int64
// - plan *datapb.CompactionPlan
func (_e *MockSessionManager_Expecter) Compaction(ctx interface{}, nodeID interface{}, plan interface{}) *MockSessionManager_Compaction_Call {
return &MockSessionManager_Compaction_Call{Call: _e.mock.On("Compaction", ctx, nodeID, plan)}
}
@ -242,7 +242,7 @@ type MockSessionManager_DeleteSession_Call struct {
}
// DeleteSession is a helper method to define mock.On call
// - node *NodeInfo
// - node *NodeInfo
func (_e *MockSessionManager_Expecter) DeleteSession(node interface{}) *MockSessionManager_DeleteSession_Call {
return &MockSessionManager_DeleteSession_Call{Call: _e.mock.On("DeleteSession", node)}
}
@ -284,8 +284,8 @@ type MockSessionManager_DropImport_Call struct {
}
// DropImport is a helper method to define mock.On call
// - nodeID int64
// - in *datapb.DropImportRequest
// - nodeID int64
// - in *datapb.DropImportRequest
func (_e *MockSessionManager_Expecter) DropImport(nodeID interface{}, in interface{}) *MockSessionManager_DropImport_Call {
return &MockSessionManager_DropImport_Call{Call: _e.mock.On("DropImport", nodeID, in)}
}
@ -318,9 +318,9 @@ type MockSessionManager_Flush_Call struct {
}
// Flush is a helper method to define mock.On call
// - ctx context.Context
// - nodeID int64
// - req *datapb.FlushSegmentsRequest
// - ctx context.Context
// - nodeID int64
// - req *datapb.FlushSegmentsRequest
func (_e *MockSessionManager_Expecter) Flush(ctx interface{}, nodeID interface{}, req interface{}) *MockSessionManager_Flush_Call {
return &MockSessionManager_Flush_Call{Call: _e.mock.On("Flush", ctx, nodeID, req)}
}
@ -362,9 +362,9 @@ type MockSessionManager_FlushChannels_Call struct {
}
// FlushChannels is a helper method to define mock.On call
// - ctx context.Context
// - nodeID int64
// - req *datapb.FlushChannelsRequest
// - ctx context.Context
// - nodeID int64
// - req *datapb.FlushChannelsRequest
func (_e *MockSessionManager_Expecter) FlushChannels(ctx interface{}, nodeID interface{}, req interface{}) *MockSessionManager_FlushChannels_Call {
return &MockSessionManager_FlushChannels_Call{Call: _e.mock.On("FlushChannels", ctx, nodeID, req)}
}
@ -545,8 +545,8 @@ type MockSessionManager_ImportV2_Call struct {
}
// ImportV2 is a helper method to define mock.On call
// - nodeID int64
// - in *datapb.ImportRequest
// - nodeID int64
// - in *datapb.ImportRequest
func (_e *MockSessionManager_Expecter) ImportV2(nodeID interface{}, in interface{}) *MockSessionManager_ImportV2_Call {
return &MockSessionManager_ImportV2_Call{Call: _e.mock.On("ImportV2", nodeID, in)}
}
@ -588,9 +588,9 @@ type MockSessionManager_NotifyChannelOperation_Call struct {
}
// NotifyChannelOperation is a helper method to define mock.On call
// - ctx context.Context
// - nodeID int64
// - req *datapb.ChannelOperationsRequest
// - ctx context.Context
// - nodeID int64
// - req *datapb.ChannelOperationsRequest
func (_e *MockSessionManager_Expecter) NotifyChannelOperation(ctx interface{}, nodeID interface{}, req interface{}) *MockSessionManager_NotifyChannelOperation_Call {
return &MockSessionManager_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation", ctx, nodeID, req)}
}
@ -632,8 +632,8 @@ type MockSessionManager_PreImport_Call struct {
}
// PreImport is a helper method to define mock.On call
// - nodeID int64
// - in *datapb.PreImportRequest
// - nodeID int64
// - in *datapb.PreImportRequest
func (_e *MockSessionManager_Expecter) PreImport(nodeID interface{}, in interface{}) *MockSessionManager_PreImport_Call {
return &MockSessionManager_PreImport_Call{Call: _e.mock.On("PreImport", nodeID, in)}
}
@ -687,8 +687,8 @@ type MockSessionManager_QueryImport_Call struct {
}
// QueryImport is a helper method to define mock.On call
// - nodeID int64
// - in *datapb.QueryImportRequest
// - nodeID int64
// - in *datapb.QueryImportRequest
func (_e *MockSessionManager_Expecter) QueryImport(nodeID interface{}, in interface{}) *MockSessionManager_QueryImport_Call {
return &MockSessionManager_QueryImport_Call{Call: _e.mock.On("QueryImport", nodeID, in)}
}
@ -742,8 +742,8 @@ type MockSessionManager_QueryPreImport_Call struct {
}
// QueryPreImport is a helper method to define mock.On call
// - nodeID int64
// - in *datapb.QueryPreImportRequest
// - nodeID int64
// - in *datapb.QueryPreImportRequest
func (_e *MockSessionManager_Expecter) QueryPreImport(nodeID interface{}, in interface{}) *MockSessionManager_QueryPreImport_Call {
return &MockSessionManager_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", nodeID, in)}
}
@ -765,6 +765,60 @@ func (_c *MockSessionManager_QueryPreImport_Call) RunAndReturn(run func(int64, *
return _c
}
// QuerySlot provides a mock function with given fields: nodeID
func (_m *MockSessionManager) QuerySlot(nodeID int64) (*datapb.QuerySlotResponse, error) {
ret := _m.Called(nodeID)
var r0 *datapb.QuerySlotResponse
var r1 error
if rf, ok := ret.Get(0).(func(int64) (*datapb.QuerySlotResponse, error)); ok {
return rf(nodeID)
}
if rf, ok := ret.Get(0).(func(int64) *datapb.QuerySlotResponse); ok {
r0 = rf(nodeID)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.QuerySlotResponse)
}
}
if rf, ok := ret.Get(1).(func(int64) error); ok {
r1 = rf(nodeID)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockSessionManager_QuerySlot_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'QuerySlot'
type MockSessionManager_QuerySlot_Call struct {
*mock.Call
}
// QuerySlot is a helper method to define mock.On call
// - nodeID int64
func (_e *MockSessionManager_Expecter) QuerySlot(nodeID interface{}) *MockSessionManager_QuerySlot_Call {
return &MockSessionManager_QuerySlot_Call{Call: _e.mock.On("QuerySlot", nodeID)}
}
func (_c *MockSessionManager_QuerySlot_Call) Run(run func(nodeID int64)) *MockSessionManager_QuerySlot_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(int64))
})
return _c
}
func (_c *MockSessionManager_QuerySlot_Call) Return(_a0 *datapb.QuerySlotResponse, _a1 error) *MockSessionManager_QuerySlot_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockSessionManager_QuerySlot_Call) RunAndReturn(run func(int64) (*datapb.QuerySlotResponse, error)) *MockSessionManager_QuerySlot_Call {
_c.Call.Return(run)
return _c
}
// SyncSegments provides a mock function with given fields: nodeID, req
func (_m *MockSessionManager) SyncSegments(nodeID int64, req *datapb.SyncSegmentsRequest) error {
ret := _m.Called(nodeID, req)
@ -785,8 +839,8 @@ type MockSessionManager_SyncSegments_Call struct {
}
// SyncSegments is a helper method to define mock.On call
// - nodeID int64
// - req *datapb.SyncSegmentsRequest
// - nodeID int64
// - req *datapb.SyncSegmentsRequest
func (_e *MockSessionManager_Expecter) SyncSegments(nodeID interface{}, req interface{}) *MockSessionManager_SyncSegments_Call {
return &MockSessionManager_SyncSegments_Call{Call: _e.mock.On("SyncSegments", nodeID, req)}
}

View File

@ -265,7 +265,7 @@ func (c *mockDataNodeClient) GetMetrics(ctx context.Context, req *milvuspb.GetMe
}, nil
}
func (c *mockDataNodeClient) Compaction(ctx context.Context, req *datapb.CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) {
func (c *mockDataNodeClient) CompactionV2(ctx context.Context, req *datapb.CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) {
if c.ch != nil {
c.ch <- struct{}{}
if c.compactionResp != nil {
@ -319,6 +319,10 @@ func (c *mockDataNodeClient) DropImport(ctx context.Context, req *datapb.DropImp
return &commonpb.Status{ErrorCode: commonpb.ErrorCode_Success}, nil
}
func (c *mockDataNodeClient) QuerySlot(ctx context.Context, req *datapb.QuerySlotRequest, opts ...grpc.CallOption) (*datapb.QuerySlotResponse, error) {
return &datapb.QuerySlotResponse{Status: merr.Success()}, nil
}
func (c *mockDataNodeClient) Stop() error {
c.state = commonpb.StateCode_Abnormal
return nil

View File

@ -523,8 +523,8 @@ func (s *Server) SetIndexNodeCreator(f func(context.Context, string, int64) (typ
}
func (s *Server) createCompactionHandler() {
s.compactionHandler = newCompactionPlanHandler(s.sessionManager, s.channelManager, s.meta, s.allocator)
triggerv2 := NewCompactionTriggerManager(s.allocator, s.compactionHandler)
s.compactionHandler = newCompactionPlanHandler(s.cluster, s.sessionManager, s.channelManager, s.meta, s.allocator)
triggerv2 := NewCompactionTriggerManager(s.allocator, s.handler, s.compactionHandler)
s.compactionViewManager = NewCompactionViewManager(s.meta, triggerv2, s.allocator)
}

View File

@ -44,8 +44,10 @@ import (
const (
flushTimeout = 15 * time.Second
importTaskTimeout = 10 * time.Second
querySlotTimeout = 10 * time.Second
)
//go:generate mockery --name=SessionManager --structname=MockSessionManager --output=./ --filename=mock_session_manager.go --with-expecter --inpackage
type SessionManager interface {
AddSession(node *NodeInfo)
DeleteSession(node *NodeInfo)
@ -65,6 +67,7 @@ type SessionManager interface {
QueryImport(nodeID int64, in *datapb.QueryImportRequest) (*datapb.QueryImportResponse, error)
DropImport(nodeID int64, in *datapb.DropImportRequest) error
CheckHealth(ctx context.Context) error
QuerySlot(nodeID int64) (*datapb.QuerySlotResponse, error)
Close()
}
@ -198,7 +201,7 @@ func (c *SessionManagerImpl) Compaction(ctx context.Context, nodeID int64, plan
return err
}
resp, err := cli.Compaction(ctx, plan)
resp, err := cli.CompactionV2(ctx, plan)
if err := VerifyResponse(resp, err); err != nil {
log.Warn("failed to execute compaction", zap.Int64("node", nodeID), zap.Error(err), zap.Int64("planID", plan.GetPlanID()))
return err
@ -474,6 +477,22 @@ func (c *SessionManagerImpl) CheckHealth(ctx context.Context) error {
return group.Wait()
}
func (c *SessionManagerImpl) QuerySlot(nodeID int64) (*datapb.QuerySlotResponse, error) {
log := log.With(zap.Int64("nodeID", nodeID))
ctx, cancel := context.WithTimeout(context.Background(), querySlotTimeout)
defer cancel()
cli, err := c.getClient(ctx, nodeID)
if err != nil {
log.Info("failed to get client", zap.Error(err))
return nil, err
}
resp, err := cli.QuerySlot(ctx, &datapb.QuerySlotRequest{})
if err = VerifyResponse(resp.GetStatus(), err); err != nil {
return nil, err
}
return resp, nil
}
// Close release sessions
func (c *SessionManagerImpl) Close() {
c.sessions.Lock()

View File

@ -21,10 +21,10 @@ import (
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
//go:generate mockery --name=Compactor --structname=MockCompactor --output=./ --filename=mock_compactor.go --with-expecter --inpackage
type Compactor interface {
Complete()
Compact() (*datapb.CompactionPlanResult, error)
InjectDone()
Stop()
GetPlanID() typeutil.UniqueID
GetCollection() typeutil.UniqueID

View File

@ -20,6 +20,7 @@ import (
"context"
"fmt"
"math"
"sync"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
@ -30,7 +31,7 @@ import (
"github.com/milvus-io/milvus/internal/datanode/allocator"
"github.com/milvus-io/milvus/internal/datanode/io"
"github.com/milvus-io/milvus/internal/datanode/metacache"
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
"github.com/milvus-io/milvus/internal/datanode/util"
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage"
@ -48,8 +49,7 @@ type LevelZeroCompactionTask struct {
io.BinlogIO
allocator allocator.Allocator
metacache metacache.MetaCache
syncmgr syncmgr.SyncManager
cm storage.ChunkManager
plan *datapb.CompactionPlan
@ -64,8 +64,7 @@ func NewLevelZeroCompactionTask(
ctx context.Context,
binlogIO io.BinlogIO,
alloc allocator.Allocator,
metaCache metacache.MetaCache,
syncmgr syncmgr.SyncManager,
cm storage.ChunkManager,
plan *datapb.CompactionPlan,
) *LevelZeroCompactionTask {
ctx, cancel := context.WithCancel(ctx)
@ -75,8 +74,7 @@ func NewLevelZeroCompactionTask(
BinlogIO: binlogIO,
allocator: alloc,
metacache: metaCache,
syncmgr: syncmgr,
cm: cm,
plan: plan,
tr: timerecord.NewTimeRecorder("levelzero compaction"),
done: make(chan struct{}, 1),
@ -101,12 +99,10 @@ func (t *LevelZeroCompactionTask) GetChannelName() string {
}
func (t *LevelZeroCompactionTask) GetCollection() int64 {
return t.metacache.Collection()
// The length of SegmentBinlogs is checked before task enqueueing.
return t.plan.GetSegmentBinlogs()[0].GetCollectionID()
}
// Do nothing for levelzero compaction
func (t *LevelZeroCompactionTask) InjectDone() {}
func (t *LevelZeroCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, "L0Compact")
defer span.End()
@ -122,13 +118,10 @@ func (t *LevelZeroCompactionTask) Compact() (*datapb.CompactionPlanResult, error
return s.Level == datapb.SegmentLevel_L0
})
targetSegIDs := lo.FilterMap(t.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) (int64, bool) {
if s.Level == datapb.SegmentLevel_L1 {
return s.GetSegmentID(), true
}
return 0, false
targetSegments := lo.Filter(t.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) bool {
return s.Level != datapb.SegmentLevel_L0
})
if len(targetSegIDs) == 0 {
if len(targetSegments) == 0 {
log.Warn("compact wrong, not target sealed segments")
return nil, errors.New("illegal compaction plan with empty target segments")
}
@ -156,7 +149,7 @@ func (t *LevelZeroCompactionTask) Compact() (*datapb.CompactionPlanResult, error
}
batchSize := getMaxBatchSize(totalSize)
resultSegments, err := t.process(ctx, batchSize, targetSegIDs, lo.Values(totalDeltalogs)...)
resultSegments, err := t.process(ctx, batchSize, targetSegments, lo.Values(totalDeltalogs)...)
if err != nil {
return nil, err
}
@ -240,7 +233,7 @@ func (t *LevelZeroCompactionTask) serializeUpload(ctx context.Context, segmentWr
func (t *LevelZeroCompactionTask) splitDelta(
ctx context.Context,
allDelta *storage.DeleteData,
targetSegIDs []int64,
segmentBfs map[int64]*metacache.BloomFilterSet,
) map[int64]*SegmentDeltaWriter {
traceCtx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "L0Compact splitDelta")
defer span.End()
@ -249,10 +242,8 @@ func (t *LevelZeroCompactionTask) splitDelta(
return segment.GetSegmentID(), segment
})
// segments shall be safe to read outside
segments := t.metacache.GetSegmentsBy(metacache.WithSegmentIDs(targetSegIDs...))
// spilt all delete data to segments
retMap := t.applyBFInParallel(traceCtx, allDelta, io.GetBFApplyPool(), segments)
retMap := t.applyBFInParallel(traceCtx, allDelta, io.GetBFApplyPool(), segmentBfs)
targetSegBuffer := make(map[int64]*SegmentDeltaWriter)
retMap.Range(func(key int, value *BatchApplyRet) bool {
@ -282,7 +273,7 @@ type BatchApplyRet = struct {
Segment2Hits map[int64][]bool
}
func (t *LevelZeroCompactionTask) applyBFInParallel(ctx context.Context, deltaData *storage.DeleteData, pool *conc.Pool[any], segmentBfs []*metacache.SegmentInfo) *typeutil.ConcurrentMap[int, *BatchApplyRet] {
func (t *LevelZeroCompactionTask) applyBFInParallel(ctx context.Context, deltaData *storage.DeleteData, pool *conc.Pool[any], segmentBfs map[int64]*metacache.BloomFilterSet) *typeutil.ConcurrentMap[int, *BatchApplyRet] {
_, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "L0Compact applyBFInParallel")
defer span.End()
batchSize := paramtable.Get().CommonCfg.BloomFilterApplyBatchSize.GetAsInt()
@ -290,9 +281,8 @@ func (t *LevelZeroCompactionTask) applyBFInParallel(ctx context.Context, deltaDa
batchPredict := func(pks []storage.PrimaryKey) map[int64][]bool {
segment2Hits := make(map[int64][]bool, 0)
lc := storage.NewBatchLocationsCache(pks)
for _, s := range segmentBfs {
segmentID := s.SegmentID()
hits := s.GetBloomFilterSet().BatchPkExist(lc)
for segmentID, bf := range segmentBfs {
hits := bf.BatchPkExist(lc)
segment2Hits[segmentID] = hits
}
return segment2Hits
@ -325,7 +315,7 @@ func (t *LevelZeroCompactionTask) applyBFInParallel(ctx context.Context, deltaDa
return retMap
}
func (t *LevelZeroCompactionTask) process(ctx context.Context, batchSize int, targetSegments []int64, deltaLogs ...[]string) ([]*datapb.CompactionSegment, error) {
func (t *LevelZeroCompactionTask) process(ctx context.Context, batchSize int, targetSegments []*datapb.CompactionSegmentBinlogs, deltaLogs ...[]string) ([]*datapb.CompactionSegment, error) {
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "L0Compact process")
defer span.End()
@ -352,7 +342,13 @@ func (t *LevelZeroCompactionTask) process(ctx context.Context, batchSize int, ta
}
batchSegments := targetSegments[left:right]
batchSegWriter := t.splitDelta(ctx, allDelta, batchSegments)
segmentBFs, err := t.loadBF(batchSegments)
if err != nil {
return nil, err
}
batchSegWriter := t.splitDelta(ctx, allDelta, segmentBFs)
batchResults, err := t.serializeUpload(ctx, batchSegWriter)
if err != nil {
log.Warn("L0 compaction serialize upload fail", zap.Error(err))
@ -389,3 +385,41 @@ func (t *LevelZeroCompactionTask) loadDelta(ctx context.Context, deltaLogs []str
return dData, nil
}
func (t *LevelZeroCompactionTask) loadBF(targetSegments []*datapb.CompactionSegmentBinlogs) (map[int64]*metacache.BloomFilterSet, error) {
log := log.Ctx(t.ctx).With(
zap.Int64("planID", t.plan.GetPlanID()),
zap.String("type", t.plan.GetType().String()),
)
var (
futures = make([]*conc.Future[any], 0, len(targetSegments))
pool = io.GetOrCreateStatsPool()
mu = &sync.Mutex{}
bfs = make(map[int64]*metacache.BloomFilterSet)
)
for _, segment := range targetSegments {
segment := segment
future := pool.Submit(func() (any, error) {
_ = binlog.DecompressBinLog(storage.StatsBinlog, segment.GetCollectionID(),
segment.GetPartitionID(), segment.GetSegmentID(), segment.GetField2StatslogPaths())
pks, err := util.LoadStats(t.ctx, t.cm,
t.plan.GetSchema(), segment.GetSegmentID(), segment.GetField2StatslogPaths())
if err != nil {
log.Warn("failed to load segment stats log", zap.Error(err))
return err, err
}
bf := metacache.NewBloomFilterSet(pks...)
mu.Lock()
defer mu.Unlock()
bfs[segment.GetSegmentID()] = bf
return nil, nil
})
futures = append(futures, future)
}
err := conc.AwaitAll(futures...)
return bfs, err
}

View File

@ -26,11 +26,14 @@ import (
"github.com/stretchr/testify/suite"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/datanode/allocator"
"github.com/milvus-io/milvus/internal/datanode/io"
"github.com/milvus-io/milvus/internal/datanode/metacache"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/timerecord"
@ -47,7 +50,6 @@ type LevelZeroCompactionTaskSuite struct {
mockBinlogIO *io.MockBinlogIO
mockAlloc *allocator.MockAllocator
mockMeta *metacache.MockMetaCache
task *LevelZeroCompactionTask
dData *storage.DeleteData
@ -57,9 +59,8 @@ type LevelZeroCompactionTaskSuite struct {
func (s *LevelZeroCompactionTaskSuite) SetupTest() {
s.mockAlloc = allocator.NewMockAllocator(s.T())
s.mockBinlogIO = io.NewMockBinlogIO(s.T())
s.mockMeta = metacache.NewMockMetaCache(s.T())
// plan of the task is unset
s.task = NewLevelZeroCompactionTask(context.Background(), s.mockBinlogIO, s.mockAlloc, s.mockMeta, nil, nil)
s.task = NewLevelZeroCompactionTask(context.Background(), s.mockBinlogIO, s.mockAlloc, nil, nil)
pk2ts := map[int64]uint64{
1: 20000,
@ -97,13 +98,22 @@ func (s *LevelZeroCompactionTaskSuite) TestProcessLoadDeltaFail() {
},
{SegmentID: 200, Level: datapb.SegmentLevel_L1},
},
Schema: &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
IsPrimaryKey: true,
},
},
},
}
s.task.plan = plan
s.task.tr = timerecord.NewTimeRecorder("test")
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return(nil, errors.New("mock download fail")).Once()
targetSegments := []int64{200}
targetSegments := lo.Filter(plan.SegmentBinlogs, func(s *datapb.CompactionSegmentBinlogs, _ int) bool {
return s.Level == datapb.SegmentLevel_L1
})
deltaLogs := map[int64][]string{100: {"a/b/c1"}}
segments, err := s.task.process(context.Background(), 1, targetSegments, lo.Values(deltaLogs)...)
@ -128,25 +138,42 @@ func (s *LevelZeroCompactionTaskSuite) TestProcessUploadFail() {
},
},
},
{SegmentID: 200, Level: datapb.SegmentLevel_L1},
{SegmentID: 200, Level: datapb.SegmentLevel_L1, Field2StatslogPaths: []*datapb.FieldBinlog{
{
Binlogs: []*datapb.Binlog{
{LogID: 9999, LogSize: 100},
},
},
}},
},
Schema: &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
IsPrimaryKey: true,
},
},
},
}
s.task.plan = plan
s.task.tr = timerecord.NewTimeRecorder("test")
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return([][]byte{s.dBlob}, nil).Once()
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(errors.New("mock upload fail")).Once()
s.mockAlloc.EXPECT().AllocOne().Return(19530, nil).Once()
s.mockMeta.EXPECT().Collection().Return(1)
s.mockMeta.EXPECT().GetSegmentsBy(mock.Anything).RunAndReturn(
func(filters ...metacache.SegmentFilter) []*metacache.SegmentInfo {
bfs1 := metacache.NewBloomFilterSetWithBatchSize(100)
bfs1.UpdatePKRange(&storage.Int64FieldData{Data: []int64{1, 2}})
segment1 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 200}, bfs1)
return []*metacache.SegmentInfo{segment1}
}).Once()
targetSegments := []int64{200}
data := &storage.Int64FieldData{
Data: []int64{1, 2, 3, 4, 5, 6, 7, 8, 9},
}
sw := &storage.StatsWriter{}
err := sw.GenerateByData(common.RowIDField, schemapb.DataType_Int64, data)
s.NoError(err)
cm := mocks.NewChunkManager(s.T())
cm.EXPECT().MultiRead(mock.Anything, mock.Anything).Return([][]byte{sw.GetBuffer()}, nil)
s.task.cm = cm
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return([][]byte{s.dBlob}, nil).Once()
mockAlloc := allocator.NewMockAllocator(s.T())
mockAlloc.EXPECT().AllocOne().Return(0, errors.New("mock alloc err"))
s.task.allocator = mockAlloc
targetSegments := lo.Filter(plan.SegmentBinlogs, func(s *datapb.CompactionSegmentBinlogs, _ int) bool {
return s.Level == datapb.SegmentLevel_L1
})
deltaLogs := map[int64][]string{100: {"a/b/c1"}}
segments, err := s.task.process(context.Background(), 2, targetSegments, lo.Values(deltaLogs)...)
@ -160,7 +187,8 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() {
Type: datapb.CompactionType_Level0DeleteCompaction,
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
{
SegmentID: 100, Level: datapb.SegmentLevel_L0, Deltalogs: []*datapb.FieldBinlog{
CollectionID: 1,
SegmentID: 100, Level: datapb.SegmentLevel_L0, Deltalogs: []*datapb.FieldBinlog{
{
Binlogs: []*datapb.Binlog{
{LogPath: "a/b/c1", LogSize: 100},
@ -172,7 +200,8 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() {
},
},
{
SegmentID: 101, Level: datapb.SegmentLevel_L0, Deltalogs: []*datapb.FieldBinlog{
CollectionID: 1,
SegmentID: 101, Level: datapb.SegmentLevel_L0, Deltalogs: []*datapb.FieldBinlog{
{
Binlogs: []*datapb.Binlog{
{LogPath: "a/d/c1", LogSize: 100},
@ -183,41 +212,52 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() {
},
},
},
{SegmentID: 200, Level: datapb.SegmentLevel_L1},
{SegmentID: 201, Level: datapb.SegmentLevel_L1},
{
CollectionID: 1,
SegmentID: 200, Level: datapb.SegmentLevel_L1, Field2StatslogPaths: []*datapb.FieldBinlog{
{
Binlogs: []*datapb.Binlog{
{LogID: 9999, LogSize: 100},
},
},
},
},
{
CollectionID: 1,
SegmentID: 201, Level: datapb.SegmentLevel_L1, Field2StatslogPaths: []*datapb.FieldBinlog{
{
Binlogs: []*datapb.Binlog{
{LogID: 9999, LogSize: 100},
},
},
},
},
},
Schema: &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
IsPrimaryKey: true,
},
},
},
}
s.task.plan = plan
s.task.tr = timerecord.NewTimeRecorder("test")
bfs1 := metacache.NewBloomFilterSetWithBatchSize(100)
bfs1.UpdatePKRange(&storage.Int64FieldData{Data: []int64{1, 2}})
segment1 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 200}, bfs1)
bfs2 := metacache.NewBloomFilterSetWithBatchSize(100)
bfs2.UpdatePKRange(&storage.Int64FieldData{Data: []int64{1, 2}})
segment2 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 201}, bfs2)
data := &storage.Int64FieldData{
Data: []int64{1, 2, 3, 4, 5, 6, 7, 8, 9},
}
sw := &storage.StatsWriter{}
err := sw.GenerateByData(common.RowIDField, schemapb.DataType_Int64, data)
s.NoError(err)
cm := mocks.NewChunkManager(s.T())
cm.EXPECT().MultiRead(mock.Anything, mock.Anything).Return([][]byte{sw.GetBuffer()}, nil)
s.task.cm = cm
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return([][]byte{s.dBlob}, nil).Times(1)
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Twice()
times := 1
s.mockMeta.EXPECT().GetSegmentsBy(mock.Anything).RunAndReturn(
func(filters ...metacache.SegmentFilter) []*metacache.SegmentInfo {
if times == 1 {
times += 1
return []*metacache.SegmentInfo{segment1}
}
if times == 2 {
times += 1
return []*metacache.SegmentInfo{segment2}
}
return []*metacache.SegmentInfo{segment1, segment2}
}).Twice()
s.mockMeta.EXPECT().Collection().Return(1)
s.mockAlloc.EXPECT().AllocOne().Return(19530, nil).Times(2)
s.Require().Equal(plan.GetPlanID(), s.task.GetPlanID())
@ -228,11 +268,8 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() {
return s.Level == datapb.SegmentLevel_L0
})
targetSegIDs := lo.FilterMap(s.task.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) (int64, bool) {
if s.Level == datapb.SegmentLevel_L1 {
return s.GetSegmentID(), true
}
return 0, false
targetSegments := lo.Filter(s.task.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) bool {
return s.Level == datapb.SegmentLevel_L1
})
totalDeltalogs := make(map[int64][]string)
@ -247,7 +284,7 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() {
totalDeltalogs[s.GetSegmentID()] = paths
}
}
segments, err := s.task.process(context.Background(), 1, targetSegIDs, lo.Values(totalDeltalogs)...)
segments, err := s.task.process(context.Background(), 1, targetSegments, lo.Values(totalDeltalogs)...)
s.NoError(err)
s.NotEmpty(segments)
s.Equal(2, len(segments))
@ -255,6 +292,9 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() {
lo.Map(segments, func(seg *datapb.CompactionSegment, _ int) int64 {
return seg.GetSegmentID()
}))
for _, segment := range segments {
s.NotNil(segment.GetDeltalogs())
}
log.Info("test segment results", zap.Any("result", segments))
}
@ -288,28 +328,44 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactBatch() {
},
},
},
{SegmentID: 200, Level: datapb.SegmentLevel_L1},
{SegmentID: 201, Level: datapb.SegmentLevel_L1},
{SegmentID: 200, Level: datapb.SegmentLevel_L1, Field2StatslogPaths: []*datapb.FieldBinlog{
{
Binlogs: []*datapb.Binlog{
{LogID: 9999, LogSize: 100},
},
},
}},
{SegmentID: 201, Level: datapb.SegmentLevel_L1, Field2StatslogPaths: []*datapb.FieldBinlog{
{
Binlogs: []*datapb.Binlog{
{LogID: 9999, LogSize: 100},
},
},
}},
},
Schema: &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
IsPrimaryKey: true,
},
},
},
}
s.task.plan = plan
s.task.tr = timerecord.NewTimeRecorder("test")
s.mockMeta.EXPECT().GetSegmentsBy(mock.Anything).RunAndReturn(
func(filters ...metacache.SegmentFilter) []*metacache.SegmentInfo {
bfs1 := metacache.NewBloomFilterSetWithBatchSize(100)
bfs1.UpdatePKRange(&storage.Int64FieldData{Data: []int64{1, 2, 3}})
segment1 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 200}, bfs1)
bfs2 := metacache.NewBloomFilterSetWithBatchSize(100)
bfs2.UpdatePKRange(&storage.Int64FieldData{Data: []int64{1, 2, 3}})
segment2 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 201}, bfs2)
return []*metacache.SegmentInfo{segment1, segment2}
})
data := &storage.Int64FieldData{
Data: []int64{1, 2, 3, 4, 5, 6, 7, 8, 9},
}
sw := &storage.StatsWriter{}
err := sw.GenerateByData(common.RowIDField, schemapb.DataType_Int64, data)
s.NoError(err)
cm := mocks.NewChunkManager(s.T())
cm.EXPECT().MultiRead(mock.Anything, mock.Anything).Return([][]byte{sw.GetBuffer()}, nil)
s.task.cm = cm
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return([][]byte{s.dBlob}, nil).Once()
s.mockMeta.EXPECT().Collection().Return(1)
s.mockAlloc.EXPECT().AllocOne().Return(19530, nil).Times(2)
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Once()
@ -317,11 +373,8 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactBatch() {
return s.Level == datapb.SegmentLevel_L0
})
targetSegIDs := lo.FilterMap(s.task.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) (int64, bool) {
if s.Level == datapb.SegmentLevel_L1 {
return s.GetSegmentID(), true
}
return 0, false
targetSegments := lo.Filter(s.task.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) bool {
return s.Level == datapb.SegmentLevel_L1
})
totalDeltalogs := make(map[int64][]string)
@ -336,7 +389,7 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactBatch() {
totalDeltalogs[s.GetSegmentID()] = paths
}
}
segments, err := s.task.process(context.TODO(), 2, targetSegIDs, lo.Values(totalDeltalogs)...)
segments, err := s.task.process(context.TODO(), 2, targetSegments, lo.Values(totalDeltalogs)...)
s.NoError(err)
s.NotEmpty(segments)
s.Equal(2, len(segments))
@ -344,12 +397,26 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactBatch() {
lo.Map(segments, func(seg *datapb.CompactionSegment, _ int) int64 {
return seg.GetSegmentID()
}))
for _, segment := range segments {
s.NotNil(segment.GetDeltalogs())
}
log.Info("test segment results", zap.Any("result", segments))
}
func (s *LevelZeroCompactionTaskSuite) TestSerializeUpload() {
ctx := context.Background()
plan := &datapb.CompactionPlan{
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
{
SegmentID: 100,
},
{
SegmentID: 101,
},
},
}
s.Run("serializeUpload allocator Alloc failed", func() {
s.SetupTest()
s.mockAlloc.EXPECT().AllocOne().Return(0, errors.New("mock alloc wrong"))
@ -364,6 +431,7 @@ func (s *LevelZeroCompactionTaskSuite) TestSerializeUpload() {
s.Run("serializeUpload Upload failed", func() {
s.SetupTest()
s.task.plan = plan
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(errors.New("mock upload failed"))
s.mockAlloc.EXPECT().AllocOne().Return(19530, nil)
@ -377,6 +445,7 @@ func (s *LevelZeroCompactionTaskSuite) TestSerializeUpload() {
s.Run("upload success", func() {
s.SetupTest()
s.task.plan = plan
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil)
s.mockAlloc.EXPECT().AllocOne().Return(19530, nil)
writer := NewSegmentDeltaWriter(100, 10, 1)
@ -394,22 +463,30 @@ func (s *LevelZeroCompactionTaskSuite) TestSerializeUpload() {
}
func (s *LevelZeroCompactionTaskSuite) TestSplitDelta() {
plan := &datapb.CompactionPlan{
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
{
CollectionID: 1,
},
},
}
s.task.plan = plan
bfs1 := metacache.NewBloomFilterSetWithBatchSize(100)
bfs1.UpdatePKRange(&storage.Int64FieldData{Data: []int64{1, 3}})
segment1 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 100}, bfs1)
bfs2 := metacache.NewBloomFilterSetWithBatchSize(100)
bfs2.UpdatePKRange(&storage.Int64FieldData{Data: []int64{3}})
segment2 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 101}, bfs2)
bfs3 := metacache.NewBloomFilterSetWithBatchSize(100)
bfs3.UpdatePKRange(&storage.Int64FieldData{Data: []int64{3}})
segment3 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 102}, bfs3)
predicted := []int64{100, 101, 102}
s.mockMeta.EXPECT().GetSegmentsBy(mock.Anything).Return([]*metacache.SegmentInfo{segment1, segment2, segment3})
s.mockMeta.EXPECT().Collection().Return(1)
targetSegIDs := predicted
deltaWriters := s.task.splitDelta(context.TODO(), s.dData, targetSegIDs)
segmentBfs := map[int64]*metacache.BloomFilterSet{
100: bfs1,
101: bfs2,
102: bfs3,
}
deltaWriters := s.task.splitDelta(context.TODO(), s.dData, segmentBfs)
s.NotEmpty(deltaWriters)
s.ElementsMatch(predicted, lo.Keys(deltaWriters))
@ -465,3 +542,92 @@ func (s *LevelZeroCompactionTaskSuite) TestLoadDelta() {
}
}
}
func (s *LevelZeroCompactionTaskSuite) TestLoadBF() {
plan := &datapb.CompactionPlan{
PlanID: 19530,
Type: datapb.CompactionType_Level0DeleteCompaction,
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
{SegmentID: 201, Level: datapb.SegmentLevel_L1, Field2StatslogPaths: []*datapb.FieldBinlog{
{
Binlogs: []*datapb.Binlog{
{LogID: 9999, LogSize: 100},
},
},
}},
},
Schema: &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
IsPrimaryKey: true,
},
},
},
}
s.task.plan = plan
data := &storage.Int64FieldData{
Data: []int64{1, 2, 3, 4, 5, 6, 7, 8, 9},
}
sw := &storage.StatsWriter{}
err := sw.GenerateByData(common.RowIDField, schemapb.DataType_Int64, data)
s.NoError(err)
cm := mocks.NewChunkManager(s.T())
cm.EXPECT().MultiRead(mock.Anything, mock.Anything).Return([][]byte{sw.GetBuffer()}, nil)
s.task.cm = cm
bfs, err := s.task.loadBF(plan.SegmentBinlogs)
s.NoError(err)
s.Len(bfs, 1)
for _, pk := range s.dData.Pks {
lc := storage.NewLocationsCache(pk)
s.True(bfs[201].PkExists(lc))
}
}
func (s *LevelZeroCompactionTaskSuite) TestFailed() {
s.Run("no primary key", func() {
plan := &datapb.CompactionPlan{
PlanID: 19530,
Type: datapb.CompactionType_Level0DeleteCompaction,
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
{SegmentID: 201, Level: datapb.SegmentLevel_L1, Field2StatslogPaths: []*datapb.FieldBinlog{
{
Binlogs: []*datapb.Binlog{
{LogID: 9999, LogSize: 100},
},
},
}},
},
Schema: &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
IsPrimaryKey: false,
},
},
},
}
s.task.plan = plan
_, err := s.task.loadBF(plan.SegmentBinlogs)
s.Error(err)
})
s.Run("no l1 segments", func() {
plan := &datapb.CompactionPlan{
PlanID: 19530,
Type: datapb.CompactionType_Level0DeleteCompaction,
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
{SegmentID: 201, Level: datapb.SegmentLevel_L0},
},
}
s.task.plan = plan
_, err := s.task.Compact()
s.Error(err)
})
}

View File

@ -21,7 +21,6 @@ import (
"fmt"
sio "io"
"strconv"
"sync"
"time"
"github.com/cockroachdb/errors"
@ -33,15 +32,12 @@ import (
"github.com/milvus-io/milvus/internal/datanode/allocator"
"github.com/milvus-io/milvus/internal/datanode/io"
iter "github.com/milvus-io/milvus/internal/datanode/iterators"
"github.com/milvus-io/milvus/internal/datanode/metacache"
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/timerecord"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
@ -51,9 +47,6 @@ import (
// for MixCompaction only
type mixCompactionTask struct {
binlogIO io.BinlogIO
Compactor
metaCache metacache.MetaCache
syncMgr syncmgr.SyncManager
allocator.Allocator
currentTs typeutil.Timestamp
@ -62,9 +55,8 @@ type mixCompactionTask struct {
ctx context.Context
cancel context.CancelFunc
injectDoneOnce sync.Once
done chan struct{}
tr *timerecord.TimeRecorder
done chan struct{}
tr *timerecord.TimeRecorder
}
// make sure compactionTask implements compactor interface
@ -73,8 +65,6 @@ var _ Compactor = (*mixCompactionTask)(nil)
func NewMixCompactionTask(
ctx context.Context,
binlogIO io.BinlogIO,
metaCache metacache.MetaCache,
syncMgr syncmgr.SyncManager,
alloc allocator.Allocator,
plan *datapb.CompactionPlan,
) *mixCompactionTask {
@ -83,8 +73,6 @@ func NewMixCompactionTask(
ctx: ctx1,
cancel: cancel,
binlogIO: binlogIO,
syncMgr: syncMgr,
metaCache: metaCache,
Allocator: alloc,
plan: plan,
tr: timerecord.NewTimeRecorder("mix compaction"),
@ -100,7 +88,6 @@ func (t *mixCompactionTask) Complete() {
func (t *mixCompactionTask) Stop() {
t.cancel()
<-t.done
t.InjectDone()
}
func (t *mixCompactionTask) GetPlanID() typeutil.UniqueID {
@ -112,18 +99,16 @@ func (t *mixCompactionTask) GetChannelName() string {
}
// return num rows of all segment compaction from
func (t *mixCompactionTask) getNumRows() (int64, error) {
func (t *mixCompactionTask) getNumRows() int64 {
numRows := int64(0)
for _, binlog := range t.plan.SegmentBinlogs {
seg, ok := t.metaCache.GetSegmentByID(binlog.GetSegmentID())
if !ok {
return 0, merr.WrapErrSegmentNotFound(binlog.GetSegmentID(), "get compaction segments num rows failed")
if len(binlog.GetFieldBinlogs()) > 0 {
for _, ct := range binlog.GetFieldBinlogs()[0].GetBinlogs() {
numRows += ct.GetEntriesNum()
}
}
numRows += seg.NumOfRows()
}
return numRows, nil
return numRows
}
func (t *mixCompactionTask) mergeDeltalogs(ctx context.Context, dpaths map[typeutil.UniqueID][]string) (map[interface{}]typeutil.Timestamp, error) {
@ -417,7 +402,19 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, fmt.Sprintf("MixCompact-%d", t.GetPlanID()))
defer span.End()
log := log.Ctx(ctx).With(zap.Int64("planID", t.plan.GetPlanID()), zap.Int32("timeout in seconds", t.plan.GetTimeoutInSeconds()))
if len(t.plan.GetSegmentBinlogs()) < 1 {
log.Warn("compact wrong, there's no segments in segment binlogs", zap.Int64("planID", t.plan.GetPlanID()))
return nil, errors.New("compaction plan is illegal")
}
collectionID := t.plan.GetSegmentBinlogs()[0].GetCollectionID()
partitionID := t.plan.GetSegmentBinlogs()[0].GetPartitionID()
log := log.Ctx(ctx).With(zap.Int64("planID", t.plan.GetPlanID()),
zap.Int64("collectionID", collectionID),
zap.Int64("partitionID", partitionID),
zap.Int32("timeout in seconds", t.plan.GetTimeoutInSeconds()))
if ok := funcutil.CheckCtxValid(ctx); !ok {
log.Warn("compact wrong, task context done or timeout")
return nil, ctx.Err()
@ -427,10 +424,6 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
defer cancelAll()
log.Info("compact start")
if len(t.plan.GetSegmentBinlogs()) < 1 {
log.Warn("compact wrong, there's no segments in segment binlogs")
return nil, errors.New("compaction plan is illegal")
}
targetSegID, err := t.AllocOne()
if err != nil {
@ -438,15 +431,9 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
return nil, err
}
previousRowCount, err := t.getNumRows()
if err != nil {
log.Warn("compact wrong, unable to get previous numRows", zap.Error(err))
return nil, err
}
previousRowCount := t.getNumRows()
partID := t.plan.GetSegmentBinlogs()[0].GetPartitionID()
writer, err := NewSegmentWriter(t.metaCache.Schema(), previousRowCount, targetSegID, partID, t.metaCache.Collection())
writer, err := NewSegmentWriter(t.plan.GetSchema(), previousRowCount, targetSegID, partitionID, collectionID)
if err != nil {
log.Warn("compact wrong, unable to init segment writer", zap.Error(err))
return nil, err
@ -455,12 +442,6 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
segIDs := lo.Map(t.plan.GetSegmentBinlogs(), func(binlogs *datapb.CompactionSegmentBinlogs, _ int) int64 {
return binlogs.GetSegmentID()
})
// Inject to stop flush
// when compaction failed, these segments need to be Unblocked by injectDone in compaction_executor
// when compaction succeeded, these segments will be Unblocked by SyncSegments from DataCoord.
for _, segID := range segIDs {
t.syncMgr.Block(segID)
}
if err := binlog.DecompressCompactionBinlogs(t.plan.GetSegmentBinlogs()); err != nil {
log.Warn("compact wrong, fail to decompress compaction binlogs", zap.Error(err))
@ -541,16 +522,9 @@ func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) {
return planResult, nil
}
func (t *mixCompactionTask) InjectDone() {
t.injectDoneOnce.Do(func() {
for _, binlog := range t.plan.SegmentBinlogs {
t.syncMgr.Unblock(binlog.SegmentID)
}
})
}
func (t *mixCompactionTask) GetCollection() typeutil.UniqueID {
return t.metaCache.Collection()
// The length of SegmentBinlogs is checked before task enqueueing.
return t.plan.GetSegmentBinlogs()[0].GetCollectionID()
}
func (t *mixCompactionTask) isExpiredEntity(ts typeutil.Timestamp) bool {

View File

@ -32,12 +32,10 @@ import (
"github.com/milvus-io/milvus/internal/datanode/allocator"
"github.com/milvus-io/milvus/internal/datanode/io"
"github.com/milvus-io/milvus/internal/datanode/metacache"
"github.com/milvus-io/milvus/internal/datanode/syncmgr"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
@ -54,8 +52,6 @@ type MixCompactionTaskSuite struct {
mockBinlogIO *io.MockBinlogIO
mockAlloc *allocator.MockAllocator
mockMeta *metacache.MockMetaCache
mockSyncMgr *syncmgr.MockSyncManager
meta *etcdpb.CollectionMeta
segWriter *SegmentWriter
@ -71,10 +67,8 @@ func (s *MixCompactionTaskSuite) SetupSuite() {
func (s *MixCompactionTaskSuite) SetupTest() {
s.mockBinlogIO = io.NewMockBinlogIO(s.T())
s.mockAlloc = allocator.NewMockAllocator(s.T())
s.mockMeta = metacache.NewMockMetaCache(s.T())
s.mockSyncMgr = syncmgr.NewMockSyncManager(s.T())
s.task = NewMixCompactionTask(context.Background(), s.mockBinlogIO, s.mockMeta, s.mockSyncMgr, s.mockAlloc, nil)
s.task = NewMixCompactionTask(context.Background(), s.mockBinlogIO, s.mockAlloc, nil)
s.meta = genTestCollectionMeta()
@ -90,6 +84,7 @@ func (s *MixCompactionTaskSuite) SetupTest() {
}},
TimeoutInSeconds: 10,
Type: datapb.CompactionType_MixCompaction,
Schema: s.meta.GetSchema(),
}
s.task.plan = s.plan
}
@ -106,26 +101,10 @@ func getMilvusBirthday() time.Time {
return time.Date(2019, time.Month(5), 30, 0, 0, 0, 0, time.UTC)
}
func (s *MixCompactionTaskSuite) TestInjectDone() {
segmentIDs := []int64{100, 200, 300}
s.task.plan.SegmentBinlogs = lo.Map(segmentIDs, func(id int64, _ int) *datapb.CompactionSegmentBinlogs {
return &datapb.CompactionSegmentBinlogs{SegmentID: id}
})
for _, segmentID := range segmentIDs {
s.mockSyncMgr.EXPECT().Unblock(segmentID).Return().Once()
}
s.task.InjectDone()
s.task.InjectDone()
}
func (s *MixCompactionTaskSuite) TestCompactDupPK() {
// Test merge compactions, two segments with the same pk, one deletion pk=1
// The merged segment 19530 should remain 3 pk without pk=100
s.mockAlloc.EXPECT().AllocOne().Return(int64(19530), nil).Twice()
s.mockMeta.EXPECT().Schema().Return(s.meta.GetSchema()).Once()
s.mockMeta.EXPECT().Collection().Return(CollectionID).Once()
segments := []int64{7, 8, 9}
dblobs, err := getInt64DeltaBlobs(
1,
@ -155,12 +134,12 @@ func (s *MixCompactionTaskSuite) TestCompactDupPK() {
err = s.segWriter.writer.Flush()
s.Require().NoError(err)
statistic := &storage.PkStatistics{
PkFilter: s.segWriter.pkstats.BF,
MinPK: s.segWriter.pkstats.MinPk,
MaxPK: s.segWriter.pkstats.MaxPk,
}
bfs := metacache.NewBloomFilterSet(statistic)
//statistic := &storage.PkStatistics{
// PkFilter: s.segWriter.pkstats.BF,
// MinPK: s.segWriter.pkstats.MinPk,
// MaxPK: s.segWriter.pkstats.MaxPk,
//}
//bfs := metacache.NewBloomFilterSet(statistic)
kvs, fBinlogs, err := s.task.serializeWrite(context.TODO(), s.segWriter)
s.Require().NoError(err)
@ -169,17 +148,12 @@ func (s *MixCompactionTaskSuite) TestCompactDupPK() {
return len(left) == 0 && len(right) == 0
})).Return(lo.Values(kvs), nil).Once()
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{
CollectionID: CollectionID,
PartitionID: PartitionID,
ID: segID,
NumOfRows: 1,
}, bfs)
s.mockMeta.EXPECT().GetSegmentByID(segID).RunAndReturn(func(id int64, filters ...metacache.SegmentFilter) (*metacache.SegmentInfo, bool) {
return seg, true
})
s.mockSyncMgr.EXPECT().Block(segID).Return().Once()
//seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{
// CollectionID: CollectionID,
// PartitionID: PartitionID,
// ID: segID,
// NumOfRows: 1,
//}, bfs)
s.plan.SegmentBinlogs = append(s.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
SegmentID: segID,
@ -206,8 +180,6 @@ func (s *MixCompactionTaskSuite) TestCompactDupPK() {
func (s *MixCompactionTaskSuite) TestCompactTwoToOne() {
s.mockAlloc.EXPECT().AllocOne().Return(int64(19530), nil).Twice()
s.mockMeta.EXPECT().Schema().Return(s.meta.GetSchema()).Once()
s.mockMeta.EXPECT().Collection().Return(CollectionID).Once()
segments := []int64{5, 6, 7}
s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(7777777, 8888888, nil)
@ -215,12 +187,12 @@ func (s *MixCompactionTaskSuite) TestCompactTwoToOne() {
s.task.plan.SegmentBinlogs = make([]*datapb.CompactionSegmentBinlogs, 0)
for _, segID := range segments {
s.initSegBuffer(segID)
statistic := &storage.PkStatistics{
PkFilter: s.segWriter.pkstats.BF,
MinPK: s.segWriter.pkstats.MinPk,
MaxPK: s.segWriter.pkstats.MaxPk,
}
bfs := metacache.NewBloomFilterSet(statistic)
//statistic := &storage.PkStatistics{
// PkFilter: s.segWriter.pkstats.BF,
// MinPK: s.segWriter.pkstats.MinPk,
// MaxPK: s.segWriter.pkstats.MaxPk,
//}
//bfs := metacache.NewBloomFilterSet(statistic)
kvs, fBinlogs, err := s.task.serializeWrite(context.TODO(), s.segWriter)
s.Require().NoError(err)
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.MatchedBy(func(keys []string) bool {
@ -228,17 +200,12 @@ func (s *MixCompactionTaskSuite) TestCompactTwoToOne() {
return len(left) == 0 && len(right) == 0
})).Return(lo.Values(kvs), nil).Once()
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{
CollectionID: CollectionID,
PartitionID: PartitionID,
ID: segID,
NumOfRows: 1,
}, bfs)
s.mockMeta.EXPECT().GetSegmentByID(segID).RunAndReturn(func(id int64, filters ...metacache.SegmentFilter) (*metacache.SegmentInfo, bool) {
return seg, true
})
s.mockSyncMgr.EXPECT().Block(segID).Return().Once()
//seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{
// CollectionID: CollectionID,
// PartitionID: PartitionID,
// ID: segID,
// NumOfRows: 1,
//}, bfs)
s.plan.SegmentBinlogs = append(s.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
SegmentID: segID,
@ -253,10 +220,7 @@ func (s *MixCompactionTaskSuite) TestCompactTwoToOne() {
ID: 99999,
NumOfRows: 0,
}, metacache.NewBloomFilterSet())
s.mockMeta.EXPECT().GetSegmentByID(seg.SegmentID()).RunAndReturn(func(id int64, filters ...metacache.SegmentFilter) (*metacache.SegmentInfo, bool) {
return seg, true
})
s.mockSyncMgr.EXPECT().Block(seg.SegmentID()).Return().Once()
s.plan.SegmentBinlogs = append(s.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{
SegmentID: seg.SegmentID(),
})
@ -533,15 +497,6 @@ func (s *MixCompactionTaskSuite) TestCompactFail() {
_, err := s.task.Compact()
s.Error(err)
})
s.Run("Test getNumRows error", func() {
s.mockAlloc.EXPECT().AllocOne().Return(19530, nil).Once()
s.mockMeta.EXPECT().GetSegmentByID(mock.Anything).Return(nil, false)
_, err := s.task.Compact()
s.Error(err)
s.ErrorIs(err, merr.ErrSegmentNotFound)
})
}
func (s *MixCompactionTaskSuite) TestIsExpiredEntity() {

View File

@ -70,12 +70,11 @@ func (c *compactionExecutor) toCompleteState(task compaction.Compactor) {
c.executing.GetAndRemove(task.GetPlanID())
}
func (c *compactionExecutor) injectDone(planID UniqueID) {
func (c *compactionExecutor) removeTask(planID UniqueID) {
c.completed.GetAndRemove(planID)
task, loaded := c.completedCompactor.GetAndRemove(planID)
if loaded {
log.Info("Compaction task inject done", zap.Int64("planID", planID), zap.String("channel", task.GetChannelName()))
task.InjectDone()
log.Info("Compaction task removed", zap.Int64("planID", planID), zap.String("channel", task.GetChannelName()))
}
}
@ -110,12 +109,11 @@ func (c *compactionExecutor) executeTask(task compaction.Compactor) {
result, err := task.Compact()
if err != nil {
task.InjectDone()
log.Warn("compaction task failed", zap.Error(err))
} else {
c.completed.Insert(result.GetPlanID(), result)
c.completedCompactor.Insert(result.GetPlanID(), task)
return
}
c.completed.Insert(result.GetPlanID(), result)
c.completedCompactor.Insert(result.GetPlanID(), task)
log.Info("end to execute compaction")
}
@ -152,7 +150,7 @@ func (c *compactionExecutor) discardPlan(channel string) {
// remove all completed plans of channel
c.completed.Range(func(planID int64, result *datapb.CompactionPlanResult) bool {
if result.GetChannel() == channel {
c.injectDone(planID)
c.removeTask(planID)
log.Info("remove compaction plan and results",
zap.String("channel", channel),
zap.Int64("planID", planID))

View File

@ -93,12 +93,6 @@ func WithLevel(level datapb.SegmentLevel) SegmentFilter {
})
}
func WithCompacted() SegmentFilter {
return SegmentFilterFunc(func(info *SegmentInfo) bool {
return info.compactTo != 0
})
}
func WithNoSyncingTask() SegmentFilter {
return SegmentFilterFunc(func(info *SegmentInfo) bool {
return info.syncingTasks == 0
@ -137,12 +131,6 @@ func RollStats(newStats ...*storage.PrimaryKeyStats) SegmentAction {
}
}
func CompactTo(compactTo int64) SegmentAction {
return func(info *SegmentInfo) {
info.compactTo = compactTo
}
}
func StartSyncing(batchSize int64) SegmentAction {
return func(info *SegmentInfo) {
info.syncingRows += batchSize

View File

@ -89,11 +89,6 @@ func (s *SegmentActionSuite) TestActions() {
action = UpdateNumOfRows(numOfRows)
action(info)
s.Equal(numOfRows, info.NumOfRows())
compactTo := int64(1002)
action = CompactTo(compactTo)
action(info)
s.Equal(compactTo, info.CompactTo())
}
func (s *SegmentActionSuite) TestMergeActions() {

View File

@ -30,6 +30,7 @@ import (
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
//go:generate mockery --name=MetaCache --structname=MockMetaCache --output=./ --filename=mock_meta_cache.go --with-expecter --inpackage
type MetaCache interface {
// Collection returns collection id of metacache.
Collection() int64
@ -41,8 +42,6 @@ type MetaCache interface {
UpdateSegments(action SegmentAction, filters ...SegmentFilter)
// RemoveSegments removes segments matches the provided filter.
RemoveSegments(filters ...SegmentFilter) []int64
// CompactSegments transfers compaction segment results inside the metacache.
CompactSegments(newSegmentID, partitionID int64, numRows int64, bfs *BloomFilterSet, oldSegmentIDs ...int64)
// GetSegmentsBy returns segments statify the provided filters.
GetSegmentsBy(filters ...SegmentFilter) []*SegmentInfo
// GetSegmentByID returns segment with provided segment id if exists.
@ -113,42 +112,6 @@ func (c *metaCacheImpl) AddSegment(segInfo *datapb.SegmentInfo, factory PkStatsF
c.segmentInfos[segInfo.GetID()] = segment
}
func (c *metaCacheImpl) CompactSegments(newSegmentID, partitionID int64, numOfRows int64, bfs *BloomFilterSet, oldSegmentIDs ...int64) {
c.mu.Lock()
defer c.mu.Unlock()
compactTo := NullSegment
if numOfRows > 0 {
compactTo = newSegmentID
if _, ok := c.segmentInfos[newSegmentID]; !ok {
c.segmentInfos[newSegmentID] = &SegmentInfo{
segmentID: newSegmentID,
partitionID: partitionID,
state: commonpb.SegmentState_Flushed,
level: datapb.SegmentLevel_L1,
flushedRows: numOfRows,
startPosRecorded: true,
bfs: bfs,
}
}
log.Info("add compactTo segment info metacache", zap.Int64("segmentID", compactTo))
}
oldSet := typeutil.NewSet(oldSegmentIDs...)
for _, segment := range c.segmentInfos {
if oldSet.Contain(segment.segmentID) ||
oldSet.Contain(segment.compactTo) {
updated := segment.Clone()
updated.compactTo = compactTo
c.segmentInfos[segment.segmentID] = updated
log.Info("update segment compactTo",
zap.Int64("segmentID", segment.segmentID),
zap.Int64("originalCompactTo", segment.compactTo),
zap.Int64("compactTo", compactTo))
}
}
}
func (c *metaCacheImpl) RemoveSegments(filters ...SegmentFilter) []int64 {
if len(filters) == 0 {
log.Warn("remove segment without filters is not allowed", zap.Stack("callstack"))

View File

@ -103,27 +103,6 @@ func (s *MetaCacheSuite) TestMetaInfo() {
s.Equal(s.collSchema, s.cache.Schema())
}
func (s *MetaCacheSuite) TestCompactSegments() {
for i, seg := range s.newSegments {
// compaction from flushed[i], unflushed[i] and invalidSeg to new[i]
s.cache.CompactSegments(seg, s.partitionIDs[i], 100, NewBloomFilterSet(), s.flushedSegments[i], s.growingSegments[i], s.invaliedSeg)
}
for i, partitionID := range s.partitionIDs {
segs := s.cache.GetSegmentsBy(WithPartitionID(partitionID))
for _, seg := range segs {
if seg.SegmentID() == s.newSegments[i] {
s.Equal(commonpb.SegmentState_Flushed, seg.State())
s.Equal(int64(100), seg.NumOfRows())
s.Equal(datapb.SegmentLevel_L1, seg.Level())
}
if seg.SegmentID() == s.flushedSegments[i] {
s.Equal(s.newSegments[i], seg.CompactTo())
}
}
}
}
func (s *MetaCacheSuite) TestAddSegment() {
testSegs := []int64{100, 101, 102}
for _, segID := range testSegs {

View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
// Code generated by mockery v2.30.1. DO NOT EDIT.
package metacache
@ -42,9 +42,9 @@ type MockMetaCache_AddSegment_Call struct {
}
// AddSegment is a helper method to define mock.On call
// - segInfo *datapb.SegmentInfo
// - factory PkStatsFactory
// - actions ...SegmentAction
// - segInfo *datapb.SegmentInfo
// - factory PkStatsFactory
// - actions ...SegmentAction
func (_e *MockMetaCache_Expecter) AddSegment(segInfo interface{}, factory interface{}, actions ...interface{}) *MockMetaCache_AddSegment_Call {
return &MockMetaCache_AddSegment_Call{Call: _e.mock.On("AddSegment",
append([]interface{}{segInfo, factory}, actions...)...)}
@ -114,57 +114,6 @@ func (_c *MockMetaCache_Collection_Call) RunAndReturn(run func() int64) *MockMet
return _c
}
// CompactSegments provides a mock function with given fields: newSegmentID, partitionID, numRows, bfs, oldSegmentIDs
func (_m *MockMetaCache) CompactSegments(newSegmentID int64, partitionID int64, numRows int64, bfs *BloomFilterSet, oldSegmentIDs ...int64) {
_va := make([]interface{}, len(oldSegmentIDs))
for _i := range oldSegmentIDs {
_va[_i] = oldSegmentIDs[_i]
}
var _ca []interface{}
_ca = append(_ca, newSegmentID, partitionID, numRows, bfs)
_ca = append(_ca, _va...)
_m.Called(_ca...)
}
// MockMetaCache_CompactSegments_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CompactSegments'
type MockMetaCache_CompactSegments_Call struct {
*mock.Call
}
// CompactSegments is a helper method to define mock.On call
// - newSegmentID int64
// - partitionID int64
// - numRows int64
// - bfs *BloomFilterSet
// - oldSegmentIDs ...int64
func (_e *MockMetaCache_Expecter) CompactSegments(newSegmentID interface{}, partitionID interface{}, numRows interface{}, bfs interface{}, oldSegmentIDs ...interface{}) *MockMetaCache_CompactSegments_Call {
return &MockMetaCache_CompactSegments_Call{Call: _e.mock.On("CompactSegments",
append([]interface{}{newSegmentID, partitionID, numRows, bfs}, oldSegmentIDs...)...)}
}
func (_c *MockMetaCache_CompactSegments_Call) Run(run func(newSegmentID int64, partitionID int64, numRows int64, bfs *BloomFilterSet, oldSegmentIDs ...int64)) *MockMetaCache_CompactSegments_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]int64, len(args)-4)
for i, a := range args[4:] {
if a != nil {
variadicArgs[i] = a.(int64)
}
}
run(args[0].(int64), args[1].(int64), args[2].(int64), args[3].(*BloomFilterSet), variadicArgs...)
})
return _c
}
func (_c *MockMetaCache_CompactSegments_Call) Return() *MockMetaCache_CompactSegments_Call {
_c.Call.Return()
return _c
}
func (_c *MockMetaCache_CompactSegments_Call) RunAndReturn(run func(int64, int64, int64, *BloomFilterSet, ...int64)) *MockMetaCache_CompactSegments_Call {
_c.Call.Return(run)
return _c
}
// GetSegmentByID provides a mock function with given fields: id, filters
func (_m *MockMetaCache) GetSegmentByID(id int64, filters ...SegmentFilter) (*SegmentInfo, bool) {
_va := make([]interface{}, len(filters))
@ -204,8 +153,8 @@ type MockMetaCache_GetSegmentByID_Call struct {
}
// GetSegmentByID is a helper method to define mock.On call
// - id int64
// - filters ...SegmentFilter
// - id int64
// - filters ...SegmentFilter
func (_e *MockMetaCache_Expecter) GetSegmentByID(id interface{}, filters ...interface{}) *MockMetaCache_GetSegmentByID_Call {
return &MockMetaCache_GetSegmentByID_Call{Call: _e.mock.On("GetSegmentByID",
append([]interface{}{id}, filters...)...)}
@ -262,7 +211,7 @@ type MockMetaCache_GetSegmentIDsBy_Call struct {
}
// GetSegmentIDsBy is a helper method to define mock.On call
// - filters ...SegmentFilter
// - filters ...SegmentFilter
func (_e *MockMetaCache_Expecter) GetSegmentIDsBy(filters ...interface{}) *MockMetaCache_GetSegmentIDsBy_Call {
return &MockMetaCache_GetSegmentIDsBy_Call{Call: _e.mock.On("GetSegmentIDsBy",
append([]interface{}{}, filters...)...)}
@ -319,7 +268,7 @@ type MockMetaCache_GetSegmentsBy_Call struct {
}
// GetSegmentsBy is a helper method to define mock.On call
// - filters ...SegmentFilter
// - filters ...SegmentFilter
func (_e *MockMetaCache_Expecter) GetSegmentsBy(filters ...interface{}) *MockMetaCache_GetSegmentsBy_Call {
return &MockMetaCache_GetSegmentsBy_Call{Call: _e.mock.On("GetSegmentsBy",
append([]interface{}{}, filters...)...)}
@ -387,8 +336,8 @@ type MockMetaCache_PredictSegments_Call struct {
}
// PredictSegments is a helper method to define mock.On call
// - pk storage.PrimaryKey
// - filters ...SegmentFilter
// - pk storage.PrimaryKey
// - filters ...SegmentFilter
func (_e *MockMetaCache_Expecter) PredictSegments(pk interface{}, filters ...interface{}) *MockMetaCache_PredictSegments_Call {
return &MockMetaCache_PredictSegments_Call{Call: _e.mock.On("PredictSegments",
append([]interface{}{pk}, filters...)...)}
@ -445,7 +394,7 @@ type MockMetaCache_RemoveSegments_Call struct {
}
// RemoveSegments is a helper method to define mock.On call
// - filters ...SegmentFilter
// - filters ...SegmentFilter
func (_e *MockMetaCache_Expecter) RemoveSegments(filters ...interface{}) *MockMetaCache_RemoveSegments_Call {
return &MockMetaCache_RemoveSegments_Call{Call: _e.mock.On("RemoveSegments",
append([]interface{}{}, filters...)...)}
@ -535,8 +484,8 @@ type MockMetaCache_UpdateSegments_Call struct {
}
// UpdateSegments is a helper method to define mock.On call
// - action SegmentAction
// - filters ...SegmentFilter
// - action SegmentAction
// - filters ...SegmentFilter
func (_e *MockMetaCache_Expecter) UpdateSegments(action interface{}, filters ...interface{}) *MockMetaCache_UpdateSegments_Call {
return &MockMetaCache_UpdateSegments_Call{Call: _e.mock.On("UpdateSegments",
append([]interface{}{action}, filters...)...)}

View File

@ -23,12 +23,6 @@ import (
"github.com/milvus-io/milvus/internal/storage"
)
const (
// NullSegment means the segment id to discard
// happens when segment compacted to 0 lines and target segment is dropped directly
NullSegment = int64(-1)
)
type SegmentInfo struct {
segmentID int64
partitionID int64
@ -40,7 +34,6 @@ type SegmentInfo struct {
bufferRows int64
syncingRows int64
bfs *BloomFilterSet
compactTo int64
level datapb.SegmentLevel
syncingTasks int32
}
@ -80,10 +73,6 @@ func (s *SegmentInfo) GetHistory() []*storage.PkStatistics {
return s.bfs.GetHistory()
}
func (s *SegmentInfo) CompactTo() int64 {
return s.compactTo
}
func (s *SegmentInfo) GetBloomFilterSet() *BloomFilterSet {
return s.bfs
}
@ -104,7 +93,6 @@ func (s *SegmentInfo) Clone() *SegmentInfo {
bufferRows: s.bufferRows,
syncingRows: s.syncingRows,
bfs: s.bfs,
compactTo: s.compactTo,
level: s.level,
syncingTasks: s.syncingTasks,
}

View File

@ -30,12 +30,8 @@ import (
"github.com/milvus-io/milvus/internal/datanode/compaction"
"github.com/milvus-io/milvus/internal/datanode/importv2"
"github.com/milvus-io/milvus/internal/datanode/io"
"github.com/milvus-io/milvus/internal/datanode/metacache"
"github.com/milvus-io/milvus/internal/datanode/util"
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/internalpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/importutilv2"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
@ -198,38 +194,18 @@ func (node *DataNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRe
}, nil
}
// Compaction handles compaction request from DataCoord
// CompactionV2 handles compaction request from DataCoord
// returns status as long as compaction task enqueued or invalid
func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) {
func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) {
log := log.Ctx(ctx).With(zap.Int64("planID", req.GetPlanID()))
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
log.Warn("DataNode.Compaction failed", zap.Int64("nodeId", node.GetNodeID()), zap.Error(err))
return merr.Status(err), nil
}
ds, ok := node.flowgraphManager.GetFlowgraphService(req.GetChannel())
if !ok {
log.Warn("illegel compaction plan, channel not in this DataNode", zap.String("channelName", req.GetChannel()))
return merr.Status(merr.WrapErrChannelNotFound(req.GetChannel(), "illegel compaction plan")), nil
}
if !node.compactionExecutor.isValidChannel(req.GetChannel()) {
log.Warn("channel of compaction is marked invalid in compaction executor", zap.String("channelName", req.GetChannel()))
return merr.Status(merr.WrapErrChannelNotFound(req.GetChannel(), "channel is dropping")), nil
}
meta := ds.metacache
for _, segment := range req.GetSegmentBinlogs() {
if segment.GetLevel() == datapb.SegmentLevel_L0 {
continue
}
_, ok := meta.GetSegmentByID(segment.GetSegmentID(), metacache.WithSegmentState(commonpb.SegmentState_Flushed))
if !ok {
log.Warn("compaction plan contains segment which is not flushed",
zap.Int64("segmentID", segment.GetSegmentID()),
)
return merr.Status(merr.WrapErrSegmentNotFound(segment.GetSegmentID(), "segment with flushed state not found")), nil
}
if len(req.GetSegmentBinlogs()) == 0 {
log.Info("no segments to compact")
return merr.Success(), nil
}
/*
@ -246,16 +222,13 @@ func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan
taskCtx,
binlogIO,
node.allocator,
ds.metacache,
node.syncMgr,
node.chunkManager,
req,
)
case datapb.CompactionType_MixCompaction:
task = compaction.NewMixCompactionTask(
taskCtx,
binlogIO,
ds.metacache,
node.syncMgr,
node.allocator,
req,
)
@ -289,10 +262,6 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments
log := log.Ctx(ctx).With(
zap.Int64("planID", req.GetPlanID()),
zap.Int64("nodeID", node.GetNodeID()),
zap.Int64("target segmentID", req.GetCompactedTo()),
zap.Int64s("compacted from", req.GetCompactedFrom()),
zap.Int64("numOfRows", req.GetNumOfRows()),
zap.String("channelName", req.GetChannelName()),
)
log.Info("DataNode receives SyncSegments")
@ -302,35 +271,8 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments
return merr.Status(err), nil
}
if len(req.GetCompactedFrom()) <= 0 {
log.Info("SyncSegments with empty compactedFrom, clearing the plan")
node.compactionExecutor.injectDone(req.GetPlanID())
return merr.Success(), nil
}
ds, ok := node.flowgraphManager.GetFlowgraphService(req.GetChannelName())
if !ok {
node.compactionExecutor.discardPlan(req.GetChannelName())
err := merr.WrapErrChannelNotFound(req.GetChannelName())
log.Warn("failed to sync segments", zap.Error(err))
return merr.Status(err), nil
}
err := binlog.DecompressBinLog(storage.StatsBinlog, req.GetCollectionId(), req.GetPartitionId(), req.GetCompactedTo(), req.GetStatsLogs())
if err != nil {
log.Warn("failed to DecompressBinLog", zap.Error(err))
return merr.Status(err), nil
}
pks, err := util.LoadStats(ctx, node.chunkManager, ds.metacache.Schema(), req.GetCompactedTo(), req.GetStatsLogs())
if err != nil {
log.Warn("failed to load segment stats log", zap.Error(err))
return merr.Status(err), nil
}
bfs := metacache.NewBloomFilterSet(pks...)
ds.metacache.CompactSegments(req.GetCompactedTo(), req.GetPartitionId(), req.GetNumOfRows(), bfs, req.GetCompactedFrom()...)
node.compactionExecutor.injectDone(req.GetPlanID())
// TODO: sheep, add a new DropCompaction interface, deprecate SyncSegments
node.compactionExecutor.removeTask(req.GetPlanID())
return merr.Success(), nil
}
@ -534,3 +476,16 @@ func (node *DataNode) DropImport(ctx context.Context, req *datapb.DropImportRequ
return merr.Success(), nil
}
func (node *DataNode) QuerySlot(ctx context.Context, req *datapb.QuerySlotRequest) (*datapb.QuerySlotResponse, error) {
if err := merr.CheckHealthy(node.GetStateCode()); err != nil {
return &datapb.QuerySlotResponse{
Status: merr.Status(err),
}, nil
}
return &datapb.QuerySlotResponse{
Status: merr.Success(),
NumSlots: Params.DataNodeCfg.SlotCap.GetAsInt64() - int64(node.compactionExecutor.executing.Len()),
}, nil
}

View File

@ -210,50 +210,7 @@ func (s *DataNodeServicesSuite) TestGetCompactionState() {
func (s *DataNodeServicesSuite) TestCompaction() {
dmChannelName := "by-dev-rootcoord-dml_0_100v0"
schema := &schemapb.CollectionSchema{
Name: "test_collection",
Fields: []*schemapb.FieldSchema{
{FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64},
{FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64},
{FieldID: common.StartOfUserFieldID, DataType: schemapb.DataType_Int64, IsPrimaryKey: true, Name: "pk"},
{FieldID: common.StartOfUserFieldID + 1, DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
{Key: common.DimKey, Value: "128"},
}},
},
}
flushedSegmentID := int64(100)
growingSegmentID := int64(101)
vchan := &datapb.VchannelInfo{
CollectionID: 1,
ChannelName: dmChannelName,
UnflushedSegmentIds: []int64{},
FlushedSegmentIds: []int64{},
}
err := s.node.flowgraphManager.AddandStartWithEtcdTickler(s.node, vchan, schema, genTestTickler())
s.Require().NoError(err)
fgservice, ok := s.node.flowgraphManager.GetFlowgraphService(dmChannelName)
s.Require().True(ok)
metaCache := metacache.NewMockMetaCache(s.T())
metaCache.EXPECT().Collection().Return(1).Maybe()
metaCache.EXPECT().Schema().Return(schema).Maybe()
s.node.writeBufferManager.Register(dmChannelName, metaCache, nil)
fgservice.metacache.AddSegment(&datapb.SegmentInfo{
ID: flushedSegmentID,
CollectionID: 1,
PartitionID: 2,
StartPosition: &msgpb.MsgPosition{},
}, func(_ *datapb.SegmentInfo) *metacache.BloomFilterSet { return metacache.NewBloomFilterSet() })
fgservice.metacache.AddSegment(&datapb.SegmentInfo{
ID: growingSegmentID,
CollectionID: 1,
PartitionID: 2,
StartPosition: &msgpb.MsgPosition{},
}, func(_ *datapb.SegmentInfo) *metacache.BloomFilterSet { return metacache.NewBloomFilterSet() })
s.Run("service_not_ready", func() {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@ -264,45 +221,12 @@ func (s *DataNodeServicesSuite) TestCompaction() {
Channel: dmChannelName,
}
resp, err := node.Compaction(ctx, req)
resp, err := node.CompactionV2(ctx, req)
s.NoError(err)
s.False(merr.Ok(resp))
})
s.Run("channel_not_match", func() {
node := s.node
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
req := &datapb.CompactionPlan{
PlanID: 1000,
Channel: dmChannelName + "other",
}
resp, err := node.Compaction(ctx, req)
s.NoError(err)
s.False(merr.Ok(resp))
})
s.Run("channel_dropped", func() {
node := s.node
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
node.compactionExecutor.dropped.Insert(dmChannelName)
defer node.compactionExecutor.dropped.Remove(dmChannelName)
req := &datapb.CompactionPlan{
PlanID: 1000,
Channel: dmChannelName,
}
resp, err := node.Compaction(ctx, req)
s.NoError(err)
s.False(merr.Ok(resp))
})
s.Run("compact_growing_segment", func() {
s.Run("unknown CompactionType", func() {
node := s.node
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
@ -312,11 +236,11 @@ func (s *DataNodeServicesSuite) TestCompaction() {
Channel: dmChannelName,
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
{SegmentID: 102, Level: datapb.SegmentLevel_L0},
{SegmentID: growingSegmentID, Level: datapb.SegmentLevel_L1},
{SegmentID: 103, Level: datapb.SegmentLevel_L1},
},
}
resp, err := node.Compaction(ctx, req)
resp, err := node.CompactionV2(ctx, req)
s.NoError(err)
s.False(merr.Ok(resp))
})
@ -506,126 +430,6 @@ func (s *DataNodeServicesSuite) TestGetMetrics() {
zap.String("response", resp.Response))
}
func (s *DataNodeServicesSuite) TestSyncSegments() {
chanName := "fake-by-dev-rootcoord-dml-test-syncsegments-1"
schema := &schemapb.CollectionSchema{
Name: "test_collection",
Fields: []*schemapb.FieldSchema{
{FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64},
{FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64},
{FieldID: common.StartOfUserFieldID, DataType: schemapb.DataType_Int64, IsPrimaryKey: true, Name: "pk"},
{FieldID: common.StartOfUserFieldID + 1, DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
{Key: common.DimKey, Value: "128"},
}},
},
}
err := s.node.flowgraphManager.AddandStartWithEtcdTickler(s.node, &datapb.VchannelInfo{
CollectionID: 1,
ChannelName: chanName,
UnflushedSegmentIds: []int64{},
FlushedSegmentIds: []int64{100, 200, 300},
}, schema, genTestTickler())
s.Require().NoError(err)
fg, ok := s.node.flowgraphManager.GetFlowgraphService(chanName)
s.Assert().True(ok)
fg.metacache.AddSegment(&datapb.SegmentInfo{ID: 100, CollectionID: 1, State: commonpb.SegmentState_Flushed}, EmptyBfsFactory)
fg.metacache.AddSegment(&datapb.SegmentInfo{ID: 101, CollectionID: 1, State: commonpb.SegmentState_Flushed}, EmptyBfsFactory)
fg.metacache.AddSegment(&datapb.SegmentInfo{ID: 200, CollectionID: 1, State: commonpb.SegmentState_Flushed}, EmptyBfsFactory)
fg.metacache.AddSegment(&datapb.SegmentInfo{ID: 201, CollectionID: 1, State: commonpb.SegmentState_Flushed}, EmptyBfsFactory)
fg.metacache.AddSegment(&datapb.SegmentInfo{ID: 300, CollectionID: 1, State: commonpb.SegmentState_Flushed}, EmptyBfsFactory)
s.Run("empty compactedFrom", func() {
req := &datapb.SyncSegmentsRequest{
CompactedTo: 400,
NumOfRows: 100,
}
req.CompactedFrom = []UniqueID{}
status, err := s.node.SyncSegments(s.ctx, req)
s.Assert().NoError(err)
s.Assert().True(merr.Ok(status))
})
s.Run("invalid compacted from", func() {
req := &datapb.SyncSegmentsRequest{
CompactedTo: 400,
NumOfRows: 100,
CompactedFrom: []UniqueID{101, 201},
}
req.CompactedFrom = []UniqueID{101, 201}
status, err := s.node.SyncSegments(s.ctx, req)
s.Assert().NoError(err)
s.Assert().False(merr.Ok(status))
})
s.Run("valid request numRows>0", func() {
req := &datapb.SyncSegmentsRequest{
CompactedFrom: []UniqueID{100, 200, 101, 201},
CompactedTo: 102,
NumOfRows: 100,
ChannelName: chanName,
CollectionId: 1,
}
status, err := s.node.SyncSegments(s.ctx, req)
s.Assert().NoError(err)
s.Assert().True(merr.Ok(status))
_, result := fg.metacache.GetSegmentByID(req.GetCompactedTo(), metacache.WithSegmentState(commonpb.SegmentState_Flushed))
s.True(result)
for _, compactFrom := range req.GetCompactedFrom() {
seg, result := fg.metacache.GetSegmentByID(compactFrom, metacache.WithSegmentState(commonpb.SegmentState_Flushed))
s.True(result)
s.Equal(req.CompactedTo, seg.CompactTo())
}
status, err = s.node.SyncSegments(s.ctx, req)
s.Assert().NoError(err)
s.Assert().True(merr.Ok(status))
})
s.Run("without_channel_meta", func() {
fg.metacache.UpdateSegments(metacache.UpdateState(commonpb.SegmentState_Flushed),
metacache.WithSegmentIDs(100, 200, 300))
req := &datapb.SyncSegmentsRequest{
CompactedFrom: []int64{100, 200},
CompactedTo: 101,
NumOfRows: 0,
}
status, err := s.node.SyncSegments(s.ctx, req)
s.Assert().NoError(err)
s.Assert().False(merr.Ok(status))
})
s.Run("valid_request_with_meta_num=0", func() {
fg.metacache.UpdateSegments(metacache.UpdateState(commonpb.SegmentState_Flushed),
metacache.WithSegmentIDs(100, 200, 300))
req := &datapb.SyncSegmentsRequest{
CompactedFrom: []int64{100, 200},
CompactedTo: 301,
NumOfRows: 0,
ChannelName: chanName,
CollectionId: 1,
}
status, err := s.node.SyncSegments(s.ctx, req)
s.Assert().NoError(err)
s.Assert().True(merr.Ok(status))
seg, result := fg.metacache.GetSegmentByID(100, metacache.WithSegmentState(commonpb.SegmentState_Flushed))
s.True(result)
s.Equal(metacache.NullSegment, seg.CompactTo())
seg, result = fg.metacache.GetSegmentByID(200, metacache.WithSegmentState(commonpb.SegmentState_Flushed))
s.True(result)
s.Equal(metacache.NullSegment, seg.CompactTo())
_, result = fg.metacache.GetSegmentByID(301, metacache.WithSegmentState(commonpb.SegmentState_Flushed))
s.False(result)
})
}
func (s *DataNodeServicesSuite) TestResendSegmentStats() {
req := &datapb.ResendSegmentStatsRequest{
Base: &commonpb.MsgBase{},
@ -666,3 +470,25 @@ func (s *DataNodeServicesSuite) TestRPCWatch() {
s.False(merr.Ok(resp.GetStatus()))
})
}
func (s *DataNodeServicesSuite) TestQuerySlot() {
s.Run("node not healthy", func() {
s.SetupTest()
s.node.UpdateStateCode(commonpb.StateCode_Abnormal)
ctx := context.Background()
resp, err := s.node.QuerySlot(ctx, nil)
s.NoError(err)
s.False(merr.Ok(resp.GetStatus()))
s.ErrorIs(merr.Error(resp.GetStatus()), merr.ErrServiceNotReady)
})
s.Run("normal case", func() {
s.SetupTest()
ctx := context.Background()
resp, err := s.node.QuerySlot(ctx, nil)
s.NoError(err)
s.True(merr.Ok(resp.GetStatus()))
s.NoError(merr.Error(resp.GetStatus()))
})
}

View File

@ -6,9 +6,9 @@ import (
"github.com/milvus-io/milvus/pkg/util/lock"
)
//go:generate mockery --name=Task --structname=MockTask --output=./ --filename=mock_task.go --with-expecter --inpackage
type Task interface {
SegmentID() int64
CalcTargetSegment() (int64, error)
Checkpoint() *msgpb.MsgPosition
StartPosition() *msgpb.MsgPosition
ChannelName() string

View File

@ -19,9 +19,6 @@ func (t *mockTask) done() {
close(t.ch)
}
func (t *mockTask) CalcTargetSegment() (int64, error) {
return t.targetID, t.err
}
func (t *mockTask) SegmentID() int64 { panic("no implementation") }
func (t *mockTask) Checkpoint() *msgpb.MsgPosition { panic("no implementation") }
func (t *mockTask) StartPosition() *msgpb.MsgPosition { panic("no implementation") }

View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
// Code generated by mockery v2.30.1. DO NOT EDIT.
package syncmgr
@ -25,39 +25,6 @@ func (_m *MockSyncManager) EXPECT() *MockSyncManager_Expecter {
return &MockSyncManager_Expecter{mock: &_m.Mock}
}
// Block provides a mock function with given fields: segmentID
func (_m *MockSyncManager) Block(segmentID int64) {
_m.Called(segmentID)
}
// MockSyncManager_Block_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Block'
type MockSyncManager_Block_Call struct {
*mock.Call
}
// Block is a helper method to define mock.On call
// - segmentID int64
func (_e *MockSyncManager_Expecter) Block(segmentID interface{}) *MockSyncManager_Block_Call {
return &MockSyncManager_Block_Call{Call: _e.mock.On("Block", segmentID)}
}
func (_c *MockSyncManager_Block_Call) Run(run func(segmentID int64)) *MockSyncManager_Block_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(int64))
})
return _c
}
func (_c *MockSyncManager_Block_Call) Return() *MockSyncManager_Block_Call {
_c.Call.Return()
return _c
}
func (_c *MockSyncManager_Block_Call) RunAndReturn(run func(int64)) *MockSyncManager_Block_Call {
_c.Call.Return(run)
return _c
}
// GetEarliestPosition provides a mock function with given fields: channel
func (_m *MockSyncManager) GetEarliestPosition(channel string) (int64, *msgpb.MsgPosition) {
ret := _m.Called(channel)
@ -90,7 +57,7 @@ type MockSyncManager_GetEarliestPosition_Call struct {
}
// GetEarliestPosition is a helper method to define mock.On call
// - channel string
// - channel string
func (_e *MockSyncManager_Expecter) GetEarliestPosition(channel interface{}) *MockSyncManager_GetEarliestPosition_Call {
return &MockSyncManager_GetEarliestPosition_Call{Call: _e.mock.On("GetEarliestPosition", channel)}
}
@ -134,8 +101,8 @@ type MockSyncManager_SyncData_Call struct {
}
// SyncData is a helper method to define mock.On call
// - ctx context.Context
// - task Task
// - ctx context.Context
// - task Task
func (_e *MockSyncManager_Expecter) SyncData(ctx interface{}, task interface{}) *MockSyncManager_SyncData_Call {
return &MockSyncManager_SyncData_Call{Call: _e.mock.On("SyncData", ctx, task)}
}
@ -157,39 +124,6 @@ func (_c *MockSyncManager_SyncData_Call) RunAndReturn(run func(context.Context,
return _c
}
// Unblock provides a mock function with given fields: segmentID
func (_m *MockSyncManager) Unblock(segmentID int64) {
_m.Called(segmentID)
}
// MockSyncManager_Unblock_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Unblock'
type MockSyncManager_Unblock_Call struct {
*mock.Call
}
// Unblock is a helper method to define mock.On call
// - segmentID int64
func (_e *MockSyncManager_Expecter) Unblock(segmentID interface{}) *MockSyncManager_Unblock_Call {
return &MockSyncManager_Unblock_Call{Call: _e.mock.On("Unblock", segmentID)}
}
func (_c *MockSyncManager_Unblock_Call) Run(run func(segmentID int64)) *MockSyncManager_Unblock_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(int64))
})
return _c
}
func (_c *MockSyncManager_Unblock_Call) Return() *MockSyncManager_Unblock_Call {
_c.Call.Return()
return _c
}
func (_c *MockSyncManager_Unblock_Call) RunAndReturn(run func(int64)) *MockSyncManager_Unblock_Call {
_c.Call.Return(run)
return _c
}
// NewMockSyncManager creates a new instance of MockSyncManager. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value.
func NewMockSyncManager(t interface {

View File

@ -1,4 +1,4 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
// Code generated by mockery v2.30.1. DO NOT EDIT.
package syncmgr
@ -20,57 +20,6 @@ func (_m *MockTask) EXPECT() *MockTask_Expecter {
return &MockTask_Expecter{mock: &_m.Mock}
}
// CalcTargetSegment provides a mock function with given fields:
func (_m *MockTask) CalcTargetSegment() (int64, error) {
ret := _m.Called()
var r0 int64
var r1 error
if rf, ok := ret.Get(0).(func() (int64, error)); ok {
return rf()
}
if rf, ok := ret.Get(0).(func() int64); ok {
r0 = rf()
} else {
r0 = ret.Get(0).(int64)
}
if rf, ok := ret.Get(1).(func() error); ok {
r1 = rf()
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockTask_CalcTargetSegment_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CalcTargetSegment'
type MockTask_CalcTargetSegment_Call struct {
*mock.Call
}
// CalcTargetSegment is a helper method to define mock.On call
func (_e *MockTask_Expecter) CalcTargetSegment() *MockTask_CalcTargetSegment_Call {
return &MockTask_CalcTargetSegment_Call{Call: _e.mock.On("CalcTargetSegment")}
}
func (_c *MockTask_CalcTargetSegment_Call) Run(run func()) *MockTask_CalcTargetSegment_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockTask_CalcTargetSegment_Call) Return(_a0 int64, _a1 error) *MockTask_CalcTargetSegment_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockTask_CalcTargetSegment_Call) RunAndReturn(run func() (int64, error)) *MockTask_CalcTargetSegment_Call {
_c.Call.Return(run)
return _c
}
// ChannelName provides a mock function with given fields:
func (_m *MockTask) ChannelName() string {
ret := _m.Called()
@ -166,7 +115,7 @@ type MockTask_HandleError_Call struct {
}
// HandleError is a helper method to define mock.On call
// - _a0 error
// - _a0 error
func (_e *MockTask_Expecter) HandleError(_a0 interface{}) *MockTask_HandleError_Call {
return &MockTask_HandleError_Call{Call: _e.mock.On("HandleError", _a0)}
}

View File

@ -249,7 +249,6 @@ func (s *StorageV1SerializerSuite) TestSerializeInsert() {
bfs := s.getBfs()
segInfo := metacache.NewSegmentInfo(&datapb.SegmentInfo{}, bfs)
metacache.UpdateNumOfRows(1000)(segInfo)
metacache.CompactTo(metacache.NullSegment)(segInfo)
s.mockCache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Run(func(action metacache.SegmentAction, filters ...metacache.SegmentFilter) {
action(segInfo)
}).Return().Once()

View File

@ -270,7 +270,6 @@ func (s *StorageV2SerializerSuite) TestSerializeInsert() {
bfs := s.getBfs()
segInfo := metacache.NewSegmentInfo(&datapb.SegmentInfo{}, bfs)
metacache.UpdateNumOfRows(1000)(segInfo)
metacache.CompactTo(metacache.NullSegment)(segInfo)
s.mockCache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Run(func(action metacache.SegmentAction, filters ...metacache.SegmentFilter) {
action(segInfo)
}).Return().Once()

View File

@ -5,7 +5,6 @@ import (
"fmt"
"strconv"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
@ -40,19 +39,15 @@ type SyncMeta struct {
metacache metacache.MetaCache
}
// SyncMangger is the interface for sync manager.
// SyncManager is the interface for sync manager.
// it processes the sync tasks inside and changes the meta.
//
//go:generate mockery --name=SyncManager --structname=MockSyncManager --output=./ --filename=mock_sync_manager.go --with-expecter --inpackage
type SyncManager interface {
// SyncData is the method to submit sync task.
SyncData(ctx context.Context, task Task) *conc.Future[struct{}]
// GetEarliestPosition returns the earliest position (normally start position) of the processing sync task of provided channel.
GetEarliestPosition(channel string) (int64, *msgpb.MsgPosition)
// Block allows caller to block tasks of provided segment id.
// normally used by compaction task.
// if levelzero delta policy is enabled, this shall be an empty operation.
Block(segmentID int64)
// Unblock is the reverse method for `Block`.
Unblock(segmentID int64)
}
type syncManager struct {
@ -115,19 +110,12 @@ func (mgr *syncManager) SyncData(ctx context.Context, task Task) *conc.Future[st
return mgr.safeSubmitTask(task)
}
// safeSubmitTask handles submitting task logic with optimistic target check logic
// when task returns errTargetSegmentNotMatch error
// perform refetch then retry logic
func (mgr *syncManager) safeSubmitTask(task Task) *conc.Future[struct{}] {
// safeSubmitTask submits task to SyncManager
func (mgr *syncManager) safeSubmitTask(task Task, callbacks ...func(error) error) *conc.Future[struct{}] {
taskKey := fmt.Sprintf("%d-%d", task.SegmentID(), task.Checkpoint().GetTimestamp())
mgr.tasks.Insert(taskKey, task)
key, err := task.CalcTargetSegment()
if err != nil {
task.HandleError(err)
return conc.Go(func() (struct{}, error) { return struct{}{}, err })
}
key := task.SegmentID()
return mgr.submit(key, task)
}
@ -137,28 +125,8 @@ func (mgr *syncManager) submit(key int64, task Task) *conc.Future[struct{}] {
if err == nil {
return nil
}
// unexpected error
if !errors.Is(err, errTargetSegmentNotMatch) {
task.HandleError(err)
return err
}
targetID, err := task.CalcTargetSegment()
// shall not reach, segment meta lost during sync
if err != nil {
task.HandleError(err)
return err
}
if targetID == key {
err = merr.WrapErrServiceInternal("recaluated with same key", fmt.Sprint(targetID))
task.HandleError(err)
return err
}
log.Info("task calculated target segment id",
zap.Int64("targetID", targetID),
zap.Int64("segmentID", task.SegmentID()),
)
return mgr.submit(targetID, task).Err()
task.HandleError(err)
return err
}
log.Info("sync mgr sumbit task with key", zap.Int64("key", key))
return mgr.Submit(key, task, handler, func(err error) error {
@ -184,11 +152,3 @@ func (mgr *syncManager) GetEarliestPosition(channel string) (int64, *msgpb.MsgPo
})
return segmentID, cp
}
func (mgr *syncManager) Block(segmentID int64) {
mgr.keyLock.Lock(segmentID)
}
func (mgr *syncManager) Unblock(segmentID int64) {
mgr.keyLock.Unlock(segmentID)
}

View File

@ -7,6 +7,7 @@ import (
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
@ -184,7 +185,6 @@ func (s *SyncManagerSuite) TestCompacted() {
bfs := metacache.NewBloomFilterSet()
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{}, bfs)
metacache.UpdateNumOfRows(1000)(seg)
metacache.CompactTo(1001)(seg)
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(seg, true)
s.metacache.EXPECT().GetSegmentsBy(mock.Anything).Return([]*metacache.SegmentInfo{seg})
s.metacache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
@ -208,52 +208,6 @@ func (s *SyncManagerSuite) TestCompacted() {
s.EqualValues(1001, segmentID.Load())
}
func (s *SyncManagerSuite) TestBlock() {
sig := make(chan struct{})
counter := atomic.NewInt32(0)
s.broker.EXPECT().SaveBinlogPaths(mock.Anything, mock.Anything).Return(nil)
bfs := metacache.NewBloomFilterSet()
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{
ID: s.segmentID,
}, bfs)
metacache.UpdateNumOfRows(1000)(seg)
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(seg, true)
s.metacache.EXPECT().GetSegmentsBy(mock.Anything).
RunAndReturn(func(...metacache.SegmentFilter) []*metacache.SegmentInfo {
return []*metacache.SegmentInfo{seg}
})
s.metacache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Run(func(_ metacache.SegmentAction, filters ...metacache.SegmentFilter) {
if counter.Inc() == 2 {
close(sig)
}
})
manager, err := NewSyncManager(s.chunkManager, s.allocator)
s.NoError(err)
// block
manager.Block(s.segmentID)
task := s.getSuiteSyncTask()
task.WithMetaWriter(BrokerMetaWriter(s.broker, 1))
task.WithTimeRange(50, 100)
task.WithCheckpoint(&msgpb.MsgPosition{
ChannelName: s.channelName,
MsgID: []byte{1, 2, 3, 4},
Timestamp: 100,
})
go manager.SyncData(context.Background(), task)
select {
case <-sig:
s.FailNow("sync task done during block")
case <-time.After(time.Second):
}
manager.Unblock(s.segmentID)
<-sig
}
func (s *SyncManagerSuite) TestResizePool() {
manager, err := NewSyncManager(s.chunkManager, s.allocator)
s.NoError(err)
@ -307,23 +261,6 @@ func (s *SyncManagerSuite) TestNewSyncManager() {
s.Error(err)
}
func (s *SyncManagerSuite) TestTargetUpdated() {
manager, err := NewSyncManager(s.chunkManager, s.allocator)
s.NoError(err)
task := NewMockTask(s.T())
task.EXPECT().SegmentID().Return(1000)
task.EXPECT().Checkpoint().Return(&msgpb.MsgPosition{})
task.EXPECT().CalcTargetSegment().Return(1000, nil).Once()
task.EXPECT().CalcTargetSegment().Return(1001, nil).Once()
task.EXPECT().Run().Return(errTargetSegmentNotMatch).Once()
task.EXPECT().Run().Return(nil).Once()
f := manager.SyncData(context.Background(), task)
_, err = f.Await()
s.NoError(err)
}
func (s *SyncManagerSuite) TestUnexpectedError() {
manager, err := NewSyncManager(s.chunkManager, s.allocator)
s.NoError(err)
@ -331,7 +268,6 @@ func (s *SyncManagerSuite) TestUnexpectedError() {
task := NewMockTask(s.T())
task.EXPECT().SegmentID().Return(1000)
task.EXPECT().Checkpoint().Return(&msgpb.MsgPosition{})
task.EXPECT().CalcTargetSegment().Return(1000, nil).Once()
task.EXPECT().Run().Return(merr.WrapErrServiceInternal("mocked")).Once()
task.EXPECT().HandleError(mock.Anything)
@ -340,46 +276,6 @@ func (s *SyncManagerSuite) TestUnexpectedError() {
s.Error(err)
}
func (s *SyncManagerSuite) TestCalcTargetError() {
s.Run("fail_before_submit", func() {
manager, err := NewSyncManager(s.chunkManager, s.allocator)
s.NoError(err)
mockErr := merr.WrapErrServiceInternal("mocked")
task := NewMockTask(s.T())
task.EXPECT().SegmentID().Return(1000)
task.EXPECT().Checkpoint().Return(&msgpb.MsgPosition{})
task.EXPECT().CalcTargetSegment().Return(0, mockErr).Once()
task.EXPECT().HandleError(mock.Anything)
f := manager.SyncData(context.Background(), task)
_, err = f.Await()
s.Error(err)
s.ErrorIs(err, mockErr)
})
s.Run("fail_during_rerun", func() {
manager, err := NewSyncManager(s.chunkManager, s.allocator)
s.NoError(err)
mockErr := merr.WrapErrServiceInternal("mocked")
task := NewMockTask(s.T())
task.EXPECT().SegmentID().Return(1000)
task.EXPECT().Checkpoint().Return(&msgpb.MsgPosition{})
task.EXPECT().CalcTargetSegment().Return(1000, nil).Once()
task.EXPECT().CalcTargetSegment().Return(0, mockErr).Once()
task.EXPECT().Run().Return(errTargetSegmentNotMatch).Once()
task.EXPECT().HandleError(mock.Anything)
f := manager.SyncData(context.Background(), task)
_, err = f.Await()
s.Error(err)
s.ErrorIs(err, mockErr)
})
}
func (s *SyncManagerSuite) TestTargetUpdateSameID() {
manager, err := NewSyncManager(s.chunkManager, s.allocator)
s.NoError(err)
@ -387,9 +283,7 @@ func (s *SyncManagerSuite) TestTargetUpdateSameID() {
task := NewMockTask(s.T())
task.EXPECT().SegmentID().Return(1000)
task.EXPECT().Checkpoint().Return(&msgpb.MsgPosition{})
task.EXPECT().CalcTargetSegment().Return(1000, nil).Once()
task.EXPECT().CalcTargetSegment().Return(1000, nil).Once()
task.EXPECT().Run().Return(errTargetSegmentNotMatch).Once()
task.EXPECT().Run().Return(errors.New("mock err")).Once()
task.EXPECT().HandleError(mock.Anything)
f := manager.SyncData(context.Background(), task)

View File

@ -21,9 +21,7 @@ import (
"fmt"
"path"
"github.com/cockroachdb/errors"
"github.com/samber/lo"
"go.uber.org/atomic"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
@ -44,8 +42,6 @@ import (
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
var errTargetSegmentNotMatch = errors.New("target segment not match")
type SyncTask struct {
chunkManager storage.ChunkManager
allocator allocator.Interface
@ -64,9 +60,6 @@ type SyncTask struct {
batchSize int64
level datapb.SegmentLevel
// targetSegmentID stores the "current" segmentID task shall be handling
targetSegmentID atomic.Int64
tsFrom typeutil.Timestamp
tsTo typeutil.Timestamp
@ -110,9 +103,6 @@ func (t *SyncTask) getLogger() *log.MLogger {
}
func (t *SyncTask) HandleError(err error) {
if errors.Is(err, errTargetSegmentNotMatch) {
return
}
if t.failureCallback != nil {
t.failureCallback(err)
}
@ -145,26 +135,6 @@ func (t *SyncTask) Run() (err error) {
return err
}
if t.segment.CompactTo() == metacache.NullSegment {
log.Info("segment compacted to zero-length segment, discard sync task")
return nil
}
if t.segment.CompactTo() > 0 {
// current task does not hold the key lock for "true" target segment id
if t.segment.CompactTo() != t.targetSegmentID.Load() {
log.Info("sync task does not hold target segment id lock, return error and retry",
zap.Int64("compactTo", t.segment.CompactTo()),
zap.Int64("currentTarget", t.targetSegmentID.Load()),
)
return errors.Wrap(errTargetSegmentNotMatch, "task does not hold target segment id lock")
}
log.Info("syncing segment compacted, update segment id", zap.Int64("compactTo", t.segment.CompactTo()))
// update sync task segment id
// it's ok to use compactTo segmentID here, since there shall be no insert for compacted segment
t.segmentID = t.segment.CompactTo()
}
err = t.prefetchIDs()
if err != nil {
log.Warn("failed allocate ids for sync task", zap.Error(err))
@ -357,20 +327,6 @@ func (t *SyncTask) SegmentID() int64 {
return t.segmentID
}
func (t *SyncTask) CalcTargetSegment() (int64, error) {
segment, has := t.metacache.GetSegmentByID(t.segmentID)
if !has {
return -1, merr.WrapErrSegmentNotFound(t.segmentID)
}
target := segment.SegmentID()
if compactTo := segment.CompactTo(); compactTo > 0 {
target = compactTo
}
t.targetSegmentID.Store(target)
return target, nil
}
func (t *SyncTask) Checkpoint() *msgpb.MsgPosition {
return t.checkpoint
}

View File

@ -292,61 +292,7 @@ func (s *SyncTaskSuite) TestRunL0Segment() {
})
}
func (s *SyncTaskSuite) TestCompactToNull() {
bfs := metacache.NewBloomFilterSet()
fd, err := storage.NewFieldData(schemapb.DataType_Int64, &schemapb.FieldSchema{
FieldID: 101,
Name: "ID",
IsPrimaryKey: true,
DataType: schemapb.DataType_Int64,
})
s.Require().NoError(err)
ids := []int64{1, 2, 3, 4, 5, 6, 7}
for _, id := range ids {
err = fd.AppendRow(id)
s.Require().NoError(err)
}
bfs.UpdatePKRange(fd)
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{}, bfs)
metacache.UpdateNumOfRows(1000)(seg)
metacache.CompactTo(metacache.NullSegment)(seg)
seg.GetBloomFilterSet().Roll()
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(seg, true)
task := s.getSuiteSyncTask()
task.WithMetaWriter(BrokerMetaWriter(s.broker, 1))
task.WithTimeRange(50, 100)
task.WithCheckpoint(&msgpb.MsgPosition{
ChannelName: s.channelName,
MsgID: []byte{1, 2, 3, 4},
Timestamp: 100,
})
err = task.Run()
s.NoError(err)
}
func (s *SyncTaskSuite) TestRunError() {
s.Run("target_segment_not_match", func() {
flag := false
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{
ID: s.segmentID,
}, metacache.NewBloomFilterSet())
metacache.CompactTo(s.segmentID + 1)(seg)
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(seg, true).Once()
handler := func(_ error) { flag = true }
task := s.getSuiteSyncTask().WithFailureCallback(handler)
task.targetSegmentID.Store(s.segmentID)
err := task.Run()
s.Error(err)
s.False(flag, "target not match shall not trigger error handler")
})
s.Run("segment_not_found", func() {
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(nil, false)
flag := false
@ -428,63 +374,6 @@ func (s *SyncTaskSuite) TestNextID() {
})
}
func (s *SyncTaskSuite) TestCalcTargetID() {
task := s.getSuiteSyncTask()
s.Run("normal_calc_segment", func() {
s.Run("not_compacted", func() {
segment := metacache.NewSegmentInfo(&datapb.SegmentInfo{
ID: s.segmentID,
PartitionID: s.partitionID,
CollectionID: s.collectionID,
}, metacache.NewBloomFilterSet())
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(segment, true).Once()
targetID, err := task.CalcTargetSegment()
s.Require().NoError(err)
s.Equal(s.segmentID, targetID)
s.Equal(s.segmentID, task.targetSegmentID.Load())
})
s.Run("compacted_normal", func() {
segment := metacache.NewSegmentInfo(&datapb.SegmentInfo{
ID: s.segmentID,
PartitionID: s.partitionID,
CollectionID: s.collectionID,
}, metacache.NewBloomFilterSet())
metacache.CompactTo(1000)(segment)
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(segment, true).Once()
targetID, err := task.CalcTargetSegment()
s.Require().NoError(err)
s.EqualValues(1000, targetID)
s.EqualValues(1000, task.targetSegmentID.Load())
})
s.Run("compacted_null", func() {
segment := metacache.NewSegmentInfo(&datapb.SegmentInfo{
ID: s.segmentID,
PartitionID: s.partitionID,
CollectionID: s.collectionID,
}, metacache.NewBloomFilterSet())
metacache.CompactTo(metacache.NullSegment)(segment)
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(segment, true).Once()
targetID, err := task.CalcTargetSegment()
s.Require().NoError(err)
s.Equal(s.segmentID, targetID)
s.Equal(s.segmentID, task.targetSegmentID.Load())
})
})
s.Run("segment_not_found", func() {
s.metacache.EXPECT().GetSegmentByID(s.segmentID).Return(nil, false).Once()
_, err := task.CalcTargetSegment()
s.Error(err)
})
}
func TestSyncTask(t *testing.T) {
suite.Run(t, new(SyncTaskSuite))
}

View File

@ -69,20 +69,13 @@ func (t *SyncTaskV2) Run() error {
log := t.getLogger()
var err error
segment, ok := t.metacache.GetSegmentByID(t.segmentID)
_, ok := t.metacache.GetSegmentByID(t.segmentID)
if !ok {
log.Warn("failed to sync data, segment not found in metacache")
t.handleError(err)
return merr.WrapErrSegmentNotFound(t.segmentID)
}
if segment.CompactTo() > 0 {
log.Info("syncing segment compacted, update segment id", zap.Int64("compactTo", segment.CompactTo()))
// update sync task segment id
// it's ok to use compactTo segmentID here, since there shall be no insert for compacted segment
t.segmentID = segment.CompactTo()
}
if err = t.writeSpace(); err != nil {
t.handleError(err)
return err

View File

@ -36,10 +36,6 @@ func (wb *bfWriteBuffer) dispatchDeleteMsgs(groups []*inData, deleteMsgs []*msgs
split := func(pks []storage.PrimaryKey, pkTss []uint64, segments []*metacache.SegmentInfo) {
lc := storage.NewBatchLocationsCache(pks)
for _, segment := range segments {
if segment.CompactTo() != 0 {
continue
}
hits := segment.GetBloomFilterSet().BatchPkExist(lc)
var deletePks []storage.PrimaryKey
var deleteTss []typeutil.Timestamp
@ -130,6 +126,5 @@ func (wb *bfWriteBuffer) BufferData(insertMsgs []*msgstream.InsertMsg, deleteMsg
_ = wb.triggerSync()
wb.cleanupCompactedSegments()
return nil
}

View File

@ -218,7 +218,6 @@ func (s *BFWriteBufferSuite) TestBufferData() {
s.metacacheInt64.EXPECT().GetSegmentByID(int64(1000)).Return(nil, false)
s.metacacheInt64.EXPECT().AddSegment(mock.Anything, mock.Anything, mock.Anything).Return()
s.metacacheInt64.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
s.metacacheInt64.EXPECT().GetSegmentIDsBy(mock.Anything, mock.Anything).Return([]int64{})
pks, msg := s.composeInsertMsg(1000, 10, 128, schemapb.DataType_Int64)
delMsg := s.composeDeleteMsg(lo.Map(pks, func(id int64, _ int) storage.PrimaryKey { return storage.NewInt64PrimaryKey(id) }))
@ -248,7 +247,6 @@ func (s *BFWriteBufferSuite) TestBufferData() {
s.metacacheVarchar.EXPECT().GetSegmentByID(int64(1000)).Return(nil, false)
s.metacacheVarchar.EXPECT().AddSegment(mock.Anything, mock.Anything, mock.Anything).Return()
s.metacacheVarchar.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
s.metacacheVarchar.EXPECT().GetSegmentIDsBy(mock.Anything, mock.Anything).Return([]int64{})
pks, msg := s.composeInsertMsg(1000, 10, 128, schemapb.DataType_VarChar)
delMsg := s.composeDeleteMsg(lo.Map(pks, func(id int64, _ int) storage.PrimaryKey { return storage.NewVarCharPrimaryKey(fmt.Sprintf("%v", id)) }))
@ -273,7 +271,6 @@ func (s *BFWriteBufferSuite) TestBufferData() {
s.metacacheInt64.EXPECT().GetSegmentByID(int64(1000)).Return(nil, false)
s.metacacheInt64.EXPECT().AddSegment(mock.Anything, mock.Anything, mock.Anything).Return()
s.metacacheInt64.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
s.metacacheInt64.EXPECT().GetSegmentIDsBy(mock.Anything, mock.Anything).Return([]int64{})
pks, msg := s.composeInsertMsg(1000, 10, 128, schemapb.DataType_VarChar)
delMsg := s.composeDeleteMsg(lo.Map(pks, func(id int64, _ int) storage.PrimaryKey { return storage.NewInt64PrimaryKey(id) }))
@ -294,7 +291,6 @@ func (s *BFWriteBufferSuite) TestBufferData() {
s.metacacheVarchar.EXPECT().GetSegmentByID(int64(1000)).Return(nil, false)
s.metacacheVarchar.EXPECT().AddSegment(mock.Anything, mock.Anything, mock.Anything).Return()
s.metacacheVarchar.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
s.metacacheVarchar.EXPECT().GetSegmentIDsBy(mock.Anything, mock.Anything).Return([]int64{})
pks, msg := s.composeInsertMsg(1000, 10, 128, schemapb.DataType_Int64)
delMsg := s.composeDeleteMsg(lo.Map(pks, func(id int64, _ int) storage.PrimaryKey { return storage.NewInt64PrimaryKey(id) }))
@ -363,7 +359,6 @@ func (s *BFWriteBufferSuite) TestBufferDataWithStorageV2() {
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 1000}, metacache.NewBloomFilterSet())
s.metacacheInt64.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg})
s.metacacheInt64.EXPECT().GetSegmentByID(int64(1000)).Return(nil, false)
s.metacacheInt64.EXPECT().GetSegmentIDsBy(mock.Anything, mock.Anything).Return([]int64{})
s.metacacheInt64.EXPECT().AddSegment(mock.Anything, mock.Anything, mock.Anything).Return()
s.metacacheInt64.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
@ -402,15 +397,12 @@ func (s *BFWriteBufferSuite) TestAutoSyncWithStorageV2() {
seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 1000}, metacache.NewBloomFilterSet())
seg1 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 1002}, metacache.NewBloomFilterSet())
segCompacted := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 1000}, metacache.NewBloomFilterSet())
metacache.CompactTo(2001)(segCompacted)
s.metacacheInt64.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg, segCompacted})
s.metacacheInt64.EXPECT().GetSegmentByID(int64(1000)).Return(nil, false).Once()
s.metacacheInt64.EXPECT().GetSegmentByID(int64(1000)).Return(seg, true).Once()
s.metacacheInt64.EXPECT().GetSegmentByID(int64(1002)).Return(seg1, true)
s.metacacheInt64.EXPECT().GetSegmentIDsBy(mock.Anything).Return([]int64{1002})
s.metacacheInt64.EXPECT().GetSegmentIDsBy(mock.Anything, mock.Anything).Return([]int64{1003}) // mocked compacted
s.metacacheInt64.EXPECT().RemoveSegments(mock.Anything).Return([]int64{1003})
s.metacacheInt64.EXPECT().AddSegment(mock.Anything, mock.Anything, mock.Anything).Return()
s.metacacheInt64.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
s.metacacheInt64.EXPECT().UpdateSegments(mock.Anything, mock.Anything, mock.Anything).Return()

View File

@ -59,9 +59,6 @@ func (wb *l0WriteBuffer) dispatchDeleteMsgs(groups []*inData, deleteMsgs []*msgs
hits := make([]bool, len(pks))
for _, segment := range partitionSegments {
if segment.CompactTo() != 0 {
continue
}
hits = segment.GetBloomFilterSet().BatchPkExistWithHits(lc, hits)
}
@ -185,7 +182,6 @@ func (wb *l0WriteBuffer) BufferData(insertMsgs []*msgstream.InsertMsg, deleteMsg
}
}
wb.cleanupCompactedSegments()
return nil
}

View File

@ -186,7 +186,6 @@ func (s *L0WriteBufferSuite) TestBufferData() {
s.metacache.EXPECT().GetSegmentByID(int64(1000)).Return(nil, false).Once()
s.metacache.EXPECT().AddSegment(mock.Anything, mock.Anything, mock.Anything).Return()
s.metacache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
s.metacache.EXPECT().GetSegmentIDsBy(mock.Anything, mock.Anything).Return([]int64{})
metrics.DataNodeFlowGraphBufferDataSize.Reset()
err = wb.BufferData([]*msgstream.InsertMsg{msg}, []*msgstream.DeleteMsg{delMsg}, &msgpb.MsgPosition{Timestamp: 100}, &msgpb.MsgPosition{Timestamp: 200})
@ -215,7 +214,6 @@ func (s *L0WriteBufferSuite) TestBufferData() {
s.metacache.EXPECT().GetSegmentsBy(mock.Anything, mock.Anything).Return([]*metacache.SegmentInfo{seg})
s.metacache.EXPECT().AddSegment(mock.Anything, mock.Anything, mock.Anything).Return()
s.metacache.EXPECT().UpdateSegments(mock.Anything, mock.Anything).Return()
s.metacache.EXPECT().GetSegmentIDsBy(mock.Anything, mock.Anything).Return([]int64{})
metrics.DataNodeFlowGraphBufferDataSize.Reset()
err = wb.BufferData([]*msgstream.InsertMsg{msg}, []*msgstream.DeleteMsg{delMsg}, &msgpb.MsgPosition{Timestamp: 100}, &msgpb.MsgPosition{Timestamp: 200})

View File

@ -40,7 +40,6 @@ func defaultWBOption(metacache metacache.MetaCache) *writeBufferOption {
syncPolicies: []SyncPolicy{
GetFullBufferPolicy(),
GetSyncStaleBufferPolicy(paramtable.Get().DataNodeCfg.SyncPeriod.GetAsDuration(time.Second)),
GetCompactedSegmentsPolicy(metacache),
GetSealedSegmentsPolicy(metacache),
GetDroppedSegmentPolicy(metacache),
},

View File

@ -56,13 +56,6 @@ func GetFullBufferPolicy() SyncPolicy {
}, "buffer full")
}
func GetCompactedSegmentsPolicy(meta metacache.MetaCache) SyncPolicy {
return wrapSelectSegmentFuncPolicy(func(buffers []*segmentBuffer, _ typeutil.Timestamp) []int64 {
segmentIDs := lo.Map(buffers, func(buffer *segmentBuffer, _ int) int64 { return buffer.segmentID })
return meta.GetSegmentIDsBy(metacache.WithSegmentIDs(segmentIDs...), metacache.WithCompacted())
}, "segment compacted")
}
func GetSyncStaleBufferPolicy(staleDuration time.Duration) SyncPolicy {
return wrapSelectSegmentFuncPolicy(func(buffers []*segmentBuffer, ts typeutil.Timestamp) []int64 {
current := tsoutil.PhysicalTime(ts)

View File

@ -93,16 +93,6 @@ func (s *SyncPolicySuite) TestSealedSegmentsPolicy() {
s.ElementsMatch(ids, result)
}
func (s *SyncPolicySuite) TestCompactedSegmentsPolicy() {
metacache := metacache.NewMockMetaCache(s.T())
policy := GetCompactedSegmentsPolicy(metacache)
ids := []int64{1, 2}
metacache.EXPECT().GetSegmentIDsBy(mock.Anything, mock.Anything).Return(ids)
result := policy.SelectSegments([]*segmentBuffer{{segmentID: 1}, {segmentID: 2}}, tsoutil.ComposeTSByTime(time.Now(), 0))
s.ElementsMatch(ids, result)
}
func (s *SyncPolicySuite) TestOlderBufferPolicy() {
policy := GetOldestBufferPolicy(2)

View File

@ -298,22 +298,6 @@ func (wb *writeBufferBase) triggerSync() (segmentIDs []int64) {
return segmentsToSync
}
func (wb *writeBufferBase) cleanupCompactedSegments() {
segmentIDs := wb.metaCache.GetSegmentIDsBy(metacache.WithCompacted(), metacache.WithNoSyncingTask())
// remove compacted only when there is no writebuffer
targetIDs := lo.Filter(segmentIDs, func(segmentID int64, _ int) bool {
_, ok := wb.buffers[segmentID]
return !ok
})
if len(targetIDs) == 0 {
return
}
removed := wb.metaCache.RemoveSegments(metacache.WithSegmentIDs(targetIDs...))
if len(removed) > 0 {
log.Info("remove compacted segments", zap.Int64s("removed", removed))
}
}
func (wb *writeBufferBase) sealSegments(_ context.Context, segmentIDs []int64) error {
// mark segment flushing if segment was growing
wb.metaCache.UpdateSegments(metacache.UpdateState(commonpb.SegmentState_Sealed),

View File

@ -173,10 +173,10 @@ func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
})
}
// Compaction return compaction by given plan
func (c *Client) Compaction(ctx context.Context, req *datapb.CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) {
// CompactionV2 return compaction by given plan
func (c *Client) CompactionV2(ctx context.Context, req *datapb.CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*commonpb.Status, error) {
return client.Compaction(ctx, req)
return client.CompactionV2(ctx, req)
})
}
@ -255,3 +255,9 @@ func (c *Client) DropImport(ctx context.Context, req *datapb.DropImportRequest,
return client.DropImport(ctx, req)
})
}
func (c *Client) QuerySlot(ctx context.Context, req *datapb.QuerySlotRequest, opts ...grpc.CallOption) (*datapb.QuerySlotResponse, error) {
return wrapGrpcCall(ctx, c, func(client datapb.DataNodeClient) (*datapb.QuerySlotResponse, error) {
return client.QuerySlot(ctx, req)
})
}

View File

@ -66,7 +66,7 @@ func Test_NewClient(t *testing.T) {
r5, err := client.GetMetrics(ctx, nil)
retCheck(retNotNil, r5, err)
r6, err := client.Compaction(ctx, nil)
r6, err := client.CompactionV2(ctx, nil)
retCheck(retNotNil, r6, err)
r8, err := client.ResendSegmentStats(ctx, nil)

View File

@ -354,8 +354,8 @@ func (s *Server) GetMetrics(ctx context.Context, request *milvuspb.GetMetricsReq
return s.datanode.GetMetrics(ctx, request)
}
func (s *Server) Compaction(ctx context.Context, request *datapb.CompactionPlan) (*commonpb.Status, error) {
return s.datanode.Compaction(ctx, request)
func (s *Server) CompactionV2(ctx context.Context, request *datapb.CompactionPlan) (*commonpb.Status, error) {
return s.datanode.CompactionV2(ctx, request)
}
// GetCompactionState gets the Compaction tasks state of DataNode
@ -402,3 +402,7 @@ func (s *Server) QueryImport(ctx context.Context, req *datapb.QueryImportRequest
func (s *Server) DropImport(ctx context.Context, req *datapb.DropImportRequest) (*commonpb.Status, error) {
return s.datanode.DropImport(ctx, req)
}
func (s *Server) QuerySlot(ctx context.Context, req *datapb.QuerySlotRequest) (*datapb.QuerySlotResponse, error) {
return s.datanode.QuerySlot(ctx, req)
}

View File

@ -126,7 +126,7 @@ func (m *MockDataNode) GetMetrics(ctx context.Context, request *milvuspb.GetMetr
return m.metricResp, m.err
}
func (m *MockDataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) {
func (m *MockDataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPlan) (*commonpb.Status, error) {
return m.status, m.err
}
@ -177,6 +177,10 @@ func (m *MockDataNode) DropImport(ctx context.Context, req *datapb.DropImportReq
return m.status, m.err
}
func (m *MockDataNode) QuerySlot(ctx context.Context, req *datapb.QuerySlotRequest) (*datapb.QuerySlotResponse, error) {
return &datapb.QuerySlotResponse{}, m.err
}
// /////////////////////////////////////////////////////////////////////////////////////////////////////////////////////
func Test_NewServer(t *testing.T) {
paramtable.Init()
@ -281,7 +285,7 @@ func Test_NewServer(t *testing.T) {
server.datanode = &MockDataNode{
status: &commonpb.Status{},
}
resp, err := server.Compaction(ctx, nil)
resp, err := server.CompactionV2(ctx, nil)
assert.NoError(t, err)
assert.NotNil(t, resp)
})

View File

@ -64,8 +64,8 @@ type MockDataNode_CheckChannelOperationProgress_Call struct {
}
// CheckChannelOperationProgress is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.ChannelWatchInfo
// - _a0 context.Context
// - _a1 *datapb.ChannelWatchInfo
func (_e *MockDataNode_Expecter) CheckChannelOperationProgress(_a0 interface{}, _a1 interface{}) *MockDataNode_CheckChannelOperationProgress_Call {
return &MockDataNode_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress", _a0, _a1)}
}
@ -87,8 +87,8 @@ func (_c *MockDataNode_CheckChannelOperationProgress_Call) RunAndReturn(run func
return _c
}
// Compaction provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) Compaction(_a0 context.Context, _a1 *datapb.CompactionPlan) (*commonpb.Status, error) {
// CompactionV2 provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) CompactionV2(_a0 context.Context, _a1 *datapb.CompactionPlan) (*commonpb.Status, error) {
ret := _m.Called(_a0, _a1)
var r0 *commonpb.Status
@ -113,31 +113,31 @@ func (_m *MockDataNode) Compaction(_a0 context.Context, _a1 *datapb.CompactionPl
return r0, r1
}
// MockDataNode_Compaction_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Compaction'
type MockDataNode_Compaction_Call struct {
// MockDataNode_CompactionV2_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CompactionV2'
type MockDataNode_CompactionV2_Call struct {
*mock.Call
}
// Compaction is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.CompactionPlan
func (_e *MockDataNode_Expecter) Compaction(_a0 interface{}, _a1 interface{}) *MockDataNode_Compaction_Call {
return &MockDataNode_Compaction_Call{Call: _e.mock.On("Compaction", _a0, _a1)}
// CompactionV2 is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.CompactionPlan
func (_e *MockDataNode_Expecter) CompactionV2(_a0 interface{}, _a1 interface{}) *MockDataNode_CompactionV2_Call {
return &MockDataNode_CompactionV2_Call{Call: _e.mock.On("CompactionV2", _a0, _a1)}
}
func (_c *MockDataNode_Compaction_Call) Run(run func(_a0 context.Context, _a1 *datapb.CompactionPlan)) *MockDataNode_Compaction_Call {
func (_c *MockDataNode_CompactionV2_Call) Run(run func(_a0 context.Context, _a1 *datapb.CompactionPlan)) *MockDataNode_CompactionV2_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.CompactionPlan))
})
return _c
}
func (_c *MockDataNode_Compaction_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataNode_Compaction_Call {
func (_c *MockDataNode_CompactionV2_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataNode_CompactionV2_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockDataNode_Compaction_Call) RunAndReturn(run func(context.Context, *datapb.CompactionPlan) (*commonpb.Status, error)) *MockDataNode_Compaction_Call {
func (_c *MockDataNode_CompactionV2_Call) RunAndReturn(run func(context.Context, *datapb.CompactionPlan) (*commonpb.Status, error)) *MockDataNode_CompactionV2_Call {
_c.Call.Return(run)
return _c
}
@ -174,8 +174,8 @@ type MockDataNode_DropImport_Call struct {
}
// DropImport is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.DropImportRequest
// - _a0 context.Context
// - _a1 *datapb.DropImportRequest
func (_e *MockDataNode_Expecter) DropImport(_a0 interface{}, _a1 interface{}) *MockDataNode_DropImport_Call {
return &MockDataNode_DropImport_Call{Call: _e.mock.On("DropImport", _a0, _a1)}
}
@ -229,8 +229,8 @@ type MockDataNode_FlushChannels_Call struct {
}
// FlushChannels is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.FlushChannelsRequest
// - _a0 context.Context
// - _a1 *datapb.FlushChannelsRequest
func (_e *MockDataNode_Expecter) FlushChannels(_a0 interface{}, _a1 interface{}) *MockDataNode_FlushChannels_Call {
return &MockDataNode_FlushChannels_Call{Call: _e.mock.On("FlushChannels", _a0, _a1)}
}
@ -284,8 +284,8 @@ type MockDataNode_FlushSegments_Call struct {
}
// FlushSegments is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.FlushSegmentsRequest
// - _a0 context.Context
// - _a1 *datapb.FlushSegmentsRequest
func (_e *MockDataNode_Expecter) FlushSegments(_a0 interface{}, _a1 interface{}) *MockDataNode_FlushSegments_Call {
return &MockDataNode_FlushSegments_Call{Call: _e.mock.On("FlushSegments", _a0, _a1)}
}
@ -380,8 +380,8 @@ type MockDataNode_GetCompactionState_Call struct {
}
// GetCompactionState is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.CompactionStateRequest
// - _a0 context.Context
// - _a1 *datapb.CompactionStateRequest
func (_e *MockDataNode_Expecter) GetCompactionState(_a0 interface{}, _a1 interface{}) *MockDataNode_GetCompactionState_Call {
return &MockDataNode_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", _a0, _a1)}
}
@ -435,8 +435,8 @@ type MockDataNode_GetComponentStates_Call struct {
}
// GetComponentStates is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *milvuspb.GetComponentStatesRequest
// - _a0 context.Context
// - _a1 *milvuspb.GetComponentStatesRequest
func (_e *MockDataNode_Expecter) GetComponentStates(_a0 interface{}, _a1 interface{}) *MockDataNode_GetComponentStates_Call {
return &MockDataNode_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", _a0, _a1)}
}
@ -490,8 +490,8 @@ type MockDataNode_GetMetrics_Call struct {
}
// GetMetrics is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *milvuspb.GetMetricsRequest
// - _a0 context.Context
// - _a1 *milvuspb.GetMetricsRequest
func (_e *MockDataNode_Expecter) GetMetrics(_a0 interface{}, _a1 interface{}) *MockDataNode_GetMetrics_Call {
return &MockDataNode_GetMetrics_Call{Call: _e.mock.On("GetMetrics", _a0, _a1)}
}
@ -627,8 +627,8 @@ type MockDataNode_GetStatisticsChannel_Call struct {
}
// GetStatisticsChannel is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *internalpb.GetStatisticsChannelRequest
// - _a0 context.Context
// - _a1 *internalpb.GetStatisticsChannelRequest
func (_e *MockDataNode_Expecter) GetStatisticsChannel(_a0 interface{}, _a1 interface{}) *MockDataNode_GetStatisticsChannel_Call {
return &MockDataNode_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", _a0, _a1)}
}
@ -682,8 +682,8 @@ type MockDataNode_ImportV2_Call struct {
}
// ImportV2 is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.ImportRequest
// - _a0 context.Context
// - _a1 *datapb.ImportRequest
func (_e *MockDataNode_Expecter) ImportV2(_a0 interface{}, _a1 interface{}) *MockDataNode_ImportV2_Call {
return &MockDataNode_ImportV2_Call{Call: _e.mock.On("ImportV2", _a0, _a1)}
}
@ -778,8 +778,8 @@ type MockDataNode_NotifyChannelOperation_Call struct {
}
// NotifyChannelOperation is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.ChannelOperationsRequest
// - _a0 context.Context
// - _a1 *datapb.ChannelOperationsRequest
func (_e *MockDataNode_Expecter) NotifyChannelOperation(_a0 interface{}, _a1 interface{}) *MockDataNode_NotifyChannelOperation_Call {
return &MockDataNode_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation", _a0, _a1)}
}
@ -833,8 +833,8 @@ type MockDataNode_PreImport_Call struct {
}
// PreImport is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.PreImportRequest
// - _a0 context.Context
// - _a1 *datapb.PreImportRequest
func (_e *MockDataNode_Expecter) PreImport(_a0 interface{}, _a1 interface{}) *MockDataNode_PreImport_Call {
return &MockDataNode_PreImport_Call{Call: _e.mock.On("PreImport", _a0, _a1)}
}
@ -888,8 +888,8 @@ type MockDataNode_QueryImport_Call struct {
}
// QueryImport is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.QueryImportRequest
// - _a0 context.Context
// - _a1 *datapb.QueryImportRequest
func (_e *MockDataNode_Expecter) QueryImport(_a0 interface{}, _a1 interface{}) *MockDataNode_QueryImport_Call {
return &MockDataNode_QueryImport_Call{Call: _e.mock.On("QueryImport", _a0, _a1)}
}
@ -943,8 +943,8 @@ type MockDataNode_QueryPreImport_Call struct {
}
// QueryPreImport is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.QueryPreImportRequest
// - _a0 context.Context
// - _a1 *datapb.QueryPreImportRequest
func (_e *MockDataNode_Expecter) QueryPreImport(_a0 interface{}, _a1 interface{}) *MockDataNode_QueryPreImport_Call {
return &MockDataNode_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", _a0, _a1)}
}
@ -966,6 +966,61 @@ func (_c *MockDataNode_QueryPreImport_Call) RunAndReturn(run func(context.Contex
return _c
}
// QuerySlot provides a mock function with given fields: _a0, _a1
func (_m *MockDataNode) QuerySlot(_a0 context.Context, _a1 *datapb.QuerySlotRequest) (*datapb.QuerySlotResponse, error) {
ret := _m.Called(_a0, _a1)
var r0 *datapb.QuerySlotResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *datapb.QuerySlotRequest) (*datapb.QuerySlotResponse, error)); ok {
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.QuerySlotRequest) *datapb.QuerySlotResponse); ok {
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.QuerySlotResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.QuerySlotRequest) error); ok {
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockDataNode_QuerySlot_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'QuerySlot'
type MockDataNode_QuerySlot_Call struct {
*mock.Call
}
// QuerySlot is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.QuerySlotRequest
func (_e *MockDataNode_Expecter) QuerySlot(_a0 interface{}, _a1 interface{}) *MockDataNode_QuerySlot_Call {
return &MockDataNode_QuerySlot_Call{Call: _e.mock.On("QuerySlot", _a0, _a1)}
}
func (_c *MockDataNode_QuerySlot_Call) Run(run func(_a0 context.Context, _a1 *datapb.QuerySlotRequest)) *MockDataNode_QuerySlot_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.QuerySlotRequest))
})
return _c
}
func (_c *MockDataNode_QuerySlot_Call) Return(_a0 *datapb.QuerySlotResponse, _a1 error) *MockDataNode_QuerySlot_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockDataNode_QuerySlot_Call) RunAndReturn(run func(context.Context, *datapb.QuerySlotRequest) (*datapb.QuerySlotResponse, error)) *MockDataNode_QuerySlot_Call {
_c.Call.Return(run)
return _c
}
// Register provides a mock function with given fields:
func (_m *MockDataNode) Register() error {
ret := _m.Called()
@ -1039,8 +1094,8 @@ type MockDataNode_ResendSegmentStats_Call struct {
}
// ResendSegmentStats is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.ResendSegmentStatsRequest
// - _a0 context.Context
// - _a1 *datapb.ResendSegmentStatsRequest
func (_e *MockDataNode_Expecter) ResendSegmentStats(_a0 interface{}, _a1 interface{}) *MockDataNode_ResendSegmentStats_Call {
return &MockDataNode_ResendSegmentStats_Call{Call: _e.mock.On("ResendSegmentStats", _a0, _a1)}
}
@ -1073,7 +1128,7 @@ type MockDataNode_SetAddress_Call struct {
}
// SetAddress is a helper method to define mock.On call
// - address string
// - address string
func (_e *MockDataNode_Expecter) SetAddress(address interface{}) *MockDataNode_SetAddress_Call {
return &MockDataNode_SetAddress_Call{Call: _e.mock.On("SetAddress", address)}
}
@ -1115,7 +1170,7 @@ type MockDataNode_SetDataCoordClient_Call struct {
}
// SetDataCoordClient is a helper method to define mock.On call
// - dataCoord types.DataCoordClient
// - dataCoord types.DataCoordClient
func (_e *MockDataNode_Expecter) SetDataCoordClient(dataCoord interface{}) *MockDataNode_SetDataCoordClient_Call {
return &MockDataNode_SetDataCoordClient_Call{Call: _e.mock.On("SetDataCoordClient", dataCoord)}
}
@ -1148,7 +1203,7 @@ type MockDataNode_SetEtcdClient_Call struct {
}
// SetEtcdClient is a helper method to define mock.On call
// - etcdClient *clientv3.Client
// - etcdClient *clientv3.Client
func (_e *MockDataNode_Expecter) SetEtcdClient(etcdClient interface{}) *MockDataNode_SetEtcdClient_Call {
return &MockDataNode_SetEtcdClient_Call{Call: _e.mock.On("SetEtcdClient", etcdClient)}
}
@ -1190,7 +1245,7 @@ type MockDataNode_SetRootCoordClient_Call struct {
}
// SetRootCoordClient is a helper method to define mock.On call
// - rootCoord types.RootCoordClient
// - rootCoord types.RootCoordClient
func (_e *MockDataNode_Expecter) SetRootCoordClient(rootCoord interface{}) *MockDataNode_SetRootCoordClient_Call {
return &MockDataNode_SetRootCoordClient_Call{Call: _e.mock.On("SetRootCoordClient", rootCoord)}
}
@ -1244,8 +1299,8 @@ type MockDataNode_ShowConfigurations_Call struct {
}
// ShowConfigurations is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *internalpb.ShowConfigurationsRequest
// - _a0 context.Context
// - _a1 *internalpb.ShowConfigurationsRequest
func (_e *MockDataNode_Expecter) ShowConfigurations(_a0 interface{}, _a1 interface{}) *MockDataNode_ShowConfigurations_Call {
return &MockDataNode_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", _a0, _a1)}
}
@ -1381,8 +1436,8 @@ type MockDataNode_SyncSegments_Call struct {
}
// SyncSegments is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.SyncSegmentsRequest
// - _a0 context.Context
// - _a1 *datapb.SyncSegmentsRequest
func (_e *MockDataNode_Expecter) SyncSegments(_a0 interface{}, _a1 interface{}) *MockDataNode_SyncSegments_Call {
return &MockDataNode_SyncSegments_Call{Call: _e.mock.On("SyncSegments", _a0, _a1)}
}
@ -1415,7 +1470,7 @@ type MockDataNode_UpdateStateCode_Call struct {
}
// UpdateStateCode is a helper method to define mock.On call
// - stateCode commonpb.StateCode
// - stateCode commonpb.StateCode
func (_e *MockDataNode_Expecter) UpdateStateCode(stateCode interface{}) *MockDataNode_UpdateStateCode_Call {
return &MockDataNode_UpdateStateCode_Call{Call: _e.mock.On("UpdateStateCode", stateCode)}
}
@ -1469,8 +1524,8 @@ type MockDataNode_WatchDmChannels_Call struct {
}
// WatchDmChannels is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *datapb.WatchDmChannelsRequest
// - _a0 context.Context
// - _a1 *datapb.WatchDmChannelsRequest
func (_e *MockDataNode_Expecter) WatchDmChannels(_a0 interface{}, _a1 interface{}) *MockDataNode_WatchDmChannels_Call {
return &MockDataNode_WatchDmChannels_Call{Call: _e.mock.On("WatchDmChannels", _a0, _a1)}
}
@ -1497,8 +1552,7 @@ func (_c *MockDataNode_WatchDmChannels_Call) RunAndReturn(run func(context.Conte
func NewMockDataNode(t interface {
mock.TestingT
Cleanup(func())
},
) *MockDataNode {
}) *MockDataNode {
mock := &MockDataNode{}
mock.Mock.Test(t)

View File

@ -70,9 +70,9 @@ type MockDataNodeClient_CheckChannelOperationProgress_Call struct {
}
// CheckChannelOperationProgress is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.ChannelWatchInfo
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.ChannelWatchInfo
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) CheckChannelOperationProgress(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_CheckChannelOperationProgress_Call {
return &MockDataNodeClient_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress",
append([]interface{}{ctx, in}, opts...)...)}
@ -142,8 +142,8 @@ func (_c *MockDataNodeClient_Close_Call) RunAndReturn(run func() error) *MockDat
return _c
}
// Compaction provides a mock function with given fields: ctx, in, opts
func (_m *MockDataNodeClient) Compaction(ctx context.Context, in *datapb.CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) {
// CompactionV2 provides a mock function with given fields: ctx, in, opts
func (_m *MockDataNodeClient) CompactionV2(ctx context.Context, in *datapb.CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) {
_va := make([]interface{}, len(opts))
for _i := range opts {
_va[_i] = opts[_i]
@ -175,21 +175,21 @@ func (_m *MockDataNodeClient) Compaction(ctx context.Context, in *datapb.Compact
return r0, r1
}
// MockDataNodeClient_Compaction_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Compaction'
type MockDataNodeClient_Compaction_Call struct {
// MockDataNodeClient_CompactionV2_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'CompactionV2'
type MockDataNodeClient_CompactionV2_Call struct {
*mock.Call
}
// Compaction is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.CompactionPlan
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) Compaction(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_Compaction_Call {
return &MockDataNodeClient_Compaction_Call{Call: _e.mock.On("Compaction",
// CompactionV2 is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.CompactionPlan
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) CompactionV2(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_CompactionV2_Call {
return &MockDataNodeClient_CompactionV2_Call{Call: _e.mock.On("CompactionV2",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockDataNodeClient_Compaction_Call) Run(run func(ctx context.Context, in *datapb.CompactionPlan, opts ...grpc.CallOption)) *MockDataNodeClient_Compaction_Call {
func (_c *MockDataNodeClient_CompactionV2_Call) Run(run func(ctx context.Context, in *datapb.CompactionPlan, opts ...grpc.CallOption)) *MockDataNodeClient_CompactionV2_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]grpc.CallOption, len(args)-2)
for i, a := range args[2:] {
@ -202,12 +202,12 @@ func (_c *MockDataNodeClient_Compaction_Call) Run(run func(ctx context.Context,
return _c
}
func (_c *MockDataNodeClient_Compaction_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataNodeClient_Compaction_Call {
func (_c *MockDataNodeClient_CompactionV2_Call) Return(_a0 *commonpb.Status, _a1 error) *MockDataNodeClient_CompactionV2_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockDataNodeClient_Compaction_Call) RunAndReturn(run func(context.Context, *datapb.CompactionPlan, ...grpc.CallOption) (*commonpb.Status, error)) *MockDataNodeClient_Compaction_Call {
func (_c *MockDataNodeClient_CompactionV2_Call) RunAndReturn(run func(context.Context, *datapb.CompactionPlan, ...grpc.CallOption) (*commonpb.Status, error)) *MockDataNodeClient_CompactionV2_Call {
_c.Call.Return(run)
return _c
}
@ -251,9 +251,9 @@ type MockDataNodeClient_DropImport_Call struct {
}
// DropImport is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.DropImportRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.DropImportRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) DropImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_DropImport_Call {
return &MockDataNodeClient_DropImport_Call{Call: _e.mock.On("DropImport",
append([]interface{}{ctx, in}, opts...)...)}
@ -321,9 +321,9 @@ type MockDataNodeClient_FlushChannels_Call struct {
}
// FlushChannels is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.FlushChannelsRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.FlushChannelsRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) FlushChannels(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_FlushChannels_Call {
return &MockDataNodeClient_FlushChannels_Call{Call: _e.mock.On("FlushChannels",
append([]interface{}{ctx, in}, opts...)...)}
@ -391,9 +391,9 @@ type MockDataNodeClient_FlushSegments_Call struct {
}
// FlushSegments is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.FlushSegmentsRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.FlushSegmentsRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) FlushSegments(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_FlushSegments_Call {
return &MockDataNodeClient_FlushSegments_Call{Call: _e.mock.On("FlushSegments",
append([]interface{}{ctx, in}, opts...)...)}
@ -461,9 +461,9 @@ type MockDataNodeClient_GetCompactionState_Call struct {
}
// GetCompactionState is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.CompactionStateRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.CompactionStateRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) GetCompactionState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetCompactionState_Call {
return &MockDataNodeClient_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState",
append([]interface{}{ctx, in}, opts...)...)}
@ -531,9 +531,9 @@ type MockDataNodeClient_GetComponentStates_Call struct {
}
// GetComponentStates is a helper method to define mock.On call
// - ctx context.Context
// - in *milvuspb.GetComponentStatesRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *milvuspb.GetComponentStatesRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) GetComponentStates(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetComponentStates_Call {
return &MockDataNodeClient_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates",
append([]interface{}{ctx, in}, opts...)...)}
@ -601,9 +601,9 @@ type MockDataNodeClient_GetMetrics_Call struct {
}
// GetMetrics is a helper method to define mock.On call
// - ctx context.Context
// - in *milvuspb.GetMetricsRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *milvuspb.GetMetricsRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) GetMetrics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetMetrics_Call {
return &MockDataNodeClient_GetMetrics_Call{Call: _e.mock.On("GetMetrics",
append([]interface{}{ctx, in}, opts...)...)}
@ -671,9 +671,9 @@ type MockDataNodeClient_GetStatisticsChannel_Call struct {
}
// GetStatisticsChannel is a helper method to define mock.On call
// - ctx context.Context
// - in *internalpb.GetStatisticsChannelRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *internalpb.GetStatisticsChannelRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) GetStatisticsChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetStatisticsChannel_Call {
return &MockDataNodeClient_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel",
append([]interface{}{ctx, in}, opts...)...)}
@ -741,9 +741,9 @@ type MockDataNodeClient_ImportV2_Call struct {
}
// ImportV2 is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.ImportRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.ImportRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) ImportV2(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ImportV2_Call {
return &MockDataNodeClient_ImportV2_Call{Call: _e.mock.On("ImportV2",
append([]interface{}{ctx, in}, opts...)...)}
@ -811,9 +811,9 @@ type MockDataNodeClient_NotifyChannelOperation_Call struct {
}
// NotifyChannelOperation is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.ChannelOperationsRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.ChannelOperationsRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) NotifyChannelOperation(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_NotifyChannelOperation_Call {
return &MockDataNodeClient_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation",
append([]interface{}{ctx, in}, opts...)...)}
@ -881,9 +881,9 @@ type MockDataNodeClient_PreImport_Call struct {
}
// PreImport is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.PreImportRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.PreImportRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) PreImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_PreImport_Call {
return &MockDataNodeClient_PreImport_Call{Call: _e.mock.On("PreImport",
append([]interface{}{ctx, in}, opts...)...)}
@ -951,9 +951,9 @@ type MockDataNodeClient_QueryImport_Call struct {
}
// QueryImport is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.QueryImportRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.QueryImportRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) QueryImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_QueryImport_Call {
return &MockDataNodeClient_QueryImport_Call{Call: _e.mock.On("QueryImport",
append([]interface{}{ctx, in}, opts...)...)}
@ -1021,9 +1021,9 @@ type MockDataNodeClient_QueryPreImport_Call struct {
}
// QueryPreImport is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.QueryPreImportRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.QueryPreImportRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) QueryPreImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_QueryPreImport_Call {
return &MockDataNodeClient_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport",
append([]interface{}{ctx, in}, opts...)...)}
@ -1052,6 +1052,76 @@ func (_c *MockDataNodeClient_QueryPreImport_Call) RunAndReturn(run func(context.
return _c
}
// QuerySlot provides a mock function with given fields: ctx, in, opts
func (_m *MockDataNodeClient) QuerySlot(ctx context.Context, in *datapb.QuerySlotRequest, opts ...grpc.CallOption) (*datapb.QuerySlotResponse, error) {
_va := make([]interface{}, len(opts))
for _i := range opts {
_va[_i] = opts[_i]
}
var _ca []interface{}
_ca = append(_ca, ctx, in)
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
var r0 *datapb.QuerySlotResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *datapb.QuerySlotRequest, ...grpc.CallOption) (*datapb.QuerySlotResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *datapb.QuerySlotRequest, ...grpc.CallOption) *datapb.QuerySlotResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*datapb.QuerySlotResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *datapb.QuerySlotRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockDataNodeClient_QuerySlot_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'QuerySlot'
type MockDataNodeClient_QuerySlot_Call struct {
*mock.Call
}
// QuerySlot is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.QuerySlotRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) QuerySlot(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_QuerySlot_Call {
return &MockDataNodeClient_QuerySlot_Call{Call: _e.mock.On("QuerySlot",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockDataNodeClient_QuerySlot_Call) Run(run func(ctx context.Context, in *datapb.QuerySlotRequest, opts ...grpc.CallOption)) *MockDataNodeClient_QuerySlot_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]grpc.CallOption, len(args)-2)
for i, a := range args[2:] {
if a != nil {
variadicArgs[i] = a.(grpc.CallOption)
}
}
run(args[0].(context.Context), args[1].(*datapb.QuerySlotRequest), variadicArgs...)
})
return _c
}
func (_c *MockDataNodeClient_QuerySlot_Call) Return(_a0 *datapb.QuerySlotResponse, _a1 error) *MockDataNodeClient_QuerySlot_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockDataNodeClient_QuerySlot_Call) RunAndReturn(run func(context.Context, *datapb.QuerySlotRequest, ...grpc.CallOption) (*datapb.QuerySlotResponse, error)) *MockDataNodeClient_QuerySlot_Call {
_c.Call.Return(run)
return _c
}
// ResendSegmentStats provides a mock function with given fields: ctx, in, opts
func (_m *MockDataNodeClient) ResendSegmentStats(ctx context.Context, in *datapb.ResendSegmentStatsRequest, opts ...grpc.CallOption) (*datapb.ResendSegmentStatsResponse, error) {
_va := make([]interface{}, len(opts))
@ -1091,9 +1161,9 @@ type MockDataNodeClient_ResendSegmentStats_Call struct {
}
// ResendSegmentStats is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.ResendSegmentStatsRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.ResendSegmentStatsRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) ResendSegmentStats(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ResendSegmentStats_Call {
return &MockDataNodeClient_ResendSegmentStats_Call{Call: _e.mock.On("ResendSegmentStats",
append([]interface{}{ctx, in}, opts...)...)}
@ -1161,9 +1231,9 @@ type MockDataNodeClient_ShowConfigurations_Call struct {
}
// ShowConfigurations is a helper method to define mock.On call
// - ctx context.Context
// - in *internalpb.ShowConfigurationsRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *internalpb.ShowConfigurationsRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) ShowConfigurations(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ShowConfigurations_Call {
return &MockDataNodeClient_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations",
append([]interface{}{ctx, in}, opts...)...)}
@ -1231,9 +1301,9 @@ type MockDataNodeClient_SyncSegments_Call struct {
}
// SyncSegments is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.SyncSegmentsRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.SyncSegmentsRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) SyncSegments(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_SyncSegments_Call {
return &MockDataNodeClient_SyncSegments_Call{Call: _e.mock.On("SyncSegments",
append([]interface{}{ctx, in}, opts...)...)}
@ -1301,9 +1371,9 @@ type MockDataNodeClient_WatchDmChannels_Call struct {
}
// WatchDmChannels is a helper method to define mock.On call
// - ctx context.Context
// - in *datapb.WatchDmChannelsRequest
// - opts ...grpc.CallOption
// - ctx context.Context
// - in *datapb.WatchDmChannelsRequest
// - opts ...grpc.CallOption
func (_e *MockDataNodeClient_Expecter) WatchDmChannels(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_WatchDmChannels_Call {
return &MockDataNodeClient_WatchDmChannels_Call{Call: _e.mock.On("WatchDmChannels",
append([]interface{}{ctx, in}, opts...)...)}
@ -1337,8 +1407,7 @@ func (_c *MockDataNodeClient_WatchDmChannels_Call) RunAndReturn(run func(context
func NewMockDataNodeClient(t interface {
mock.TestingT
Cleanup(func())
},
) *MockDataNodeClient {
}) *MockDataNodeClient {
mock := &MockDataNodeClient{}
mock.Mock.Test(t)

View File

@ -110,7 +110,7 @@ service DataNode {
// https://wiki.lfaidata.foundation/display/MIL/MEP+8+--+Add+metrics+for+proxy
rpc GetMetrics(milvus.GetMetricsRequest) returns (milvus.GetMetricsResponse) {}
rpc Compaction(CompactionPlan) returns (common.Status) {}
rpc CompactionV2(CompactionPlan) returns (common.Status) {}
rpc GetCompactionState(CompactionStateRequest) returns (CompactionStateResponse) {}
rpc SyncSegments(SyncSegmentsRequest) returns (common.Status) {}
@ -127,6 +127,8 @@ service DataNode {
rpc QueryPreImport(QueryPreImportRequest) returns(QueryPreImportResponse) {}
rpc QueryImport(QueryImportRequest) returns(QueryImportResponse) {}
rpc DropImport(DropImportRequest) returns(common.Status) {}
rpc QuerySlot(QuerySlotRequest) returns(QuerySlotResponse) {}
}
message FlushRequest {
@ -526,6 +528,7 @@ message CompactionPlan {
string channel = 7;
int64 collection_ttl = 8;
int64 total_rows = 9;
schema.CollectionSchema schema = 10;
}
message CompactionSegment {
@ -833,3 +836,10 @@ message GcControlRequest {
GcCommand command = 2;
repeated common.KeyValuePair params = 3;
}
message QuerySlotRequest {}
message QuerySlotResponse {
common.Status status = 1;
int64 num_slots = 2;
}

View File

@ -57,7 +57,7 @@ func (m *GrpcDataNodeClient) GetMetrics(ctx context.Context, in *milvuspb.GetMet
return &milvuspb.GetMetricsResponse{}, m.Err
}
func (m *GrpcDataNodeClient) Compaction(ctx context.Context, req *datapb.CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) {
func (m *GrpcDataNodeClient) CompactionV2(ctx context.Context, req *datapb.CompactionPlan, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}
@ -104,3 +104,7 @@ func (m *GrpcDataNodeClient) QueryImport(ctx context.Context, req *datapb.QueryI
func (m *GrpcDataNodeClient) DropImport(ctx context.Context, req *datapb.DropImportRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}
func (m *GrpcDataNodeClient) QuerySlot(ctx context.Context, req *datapb.QuerySlotRequest, opts ...grpc.CallOption) (*datapb.QuerySlotResponse, error) {
return &datapb.QuerySlotResponse{}, m.Err
}

View File

@ -3482,6 +3482,9 @@ type dataNodeConfig struct {
GracefulStopTimeout ParamItem `refreshable:"true"`
BloomFilterApplyParallelFactor ParamItem `refreshable:"true"`
// slot
SlotCap ParamItem `refreshable:"true"`
}
func (p *dataNodeConfig) init(base *BaseTable) {
@ -3796,6 +3799,15 @@ if this parameter <= 0, will set it as 10`,
Export: true,
}
p.BloomFilterApplyParallelFactor.Init(base.mgr)
p.SlotCap = ParamItem{
Key: "dataNode.slot.slotCap",
Version: "2.4.2",
DefaultValue: "2",
Doc: "The maximum number of tasks(e.g. compaction, importing) allowed to run concurrently on a datanode",
Export: true,
}
p.SlotCap.Init(base.mgr)
}
// /////////////////////////////////////////////////////////////////////////////

View File

@ -496,6 +496,7 @@ func TestComponentParam(t *testing.T) {
params.Save("datanode.gracefulStopTimeout", "100")
assert.Equal(t, 100*time.Second, Params.GracefulStopTimeout.GetAsDuration(time.Second))
assert.Equal(t, 4, Params.BloomFilterApplyParallelFactor.GetAsInt())
assert.Equal(t, 2, Params.SlotCap.GetAsInt())
})
t.Run("test indexNodeConfig", func(t *testing.T) {

View File

@ -0,0 +1,47 @@
// 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.
package compaction
import (
"testing"
"github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/tests/integration"
)
type CompactionSuite struct {
integration.MiniClusterSuite
}
func (s *CompactionSuite) SetupSuite() {
s.MiniClusterSuite.SetupSuite()
paramtable.Init()
paramtable.Get().Save(paramtable.Get().DataCoordCfg.GlobalCompactionInterval.Key, "1")
}
func (s *CompactionSuite) TearDownSuite() {
s.MiniClusterSuite.TearDownSuite()
paramtable.Get().Reset(paramtable.Get().DataCoordCfg.GlobalCompactionInterval.Key)
}
func TestCompaction(t *testing.T) {
suite.Run(t, new(CompactionSuite))
}

View File

@ -0,0 +1,238 @@
// 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.
package compaction
import (
"context"
"fmt"
"time"
"github.com/golang/protobuf/proto"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metric"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/tests/integration"
)
func (s *CompactionSuite) TestL0Compaction() {
ctx, cancel := context.WithTimeout(context.Background(), time.Minute*10)
defer cancel()
c := s.Cluster
const (
dim = 128
dbName = ""
rowNum = 100000
deleteCnt = 50000
indexType = integration.IndexFaissIvfFlat
metricType = metric.L2
vecType = schemapb.DataType_FloatVector
)
paramtable.Get().Save(paramtable.Get().DataCoordCfg.LevelZeroCompactionTriggerDeltalogMinNum.Key, "1")
defer paramtable.Get().Reset(paramtable.Get().DataCoordCfg.LevelZeroCompactionTriggerDeltalogMinNum.Key)
collectionName := "TestCompaction_" + funcutil.GenRandomStr()
schema := integration.ConstructSchemaOfVecDataType(collectionName, dim, false, vecType)
marshaledSchema, err := proto.Marshal(schema)
s.NoError(err)
// create collection
createCollectionStatus, err := c.Proxy.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
DbName: dbName,
CollectionName: collectionName,
Schema: marshaledSchema,
ShardsNum: common.DefaultShardsNum,
ConsistencyLevel: commonpb.ConsistencyLevel_Strong,
})
err = merr.CheckRPCCall(createCollectionStatus, err)
s.NoError(err)
log.Info("CreateCollection result", zap.Any("createCollectionStatus", createCollectionStatus))
// show collection
showCollectionsResp, err := c.Proxy.ShowCollections(ctx, &milvuspb.ShowCollectionsRequest{})
err = merr.CheckRPCCall(showCollectionsResp, err)
s.NoError(err)
log.Info("ShowCollections result", zap.Any("showCollectionsResp", showCollectionsResp))
// insert
pkColumn := integration.NewInt64FieldData(integration.Int64Field, rowNum)
fVecColumn := integration.NewFloatVectorFieldData(integration.FloatVecField, rowNum, dim)
hashKeys := integration.GenerateHashKeys(rowNum)
insertResult, err := c.Proxy.Insert(ctx, &milvuspb.InsertRequest{
DbName: dbName,
CollectionName: collectionName,
FieldsData: []*schemapb.FieldData{pkColumn, fVecColumn},
HashKeys: hashKeys,
NumRows: uint32(rowNum),
})
err = merr.CheckRPCCall(insertResult, err)
s.NoError(err)
s.Equal(int64(rowNum), insertResult.GetInsertCnt())
// flush
flushResp, err := c.Proxy.Flush(ctx, &milvuspb.FlushRequest{
DbName: dbName,
CollectionNames: []string{collectionName},
})
err = merr.CheckRPCCall(flushResp, err)
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
ids := segmentIDs.GetData()
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
// create index
createIndexStatus, err := c.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{
CollectionName: collectionName,
FieldName: integration.FloatVecField,
IndexName: "_default",
ExtraParams: integration.ConstructIndexParam(dim, indexType, metricType),
})
err = merr.CheckRPCCall(createIndexStatus, err)
s.NoError(err)
s.WaitForIndexBuilt(ctx, collectionName, integration.FloatVecField)
segments, err := c.MetaWatcher.ShowSegments()
s.NoError(err)
s.NotEmpty(segments)
s.Equal(1, len(segments))
s.Equal(int64(rowNum), segments[0].GetNumOfRows())
// load
loadStatus, err := c.Proxy.LoadCollection(ctx, &milvuspb.LoadCollectionRequest{
DbName: dbName,
CollectionName: collectionName,
})
err = merr.CheckRPCCall(loadStatus, err)
s.NoError(err)
s.WaitForLoad(ctx, collectionName)
// delete
deleteResult, err := c.Proxy.Delete(ctx, &milvuspb.DeleteRequest{
DbName: dbName,
CollectionName: collectionName,
Expr: fmt.Sprintf("%s < %d", integration.Int64Field, deleteCnt),
})
err = merr.CheckRPCCall(deleteResult, err)
s.NoError(err)
// flush l0
flushResp, err = c.Proxy.Flush(ctx, &milvuspb.FlushRequest{
DbName: dbName,
CollectionNames: []string{collectionName},
})
err = merr.CheckRPCCall(flushResp, err)
s.NoError(err)
flushTs, has = flushResp.GetCollFlushTs()[collectionName]
s.True(has)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
// query
queryResult, err := c.Proxy.Query(ctx, &milvuspb.QueryRequest{
DbName: dbName,
CollectionName: collectionName,
Expr: "",
OutputFields: []string{"count(*)"},
})
err = merr.CheckRPCCall(queryResult, err)
s.NoError(err)
s.Equal(int64(rowNum-deleteCnt), queryResult.GetFieldsData()[0].GetScalars().GetLongData().GetData()[0])
// wait for l0 compaction completed
showSegments := func() bool {
segments, err = c.MetaWatcher.ShowSegments()
s.NoError(err)
s.NotEmpty(segments)
log.Info("ShowSegments result", zap.Any("segments", segments))
flushed := lo.Filter(segments, func(segment *datapb.SegmentInfo, _ int) bool {
return segment.GetState() == commonpb.SegmentState_Flushed
})
if len(flushed) == 1 &&
flushed[0].GetLevel() == datapb.SegmentLevel_L1 &&
flushed[0].GetNumOfRows() == rowNum {
log.Info("l0 compaction done, wait for single compaction")
}
return len(flushed) == 1 &&
flushed[0].GetLevel() == datapb.SegmentLevel_L1 &&
flushed[0].GetNumOfRows() == rowNum-deleteCnt
}
for !showSegments() {
select {
case <-ctx.Done():
s.Fail("waiting for compaction timeout")
return
case <-time.After(1 * time.Second):
}
}
// search
expr := fmt.Sprintf("%s > 0", integration.Int64Field)
nq := 10
topk := 10
roundDecimal := -1
params := integration.GetSearchParams(indexType, metricType)
searchReq := integration.ConstructSearchRequest("", collectionName, expr,
integration.FloatVecField, vecType, nil, metricType, params, nq, dim, topk, roundDecimal)
searchResult, err := c.Proxy.Search(ctx, searchReq)
err = merr.CheckRPCCall(searchResult, err)
s.NoError(err)
s.Equal(nq*topk, len(searchResult.GetResults().GetScores()))
// query
queryResult, err = c.Proxy.Query(ctx, &milvuspb.QueryRequest{
DbName: dbName,
CollectionName: collectionName,
Expr: "",
OutputFields: []string{"count(*)"},
})
err = merr.CheckRPCCall(queryResult, err)
s.NoError(err)
s.Equal(int64(rowNum-deleteCnt), queryResult.GetFieldsData()[0].GetScalars().GetLongData().GetData()[0])
// release collection
status, err := c.Proxy.ReleaseCollection(ctx, &milvuspb.ReleaseCollectionRequest{
CollectionName: collectionName,
})
err = merr.CheckRPCCall(status, err)
s.NoError(err)
// drop collection
status, err = c.Proxy.DropCollection(ctx, &milvuspb.DropCollectionRequest{
CollectionName: collectionName,
})
err = merr.CheckRPCCall(status, err)
s.NoError(err)
log.Info("Test compaction succeed")
}

View File

@ -0,0 +1,205 @@
// 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.
package compaction
import (
"context"
"fmt"
"time"
"github.com/golang/protobuf/proto"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metric"
"github.com/milvus-io/milvus/tests/integration"
)
func (s *CompactionSuite) TestMixCompaction() {
ctx, cancel := context.WithTimeout(context.Background(), time.Minute*10)
defer cancel()
c := s.Cluster
const (
dim = 128
dbName = ""
rowNum = 10000
batch = 1000
indexType = integration.IndexFaissIvfFlat
metricType = metric.L2
vecType = schemapb.DataType_FloatVector
)
collectionName := "TestCompaction_" + funcutil.GenRandomStr()
schema := integration.ConstructSchemaOfVecDataType(collectionName, dim, true, vecType)
marshaledSchema, err := proto.Marshal(schema)
s.NoError(err)
// create collection
createCollectionStatus, err := c.Proxy.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
DbName: dbName,
CollectionName: collectionName,
Schema: marshaledSchema,
ShardsNum: common.DefaultShardsNum,
ConsistencyLevel: commonpb.ConsistencyLevel_Strong,
})
err = merr.CheckRPCCall(createCollectionStatus, err)
s.NoError(err)
log.Info("CreateCollection result", zap.Any("createCollectionStatus", createCollectionStatus))
// show collection
showCollectionsResp, err := c.Proxy.ShowCollections(ctx, &milvuspb.ShowCollectionsRequest{})
err = merr.CheckRPCCall(showCollectionsResp, err)
s.NoError(err)
log.Info("ShowCollections result", zap.Any("showCollectionsResp", showCollectionsResp))
for i := 0; i < rowNum/batch; i++ {
// insert
fVecColumn := integration.NewFloatVectorFieldData(integration.FloatVecField, batch, dim)
hashKeys := integration.GenerateHashKeys(batch)
insertResult, err := c.Proxy.Insert(ctx, &milvuspb.InsertRequest{
DbName: dbName,
CollectionName: collectionName,
FieldsData: []*schemapb.FieldData{fVecColumn},
HashKeys: hashKeys,
NumRows: uint32(batch),
})
err = merr.CheckRPCCall(insertResult, err)
s.NoError(err)
s.Equal(int64(batch), insertResult.GetInsertCnt())
// flush
flushResp, err := c.Proxy.Flush(ctx, &milvuspb.FlushRequest{
DbName: dbName,
CollectionNames: []string{collectionName},
})
err = merr.CheckRPCCall(flushResp, err)
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
ids := segmentIDs.GetData()
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
log.Info("insert done", zap.Int("i", i))
}
// create index
createIndexStatus, err := c.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{
CollectionName: collectionName,
FieldName: integration.FloatVecField,
IndexName: "_default",
ExtraParams: integration.ConstructIndexParam(dim, indexType, metricType),
})
err = merr.CheckRPCCall(createIndexStatus, err)
s.NoError(err)
s.WaitForIndexBuilt(ctx, collectionName, integration.FloatVecField)
segments, err := c.MetaWatcher.ShowSegments()
s.NoError(err)
s.NotEmpty(segments)
s.Equal(rowNum/batch, len(segments))
for _, segment := range segments {
log.Info("show segment result", zap.String("segment", segment.String()))
}
// wait for compaction completed
showSegments := func() bool {
segments, err = c.MetaWatcher.ShowSegments()
s.NoError(err)
s.NotEmpty(segments)
compactFromSegments := lo.Filter(segments, func(segment *datapb.SegmentInfo, _ int) bool {
return segment.GetState() == commonpb.SegmentState_Dropped
})
compactToSegments := lo.Filter(segments, func(segment *datapb.SegmentInfo, _ int) bool {
return segment.GetState() == commonpb.SegmentState_Flushed
})
log.Info("ShowSegments result", zap.Int("len(compactFromSegments)", len(compactFromSegments)),
zap.Int("len(compactToSegments)", len(compactToSegments)))
return len(compactToSegments) == 1
}
for !showSegments() {
select {
case <-ctx.Done():
s.Fail("waiting for compaction timeout")
return
case <-time.After(1 * time.Second):
}
}
// load
loadStatus, err := c.Proxy.LoadCollection(ctx, &milvuspb.LoadCollectionRequest{
DbName: dbName,
CollectionName: collectionName,
})
err = merr.CheckRPCCall(loadStatus, err)
s.NoError(err)
s.WaitForLoad(ctx, collectionName)
// search
expr := fmt.Sprintf("%s > 0", integration.Int64Field)
nq := 10
topk := 10
roundDecimal := -1
params := integration.GetSearchParams(indexType, metricType)
searchReq := integration.ConstructSearchRequest("", collectionName, expr,
integration.FloatVecField, vecType, nil, metricType, params, nq, dim, topk, roundDecimal)
searchResult, err := c.Proxy.Search(ctx, searchReq)
err = merr.CheckRPCCall(searchResult, err)
s.NoError(err)
s.Equal(nq*topk, len(searchResult.GetResults().GetScores()))
// query
queryResult, err := c.Proxy.Query(ctx, &milvuspb.QueryRequest{
DbName: dbName,
CollectionName: collectionName,
Expr: "",
OutputFields: []string{"count(*)"},
})
err = merr.CheckRPCCall(queryResult, err)
s.NoError(err)
s.Equal(int64(rowNum), queryResult.GetFieldsData()[0].GetScalars().GetLongData().GetData()[0])
// release collection
status, err := c.Proxy.ReleaseCollection(ctx, &milvuspb.ReleaseCollectionRequest{
CollectionName: collectionName,
})
err = merr.CheckRPCCall(status, err)
s.NoError(err)
// drop collection
status, err = c.Proxy.DropCollection(ctx, &milvuspb.DropCollectionRequest{
CollectionName: collectionName,
})
err = merr.CheckRPCCall(status, err)
s.NoError(err)
log.Info("Test compaction succeed")
}