Skip to content

Commit

Permalink
enhance: [cherry-pick] Batch pick PRs related to compaction (milvus-i…
Browse files Browse the repository at this point in the history
…o#34315)

This PR cherry-picks the following commits related to compaction:

- Use a pool for CompactionExecutor.
milvus-io#33558
- Move compaction executor to compaction pacakge.
milvus-io#33778
- Ensure the idempotency of compaction tasks.
milvus-io#33872
- Add comment for channel cp updater.
milvus-io#33759

issue: milvus-io#33182,
milvus-io#32451

pr: milvus-io#33558,
milvus-io#33778,
milvus-io#33872,
milvus-io#33759

---------

Signed-off-by: coldWater <254244460@qq.com>
Signed-off-by: bigsheeper <yihao.dai@zilliz.com>
Co-authored-by: coldWater <254244460@qq.com>
  • Loading branch information
bigsheeper and forsaken628 committed Jul 2, 2024
1 parent c924b0b commit 1b95ee7
Show file tree
Hide file tree
Showing 10 changed files with 181 additions and 142 deletions.
7 changes: 6 additions & 1 deletion internal/datanode/channel_checkpoint_updater.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ const (
type channelCPUpdateTask struct {
pos *msgpb.MsgPosition
callback func()
flush bool
flush bool // indicates whether the task originates from flush
}

type channelCheckpointUpdater struct {
Expand Down Expand Up @@ -74,6 +74,7 @@ func (ccu *channelCheckpointUpdater) start() {
ccu.mu.Lock()
for _, task := range ccu.tasks {
if task.flush {
// reset flush flag to make next flush valid
task.flush = false
tasks = append(tasks, task)
}
Expand Down Expand Up @@ -142,6 +143,7 @@ func (ccu *channelCheckpointUpdater) updateCheckpoints(tasks []*channelCPUpdateT
defer ccu.mu.Unlock()
finished.Range(func(_ string, task *channelCPUpdateTask) bool {
channel := task.pos.GetChannelName()
// delete the task if no new task has been added
if ccu.tasks[channel].pos.GetTimestamp() <= task.pos.GetTimestamp() {
delete(ccu.tasks, channel)
}
Expand All @@ -163,6 +165,7 @@ func (ccu *channelCheckpointUpdater) AddTask(channelPos *msgpb.MsgPosition, flus
return
}
if flush {
// trigger update to accelerate flush
defer ccu.trigger()
}
channel := channelPos.GetChannelName()
Expand All @@ -184,6 +187,8 @@ func (ccu *channelCheckpointUpdater) AddTask(channelPos *msgpb.MsgPosition, flus
}
return b
}
// 1. `task.pos.GetTimestamp() < channelPos.GetTimestamp()`: position updated, update task position
// 2. `flush && !task.flush`: position not being updated, but flush is triggered, update task flush flag
if task.pos.GetTimestamp() < channelPos.GetTimestamp() || (flush && !task.flush) {
ccu.mu.Lock()
defer ccu.mu.Unlock()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,95 +14,117 @@
// See the License for the specific language governing permissions and
// limitations under the License.

package datanode
package compaction

import (
"context"
"sync"

"github.com/samber/lo"
"go.uber.org/zap"
"golang.org/x/sync/semaphore"

"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/datanode/compaction"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)

const (
maxTaskNum = 1024
maxTaskQueueNum = 1024
maxParallelTaskNum = 10
)

type compactionExecutor struct {
executing *typeutil.ConcurrentMap[int64, compaction.Compactor] // planID to compactor
completedCompactor *typeutil.ConcurrentMap[int64, compaction.Compactor] // planID to compactor
type Executor interface {
Start(ctx context.Context)
Execute(task Compactor)
Slots() int64
RemoveTask(planID int64)
GetResults(planID int64) []*datapb.CompactionPlanResult
DiscardByDroppedChannel(channel string)
DiscardPlan(channel string)
}

type executor struct {
executing *typeutil.ConcurrentMap[int64, Compactor] // planID to compactor
completedCompactor *typeutil.ConcurrentMap[int64, Compactor] // planID to compactor
completed *typeutil.ConcurrentMap[int64, *datapb.CompactionPlanResult] // planID to CompactionPlanResult
taskCh chan compaction.Compactor
taskCh chan Compactor
taskSem *semaphore.Weighted
dropped *typeutil.ConcurrentSet[string] // vchannel dropped

// To prevent concurrency of release channel and compaction get results
// all released channel's compaction tasks will be discarded
resultGuard sync.RWMutex
}

func newCompactionExecutor() *compactionExecutor {
return &compactionExecutor{
executing: typeutil.NewConcurrentMap[int64, compaction.Compactor](),
completedCompactor: typeutil.NewConcurrentMap[int64, compaction.Compactor](),
func NewExecutor() *executor {
return &executor{
executing: typeutil.NewConcurrentMap[int64, Compactor](),
completedCompactor: typeutil.NewConcurrentMap[int64, Compactor](),
completed: typeutil.NewConcurrentMap[int64, *datapb.CompactionPlanResult](),
taskCh: make(chan compaction.Compactor, maxTaskNum),
taskCh: make(chan Compactor, maxTaskQueueNum),
taskSem: semaphore.NewWeighted(maxParallelTaskNum),
dropped: typeutil.NewConcurrentSet[string](),
}
}

func (c *compactionExecutor) execute(task compaction.Compactor) {
c.taskCh <- task
c.toExecutingState(task)
func (e *executor) Execute(task Compactor) {
_, ok := e.executing.GetOrInsert(task.GetPlanID(), task)
if ok {
log.Warn("duplicated compaction task",
zap.Int64("planID", task.GetPlanID()),
zap.String("channel", task.GetChannelName()))
return
}
e.taskCh <- task
}

func (c *compactionExecutor) toExecutingState(task compaction.Compactor) {
c.executing.Insert(task.GetPlanID(), task)
func (e *executor) Slots() int64 {
return paramtable.Get().DataNodeCfg.SlotCap.GetAsInt64() - int64(e.executing.Len())
}

func (c *compactionExecutor) toCompleteState(task compaction.Compactor) {
func (e *executor) toCompleteState(task Compactor) {
task.Complete()
c.executing.GetAndRemove(task.GetPlanID())
e.executing.GetAndRemove(task.GetPlanID())
}

func (c *compactionExecutor) removeTask(planID UniqueID) {
c.completed.GetAndRemove(planID)
task, loaded := c.completedCompactor.GetAndRemove(planID)
func (e *executor) RemoveTask(planID int64) {
e.completed.GetAndRemove(planID)
task, loaded := e.completedCompactor.GetAndRemove(planID)
if loaded {
log.Info("Compaction task removed", zap.Int64("planID", planID), zap.String("channel", task.GetChannelName()))
}
}

// These two func are bounded for waitGroup
func (c *compactionExecutor) executeWithState(task compaction.Compactor) {
go c.executeTask(task)
}

func (c *compactionExecutor) start(ctx context.Context) {
func (e *executor) Start(ctx context.Context) {
for {
select {
case <-ctx.Done():
return
case task := <-c.taskCh:
c.executeWithState(task)
case task := <-e.taskCh:
err := e.taskSem.Acquire(ctx, 1)
if err != nil {
return
}
go func() {
defer e.taskSem.Release(1)
e.executeTask(task)
}()
}
}
}

func (c *compactionExecutor) executeTask(task compaction.Compactor) {
func (e *executor) executeTask(task Compactor) {
log := log.With(
zap.Int64("planID", task.GetPlanID()),
zap.Int64("Collection", task.GetCollection()),
zap.String("channel", task.GetChannelName()),
)

defer func() {
c.toCompleteState(task)
e.toCompleteState(task)
}()

log.Info("start to execute compaction")
Expand All @@ -112,45 +134,45 @@ func (c *compactionExecutor) executeTask(task compaction.Compactor) {
log.Warn("compaction task failed", zap.Error(err))
return
}
c.completed.Insert(result.GetPlanID(), result)
c.completedCompactor.Insert(result.GetPlanID(), task)
e.completed.Insert(result.GetPlanID(), result)
e.completedCompactor.Insert(result.GetPlanID(), task)

log.Info("end to execute compaction")
}

func (c *compactionExecutor) stopTask(planID UniqueID) {
task, loaded := c.executing.GetAndRemove(planID)
func (e *executor) stopTask(planID int64) {
task, loaded := e.executing.GetAndRemove(planID)
if loaded {
log.Warn("compaction executor stop task", zap.Int64("planID", planID), zap.String("vChannelName", task.GetChannelName()))
task.Stop()
}
}

func (c *compactionExecutor) isValidChannel(channel string) bool {
func (e *executor) isValidChannel(channel string) bool {
// if vchannel marked dropped, compaction should not proceed
return !c.dropped.Contain(channel)
return !e.dropped.Contain(channel)
}

func (c *compactionExecutor) discardByDroppedChannel(channel string) {
c.dropped.Insert(channel)
c.discardPlan(channel)
func (e *executor) DiscardByDroppedChannel(channel string) {
e.dropped.Insert(channel)
e.DiscardPlan(channel)
}

func (c *compactionExecutor) discardPlan(channel string) {
c.resultGuard.Lock()
defer c.resultGuard.Unlock()
func (e *executor) DiscardPlan(channel string) {
e.resultGuard.Lock()
defer e.resultGuard.Unlock()

c.executing.Range(func(planID int64, task compaction.Compactor) bool {
e.executing.Range(func(planID int64, task Compactor) bool {
if task.GetChannelName() == channel {
c.stopTask(planID)
e.stopTask(planID)
}
return true
})

// remove all completed plans of channel
c.completed.Range(func(planID int64, result *datapb.CompactionPlanResult) bool {
e.completed.Range(func(planID int64, result *datapb.CompactionPlanResult) bool {
if result.GetChannel() == channel {
c.removeTask(planID)
e.RemoveTask(planID)
log.Info("remove compaction plan and results",
zap.String("channel", channel),
zap.Int64("planID", planID))
Expand All @@ -159,35 +181,43 @@ func (c *compactionExecutor) discardPlan(channel string) {
})
}

func (c *compactionExecutor) getCompactionResult(planID int64) *datapb.CompactionPlanResult {
c.resultGuard.RLock()
defer c.resultGuard.RUnlock()
_, ok := c.executing.Get(planID)
func (e *executor) GetResults(planID int64) []*datapb.CompactionPlanResult {
if planID != 0 {
result := e.getCompactionResult(planID)
return []*datapb.CompactionPlanResult{result}
}
return e.getAllCompactionResults()
}

func (e *executor) getCompactionResult(planID int64) *datapb.CompactionPlanResult {
e.resultGuard.RLock()
defer e.resultGuard.RUnlock()
_, ok := e.executing.Get(planID)
if ok {
result := &datapb.CompactionPlanResult{
State: commonpb.CompactionState_Executing,
PlanID: planID,
}
return result
}
result, ok2 := c.completed.Get(planID)
result, ok2 := e.completed.Get(planID)
if !ok2 {
return &datapb.CompactionPlanResult{}
}
return result
}

func (c *compactionExecutor) getAllCompactionResults() []*datapb.CompactionPlanResult {
c.resultGuard.RLock()
defer c.resultGuard.RUnlock()
func (e *executor) getAllCompactionResults() []*datapb.CompactionPlanResult {
e.resultGuard.RLock()
defer e.resultGuard.RUnlock()
var (
executing []int64
completed []int64
completedLevelZero []int64
)
results := make([]*datapb.CompactionPlanResult, 0)
// get executing results
c.executing.Range(func(planID int64, task compaction.Compactor) bool {
e.executing.Range(func(planID int64, task Compactor) bool {
executing = append(executing, planID)
results = append(results, &datapb.CompactionPlanResult{
State: commonpb.CompactionState_Executing,
Expand All @@ -197,7 +227,7 @@ func (c *compactionExecutor) getAllCompactionResults() []*datapb.CompactionPlanR
})

// get completed results
c.completed.Range(func(planID int64, result *datapb.CompactionPlanResult) bool {
e.completed.Range(func(planID int64, result *datapb.CompactionPlanResult) bool {
completed = append(completed, planID)
results = append(results, result)

Expand All @@ -209,8 +239,8 @@ func (c *compactionExecutor) getAllCompactionResults() []*datapb.CompactionPlanR

// remove level zero results
lo.ForEach(completedLevelZero, func(planID int64, _ int) {
c.completed.Remove(planID)
c.completedCompactor.Remove(planID)
e.completed.Remove(planID)
e.completedCompactor.Remove(planID)
})

if len(results) > 0 {
Expand Down
Loading

0 comments on commit 1b95ee7

Please sign in to comment.