2021-12-29 11:38:57 +08:00
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
2021-11-05 22:25:00 +08:00
package datacoord
import (
"context"
2022-09-25 20:12:52 +08:00
"fmt"
2022-06-15 23:14:10 +08:00
"sort"
2021-11-05 22:25:00 +08:00
"sync"
"time"
2023-01-04 19:37:36 +08:00
"github.com/samber/lo"
"go.uber.org/zap"
2023-06-09 01:28:37 +08:00
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
2023-11-23 17:30:25 +08:00
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
2024-01-29 10:17:02 +08:00
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/metastore/model"
2021-11-05 22:25:00 +08:00
"github.com/milvus-io/milvus/internal/proto/datapb"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
2024-07-11 14:44:52 +08:00
"github.com/milvus-io/milvus/pkg/util/lifetime"
2024-05-15 16:33:34 +08:00
"github.com/milvus-io/milvus/pkg/util/lock"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/pkg/util/logutil"
2024-03-05 13:39:01 +08:00
"github.com/milvus-io/milvus/pkg/util/paramtable"
2023-09-21 09:45:27 +08:00
"github.com/milvus-io/milvus/pkg/util/tsoutil"
2024-01-29 10:17:02 +08:00
"github.com/milvus-io/milvus/pkg/util/typeutil"
2021-11-05 22:25:00 +08:00
)
2022-06-15 23:14:10 +08:00
type compactTime struct {
2024-06-05 10:17:50 +08:00
startTime Timestamp
2022-10-10 20:31:22 +08:00
expireTime Timestamp
collectionTTL time . Duration
2021-11-05 22:25:00 +08:00
}
2024-06-05 10:17:50 +08:00
// todo: migrate to compaction_trigger_v2
2021-11-05 22:25:00 +08:00
type trigger interface {
start ( )
stop ( )
2021-12-16 09:57:25 +08:00
// triggerSingleCompaction triggers a compaction bundled with collection-partition-channel-segment
2023-12-12 10:16:39 +08:00
triggerSingleCompaction ( collectionID , partitionID , segmentID int64 , channel string , blockToSendSignal bool ) error
2024-06-05 10:17:50 +08:00
// triggerManualCompaction force to start a compaction
triggerManualCompaction ( collectionID int64 ) ( UniqueID , error )
2021-11-05 22:25:00 +08:00
}
type compactionSignal struct {
id UniqueID
isForce bool
isGlobal bool
collectionID UniqueID
partitionID UniqueID
channel string
2023-11-23 17:30:25 +08:00
segmentID UniqueID
pos * msgpb . MsgPosition
2021-11-05 22:25:00 +08:00
}
var _ trigger = ( * compactionTrigger ) ( nil )
type compactionTrigger struct {
2023-01-04 19:37:36 +08:00
handler Handler
meta * meta
allocator allocator
signals chan * compactionSignal
compactionHandler compactionPlanContext
globalTrigger * time . Ticker
2024-05-15 16:33:34 +08:00
forceMu lock . Mutex
2024-07-11 14:44:52 +08:00
closeCh lifetime . SafeChan
closeWaiter sync . WaitGroup
2023-11-21 09:26:22 +08:00
indexEngineVersionManager IndexEngineVersionManager
2022-12-13 15:39:21 +08:00
estimateNonDiskSegmentPolicy calUpperLimitPolicy
estimateDiskSegmentPolicy calUpperLimitPolicy
// A sloopy hack, so we can test with different segment row count without worrying that
// they are re-calculated in every compaction.
testingOnly bool
2021-11-05 22:25:00 +08:00
}
2022-09-16 11:32:48 +08:00
func newCompactionTrigger (
meta * meta ,
compactionHandler compactionPlanContext ,
allocator allocator ,
2022-10-11 21:39:24 +08:00
handler Handler ,
2023-11-21 09:26:22 +08:00
indexVersionManager IndexEngineVersionManager ,
2022-09-16 11:32:48 +08:00
) * compactionTrigger {
2021-11-05 22:25:00 +08:00
return & compactionTrigger {
2023-11-21 09:26:22 +08:00
meta : meta ,
allocator : allocator ,
signals : make ( chan * compactionSignal , 100 ) ,
compactionHandler : compactionHandler ,
indexEngineVersionManager : indexVersionManager ,
2022-12-13 15:39:21 +08:00
estimateDiskSegmentPolicy : calBySchemaPolicyWithDiskIndex ,
estimateNonDiskSegmentPolicy : calBySchemaPolicy ,
handler : handler ,
2024-07-11 14:44:52 +08:00
closeCh : lifetime . NewSafeChan ( ) ,
2021-11-05 22:25:00 +08:00
}
}
func ( t * compactionTrigger ) start ( ) {
2022-12-07 18:01:19 +08:00
t . globalTrigger = time . NewTicker ( Params . DataCoordCfg . GlobalCompactionInterval . GetAsDuration ( time . Second ) )
2024-07-11 14:44:52 +08:00
t . closeWaiter . Add ( 2 )
2021-11-05 22:25:00 +08:00
go func ( ) {
defer logutil . LogPanic ( )
2024-07-11 14:44:52 +08:00
defer t . closeWaiter . Done ( )
2021-11-05 22:25:00 +08:00
for {
select {
2024-07-11 14:44:52 +08:00
case <- t . closeCh . CloseCh ( ) :
2022-03-02 15:35:55 +08:00
log . Info ( "compaction trigger quit" )
2021-11-05 22:25:00 +08:00
return
case signal := <- t . signals :
switch {
case signal . isGlobal :
2023-11-30 17:06:31 +08:00
// ManualCompaction also use use handleGlobalSignal
// so throw err here
err := t . handleGlobalSignal ( signal )
if err != nil {
log . Warn ( "unable to handleGlobalSignal" , zap . Error ( err ) )
}
2021-11-05 22:25:00 +08:00
default :
2023-11-30 17:06:31 +08:00
// no need to handle err in handleSignal
2021-11-05 22:25:00 +08:00
t . handleSignal ( signal )
2022-06-15 23:14:10 +08:00
// shouldn't reset, otherwise a frequent flushed collection will affect other collections
// t.globalTrigger.Reset(Params.DataCoordCfg.GlobalCompactionInterval)
2021-11-05 22:25:00 +08:00
}
}
}
} ( )
go t . startGlobalCompactionLoop ( )
}
func ( t * compactionTrigger ) startGlobalCompactionLoop ( ) {
defer logutil . LogPanic ( )
2024-07-11 14:44:52 +08:00
defer t . closeWaiter . Done ( )
2021-11-05 22:25:00 +08:00
2021-12-16 09:59:12 +08:00
// If AutoCompaction disabled, global loop will not start
2022-12-07 18:01:19 +08:00
if ! Params . DataCoordCfg . EnableAutoCompaction . GetAsBool ( ) {
2021-12-08 19:47:05 +08:00
return
}
2021-11-05 22:25:00 +08:00
for {
select {
2024-07-11 14:44:52 +08:00
case <- t . closeCh . CloseCh ( ) :
2021-11-05 22:25:00 +08:00
t . globalTrigger . Stop ( )
log . Info ( "global compaction loop exit" )
return
case <- t . globalTrigger . C :
2022-10-10 20:31:22 +08:00
err := t . triggerCompaction ( )
2022-01-07 17:41:40 +08:00
if err != nil {
log . Warn ( "unable to triggerCompaction" , zap . Error ( err ) )
}
2021-11-05 22:25:00 +08:00
}
}
}
func ( t * compactionTrigger ) stop ( ) {
2024-07-11 14:44:52 +08:00
t . closeCh . Close ( )
t . closeWaiter . Wait ( )
2021-11-05 22:25:00 +08:00
}
2023-05-10 17:45:20 +08:00
func ( t * compactionTrigger ) getCollection ( collectionID UniqueID ) ( * collectionInfo , error ) {
2022-10-11 21:39:24 +08:00
ctx , cancel := context . WithTimeout ( context . Background ( ) , time . Second )
defer cancel ( )
coll , err := t . handler . GetCollection ( ctx , collectionID )
if err != nil {
return nil , fmt . Errorf ( "collection ID %d not found, err: %w" , collectionID , err )
2022-10-10 20:31:22 +08:00
}
2023-05-10 17:45:20 +08:00
return coll , nil
}
func ( t * compactionTrigger ) isCollectionAutoCompactionEnabled ( coll * collectionInfo ) bool {
enabled , err := getCollectionAutoCompactionEnabled ( coll . Properties )
if err != nil {
log . Warn ( "collection properties auto compaction not valid, returning false" , zap . Error ( err ) )
return false
}
return enabled
}
2022-10-10 20:31:22 +08:00
2024-03-05 13:39:01 +08:00
func ( t * compactionTrigger ) isChannelCheckpointHealthy ( vchanName string ) bool {
if paramtable . Get ( ) . DataCoordCfg . ChannelCheckpointMaxLag . GetAsInt64 ( ) <= 0 {
return true
}
checkpoint := t . meta . GetChannelCheckpoint ( vchanName )
if checkpoint == nil {
log . Warn ( "channel checkpoint not found" , zap . String ( "channel" , vchanName ) )
return false
}
cpTime := tsoutil . PhysicalTime ( checkpoint . GetTimestamp ( ) )
return time . Since ( cpTime ) < paramtable . Get ( ) . DataCoordCfg . ChannelCheckpointMaxLag . GetAsDuration ( time . Second )
}
2024-06-05 10:17:50 +08:00
func getCompactTime ( ts Timestamp , coll * collectionInfo ) ( * compactTime , error ) {
2022-10-10 20:31:22 +08:00
collectionTTL , err := getCollectionTTL ( coll . Properties )
if err != nil {
return nil , err
}
pts , _ := tsoutil . ParseTS ( ts )
if collectionTTL > 0 {
ttexpired := pts . Add ( - collectionTTL )
ttexpiredLogic := tsoutil . ComposeTS ( ttexpired . UnixNano ( ) / int64 ( time . Millisecond ) , 0 )
2024-06-05 10:17:50 +08:00
return & compactTime { ts , ttexpiredLogic , collectionTTL } , nil
2022-10-10 20:31:22 +08:00
}
// no expiration time
2024-06-05 10:17:50 +08:00
return & compactTime { ts , 0 , 0 } , nil
2022-10-10 20:31:22 +08:00
}
2021-11-05 22:25:00 +08:00
// triggerCompaction trigger a compaction if any compaction condition satisfy.
2022-10-10 20:31:22 +08:00
func ( t * compactionTrigger ) triggerCompaction ( ) error {
2021-11-05 22:25:00 +08:00
id , err := t . allocSignalID ( )
if err != nil {
return err
}
signal := & compactionSignal {
2022-10-10 20:31:22 +08:00
id : id ,
isForce : false ,
isGlobal : true ,
2021-11-05 22:25:00 +08:00
}
t . signals <- signal
return nil
}
2024-06-05 10:17:50 +08:00
// triggerSingleCompaction trigger a compaction bundled with collection-partition-channel-segment
2023-12-12 10:16:39 +08:00
func ( t * compactionTrigger ) triggerSingleCompaction ( collectionID , partitionID , segmentID int64 , channel string , blockToSendSignal bool ) error {
2023-03-03 14:13:49 +08:00
// If AutoCompaction disabled, flush request will not trigger compaction
2024-07-11 14:44:52 +08:00
if ! paramtable . Get ( ) . DataCoordCfg . EnableAutoCompaction . GetAsBool ( ) && ! paramtable . Get ( ) . DataCoordCfg . EnableCompaction . GetAsBool ( ) {
2021-12-08 19:47:05 +08:00
return nil
}
2021-11-05 22:25:00 +08:00
id , err := t . allocSignalID ( )
if err != nil {
return err
}
signal := & compactionSignal {
id : id ,
isForce : false ,
isGlobal : false ,
collectionID : collectionID ,
partitionID : partitionID ,
segmentID : segmentID ,
channel : channel ,
}
2023-12-12 10:16:39 +08:00
if blockToSendSignal {
t . signals <- signal
return nil
}
select {
case t . signals <- signal :
default :
log . Info ( "no space to send compaction signal" , zap . Int64 ( "collectionID" , collectionID ) , zap . Int64 ( "segmentID" , segmentID ) , zap . String ( "channel" , channel ) )
}
2021-11-05 22:25:00 +08:00
return nil
}
2024-06-05 10:17:50 +08:00
// triggerManualCompaction force to start a compaction
2022-02-18 14:51:49 +08:00
// invoked by user `ManualCompaction` operation
2024-06-05 10:17:50 +08:00
func ( t * compactionTrigger ) triggerManualCompaction ( collectionID int64 ) ( UniqueID , error ) {
2021-11-05 22:25:00 +08:00
id , err := t . allocSignalID ( )
if err != nil {
return - 1 , err
}
signal := & compactionSignal {
id : id ,
isForce : true ,
2022-02-18 14:51:49 +08:00
isGlobal : true ,
2021-11-05 22:25:00 +08:00
collectionID : collectionID ,
}
2023-11-30 17:06:31 +08:00
err = t . handleGlobalSignal ( signal )
if err != nil {
2024-06-05 10:17:50 +08:00
log . Warn ( "unable to handle compaction signal" , zap . Error ( err ) )
2023-11-30 17:06:31 +08:00
return - 1 , err
}
2021-11-05 22:25:00 +08:00
return id , nil
}
func ( t * compactionTrigger ) allocSignalID ( ) ( UniqueID , error ) {
ctx , cancel := context . WithTimeout ( context . Background ( ) , 5 * time . Second )
defer cancel ( )
return t . allocator . allocID ( ctx )
}
2024-04-23 12:11:24 +08:00
func ( t * compactionTrigger ) getExpectedSegmentSize ( collectionID int64 ) int64 {
indexInfos := t . meta . indexMeta . GetIndexesForCollection ( collectionID , "" )
2022-09-25 20:12:52 +08:00
2024-01-29 10:17:02 +08:00
ctx , cancel := context . WithTimeout ( context . Background ( ) , time . Second )
defer cancel ( )
collMeta , err := t . handler . GetCollection ( ctx , collectionID )
if err != nil {
2024-04-23 12:11:24 +08:00
log . Warn ( "failed to get collection" , zap . Int64 ( "collectionID" , collectionID ) , zap . Error ( err ) )
return Params . DataCoordCfg . SegmentMaxSize . GetAsInt64 ( ) * 1024 * 1024
2024-01-29 10:17:02 +08:00
}
2024-04-23 12:11:24 +08:00
2024-01-29 10:17:02 +08:00
vectorFields := typeutil . GetVectorFieldSchemas ( collMeta . Schema )
fieldIndexTypes := lo . SliceToMap ( indexInfos , func ( t * model . Index ) ( int64 , indexparamcheck . IndexType ) {
2024-03-18 15:17:10 +08:00
return t . FieldID , GetIndexType ( t . IndexParams )
2024-01-29 10:17:02 +08:00
} )
vectorFieldsWithDiskIndex := lo . Filter ( vectorFields , func ( field * schemapb . FieldSchema , _ int ) bool {
if indexType , ok := fieldIndexTypes [ field . FieldID ] ; ok {
return indexparamcheck . IsDiskIndex ( indexType )
}
return false
} )
allDiskIndex := len ( vectorFields ) == len ( vectorFieldsWithDiskIndex )
if allDiskIndex {
// Only if all vector fields index type are DiskANN, recalc segment max size here.
2024-04-23 12:11:24 +08:00
return Params . DataCoordCfg . DiskSegmentMaxSize . GetAsInt64 ( ) * 1024 * 1024
2022-09-25 20:12:52 +08:00
}
2024-01-29 10:17:02 +08:00
// If some vector fields index type are not DiskANN, recalc segment max size using default policy.
2024-04-23 12:11:24 +08:00
return Params . DataCoordCfg . SegmentMaxSize . GetAsInt64 ( ) * 1024 * 1024
2022-09-25 20:12:52 +08:00
}
2023-11-30 17:06:31 +08:00
func ( t * compactionTrigger ) handleGlobalSignal ( signal * compactionSignal ) error {
2021-11-05 22:25:00 +08:00
t . forceMu . Lock ( )
defer t . forceMu . Unlock ( )
2023-11-30 17:06:31 +08:00
log := log . With ( zap . Int64 ( "compactionID" , signal . id ) ,
zap . Int64 ( "signal.collectionID" , signal . collectionID ) ,
zap . Int64 ( "signal.partitionID" , signal . partitionID ) ,
zap . Int64 ( "signal.segmentID" , signal . segmentID ) )
2024-06-05 10:17:50 +08:00
partSegments := t . meta . GetSegmentsChanPart ( func ( segment * SegmentInfo ) bool {
2022-02-18 14:51:49 +08:00
return ( signal . collectionID == 0 || segment . CollectionID == signal . collectionID ) &&
isSegmentHealthy ( segment ) &&
isFlush ( segment ) &&
2022-09-26 18:06:54 +08:00
! segment . isCompacting && // not compacting now
2023-12-05 18:44:37 +08:00
! segment . GetIsImporting ( ) && // not importing now
2024-06-10 21:34:08 +08:00
segment . GetLevel ( ) != datapb . SegmentLevel_L0 && // ignore level zero segments
segment . GetLevel ( ) != datapb . SegmentLevel_L2 // ignore l2 segment
2024-06-05 10:17:50 +08:00
} ) // partSegments is list of chanPartSegments, which is channel-partition organized segments
2022-09-22 15:48:50 +08:00
2024-06-05 10:17:50 +08:00
if len ( partSegments ) == 0 {
2023-11-30 17:06:31 +08:00
log . Info ( "the length of SegmentsChanPart is 0, skip to handle compaction" )
return nil
2022-10-10 20:31:22 +08:00
}
2024-03-05 13:39:01 +08:00
channelCheckpointOK := make ( map [ string ] bool )
isChannelCPOK := func ( channelName string ) bool {
cached , ok := channelCheckpointOK [ channelName ]
if ok {
return cached
}
return t . isChannelCheckpointHealthy ( channelName )
}
2024-06-05 10:17:50 +08:00
for _ , group := range partSegments {
2023-11-30 17:06:31 +08:00
log := log . With ( zap . Int64 ( "collectionID" , group . collectionID ) ,
zap . Int64 ( "partitionID" , group . partitionID ) ,
zap . String ( "channel" , group . channelName ) )
2022-02-18 14:51:49 +08:00
if ! signal . isForce && t . compactionHandler . isFull ( ) {
2023-11-30 17:06:31 +08:00
log . Warn ( "compaction plan skipped due to handler full" )
2022-02-18 14:51:49 +08:00
break
}
2024-03-05 13:39:01 +08:00
if ! isChannelCPOK ( group . channelName ) && ! signal . isForce {
log . Warn ( "compaction plan skipped due to channel checkpoint lag" , zap . String ( "channel" , signal . channel ) )
continue
}
2023-07-23 21:31:00 +08:00
if Params . DataCoordCfg . IndexBasedCompaction . GetAsBool ( ) {
group . segments = FilterInIndexedSegments ( t . handler , t . meta , group . segments ... )
}
2022-09-25 20:12:52 +08:00
2023-05-10 17:45:20 +08:00
coll , err := t . getCollection ( group . collectionID )
if err != nil {
2023-11-30 17:06:31 +08:00
log . Warn ( "get collection info failed, skip handling compaction" , zap . Error ( err ) )
return err
2023-05-10 17:45:20 +08:00
}
if ! signal . isForce && ! t . isCollectionAutoCompactionEnabled ( coll ) {
2024-07-12 00:55:34 +08:00
log . RatedInfo ( 20 , "collection auto compaction disabled" )
2023-11-30 17:06:31 +08:00
return nil
2023-05-10 17:45:20 +08:00
}
2024-06-05 10:17:50 +08:00
ct , err := getCompactTime ( tsoutil . ComposeTSByTime ( time . Now ( ) , 0 ) , coll )
2022-10-10 20:31:22 +08:00
if err != nil {
2024-07-12 00:55:34 +08:00
log . Warn ( "get compact time failed, skip to handle compaction" )
2023-11-30 17:06:31 +08:00
return err
2022-10-10 20:31:22 +08:00
}
2024-06-05 10:17:50 +08:00
plans := t . generatePlans ( group . segments , signal , ct )
2024-07-17 13:23:42 +08:00
currentID , _ , err := t . allocator . allocN ( int64 ( len ( plans ) * 2 ) )
2024-06-05 10:17:50 +08:00
if err != nil {
return err
}
2022-02-18 14:51:49 +08:00
for _ , plan := range plans {
2024-06-05 10:17:50 +08:00
totalRows := plan . A
segIDs := plan . B
2022-02-18 14:51:49 +08:00
if ! signal . isForce && t . compactionHandler . isFull ( ) {
2024-07-12 00:55:34 +08:00
log . Warn ( "compaction plan skipped due to handler full" , zap . Int64s ( "segmentIDs" , segIDs ) )
2022-02-18 14:51:49 +08:00
break
}
start := time . Now ( )
2024-06-05 10:17:50 +08:00
planID := currentID
currentID ++
2024-07-17 13:23:42 +08:00
targetSegmentID := currentID
currentID ++
2024-06-05 10:17:50 +08:00
pts , _ := tsoutil . ParseTS ( ct . startTime )
task := & datapb . CompactionTask {
PlanID : planID ,
TriggerID : signal . id ,
State : datapb . CompactionTaskState_pipelining ,
StartTime : pts . Unix ( ) ,
TimeoutInSeconds : Params . DataCoordCfg . CompactionTimeoutInSeconds . GetAsInt32 ( ) ,
Type : datapb . CompactionType_MixCompaction ,
CollectionTtl : ct . collectionTTL . Nanoseconds ( ) ,
2024-07-12 00:55:34 +08:00
CollectionID : group . collectionID ,
2024-06-05 10:17:50 +08:00
PartitionID : group . partitionID ,
Channel : group . channelName ,
InputSegments : segIDs ,
2024-07-17 13:23:42 +08:00
ResultSegments : [ ] int64 { targetSegmentID } , // pre-allocated target segment
2024-06-05 10:17:50 +08:00
TotalRows : totalRows ,
Schema : coll . Schema ,
}
err := t . compactionHandler . enqueueCompaction ( task )
if err != nil {
log . Warn ( "failed to execute compaction task" ,
2023-07-14 15:56:31 +08:00
zap . Int64s ( "segmentIDs" , segIDs ) ,
2022-11-15 11:13:07 +08:00
zap . Error ( err ) )
2022-02-18 14:51:49 +08:00
continue
}
2024-06-05 10:17:50 +08:00
2022-11-15 11:13:07 +08:00
log . Info ( "time cost of generating global compaction" ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "time cost" , time . Since ( start ) . Milliseconds ( ) ) ,
zap . Int64s ( "segmentIDs" , segIDs ) )
2022-02-18 14:51:49 +08:00
}
2021-12-29 10:06:47 +08:00
}
2023-11-30 17:06:31 +08:00
return nil
2021-11-05 22:25:00 +08:00
}
2022-02-18 14:51:49 +08:00
// handleSignal processes segment flush caused partition-chan level compaction signal
2021-11-05 22:25:00 +08:00
func ( t * compactionTrigger ) handleSignal ( signal * compactionSignal ) {
t . forceMu . Lock ( )
defer t . forceMu . Unlock ( )
// 1. check whether segment's binlogs should be compacted or not
if t . compactionHandler . isFull ( ) {
2023-11-30 17:06:31 +08:00
log . Warn ( "compaction plan skipped due to handler full" )
2021-11-05 22:25:00 +08:00
return
}
2024-03-05 13:39:01 +08:00
if ! t . isChannelCheckpointHealthy ( signal . channel ) {
log . Warn ( "compaction plan skipped due to channel checkpoint lag" , zap . String ( "channel" , signal . channel ) )
return
}
2023-03-03 14:13:49 +08:00
segment := t . meta . GetHealthySegment ( signal . segmentID )
2021-12-22 21:23:10 +08:00
if segment == nil {
log . Warn ( "segment in compaction signal not found in meta" , zap . Int64 ( "segmentID" , signal . segmentID ) )
return
}
2021-11-05 22:25:00 +08:00
channel := segment . GetInsertChannel ( )
partitionID := segment . GetPartitionID ( )
2023-05-10 17:45:20 +08:00
collectionID := segment . GetCollectionID ( )
2021-11-05 22:25:00 +08:00
segments := t . getCandidateSegments ( channel , partitionID )
2022-09-25 20:12:52 +08:00
2022-10-10 20:31:22 +08:00
if len ( segments ) == 0 {
2024-03-14 05:32:54 +08:00
log . Info ( "the number of candidate segments is 0, skip to handle compaction" )
2022-10-10 20:31:22 +08:00
return
}
2023-05-10 17:45:20 +08:00
coll , err := t . getCollection ( collectionID )
if err != nil {
log . Warn ( "get collection info failed, skip handling compaction" ,
zap . Int64 ( "collectionID" , collectionID ) ,
zap . Int64 ( "partitionID" , partitionID ) ,
zap . String ( "channel" , channel ) ,
zap . Error ( err ) ,
)
return
}
if ! signal . isForce && ! t . isCollectionAutoCompactionEnabled ( coll ) {
log . RatedInfo ( 20 , "collection auto compaction disabled" ,
zap . Int64 ( "collectionID" , collectionID ) ,
)
return
}
2024-06-05 10:17:50 +08:00
ts := tsoutil . ComposeTSByTime ( time . Now ( ) , 0 )
ct , err := getCompactTime ( ts , coll )
2022-10-10 20:31:22 +08:00
if err != nil {
log . Warn ( "get compact time failed, skip to handle compaction" , zap . Int64 ( "collectionID" , segment . GetCollectionID ( ) ) ,
zap . Int64 ( "partitionID" , partitionID ) , zap . String ( "channel" , channel ) )
return
}
2024-06-05 10:17:50 +08:00
plans := t . generatePlans ( segments , signal , ct )
2024-07-17 13:23:42 +08:00
currentID , _ , err := t . allocator . allocN ( int64 ( len ( plans ) * 2 ) )
2024-06-05 10:17:50 +08:00
if err != nil {
log . Warn ( "fail to allocate id" , zap . Error ( err ) )
return
}
2022-02-18 14:51:49 +08:00
for _ , plan := range plans {
if t . compactionHandler . isFull ( ) {
2024-06-05 10:17:50 +08:00
log . Warn ( "compaction plan skipped due to handler full" , zap . Int64 ( "collection" , signal . collectionID ) )
2022-02-18 14:51:49 +08:00
break
}
2024-06-05 10:17:50 +08:00
totalRows := plan . A
segmentIDS := plan . B
2022-02-18 14:51:49 +08:00
start := time . Now ( )
2024-06-05 10:17:50 +08:00
planID := currentID
currentID ++
2024-07-17 13:23:42 +08:00
targetSegmentID := currentID
currentID ++
2024-06-05 10:17:50 +08:00
pts , _ := tsoutil . ParseTS ( ct . startTime )
if err := t . compactionHandler . enqueueCompaction ( & datapb . CompactionTask {
PlanID : planID ,
TriggerID : signal . id ,
State : datapb . CompactionTaskState_pipelining ,
StartTime : pts . Unix ( ) ,
TimeoutInSeconds : Params . DataCoordCfg . CompactionTimeoutInSeconds . GetAsInt32 ( ) ,
Type : datapb . CompactionType_MixCompaction ,
CollectionTtl : ct . collectionTTL . Nanoseconds ( ) ,
CollectionID : collectionID ,
PartitionID : partitionID ,
Channel : channel ,
InputSegments : segmentIDS ,
2024-07-17 13:23:42 +08:00
ResultSegments : [ ] int64 { targetSegmentID } , // pre-allocated target segment
2024-06-05 10:17:50 +08:00
TotalRows : totalRows ,
Schema : coll . Schema ,
} ) ; err != nil {
log . Warn ( "failed to execute compaction task" ,
zap . Int64 ( "collection" , collectionID ) ,
zap . Int64 ( "planID" , planID ) ,
zap . Int64s ( "segmentIDs" , segmentIDS ) ,
zap . Error ( err ) )
2022-02-18 14:51:49 +08:00
continue
}
2022-11-15 11:13:07 +08:00
log . Info ( "time cost of generating compaction" ,
2024-06-05 10:17:50 +08:00
zap . Int64 ( "planID" , planID ) ,
2024-01-05 16:12:48 +08:00
zap . Int64 ( "time cost" , time . Since ( start ) . Milliseconds ( ) ) ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "collectionID" , signal . collectionID ) ,
2022-11-15 11:13:07 +08:00
zap . String ( "channel" , channel ) ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "partitionID" , partitionID ) ,
2024-06-05 10:17:50 +08:00
zap . Int64s ( "segmentIDs" , segmentIDS ) )
2021-11-05 22:25:00 +08:00
}
}
2024-06-05 10:17:50 +08:00
func ( t * compactionTrigger ) generatePlans ( segments [ ] * SegmentInfo , signal * compactionSignal , compactTime * compactTime ) [ ] * typeutil . Pair [ int64 , [ ] int64 ] {
2024-04-23 12:11:24 +08:00
if len ( segments ) == 0 {
log . Warn ( "the number of candidate segments is 0, skip to generate compaction plan" )
2024-06-05 10:17:50 +08:00
return [ ] * typeutil . Pair [ int64 , [ ] int64 ] { }
2024-04-23 12:11:24 +08:00
}
2022-02-18 14:51:49 +08:00
// find segments need internal compaction
2022-06-15 23:14:10 +08:00
// TODO add low priority candidates, for example if the segment is smaller than full 0.9 * max segment size but larger than small segment boundary, we only execute compaction when there are no compaction running actively
var prioritizedCandidates [ ] * SegmentInfo
var smallCandidates [ ] * SegmentInfo
2022-12-09 16:03:20 +08:00
var nonPlannedSegments [ ] * SegmentInfo
2022-06-15 23:14:10 +08:00
2024-04-23 12:11:24 +08:00
expectedSize := t . getExpectedSegmentSize ( segments [ 0 ] . CollectionID )
2022-06-15 23:14:10 +08:00
// TODO, currently we lack of the measurement of data distribution, there should be another compaction help on redistributing segment based on scalar/vector field distribution
2022-02-18 14:51:49 +08:00
for _ , segment := range segments {
segment := segment . ShadowClone ( )
2022-06-15 23:14:10 +08:00
// TODO should we trigger compaction periodically even if the segment has no obvious reason to be compacted?
2024-06-05 10:17:50 +08:00
if signal . isForce || t . ShouldDoSingleCompaction ( segment , compactTime ) {
2022-06-15 23:14:10 +08:00
prioritizedCandidates = append ( prioritizedCandidates , segment )
2024-04-23 12:11:24 +08:00
} else if t . isSmallSegment ( segment , expectedSize ) {
2022-06-15 23:14:10 +08:00
smallCandidates = append ( smallCandidates , segment )
2022-12-09 16:03:20 +08:00
} else {
nonPlannedSegments = append ( nonPlannedSegments , segment )
2022-02-18 14:51:49 +08:00
}
}
2023-09-04 17:41:48 +08:00
2024-04-23 12:11:24 +08:00
buckets := [ ] [ ] * SegmentInfo { }
2022-06-15 23:14:10 +08:00
// sort segment from large to small
sort . Slice ( prioritizedCandidates , func ( i , j int ) bool {
2024-04-23 12:11:24 +08:00
if prioritizedCandidates [ i ] . getSegmentSize ( ) != prioritizedCandidates [ j ] . getSegmentSize ( ) {
return prioritizedCandidates [ i ] . getSegmentSize ( ) > prioritizedCandidates [ j ] . getSegmentSize ( )
2022-06-15 23:14:10 +08:00
}
return prioritizedCandidates [ i ] . GetID ( ) < prioritizedCandidates [ j ] . GetID ( )
} )
2022-02-18 14:51:49 +08:00
2022-06-15 23:14:10 +08:00
sort . Slice ( smallCandidates , func ( i , j int ) bool {
2024-04-23 12:11:24 +08:00
if smallCandidates [ i ] . getSegmentSize ( ) != smallCandidates [ j ] . getSegmentSize ( ) {
return smallCandidates [ i ] . getSegmentSize ( ) > smallCandidates [ j ] . getSegmentSize ( )
2022-06-15 23:14:10 +08:00
}
return smallCandidates [ i ] . GetID ( ) < smallCandidates [ j ] . GetID ( )
} )
2022-12-09 16:03:20 +08:00
// Sort non-planned from small to large.
sort . Slice ( nonPlannedSegments , func ( i , j int ) bool {
2024-04-23 12:11:24 +08:00
if nonPlannedSegments [ i ] . getSegmentSize ( ) != nonPlannedSegments [ j ] . getSegmentSize ( ) {
return nonPlannedSegments [ i ] . getSegmentSize ( ) < nonPlannedSegments [ j ] . getSegmentSize ( )
2022-12-09 16:03:20 +08:00
}
return nonPlannedSegments [ i ] . GetID ( ) > nonPlannedSegments [ j ] . GetID ( )
} )
2022-06-15 23:14:10 +08:00
// greedy pick from large segment to small, the goal is to fill each segment to reach 512M
// we must ensure all prioritized candidates is in a plan
2023-09-21 09:45:27 +08:00
// TODO the compaction selection policy should consider if compaction workload is high
2022-06-15 23:14:10 +08:00
for len ( prioritizedCandidates ) > 0 {
2022-02-18 14:51:49 +08:00
var bucket [ ] * SegmentInfo
2022-06-15 23:14:10 +08:00
// pop out the first element
segment := prioritizedCandidates [ 0 ]
2022-02-18 14:51:49 +08:00
bucket = append ( bucket , segment )
2022-06-15 23:14:10 +08:00
prioritizedCandidates = prioritizedCandidates [ 1 : ]
// only do single file compaction if segment is already large enough
2024-04-23 12:11:24 +08:00
if segment . getSegmentSize ( ) < expectedSize {
2022-06-15 23:14:10 +08:00
var result [ ] * SegmentInfo
2024-04-23 12:11:24 +08:00
free := expectedSize - segment . getSegmentSize ( )
2022-12-07 18:01:19 +08:00
maxNum := Params . DataCoordCfg . MaxSegmentToMerge . GetAsInt ( ) - 1
2022-06-15 23:14:10 +08:00
prioritizedCandidates , result , free = greedySelect ( prioritizedCandidates , free , maxNum )
bucket = append ( bucket , result ... )
maxNum -= len ( result )
if maxNum > 0 {
smallCandidates , result , _ = greedySelect ( smallCandidates , free , maxNum )
bucket = append ( bucket , result ... )
}
}
// since this is priority compaction, we will execute even if there is only segment
2024-05-07 19:01:31 +08:00
log . Info ( "pick priority candidate for compaction" ,
zap . Int64 ( "prioritized segmentID" , segment . GetID ( ) ) ,
zap . Int64s ( "picked segmentIDs" , lo . Map ( bucket , func ( s * SegmentInfo , _ int ) int64 { return s . GetID ( ) } ) ) ,
zap . Int64 ( "target size" , lo . SumBy ( bucket , func ( s * SegmentInfo ) int64 { return s . getSegmentSize ( ) } ) ) ,
zap . Int64 ( "target count" , lo . SumBy ( bucket , func ( s * SegmentInfo ) int64 { return s . GetNumOfRows ( ) } ) ) ,
)
2024-04-23 12:11:24 +08:00
buckets = append ( buckets , bucket )
2022-02-18 14:51:49 +08:00
}
2022-12-09 16:03:20 +08:00
var remainingSmallSegs [ ] * SegmentInfo
2022-06-15 23:14:10 +08:00
// check if there are small candidates left can be merged into large segments
for len ( smallCandidates ) > 0 {
var bucket [ ] * SegmentInfo
// pop out the first element
segment := smallCandidates [ 0 ]
bucket = append ( bucket , segment )
smallCandidates = smallCandidates [ 1 : ]
var result [ ] * SegmentInfo
2024-04-23 12:11:24 +08:00
free := expectedSize - segment . getSegmentSize ( )
2022-06-15 23:14:10 +08:00
// for small segment merge, we pick one largest segment and merge as much as small segment together with it
// Why reverse? try to merge as many segments as expected.
// for instance, if a 255M and 255M is the largest small candidates, they will never be merged because of the MinSegmentToMerge limit.
2022-12-07 18:01:19 +08:00
smallCandidates , result , _ = reverseGreedySelect ( smallCandidates , free , Params . DataCoordCfg . MaxSegmentToMerge . GetAsInt ( ) - 1 )
2022-06-15 23:14:10 +08:00
bucket = append ( bucket , result ... )
2022-02-18 14:51:49 +08:00
2024-05-07 19:01:31 +08:00
// only merge if candidate number is large than MinSegmentToMerge or if target size is large enough
targetSize := lo . SumBy ( bucket , func ( s * SegmentInfo ) int64 { return s . getSegmentSize ( ) } )
2022-12-09 16:03:20 +08:00
if len ( bucket ) >= Params . DataCoordCfg . MinSegmentToMerge . GetAsInt ( ) ||
2024-04-23 12:11:24 +08:00
len ( bucket ) > 1 && t . isCompactableSegment ( targetSize , expectedSize ) {
buckets = append ( buckets , bucket )
2022-12-09 16:03:20 +08:00
} else {
remainingSmallSegs = append ( remainingSmallSegs , bucket ... )
}
}
2024-04-23 12:11:24 +08:00
2024-05-07 19:01:31 +08:00
remainingSmallSegs = t . squeezeSmallSegmentsToBuckets ( remainingSmallSegs , buckets , expectedSize )
2022-12-09 16:03:20 +08:00
// If there are still remaining small segments, try adding them to non-planned segments.
for _ , npSeg := range nonPlannedSegments {
bucket := [ ] * SegmentInfo { npSeg }
2024-04-23 12:11:24 +08:00
targetSize := npSeg . getSegmentSize ( )
2022-12-09 16:03:20 +08:00
for i := len ( remainingSmallSegs ) - 1 ; i >= 0 ; i -- {
// Note: could also simply use MaxRowNum as limit.
2024-04-23 12:11:24 +08:00
if targetSize + remainingSmallSegs [ i ] . getSegmentSize ( ) <=
int64 ( Params . DataCoordCfg . SegmentExpansionRate . GetAsFloat ( ) * float64 ( expectedSize ) ) {
2022-12-09 16:03:20 +08:00
bucket = append ( bucket , remainingSmallSegs [ i ] )
2024-04-23 12:11:24 +08:00
targetSize += remainingSmallSegs [ i ] . getSegmentSize ( )
2022-12-09 16:03:20 +08:00
remainingSmallSegs = append ( remainingSmallSegs [ : i ] , remainingSmallSegs [ i + 1 : ] ... )
}
}
if len ( bucket ) > 1 {
2024-04-23 12:11:24 +08:00
buckets = append ( buckets , bucket )
2022-06-15 23:14:10 +08:00
}
2022-02-18 14:51:49 +08:00
}
2024-04-23 12:11:24 +08:00
2024-06-05 10:17:50 +08:00
tasks := make ( [ ] * typeutil . Pair [ int64 , [ ] int64 ] , len ( buckets ) )
2024-04-23 12:11:24 +08:00
for i , b := range buckets {
2024-06-05 10:17:50 +08:00
segmentIDs := make ( [ ] int64 , 0 )
var totalRows int64
for _ , s := range b {
totalRows += s . GetNumOfRows ( )
segmentIDs = append ( segmentIDs , s . GetID ( ) )
2022-02-18 14:51:49 +08:00
}
2024-06-05 10:17:50 +08:00
pair := typeutil . NewPair ( totalRows , segmentIDs )
tasks [ i ] = & pair
2022-02-18 14:51:49 +08:00
}
2024-06-05 10:17:50 +08:00
log . Info ( "generatePlans" , zap . Int64 ( "collectionID" , signal . collectionID ) , zap . Int ( "plan_num" , len ( tasks ) ) )
return tasks
2022-02-18 14:51:49 +08:00
}
2022-06-15 23:14:10 +08:00
func greedySelect ( candidates [ ] * SegmentInfo , free int64 , maxSegment int ) ( [ ] * SegmentInfo , [ ] * SegmentInfo , int64 ) {
var result [ ] * SegmentInfo
for i := 0 ; i < len ( candidates ) ; {
candidate := candidates [ i ]
2024-04-23 12:11:24 +08:00
if len ( result ) < maxSegment && candidate . getSegmentSize ( ) < free {
2022-06-15 23:14:10 +08:00
result = append ( result , candidate )
2024-04-23 12:11:24 +08:00
free -= candidate . getSegmentSize ( )
2022-06-15 23:14:10 +08:00
candidates = append ( candidates [ : i ] , candidates [ i + 1 : ] ... )
} else {
i ++
}
}
return candidates , result , free
}
func reverseGreedySelect ( candidates [ ] * SegmentInfo , free int64 , maxSegment int ) ( [ ] * SegmentInfo , [ ] * SegmentInfo , int64 ) {
2022-02-18 14:51:49 +08:00
var result [ ] * SegmentInfo
2022-06-15 23:14:10 +08:00
for i := len ( candidates ) - 1 ; i >= 0 ; i -- {
candidate := candidates [ i ]
2024-04-23 12:11:24 +08:00
if ( len ( result ) < maxSegment ) && ( candidate . getSegmentSize ( ) < free ) {
2022-06-15 23:14:10 +08:00
result = append ( result , candidate )
2024-04-23 12:11:24 +08:00
free -= candidate . getSegmentSize ( )
2022-06-15 23:14:10 +08:00
candidates = append ( candidates [ : i ] , candidates [ i + 1 : ] ... )
}
2022-02-18 14:51:49 +08:00
}
2022-06-15 23:14:10 +08:00
return candidates , result , free
2021-11-05 22:25:00 +08:00
}
func ( t * compactionTrigger ) getCandidateSegments ( channel string , partitionID UniqueID ) [ ] * SegmentInfo {
segments := t . meta . GetSegmentsByChannel ( channel )
2023-07-23 21:31:00 +08:00
if Params . DataCoordCfg . IndexBasedCompaction . GetAsBool ( ) {
segments = FilterInIndexedSegments ( t . handler , t . meta , segments ... )
}
2022-02-18 14:51:49 +08:00
var res [ ] * SegmentInfo
2021-11-05 22:25:00 +08:00
for _ , s := range segments {
2022-09-22 18:34:50 +08:00
if ! isSegmentHealthy ( s ) ||
! isFlush ( s ) ||
s . GetInsertChannel ( ) != channel ||
s . GetPartitionID ( ) != partitionID ||
2022-09-26 18:06:54 +08:00
s . isCompacting ||
2023-12-05 18:44:37 +08:00
s . GetIsImporting ( ) ||
2024-06-10 21:34:08 +08:00
s . GetLevel ( ) == datapb . SegmentLevel_L0 ||
s . GetLevel ( ) == datapb . SegmentLevel_L2 {
2021-11-05 22:25:00 +08:00
continue
}
res = append ( res , s )
}
2022-09-16 11:32:48 +08:00
2021-11-05 22:25:00 +08:00
return res
}
2024-04-23 12:11:24 +08:00
func ( t * compactionTrigger ) isSmallSegment ( segment * SegmentInfo , expectedSize int64 ) bool {
return segment . getSegmentSize ( ) < int64 ( float64 ( expectedSize ) * Params . DataCoordCfg . SegmentSmallProportion . GetAsFloat ( ) )
2021-11-05 22:25:00 +08:00
}
2024-04-23 12:11:24 +08:00
func ( t * compactionTrigger ) isCompactableSegment ( targetSize , expectedSize int64 ) bool {
2023-07-26 14:49:01 +08:00
smallProportion := Params . DataCoordCfg . SegmentSmallProportion . GetAsFloat ( )
compactableProportion := Params . DataCoordCfg . SegmentCompactableProportion . GetAsFloat ( )
// avoid invalid single segment compaction
if compactableProportion < smallProportion {
compactableProportion = smallProportion
}
2024-04-23 12:11:24 +08:00
return targetSize > int64 ( float64 ( expectedSize ) * compactableProportion )
2022-12-09 16:03:20 +08:00
}
2024-04-23 12:11:24 +08:00
func isExpandableSmallSegment ( segment * SegmentInfo , expectedSize int64 ) bool {
return segment . getSegmentSize ( ) < int64 ( float64 ( expectedSize ) * ( Params . DataCoordCfg . SegmentExpansionRate . GetAsFloat ( ) - 1 ) )
2022-10-31 19:13:34 +08:00
}
2024-04-23 12:11:24 +08:00
func ( t * compactionTrigger ) ShouldDoSingleCompaction ( segment * SegmentInfo , compactTime * compactTime ) bool {
2022-12-12 10:33:26 +08:00
// no longer restricted binlog numbers because this is now related to field numbers
2023-11-23 17:30:25 +08:00
binlogCount := GetBinlogCount ( segment . GetBinlogs ( ) )
deltaLogCount := GetBinlogCount ( segment . GetDeltalogs ( ) )
if deltaLogCount > Params . DataCoordCfg . SingleCompactionDeltalogMaxNum . GetAsInt ( ) {
log . Info ( "total delta number is too much, trigger compaction" , zap . Int64 ( "segmentID" , segment . ID ) , zap . Int ( "Bin logs" , binlogCount ) , zap . Int ( "Delta logs" , deltaLogCount ) )
2022-06-15 23:14:10 +08:00
return true
}
// if expire time is enabled, put segment into compaction candidate
totalExpiredSize := int64 ( 0 )
2022-06-22 19:00:14 +08:00
totalExpiredRows := 0
2022-06-15 23:14:10 +08:00
for _ , binlogs := range segment . GetBinlogs ( ) {
for _ , l := range binlogs . GetBinlogs ( ) {
// TODO, we should probably estimate expired log entries by total rows in binlog and the ralationship of timeTo, timeFrom and expire time
if l . TimestampTo < compactTime . expireTime {
2023-08-08 21:17:21 +08:00
log . RatedDebug ( 10 , "mark binlog as expired" ,
zap . Int64 ( "segmentID" , segment . ID ) ,
zap . Int64 ( "binlogID" , l . GetLogID ( ) ) ,
zap . Uint64 ( "binlogTimestampTo" , l . TimestampTo ) ,
zap . Uint64 ( "compactExpireTime" , compactTime . expireTime ) )
2022-06-22 19:00:14 +08:00
totalExpiredRows += int ( l . GetEntriesNum ( ) )
2024-05-15 12:59:34 +08:00
totalExpiredSize += l . GetMemorySize ( )
2022-06-15 23:14:10 +08:00
}
}
}
2023-08-08 21:17:21 +08:00
if float64 ( totalExpiredRows ) / float64 ( segment . GetNumOfRows ( ) ) >= Params . DataCoordCfg . SingleCompactionRatioThreshold . GetAsFloat ( ) ||
totalExpiredSize > Params . DataCoordCfg . SingleCompactionExpiredLogMaxSize . GetAsInt64 ( ) {
2023-07-14 15:56:31 +08:00
log . Info ( "total expired entities is too much, trigger compaction" , zap . Int64 ( "segmentID" , segment . ID ) ,
2023-08-08 21:17:21 +08:00
zap . Int ( "expiredRows" , totalExpiredRows ) , zap . Int64 ( "expiredLogSize" , totalExpiredSize ) ,
zap . Bool ( "createdByCompaction" , segment . CreatedByCompaction ) , zap . Int64s ( "compactionFrom" , segment . CompactionFrom ) )
2022-06-15 23:14:10 +08:00
return true
}
2021-11-05 22:25:00 +08:00
totalDeletedRows := 0
totalDeleteLogSize := int64 ( 0 )
2022-06-15 23:14:10 +08:00
for _ , deltaLogs := range segment . GetDeltalogs ( ) {
for _ , l := range deltaLogs . GetBinlogs ( ) {
2023-09-04 17:41:48 +08:00
totalDeletedRows += int ( l . GetEntriesNum ( ) )
2024-05-15 12:59:34 +08:00
totalDeleteLogSize += l . GetMemorySize ( )
2021-11-05 22:25:00 +08:00
}
}
// currently delta log size and delete ratio policy is applied
2022-12-07 18:01:19 +08:00
if float64 ( totalDeletedRows ) / float64 ( segment . GetNumOfRows ( ) ) >= Params . DataCoordCfg . SingleCompactionRatioThreshold . GetAsFloat ( ) || totalDeleteLogSize > Params . DataCoordCfg . SingleCompactionDeltaLogMaxSize . GetAsInt64 ( ) {
2023-09-04 17:41:48 +08:00
log . Info ( "total delete entities is too much, trigger compaction" ,
zap . Int64 ( "segmentID" , segment . ID ) ,
zap . Int64 ( "numRows" , segment . GetNumOfRows ( ) ) ,
zap . Int ( "deleted rows" , totalDeletedRows ) ,
zap . Int64 ( "delete log size" , totalDeleteLogSize ) )
2022-06-15 23:14:10 +08:00
return true
2022-02-18 14:51:49 +08:00
}
2023-12-12 11:56:37 +08:00
if Params . DataCoordCfg . AutoUpgradeSegmentIndex . GetAsBool ( ) {
// index version of segment lower than current version and IndexFileKeys should have value, trigger compaction
2024-03-04 16:56:59 +08:00
indexIDToSegIdxes := t . meta . indexMeta . GetSegmentIndexes ( segment . CollectionID , segment . ID )
for _ , index := range indexIDToSegIdxes {
2023-12-12 11:56:37 +08:00
if index . CurrentIndexVersion < t . indexEngineVersionManager . GetCurrentIndexEngineVersion ( ) &&
len ( index . IndexFileKeys ) > 0 {
log . Info ( "index version is too old, trigger compaction" ,
zap . Int64 ( "segmentID" , segment . ID ) ,
zap . Int64 ( "indexID" , index . IndexID ) ,
zap . Strings ( "indexFileKeys" , index . IndexFileKeys ) ,
zap . Int32 ( "currentIndexVersion" , index . CurrentIndexVersion ) ,
zap . Int32 ( "currentEngineVersion" , t . indexEngineVersionManager . GetCurrentIndexEngineVersion ( ) ) )
return true
}
2023-11-21 09:26:22 +08:00
}
}
2022-02-18 14:51:49 +08:00
return false
}
2021-12-14 13:55:07 +08:00
func isFlush ( segment * SegmentInfo ) bool {
return segment . GetState ( ) == commonpb . SegmentState_Flushed || segment . GetState ( ) == commonpb . SegmentState_Flushing
}
2022-11-15 11:13:07 +08:00
2024-07-01 22:42:08 +08:00
func needSync ( segment * SegmentInfo ) bool {
return segment . GetState ( ) == commonpb . SegmentState_Flushed || segment . GetState ( ) == commonpb . SegmentState_Flushing || segment . GetState ( ) == commonpb . SegmentState_Sealed
}
2024-05-07 19:01:31 +08:00
// buckets will be updated inplace
func ( t * compactionTrigger ) squeezeSmallSegmentsToBuckets ( small [ ] * SegmentInfo , buckets [ ] [ ] * SegmentInfo , expectedSize int64 ) ( remaining [ ] * SegmentInfo ) {
for i := len ( small ) - 1 ; i >= 0 ; i -- {
s := small [ i ]
if ! isExpandableSmallSegment ( s , expectedSize ) {
continue
}
// Try squeeze this segment into existing plans. This could cause segment size to exceed maxSize.
for bidx , b := range buckets {
totalSize := lo . SumBy ( b , func ( s * SegmentInfo ) int64 { return s . getSegmentSize ( ) } )
if totalSize + s . getSegmentSize ( ) > int64 ( Params . DataCoordCfg . SegmentExpansionRate . GetAsFloat ( ) * float64 ( expectedSize ) ) {
continue
}
buckets [ bidx ] = append ( buckets [ bidx ] , s )
small = append ( small [ : i ] , small [ i + 1 : ] ... )
break
}
}
return small
}