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"
2021-11-10 19:03:38 +08:00
"sync"
"time"
2024-04-25 20:41:27 +08:00
"github.com/cockroachdb/errors"
2023-01-04 19:37:36 +08:00
"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"
2024-04-25 20:41:27 +08:00
"github.com/milvus-io/milvus/internal/metastore/model"
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"
2024-07-11 23:01:35 +08:00
"github.com/milvus-io/milvus/pkg/util/funcutil"
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-03-19 11:27:06 +08:00
scanInterval time . Duration // interval for scan residue for interupted log wrttien
2024-01-03 13:16:57 +08:00
2024-04-25 20:41:27 +08:00
removeObjectPool * 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 {
2024-04-25 20:41:27 +08:00
ctx context . Context
cancel context . CancelFunc
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
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 {
2024-03-19 11:27:06 +08:00
log . Info ( "GC with option" ,
zap . Bool ( "enabled" , opt . enabled ) ,
zap . Duration ( "interval" , opt . checkInterval ) ,
zap . Duration ( "scanInterval" , opt . scanInterval ) ,
zap . Duration ( "missingTolerance" , opt . missingTolerance ) ,
zap . Duration ( "dropTolerance" , opt . dropTolerance ) )
2024-04-25 20:41:27 +08:00
opt . removeObjectPool = conc . NewPool [ struct { } ] ( Params . DataCoordCfg . GCRemoveConcurrent . GetAsInt ( ) , conc . WithExpiryDuration ( time . Minute ) )
ctx , cancel := context . WithCancel ( context . Background ( ) )
2021-11-10 19:03:38 +08:00
return & garbageCollector {
2024-04-25 20:41:27 +08:00
ctx : ctx ,
cancel : cancel ,
2023-01-04 19:37:36 +08:00
meta : meta ,
handler : handler ,
option : opt ,
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 ( ) {
2024-04-25 20:41:27 +08:00
gc . work ( gc . ctx )
2021-11-10 19:03:38 +08:00
} )
}
}
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
2024-04-25 20:41:27 +08:00
func ( gc * garbageCollector ) work ( ctx context . Context ) {
// TODO: fast cancel for gc when closing.
// Run gc tasks in parallel.
gc . wg . Add ( 3 )
go func ( ) {
defer gc . wg . Done ( )
gc . runRecycleTaskWithPauser ( ctx , "meta" , gc . option . checkInterval , func ( ctx context . Context ) {
gc . recycleDroppedSegments ( ctx )
2024-07-11 23:01:35 +08:00
gc . recycleChannelCPMeta ( ctx )
2024-04-25 20:41:27 +08:00
gc . recycleUnusedIndexes ( ctx )
gc . recycleUnusedSegIndexes ( ctx )
2024-07-29 17:03:48 +08:00
gc . recycleUnusedAnalyzeFiles ( ctx )
2024-04-25 20:41:27 +08:00
} )
} ( )
go func ( ) {
defer gc . wg . Done ( )
gc . runRecycleTaskWithPauser ( ctx , "orphan" , gc . option . scanInterval , func ( ctx context . Context ) {
gc . recycleUnusedBinlogFiles ( ctx )
gc . recycleUnusedIndexFiles ( ctx )
} )
} ( )
go func ( ) {
defer gc . wg . Done ( )
gc . startControlLoop ( ctx )
} ( )
}
// startControlLoop start a control loop for garbageCollector.
2024-04-30 14:19:27 +08:00
func ( gc * garbageCollector ) startControlLoop ( _ context . Context ) {
2021-11-10 19:03:38 +08:00
for {
select {
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 )
2024-04-25 20:41:27 +08:00
case <- gc . ctx . Done ( ) :
log . Warn ( "garbage collector control loop quit" )
return
}
}
}
// runRecycleTaskWithPauser is a helper function to create a task with pauser
func ( gc * garbageCollector ) runRecycleTaskWithPauser ( ctx context . Context , name string , interval time . Duration , task func ( ctx context . Context ) ) {
logger := log . With ( zap . String ( "gcType" , name ) ) . With ( zap . Duration ( "interval" , interval ) )
2024-04-30 14:19:27 +08:00
timer := time . NewTicker ( interval )
2024-04-25 20:41:27 +08:00
defer timer . Stop ( )
for {
select {
case <- ctx . Done ( ) :
2021-11-10 19:03:38 +08:00
return
2024-04-25 20:41:27 +08:00
case <- timer . C :
if time . Now ( ) . Before ( gc . pauseUntil . Load ( ) ) {
logger . Info ( "garbage collector paused" , zap . Time ( "until" , gc . pauseUntil . Load ( ) ) )
continue
}
logger . Info ( "garbage collector recycle task start..." )
start := time . Now ( )
task ( ctx )
logger . Info ( "garbage collector recycle task done" , zap . Duration ( "timeCost" , time . Since ( start ) ) )
2021-11-10 19:03:38 +08:00
}
}
}
2024-04-25 20:41:27 +08:00
// close stop the garbage collector.
2021-11-10 19:03:38 +08:00
func ( gc * garbageCollector ) close ( ) {
gc . stopOnce . Do ( func ( ) {
2024-04-25 20:41:27 +08:00
gc . cancel ( )
2021-11-10 19:03:38 +08:00
gc . wg . Wait ( )
} )
}
2024-04-25 20:41:27 +08:00
// recycleUnusedBinlogFiles load meta file info and compares OSS keys
2021-11-23 11:23:15 +08:00
// if missing found, performs gc cleanup
2024-04-25 20:41:27 +08:00
func ( gc * garbageCollector ) recycleUnusedBinlogFiles ( ctx context . Context ) {
start := time . Now ( )
log := log . With ( zap . String ( "gcName" , "recycleUnusedBinlogFiles" ) , zap . Time ( "startAt" , start ) )
log . Info ( "start recycleUnusedBinlogFiles..." )
defer func ( ) { log . Info ( "recycleUnusedBinlogFiles done" , zap . Duration ( "timeCost" , time . Since ( start ) ) ) } ( )
type scanTask struct {
prefix string
checker func ( objectInfo * storage . ChunkObjectInfo , segment * SegmentInfo ) bool
label string
}
scanTasks := [ ] scanTask {
{
prefix : path . Join ( gc . option . cli . RootPath ( ) , common . SegmentInsertLogPath ) ,
checker : func ( objectInfo * storage . ChunkObjectInfo , segment * SegmentInfo ) bool {
return segment != nil
} ,
label : metrics . InsertFileLabel ,
} ,
{
prefix : path . Join ( gc . option . cli . RootPath ( ) , common . SegmentStatslogPath ) ,
checker : func ( objectInfo * storage . ChunkObjectInfo , segment * SegmentInfo ) bool {
logID , err := binlog . GetLogIDFromBingLogPath ( objectInfo . FilePath )
if err != nil {
log . Warn ( "garbageCollector find dirty stats log" , zap . String ( "filePath" , objectInfo . FilePath ) , zap . Error ( err ) )
return false
}
return segment != nil && segment . IsStatsLogExists ( logID )
} ,
label : metrics . StatFileLabel ,
} ,
{
prefix : path . Join ( gc . option . cli . RootPath ( ) , common . SegmentDeltaLogPath ) ,
checker : func ( objectInfo * storage . ChunkObjectInfo , segment * SegmentInfo ) bool {
logID , err := binlog . GetLogIDFromBingLogPath ( objectInfo . FilePath )
if err != nil {
log . Warn ( "garbageCollector find dirty dleta log" , zap . String ( "filePath" , objectInfo . FilePath ) , zap . Error ( err ) )
return false
}
return segment != nil && segment . IsDeltaLogExists ( logID )
} ,
label : metrics . DeleteFileLabel ,
} ,
}
for _ , task := range scanTasks {
gc . recycleUnusedBinLogWithChecker ( ctx , task . prefix , task . label , task . checker )
2021-11-10 19:03:38 +08:00
}
2024-04-25 20:41:27 +08:00
metrics . GarbageCollectorRunCount . WithLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) ) . Add ( 1 )
}
// recycleUnusedBinLogWithChecker scans the prefix and checks the path with checker.
// GC the file if checker returns false.
func ( gc * garbageCollector ) recycleUnusedBinLogWithChecker ( ctx context . Context , prefix string , label string , checker func ( objectInfo * storage . ChunkObjectInfo , segment * SegmentInfo ) bool ) {
logger := log . With ( zap . String ( "prefix" , prefix ) )
logger . Info ( "garbageCollector recycleUnusedBinlogFiles start" , zap . String ( "prefix" , prefix ) )
lastFilePath := ""
total := 0
valid := 0
unexpectedFailure := atomic . NewInt32 ( 0 )
removed := atomic . NewInt32 ( 0 )
start := time . Now ( )
futures := make ( [ ] * conc . Future [ struct { } ] , 0 )
err := gc . option . cli . WalkWithPrefix ( ctx , prefix , true , func ( chunkInfo * storage . ChunkObjectInfo ) bool {
total ++
lastFilePath = chunkInfo . FilePath
// Check file tolerance first to avoid unnecessary operation.
if time . Since ( chunkInfo . ModifyTime ) <= gc . option . missingTolerance {
logger . Info ( "garbageCollector recycleUnusedBinlogFiles skip file since it is not expired" , zap . String ( "filePath" , chunkInfo . FilePath ) , zap . Time ( "modifyTime" , chunkInfo . ModifyTime ) )
return true
}
2021-11-10 19:03:38 +08:00
2024-04-25 20:41:27 +08:00
// Parse segmentID from file path.
// TODO: Does all files in the same segment have the same segmentID?
segmentID , err := storage . ParseSegmentIDByBinlog ( gc . option . cli . RootPath ( ) , chunkInfo . FilePath )
2022-07-22 22:10:28 +08:00
if err != nil {
2024-04-25 20:41:27 +08:00
unexpectedFailure . Inc ( )
logger . Warn ( "garbageCollector recycleUnusedBinlogFiles parse segment id error" ,
zap . String ( "filePath" , chunkInfo . FilePath ) ,
zap . Error ( err ) )
return true
2022-07-22 22:10:28 +08:00
}
2023-04-12 19:38:28 +08:00
2024-04-25 20:41:27 +08:00
segment := gc . meta . GetSegment ( segmentID )
if checker ( chunkInfo , segment ) {
valid ++
logger . Info ( "garbageCollector recycleUnusedBinlogFiles skip file since it is valid" , zap . String ( "filePath" , chunkInfo . FilePath ) , zap . Int64 ( "segmentID" , segmentID ) )
return true
}
2023-04-12 19:38:28 +08:00
2024-04-25 20:41:27 +08:00
// ignore error since it could be cleaned up next time
file := chunkInfo . FilePath
future := gc . option . removeObjectPool . Submit ( func ( ) ( struct { } , error ) {
logger := logger . With ( zap . String ( "file" , file ) )
logger . Info ( "garbageCollector recycleUnusedBinlogFiles remove file..." )
2022-10-15 15:13:24 +08:00
2024-04-25 20:41:27 +08:00
if err = gc . option . cli . Remove ( ctx , file ) ; err != nil {
log . Warn ( "garbageCollector recycleUnusedBinlogFiles remove file failed" , zap . Error ( err ) )
unexpectedFailure . Inc ( )
return struct { } { } , err
2021-11-24 09:55:15 +08:00
}
2024-04-25 20:41:27 +08:00
log . Info ( "garbageCollector recycleUnusedBinlogFiles remove file success" )
removed . Inc ( )
return struct { } { } , nil
} )
futures = append ( futures , future )
return true
} )
// Wait for all remove tasks done.
if err := conc . BlockOnAll ( futures ... ) ; err != nil {
// error is logged, and can be ignored here.
logger . Warn ( "some task failure in remove object pool" , zap . Error ( err ) )
2021-11-10 19:03:38 +08:00
}
2024-04-25 20:41:27 +08:00
cost := time . Since ( start )
logger . Info ( "garbageCollector recycleUnusedBinlogFiles done" ,
2022-10-15 15:13:24 +08:00
zap . Int ( "total" , total ) ,
zap . Int ( "valid" , valid ) ,
2024-04-25 20:41:27 +08:00
zap . Int ( "unexpectedFailure" , int ( unexpectedFailure . Load ( ) ) ) ,
zap . Int ( "removed" , int ( removed . Load ( ) ) ) ,
zap . String ( "lastFilePath" , lastFilePath ) ,
zap . Duration ( "cost" , cost ) ,
zap . Error ( err ) )
metrics . GarbageCollectorFileScanDuration .
WithLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) , label ) .
Observe ( float64 ( cost . Milliseconds ( ) ) )
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 ) )
2024-05-06 20:29:30 +08:00
if ! gc . isExpire ( segment . GetDroppedAt ( ) ) {
return false
}
2023-08-09 19:39:15 +08:00
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
}
}
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
}
2024-04-25 20:41:27 +08:00
// recycleDroppedSegments scans all segments and remove those dropped segments from meta and oss.
func ( gc * garbageCollector ) recycleDroppedSegments ( ctx context . Context ) {
start := time . Now ( )
log := log . With ( zap . String ( "gcName" , "recycleDroppedSegments" ) , zap . Time ( "startAt" , start ) )
log . Info ( "start clear dropped segments..." )
defer func ( ) { log . Info ( "clear dropped segments done" , zap . Duration ( "timeCost" , time . Since ( start ) ) ) } ( )
2024-05-08 21:37:29 +08:00
all := gc . meta . SelectSegments ( )
2022-09-16 11:32:48 +08:00
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 ( )
}
2024-04-25 20:41:27 +08:00
log . Info ( "start to GC segments" , zap . Int ( "drop_num" , len ( drops ) ) )
for segmentID , segment := range drops {
if ctx . Err ( ) != nil {
// process canceled, stop.
return
2021-11-23 11:23:15 +08:00
}
2023-08-09 19:39:15 +08:00
2024-04-25 20:41:27 +08:00
log := log . With ( zap . Int64 ( "segmentID" , segmentID ) )
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-04-25 20:41:27 +08:00
log . Info ( "GC segment start..." , zap . Int ( "insert_logs" , len ( segment . GetBinlogs ( ) ) ) ,
2024-01-03 13:16:57 +08:00
zap . Int ( "delta_logs" , len ( segment . GetDeltalogs ( ) ) ) ,
zap . Int ( "stats_logs" , len ( segment . GetStatslogs ( ) ) ) )
2024-04-25 20:41:27 +08:00
if err := gc . removeObjectFiles ( ctx , logs ) ; err != nil {
log . Warn ( "GC segment remove logs failed" , zap . Error ( err ) )
continue
2021-11-23 11:23:15 +08:00
}
2024-04-25 20:41:27 +08:00
if err := gc . meta . DropSegment ( segment . GetID ( ) ) ; err != nil {
log . Warn ( "GC segment meta failed to drop segment" , zap . Error ( err ) )
continue
}
log . Info ( "GC segment meta drop segment done" )
2024-07-11 23:01:35 +08:00
}
}
2024-04-25 20:41:27 +08:00
2024-07-11 23:01:35 +08:00
func ( gc * garbageCollector ) recycleChannelCPMeta ( ctx context . Context ) {
channelCPs , err := gc . meta . catalog . ListChannelCheckpoint ( ctx )
if err != nil {
log . Warn ( "list channel cp fail during GC" , zap . Error ( err ) )
return
}
collectionID2GcStatus := make ( map [ int64 ] bool )
skippedCnt := 0
log . Info ( "start to GC channel cp" , zap . Int ( "vchannelCnt" , len ( channelCPs ) ) )
for vChannel := range channelCPs {
collectionID := funcutil . GetCollectionIDFromVChannel ( vChannel )
// !!! Skip to GC if vChannel format is illegal, it will lead meta leak in this case
if collectionID == - 1 {
skippedCnt ++
log . Warn ( "parse collection id fail, skip to gc channel cp" , zap . String ( "vchannel" , vChannel ) )
continue
}
if _ , ok := collectionID2GcStatus [ collectionID ] ; ! ok {
collectionID2GcStatus [ collectionID ] = gc . meta . catalog . GcConfirm ( ctx , collectionID , - 1 )
}
// Skip to GC if all segments meta of the corresponding collection are not removed
if gcConfirmed , _ := collectionID2GcStatus [ collectionID ] ; ! gcConfirmed {
skippedCnt ++
continue
}
if err := gc . meta . DropChannelCheckpoint ( vChannel ) ; err != nil {
// Try to GC in the next gc cycle if drop channel cp meta fail.
log . Warn ( "failed to drop channel check point during gc" , zap . String ( "vchannel" , vChannel ) , zap . Error ( err ) )
2023-03-09 14:13:52 +08:00
}
2021-11-23 11:23:15 +08:00
}
2024-07-11 23:01:35 +08:00
log . Info ( "GC channel cp done" , zap . Int ( "skippedChannelCP" , skippedCnt ) )
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
}
2024-04-25 20:41:27 +08:00
func getLogs ( sinfo * SegmentInfo ) map [ string ] struct { } {
logs := make ( map [ string ] struct { } )
2021-11-23 11:23:15 +08:00
for _ , flog := range sinfo . GetBinlogs ( ) {
2024-04-25 20:41:27 +08:00
for _ , l := range flog . GetBinlogs ( ) {
logs [ l . GetLogPath ( ) ] = struct { } { }
}
2021-11-23 11:23:15 +08:00
}
for _ , flog := range sinfo . GetStatslogs ( ) {
2024-04-25 20:41:27 +08:00
for _ , l := range flog . GetBinlogs ( ) {
logs [ l . GetLogPath ( ) ] = struct { } { }
}
2021-11-23 11:23:15 +08:00
}
2021-12-19 20:00:42 +08:00
for _ , flog := range sinfo . GetDeltalogs ( ) {
2024-04-25 20:41:27 +08:00
for _ , l := range flog . GetBinlogs ( ) {
logs [ l . GetLogPath ( ) ] = struct { } { }
}
2021-11-23 11:23:15 +08:00
}
return logs
}
2024-04-25 20:41:27 +08:00
// removeObjectFiles remove file from oss storage, return error if any log failed to remove.
func ( gc * garbageCollector ) removeObjectFiles ( ctx context . Context , filePaths map [ string ] struct { } ) error {
futures := make ( [ ] * conc . Future [ struct { } ] , 0 )
for filePath := range filePaths {
filePath := filePath
future := gc . option . removeObjectPool . Submit ( func ( ) ( struct { } , error ) {
err := gc . option . cli . Remove ( ctx , filePath )
// ignore the error Key Not Found
if err != nil {
if ! errors . Is ( err , merr . ErrIoKeyNotFound ) {
return struct { } { } , err
2024-01-03 13:16:57 +08:00
}
2024-04-25 20:41:27 +08:00
log . Info ( "remove log failed, key not found, may be removed at previous GC, ignore the error" ,
zap . String ( "path" , filePath ) ,
zap . Error ( err ) )
2022-07-22 22:10:28 +08:00
}
2024-04-25 20:41:27 +08:00
return struct { } { } , nil
2024-01-03 13:16:57 +08:00
} )
2024-04-25 20:41:27 +08:00
futures = append ( futures , future )
2024-01-03 13:16:57 +08:00
}
2024-04-25 20:41:27 +08:00
return conc . BlockOnAll ( futures ... )
2021-11-10 19:03:38 +08:00
}
2023-01-04 19:37:36 +08:00
2024-04-25 20:41:27 +08:00
// recycleUnusedIndexes is used to delete those indexes that is deleted by collection.
func ( gc * garbageCollector ) recycleUnusedIndexes ( ctx context . Context ) {
start := time . Now ( )
log := log . With ( zap . String ( "gcName" , "recycleUnusedIndexes" ) , zap . Time ( "startAt" , start ) )
log . Info ( "start recycleUnusedIndexes..." )
defer func ( ) { log . Info ( "recycleUnusedIndexes done" , zap . Duration ( "timeCost" , time . Since ( start ) ) ) } ( )
2024-03-04 16:56:59 +08:00
deletedIndexes := gc . meta . indexMeta . GetDeletedIndexes ( )
2023-01-04 19:37:36 +08:00
for _ , index := range deletedIndexes {
2024-04-25 20:41:27 +08:00
if ctx . Err ( ) != nil {
// process canceled.
return
}
log := log . With ( zap . Int64 ( "collectionID" , index . CollectionID ) , zap . Int64 ( "fieldID" , index . FieldID ) , zap . Int64 ( "indexID" , index . IndexID ) )
2024-03-04 16:56:59 +08:00
if err := gc . meta . indexMeta . RemoveIndex ( index . CollectionID , index . IndexID ) ; err != nil {
2024-04-25 20:41:27 +08:00
log . Warn ( "remove index on collection fail" , zap . Error ( err ) )
2023-01-04 19:37:36 +08:00
continue
}
2024-04-25 20:41:27 +08:00
log . Info ( "remove index on collection done" )
2023-01-04 19:37:36 +08:00
}
}
2024-04-25 20:41:27 +08:00
// recycleUnusedSegIndexes remove the index of segment if index is deleted or segment itself is deleted.
func ( gc * garbageCollector ) recycleUnusedSegIndexes ( ctx context . Context ) {
start := time . Now ( )
log := log . With ( zap . String ( "gcName" , "recycleUnusedSegIndexes" ) , zap . Time ( "startAt" , start ) )
log . Info ( "start recycleUnusedSegIndexes..." )
defer func ( ) { log . Info ( "recycleUnusedSegIndexes done" , zap . Duration ( "timeCost" , time . Since ( start ) ) ) } ( )
2024-03-04 16:56:59 +08:00
segIndexes := gc . meta . indexMeta . GetAllSegIndexes ( )
2023-01-04 19:37:36 +08:00
for _ , segIdx := range segIndexes {
2024-04-25 20:41:27 +08:00
if ctx . Err ( ) != nil {
// process canceled.
return
}
// 1. segment belongs to is deleted.
// 2. index is deleted.
2024-03-04 16:56:59 +08:00
if gc . meta . GetSegment ( segIdx . SegmentID ) == nil || ! gc . meta . indexMeta . IsIndexExist ( segIdx . CollectionID , segIdx . IndexID ) {
2024-04-25 20:41:27 +08:00
indexFiles := gc . getAllIndexFilesOfIndex ( segIdx )
log := log . With ( zap . Int64 ( "collectionID" , segIdx . CollectionID ) ,
zap . Int64 ( "partitionID" , segIdx . PartitionID ) ,
zap . Int64 ( "segmentID" , segIdx . SegmentID ) ,
zap . Int64 ( "indexID" , segIdx . IndexID ) ,
zap . Int64 ( "buildID" , segIdx . BuildID ) ,
zap . Int64 ( "nodeID" , segIdx . NodeID ) ,
zap . Int ( "indexFiles" , len ( indexFiles ) ) )
log . Info ( "GC Segment Index file start..." )
// Remove index files first.
if err := gc . removeObjectFiles ( ctx , indexFiles ) ; err != nil {
log . Warn ( "fail to remove index files for index" , zap . Error ( err ) )
continue
}
// Remove meta from index meta.
2024-03-04 16:56:59 +08:00
if err := gc . meta . indexMeta . RemoveSegmentIndex ( segIdx . CollectionID , segIdx . PartitionID , segIdx . SegmentID , segIdx . IndexID , segIdx . BuildID ) ; err != nil {
2024-04-25 20:41:27 +08:00
log . Warn ( "delete index meta from etcd failed, wait to retry" , zap . Error ( err ) )
2023-01-04 19:37:36 +08:00
continue
}
2024-04-25 20:41:27 +08:00
log . Info ( "index meta recycle success" )
2023-01-04 19:37:36 +08:00
}
}
}
// recycleUnusedIndexFiles is used to delete those index files that no longer exist in the meta.
2024-04-25 20:41:27 +08:00
func ( gc * garbageCollector ) recycleUnusedIndexFiles ( ctx context . Context ) {
start := time . Now ( )
log := log . With ( zap . String ( "gcName" , "recycleUnusedIndexFiles" ) , zap . Time ( "startAt" , start ) )
log . Info ( "start recycleUnusedIndexFiles..." )
2023-01-04 19:37:36 +08:00
prefix := path . Join ( gc . option . cli . RootPath ( ) , common . SegmentIndexPath ) + "/"
// list dir first
2024-04-25 20:41:27 +08:00
keyCount := 0
err := gc . option . cli . WalkWithPrefix ( ctx , prefix , false , func ( indexPathInfo * storage . ChunkObjectInfo ) bool {
key := indexPathInfo . FilePath
keyCount ++
logger := log . With ( zap . String ( "prefix" , prefix ) , zap . String ( "key" , key ) )
2023-01-04 19:37:36 +08:00
buildID , err := parseBuildIDFromFilePath ( key )
if err != nil {
2024-04-25 20:41:27 +08:00
logger . Warn ( "garbageCollector recycleUnusedIndexFiles parseIndexFileKey" , zap . Error ( err ) )
return true
2023-01-04 19:37:36 +08:00
}
2024-04-25 20:41:27 +08:00
logger = logger . With ( zap . Int64 ( "buildID" , buildID ) )
logger . Info ( "garbageCollector will recycle index files" )
2024-06-06 17:37:51 +08:00
canRecycle , segIdx := gc . meta . indexMeta . CheckCleanSegmentIndex ( buildID )
2023-01-04 19:37:36 +08:00
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.
2024-04-25 20:41:27 +08:00
logger . Info ( "garbageCollector can not recycle index files" )
return true
2023-01-04 19:37:36 +08:00
}
if segIdx == nil {
// buildID no longer exists in meta, remove all index files
2024-04-25 20:41:27 +08:00
logger . Info ( "garbageCollector recycleUnusedIndexFiles find meta has not exist, remove index files" )
2023-01-04 19:37:36 +08:00
err = gc . option . cli . RemoveWithPrefix ( ctx , key )
if err != nil {
2024-04-25 20:41:27 +08:00
logger . Warn ( "garbageCollector recycleUnusedIndexFiles remove index files failed" , zap . Error ( err ) )
return true
2023-01-04 19:37:36 +08:00
}
2024-04-25 20:41:27 +08:00
logger . Info ( "garbageCollector recycleUnusedIndexFiles remove index files success" )
return true
2023-01-04 19:37:36 +08:00
}
2024-04-25 20:41:27 +08:00
filesMap := gc . getAllIndexFilesOfIndex ( segIdx )
logger . Info ( "recycle index files" , zap . Int ( "meta files num" , len ( filesMap ) ) )
deletedFilesNum := atomic . NewInt32 ( 0 )
fileNum := 0
futures := make ( [ ] * conc . Future [ struct { } ] , 0 )
err = gc . option . cli . WalkWithPrefix ( ctx , key , true , func ( indexFile * storage . ChunkObjectInfo ) bool {
fileNum ++
file := indexFile . FilePath
2023-01-04 19:37:36 +08:00
if _ , ok := filesMap [ file ] ; ! ok {
2024-04-25 20:41:27 +08:00
future := gc . option . removeObjectPool . Submit ( func ( ) ( struct { } , error ) {
logger := logger . With ( zap . String ( "file" , file ) )
logger . Info ( "garbageCollector recycleUnusedIndexFiles remove file..." )
if err := gc . option . cli . Remove ( ctx , file ) ; err != nil {
logger . Warn ( "garbageCollector recycleUnusedIndexFiles remove file failed" , zap . Error ( err ) )
return struct { } { } , err
}
deletedFilesNum . Inc ( )
logger . Info ( "garbageCollector recycleUnusedIndexFiles remove file success" )
return struct { } { } , nil
} )
futures = append ( futures , future )
2023-01-04 19:37:36 +08:00
}
2024-04-25 20:41:27 +08:00
return true
} )
// Wait for all remove tasks done.
if err := conc . BlockOnAll ( futures ... ) ; err != nil {
// error is logged, and can be ignored here.
logger . Warn ( "some task failure in remove object pool" , zap . Error ( err ) )
}
logger = logger . With ( zap . Int ( "deleteIndexFilesNum" , int ( deletedFilesNum . Load ( ) ) ) , zap . Int ( "walkFileNum" , fileNum ) )
if err != nil {
logger . Warn ( "index files recycle failed when walk with prefix" , zap . Error ( err ) )
return true
2023-01-04 19:37:36 +08:00
}
2024-04-25 20:41:27 +08:00
logger . Info ( "index files recycle done" )
return true
} )
log = log . With ( zap . Duration ( "timeCost" , time . Since ( start ) ) , zap . Int ( "keyCount" , keyCount ) , zap . Error ( err ) )
if err != nil {
log . Warn ( "garbageCollector recycleUnusedIndexFiles failed" , zap . Error ( err ) )
return
}
log . Info ( "recycleUnusedIndexFiles done" )
}
// getAllIndexFilesOfIndex returns the all index files of index.
func ( gc * garbageCollector ) getAllIndexFilesOfIndex ( segmentIndex * model . SegmentIndex ) map [ string ] struct { } {
filesMap := make ( map [ string ] struct { } )
for _ , fileID := range segmentIndex . IndexFileKeys {
filepath := metautil . BuildSegmentIndexFilePath ( gc . option . cli . RootPath ( ) , segmentIndex . BuildID , segmentIndex . IndexVersion ,
segmentIndex . PartitionID , segmentIndex . SegmentID , fileID )
filesMap [ filepath ] = struct { } { }
2023-01-04 19:37:36 +08:00
}
2024-04-25 20:41:27 +08:00
return filesMap
2023-01-04 19:37:36 +08:00
}
2024-06-10 21:34:08 +08:00
// recycleUnusedAnalyzeFiles is used to delete those analyze stats files that no longer exist in the meta.
2024-07-29 17:03:48 +08:00
func ( gc * garbageCollector ) recycleUnusedAnalyzeFiles ( ctx context . Context ) {
2024-06-10 21:34:08 +08:00
log . Info ( "start recycleUnusedAnalyzeFiles" )
startTs := time . Now ( )
prefix := path . Join ( gc . option . cli . RootPath ( ) , common . AnalyzeStatsPath ) + "/"
// list dir first
keys := make ( [ ] string , 0 )
err := gc . option . cli . WalkWithPrefix ( ctx , prefix , false , func ( chunkInfo * storage . ChunkObjectInfo ) bool {
keys = append ( keys , chunkInfo . FilePath )
return true
} )
if err != nil {
log . Warn ( "garbageCollector recycleUnusedAnalyzeFiles list keys from chunk manager failed" , zap . Error ( err ) )
return
}
log . Info ( "recycleUnusedAnalyzeFiles, finish list object" , zap . Duration ( "time spent" , time . Since ( startTs ) ) , zap . Int ( "task ids" , len ( keys ) ) )
for _ , key := range keys {
2024-07-29 17:03:48 +08:00
if ctx . Err ( ) != nil {
// process canceled
return
}
2024-06-10 21:34:08 +08:00
log . Debug ( "analyze keys" , zap . String ( "key" , key ) )
taskID , err := parseBuildIDFromFilePath ( key )
if err != nil {
log . Warn ( "garbageCollector recycleUnusedAnalyzeFiles parseAnalyzeResult failed" , zap . String ( "key" , key ) , zap . Error ( err ) )
continue
}
log . Info ( "garbageCollector will recycle analyze stats files" , zap . Int64 ( "taskID" , taskID ) )
canRecycle , task := gc . meta . analyzeMeta . CheckCleanAnalyzeTask ( taskID )
if ! canRecycle {
// Even if the analysis task is marked as deleted, the analysis stats file will not be recycled, wait for the next gc,
// and delete all index files about the taskID at one time.
log . Info ( "garbageCollector no need to recycle analyze stats files" , zap . Int64 ( "taskID" , taskID ) )
continue
}
if task == nil {
// taskID no longer exists in meta, remove all analysis files
log . Info ( "garbageCollector recycleUnusedAnalyzeFiles find meta has not exist, remove index files" ,
zap . Int64 ( "taskID" , taskID ) )
err = gc . option . cli . RemoveWithPrefix ( ctx , key )
if err != nil {
log . Warn ( "garbageCollector recycleUnusedAnalyzeFiles remove analyze stats files failed" ,
zap . Int64 ( "taskID" , taskID ) , zap . String ( "prefix" , key ) , zap . Error ( err ) )
continue
}
log . Info ( "garbageCollector recycleUnusedAnalyzeFiles remove analyze stats files success" ,
zap . Int64 ( "taskID" , taskID ) , zap . String ( "prefix" , key ) )
continue
}
log . Info ( "remove analyze stats files which version is less than current task" ,
zap . Int64 ( "taskID" , taskID ) , zap . Int64 ( "current version" , task . Version ) )
var i int64
for i = 0 ; i < task . Version ; i ++ {
2024-07-29 17:03:48 +08:00
if ctx . Err ( ) != nil {
// process canceled.
return
}
2024-06-10 21:34:08 +08:00
removePrefix := prefix + fmt . Sprintf ( "%d/" , task . Version )
if err := gc . option . cli . RemoveWithPrefix ( ctx , removePrefix ) ; err != nil {
log . Warn ( "garbageCollector recycleUnusedAnalyzeFiles remove files with prefix failed" ,
zap . Int64 ( "taskID" , taskID ) , zap . String ( "removePrefix" , removePrefix ) )
continue
}
}
log . Info ( "analyze stats files recycle success" , zap . Int64 ( "taskID" , taskID ) )
}
}