2021-11-10 19:03:38 +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.
package datacoord
import (
2022-09-29 16:18:56 +08:00
"context"
2023-09-22 18:47:25 +08:00
"fmt"
2021-11-24 09:55:15 +08:00
"path"
2023-07-23 21:31:00 +08:00
"sort"
2022-10-15 15:13:24 +08:00
"strings"
2021-11-10 19:03:38 +08:00
"sync"
"time"
2023-01-04 19:37:36 +08:00
"github.com/minio/minio-go/v7"
"github.com/samber/lo"
2023-12-14 19:26:39 +08:00
"go.uber.org/atomic"
2023-01-04 19:37:36 +08:00
"go.uber.org/zap"
2023-06-09 01:28:37 +08:00
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
2024-01-18 22:06:31 +08:00
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
2021-12-19 20:00:42 +08:00
"github.com/milvus-io/milvus/internal/proto/datapb"
2022-07-15 13:54:26 +08:00
"github.com/milvus-io/milvus/internal/storage"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
2023-09-22 18:47:25 +08:00
"github.com/milvus-io/milvus/pkg/metrics"
2024-01-03 13:16:57 +08:00
"github.com/milvus-io/milvus/pkg/util/conc"
2023-12-14 19:26:39 +08:00
"github.com/milvus-io/milvus/pkg/util/merr"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/pkg/util/metautil"
2023-09-22 18:47:25 +08:00
"github.com/milvus-io/milvus/pkg/util/paramtable"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/pkg/util/typeutil"
2021-11-10 19:03:38 +08:00
)
// GcOption garbage collection options
type GcOption struct {
2022-07-22 22:10:28 +08:00
cli storage . ChunkManager // client
enabled bool // enable switch
checkInterval time . Duration // each interval
missingTolerance time . Duration // key missing in meta tolerance time
dropTolerance time . Duration // dropped segment related key tolerance time
2024-01-03 13:16:57 +08:00
removeLogPool * conc . Pool [ struct { } ]
2021-11-10 19:03:38 +08:00
}
// garbageCollector handles garbage files in object storage
2021-12-15 18:25:43 +08:00
// which could be dropped collection remanent or data node failure traces
2021-11-10 19:03:38 +08:00
type garbageCollector struct {
2023-01-04 19:37:36 +08:00
option GcOption
meta * meta
handler Handler
2021-11-10 19:03:38 +08:00
2023-12-14 19:26:39 +08:00
startOnce sync . Once
stopOnce sync . Once
wg sync . WaitGroup
closeCh chan struct { }
cmdCh chan gcCmd
pauseUntil atomic . Time
}
type gcCmd struct {
cmdType datapb . GcCommand
duration time . Duration
done chan struct { }
2021-11-10 19:03:38 +08:00
}
// newGarbageCollector create garbage collector with meta and option
2023-01-04 19:37:36 +08:00
func newGarbageCollector ( meta * meta , handler Handler , opt GcOption ) * garbageCollector {
2021-11-18 22:29:40 +08:00
log . Info ( "GC with option" , zap . Bool ( "enabled" , opt . enabled ) , zap . Duration ( "interval" , opt . checkInterval ) ,
zap . Duration ( "missingTolerance" , opt . missingTolerance ) , zap . Duration ( "dropTolerance" , opt . dropTolerance ) )
2024-01-03 13:16:57 +08:00
opt . removeLogPool = conc . NewPool [ struct { } ] ( Params . DataCoordCfg . GCRemoveConcurrent . GetAsInt ( ) , conc . WithExpiryDuration ( time . Minute ) )
2021-11-10 19:03:38 +08:00
return & garbageCollector {
2023-01-04 19:37:36 +08:00
meta : meta ,
handler : handler ,
option : opt ,
closeCh : make ( chan struct { } ) ,
2023-12-14 19:26:39 +08:00
cmdCh : make ( chan gcCmd ) ,
2021-11-10 19:03:38 +08:00
}
}
// start a goroutine and perform gc check every `checkInterval`
func ( gc * garbageCollector ) start ( ) {
if gc . option . enabled {
if gc . option . cli == nil {
2021-12-20 17:42:07 +08:00
log . Warn ( "DataCoord gc enabled, but SSO client is not provided" )
2021-11-10 19:03:38 +08:00
return
}
gc . startOnce . Do ( func ( ) {
gc . wg . Add ( 1 )
go gc . work ( )
} )
}
}
2023-12-14 19:26:39 +08:00
func ( gc * garbageCollector ) Pause ( ctx context . Context , pauseDuration time . Duration ) error {
if ! gc . option . enabled {
log . Info ( "garbage collection not enabled" )
return nil
}
done := make ( chan struct { } )
select {
case gc . cmdCh <- gcCmd {
cmdType : datapb . GcCommand_Pause ,
duration : pauseDuration ,
done : done ,
} :
<- done
return nil
case <- ctx . Done ( ) :
return ctx . Err ( )
}
}
func ( gc * garbageCollector ) Resume ( ctx context . Context ) error {
if ! gc . option . enabled {
log . Warn ( "garbage collection not enabled, cannot resume" )
return merr . WrapErrServiceUnavailable ( "garbage collection not enabled" )
}
done := make ( chan struct { } )
select {
case gc . cmdCh <- gcCmd {
cmdType : datapb . GcCommand_Resume ,
done : done ,
} :
<- done
return nil
case <- ctx . Done ( ) :
return ctx . Err ( )
}
}
2021-11-10 19:03:38 +08:00
// work contains actual looping check logic
func ( gc * garbageCollector ) work ( ) {
defer gc . wg . Done ( )
2023-02-23 18:59:45 +08:00
ticker := time . NewTicker ( gc . option . checkInterval )
defer ticker . Stop ( )
2021-11-10 19:03:38 +08:00
for {
select {
2023-02-23 18:59:45 +08:00
case <- ticker . C :
2023-12-14 19:26:39 +08:00
if time . Now ( ) . Before ( gc . pauseUntil . Load ( ) ) {
log . Info ( "garbage collector paused" , zap . Time ( "until" , gc . pauseUntil . Load ( ) ) )
continue
}
2021-11-23 11:23:15 +08:00
gc . clearEtcd ( )
2023-01-04 19:37:36 +08:00
gc . recycleUnusedIndexes ( )
gc . recycleUnusedSegIndexes ( )
2021-11-10 19:03:38 +08:00
gc . scan ( )
2023-01-04 19:37:36 +08:00
gc . recycleUnusedIndexFiles ( )
2023-12-14 19:26:39 +08:00
case cmd := <- gc . cmdCh :
switch cmd . cmdType {
case datapb . GcCommand_Pause :
pauseUntil := time . Now ( ) . Add ( cmd . duration )
if pauseUntil . After ( gc . pauseUntil . Load ( ) ) {
log . Info ( "garbage collection paused" , zap . Duration ( "duration" , cmd . duration ) , zap . Time ( "pauseUntil" , pauseUntil ) )
gc . pauseUntil . Store ( pauseUntil )
} else {
log . Info ( "new pause until before current value" , zap . Duration ( "duration" , cmd . duration ) , zap . Time ( "pauseUntil" , pauseUntil ) , zap . Time ( "oldPauseUntil" , gc . pauseUntil . Load ( ) ) )
}
case datapb . GcCommand_Resume :
// reset to zero value
gc . pauseUntil . Store ( time . Time { } )
2024-02-05 17:09:15 +08:00
log . Info ( "garbage collection resumed" )
2023-12-14 19:26:39 +08:00
}
close ( cmd . done )
2021-11-10 19:03:38 +08:00
case <- gc . closeCh :
log . Warn ( "garbage collector quit" )
return
}
}
}
func ( gc * garbageCollector ) close ( ) {
gc . stopOnce . Do ( func ( ) {
close ( gc . closeCh )
gc . wg . Wait ( )
} )
}
// scan load meta file info and compares OSS keys
2021-11-23 11:23:15 +08:00
// if missing found, performs gc cleanup
2021-11-10 19:03:38 +08:00
func ( gc * garbageCollector ) scan ( ) {
2022-09-29 16:18:56 +08:00
ctx , cancel := context . WithCancel ( context . Background ( ) )
defer cancel ( )
2022-10-15 15:13:24 +08:00
var (
total = 0
valid = 0
missing = 0
)
2023-09-19 17:05:34 +08:00
getMetaMap := func ( ) ( typeutil . UniqueSet , typeutil . Set [ string ] ) {
segmentMap := typeutil . NewUniqueSet ( )
filesMap := typeutil . NewSet [ string ] ( )
segments := gc . meta . GetAllSegmentsUnsafe ( )
for _ , segment := range segments {
2024-01-18 22:06:31 +08:00
cloned := segment . Clone ( )
binlog . DecompressBinLogs ( cloned . SegmentInfo )
2023-09-19 17:05:34 +08:00
segmentMap . Insert ( segment . GetID ( ) )
2024-01-18 22:06:31 +08:00
for _ , log := range getLogs ( cloned ) {
2023-09-19 17:05:34 +08:00
filesMap . Insert ( log . GetLogPath ( ) )
}
2022-10-15 15:13:24 +08:00
}
2023-09-19 17:05:34 +08:00
return segmentMap , filesMap
2021-11-10 19:03:38 +08:00
}
2021-11-24 09:55:15 +08:00
// walk only data cluster related prefixes
prefixes := make ( [ ] string , 0 , 3 )
2023-09-19 10:01:23 +08:00
prefixes = append ( prefixes , path . Join ( gc . option . cli . RootPath ( ) , common . SegmentInsertLogPath ) )
prefixes = append ( prefixes , path . Join ( gc . option . cli . RootPath ( ) , common . SegmentStatslogPath ) )
prefixes = append ( prefixes , path . Join ( gc . option . cli . RootPath ( ) , common . SegmentDeltaLogPath ) )
2023-09-22 18:47:25 +08:00
labels := [ ] string { metrics . InsertFileLabel , metrics . StatFileLabel , metrics . DeleteFileLabel }
2022-02-18 11:59:49 +08:00
var removedKeys [ ] string
2021-11-24 09:55:15 +08:00
2023-09-22 18:47:25 +08:00
for idx , prefix := range prefixes {
2023-07-12 10:14:29 +08:00
startTs := time . Now ( )
2023-04-12 19:38:28 +08:00
infoKeys , modTimes , err := gc . option . cli . ListWithPrefix ( ctx , prefix , true )
2022-07-22 22:10:28 +08:00
if err != nil {
2023-04-12 19:38:28 +08:00
log . Error ( "failed to list files with prefix" ,
2022-10-15 15:13:24 +08:00
zap . String ( "prefix" , prefix ) ,
2023-09-15 10:09:21 +08:00
zap . Error ( err ) ,
2022-10-15 15:13:24 +08:00
)
2022-07-22 22:10:28 +08:00
}
2023-09-22 18:47:25 +08:00
cost := time . Since ( startTs )
2023-09-19 17:05:34 +08:00
segmentMap , filesMap := getMetaMap ( )
2023-09-22 18:47:25 +08:00
metrics . GarbageCollectorListLatency .
WithLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) , labels [ idx ] ) .
Observe ( float64 ( cost . Milliseconds ( ) ) )
log . Info ( "gc scan finish list object" , zap . String ( "prefix" , prefix ) , zap . Duration ( "time spent" , cost ) , zap . Int ( "keys" , len ( infoKeys ) ) )
2023-04-12 19:38:28 +08:00
for i , infoKey := range infoKeys {
total ++
_ , has := filesMap [ infoKey ]
if has {
valid ++
2021-11-24 09:55:15 +08:00
continue
}
2023-04-12 19:38:28 +08:00
segmentID , err := storage . ParseSegmentIDByBinlog ( gc . option . cli . RootPath ( ) , infoKey )
2022-09-30 14:18:55 +08:00
if err != nil {
2023-04-12 19:38:28 +08:00
missing ++
log . Warn ( "parse segment id error" ,
zap . String ( "infoKey" , infoKey ) ,
zap . Error ( err ) )
continue
}
2023-09-19 10:01:23 +08:00
if strings . Contains ( prefix , common . SegmentInsertLogPath ) &&
2023-04-12 19:38:28 +08:00
segmentMap . Contain ( segmentID ) {
valid ++
2022-10-15 15:13:24 +08:00
continue
}
2023-04-12 19:38:28 +08:00
// not found in meta, check last modified time exceeds tolerance duration
if time . Since ( modTimes [ i ] ) > gc . option . missingTolerance {
// ignore error since it could be cleaned up next time
removedKeys = append ( removedKeys , infoKey )
err = gc . option . cli . Remove ( ctx , infoKey )
2022-07-15 13:54:26 +08:00
if err != nil {
2022-10-13 14:43:23 +08:00
missing ++
2023-04-12 19:38:28 +08:00
log . Error ( "failed to remove object" ,
2022-10-15 15:13:24 +08:00
zap . String ( "infoKey" , infoKey ) ,
zap . Error ( err ) )
2022-07-15 13:54:26 +08:00
}
2021-11-24 09:55:15 +08:00
}
2021-11-10 19:03:38 +08:00
}
}
2023-09-22 18:47:25 +08:00
metrics . GarbageCollectorRunCount . WithLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) ) . Add ( 1 )
2022-10-15 15:13:24 +08:00
log . Info ( "scan file to do garbage collection" ,
zap . Int ( "total" , total ) ,
zap . Int ( "valid" , valid ) ,
zap . Int ( "missing" , missing ) ,
zap . Strings ( "removedKeys" , removedKeys ) )
2021-11-23 11:23:15 +08:00
}
2023-08-09 19:39:15 +08:00
func ( gc * garbageCollector ) checkDroppedSegmentGC ( segment * SegmentInfo ,
childSegment * SegmentInfo ,
indexSet typeutil . UniqueSet ,
2023-09-21 09:45:27 +08:00
cpTimestamp Timestamp ,
) bool {
2023-08-09 19:39:15 +08:00
log := log . With ( zap . Int64 ( "segmentID" , segment . ID ) )
isCompacted := childSegment != nil || segment . GetCompacted ( )
if isCompacted {
// For compact A, B -> C, don't GC A or B if C is not indexed,
// guarantee replacing A, B with C won't downgrade performance
// If the child is GC'ed first, then childSegment will be nil.
if childSegment != nil && ! indexSet . Contain ( childSegment . GetID ( ) ) {
log . WithRateGroup ( "GC_FAIL_COMPACT_TO_NOT_INDEXED" , 1 , 60 ) .
2023-08-15 17:49:33 +08:00
RatedInfo ( 60 , "skipping GC when compact target segment is not indexed" ,
2023-08-09 19:39:15 +08:00
zap . Int64 ( "child segment ID" , childSegment . GetID ( ) ) )
return false
}
} else {
if ! gc . isExpire ( segment . GetDroppedAt ( ) ) {
return false
}
}
segInsertChannel := segment . GetInsertChannel ( )
// Ignore segments from potentially dropped collection. Check if collection is to be dropped by checking if channel is dropped.
// We do this because collection meta drop relies on all segment being GCed.
if gc . meta . catalog . ChannelExists ( context . Background ( ) , segInsertChannel ) &&
segment . GetDmlPosition ( ) . GetTimestamp ( ) > cpTimestamp {
// segment gc shall only happen when channel cp is after segment dml cp.
log . WithRateGroup ( "GC_FAIL_CP_BEFORE" , 1 , 60 ) .
RatedInfo ( 60 , "dropped segment dml position after channel cp, skip meta gc" ,
zap . Uint64 ( "dmlPosTs" , segment . GetDmlPosition ( ) . GetTimestamp ( ) ) ,
zap . Uint64 ( "channelCpTs" , cpTimestamp ) ,
)
return false
}
return true
}
2021-11-23 11:23:15 +08:00
func ( gc * garbageCollector ) clearEtcd ( ) {
2022-09-16 11:32:48 +08:00
all := gc . meta . SelectSegments ( func ( si * SegmentInfo ) bool { return true } )
drops := make ( map [ int64 ] * SegmentInfo , 0 )
compactTo := make ( map [ int64 ] * SegmentInfo )
2023-01-06 21:33:36 +08:00
channels := typeutil . NewSet [ string ] ( )
2022-09-16 11:32:48 +08:00
for _ , segment := range all {
2024-01-18 22:06:31 +08:00
cloned := segment . Clone ( )
binlog . DecompressBinLogs ( cloned . SegmentInfo )
if cloned . GetState ( ) == commonpb . SegmentState_Dropped {
drops [ cloned . GetID ( ) ] = cloned
channels . Insert ( cloned . GetInsertChannel ( ) )
2023-09-21 09:45:27 +08:00
// continue
2023-01-04 19:37:36 +08:00
// A(indexed), B(indexed) -> C(no indexed), D(no indexed) -> E(no indexed), A, B can not be GC
2022-09-16 11:32:48 +08:00
}
2024-01-18 22:06:31 +08:00
for _ , from := range cloned . GetCompactionFrom ( ) {
compactTo [ from ] = cloned
2022-09-16 11:32:48 +08:00
}
}
droppedCompactTo := make ( map [ * SegmentInfo ] struct { } )
for id := range drops {
if to , ok := compactTo [ id ] ; ok {
droppedCompactTo [ to ] = struct { } { }
}
}
2023-01-04 19:37:36 +08:00
indexedSegments := FilterInIndexedSegments ( gc . handler , gc . meta , lo . Keys ( droppedCompactTo ) ... )
2022-09-16 11:32:48 +08:00
indexedSet := make ( typeutil . UniqueSet )
for _ , segment := range indexedSegments {
indexedSet . Insert ( segment . GetID ( ) )
}
2021-11-23 11:23:15 +08:00
2023-01-06 21:33:36 +08:00
channelCPs := make ( map [ string ] uint64 )
for channel := range channels {
pos := gc . meta . GetChannelCheckpoint ( channel )
channelCPs [ channel ] = pos . GetTimestamp ( )
}
2023-07-23 21:31:00 +08:00
dropIDs := lo . Keys ( drops )
sort . Slice ( dropIDs , func ( i , j int ) bool {
return dropIDs [ i ] < dropIDs [ j ]
} )
2024-01-03 13:16:57 +08:00
log . Info ( "start to GC segments" , zap . Int ( "drop_num" , len ( dropIDs ) ) )
2023-07-23 21:31:00 +08:00
for _ , segmentID := range dropIDs {
segment , ok := drops [ segmentID ]
if ! ok {
log . Warn ( "segmentID is not in drops" , zap . Int64 ( "segmentID" , segmentID ) )
2021-11-23 11:23:15 +08:00
continue
}
2023-08-09 19:39:15 +08:00
2023-03-09 14:13:52 +08:00
segInsertChannel := segment . GetInsertChannel ( )
2023-08-09 19:39:15 +08:00
if ! gc . checkDroppedSegmentGC ( segment , compactTo [ segment . GetID ( ) ] , indexedSet , channelCPs [ segInsertChannel ] ) {
2022-09-16 11:32:48 +08:00
continue
}
2023-08-09 19:39:15 +08:00
2022-10-15 15:13:24 +08:00
logs := getLogs ( segment )
2024-01-03 13:16:57 +08:00
log . Info ( "GC segment" , zap . Int64 ( "segmentID" , segment . GetID ( ) ) ,
zap . Int ( "insert_logs" , len ( segment . GetBinlogs ( ) ) ) ,
zap . Int ( "delta_logs" , len ( segment . GetDeltalogs ( ) ) ) ,
zap . Int ( "stats_logs" , len ( segment . GetStatslogs ( ) ) ) )
2021-11-23 11:23:15 +08:00
if gc . removeLogs ( logs ) {
2023-08-09 19:39:15 +08:00
err := gc . meta . DropSegment ( segment . GetID ( ) )
if err != nil {
log . Info ( "GC segment meta failed to drop segment" , zap . Int64 ( "segment id" , segment . GetID ( ) ) , zap . Error ( err ) )
} else {
log . Info ( "GC segment meta drop semgent" , zap . Int64 ( "segment id" , segment . GetID ( ) ) )
2023-07-25 10:39:01 +08:00
}
2021-11-23 11:23:15 +08:00
}
2023-03-09 14:13:52 +08:00
if segList := gc . meta . GetSegmentsByChannel ( segInsertChannel ) ; len ( segList ) == 0 &&
! gc . meta . catalog . ChannelExists ( context . Background ( ) , segInsertChannel ) {
2023-07-12 10:14:29 +08:00
log . Info ( "empty channel found during gc, manually cleanup channel checkpoints" , zap . String ( "vChannel" , segInsertChannel ) )
2023-03-09 14:13:52 +08:00
if err := gc . meta . DropChannelCheckpoint ( segInsertChannel ) ; err != nil {
2023-08-09 19:39:15 +08:00
log . Info ( "failed to drop channel check point during segment garbage collection" , zap . String ( "vchannel" , segInsertChannel ) , zap . Error ( err ) )
2023-03-09 14:13:52 +08:00
}
}
2021-11-23 11:23:15 +08:00
}
}
func ( gc * garbageCollector ) isExpire ( dropts Timestamp ) bool {
droptime := time . Unix ( 0 , int64 ( dropts ) )
return time . Since ( droptime ) > gc . option . dropTolerance
}
2021-12-19 20:00:42 +08:00
func getLogs ( sinfo * SegmentInfo ) [ ] * datapb . Binlog {
var logs [ ] * datapb . Binlog
2021-11-23 11:23:15 +08:00
for _ , flog := range sinfo . GetBinlogs ( ) {
logs = append ( logs , flog . GetBinlogs ( ) ... )
}
for _ , flog := range sinfo . GetStatslogs ( ) {
logs = append ( logs , flog . GetBinlogs ( ) ... )
}
2021-12-19 20:00:42 +08:00
for _ , flog := range sinfo . GetDeltalogs ( ) {
logs = append ( logs , flog . GetBinlogs ( ) ... )
2021-11-23 11:23:15 +08:00
}
return logs
}
2021-12-19 20:00:42 +08:00
func ( gc * garbageCollector ) removeLogs ( logs [ ] * datapb . Binlog ) bool {
2022-09-29 16:18:56 +08:00
ctx , cancel := context . WithCancel ( context . Background ( ) )
defer cancel ( )
2024-01-03 13:16:57 +08:00
var w sync . WaitGroup
w . Add ( len ( logs ) )
2021-11-23 11:23:15 +08:00
for _ , l := range logs {
2024-01-03 13:16:57 +08:00
tmpLog := l
gc . option . removeLogPool . Submit ( func ( ) ( struct { } , error ) {
defer w . Done ( )
select {
case <- ctx . Done ( ) :
return struct { } { } , nil
2022-07-22 22:10:28 +08:00
default :
2024-01-03 13:16:57 +08:00
err := gc . option . cli . Remove ( ctx , tmpLog . GetLogPath ( ) )
if err != nil {
switch err . ( type ) {
case minio . ErrorResponse :
errResp := minio . ToErrorResponse ( err )
if errResp . Code != "" && errResp . Code != "NoSuchKey" {
cancel ( )
}
default :
cancel ( )
}
}
return struct { } { } , nil
2022-07-22 22:10:28 +08:00
}
2024-01-03 13:16:57 +08:00
} )
}
w . Wait ( )
select {
case <- ctx . Done ( ) :
return false
default :
return true
2021-11-23 11:23:15 +08:00
}
2021-11-10 19:03:38 +08:00
}
2023-01-04 19:37:36 +08:00
func ( gc * garbageCollector ) recycleUnusedIndexes ( ) {
log . Info ( "start recycleUnusedIndexes" )
deletedIndexes := gc . meta . GetDeletedIndexes ( )
for _ , index := range deletedIndexes {
if err := gc . meta . RemoveIndex ( index . CollectionID , index . IndexID ) ; err != nil {
2023-07-14 15:56:31 +08:00
log . Warn ( "remove index on collection fail" , zap . Int64 ( "collectionID" , index . CollectionID ) ,
2023-01-04 19:37:36 +08:00
zap . Int64 ( "indexID" , index . IndexID ) , zap . Error ( err ) )
continue
}
}
}
func ( gc * garbageCollector ) recycleUnusedSegIndexes ( ) {
segIndexes := gc . meta . GetAllSegIndexes ( )
for _ , segIdx := range segIndexes {
2023-03-03 14:13:49 +08:00
if gc . meta . GetSegment ( segIdx . SegmentID ) == nil || ! gc . meta . IsIndexExist ( segIdx . CollectionID , segIdx . IndexID ) {
2023-01-04 19:37:36 +08:00
if err := gc . meta . RemoveSegmentIndex ( segIdx . CollectionID , segIdx . PartitionID , segIdx . SegmentID , segIdx . IndexID , segIdx . BuildID ) ; err != nil {
log . Warn ( "delete index meta from etcd failed, wait to retry" , zap . Int64 ( "buildID" , segIdx . BuildID ) ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "segmentID" , segIdx . SegmentID ) , zap . Int64 ( "nodeID" , segIdx . NodeID ) , zap . Error ( err ) )
2023-01-04 19:37:36 +08:00
continue
}
log . Info ( "index meta recycle success" , zap . Int64 ( "buildID" , segIdx . BuildID ) ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "segmentID" , segIdx . SegmentID ) )
2023-01-04 19:37:36 +08:00
}
}
}
// recycleUnusedIndexFiles is used to delete those index files that no longer exist in the meta.
func ( gc * garbageCollector ) recycleUnusedIndexFiles ( ) {
log . Info ( "start recycleUnusedIndexFiles" )
ctx , cancel := context . WithCancel ( context . Background ( ) )
defer cancel ( )
2023-07-12 10:14:29 +08:00
startTs := time . Now ( )
2023-01-04 19:37:36 +08:00
prefix := path . Join ( gc . option . cli . RootPath ( ) , common . SegmentIndexPath ) + "/"
// list dir first
keys , _ , err := gc . option . cli . ListWithPrefix ( ctx , prefix , false )
if err != nil {
2023-07-12 10:14:29 +08:00
log . Warn ( "garbageCollector recycleUnusedIndexFiles list keys from chunk manager failed" , zap . Error ( err ) )
2023-01-04 19:37:36 +08:00
return
}
2023-07-12 10:14:29 +08:00
log . Info ( "recycleUnusedIndexFiles, finish list object" , zap . Duration ( "time spent" , time . Since ( startTs ) ) , zap . Int ( "build ids" , len ( keys ) ) )
2023-01-04 19:37:36 +08:00
for _ , key := range keys {
log . Debug ( "indexFiles keys" , zap . String ( "key" , key ) )
buildID , err := parseBuildIDFromFilePath ( key )
if err != nil {
2023-07-12 10:14:29 +08:00
log . Warn ( "garbageCollector recycleUnusedIndexFiles parseIndexFileKey" , zap . String ( "key" , key ) , zap . Error ( err ) )
2023-01-04 19:37:36 +08:00
continue
}
log . Info ( "garbageCollector will recycle index files" , zap . Int64 ( "buildID" , buildID ) )
canRecycle , segIdx := gc . meta . CleanSegmentIndex ( buildID )
if ! canRecycle {
// Even if the index is marked as deleted, the index file will not be recycled, wait for the next gc,
// and delete all index files about the buildID at one time.
2023-08-15 17:49:33 +08:00
log . Info ( "garbageCollector can not recycle index files" , zap . Int64 ( "buildID" , buildID ) )
2023-01-04 19:37:36 +08:00
continue
}
if segIdx == nil {
// buildID no longer exists in meta, remove all index files
log . Info ( "garbageCollector recycleUnusedIndexFiles find meta has not exist, remove index files" ,
zap . Int64 ( "buildID" , buildID ) )
err = gc . option . cli . RemoveWithPrefix ( ctx , key )
if err != nil {
log . Warn ( "garbageCollector recycleUnusedIndexFiles remove index files failed" ,
zap . Int64 ( "buildID" , buildID ) , zap . String ( "prefix" , key ) , zap . Error ( err ) )
continue
}
2023-04-09 16:22:30 +08:00
log . Info ( "garbageCollector recycleUnusedIndexFiles remove index files success" ,
zap . Int64 ( "buildID" , buildID ) , zap . String ( "prefix" , key ) )
2023-01-04 19:37:36 +08:00
continue
}
filesMap := make ( map [ string ] struct { } )
for _ , fileID := range segIdx . IndexFileKeys {
filepath := metautil . BuildSegmentIndexFilePath ( gc . option . cli . RootPath ( ) , segIdx . BuildID , segIdx . IndexVersion ,
segIdx . PartitionID , segIdx . SegmentID , fileID )
filesMap [ filepath ] = struct { } { }
}
files , _ , err := gc . option . cli . ListWithPrefix ( ctx , key , true )
if err != nil {
log . Warn ( "garbageCollector recycleUnusedIndexFiles list files failed" ,
zap . Int64 ( "buildID" , buildID ) , zap . String ( "prefix" , key ) , zap . Error ( err ) )
continue
}
log . Info ( "recycle index files" , zap . Int64 ( "buildID" , buildID ) , zap . Int ( "meta files num" , len ( filesMap ) ) ,
zap . Int ( "chunkManager files num" , len ( files ) ) )
deletedFilesNum := 0
for _ , file := range files {
if _ , ok := filesMap [ file ] ; ! ok {
if err = gc . option . cli . Remove ( ctx , file ) ; err != nil {
log . Warn ( "garbageCollector recycleUnusedIndexFiles remove file failed" ,
zap . Int64 ( "buildID" , buildID ) , zap . String ( "file" , file ) , zap . Error ( err ) )
continue
}
deletedFilesNum ++
}
}
log . Info ( "index files recycle success" , zap . Int64 ( "buildID" , buildID ) ,
zap . Int ( "delete index files num" , deletedFilesNum ) )
}
}