2021-10-25 20:21:18 +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
2021-04-19 11:35:38 +08:00
// with the License. You may obtain a copy of the License at
//
2021-10-25 20:21:18 +08:00
// http://www.apache.org/licenses/LICENSE-2.0
2021-04-19 11:35:38 +08:00
//
2021-10-25 20:21:18 +08:00
// 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-10-04 17:16:54 +08:00
2021-10-05 22:33:01 +08:00
// Package datacoord contains core functions in datacoord
2021-06-22 10:42:07 +08:00
package datacoord
2021-01-15 17:09:41 +08:00
import (
2022-08-20 10:24:51 +08:00
"context"
2021-01-15 17:09:41 +08:00
"fmt"
2023-12-05 18:44:37 +08:00
"math"
2024-06-10 21:34:08 +08:00
"path"
"strconv"
2021-08-19 14:08:10 +08:00
"time"
2021-01-15 17:09:41 +08:00
2023-09-18 09:53:22 +08:00
"github.com/cockroachdb/errors"
2022-10-10 20:31:22 +08:00
"github.com/golang/protobuf/proto"
2023-01-04 19:37:36 +08:00
"github.com/samber/lo"
"go.uber.org/zap"
2022-11-10 22:13:04 +08:00
"golang.org/x/exp/maps"
2022-08-25 15:48:54 +08:00
2023-06-09 01:28:37 +08:00
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/msgpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
2024-05-28 10:09:48 +08:00
"github.com/milvus-io/milvus/internal/datacoord/broker"
2022-08-20 10:24:51 +08:00
"github.com/milvus-io/milvus/internal/metastore"
2021-04-22 14:45:57 +08:00
"github.com/milvus-io/milvus/internal/proto/datapb"
2022-11-17 20:37:10 +08:00
"github.com/milvus-io/milvus/internal/storage"
2023-02-27 10:41:46 +08:00
"github.com/milvus-io/milvus/internal/util/segmentutil"
2023-04-06 19:14:32 +08:00
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
2024-05-28 10:09:48 +08:00
"github.com/milvus-io/milvus/pkg/util/funcutil"
2024-05-15 16:33:34 +08:00
"github.com/milvus-io/milvus/pkg/util/lock"
2024-02-28 19:03:01 +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-11-28 07:00:28 +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/timerecord"
2023-09-21 09:45:27 +08:00
"github.com/milvus-io/milvus/pkg/util/tsoutil"
2021-04-09 09:55:04 +08:00
)
2024-06-05 10:17:50 +08:00
type CompactionMeta interface {
GetSegment ( segID UniqueID ) * SegmentInfo
SelectSegments ( filters ... SegmentFilter ) [ ] * SegmentInfo
GetHealthySegment ( segID UniqueID ) * SegmentInfo
UpdateSegmentsInfo ( operators ... UpdateOperator ) error
SetSegmentCompacting ( segmentID int64 , compacting bool )
CheckAndSetSegmentsCompacting ( segmentIDs [ ] int64 ) ( bool , bool )
CompleteCompactionMutation ( plan * datapb . CompactionPlan , result * datapb . CompactionPlanResult ) ( [ ] * SegmentInfo , * segMetricMutation , error )
2024-06-10 21:34:08 +08:00
CleanPartitionStatsInfo ( info * datapb . PartitionStatsInfo ) error
2024-06-06 17:37:51 +08:00
2024-06-05 10:17:50 +08:00
SaveCompactionTask ( task * datapb . CompactionTask ) error
DropCompactionTask ( task * datapb . CompactionTask ) error
GetCompactionTasks ( ) map [ int64 ] [ ] * datapb . CompactionTask
GetCompactionTasksByTriggerID ( triggerID int64 ) [ ] * datapb . CompactionTask
2024-06-06 17:37:51 +08:00
GetIndexMeta ( ) * indexMeta
GetAnalyzeMeta ( ) * analyzeMeta
2024-06-10 21:34:08 +08:00
GetPartitionStatsMeta ( ) * partitionStatsMeta
2024-06-06 17:37:51 +08:00
GetCompactionTaskMeta ( ) * compactionTaskMeta
2024-06-05 10:17:50 +08:00
}
var _ CompactionMeta = ( * meta ) ( nil )
2021-06-22 18:24:08 +08:00
type meta struct {
2024-05-15 16:33:34 +08:00
lock . RWMutex
2024-03-07 20:39:02 +08:00
ctx context . Context
catalog metastore . DataCoordCatalog
collections map [ UniqueID ] * collectionInfo // collection id to collection info
segments * SegmentsInfo // segment id to segment info
channelCPs * channelCPs // vChannel -> channel checkpoint/see position
chunkManager storage . ChunkManager
2023-01-04 19:37:36 +08:00
2024-06-05 10:17:50 +08:00
indexMeta * indexMeta
2024-06-06 17:37:51 +08:00
analyzeMeta * analyzeMeta
2024-06-10 21:34:08 +08:00
partitionStatsMeta * partitionStatsMeta
2024-06-05 10:17:50 +08:00
compactionTaskMeta * compactionTaskMeta
2022-10-10 20:31:22 +08:00
}
2024-06-06 17:37:51 +08:00
func ( m * meta ) GetIndexMeta ( ) * indexMeta {
return m . indexMeta
}
func ( m * meta ) GetAnalyzeMeta ( ) * analyzeMeta {
return m . analyzeMeta
}
2024-06-10 21:34:08 +08:00
func ( m * meta ) GetPartitionStatsMeta ( ) * partitionStatsMeta {
return m . partitionStatsMeta
}
2024-06-06 17:37:51 +08:00
func ( m * meta ) GetCompactionTaskMeta ( ) * compactionTaskMeta {
return m . compactionTaskMeta
}
2024-03-07 20:39:02 +08:00
type channelCPs struct {
2024-05-15 16:33:34 +08:00
lock . RWMutex
2024-03-07 20:39:02 +08:00
checkpoints map [ string ] * msgpb . MsgPosition
}
func newChannelCps ( ) * channelCPs {
return & channelCPs {
checkpoints : make ( map [ string ] * msgpb . MsgPosition ) ,
}
}
2022-11-22 19:21:13 +08:00
// A local cache of segment metric update. Must call commit() to take effect.
type segMetricMutation struct {
2023-11-24 15:58:24 +08:00
stateChange map [ string ] map [ string ] int // segment state, seg level -> state change count (to increase or decrease).
rowCountChange int64 // Change in # of rows.
rowCountAccChange int64 // Total # of historical added rows, accumulated.
2022-11-22 19:21:13 +08:00
}
2022-10-10 20:31:22 +08:00
type collectionInfo struct {
ID int64
Schema * schemapb . CollectionSchema
Partitions [ ] int64
StartPositions [ ] * commonpb . KeyDataPair
Properties map [ string ] string
2023-03-08 18:53:51 +08:00
CreatedAt Timestamp
2024-04-02 14:27:13 +08:00
DatabaseName string
2024-04-12 16:01:19 +08:00
DatabaseID int64
2024-04-30 09:45:27 +08:00
VChannelNames [ ] string
2021-06-22 18:24:08 +08:00
}
2021-12-09 22:07:10 +08:00
// NewMeta creates meta from provided `kv.TxnKV`
2023-01-06 14:33:36 +08:00
func newMeta ( ctx context . Context , catalog metastore . DataCoordCatalog , chunkManager storage . ChunkManager ) ( * meta , error ) {
2024-06-06 17:37:51 +08:00
im , err := newIndexMeta ( ctx , catalog )
2024-03-04 16:56:59 +08:00
if err != nil {
return nil , err
}
2024-06-06 17:37:51 +08:00
am , err := newAnalyzeMeta ( ctx , catalog )
2024-06-05 10:17:50 +08:00
if err != nil {
return nil , err
}
2024-06-10 21:34:08 +08:00
psm , err := newPartitionStatsMeta ( ctx , catalog )
if err != nil {
return nil , err
}
2024-06-06 17:37:51 +08:00
ctm , err := newCompactionTaskMeta ( ctx , catalog )
if err != nil {
return nil , err
}
2021-01-15 17:09:41 +08:00
mt := & meta {
2024-06-05 10:17:50 +08:00
ctx : ctx ,
catalog : catalog ,
collections : make ( map [ UniqueID ] * collectionInfo ) ,
segments : NewSegmentsInfo ( ) ,
channelCPs : newChannelCps ( ) ,
2024-06-06 17:37:51 +08:00
indexMeta : im ,
analyzeMeta : am ,
2024-06-05 10:17:50 +08:00
chunkManager : chunkManager ,
2024-06-10 21:34:08 +08:00
partitionStatsMeta : psm ,
2024-06-05 10:17:50 +08:00
compactionTaskMeta : ctm ,
2024-03-04 16:56:59 +08:00
}
err = mt . reloadFromKV ( )
2021-01-15 17:09:41 +08:00
if err != nil {
return nil , err
}
return mt , nil
}
2021-12-09 22:07:10 +08:00
// reloadFromKV loads meta from KV storage
2021-05-25 15:35:37 +08:00
func ( m * meta ) reloadFromKV ( ) error {
2022-11-08 20:13:03 +08:00
record := timerecord . NewTimeRecorder ( "datacoord" )
2022-08-20 10:24:51 +08:00
segments , err := m . catalog . ListSegments ( m . ctx )
2021-01-15 17:09:41 +08:00
if err != nil {
return err
}
2022-03-15 21:51:21 +08:00
metrics . DataCoordNumCollections . WithLabelValues ( ) . Set ( 0 )
2023-11-24 15:58:24 +08:00
metrics . DataCoordNumSegments . Reset ( )
2022-03-15 21:51:21 +08:00
numStoredRows := int64 ( 0 )
2022-08-20 10:24:51 +08:00
for _ , segment := range segments {
2024-01-18 22:06:31 +08:00
// segments from catalog.ListSegments will not have logPath
2022-08-20 10:24:51 +08:00
m . segments . SetSegment ( segment . ID , NewSegmentInfo ( segment ) )
2023-12-20 11:20:44 +08:00
metrics . DataCoordNumSegments . WithLabelValues ( segment . GetState ( ) . String ( ) , segment . GetLevel ( ) . String ( ) ) . Inc ( )
2022-08-20 10:24:51 +08:00
if segment . State == commonpb . SegmentState_Flushed {
numStoredRows += segment . NumOfRows
2023-06-26 17:52:44 +08:00
insertFileNum := 0
for _ , fieldBinlog := range segment . GetBinlogs ( ) {
insertFileNum += len ( fieldBinlog . GetBinlogs ( ) )
}
metrics . FlushedSegmentFileNum . WithLabelValues ( metrics . InsertFileLabel ) . Observe ( float64 ( insertFileNum ) )
statFileNum := 0
for _ , fieldBinlog := range segment . GetStatslogs ( ) {
statFileNum += len ( fieldBinlog . GetBinlogs ( ) )
}
metrics . FlushedSegmentFileNum . WithLabelValues ( metrics . StatFileLabel ) . Observe ( float64 ( statFileNum ) )
deleteFileNum := 0
for _ , filedBinlog := range segment . GetDeltalogs ( ) {
deleteFileNum += len ( filedBinlog . GetBinlogs ( ) )
}
metrics . FlushedSegmentFileNum . WithLabelValues ( metrics . DeleteFileLabel ) . Observe ( float64 ( deleteFileNum ) )
2022-03-15 21:51:21 +08:00
}
2021-01-15 17:09:41 +08:00
}
2022-11-10 22:13:04 +08:00
channelCPs , err := m . catalog . ListChannelCheckpoint ( m . ctx )
if err != nil {
return err
}
for vChannel , pos := range channelCPs {
2023-02-16 16:46:35 +08:00
// for 2.2.2 issue https://github.com/milvus-io/milvus/issues/22181
pos . ChannelName = vChannel
2024-03-07 20:39:02 +08:00
m . channelCPs . checkpoints [ vChannel ] = pos
2022-11-10 22:13:04 +08:00
}
2024-06-05 10:17:50 +08:00
2023-03-30 18:56:22 +08:00
log . Info ( "DataCoord meta reloadFromKV done" , zap . Duration ( "duration" , record . ElapseSpan ( ) ) )
2021-01-15 17:09:41 +08:00
return nil
}
2024-05-28 10:09:48 +08:00
func ( m * meta ) reloadCollectionsFromRootcoord ( ctx context . Context , broker broker . Broker ) error {
resp , err := broker . ListDatabases ( ctx )
if err != nil {
return err
}
for _ , dbName := range resp . GetDbNames ( ) {
resp , err := broker . ShowCollections ( ctx , dbName )
if err != nil {
return err
}
for _ , collectionID := range resp . GetCollectionIds ( ) {
resp , err := broker . DescribeCollectionInternal ( ctx , collectionID )
if err != nil {
return err
}
partitionIDs , err := broker . ShowPartitionsInternal ( ctx , collectionID )
if err != nil {
return err
}
collection := & collectionInfo {
ID : collectionID ,
Schema : resp . GetSchema ( ) ,
Partitions : partitionIDs ,
StartPositions : resp . GetStartPositions ( ) ,
Properties : funcutil . KeyValuePair2Map ( resp . GetProperties ( ) ) ,
CreatedAt : resp . GetCreatedTimestamp ( ) ,
DatabaseName : resp . GetDbName ( ) ,
DatabaseID : resp . GetDbId ( ) ,
VChannelNames : resp . GetVirtualChannelNames ( ) ,
}
m . AddCollection ( collection )
}
}
return nil
}
2021-12-09 22:07:10 +08:00
// AddCollection adds a collection into meta
2021-09-08 11:35:59 +08:00
// Note that collection info is just for caching and will not be set into etcd from datacoord
2022-10-10 20:31:22 +08:00
func ( m * meta ) AddCollection ( collection * collectionInfo ) {
2024-04-15 16:33:19 +08:00
log . Info ( "meta update: add collection" , zap . Int64 ( "collectionID" , collection . ID ) )
2021-05-25 15:35:37 +08:00
m . Lock ( )
defer m . Unlock ( )
m . collections [ collection . ID ] = collection
2022-03-15 21:51:21 +08:00
metrics . DataCoordNumCollections . WithLabelValues ( ) . Set ( float64 ( len ( m . collections ) ) )
2023-07-14 15:56:31 +08:00
log . Info ( "meta update: add collection - complete" , zap . Int64 ( "collectionID" , collection . ID ) )
2021-01-15 17:09:41 +08:00
}
2024-04-15 16:33:19 +08:00
// DropCollection drop a collection from meta
func ( m * meta ) DropCollection ( collectionID int64 ) {
log . Info ( "meta update: drop collection" , zap . Int64 ( "collectionID" , collectionID ) )
m . Lock ( )
defer m . Unlock ( )
delete ( m . collections , collectionID )
2024-05-20 22:03:39 +08:00
metrics . CleanupDataCoordWithCollectionID ( collectionID )
2024-04-15 16:33:19 +08:00
metrics . DataCoordNumCollections . WithLabelValues ( ) . Set ( float64 ( len ( m . collections ) ) )
log . Info ( "meta update: drop collection - complete" , zap . Int64 ( "collectionID" , collectionID ) )
}
2021-12-15 16:57:23 +08:00
// GetCollection returns collection info with provided collection id from local cache
2022-10-10 20:31:22 +08:00
func ( m * meta ) GetCollection ( collectionID UniqueID ) * collectionInfo {
2021-05-25 15:35:37 +08:00
m . RLock ( )
defer m . RUnlock ( )
collection , ok := m . collections [ collectionID ]
2021-01-19 15:35:40 +08:00
if ! ok {
2021-07-07 14:02:01 +08:00
return nil
2021-01-19 15:35:40 +08:00
}
2021-07-07 14:02:01 +08:00
return collection
2021-01-19 15:35:40 +08:00
}
2021-01-15 17:09:41 +08:00
2024-06-05 10:17:50 +08:00
// GetCollections returns collections from local cache
func ( m * meta ) GetCollections ( ) [ ] * collectionInfo {
m . RLock ( )
defer m . RUnlock ( )
collections := make ( [ ] * collectionInfo , 0 )
for _ , coll := range m . collections {
collections = append ( collections , coll )
}
return collections
}
2022-10-10 20:31:22 +08:00
func ( m * meta ) GetClonedCollectionInfo ( collectionID UniqueID ) * collectionInfo {
m . RLock ( )
defer m . RUnlock ( )
coll , ok := m . collections [ collectionID ]
if ! ok {
return nil
}
clonedProperties := make ( map [ string ] string )
maps . Copy ( clonedProperties , coll . Properties )
cloneColl := & collectionInfo {
ID : coll . ID ,
Schema : proto . Clone ( coll . Schema ) . ( * schemapb . CollectionSchema ) ,
Partitions : coll . Partitions ,
StartPositions : common . CloneKeyDataPairs ( coll . StartPositions ) ,
Properties : clonedProperties ,
2024-04-02 14:27:13 +08:00
DatabaseName : coll . DatabaseName ,
2024-04-12 16:01:19 +08:00
DatabaseID : coll . DatabaseID ,
2024-04-30 09:45:27 +08:00
VChannelNames : coll . VChannelNames ,
2022-10-10 20:31:22 +08:00
}
return cloneColl
}
2022-09-30 11:48:55 +08:00
// GetSegmentsChanPart returns segments organized in Channel-Partition dimension with selector applied
2021-11-05 22:25:00 +08:00
func ( m * meta ) GetSegmentsChanPart ( selector SegmentInfoSelector ) [ ] * chanPartSegments {
m . RLock ( )
defer m . RUnlock ( )
mDimEntry := make ( map [ string ] * chanPartSegments )
2024-06-05 10:17:50 +08:00
log . Debug ( "GetSegmentsChanPart segment number" , zap . Int ( "length" , len ( m . segments . GetSegments ( ) ) ) )
2021-11-05 22:25:00 +08:00
for _ , segmentInfo := range m . segments . segments {
if ! selector ( segmentInfo ) {
continue
}
2024-02-04 10:21:06 +08:00
cloned := segmentInfo . Clone ( )
dim := fmt . Sprintf ( "%d-%s" , cloned . PartitionID , cloned . InsertChannel )
2021-11-05 22:25:00 +08:00
entry , ok := mDimEntry [ dim ]
if ! ok {
entry = & chanPartSegments {
2024-02-04 10:21:06 +08:00
collectionID : cloned . CollectionID ,
partitionID : cloned . PartitionID ,
channelName : cloned . InsertChannel ,
2021-11-05 22:25:00 +08:00
}
mDimEntry [ dim ] = entry
}
2024-02-04 10:21:06 +08:00
entry . segments = append ( entry . segments , cloned )
2021-11-05 22:25:00 +08:00
}
result := make ( [ ] * chanPartSegments , 0 , len ( mDimEntry ) )
for _ , entry := range mDimEntry {
result = append ( result , entry )
}
return result
}
2024-03-19 15:23:06 +08:00
func ( m * meta ) getNumRowsOfCollectionUnsafe ( collectionID UniqueID ) int64 {
2021-12-14 15:31:07 +08:00
var ret int64
2021-07-07 14:02:01 +08:00
segments := m . segments . GetSegments ( )
for _ , segment := range segments {
2021-11-12 00:22:42 +08:00
if isSegmentHealthy ( segment ) && segment . GetCollectionID ( ) == collectionID {
2021-06-22 18:24:08 +08:00
ret += segment . GetNumOfRows ( )
2021-02-02 14:25:58 +08:00
}
}
2021-07-07 14:02:01 +08:00
return ret
2021-02-02 14:25:58 +08:00
}
2024-03-19 15:23:06 +08:00
// GetNumRowsOfCollection returns total rows count of segments belongs to provided collection
func ( m * meta ) GetNumRowsOfCollection ( collectionID UniqueID ) int64 {
m . RLock ( )
defer m . RUnlock ( )
return m . getNumRowsOfCollectionUnsafe ( collectionID )
}
2023-04-26 21:52:36 +08:00
// GetCollectionBinlogSize returns the total binlog size and binlog size of collections.
2024-04-12 16:01:19 +08:00
func ( m * meta ) GetCollectionBinlogSize ( ) ( int64 , map [ UniqueID ] int64 , map [ UniqueID ] map [ UniqueID ] int64 ) {
2022-10-12 10:03:22 +08:00
m . RLock ( )
defer m . RUnlock ( )
2023-04-26 21:52:36 +08:00
collectionBinlogSize := make ( map [ UniqueID ] int64 )
2024-04-12 16:01:19 +08:00
partitionBinlogSize := make ( map [ UniqueID ] map [ UniqueID ] int64 )
2023-08-17 15:52:17 +08:00
collectionRowsNum := make ( map [ UniqueID ] map [ commonpb . SegmentState ] int64 )
2022-10-12 10:03:22 +08:00
segments := m . segments . GetSegments ( )
2023-04-26 21:52:36 +08:00
var total int64
2022-10-12 10:03:22 +08:00
for _ , segment := range segments {
2023-04-26 21:52:36 +08:00
segmentSize := segment . getSegmentSize ( )
2024-03-15 14:43:03 +08:00
if isSegmentHealthy ( segment ) && ! segment . GetIsImporting ( ) {
2023-04-26 21:52:36 +08:00
total += segmentSize
collectionBinlogSize [ segment . GetCollectionID ( ) ] += segmentSize
2024-04-10 21:01:20 +08:00
2024-04-12 16:01:19 +08:00
partBinlogSize , ok := partitionBinlogSize [ segment . GetCollectionID ( ) ]
if ! ok {
partBinlogSize = make ( map [ int64 ] int64 )
partitionBinlogSize [ segment . GetCollectionID ( ) ] = partBinlogSize
}
partBinlogSize [ segment . GetPartitionID ( ) ] += segmentSize
2024-04-10 21:01:20 +08:00
coll , ok := m . collections [ segment . GetCollectionID ( ) ]
if ok {
metrics . DataCoordStoredBinlogSize . WithLabelValues ( coll . DatabaseName ,
fmt . Sprint ( segment . GetCollectionID ( ) ) , fmt . Sprint ( segment . GetID ( ) ) ) . Set ( float64 ( segmentSize ) )
} else {
log . Warn ( "not found database name" , zap . Int64 ( "collectionID" , segment . GetCollectionID ( ) ) )
}
2023-08-17 15:52:17 +08:00
if _ , ok := collectionRowsNum [ segment . GetCollectionID ( ) ] ; ! ok {
collectionRowsNum [ segment . GetCollectionID ( ) ] = make ( map [ commonpb . SegmentState ] int64 )
}
collectionRowsNum [ segment . GetCollectionID ( ) ] [ segment . GetState ( ) ] += segment . GetNumOfRows ( )
}
}
2024-05-20 22:03:39 +08:00
metrics . DataCoordNumStoredRows . Reset ( )
2024-04-02 14:27:13 +08:00
for collectionID , statesRows := range collectionRowsNum {
2023-08-17 15:52:17 +08:00
for state , rows := range statesRows {
2024-04-02 14:27:13 +08:00
coll , ok := m . collections [ collectionID ]
if ok {
metrics . DataCoordNumStoredRows . WithLabelValues ( coll . DatabaseName , fmt . Sprint ( collectionID ) , state . String ( ) ) . Set ( float64 ( rows ) )
}
2022-11-14 21:03:09 +08:00
}
}
2024-04-12 16:01:19 +08:00
return total , collectionBinlogSize , partitionBinlogSize
2022-10-12 10:03:22 +08:00
}
2024-05-13 17:59:32 +08:00
// GetCollectionIndexFilesSize returns the total index files size of all segment for each collection.
func ( m * meta ) GetCollectionIndexFilesSize ( ) uint64 {
2024-05-14 18:49:33 +08:00
m . RLock ( )
defer m . RUnlock ( )
2024-05-13 17:59:32 +08:00
var total uint64
for _ , segmentIdx := range m . indexMeta . GetAllSegIndexes ( ) {
coll , ok := m . collections [ segmentIdx . CollectionID ]
if ok {
metrics . DataCoordStoredIndexFilesSize . WithLabelValues ( coll . DatabaseName ,
fmt . Sprint ( segmentIdx . CollectionID ) , fmt . Sprint ( segmentIdx . SegmentID ) ) . Set ( float64 ( segmentIdx . IndexSize ) )
total += segmentIdx . IndexSize
}
}
return total
}
2024-03-19 15:23:06 +08:00
func ( m * meta ) GetAllCollectionNumRows ( ) map [ int64 ] int64 {
m . RLock ( )
defer m . RUnlock ( )
ret := make ( map [ int64 ] int64 , len ( m . collections ) )
2024-04-20 07:55:21 +08:00
segments := m . segments . GetSegments ( )
for _ , segment := range segments {
if isSegmentHealthy ( segment ) {
ret [ segment . GetCollectionID ( ) ] += segment . GetNumOfRows ( )
}
2024-03-19 15:23:06 +08:00
}
return ret
}
2021-09-08 11:35:59 +08:00
// AddSegment records segment info, persisting info into kv store
2023-09-18 09:53:22 +08:00
func ( m * meta ) AddSegment ( ctx context . Context , segment * SegmentInfo ) error {
log := log . Ctx ( ctx )
log . Info ( "meta update: adding segment - Start" , zap . Int64 ( "segmentID" , segment . GetID ( ) ) )
2021-05-25 15:35:37 +08:00
m . Lock ( )
defer m . Unlock ( )
2022-08-20 10:24:51 +08:00
if err := m . catalog . AddSegment ( m . ctx , segment . SegmentInfo ) ; err != nil {
2022-09-30 11:48:55 +08:00
log . Error ( "meta update: adding segment failed" ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "segmentID" , segment . GetID ( ) ) ,
2022-09-30 11:48:55 +08:00
zap . Error ( err ) )
2021-01-15 17:09:41 +08:00
return err
}
2022-07-08 13:48:21 +08:00
m . segments . SetSegment ( segment . GetID ( ) , segment )
2024-01-18 22:06:31 +08:00
2023-11-24 15:58:24 +08:00
metrics . DataCoordNumSegments . WithLabelValues ( segment . GetState ( ) . String ( ) , segment . GetLevel ( ) . String ( ) ) . Inc ( )
2023-07-14 15:56:31 +08:00
log . Info ( "meta update: adding segment - complete" , zap . Int64 ( "segmentID" , segment . GetID ( ) ) )
2021-01-15 17:09:41 +08:00
return nil
}
2021-09-08 11:35:59 +08:00
// DropSegment remove segment with provided id, etcd persistence also removed
2021-05-25 15:35:37 +08:00
func ( m * meta ) DropSegment ( segmentID UniqueID ) error {
2023-07-14 15:56:31 +08:00
log . Debug ( "meta update: dropping segment" , zap . Int64 ( "segmentID" , segmentID ) )
2021-05-25 15:35:37 +08:00
m . Lock ( )
defer m . Unlock ( )
2021-07-07 14:02:01 +08:00
segment := m . segments . GetSegment ( segmentID )
2021-09-08 11:35:59 +08:00
if segment == nil {
2022-09-30 11:48:55 +08:00
log . Warn ( "meta update: dropping segment failed - segment not found" ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "segmentID" , segmentID ) )
2021-09-08 11:35:59 +08:00
return nil
}
2022-08-20 10:24:51 +08:00
if err := m . catalog . DropSegment ( m . ctx , segment . SegmentInfo ) ; err != nil {
2022-09-30 11:48:55 +08:00
log . Warn ( "meta update: dropping segment failed" ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "segmentID" , segmentID ) ,
2022-09-30 11:48:55 +08:00
zap . Error ( err ) )
2021-02-02 14:25:58 +08:00
return err
}
2023-11-24 15:58:24 +08:00
metrics . DataCoordNumSegments . WithLabelValues ( segment . GetState ( ) . String ( ) , segment . GetLevel ( ) . String ( ) ) . Dec ( )
2024-05-14 18:49:33 +08:00
coll , ok := m . collections [ segment . CollectionID ]
if ok {
metrics . CleanupDataCoordSegmentMetrics ( coll . DatabaseName , segment . CollectionID , segment . ID )
}
2021-11-23 11:23:15 +08:00
m . segments . DropSegment ( segmentID )
2022-10-25 19:31:30 +08:00
log . Info ( "meta update: dropping segment - complete" ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "segmentID" , segmentID ) )
2021-01-19 15:35:40 +08:00
return nil
}
2023-03-03 14:13:49 +08:00
// GetHealthySegment returns segment info with provided id
2021-09-08 11:35:59 +08:00
// if not segment is found, nil will be returned
2023-03-03 14:13:49 +08:00
func ( m * meta ) GetHealthySegment ( segID UniqueID ) * SegmentInfo {
2021-05-25 15:35:37 +08:00
m . RLock ( )
defer m . RUnlock ( )
2021-11-12 00:22:42 +08:00
segment := m . segments . GetSegment ( segID )
if segment != nil && isSegmentHealthy ( segment ) {
2024-03-01 13:59:02 +08:00
return segment
2021-11-12 00:22:42 +08:00
}
return nil
2021-01-15 17:09:41 +08:00
}
2024-03-05 14:47:00 +08:00
// Get segments By filter function
func ( m * meta ) GetSegments ( segIDs [ ] UniqueID , filterFunc SegmentInfoSelector ) [ ] UniqueID {
m . RLock ( )
defer m . RUnlock ( )
var result [ ] UniqueID
for _ , id := range segIDs {
segment := m . segments . GetSegment ( id )
if segment != nil && filterFunc ( segment ) {
result = append ( result , id )
}
}
return result
}
2023-03-03 14:13:49 +08:00
// GetSegment returns segment info with provided id
2022-10-15 15:13:24 +08:00
// include the unhealthy segment
2022-09-16 11:32:48 +08:00
// if not segment is found, nil will be returned
2023-03-03 14:13:49 +08:00
func ( m * meta ) GetSegment ( segID UniqueID ) * SegmentInfo {
2022-09-16 11:32:48 +08:00
m . RLock ( )
defer m . RUnlock ( )
return m . segments . GetSegment ( segID )
}
2022-10-15 15:13:24 +08:00
// GetAllSegmentsUnsafe returns all segments
func ( m * meta ) GetAllSegmentsUnsafe ( ) [ ] * SegmentInfo {
2022-06-17 18:24:12 +08:00
m . RLock ( )
defer m . RUnlock ( )
2022-10-15 15:13:24 +08:00
return m . segments . GetSegments ( )
2022-06-17 18:24:12 +08:00
}
2024-03-01 18:31:02 +08:00
func ( m * meta ) GetSegmentsTotalCurrentRows ( segmentIDs [ ] UniqueID ) int64 {
m . RLock ( )
defer m . RUnlock ( )
var sum int64 = 0
for _ , segmentID := range segmentIDs {
segment := m . segments . GetSegment ( segmentID )
if segment == nil {
log . Warn ( "cannot find segment" , zap . Int64 ( "segmentID" , segmentID ) )
continue
}
sum += segment . currRows
}
return sum
}
func ( m * meta ) GetSegmentsChannels ( segmentIDs [ ] UniqueID ) ( map [ int64 ] string , error ) {
m . RLock ( )
defer m . RUnlock ( )
segChannels := make ( map [ int64 ] string )
for _ , segmentID := range segmentIDs {
segment := m . segments . GetSegment ( segmentID )
if segment == nil {
return nil , errors . New ( fmt . Sprintf ( "cannot find segment %d" , segmentID ) )
}
segChannels [ segmentID ] = segment . GetInsertChannel ( )
}
return segChannels , nil
}
2021-09-08 11:35:59 +08:00
// SetState setting segment with provided ID state
2022-09-26 18:06:54 +08:00
func ( m * meta ) SetState ( segmentID UniqueID , targetState commonpb . SegmentState ) error {
2023-07-14 15:56:31 +08:00
log . Debug ( "meta update: setting segment state" ,
zap . Int64 ( "segmentID" , segmentID ) ,
2022-09-30 11:48:55 +08:00
zap . Any ( "target state" , targetState ) )
2021-05-25 15:35:37 +08:00
m . Lock ( )
defer m . Unlock ( )
2022-03-15 21:51:21 +08:00
curSegInfo := m . segments . GetSegment ( segmentID )
if curSegInfo == nil {
2022-09-30 11:48:55 +08:00
log . Warn ( "meta update: setting segment state - segment not found" ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "segmentID" , segmentID ) ,
2022-09-30 11:48:55 +08:00
zap . Any ( "target state" , targetState ) )
2023-01-31 12:41:53 +08:00
// idempotent drop
if targetState == commonpb . SegmentState_Dropped {
return nil
}
return fmt . Errorf ( "segment is not exist with ID = %d" , segmentID )
2022-03-15 21:51:21 +08:00
}
2022-09-26 18:06:54 +08:00
// Persist segment updates first.
clonedSegment := curSegInfo . Clone ( )
2022-11-22 19:21:13 +08:00
metricMutation := & segMetricMutation {
2023-11-24 15:58:24 +08:00
stateChange : make ( map [ string ] map [ string ] int ) ,
2022-11-22 19:21:13 +08:00
}
2022-09-26 18:06:54 +08:00
if clonedSegment != nil && isSegmentHealthy ( clonedSegment ) {
2023-07-21 14:21:05 +08:00
// Update segment state and prepare segment metric update.
updateSegStateAndPrepareMetrics ( clonedSegment , targetState , metricMutation )
2023-07-06 20:46:25 +08:00
if err := m . catalog . AlterSegments ( m . ctx , [ ] * datapb . SegmentInfo { clonedSegment . SegmentInfo } ) ; err != nil {
2023-07-14 15:56:31 +08:00
log . Warn ( "meta update: setting segment state - failed to alter segments" ,
zap . Int64 ( "segmentID" , segmentID ) ,
2022-09-30 11:48:55 +08:00
zap . String ( "target state" , targetState . String ( ) ) ,
2022-09-26 18:06:54 +08:00
zap . Error ( err ) )
return err
}
2022-11-22 19:21:13 +08:00
// Apply segment metric update after successful meta update.
metricMutation . commit ( )
2023-07-21 14:21:05 +08:00
// Update in-memory meta.
m . segments . SetState ( segmentID , targetState )
2022-09-26 18:06:54 +08:00
}
2022-10-25 19:31:30 +08:00
log . Info ( "meta update: setting segment state - complete" ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "segmentID" , segmentID ) ,
2022-09-30 11:48:55 +08:00
zap . String ( "target state" , targetState . String ( ) ) )
2022-09-26 18:06:54 +08:00
return nil
}
2024-03-01 13:59:02 +08:00
func ( m * meta ) UpdateSegment ( segmentID int64 , operators ... SegmentOperator ) error {
m . Lock ( )
defer m . Unlock ( )
info := m . segments . GetSegment ( segmentID )
if info == nil {
log . Warn ( "meta update: UpdateSegment - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) )
return merr . WrapErrSegmentNotFound ( segmentID )
}
// Persist segment updates first.
cloned := info . Clone ( )
var updated bool
for _ , operator := range operators {
updated = updated || operator ( cloned )
}
if ! updated {
log . Warn ( "meta update:UpdateSegmnt skipped, no update" ,
zap . Int64 ( "segmentID" , segmentID ) ,
)
return nil
}
if err := m . catalog . AlterSegments ( m . ctx , [ ] * datapb . SegmentInfo { cloned . SegmentInfo } ) ; err != nil {
log . Warn ( "meta update: update segment - failed to alter segments" ,
zap . Int64 ( "segmentID" , segmentID ) ,
zap . Error ( err ) )
return err
}
// Update in-memory meta.
m . segments . SetSegment ( segmentID , cloned )
log . Info ( "meta update: update segment - complete" ,
zap . Int64 ( "segmentID" , segmentID ) )
return nil
}
2023-11-21 16:48:22 +08:00
type updateSegmentPack struct {
2023-11-06 15:26:16 +08:00
meta * meta
segments map [ int64 ] * SegmentInfo
// for update etcd binlog paths
increments map [ int64 ] metastore . BinlogsIncrement
// for update segment metric after alter segments
metricMutation * segMetricMutation
}
2022-09-30 11:48:55 +08:00
2023-11-21 16:48:22 +08:00
func ( p * updateSegmentPack ) Get ( segmentID int64 ) * SegmentInfo {
2023-11-06 15:26:16 +08:00
if segment , ok := p . segments [ segmentID ] ; ok {
return segment
}
segment := p . meta . segments . GetSegment ( segmentID )
2021-11-12 00:22:42 +08:00
if segment == nil || ! isSegmentHealthy ( segment ) {
2023-11-06 15:26:16 +08:00
log . Warn ( "meta update: get segment failed - segment not found" ,
2023-07-14 15:56:31 +08:00
zap . Int64 ( "segmentID" , segmentID ) ,
2022-09-30 11:48:55 +08:00
zap . Bool ( "segment nil" , segment == nil ) ,
zap . Bool ( "segment unhealthy" , ! isSegmentHealthy ( segment ) ) )
2021-08-19 13:00:12 +08:00
return nil
2021-05-31 18:47:32 +08:00
}
2021-08-19 13:00:12 +08:00
2023-11-06 15:26:16 +08:00
p . segments [ segmentID ] = segment . Clone ( )
return p . segments [ segmentID ]
}
2023-11-21 16:48:22 +08:00
type UpdateOperator func ( * updateSegmentPack ) bool
2023-11-06 15:26:16 +08:00
func CreateL0Operator ( collectionID , partitionID , segmentID int64 , channel string ) UpdateOperator {
2023-11-21 16:48:22 +08:00
return func ( modPack * updateSegmentPack ) bool {
segment := modPack . meta . segments . GetSegment ( segmentID )
2023-11-06 15:26:16 +08:00
if segment == nil {
log . Info ( "meta update: add new l0 segment" ,
zap . Int64 ( "collectionID" , collectionID ) ,
zap . Int64 ( "partitionID" , partitionID ) ,
zap . Int64 ( "segmentID" , segmentID ) )
2023-11-21 16:48:22 +08:00
modPack . segments [ segmentID ] = & SegmentInfo {
SegmentInfo : & datapb . SegmentInfo {
ID : segmentID ,
CollectionID : collectionID ,
PartitionID : partitionID ,
InsertChannel : channel ,
NumOfRows : 0 ,
2024-02-04 10:21:06 +08:00
State : commonpb . SegmentState_Flushed ,
2023-11-21 16:48:22 +08:00
Level : datapb . SegmentLevel_L0 ,
} ,
}
2024-02-04 10:21:06 +08:00
modPack . metricMutation . addNewSeg ( commonpb . SegmentState_Flushed , datapb . SegmentLevel_L0 , 0 )
2021-08-19 13:00:12 +08:00
}
2023-11-06 15:26:16 +08:00
return true
2021-08-19 13:00:12 +08:00
}
2023-11-06 15:26:16 +08:00
}
2023-11-23 17:26:24 +08:00
func UpdateStorageVersionOperator ( segmentID int64 , version int64 ) UpdateOperator {
return func ( modPack * updateSegmentPack ) bool {
2024-01-12 18:10:51 +08:00
segment := modPack . Get ( segmentID )
2023-11-23 17:26:24 +08:00
if segment == nil {
log . Info ( "meta update: update storage version - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) )
return false
}
segment . StorageVersion = version
return true
}
}
2023-11-06 15:26:16 +08:00
// Set status of segment
// and record dropped time when change segment status to dropped
func UpdateStatusOperator ( segmentID int64 , status commonpb . SegmentState ) UpdateOperator {
2023-11-21 16:48:22 +08:00
return func ( modPack * updateSegmentPack ) bool {
2023-11-06 15:26:16 +08:00
segment := modPack . Get ( segmentID )
if segment == nil {
log . Warn ( "meta update: update status failed - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) ,
zap . String ( "status" , status . String ( ) ) )
return false
2022-10-31 17:41:34 +08:00
}
2023-11-06 15:26:16 +08:00
updateSegStateAndPrepareMetrics ( segment , status , modPack . metricMutation )
if status == commonpb . SegmentState_Dropped {
segment . DroppedAt = uint64 ( time . Now ( ) . UnixNano ( ) )
2021-12-19 20:00:42 +08:00
}
2023-11-06 15:26:16 +08:00
return true
2021-12-19 20:00:42 +08:00
}
2023-11-06 15:26:16 +08:00
}
2024-01-09 10:52:49 +08:00
func UpdateCompactedOperator ( segmentID int64 ) UpdateOperator {
return func ( modPack * updateSegmentPack ) bool {
segment := modPack . Get ( segmentID )
if segment == nil {
log . Warn ( "meta update: update binlog failed - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) )
return false
}
segment . Compacted = true
return true
}
}
2024-06-10 21:34:08 +08:00
func UpdateSegmentLevelOperator ( segmentID int64 , level datapb . SegmentLevel ) UpdateOperator {
return func ( modPack * updateSegmentPack ) bool {
segment := modPack . Get ( segmentID )
if segment == nil {
log . Warn ( "meta update: update level fail - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) )
return false
}
segment . LastLevel = segment . Level
segment . Level = level
return true
}
}
func UpdateSegmentPartitionStatsVersionOperator ( segmentID int64 , version int64 ) UpdateOperator {
return func ( modPack * updateSegmentPack ) bool {
segment := modPack . Get ( segmentID )
if segment == nil {
log . Warn ( "meta update: update partition stats version fail - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) )
return false
}
segment . LastPartitionStatsVersion = segment . PartitionStatsVersion
segment . PartitionStatsVersion = version
return true
}
}
func RevertSegmentLevelOperator ( segmentID int64 ) UpdateOperator {
return func ( modPack * updateSegmentPack ) bool {
segment := modPack . Get ( segmentID )
if segment == nil {
log . Warn ( "meta update: revert level fail - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) )
return false
}
segment . Level = segment . LastLevel
return true
}
}
func RevertSegmentPartitionStatsVersionOperator ( segmentID int64 ) UpdateOperator {
return func ( modPack * updateSegmentPack ) bool {
segment := modPack . Get ( segmentID )
if segment == nil {
log . Warn ( "meta update: revert level fail - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) )
return false
}
segment . PartitionStatsVersion = segment . LastPartitionStatsVersion
return true
}
}
2024-03-20 17:53:14 +08:00
// Add binlogs in segmentInfo
func AddBinlogsOperator ( segmentID int64 , binlogs , statslogs , deltalogs [ ] * datapb . FieldBinlog ) UpdateOperator {
2023-11-21 16:48:22 +08:00
return func ( modPack * updateSegmentPack ) bool {
2023-11-06 15:26:16 +08:00
segment := modPack . Get ( segmentID )
if segment == nil {
2024-03-20 17:53:14 +08:00
log . Warn ( "meta update: add binlog failed - segment not found" ,
2023-11-06 15:26:16 +08:00
zap . Int64 ( "segmentID" , segmentID ) )
return false
2021-10-23 13:31:09 +08:00
}
2023-11-06 15:26:16 +08:00
segment . Binlogs = mergeFieldBinlogs ( segment . GetBinlogs ( ) , binlogs )
segment . Statslogs = mergeFieldBinlogs ( segment . GetStatslogs ( ) , statslogs )
segment . Deltalogs = mergeFieldBinlogs ( segment . GetDeltalogs ( ) , deltalogs )
modPack . increments [ segmentID ] = metastore . BinlogsIncrement {
Segment : segment . SegmentInfo ,
2021-10-23 13:31:09 +08:00
}
2023-11-06 15:26:16 +08:00
return true
2021-10-23 13:31:09 +08:00
}
2023-11-06 15:26:16 +08:00
}
2024-03-20 17:53:14 +08:00
func UpdateBinlogsOperator ( segmentID int64 , binlogs , statslogs , deltalogs [ ] * datapb . FieldBinlog ) UpdateOperator {
2024-03-01 18:31:02 +08:00
return func ( modPack * updateSegmentPack ) bool {
segment := modPack . Get ( segmentID )
if segment == nil {
2024-03-20 17:53:14 +08:00
log . Warn ( "meta update: update binlog failed - segment not found" ,
2024-03-01 18:31:02 +08:00
zap . Int64 ( "segmentID" , segmentID ) )
return false
}
segment . Binlogs = binlogs
segment . Statslogs = statslogs
segment . Deltalogs = deltalogs
modPack . increments [ segmentID ] = metastore . BinlogsIncrement {
Segment : segment . SegmentInfo ,
}
return true
}
}
2023-11-06 15:26:16 +08:00
// update startPosition
func UpdateStartPosition ( startPositions [ ] * datapb . SegmentStartPosition ) UpdateOperator {
2023-11-21 16:48:22 +08:00
return func ( modPack * updateSegmentPack ) bool {
2023-11-06 15:26:16 +08:00
for _ , pos := range startPositions {
if len ( pos . GetStartPosition ( ) . GetMsgID ( ) ) == 0 {
continue
}
s := modPack . Get ( pos . GetSegmentID ( ) )
if s == nil {
continue
}
s . StartPosition = pos . GetStartPosition ( )
2021-06-15 11:06:42 +08:00
}
2023-11-06 15:26:16 +08:00
return true
}
}
2024-03-01 18:31:02 +08:00
func UpdateDmlPosition ( segmentID int64 , dmlPosition * msgpb . MsgPosition ) UpdateOperator {
return func ( modPack * updateSegmentPack ) bool {
if len ( dmlPosition . GetMsgID ( ) ) == 0 {
log . Warn ( "meta update: update dml position failed - nil position msg id" ,
zap . Int64 ( "segmentID" , segmentID ) )
return false
}
segment := modPack . Get ( segmentID )
if segment == nil {
log . Warn ( "meta update: update dml position failed - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) )
return false
}
segment . DmlPosition = dmlPosition
return true
}
}
2024-03-22 15:29:09 +08:00
// UpdateCheckPointOperator updates segment checkpoint and num rows
func UpdateCheckPointOperator ( segmentID int64 , checkpoints [ ] * datapb . CheckPoint ) UpdateOperator {
2023-11-21 16:48:22 +08:00
return func ( modPack * updateSegmentPack ) bool {
2023-11-06 15:26:16 +08:00
segment := modPack . Get ( segmentID )
if segment == nil {
log . Warn ( "meta update: update checkpoint failed - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) )
return false
2021-10-12 23:46:35 +08:00
}
2024-03-22 15:29:09 +08:00
for _ , cp := range checkpoints {
if cp . SegmentID != segmentID {
// Don't think this is gonna to happen, ignore for now.
log . Warn ( "checkpoint in segment is not same as flush segment to update, igreo" , zap . Int64 ( "current" , segmentID ) , zap . Int64 ( "checkpoint segment" , cp . SegmentID ) )
continue
}
2023-11-06 15:26:16 +08:00
2024-03-22 15:29:09 +08:00
if segment . DmlPosition != nil && segment . DmlPosition . Timestamp >= cp . Position . Timestamp {
log . Warn ( "checkpoint in segment is larger than reported" , zap . Any ( "current" , segment . GetDmlPosition ( ) ) , zap . Any ( "reported" , cp . GetPosition ( ) ) )
// segment position in etcd is larger than checkpoint, then dont change it
continue
2023-11-06 15:26:16 +08:00
}
2024-03-22 15:29:09 +08:00
segment . NumOfRows = cp . NumOfRows
segment . DmlPosition = cp . GetPosition ( )
2023-11-06 15:26:16 +08:00
}
2021-10-12 23:46:35 +08:00
2023-11-06 15:26:16 +08:00
count := segmentutil . CalcRowCountFromBinLog ( segment . SegmentInfo )
2023-08-22 14:16:22 +08:00
if count != segment . currRows && count > 0 {
2023-02-27 10:41:46 +08:00
log . Info ( "check point reported inconsistent with bin log row count" ,
zap . Int64 ( "current rows (wrong)" , segment . currRows ) ,
zap . Int64 ( "segment bin log row count (correct)" , count ) )
2023-11-06 15:26:16 +08:00
segment . NumOfRows = count
2021-06-04 11:45:45 +08:00
}
2023-11-06 15:26:16 +08:00
return true
}
}
2021-10-12 23:46:35 +08:00
2024-03-01 18:31:02 +08:00
func UpdateImportedRows ( segmentID int64 , rows int64 ) UpdateOperator {
return func ( modPack * updateSegmentPack ) bool {
segment := modPack . Get ( segmentID )
if segment == nil {
log . Warn ( "meta update: update NumOfRows failed - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) )
return false
}
segment . currRows = rows
segment . NumOfRows = rows
segment . MaxRowNum = rows
return true
}
}
func UpdateIsImporting ( segmentID int64 , isImporting bool ) UpdateOperator {
return func ( modPack * updateSegmentPack ) bool {
segment := modPack . Get ( segmentID )
if segment == nil {
log . Warn ( "meta update: update isImporting failed - segment not found" ,
zap . Int64 ( "segmentID" , segmentID ) )
return false
}
segment . IsImporting = isImporting
return true
}
}
2023-11-06 15:26:16 +08:00
// updateSegmentsInfo update segment infos
// will exec all operators, and update all changed segments
func ( m * meta ) UpdateSegmentsInfo ( operators ... UpdateOperator ) error {
m . Lock ( )
defer m . Unlock ( )
2023-11-21 16:48:22 +08:00
updatePack := & updateSegmentPack {
2023-11-06 15:26:16 +08:00
meta : m ,
segments : make ( map [ int64 ] * SegmentInfo ) ,
increments : make ( map [ int64 ] metastore . BinlogsIncrement ) ,
metricMutation : & segMetricMutation {
2023-11-24 15:58:24 +08:00
stateChange : make ( map [ string ] map [ string ] int ) ,
2023-11-06 15:26:16 +08:00
} ,
}
2023-02-27 10:41:46 +08:00
2023-11-06 15:26:16 +08:00
for _ , operator := range operators {
ok := operator ( updatePack )
if ! ok {
return nil
2023-02-27 10:41:46 +08:00
}
2021-06-15 11:06:42 +08:00
}
2023-11-06 15:26:16 +08:00
segments := lo . MapToSlice ( updatePack . segments , func ( _ int64 , segment * SegmentInfo ) * datapb . SegmentInfo { return segment . SegmentInfo } )
increments := lo . Values ( updatePack . increments )
if err := m . catalog . AlterSegments ( m . ctx , segments , increments ... ) ; err != nil {
2022-09-30 11:48:55 +08:00
log . Error ( "meta update: update flush segments info - failed to store flush segment info into Etcd" ,
zap . Error ( err ) )
2021-05-25 15:35:37 +08:00
return err
}
2022-11-22 19:21:13 +08:00
// Apply metric mutation after a successful meta update.
2023-11-06 15:26:16 +08:00
updatePack . metricMutation . commit ( )
2021-10-12 23:46:35 +08:00
// update memory status
2023-11-06 15:26:16 +08:00
for id , s := range updatePack . segments {
2021-10-12 23:46:35 +08:00
m . segments . SetSegment ( id , s )
}
2023-11-06 15:26:16 +08:00
log . Info ( "meta update: update flush segments info - update flush segments info successfully" )
2021-05-25 15:35:37 +08:00
return nil
}
2021-11-29 22:35:41 +08:00
// UpdateDropChannelSegmentInfo updates segment checkpoints and binlogs before drop
// reusing segment info to pass segment id, binlogs, statslog, deltalog, start position and checkpoint
func ( m * meta ) UpdateDropChannelSegmentInfo ( channel string , segments [ ] * SegmentInfo ) error {
2023-07-14 15:56:31 +08:00
log . Debug ( "meta update: update drop channel segment info" ,
2022-09-30 11:48:55 +08:00
zap . String ( "channel" , channel ) )
2021-11-29 22:35:41 +08:00
m . Lock ( )
defer m . Unlock ( )
2022-11-22 19:21:13 +08:00
// Prepare segment metric mutation.
metricMutation := & segMetricMutation {
2023-11-24 15:58:24 +08:00
stateChange : make ( map [ string ] map [ string ] int ) ,
2022-11-22 19:21:13 +08:00
}
2021-11-29 22:35:41 +08:00
modSegments := make ( map [ UniqueID ] * SegmentInfo )
2022-08-20 10:24:51 +08:00
// save new segments flushed from buffer data
2021-11-29 22:35:41 +08:00
for _ , seg2Drop := range segments {
2022-11-22 19:21:13 +08:00
var segment * SegmentInfo
segment , metricMutation = m . mergeDropSegment ( seg2Drop )
2021-11-29 22:35:41 +08:00
if segment != nil {
modSegments [ seg2Drop . GetID ( ) ] = segment
}
}
2022-08-20 10:24:51 +08:00
// set existed segments of channel to Dropped
2021-12-03 18:55:34 +08:00
for _ , seg := range m . segments . segments {
if seg . InsertChannel != channel {
continue
}
_ , ok := modSegments [ seg . ID ]
// seg inf mod segments are all in dropped state
if ! ok {
clonedSeg := seg . Clone ( )
2022-11-22 19:21:13 +08:00
updateSegStateAndPrepareMetrics ( clonedSeg , commonpb . SegmentState_Dropped , metricMutation )
2021-12-03 18:55:34 +08:00
modSegments [ seg . ID ] = clonedSeg
}
}
2022-03-15 21:51:21 +08:00
err := m . batchSaveDropSegments ( channel , modSegments )
2022-09-30 11:48:55 +08:00
if err != nil {
2023-07-14 15:56:31 +08:00
log . Warn ( "meta update: update drop channel segment info failed" ,
2022-09-30 11:48:55 +08:00
zap . String ( "channel" , channel ) ,
zap . Error ( err ) )
} else {
2022-10-25 19:31:30 +08:00
log . Info ( "meta update: update drop channel segment info - complete" ,
2022-09-30 11:48:55 +08:00
zap . String ( "channel" , channel ) )
2023-02-03 19:33:52 +08:00
// Apply segment metric mutation on successful meta update.
metricMutation . commit ( )
2022-09-30 11:48:55 +08:00
}
2022-03-15 21:51:21 +08:00
return err
2021-11-29 22:35:41 +08:00
}
// mergeDropSegment merges drop segment information with meta segments
2022-11-22 19:21:13 +08:00
func ( m * meta ) mergeDropSegment ( seg2Drop * SegmentInfo ) ( * SegmentInfo , * segMetricMutation ) {
2022-11-24 17:37:13 +08:00
metricMutation := & segMetricMutation {
2023-11-24 15:58:24 +08:00
stateChange : make ( map [ string ] map [ string ] int ) ,
2022-11-24 17:37:13 +08:00
}
2021-11-29 22:35:41 +08:00
segment := m . segments . GetSegment ( seg2Drop . ID )
// healthy check makes sure the Idempotence
if segment == nil || ! isSegmentHealthy ( segment ) {
log . Warn ( "UpdateDropChannel skipping nil or unhealthy" , zap . Bool ( "is nil" , segment == nil ) ,
zap . Bool ( "isHealthy" , isSegmentHealthy ( segment ) ) )
2022-11-24 17:37:13 +08:00
return nil , metricMutation
2021-11-29 22:35:41 +08:00
}
clonedSegment := segment . Clone ( )
2022-11-22 19:21:13 +08:00
updateSegStateAndPrepareMetrics ( clonedSegment , commonpb . SegmentState_Dropped , metricMutation )
2021-11-29 22:35:41 +08:00
currBinlogs := clonedSegment . GetBinlogs ( )
2023-09-21 09:45:27 +08:00
getFieldBinlogs := func ( id UniqueID , binlogs [ ] * datapb . FieldBinlog ) * datapb . FieldBinlog {
2021-11-29 22:35:41 +08:00
for _ , binlog := range binlogs {
if id == binlog . GetFieldID ( ) {
return binlog
}
}
return nil
}
// binlogs
for _ , tBinlogs := range seg2Drop . GetBinlogs ( ) {
fieldBinlogs := getFieldBinlogs ( tBinlogs . GetFieldID ( ) , currBinlogs )
if fieldBinlogs == nil {
currBinlogs = append ( currBinlogs , tBinlogs )
} else {
fieldBinlogs . Binlogs = append ( fieldBinlogs . Binlogs , tBinlogs . Binlogs ... )
}
}
clonedSegment . Binlogs = currBinlogs
// statlogs
currStatsLogs := clonedSegment . GetStatslogs ( )
for _ , tStatsLogs := range seg2Drop . GetStatslogs ( ) {
fieldStatsLog := getFieldBinlogs ( tStatsLogs . GetFieldID ( ) , currStatsLogs )
if fieldStatsLog == nil {
currStatsLogs = append ( currStatsLogs , tStatsLogs )
} else {
fieldStatsLog . Binlogs = append ( fieldStatsLog . Binlogs , tStatsLogs . Binlogs ... )
}
}
clonedSegment . Statslogs = currStatsLogs
// deltalogs
clonedSegment . Deltalogs = append ( clonedSegment . Deltalogs , seg2Drop . GetDeltalogs ( ) ... )
// start position
if seg2Drop . GetStartPosition ( ) != nil {
clonedSegment . StartPosition = seg2Drop . GetStartPosition ( )
}
// checkpoint
if seg2Drop . GetDmlPosition ( ) != nil {
clonedSegment . DmlPosition = seg2Drop . GetDmlPosition ( )
}
clonedSegment . currRows = seg2Drop . currRows
2022-11-22 19:21:13 +08:00
clonedSegment . NumOfRows = seg2Drop . currRows
return clonedSegment , metricMutation
2021-11-29 22:35:41 +08:00
}
// batchSaveDropSegments saves drop segments info with channel removal flag
// since the channel unwatching operation is not atomic here
// ** the removal flag is always with last batch
// ** the last batch must contains at least one segment
2023-03-03 14:13:49 +08:00
// 1. when failure occurs between batches, failover mechanism will continue with the earliest checkpoint of this channel
2022-11-03 12:15:35 +08:00
// since the flag is not marked so DataNode can re-consume the drop collection msg
// 2. when failure occurs between save meta and unwatch channel, the removal flag shall be check before let datanode watch this channel
2021-11-29 22:35:41 +08:00
func ( m * meta ) batchSaveDropSegments ( channel string , modSegments map [ int64 ] * SegmentInfo ) error {
2022-11-03 12:15:35 +08:00
var modSegIDs [ ] int64
for k := range modSegments {
modSegIDs = append ( modSegIDs , k )
}
log . Info ( "meta update: batch save drop segments" ,
zap . Int64s ( "drop segments" , modSegIDs ) )
2022-08-25 16:48:53 +08:00
segments := make ( [ ] * datapb . SegmentInfo , 0 )
for _ , seg := range modSegments {
segments = append ( segments , seg . SegmentInfo )
}
err := m . catalog . SaveDroppedSegmentsInBatch ( m . ctx , segments )
2021-11-29 22:35:41 +08:00
if err != nil {
return err
}
2022-08-25 16:48:53 +08:00
if err = m . catalog . MarkChannelDeleted ( m . ctx , channel ) ; err != nil {
return err
2022-08-20 10:24:51 +08:00
}
2021-11-29 22:35:41 +08:00
// update memory info
2022-08-25 16:48:53 +08:00
for id , segment := range modSegments {
m . segments . SetSegment ( id , segment )
2021-11-29 22:35:41 +08:00
}
2022-08-20 10:24:51 +08:00
return nil
2021-12-02 16:39:33 +08:00
}
2021-09-08 11:35:59 +08:00
// GetSegmentsByChannel returns all segment info which insert channel equals provided `dmlCh`
2023-11-23 17:30:25 +08:00
func ( m * meta ) GetSegmentsByChannel ( channel string ) [ ] * SegmentInfo {
2024-05-08 21:37:29 +08:00
return m . SelectSegments ( SegmentFilterFunc ( isSegmentHealthy ) , WithChannel ( channel ) )
2021-05-31 18:47:32 +08:00
}
2021-11-05 22:25:00 +08:00
// GetSegmentsOfCollection get all segments of collection
func ( m * meta ) GetSegmentsOfCollection ( collectionID UniqueID ) [ ] * SegmentInfo {
2024-05-08 21:37:29 +08:00
return m . SelectSegments ( SegmentFilterFunc ( isSegmentHealthy ) , WithCollection ( collectionID ) )
2021-11-05 22:25:00 +08:00
}
// GetSegmentsIDOfCollection returns all segment ids which collection equals to provided `collectionID`
func ( m * meta ) GetSegmentsIDOfCollection ( collectionID UniqueID ) [ ] UniqueID {
2024-05-08 21:37:29 +08:00
segments := m . SelectSegments ( SegmentFilterFunc ( isSegmentHealthy ) , WithCollection ( collectionID ) )
2023-11-23 17:30:25 +08:00
return lo . Map ( segments , func ( segment * SegmentInfo , _ int ) int64 {
return segment . ID
} )
2021-01-19 15:35:40 +08:00
}
2022-09-16 11:32:48 +08:00
// GetSegmentsIDOfCollection returns all segment ids which collection equals to provided `collectionID`
func ( m * meta ) GetSegmentsIDOfCollectionWithDropped ( collectionID UniqueID ) [ ] UniqueID {
2024-05-08 21:37:29 +08:00
segments := m . SelectSegments ( WithCollection ( collectionID ) , SegmentFilterFunc ( func ( segment * SegmentInfo ) bool {
2023-11-23 17:30:25 +08:00
return segment != nil &&
2022-09-16 11:32:48 +08:00
segment . GetState ( ) != commonpb . SegmentState_SegmentStateNone &&
2024-05-08 21:37:29 +08:00
segment . GetState ( ) != commonpb . SegmentState_NotExist
} ) )
2023-11-23 17:30:25 +08:00
return lo . Map ( segments , func ( segment * SegmentInfo , _ int ) int64 {
return segment . ID
} )
2022-09-16 11:32:48 +08:00
}
2021-11-05 22:25:00 +08:00
// GetSegmentsIDOfPartition returns all segments ids which collection & partition equals to provided `collectionID`, `partitionID`
func ( m * meta ) GetSegmentsIDOfPartition ( collectionID , partitionID UniqueID ) [ ] UniqueID {
2024-05-08 21:37:29 +08:00
segments := m . SelectSegments ( WithCollection ( collectionID ) , SegmentFilterFunc ( func ( segment * SegmentInfo ) bool {
2023-11-23 17:30:25 +08:00
return isSegmentHealthy ( segment ) &&
segment . PartitionID == partitionID
2024-05-08 21:37:29 +08:00
} ) )
2023-11-23 17:30:25 +08:00
return lo . Map ( segments , func ( segment * SegmentInfo , _ int ) int64 {
return segment . ID
} )
2021-01-19 15:35:40 +08:00
}
2022-09-16 11:32:48 +08:00
// GetSegmentsIDOfPartition returns all segments ids which collection & partition equals to provided `collectionID`, `partitionID`
func ( m * meta ) GetSegmentsIDOfPartitionWithDropped ( collectionID , partitionID UniqueID ) [ ] UniqueID {
2024-05-08 21:37:29 +08:00
segments := m . SelectSegments ( WithCollection ( collectionID ) , SegmentFilterFunc ( func ( segment * SegmentInfo ) bool {
2023-11-23 17:30:25 +08:00
return segment . GetState ( ) != commonpb . SegmentState_SegmentStateNone &&
2022-09-16 11:32:48 +08:00
segment . GetState ( ) != commonpb . SegmentState_NotExist &&
2023-11-23 17:30:25 +08:00
segment . PartitionID == partitionID
2024-05-08 21:37:29 +08:00
} ) )
2023-11-23 17:30:25 +08:00
return lo . Map ( segments , func ( segment * SegmentInfo , _ int ) int64 {
return segment . ID
} )
2022-09-16 11:32:48 +08:00
}
2021-09-08 11:35:59 +08:00
// GetNumRowsOfPartition returns row count of segments belongs to provided collection & partition
2021-07-07 14:02:01 +08:00
func ( m * meta ) GetNumRowsOfPartition ( collectionID UniqueID , partitionID UniqueID ) int64 {
2021-12-14 15:31:07 +08:00
var ret int64
2024-05-08 21:37:29 +08:00
segments := m . SelectSegments ( WithCollection ( collectionID ) , SegmentFilterFunc ( func ( si * SegmentInfo ) bool {
return isSegmentHealthy ( si ) && si . GetPartitionID ( ) == partitionID
} ) )
2021-11-12 00:22:42 +08:00
for _ , segment := range segments {
2024-05-08 21:37:29 +08:00
ret += segment . NumOfRows
2021-02-02 14:25:58 +08:00
}
2021-07-07 14:02:01 +08:00
return ret
2021-02-02 14:25:58 +08:00
}
2021-09-08 11:35:59 +08:00
// GetUnFlushedSegments get all segments which state is not `Flushing` nor `Flushed`
2021-07-12 17:24:25 +08:00
func ( m * meta ) GetUnFlushedSegments ( ) [ ] * SegmentInfo {
2024-05-08 21:37:29 +08:00
return m . SelectSegments ( SegmentFilterFunc ( func ( segment * SegmentInfo ) bool {
2023-11-23 17:30:25 +08:00
return segment . GetState ( ) == commonpb . SegmentState_Growing || segment . GetState ( ) == commonpb . SegmentState_Sealed
2024-05-08 21:37:29 +08:00
} ) )
2021-05-28 09:55:21 +08:00
}
2021-09-08 11:35:59 +08:00
// GetFlushingSegments get all segments which state is `Flushing`
2021-07-12 17:24:25 +08:00
func ( m * meta ) GetFlushingSegments ( ) [ ] * SegmentInfo {
2024-05-08 21:37:29 +08:00
return m . SelectSegments ( SegmentFilterFunc ( func ( segment * SegmentInfo ) bool {
2023-11-23 17:30:25 +08:00
return segment . GetState ( ) == commonpb . SegmentState_Flushing
2024-05-08 21:37:29 +08:00
} ) )
2021-05-21 18:30:41 +08:00
}
2021-10-20 15:02:36 +08:00
// SelectSegments select segments with selector
2024-05-08 21:37:29 +08:00
func ( m * meta ) SelectSegments ( filters ... SegmentFilter ) [ ] * SegmentInfo {
2021-10-20 15:02:36 +08:00
m . RLock ( )
defer m . RUnlock ( )
2024-05-08 21:37:29 +08:00
return m . segments . GetSegmentsBySelector ( filters ... )
2021-10-20 15:02:36 +08:00
}
2024-06-03 14:47:47 +08:00
func ( m * meta ) GetRealSegmentsForChannel ( channel string ) [ ] * SegmentInfo {
m . RLock ( )
defer m . RUnlock ( )
return m . segments . GetRealSegmentsForChannel ( channel )
}
2021-09-08 11:35:59 +08:00
// AddAllocation add allocation in segment
2021-07-12 17:24:25 +08:00
func ( m * meta ) AddAllocation ( segmentID UniqueID , allocation * Allocation ) error {
2023-07-14 15:56:31 +08:00
log . Debug ( "meta update: add allocation" ,
2022-09-30 11:48:55 +08:00
zap . Int64 ( "segmentID" , segmentID ) ,
zap . Any ( "allocation" , allocation ) )
2021-07-12 17:24:25 +08:00
m . Lock ( )
defer m . Unlock ( )
2022-09-26 18:06:54 +08:00
curSegInfo := m . segments . GetSegment ( segmentID )
if curSegInfo == nil {
// TODO: Error handling.
2023-09-18 09:53:22 +08:00
log . Error ( "meta update: add allocation failed - segment not found" , zap . Int64 ( "segmentID" , segmentID ) )
return errors . New ( "meta update: add allocation failed - segment not found" )
2022-09-26 18:06:54 +08:00
}
2023-07-24 14:11:07 +08:00
// As we use global segment lastExpire to guarantee data correctness after restart
// there is no need to persist allocation to meta store, only update allocation in-memory meta.
2022-09-26 18:06:54 +08:00
m . segments . AddAllocation ( segmentID , allocation )
2023-07-24 14:11:07 +08:00
log . Info ( "meta update: add allocation - complete" , zap . Int64 ( "segmentID" , segmentID ) )
2021-07-12 17:24:25 +08:00
return nil
}
2021-09-08 11:35:59 +08:00
// SetAllocations set Segment allocations, will overwrite ALL original allocations
// Note that allocations is not persisted in KV store
2021-07-12 17:24:25 +08:00
func ( m * meta ) SetAllocations ( segmentID UniqueID , allocations [ ] * Allocation ) {
m . Lock ( )
defer m . Unlock ( )
m . segments . SetAllocations ( segmentID , allocations )
}
2021-09-08 11:35:59 +08:00
// SetCurrentRows set current row count for segment with provided `segmentID`
// Note that currRows is not persisted in KV store
2021-07-12 17:24:25 +08:00
func ( m * meta ) SetCurrentRows ( segmentID UniqueID , rows int64 ) {
m . Lock ( )
defer m . Unlock ( )
m . segments . SetCurrentRows ( segmentID , rows )
}
2023-07-24 14:11:07 +08:00
// SetLastExpire set lastExpire time for segment
// Note that last is not necessary to store in KV meta
func ( m * meta ) SetLastExpire ( segmentID UniqueID , lastExpire uint64 ) {
m . Lock ( )
defer m . Unlock ( )
clonedSegment := m . segments . GetSegment ( segmentID ) . Clone ( )
clonedSegment . LastExpireTime = lastExpire
m . segments . SetSegment ( segmentID , clonedSegment )
}
2021-09-08 11:35:59 +08:00
// SetLastFlushTime set LastFlushTime for segment with provided `segmentID`
// Note that lastFlushTime is not persisted in KV store
2021-08-19 14:08:10 +08:00
func ( m * meta ) SetLastFlushTime ( segmentID UniqueID , t time . Time ) {
m . Lock ( )
defer m . Unlock ( )
m . segments . SetFlushTime ( segmentID , t )
}
2021-11-05 22:25:00 +08:00
// SetSegmentCompacting sets compaction state for segment
func ( m * meta ) SetSegmentCompacting ( segmentID UniqueID , compacting bool ) {
m . Lock ( )
defer m . Unlock ( )
m . segments . SetIsCompacting ( segmentID , compacting )
}
2024-06-05 10:17:50 +08:00
// CheckAndSetSegmentsCompacting check all segments are not compacting
// if true, set them compacting and return true
// if false, skip setting and
func ( m * meta ) CheckAndSetSegmentsCompacting ( segmentIDs [ ] UniqueID ) ( exist , hasCompactingSegment bool ) {
m . Lock ( )
defer m . Unlock ( )
for _ , segmentID := range segmentIDs {
seg := m . segments . GetSegment ( segmentID )
if seg != nil {
hasCompactingSegment = seg . isCompacting
} else {
return false , false
}
}
if hasCompactingSegment {
return true , false
}
for _ , segmentID := range segmentIDs {
m . segments . SetIsCompacting ( segmentID , true )
}
return true , true
}
func ( m * meta ) SetSegmentsCompacting ( segmentIDs [ ] UniqueID , compacting bool ) {
m . Lock ( )
defer m . Unlock ( )
for _ , segmentID := range segmentIDs {
m . segments . SetIsCompacting ( segmentID , compacting )
}
}
2024-06-10 21:34:08 +08:00
// SetSegmentLevel sets level for segment
func ( m * meta ) SetSegmentLevel ( segmentID UniqueID , level datapb . SegmentLevel ) {
m . Lock ( )
defer m . Unlock ( )
m . segments . SetLevel ( segmentID , level )
}
2024-02-28 19:03:01 +08:00
func ( m * meta ) CompleteCompactionMutation ( plan * datapb . CompactionPlan , result * datapb . CompactionPlanResult ) ( [ ] * SegmentInfo , * segMetricMutation , error ) {
2024-01-26 16:59:00 +08:00
m . Lock ( )
defer m . Unlock ( )
2021-11-05 22:25:00 +08:00
2024-02-28 19:03:01 +08:00
log := log . With ( zap . Int64 ( "planID" , plan . GetPlanID ( ) ) , zap . String ( "type" , plan . GetType ( ) . String ( ) ) )
2022-09-27 16:02:53 +08:00
2024-02-28 19:03:01 +08:00
metricMutation := & segMetricMutation { stateChange : make ( map [ string ] map [ string ] int ) }
var compactFromSegIDs [ ] int64
var latestCompactFromSegments [ ] * SegmentInfo
for _ , segmentBinlogs := range plan . GetSegmentBinlogs ( ) {
segment := m . segments . GetSegment ( segmentBinlogs . GetSegmentID ( ) )
if segment == nil {
return nil , nil , merr . WrapErrSegmentNotFound ( segmentBinlogs . GetSegmentID ( ) )
2021-11-05 22:25:00 +08:00
}
2024-02-28 19:03:01 +08:00
cloned := segment . Clone ( )
cloned . DroppedAt = uint64 ( time . Now ( ) . UnixNano ( ) )
cloned . Compacted = true
2021-11-25 17:19:17 +08:00
2024-02-28 19:03:01 +08:00
latestCompactFromSegments = append ( latestCompactFromSegments , cloned )
compactFromSegIDs = append ( compactFromSegIDs , cloned . GetID ( ) )
2021-11-05 22:25:00 +08:00
2024-02-28 19:03:01 +08:00
// metrics mutation for compaction from segments
updateSegStateAndPrepareMetrics ( cloned , commonpb . SegmentState_Dropped , metricMutation )
2021-11-05 22:25:00 +08:00
}
2024-02-28 19:03:01 +08:00
logIDsFromPlan := make ( map [ int64 ] struct { } )
for _ , segBinlogs := range plan . GetSegmentBinlogs ( ) {
for _ , fieldBinlog := range segBinlogs . GetDeltalogs ( ) {
for _ , binlog := range fieldBinlog . GetBinlogs ( ) {
logIDsFromPlan [ binlog . GetLogID ( ) ] = struct { } { }
}
}
2021-11-05 22:25:00 +08:00
}
2024-06-10 21:34:08 +08:00
getMinPosition := func ( positions [ ] * msgpb . MsgPosition ) * msgpb . MsgPosition {
var minPos * msgpb . MsgPosition
for _ , pos := range positions {
if minPos == nil ||
pos != nil && pos . GetTimestamp ( ) < minPos . GetTimestamp ( ) {
minPos = pos
}
}
return minPos
}
if plan . GetType ( ) == datapb . CompactionType_ClusteringCompaction {
newSegments := make ( [ ] * SegmentInfo , 0 )
for _ , seg := range result . GetSegments ( ) {
segmentInfo := & datapb . SegmentInfo {
ID : seg . GetSegmentID ( ) ,
CollectionID : latestCompactFromSegments [ 0 ] . CollectionID ,
PartitionID : latestCompactFromSegments [ 0 ] . PartitionID ,
InsertChannel : plan . GetChannel ( ) ,
NumOfRows : seg . NumOfRows ,
State : commonpb . SegmentState_Flushed ,
MaxRowNum : latestCompactFromSegments [ 0 ] . MaxRowNum ,
Binlogs : seg . GetInsertLogs ( ) ,
Statslogs : seg . GetField2StatslogPaths ( ) ,
CreatedByCompaction : true ,
CompactionFrom : compactFromSegIDs ,
LastExpireTime : tsoutil . ComposeTSByTime ( time . Unix ( plan . GetStartTime ( ) , 0 ) , 0 ) ,
Level : datapb . SegmentLevel_L2 ,
StartPosition : getMinPosition ( lo . Map ( latestCompactFromSegments , func ( info * SegmentInfo , _ int ) * msgpb . MsgPosition {
return info . GetStartPosition ( )
} ) ) ,
DmlPosition : getMinPosition ( lo . Map ( latestCompactFromSegments , func ( info * SegmentInfo , _ int ) * msgpb . MsgPosition {
return info . GetDmlPosition ( )
} ) ) ,
}
segment := NewSegmentInfo ( segmentInfo )
newSegments = append ( newSegments , segment )
metricMutation . addNewSeg ( segment . GetState ( ) , segment . GetLevel ( ) , segment . GetNumOfRows ( ) )
}
compactionTo := make ( [ ] UniqueID , 0 , len ( newSegments ) )
for _ , s := range newSegments {
compactionTo = append ( compactionTo , s . GetID ( ) )
}
log . Info ( "meta update: prepare for complete compaction mutation - complete" ,
zap . Int64 ( "collectionID" , latestCompactFromSegments [ 0 ] . CollectionID ) ,
zap . Int64 ( "partitionID" , latestCompactFromSegments [ 0 ] . PartitionID ) ,
zap . Any ( "compacted from" , compactFromSegIDs ) ,
zap . Any ( "compacted to" , compactionTo ) )
compactFromInfos := lo . Map ( latestCompactFromSegments , func ( info * SegmentInfo , _ int ) * datapb . SegmentInfo {
return info . SegmentInfo
} )
newSegmentInfos := lo . Map ( newSegments , func ( info * SegmentInfo , _ int ) * datapb . SegmentInfo {
return info . SegmentInfo
} )
binlogs := make ( [ ] metastore . BinlogsIncrement , 0 )
for _ , seg := range newSegmentInfos {
binlogs = append ( binlogs , metastore . BinlogsIncrement { Segment : seg } )
}
if err := m . catalog . AlterSegments ( m . ctx , append ( compactFromInfos , newSegmentInfos ... ) , binlogs ... ) ; err != nil {
log . Warn ( "fail to alter segments and new segment" , zap . Error ( err ) )
return nil , nil , err
}
lo . ForEach ( latestCompactFromSegments , func ( info * SegmentInfo , _ int ) {
m . segments . SetSegment ( info . GetID ( ) , info )
} )
lo . ForEach ( newSegments , func ( info * SegmentInfo , _ int ) {
m . segments . SetSegment ( info . GetID ( ) , info )
} )
return newSegments , metricMutation , nil
}
2023-11-14 15:56:19 +08:00
// MixCompaction / MergeCompaction will generates one and only one segment
compactToSegment := result . GetSegments ( ) [ 0 ]
2024-02-28 19:03:01 +08:00
// copy new deltalogs in compactFrom segments to compactTo segments.
// TODO: Not needed when enable L0 segments.
newDeltalogs , err := m . copyNewDeltalogs ( latestCompactFromSegments , logIDsFromPlan , compactToSegment . GetSegmentID ( ) )
2022-11-17 20:37:10 +08:00
if err != nil {
2024-02-28 19:03:01 +08:00
return nil , nil , err
}
if len ( newDeltalogs ) > 0 {
compactToSegment . Deltalogs = append ( compactToSegment . GetDeltalogs ( ) , & datapb . FieldBinlog { Binlogs : newDeltalogs } )
2024-01-24 14:23:00 +08:00
}
2024-02-28 19:03:01 +08:00
compactToSegmentInfo := NewSegmentInfo (
& datapb . SegmentInfo {
ID : compactToSegment . GetSegmentID ( ) ,
CollectionID : latestCompactFromSegments [ 0 ] . CollectionID ,
PartitionID : latestCompactFromSegments [ 0 ] . PartitionID ,
InsertChannel : plan . GetChannel ( ) ,
NumOfRows : compactToSegment . NumOfRows ,
State : commonpb . SegmentState_Flushed ,
MaxRowNum : latestCompactFromSegments [ 0 ] . MaxRowNum ,
Binlogs : compactToSegment . GetInsertLogs ( ) ,
Statslogs : compactToSegment . GetField2StatslogPaths ( ) ,
Deltalogs : compactToSegment . GetDeltalogs ( ) ,
CreatedByCompaction : true ,
CompactionFrom : compactFromSegIDs ,
2024-06-05 10:17:50 +08:00
LastExpireTime : tsoutil . ComposeTSByTime ( time . Unix ( plan . GetStartTime ( ) , 0 ) , 0 ) ,
2024-02-28 19:03:01 +08:00
Level : datapb . SegmentLevel_L1 ,
StartPosition : getMinPosition ( lo . Map ( latestCompactFromSegments , func ( info * SegmentInfo , _ int ) * msgpb . MsgPosition {
return info . GetStartPosition ( )
} ) ) ,
DmlPosition : getMinPosition ( lo . Map ( latestCompactFromSegments , func ( info * SegmentInfo , _ int ) * msgpb . MsgPosition {
return info . GetDmlPosition ( )
} ) ) ,
} )
2021-11-05 22:25:00 +08:00
2024-02-28 19:03:01 +08:00
// L1 segment with NumRows=0 will be discarded, so no need to change the metric
if compactToSegmentInfo . GetNumOfRows ( ) > 0 {
// metrics mutation for compactTo segments
metricMutation . addNewSeg ( compactToSegmentInfo . GetState ( ) , compactToSegmentInfo . GetLevel ( ) , compactToSegmentInfo . GetNumOfRows ( ) )
} else {
compactToSegmentInfo . State = commonpb . SegmentState_Dropped
2023-03-17 17:27:56 +08:00
}
2023-07-13 14:08:29 +08:00
2024-02-28 19:03:01 +08:00
log = log . With (
zap . String ( "channel" , plan . GetChannel ( ) ) ,
zap . Int64 ( "partitionID" , compactToSegmentInfo . GetPartitionID ( ) ) ,
zap . Int64 ( "compactTo segmentID" , compactToSegmentInfo . GetID ( ) ) ,
zap . Int64 ( "compactTo segment numRows" , compactToSegmentInfo . GetNumOfRows ( ) ) ,
zap . Any ( "compactFrom segments(to be updated as dropped)" , compactFromSegIDs ) ,
)
2023-03-03 14:13:49 +08:00
2024-02-28 19:03:01 +08:00
log . Debug ( "meta update: prepare for meta mutation - complete" )
compactFromInfos := lo . Map ( latestCompactFromSegments , func ( info * SegmentInfo , _ int ) * datapb . SegmentInfo {
return info . SegmentInfo
} )
2024-02-19 20:52:50 +08:00
2024-02-28 19:03:01 +08:00
log . Debug ( "meta update: alter meta store for compaction updates" ,
zap . Int ( "binlog count" , len ( compactToSegmentInfo . GetBinlogs ( ) ) ) ,
zap . Int ( "statslog count" , len ( compactToSegmentInfo . GetStatslogs ( ) ) ) ,
zap . Int ( "deltalog count" , len ( compactToSegmentInfo . GetDeltalogs ( ) ) ) ,
)
if err := m . catalog . AlterSegments ( m . ctx , append ( compactFromInfos , compactToSegmentInfo . SegmentInfo ) ,
metastore . BinlogsIncrement { Segment : compactToSegmentInfo . SegmentInfo } ,
) ; err != nil {
2023-03-17 17:27:56 +08:00
log . Warn ( "fail to alter segments and new segment" , zap . Error ( err ) )
2024-02-28 19:03:01 +08:00
return nil , nil , err
2022-11-24 10:19:12 +08:00
}
2024-02-28 19:03:01 +08:00
lo . ForEach ( latestCompactFromSegments , func ( info * SegmentInfo , _ int ) {
m . segments . SetSegment ( info . GetID ( ) , info )
} )
m . segments . SetSegment ( compactToSegmentInfo . GetID ( ) , compactToSegmentInfo )
2024-02-19 20:52:50 +08:00
2024-02-28 19:03:01 +08:00
log . Info ( "meta update: alter in memory meta after compaction - complete" )
return [ ] * SegmentInfo { compactToSegmentInfo } , metricMutation , nil
2021-11-05 22:25:00 +08:00
}
2024-02-28 19:03:01 +08:00
func ( m * meta ) copyNewDeltalogs ( latestCompactFromInfos [ ] * SegmentInfo , logIDsInPlan map [ int64 ] struct { } , toSegment int64 ) ( [ ] * datapb . Binlog , error ) {
newBinlogs := [ ] * datapb . Binlog { }
for _ , seg := range latestCompactFromInfos {
for _ , fieldLog := range seg . GetDeltalogs ( ) {
for _ , l := range fieldLog . GetBinlogs ( ) {
if _ , ok := logIDsInPlan [ l . GetLogID ( ) ] ; ! ok {
fromKey := metautil . BuildDeltaLogPath ( m . chunkManager . RootPath ( ) , seg . CollectionID , seg . PartitionID , seg . ID , l . GetLogID ( ) )
toKey := metautil . BuildDeltaLogPath ( m . chunkManager . RootPath ( ) , seg . CollectionID , seg . PartitionID , toSegment , l . GetLogID ( ) )
log . Warn ( "found new deltalog in compactFrom segment, copying it..." ,
zap . Any ( "deltalog" , l ) ,
zap . Int64 ( "copyFrom segmentID" , seg . GetID ( ) ) ,
zap . Int64 ( "copyTo segmentID" , toSegment ) ,
zap . String ( "copyFrom key" , fromKey ) ,
zap . String ( "copyTo key" , toKey ) ,
)
blob , err := m . chunkManager . Read ( m . ctx , fromKey )
if err != nil {
return nil , err
}
if err := m . chunkManager . Write ( m . ctx , toKey , blob ) ; err != nil {
return nil , err
}
newBinlogs = append ( newBinlogs , l )
2021-12-19 20:00:42 +08:00
}
}
}
2021-11-05 22:25:00 +08:00
}
2024-02-28 19:03:01 +08:00
return newBinlogs , nil
2021-11-05 22:25:00 +08:00
}
2021-09-08 11:35:59 +08:00
// buildSegment utility function for compose datapb.SegmentInfo struct with provided info
2024-03-22 15:29:09 +08:00
func buildSegment ( collectionID UniqueID , partitionID UniqueID , segmentID UniqueID , channelName string ) * SegmentInfo {
2021-07-12 17:24:25 +08:00
info := & datapb . SegmentInfo {
2021-04-09 09:55:04 +08:00
ID : segmentID ,
2021-02-02 18:53:10 +08:00
CollectionID : collectionID ,
PartitionID : partitionID ,
InsertChannel : channelName ,
2021-06-04 11:45:45 +08:00
NumOfRows : 0 ,
2021-03-11 14:14:29 +08:00
State : commonpb . SegmentState_Growing ,
2021-07-07 14:02:01 +08:00
}
2021-07-12 17:24:25 +08:00
return NewSegmentInfo ( info )
2021-01-25 15:17:17 +08:00
}
2021-11-12 00:22:42 +08:00
func isSegmentHealthy ( segment * SegmentInfo ) bool {
2022-06-29 16:54:17 +08:00
return segment != nil &&
segment . GetState ( ) != commonpb . SegmentState_SegmentStateNone &&
2021-11-29 22:35:41 +08:00
segment . GetState ( ) != commonpb . SegmentState_NotExist &&
2021-11-12 00:22:42 +08:00
segment . GetState ( ) != commonpb . SegmentState_Dropped
}
2022-05-31 16:36:03 +08:00
func ( m * meta ) HasSegments ( segIDs [ ] UniqueID ) ( bool , error ) {
m . RLock ( )
defer m . RUnlock ( )
for _ , segID := range segIDs {
if _ , ok := m . segments . segments [ segID ] ; ! ok {
return false , fmt . Errorf ( "segment is not exist with ID = %d" , segID )
}
}
return true , nil
}
2022-11-01 21:01:34 +08:00
2024-03-05 10:04:59 +08:00
// GetCompactionTo returns the segment info of the segment to be compacted to.
func ( m * meta ) GetCompactionTo ( segmentID int64 ) ( * SegmentInfo , bool ) {
2022-11-01 21:01:34 +08:00
m . RLock ( )
defer m . RUnlock ( )
2024-03-05 10:04:59 +08:00
return m . segments . GetCompactionTo ( segmentID )
2022-11-01 21:01:34 +08:00
}
2022-11-10 22:13:04 +08:00
// UpdateChannelCheckpoint updates and saves channel checkpoint.
2023-03-04 23:21:50 +08:00
func ( m * meta ) UpdateChannelCheckpoint ( vChannel string , pos * msgpb . MsgPosition ) error {
2023-07-25 10:43:04 +08:00
if pos == nil || pos . GetMsgID ( ) == nil {
2022-11-10 22:13:04 +08:00
return fmt . Errorf ( "channelCP is nil, vChannel=%s" , vChannel )
}
2024-03-07 20:39:02 +08:00
m . channelCPs . Lock ( )
defer m . channelCPs . Unlock ( )
2022-11-12 21:09:04 +08:00
2024-03-07 20:39:02 +08:00
oldPosition , ok := m . channelCPs . checkpoints [ vChannel ]
2022-11-10 22:13:04 +08:00
if ! ok || oldPosition . Timestamp < pos . Timestamp {
err := m . catalog . SaveChannelCheckpoint ( m . ctx , vChannel , pos )
if err != nil {
return err
}
2024-03-07 20:39:02 +08:00
m . channelCPs . checkpoints [ vChannel ] = pos
2022-11-10 22:13:04 +08:00
ts , _ := tsoutil . ParseTS ( pos . Timestamp )
2023-07-25 10:43:04 +08:00
log . Info ( "UpdateChannelCheckpoint done" ,
2023-03-09 14:13:52 +08:00
zap . String ( "vChannel" , vChannel ) ,
2023-07-25 10:43:04 +08:00
zap . Uint64 ( "ts" , pos . GetTimestamp ( ) ) ,
zap . ByteString ( "msgID" , pos . GetMsgID ( ) ) ,
2023-03-09 14:13:52 +08:00
zap . Time ( "time" , ts ) )
2023-12-20 11:04:45 +08:00
metrics . DataCoordCheckpointUnixSeconds . WithLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) , vChannel ) .
Set ( float64 ( ts . Unix ( ) ) )
2022-11-10 22:13:04 +08:00
}
return nil
}
2024-05-21 11:59:39 +08:00
// MarkChannelCheckpointDropped set channel checkpoint to MaxUint64 preventing future update
// and remove the metrics for channel checkpoint lag.
func ( m * meta ) MarkChannelCheckpointDropped ( ctx context . Context , channel string ) error {
m . channelCPs . Lock ( )
defer m . channelCPs . Unlock ( )
cp := & msgpb . MsgPosition {
ChannelName : channel ,
Timestamp : math . MaxUint64 ,
}
err := m . catalog . SaveChannelCheckpoints ( ctx , [ ] * msgpb . MsgPosition { cp } )
if err != nil {
return err
}
m . channelCPs . checkpoints [ channel ] = cp
metrics . DataCoordCheckpointUnixSeconds . DeleteLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) , channel )
return nil
}
2024-03-07 20:39:02 +08:00
// UpdateChannelCheckpoints updates and saves channel checkpoints.
func ( m * meta ) UpdateChannelCheckpoints ( positions [ ] * msgpb . MsgPosition ) error {
m . channelCPs . Lock ( )
defer m . channelCPs . Unlock ( )
toUpdates := lo . Filter ( positions , func ( pos * msgpb . MsgPosition , _ int ) bool {
if pos == nil || pos . GetMsgID ( ) == nil || pos . GetChannelName ( ) == "" {
log . Warn ( "illegal channel cp" , zap . Any ( "pos" , pos ) )
return false
}
vChannel := pos . GetChannelName ( )
oldPosition , ok := m . channelCPs . checkpoints [ vChannel ]
return ! ok || oldPosition . Timestamp < pos . Timestamp
} )
err := m . catalog . SaveChannelCheckpoints ( m . ctx , toUpdates )
if err != nil {
return err
}
for _ , pos := range toUpdates {
channel := pos . GetChannelName ( )
m . channelCPs . checkpoints [ channel ] = pos
log . Info ( "UpdateChannelCheckpoint done" , zap . String ( "channel" , channel ) ,
zap . Uint64 ( "ts" , pos . GetTimestamp ( ) ) ,
zap . Time ( "time" , tsoutil . PhysicalTime ( pos . GetTimestamp ( ) ) ) )
ts , _ := tsoutil . ParseTS ( pos . Timestamp )
metrics . DataCoordCheckpointUnixSeconds . WithLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) , channel ) . Set ( float64 ( ts . Unix ( ) ) )
}
return nil
}
2023-03-04 23:21:50 +08:00
func ( m * meta ) GetChannelCheckpoint ( vChannel string ) * msgpb . MsgPosition {
2024-03-07 20:39:02 +08:00
m . channelCPs . RLock ( )
defer m . channelCPs . RUnlock ( )
cp , ok := m . channelCPs . checkpoints [ vChannel ]
2023-11-30 11:00:28 +08:00
if ! ok {
2022-11-10 22:13:04 +08:00
return nil
}
2024-03-07 20:39:02 +08:00
return proto . Clone ( cp ) . ( * msgpb . MsgPosition )
2022-11-10 22:13:04 +08:00
}
func ( m * meta ) DropChannelCheckpoint ( vChannel string ) error {
2024-03-07 20:39:02 +08:00
m . channelCPs . Lock ( )
defer m . channelCPs . Unlock ( )
2022-11-10 22:13:04 +08:00
err := m . catalog . DropChannelCheckpoint ( m . ctx , vChannel )
if err != nil {
return err
}
2024-03-07 20:39:02 +08:00
delete ( m . channelCPs . checkpoints , vChannel )
2024-05-08 14:17:50 +08:00
log . Info ( "DropChannelCheckpoint done" , zap . String ( "vChannel" , vChannel ) )
2022-11-10 22:13:04 +08:00
return nil
}
2022-11-22 19:21:13 +08:00
2023-01-12 09:55:42 +08:00
func ( m * meta ) GcConfirm ( ctx context . Context , collectionID , partitionID UniqueID ) bool {
return m . catalog . GcConfirm ( ctx , collectionID , partitionID )
}
2023-11-23 17:30:25 +08:00
func ( m * meta ) GetCompactableSegmentGroupByCollection ( ) map [ int64 ] [ ] * SegmentInfo {
2024-05-08 21:37:29 +08:00
allSegs := m . SelectSegments ( SegmentFilterFunc ( func ( segment * SegmentInfo ) bool {
2023-11-23 17:30:25 +08:00
return isSegmentHealthy ( segment ) &&
isFlush ( segment ) && // sealed segment
! segment . isCompacting && // not compacting now
! segment . GetIsImporting ( ) // not importing now
2024-05-08 21:37:29 +08:00
} ) )
2023-11-23 17:30:25 +08:00
ret := make ( map [ int64 ] [ ] * SegmentInfo )
for _ , seg := range allSegs {
if _ , ok := ret [ seg . CollectionID ] ; ! ok {
ret [ seg . CollectionID ] = make ( [ ] * SegmentInfo , 0 )
}
ret [ seg . CollectionID ] = append ( ret [ seg . CollectionID ] , seg )
}
return ret
}
func ( m * meta ) GetEarliestStartPositionOfGrowingSegments ( label * CompactionGroupLabel ) * msgpb . MsgPosition {
2024-05-08 21:37:29 +08:00
segments := m . SelectSegments ( WithCollection ( label . CollectionID ) , SegmentFilterFunc ( func ( segment * SegmentInfo ) bool {
2023-11-23 17:30:25 +08:00
return segment . GetState ( ) == commonpb . SegmentState_Growing &&
segment . GetPartitionID ( ) == label . PartitionID &&
segment . GetInsertChannel ( ) == label . Channel
2024-05-08 21:37:29 +08:00
} ) )
2023-11-23 17:30:25 +08:00
2023-12-05 18:44:37 +08:00
earliest := & msgpb . MsgPosition { Timestamp : math . MaxUint64 }
2023-11-23 17:30:25 +08:00
for _ , seg := range segments {
if earliest == nil || earliest . GetTimestamp ( ) > seg . GetStartPosition ( ) . GetTimestamp ( ) {
earliest = seg . GetStartPosition ( )
}
}
return earliest
}
2022-11-22 19:21:13 +08:00
// addNewSeg update metrics update for a new segment.
2023-11-24 15:58:24 +08:00
func ( s * segMetricMutation ) addNewSeg ( state commonpb . SegmentState , level datapb . SegmentLevel , rowCount int64 ) {
if _ , ok := s . stateChange [ level . String ( ) ] ; ! ok {
s . stateChange [ level . String ( ) ] = make ( map [ string ] int )
}
s . stateChange [ level . String ( ) ] [ state . String ( ) ] += 1
2022-11-22 19:21:13 +08:00
s . rowCountChange += rowCount
s . rowCountAccChange += rowCount
}
// commit persists all updates in current segMetricMutation, should and must be called AFTER segment state change
// has persisted in Etcd.
func ( s * segMetricMutation ) commit ( ) {
2023-11-24 15:58:24 +08:00
for level , submap := range s . stateChange {
for state , change := range submap {
metrics . DataCoordNumSegments . WithLabelValues ( state , level ) . Add ( float64 ( change ) )
}
2022-11-22 19:21:13 +08:00
}
}
// append updates current segMetricMutation when segment state change happens.
2023-11-24 15:58:24 +08:00
func ( s * segMetricMutation ) append ( oldState , newState commonpb . SegmentState , level datapb . SegmentLevel , rowCountUpdate int64 ) {
2022-11-22 19:21:13 +08:00
if oldState != newState {
2023-11-24 15:58:24 +08:00
if _ , ok := s . stateChange [ level . String ( ) ] ; ! ok {
s . stateChange [ level . String ( ) ] = make ( map [ string ] int )
}
s . stateChange [ level . String ( ) ] [ oldState . String ( ) ] -= 1
s . stateChange [ level . String ( ) ] [ newState . String ( ) ] += 1
2022-11-22 19:21:13 +08:00
}
// Update # of rows on new flush operations and drop operations.
if isFlushState ( newState ) && ! isFlushState ( oldState ) {
// If new flush.
s . rowCountChange += rowCountUpdate
s . rowCountAccChange += rowCountUpdate
} else if newState == commonpb . SegmentState_Dropped && oldState != newState {
// If new drop.
s . rowCountChange -= rowCountUpdate
}
}
func isFlushState ( state commonpb . SegmentState ) bool {
return state == commonpb . SegmentState_Flushing || state == commonpb . SegmentState_Flushed
}
// updateSegStateAndPrepareMetrics updates a segment's in-memory state and prepare for the corresponding metric update.
func updateSegStateAndPrepareMetrics ( segToUpdate * SegmentInfo , targetState commonpb . SegmentState , metricMutation * segMetricMutation ) {
log . Debug ( "updating segment state and updating metrics" ,
2023-05-04 12:22:40 +08:00
zap . Int64 ( "segmentID" , segToUpdate . GetID ( ) ) ,
2022-11-22 19:21:13 +08:00
zap . String ( "old state" , segToUpdate . GetState ( ) . String ( ) ) ,
zap . String ( "new state" , targetState . String ( ) ) ,
zap . Int64 ( "# of rows" , segToUpdate . GetNumOfRows ( ) ) )
2023-11-24 15:58:24 +08:00
metricMutation . append ( segToUpdate . GetState ( ) , targetState , segToUpdate . GetLevel ( ) , segToUpdate . GetNumOfRows ( ) )
2022-11-22 19:21:13 +08:00
segToUpdate . State = targetState
2024-06-12 20:21:56 +08:00
if targetState == commonpb . SegmentState_Dropped {
segToUpdate . DroppedAt = uint64 ( time . Now ( ) . UnixNano ( ) )
}
2022-11-22 19:21:13 +08:00
}
2024-05-30 13:37:44 +08:00
func ( m * meta ) ListCollections ( ) [ ] int64 {
m . RLock ( )
defer m . RUnlock ( )
return lo . Keys ( m . collections )
}
2024-06-05 10:17:50 +08:00
func ( m * meta ) DropCompactionTask ( task * datapb . CompactionTask ) error {
return m . compactionTaskMeta . DropCompactionTask ( task )
}
func ( m * meta ) SaveCompactionTask ( task * datapb . CompactionTask ) error {
return m . compactionTaskMeta . SaveCompactionTask ( task )
}
func ( m * meta ) GetCompactionTasks ( ) map [ int64 ] [ ] * datapb . CompactionTask {
return m . compactionTaskMeta . GetCompactionTasks ( )
}
func ( m * meta ) GetCompactionTasksByTriggerID ( triggerID int64 ) [ ] * datapb . CompactionTask {
return m . compactionTaskMeta . GetCompactionTasksByTriggerID ( triggerID )
}
2024-06-10 21:34:08 +08:00
func ( m * meta ) CleanPartitionStatsInfo ( info * datapb . PartitionStatsInfo ) error {
removePaths := make ( [ ] string , 0 )
partitionStatsPath := path . Join ( m . chunkManager . RootPath ( ) , common . PartitionStatsPath ,
metautil . JoinIDPath ( info . CollectionID , info . PartitionID ) ,
info . GetVChannel ( ) , strconv . FormatInt ( info . GetVersion ( ) , 10 ) )
removePaths = append ( removePaths , partitionStatsPath )
analyzeT := m . analyzeMeta . GetTask ( info . GetAnalyzeTaskID ( ) )
if analyzeT != nil {
centroidsFilePath := path . Join ( m . chunkManager . RootPath ( ) , common . AnalyzeStatsPath ,
metautil . JoinIDPath ( analyzeT . GetTaskID ( ) , analyzeT . GetVersion ( ) , analyzeT . GetCollectionID ( ) ,
analyzeT . GetPartitionID ( ) , analyzeT . GetFieldID ( ) ) ,
"centroids" ,
)
removePaths = append ( removePaths , centroidsFilePath )
for _ , segID := range info . GetSegmentIDs ( ) {
segmentOffsetMappingFilePath := path . Join ( m . chunkManager . RootPath ( ) , common . AnalyzeStatsPath ,
metautil . JoinIDPath ( analyzeT . GetTaskID ( ) , analyzeT . GetVersion ( ) , analyzeT . GetCollectionID ( ) ,
analyzeT . GetPartitionID ( ) , analyzeT . GetFieldID ( ) , segID ) ,
"offset_mapping" ,
)
removePaths = append ( removePaths , segmentOffsetMappingFilePath )
}
}
log . Debug ( "remove clustering compaction stats files" ,
zap . Int64 ( "collectionID" , info . GetCollectionID ( ) ) ,
zap . Int64 ( "partitionID" , info . GetPartitionID ( ) ) ,
zap . String ( "vChannel" , info . GetVChannel ( ) ) ,
zap . Int64 ( "planID" , info . GetVersion ( ) ) ,
zap . Strings ( "removePaths" , removePaths ) )
err := m . chunkManager . MultiRemove ( context . Background ( ) , removePaths )
if err != nil {
log . Warn ( "remove clustering compaction stats files failed" , zap . Error ( err ) )
return err
}
// first clean analyze task
if err = m . analyzeMeta . DropAnalyzeTask ( info . GetAnalyzeTaskID ( ) ) ; err != nil {
log . Warn ( "remove analyze task failed" , zap . Int64 ( "analyzeTaskID" , info . GetAnalyzeTaskID ( ) ) , zap . Error ( err ) )
return err
}
// finally, clean up the partition stats info, and make sure the analysis task is cleaned up
err = m . partitionStatsMeta . DropPartitionStatsInfo ( info )
log . Debug ( "drop partition stats meta" ,
zap . Int64 ( "collectionID" , info . GetCollectionID ( ) ) ,
zap . Int64 ( "partitionID" , info . GetPartitionID ( ) ) ,
zap . String ( "vChannel" , info . GetVChannel ( ) ) ,
zap . Int64 ( "planID" , info . GetVersion ( ) ) )
if err != nil {
return err
}
return nil
}