2023-03-26 16:42:00 +00: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 segments
/ *
2024-08-23 04:35:02 +00:00
# cgo pkg - config : milvus_core
2023-03-26 16:42:00 +00:00
2024-06-22 01:38:02 +00:00
# include "futures/future_c.h"
2023-03-26 16:42:00 +00:00
# include "segcore/collection_c.h"
# include "segcore/plan_c.h"
# include "segcore/reduce_c.h"
* /
import "C"
2023-04-06 11:14:32 +00:00
2023-03-26 16:42:00 +00:00
import (
"context"
"fmt"
2024-05-07 02:05:35 +00:00
"runtime"
2024-01-25 11:55:39 +00:00
"strings"
2023-03-26 16:42:00 +00:00
"unsafe"
"github.com/cockroachdb/errors"
2024-01-10 13:58:51 +00:00
"go.opentelemetry.io/otel"
2023-03-26 16:42:00 +00:00
"go.uber.org/atomic"
"go.uber.org/zap"
2024-07-29 03:31:51 +00:00
"google.golang.org/protobuf/proto"
2023-03-26 16:42:00 +00:00
2023-06-08 17:28:37 +00: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-31 01:47:47 +00:00
"github.com/milvus-io/milvus/internal/proto/cgopb"
2023-03-26 16:42:00 +00:00
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/querypb"
"github.com/milvus-io/milvus/internal/proto/segcorepb"
2024-05-06 12:29:30 +00:00
"github.com/milvus-io/milvus/internal/querycoordv2/params"
2023-09-22 10:31:25 +00:00
"github.com/milvus-io/milvus/internal/querynodev2/pkoracle"
2024-04-08 09:09:16 +00:00
"github.com/milvus-io/milvus/internal/querynodev2/segments/state"
2023-03-26 16:42:00 +00:00
"github.com/milvus-io/milvus/internal/storage"
2024-06-22 01:38:02 +00:00
"github.com/milvus-io/milvus/internal/util/cgo"
2024-01-12 10:10:51 +00:00
"github.com/milvus-io/milvus/pkg/common"
2023-04-06 11:14:32 +00:00
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
2024-05-31 01:47:47 +00:00
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
"github.com/milvus-io/milvus/pkg/util/indexparams"
2023-09-21 01:45:27 +00:00
"github.com/milvus-io/milvus/pkg/util/merr"
2024-05-07 11:13:35 +00:00
"github.com/milvus-io/milvus/pkg/util/metautil"
2023-04-06 11:14:32 +00:00
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/timerecord"
"github.com/milvus-io/milvus/pkg/util/typeutil"
2023-03-26 16:42:00 +00:00
)
type SegmentType = commonpb . SegmentState
const (
SegmentTypeGrowing = commonpb . SegmentState_Growing
SegmentTypeSealed = commonpb . SegmentState_Sealed
)
2023-09-21 01:45:27 +00:00
var ErrSegmentUnhealthy = errors . New ( "segment unhealthy" )
2023-03-26 16:42:00 +00:00
// IndexedFieldInfo contains binlog info of vector field
type IndexedFieldInfo struct {
FieldBinlog * datapb . FieldBinlog
IndexInfo * querypb . FieldIndexInfo
2024-05-06 12:29:30 +00:00
IsLoaded bool
2023-03-26 16:42:00 +00:00
}
type baseSegment struct {
2024-03-19 03:53:05 +00:00
collection * Collection
version * atomic . Int64
2024-02-27 12:58:40 +00:00
2024-03-19 03:53:05 +00:00
segmentType SegmentType
2023-03-26 16:42:00 +00:00
bloomFilterSet * pkoracle . BloomFilterSet
2024-05-06 12:29:30 +00:00
loadInfo * atomic . Pointer [ querypb . SegmentLoadInfo ]
2024-03-28 03:23:10 +00:00
isLazyLoad bool
2024-05-07 11:13:35 +00:00
channel metautil . Channel
2024-03-19 03:53:05 +00:00
resourceUsageCache * atomic . Pointer [ ResourceUsage ]
2024-05-06 12:29:30 +00:00
needUpdatedVersion * atomic . Int64 // only for lazy load mode update index
2023-03-26 16:42:00 +00:00
}
2024-05-07 11:13:35 +00:00
func newBaseSegment ( collection * Collection , segmentType SegmentType , version int64 , loadInfo * querypb . SegmentLoadInfo ) ( baseSegment , error ) {
channel , err := metautil . ParseChannel ( loadInfo . GetInsertChannel ( ) , channelMapper )
if err != nil {
return baseSegment { } , err
}
bs := baseSegment {
collection : collection ,
loadInfo : atomic . NewPointer [ querypb . SegmentLoadInfo ] ( loadInfo ) ,
version : atomic . NewInt64 ( version ) ,
segmentType : segmentType ,
bloomFilterSet : pkoracle . NewBloomFilterSet ( loadInfo . GetSegmentID ( ) , loadInfo . GetPartitionID ( ) , segmentType ) ,
channel : channel ,
isLazyLoad : isLazyLoad ( collection , segmentType ) ,
2024-03-19 03:53:05 +00:00
resourceUsageCache : atomic . NewPointer [ ResourceUsage ] ( nil ) ,
2024-05-06 12:29:30 +00:00
needUpdatedVersion : atomic . NewInt64 ( 0 ) ,
2023-03-26 16:42:00 +00:00
}
2024-05-07 11:13:35 +00:00
return bs , nil
2023-03-26 16:42:00 +00:00
}
2024-05-06 12:29:30 +00:00
// isLazyLoad checks if the segment is lazy load
func isLazyLoad ( collection * Collection , segmentType SegmentType ) bool {
return segmentType == SegmentTypeSealed && // only sealed segment enable lazy load
( common . IsCollectionLazyLoadEnabled ( collection . Schema ( ) . Properties ... ) || // collection level lazy load
( ! common . HasLazyload ( collection . Schema ( ) . Properties ) &&
params . Params . QueryNodeCfg . LazyLoadEnabled . GetAsBool ( ) ) ) // global level lazy load
}
2023-03-26 16:42:00 +00:00
// ID returns the identity number.
func ( s * baseSegment ) ID ( ) int64 {
2024-05-06 12:29:30 +00:00
return s . loadInfo . Load ( ) . GetSegmentID ( )
2023-03-26 16:42:00 +00:00
}
func ( s * baseSegment ) Collection ( ) int64 {
2024-05-06 12:29:30 +00:00
return s . loadInfo . Load ( ) . GetCollectionID ( )
}
func ( s * baseSegment ) GetCollection ( ) * Collection {
return s . collection
2023-03-26 16:42:00 +00:00
}
func ( s * baseSegment ) Partition ( ) int64 {
2024-05-06 12:29:30 +00:00
return s . loadInfo . Load ( ) . GetPartitionID ( )
2023-03-26 16:42:00 +00:00
}
2024-04-01 02:21:21 +00:00
func ( s * baseSegment ) DatabaseName ( ) string {
return s . collection . GetDBName ( )
}
func ( s * baseSegment ) ResourceGroup ( ) string {
return s . collection . GetResourceGroup ( )
}
2024-05-07 11:13:35 +00:00
func ( s * baseSegment ) Shard ( ) metautil . Channel {
return s . channel
2023-03-26 16:42:00 +00:00
}
func ( s * baseSegment ) Type ( ) SegmentType {
2024-03-19 03:53:05 +00:00
return s . segmentType
2023-03-26 16:42:00 +00:00
}
2023-11-06 17:44:18 +00:00
func ( s * baseSegment ) Level ( ) datapb . SegmentLevel {
2024-05-06 12:29:30 +00:00
return s . loadInfo . Load ( ) . GetLevel ( )
2023-11-06 17:44:18 +00:00
}
2023-03-26 16:42:00 +00:00
func ( s * baseSegment ) StartPosition ( ) * msgpb . MsgPosition {
2024-05-06 12:29:30 +00:00
return s . loadInfo . Load ( ) . GetStartPosition ( )
2023-03-26 16:42:00 +00:00
}
func ( s * baseSegment ) Version ( ) int64 {
2023-08-16 08:30:18 +00:00
return s . version . Load ( )
2023-03-26 16:42:00 +00:00
}
2023-08-29 07:46:27 +00:00
func ( s * baseSegment ) CASVersion ( old , newVersion int64 ) bool {
return s . version . CompareAndSwap ( old , newVersion )
2023-08-11 03:21:32 +00:00
}
2024-02-27 12:58:40 +00:00
func ( s * baseSegment ) LoadInfo ( ) * querypb . SegmentLoadInfo {
2024-05-06 12:29:30 +00:00
return s . loadInfo . Load ( )
2024-02-27 12:58:40 +00:00
}
2023-03-26 16:42:00 +00:00
func ( s * baseSegment ) UpdateBloomFilter ( pks [ ] storage . PrimaryKey ) {
s . bloomFilterSet . UpdateBloomFilter ( pks )
}
// MayPkExist returns true if the given PK exists in the PK range and being positive through the bloom filter,
// false otherwise,
// may returns true even the PK doesn't exist actually
2024-06-13 09:57:56 +00:00
func ( s * baseSegment ) MayPkExist ( pk * storage . LocationsCache ) bool {
return s . bloomFilterSet . MayPkExist ( pk )
}
func ( s * baseSegment ) BatchPkExist ( lc * storage . BatchLocationsCache ) [ ] bool {
return s . bloomFilterSet . BatchPkExist ( lc )
2024-05-07 13:13:47 +00:00
}
2024-03-19 03:53:05 +00:00
// ResourceUsageEstimate returns the estimated resource usage of the segment.
func ( s * baseSegment ) ResourceUsageEstimate ( ) ResourceUsage {
if s . segmentType == SegmentTypeGrowing {
// Growing segment cannot do resource usage estimate.
return ResourceUsage { }
}
cache := s . resourceUsageCache . Load ( )
if cache != nil {
return * cache
}
2024-05-06 12:29:30 +00:00
usage , err := getResourceUsageEstimateOfSegment ( s . collection . Schema ( ) , s . LoadInfo ( ) , resourceEstimateFactor {
2024-03-19 03:53:05 +00:00
memoryUsageFactor : 1.0 ,
memoryIndexUsageFactor : 1.0 ,
enableTempSegmentIndex : false ,
deltaDataExpansionFactor : paramtable . Get ( ) . QueryNodeCfg . DeltaDataExpansionRate . GetAsFloat ( ) ,
} )
if err != nil {
// Should never failure, if failed, segment should never be loaded.
log . Warn ( "unreachable: failed to get resource usage estimate of segment" , zap . Error ( err ) , zap . Int64 ( "collectionID" , s . Collection ( ) ) , zap . Int64 ( "segmentID" , s . ID ( ) ) )
return ResourceUsage { }
}
s . resourceUsageCache . Store ( usage )
return * usage
}
2024-05-06 12:29:30 +00:00
func ( s * baseSegment ) IsLazyLoad ( ) bool {
return s . isLazyLoad
}
func ( s * baseSegment ) NeedUpdatedVersion ( ) int64 {
return s . needUpdatedVersion . Load ( )
}
func ( s * baseSegment ) SetLoadInfo ( loadInfo * querypb . SegmentLoadInfo ) {
s . loadInfo . Store ( loadInfo )
}
func ( s * baseSegment ) SetNeedUpdatedVersion ( version int64 ) {
s . needUpdatedVersion . Store ( version )
}
2024-03-28 03:23:10 +00:00
2024-02-27 12:58:40 +00:00
type FieldInfo struct {
2024-07-30 07:53:51 +00:00
* datapb . FieldBinlog
2024-02-27 12:58:40 +00:00
RowCount int64
}
2023-03-26 16:42:00 +00:00
var _ Segment = ( * LocalSegment ) ( nil )
// Segment is a wrapper of the underlying C-structure segment.
type LocalSegment struct {
baseSegment
2024-04-08 09:09:16 +00:00
ptrLock * state . LoadStateLock
2023-08-16 10:38:17 +00:00
ptr C . CSegmentInterface
2023-03-26 16:42:00 +00:00
2023-10-30 09:54:14 +00:00
// cached results, to avoid too many CGO calls
memSize * atomic . Int64
rowNum * atomic . Int64
insertCount * atomic . Int64
2023-03-26 16:42:00 +00:00
lastDeltaTimestamp * atomic . Uint64
2024-02-27 12:58:40 +00:00
fields * typeutil . ConcurrentMap [ int64 , * FieldInfo ]
2023-03-26 16:42:00 +00:00
fieldIndexes * typeutil . ConcurrentMap [ int64 , * IndexedFieldInfo ]
}
2023-12-27 08:10:47 +00:00
func NewSegment ( ctx context . Context ,
collection * Collection ,
2023-03-26 16:42:00 +00:00
segmentType SegmentType ,
version int64 ,
2024-03-19 03:53:05 +00:00
loadInfo * querypb . SegmentLoadInfo ,
2023-11-06 17:44:18 +00:00
) ( Segment , error ) {
2023-12-27 08:10:47 +00:00
log := log . Ctx ( ctx )
2023-03-26 16:42:00 +00:00
/ *
2023-12-07 05:34:35 +00:00
CStatus
NewSegment ( CCollection collection , uint64_t segment_id , SegmentType seg_type , CSegmentInterface * newSegment ) ;
2023-03-26 16:42:00 +00:00
* /
2024-03-19 03:53:05 +00:00
if loadInfo . GetLevel ( ) == datapb . SegmentLevel_L0 {
return NewL0Segment ( collection , segmentType , version , loadInfo )
2023-11-06 17:44:18 +00:00
}
2024-05-07 11:13:35 +00:00
base , err := newBaseSegment ( collection , segmentType , version , loadInfo )
if err != nil {
return nil , err
}
2023-12-07 05:34:35 +00:00
var cSegType C . SegmentType
2024-04-08 09:09:16 +00:00
var locker * state . LoadStateLock
2023-03-26 16:42:00 +00:00
switch segmentType {
case SegmentTypeSealed :
2023-12-07 05:34:35 +00:00
cSegType = C . Sealed
2024-04-08 09:09:16 +00:00
locker = state . NewLoadStateLock ( state . LoadStateOnlyMeta )
2023-03-26 16:42:00 +00:00
case SegmentTypeGrowing :
2024-04-08 09:09:16 +00:00
locker = state . NewLoadStateLock ( state . LoadStateDataLoaded )
2023-12-07 05:34:35 +00:00
cSegType = C . Growing
2023-03-26 16:42:00 +00:00
default :
2024-03-19 03:53:05 +00:00
return nil , fmt . Errorf ( "illegal segment type %d when create segment %d" , segmentType , loadInfo . GetSegmentID ( ) )
2023-03-26 16:42:00 +00:00
}
2023-12-07 05:34:35 +00:00
var newPtr C . CSegmentInterface
2024-05-07 11:13:35 +00:00
_ , err = GetDynamicPool ( ) . Submit ( func ( ) ( any , error ) {
2024-03-19 03:53:05 +00:00
status := C . NewSegment ( collection . collectionPtr , cSegType , C . int64_t ( loadInfo . GetSegmentID ( ) ) , & newPtr )
2023-12-27 08:10:47 +00:00
err := HandleCStatus ( ctx , & status , "NewSegmentFailed" ,
2024-03-19 03:53:05 +00:00
zap . Int64 ( "collectionID" , loadInfo . GetCollectionID ( ) ) ,
zap . Int64 ( "partitionID" , loadInfo . GetPartitionID ( ) ) ,
zap . Int64 ( "segmentID" , loadInfo . GetSegmentID ( ) ) ,
2023-12-26 06:06:46 +00:00
zap . String ( "segmentType" , segmentType . String ( ) ) )
2023-12-20 02:06:43 +00:00
return nil , err
} ) . Await ( )
if err != nil {
2023-12-07 05:34:35 +00:00
return nil , err
}
2023-03-26 16:42:00 +00:00
log . Info ( "create segment" ,
2024-03-19 03:53:05 +00:00
zap . Int64 ( "collectionID" , loadInfo . GetCollectionID ( ) ) ,
zap . Int64 ( "partitionID" , loadInfo . GetPartitionID ( ) ) ,
zap . Int64 ( "segmentID" , loadInfo . GetSegmentID ( ) ) ,
2024-01-08 06:16:48 +00:00
zap . String ( "segmentType" , segmentType . String ( ) ) ,
2024-03-19 03:53:05 +00:00
zap . String ( "level" , loadInfo . GetLevel ( ) . String ( ) ) ,
2024-01-08 06:16:48 +00:00
)
2023-03-26 16:42:00 +00:00
2023-09-21 01:45:27 +00:00
segment := & LocalSegment {
2024-05-07 11:13:35 +00:00
baseSegment : base ,
2024-04-08 09:09:16 +00:00
ptrLock : locker ,
2023-12-07 05:34:35 +00:00
ptr : newPtr ,
2023-09-19 08:21:23 +00:00
lastDeltaTimestamp : atomic . NewUint64 ( 0 ) ,
2024-02-27 12:58:40 +00:00
fields : typeutil . NewConcurrentMap [ int64 , * FieldInfo ] ( ) ,
2023-03-26 16:42:00 +00:00
fieldIndexes : typeutil . NewConcurrentMap [ int64 , * IndexedFieldInfo ] ( ) ,
2023-10-30 09:54:14 +00:00
memSize : atomic . NewInt64 ( - 1 ) ,
rowNum : atomic . NewInt64 ( - 1 ) ,
insertCount : atomic . NewInt64 ( 0 ) ,
2023-03-26 16:42:00 +00:00
}
2024-05-06 12:29:30 +00:00
if err := segment . initializeSegment ( ) ; err != nil {
return nil , err
2024-02-27 12:58:40 +00:00
}
2023-03-26 16:42:00 +00:00
return segment , nil
}
2024-05-06 12:29:30 +00:00
func ( s * LocalSegment ) initializeSegment ( ) error {
loadInfo := s . loadInfo . Load ( )
indexedFieldInfos , fieldBinlogs := separateIndexAndBinlog ( loadInfo )
schemaHelper , _ := typeutil . CreateSchemaHelper ( s . collection . Schema ( ) )
for fieldID , info := range indexedFieldInfos {
field , err := schemaHelper . GetFieldFromID ( fieldID )
if err != nil {
return err
}
indexInfo := info . IndexInfo
s . fieldIndexes . Insert ( indexInfo . GetFieldID ( ) , & IndexedFieldInfo {
FieldBinlog : & datapb . FieldBinlog {
FieldID : indexInfo . GetFieldID ( ) ,
} ,
IndexInfo : indexInfo ,
IsLoaded : false ,
} )
if ! typeutil . IsVectorType ( field . GetDataType ( ) ) && ! s . HasRawData ( fieldID ) {
s . fields . Insert ( fieldID , & FieldInfo {
2024-07-30 07:53:51 +00:00
FieldBinlog : info . FieldBinlog ,
2024-05-06 12:29:30 +00:00
RowCount : loadInfo . GetNumOfRows ( ) ,
} )
}
}
for _ , binlogs := range fieldBinlogs {
s . fields . Insert ( binlogs . FieldID , & FieldInfo {
2024-07-30 07:53:51 +00:00
FieldBinlog : binlogs ,
2024-05-06 12:29:30 +00:00
RowCount : loadInfo . GetNumOfRows ( ) ,
} )
}
// Update the insert count when initialize the segment and update the metrics.
s . insertCount . Store ( loadInfo . GetNumOfRows ( ) )
return nil
}
2024-04-08 09:09:16 +00:00
// PinIfNotReleased acquires the `ptrLock` and returns true if the pointer is valid
2023-08-16 10:38:17 +00:00
// Provide ONLY the read lock operations,
// don't make `ptrLock` public to avoid abusing of the mutex.
2024-04-08 09:09:16 +00:00
func ( s * LocalSegment ) PinIfNotReleased ( ) error {
if ! s . ptrLock . PinIfNotReleased ( ) {
2023-08-16 10:38:17 +00:00
return merr . WrapErrSegmentNotLoaded ( s . ID ( ) , "segment released" )
}
return nil
}
2024-04-08 09:09:16 +00:00
func ( s * LocalSegment ) Unpin ( ) {
s . ptrLock . Unpin ( )
2023-08-16 10:38:17 +00:00
}
2023-03-26 16:42:00 +00:00
func ( s * LocalSegment ) InsertCount ( ) int64 {
2023-10-30 09:54:14 +00:00
return s . insertCount . Load ( )
2023-03-26 16:42:00 +00:00
}
func ( s * LocalSegment ) RowNum ( ) int64 {
2024-04-08 09:09:16 +00:00
// if segment is not loaded, return 0 (maybe not loaded or release by lru)
if ! s . ptrLock . RLockIf ( state . IsDataLoaded ) {
2023-05-24 11:23:27 +00:00
return 0
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2023-03-26 16:42:00 +00:00
2023-10-30 09:54:14 +00:00
rowNum := s . rowNum . Load ( )
if rowNum < 0 {
var rowCount C . int64_t
GetDynamicPool ( ) . Submit ( func ( ) ( any , error ) {
rowCount = C . GetRealCount ( s . ptr )
s . rowNum . Store ( int64 ( rowCount ) )
return nil , nil
} ) . Await ( )
rowNum = int64 ( rowCount )
}
return rowNum
2023-03-26 16:42:00 +00:00
}
func ( s * LocalSegment ) MemSize ( ) int64 {
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
2023-05-24 11:23:27 +00:00
return 0
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2023-03-26 16:42:00 +00:00
2023-10-30 09:54:14 +00:00
memSize := s . memSize . Load ( )
if memSize < 0 {
var cMemSize C . int64_t
GetDynamicPool ( ) . Submit ( func ( ) ( any , error ) {
cMemSize = C . GetMemoryUsageInBytes ( s . ptr )
s . memSize . Store ( int64 ( cMemSize ) )
return nil , nil
} ) . Await ( )
memSize = int64 ( cMemSize )
}
return memSize
2023-03-26 16:42:00 +00:00
}
func ( s * LocalSegment ) LastDeltaTimestamp ( ) uint64 {
return s . lastDeltaTimestamp . Load ( )
}
func ( s * LocalSegment ) GetIndex ( fieldID int64 ) * IndexedFieldInfo {
info , _ := s . fieldIndexes . Get ( fieldID )
return info
}
func ( s * LocalSegment ) ExistIndex ( fieldID int64 ) bool {
fieldInfo , ok := s . fieldIndexes . Get ( fieldID )
if ! ok {
return false
}
2024-05-07 09:11:30 +00:00
return fieldInfo . IndexInfo != nil
2023-03-26 16:42:00 +00:00
}
2023-04-23 01:00:32 +00:00
func ( s * LocalSegment ) HasRawData ( fieldID int64 ) bool {
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
2023-05-24 11:23:27 +00:00
return false
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2023-04-23 01:00:32 +00:00
ret := C . HasRawData ( s . ptr , C . int64_t ( fieldID ) )
return bool ( ret )
}
2023-03-26 16:42:00 +00:00
func ( s * LocalSegment ) Indexes ( ) [ ] * IndexedFieldInfo {
var result [ ] * IndexedFieldInfo
s . fieldIndexes . Range ( func ( key int64 , value * IndexedFieldInfo ) bool {
result = append ( result , value )
return true
} )
return result
}
2024-04-10 07:15:19 +00:00
func ( s * LocalSegment ) ResetIndexesLazyLoad ( lazyState bool ) {
for _ , indexInfo := range s . Indexes ( ) {
2024-05-06 12:29:30 +00:00
indexInfo . IsLoaded = lazyState
2024-04-10 07:15:19 +00:00
}
}
2023-03-26 16:42:00 +00:00
func ( s * LocalSegment ) Search ( ctx context . Context , searchReq * SearchRequest ) ( * SearchResult , error ) {
/ *
CStatus
Search ( void * plan ,
void * placeholder_groups ,
uint64_t * timestamps ,
int num_groups ,
long int * result_ids ,
float * result_distances ) ;
* /
log := log . Ctx ( ctx ) . With (
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
2024-03-19 03:53:05 +00:00
zap . String ( "segmentType" , s . segmentType . String ( ) ) ,
2023-03-26 16:42:00 +00:00
)
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
// TODO: check if the segment is readable but not released. too many related logic need to be refactor.
2024-03-19 03:53:05 +00:00
return nil , merr . WrapErrSegmentNotLoaded ( s . ID ( ) , "segment released" )
2023-03-26 16:42:00 +00:00
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2023-03-26 16:42:00 +00:00
2024-02-28 10:59:00 +00:00
traceCtx := ParseCTraceContext ( ctx )
2024-06-22 01:38:02 +00:00
defer runtime . KeepAlive ( traceCtx )
defer runtime . KeepAlive ( searchReq )
2023-03-26 16:42:00 +00:00
hasIndex := s . ExistIndex ( searchReq . searchFieldID )
log = log . With ( zap . Bool ( "withIndex" , hasIndex ) )
log . Debug ( "search segment..." )
2024-06-22 01:38:02 +00:00
tr := timerecord . NewTimeRecorder ( "cgoSearch" )
future := cgo . Async (
ctx ,
func ( ) cgo . CFuturePtr {
return ( cgo . CFuturePtr ) ( C . AsyncSearch (
traceCtx . ctx ,
s . ptr ,
searchReq . plan . cSearchPlan ,
searchReq . cPlaceholderGroup ,
C . uint64_t ( searchReq . mvccTimestamp ) ,
) )
} ,
cgo . WithName ( "search" ) ,
)
defer future . Release ( )
result , err := future . BlockAndLeakyGet ( )
if err != nil {
log . Warn ( "Search failed" )
2023-03-26 16:42:00 +00:00
return nil , err
}
2024-06-22 01:38:02 +00:00
metrics . QueryNodeSQSegmentLatencyInCore . WithLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) , metrics . SearchLabel ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-03-26 16:42:00 +00:00
log . Debug ( "search segment done" )
2024-06-22 01:38:02 +00:00
return & SearchResult {
cSearchResult : ( C . CSearchResult ) ( result ) ,
} , nil
2023-03-26 16:42:00 +00:00
}
2023-03-29 10:10:02 +00:00
func ( s * LocalSegment ) Retrieve ( ctx context . Context , plan * RetrievePlan ) ( * segcorepb . RetrieveResults , error ) {
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
// TODO: check if the segment is readable but not released. too many related logic need to be refactor.
2024-03-19 03:53:05 +00:00
return nil , merr . WrapErrSegmentNotLoaded ( s . ID ( ) , "segment released" )
2023-03-26 16:42:00 +00:00
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2023-03-26 16:42:00 +00:00
2023-10-19 11:22:10 +00:00
log := log . Ctx ( ctx ) . With (
2023-03-26 16:42:00 +00:00
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
2023-05-19 10:19:24 +00:00
zap . Int64 ( "msgID" , plan . msgID ) ,
2024-03-19 03:53:05 +00:00
zap . String ( "segmentType" , s . segmentType . String ( ) ) ,
2023-03-26 16:42:00 +00:00
)
2024-04-25 01:49:26 +00:00
log . Debug ( "begin to retrieve" )
2023-03-26 16:42:00 +00:00
2024-02-28 10:59:00 +00:00
traceCtx := ParseCTraceContext ( ctx )
2024-06-22 01:38:02 +00:00
defer runtime . KeepAlive ( traceCtx )
defer runtime . KeepAlive ( plan )
2023-03-29 10:10:02 +00:00
2023-08-10 06:11:15 +00:00
maxLimitSize := paramtable . Get ( ) . QuotaConfig . MaxOutputSize . GetAsInt64 ( )
2024-06-22 01:38:02 +00:00
tr := timerecord . NewTimeRecorder ( "cgoRetrieve" )
future := cgo . Async (
ctx ,
func ( ) cgo . CFuturePtr {
return ( cgo . CFuturePtr ) ( C . AsyncRetrieve (
traceCtx . ctx ,
s . ptr ,
plan . cRetrievePlan ,
C . uint64_t ( plan . Timestamp ) ,
C . int64_t ( maxLimitSize ) ,
C . bool ( plan . ignoreNonPk ) ,
) )
} ,
cgo . WithName ( "retrieve" ) ,
)
defer future . Release ( )
result , err := future . BlockAndLeakyGet ( )
if err != nil {
log . Warn ( "Retrieve failed" )
2023-03-26 16:42:00 +00:00
return nil , err
}
2024-06-22 01:38:02 +00:00
defer C . DeleteRetrieveResult ( ( * C . CRetrieveResult ) ( result ) )
metrics . QueryNodeSQSegmentLatencyInCore . WithLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) ,
metrics . QueryLabel ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
2023-04-19 15:56:31 +00:00
2024-04-25 01:49:26 +00:00
_ , span := otel . Tracer ( typeutil . QueryNodeRole ) . Start ( ctx , "partial-segcore-results-deserialization" )
defer span . End ( )
2024-06-22 01:38:02 +00:00
retrieveResult := new ( segcorepb . RetrieveResults )
if err := UnmarshalCProto ( ( * C . CRetrieveResult ) ( result ) , retrieveResult ) ; err != nil {
log . Warn ( "unmarshal retrieve result failed" , zap . Error ( err ) )
2023-03-26 16:42:00 +00:00
return nil , err
}
2023-05-19 10:19:24 +00:00
log . Debug ( "retrieve segment done" ,
2024-06-22 01:38:02 +00:00
zap . Int ( "resultNum" , len ( retrieveResult . Offset ) ) ,
2023-03-26 16:42:00 +00:00
)
2023-08-01 12:29:05 +00:00
// Sort was done by the segcore.
// sort.Sort(&byPK{result})
2024-06-22 01:38:02 +00:00
return retrieveResult , nil
2023-03-26 16:42:00 +00:00
}
2024-04-25 01:49:26 +00:00
func ( s * LocalSegment ) RetrieveByOffsets ( ctx context . Context , plan * RetrievePlan , offsets [ ] int64 ) ( * segcorepb . RetrieveResults , error ) {
2024-06-22 01:38:02 +00:00
if len ( offsets ) == 0 {
return nil , merr . WrapErrParameterInvalid ( "segment offsets" , "empty offsets" )
}
2024-04-25 01:49:26 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
// TODO: check if the segment is readable but not released. too many related logic need to be refactor.
return nil , merr . WrapErrSegmentNotLoaded ( s . ID ( ) , "segment released" )
}
defer s . ptrLock . RUnlock ( )
fields := [ ] zap . Field {
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
zap . Int64 ( "msgID" , plan . msgID ) ,
zap . String ( "segmentType" , s . segmentType . String ( ) ) ,
zap . Int ( "resultNum" , len ( offsets ) ) ,
}
log := log . Ctx ( ctx ) . With ( fields ... )
log . Debug ( "begin to retrieve by offsets" )
tr := timerecord . NewTimeRecorder ( "cgoRetrieveByOffsets" )
2024-06-22 01:38:02 +00:00
traceCtx := ParseCTraceContext ( ctx )
defer runtime . KeepAlive ( traceCtx )
defer runtime . KeepAlive ( plan )
defer runtime . KeepAlive ( offsets )
future := cgo . Async (
ctx ,
func ( ) cgo . CFuturePtr {
return ( cgo . CFuturePtr ) ( C . AsyncRetrieveByOffsets (
traceCtx . ctx ,
s . ptr ,
plan . cRetrievePlan ,
( * C . int64_t ) ( unsafe . Pointer ( & offsets [ 0 ] ) ) ,
C . int64_t ( len ( offsets ) ) ,
) )
} ,
cgo . WithName ( "retrieve-by-offsets" ) ,
)
defer future . Release ( )
result , err := future . BlockAndLeakyGet ( )
if err != nil {
log . Warn ( "RetrieveByOffsets failed" )
return nil , err
}
defer C . DeleteRetrieveResult ( ( * C . CRetrieveResult ) ( result ) )
2024-04-25 01:49:26 +00:00
metrics . QueryNodeSQSegmentLatencyInCore . WithLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) ,
metrics . QueryLabel ) . Observe ( float64 ( tr . ElapseSpan ( ) . Milliseconds ( ) ) )
_ , span := otel . Tracer ( typeutil . QueryNodeRole ) . Start ( ctx , "reduced-segcore-results-deserialization" )
defer span . End ( )
2024-06-22 01:38:02 +00:00
retrieveResult := new ( segcorepb . RetrieveResults )
if err := UnmarshalCProto ( ( * C . CRetrieveResult ) ( result ) , retrieveResult ) ; err != nil {
log . Warn ( "unmarshal retrieve by offsets result failed" , zap . Error ( err ) )
2024-04-25 01:49:26 +00:00
return nil , err
}
2024-06-22 01:38:02 +00:00
log . Debug ( "retrieve by segment offsets done" ,
zap . Int ( "resultNum" , len ( retrieveResult . Offset ) ) ,
)
return retrieveResult , nil
2024-04-25 01:49:26 +00:00
}
2023-03-26 16:42:00 +00:00
func ( s * LocalSegment ) GetFieldDataPath ( index * IndexedFieldInfo , offset int64 ) ( dataPath string , offsetInBinlog int64 ) {
offsetInBinlog = offset
for _ , binlog := range index . FieldBinlog . Binlogs {
if offsetInBinlog < binlog . EntriesNum {
dataPath = binlog . GetLogPath ( )
break
} else {
offsetInBinlog -= binlog . EntriesNum
}
}
return dataPath , offsetInBinlog
}
// -------------------------------------------------------------------------------------- interfaces for growing segment
2023-12-27 08:10:47 +00:00
func ( s * LocalSegment ) preInsert ( ctx context . Context , numOfRecords int ) ( int64 , error ) {
2023-03-26 16:42:00 +00:00
/ *
long int
PreInsert ( CSegmentInterface c_segment , long int size ) ;
* /
var offset int64
cOffset := ( * C . int64_t ) ( & offset )
2023-04-19 15:56:31 +00:00
var status C . CStatus
2023-08-01 01:19:05 +00:00
GetDynamicPool ( ) . Submit ( func ( ) ( any , error ) {
2023-04-19 15:56:31 +00:00
status = C . PreInsert ( s . ptr , C . int64_t ( int64 ( numOfRecords ) ) , cOffset )
return nil , nil
} ) . Await ( )
2023-12-27 08:10:47 +00:00
if err := HandleCStatus ( ctx , & status , "PreInsert failed" ) ; err != nil {
2023-03-26 16:42:00 +00:00
return 0 , err
}
return offset , nil
}
2023-12-27 08:10:47 +00:00
func ( s * LocalSegment ) Insert ( ctx context . Context , rowIDs [ ] int64 , timestamps [ ] typeutil . Timestamp , record * segcorepb . InsertRecord ) error {
2023-03-26 16:42:00 +00:00
if s . Type ( ) != SegmentTypeGrowing {
2024-03-19 03:53:05 +00:00
return fmt . Errorf ( "unexpected segmentType when segmentInsert, segmentType = %s" , s . segmentType . String ( ) )
2023-03-26 16:42:00 +00:00
}
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
2024-03-19 03:53:05 +00:00
return merr . WrapErrSegmentNotLoaded ( s . ID ( ) , "segment released" )
2023-03-26 16:42:00 +00:00
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2023-03-26 16:42:00 +00:00
2023-12-27 08:10:47 +00:00
offset , err := s . preInsert ( ctx , len ( rowIDs ) )
2023-03-26 16:42:00 +00:00
if err != nil {
return err
}
insertRecordBlob , err := proto . Marshal ( record )
if err != nil {
return fmt . Errorf ( "failed to marshal insert record: %s" , err )
}
2023-09-21 01:45:27 +00:00
numOfRow := len ( rowIDs )
cOffset := C . int64_t ( offset )
cNumOfRows := C . int64_t ( numOfRow )
2024-03-27 22:33:11 +00:00
cEntityIDsPtr := ( * C . int64_t ) ( & ( rowIDs ) [ 0 ] )
2023-09-21 01:45:27 +00:00
cTimestampsPtr := ( * C . uint64_t ) ( & ( timestamps ) [ 0 ] )
2023-03-26 16:42:00 +00:00
2023-04-19 15:56:31 +00:00
var status C . CStatus
2023-08-01 01:19:05 +00:00
GetDynamicPool ( ) . Submit ( func ( ) ( any , error ) {
2023-04-19 15:56:31 +00:00
status = C . Insert ( s . ptr ,
cOffset ,
cNumOfRows ,
2024-03-27 22:33:11 +00:00
cEntityIDsPtr ,
2023-04-19 15:56:31 +00:00
cTimestampsPtr ,
( * C . uint8_t ) ( unsafe . Pointer ( & insertRecordBlob [ 0 ] ) ) ,
( C . uint64_t ) ( len ( insertRecordBlob ) ) ,
)
return nil , nil
} ) . Await ( )
2023-12-27 08:10:47 +00:00
if err := HandleCStatus ( ctx , & status , "Insert failed" ) ; err != nil {
2023-03-26 16:42:00 +00:00
return err
}
2023-10-30 09:54:14 +00:00
s . insertCount . Add ( int64 ( numOfRow ) )
s . rowNum . Store ( - 1 )
s . memSize . Store ( - 1 )
2023-03-26 16:42:00 +00:00
return nil
}
2023-12-27 08:10:47 +00:00
func ( s * LocalSegment ) Delete ( ctx context . Context , primaryKeys [ ] storage . PrimaryKey , timestamps [ ] typeutil . Timestamp ) error {
2023-03-26 16:42:00 +00:00
/ *
CStatus
Delete ( CSegmentInterface c_segment ,
long int reserved_offset ,
long size ,
const long * primary_keys ,
const unsigned long * timestamps ) ;
* /
2023-04-20 03:32:31 +00:00
2023-11-08 08:42:21 +00:00
if len ( primaryKeys ) == 0 {
return nil
}
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
2024-03-19 03:53:05 +00:00
return merr . WrapErrSegmentNotLoaded ( s . ID ( ) , "segment released" )
2023-03-26 16:42:00 +00:00
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2023-03-26 16:42:00 +00:00
2023-09-21 01:45:27 +00:00
cOffset := C . int64_t ( 0 ) // depre
cSize := C . int64_t ( len ( primaryKeys ) )
cTimestampsPtr := ( * C . uint64_t ) ( & ( timestamps ) [ 0 ] )
2023-03-26 16:42:00 +00:00
ids := & schemapb . IDs { }
pkType := primaryKeys [ 0 ] . Type ( )
switch pkType {
case schemapb . DataType_Int64 :
int64Pks := make ( [ ] int64 , len ( primaryKeys ) )
for index , pk := range primaryKeys {
int64Pks [ index ] = pk . ( * storage . Int64PrimaryKey ) . Value
}
ids . IdField = & schemapb . IDs_IntId {
IntId : & schemapb . LongArray {
Data : int64Pks ,
} ,
}
case schemapb . DataType_VarChar :
varCharPks := make ( [ ] string , len ( primaryKeys ) )
for index , entity := range primaryKeys {
varCharPks [ index ] = entity . ( * storage . VarCharPrimaryKey ) . Value
}
ids . IdField = & schemapb . IDs_StrId {
StrId : & schemapb . StringArray {
Data : varCharPks ,
} ,
}
default :
return fmt . Errorf ( "invalid data type of primary keys" )
}
dataBlob , err := proto . Marshal ( ids )
if err != nil {
return fmt . Errorf ( "failed to marshal ids: %s" , err )
}
2023-04-19 15:56:31 +00:00
var status C . CStatus
2023-08-01 01:19:05 +00:00
GetDynamicPool ( ) . Submit ( func ( ) ( any , error ) {
2023-04-19 15:56:31 +00:00
status = C . Delete ( s . ptr ,
cOffset ,
cSize ,
( * C . uint8_t ) ( unsafe . Pointer ( & dataBlob [ 0 ] ) ) ,
( C . uint64_t ) ( len ( dataBlob ) ) ,
cTimestampsPtr ,
)
return nil , nil
} ) . Await ( )
2023-03-26 16:42:00 +00:00
2023-12-27 08:10:47 +00:00
if err := HandleCStatus ( ctx , & status , "Delete failed" ) ; err != nil {
2023-03-26 16:42:00 +00:00
return err
}
2023-10-30 09:54:14 +00:00
s . rowNum . Store ( - 1 )
2023-03-26 16:42:00 +00:00
s . lastDeltaTimestamp . Store ( timestamps [ len ( timestamps ) - 1 ] )
return nil
}
// -------------------------------------------------------------------------------------- interfaces for sealed segment
2024-05-06 12:29:30 +00:00
func ( s * LocalSegment ) LoadMultiFieldData ( ctx context . Context ) error {
loadInfo := s . loadInfo . Load ( )
rowCount := loadInfo . GetNumOfRows ( )
fields := loadInfo . GetBinlogPaths ( )
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
2024-03-19 03:53:05 +00:00
return merr . WrapErrSegmentNotLoaded ( s . ID ( ) , "segment released" )
2023-06-25 06:38:44 +00:00
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2023-06-25 06:38:44 +00:00
2023-12-27 08:10:47 +00:00
log := log . Ctx ( ctx ) . With (
2023-06-25 06:38:44 +00:00
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
)
2023-12-27 08:10:47 +00:00
loadFieldDataInfo , err := newLoadFieldDataInfo ( ctx )
2023-06-25 06:38:44 +00:00
defer deleteFieldDataInfo ( loadFieldDataInfo )
if err != nil {
return err
2023-03-26 16:42:00 +00:00
}
2023-06-25 06:38:44 +00:00
for _ , field := range fields {
fieldID := field . FieldID
2023-12-27 08:10:47 +00:00
err = loadFieldDataInfo . appendLoadFieldInfo ( ctx , fieldID , rowCount )
2023-06-25 06:38:44 +00:00
if err != nil {
return err
}
for _ , binlog := range field . Binlogs {
2023-12-27 08:10:47 +00:00
err = loadFieldDataInfo . appendLoadFieldDataPath ( ctx , fieldID , binlog )
2023-06-25 06:38:44 +00:00
if err != nil {
return err
}
}
loadFieldDataInfo . appendMMapDirPath ( paramtable . Get ( ) . QueryNodeCfg . MmapDirPath . GetValue ( ) )
}
var status C . CStatus
2023-11-17 11:56:21 +00:00
GetLoadPool ( ) . Submit ( func ( ) ( any , error ) {
2024-08-01 04:08:13 +00:00
status = C . LoadFieldData ( s . ptr , loadFieldDataInfo . cLoadFieldDataInfo )
2023-06-25 06:38:44 +00:00
return nil , nil
} ) . Await ( )
2023-12-27 08:10:47 +00:00
if err := HandleCStatus ( ctx , & status , "LoadMultiFieldData failed" ,
2023-12-26 06:06:46 +00:00
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ) ; err != nil {
2023-06-25 06:38:44 +00:00
return err
}
log . Info ( "load mutil field done" ,
zap . Int64 ( "row count" , rowCount ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) )
return nil
}
2024-08-20 16:22:54 +00:00
func ( s * LocalSegment ) LoadFieldData ( ctx context . Context , fieldID int64 , rowCount int64 , field * datapb . FieldBinlog ) error {
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
return merr . WrapErrSegmentNotLoaded ( s . ID ( ) , "segment released" )
}
2023-08-16 10:38:17 +00:00
defer s . ptrLock . RUnlock ( )
2024-02-27 12:58:40 +00:00
2024-03-19 03:53:05 +00:00
ctx , sp := otel . Tracer ( typeutil . QueryNodeRole ) . Start ( ctx , fmt . Sprintf ( "LoadFieldData-%d-%d" , s . ID ( ) , fieldID ) )
2024-01-10 13:58:51 +00:00
defer sp . End ( )
2023-03-26 16:42:00 +00:00
2023-12-27 08:10:47 +00:00
log := log . Ctx ( ctx ) . With (
2023-03-26 16:42:00 +00:00
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
2023-08-31 04:03:00 +00:00
zap . Int64 ( "fieldID" , fieldID ) ,
zap . Int64 ( "rowCount" , rowCount ) ,
2023-03-26 16:42:00 +00:00
)
2023-08-31 04:03:00 +00:00
log . Info ( "start loading field data for field" )
2023-03-26 16:42:00 +00:00
2023-12-27 08:10:47 +00:00
loadFieldDataInfo , err := newLoadFieldDataInfo ( ctx )
2023-06-25 06:38:44 +00:00
defer deleteFieldDataInfo ( loadFieldDataInfo )
2023-03-26 16:42:00 +00:00
if err != nil {
return err
}
2023-12-27 08:10:47 +00:00
err = loadFieldDataInfo . appendLoadFieldInfo ( ctx , fieldID , rowCount )
2023-06-25 06:38:44 +00:00
if err != nil {
return err
2023-03-28 13:30:05 +00:00
}
2024-01-12 10:10:51 +00:00
if field != nil {
for _ , binlog := range field . Binlogs {
err = loadFieldDataInfo . appendLoadFieldDataPath ( ctx , fieldID , binlog )
if err != nil {
return err
}
2023-06-25 06:38:44 +00:00
}
2023-03-26 16:42:00 +00:00
}
2024-01-12 10:10:51 +00:00
2024-08-20 16:22:54 +00:00
// TODO retrieve_enable should be considered
2024-05-06 12:29:30 +00:00
collection := s . collection
2024-08-20 16:22:54 +00:00
fieldSchema , err := getFieldSchema ( collection . Schema ( ) , fieldID )
if err != nil {
return err
}
mmapEnabled := isDataMmapEnable ( fieldSchema )
2023-06-25 06:38:44 +00:00
loadFieldDataInfo . appendMMapDirPath ( paramtable . Get ( ) . QueryNodeCfg . MmapDirPath . GetValue ( ) )
2023-12-06 12:36:36 +00:00
loadFieldDataInfo . enableMmap ( fieldID , mmapEnabled )
2023-03-26 16:42:00 +00:00
2023-04-19 15:56:31 +00:00
var status C . CStatus
2023-11-17 11:56:21 +00:00
GetLoadPool ( ) . Submit ( func ( ) ( any , error ) {
2024-03-05 08:33:00 +00:00
log . Info ( "submitted loadFieldData task to load pool" )
2024-08-01 04:08:13 +00:00
status = C . LoadFieldData ( s . ptr , loadFieldDataInfo . cLoadFieldDataInfo )
2023-04-19 15:56:31 +00:00
return nil , nil
} ) . Await ( )
2023-12-27 08:10:47 +00:00
if err := HandleCStatus ( ctx , & status , "LoadFieldData failed" ,
2023-12-26 06:06:46 +00:00
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
zap . Int64 ( "fieldID" , fieldID ) ) ; err != nil {
2023-03-26 16:42:00 +00:00
return err
}
2023-08-31 04:03:00 +00:00
log . Info ( "load field done" )
2023-03-26 16:42:00 +00:00
return nil
}
2023-12-27 08:10:47 +00:00
func ( s * LocalSegment ) AddFieldDataInfo ( ctx context . Context , rowCount int64 , fields [ ] * datapb . FieldBinlog ) error {
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
2024-03-19 03:53:05 +00:00
return merr . WrapErrSegmentNotLoaded ( s . ID ( ) , "segment released" )
2023-09-15 02:21:20 +00:00
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2023-09-15 02:21:20 +00:00
2023-12-27 08:10:47 +00:00
log := log . Ctx ( ctx ) . With (
2023-09-15 02:21:20 +00:00
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
zap . Int64 ( "row count" , rowCount ) ,
)
2023-12-27 08:10:47 +00:00
loadFieldDataInfo , err := newLoadFieldDataInfo ( ctx )
2023-09-15 02:21:20 +00:00
if err != nil {
return err
}
2024-03-05 08:33:00 +00:00
defer deleteFieldDataInfo ( loadFieldDataInfo )
2023-09-15 02:21:20 +00:00
for _ , field := range fields {
fieldID := field . FieldID
2023-12-27 08:10:47 +00:00
err = loadFieldDataInfo . appendLoadFieldInfo ( ctx , fieldID , rowCount )
2023-09-15 02:21:20 +00:00
if err != nil {
return err
}
for _ , binlog := range field . Binlogs {
2023-12-27 08:10:47 +00:00
err = loadFieldDataInfo . appendLoadFieldDataPath ( ctx , fieldID , binlog )
2023-09-15 02:21:20 +00:00
if err != nil {
return err
}
}
}
var status C . CStatus
2023-11-17 11:56:21 +00:00
GetLoadPool ( ) . Submit ( func ( ) ( any , error ) {
2023-09-15 02:21:20 +00:00
status = C . AddFieldDataInfoForSealed ( s . ptr , loadFieldDataInfo . cLoadFieldDataInfo )
return nil , nil
} ) . Await ( )
2023-12-27 08:10:47 +00:00
if err := HandleCStatus ( ctx , & status , "AddFieldDataInfo failed" ,
2023-12-26 06:06:46 +00:00
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ) ; err != nil {
2023-09-15 02:21:20 +00:00
return err
}
log . Info ( "add field data info done" )
return nil
}
2023-12-27 08:10:47 +00:00
func ( s * LocalSegment ) LoadDeltaData ( ctx context . Context , deltaData * storage . DeleteData ) error {
2023-03-26 16:42:00 +00:00
pks , tss := deltaData . Pks , deltaData . Tss
rowNum := deltaData . RowCount
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
2024-03-19 03:53:05 +00:00
return merr . WrapErrSegmentNotLoaded ( s . ID ( ) , "segment released" )
2023-03-26 16:42:00 +00:00
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2023-03-26 16:42:00 +00:00
2023-12-27 08:10:47 +00:00
log := log . Ctx ( ctx ) . With (
2023-03-26 16:42:00 +00:00
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
)
pkType := pks [ 0 ] . Type ( )
ids := & schemapb . IDs { }
switch pkType {
case schemapb . DataType_Int64 :
int64Pks := make ( [ ] int64 , len ( pks ) )
for index , pk := range pks {
int64Pks [ index ] = pk . ( * storage . Int64PrimaryKey ) . Value
}
ids . IdField = & schemapb . IDs_IntId {
IntId : & schemapb . LongArray {
Data : int64Pks ,
} ,
}
case schemapb . DataType_VarChar :
varCharPks := make ( [ ] string , len ( pks ) )
for index , pk := range pks {
varCharPks [ index ] = pk . ( * storage . VarCharPrimaryKey ) . Value
}
ids . IdField = & schemapb . IDs_StrId {
StrId : & schemapb . StringArray {
Data : varCharPks ,
} ,
}
default :
return fmt . Errorf ( "invalid data type of primary keys" )
}
idsBlob , err := proto . Marshal ( ids )
if err != nil {
return err
}
loadInfo := C . CLoadDeletedRecordInfo {
timestamps : unsafe . Pointer ( & tss [ 0 ] ) ,
primary_keys : ( * C . uint8_t ) ( unsafe . Pointer ( & idsBlob [ 0 ] ) ) ,
primary_keys_size : C . uint64_t ( len ( idsBlob ) ) ,
row_count : C . int64_t ( rowNum ) ,
}
/ *
CStatus
LoadDeletedRecord ( CSegmentInterface c_segment , CLoadDeletedRecordInfo deleted_record_info )
* /
2023-04-19 15:56:31 +00:00
var status C . CStatus
2023-08-01 01:19:05 +00:00
GetDynamicPool ( ) . Submit ( func ( ) ( any , error ) {
2023-04-19 15:56:31 +00:00
status = C . LoadDeletedRecord ( s . ptr , loadInfo )
return nil , nil
} ) . Await ( )
2023-12-27 08:10:47 +00:00
if err := HandleCStatus ( ctx , & status , "LoadDeletedRecord failed" ,
2023-12-26 06:06:46 +00:00
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ) ; err != nil {
2023-03-26 16:42:00 +00:00
return err
}
2023-10-30 09:54:14 +00:00
s . rowNum . Store ( - 1 )
2023-09-19 08:21:23 +00:00
s . lastDeltaTimestamp . Store ( tss [ len ( tss ) - 1 ] )
2023-03-26 16:42:00 +00:00
log . Info ( "load deleted record done" ,
zap . Int64 ( "rowNum" , rowNum ) ,
zap . String ( "segmentType" , s . Type ( ) . String ( ) ) )
return nil
}
2024-05-06 12:29:30 +00:00
func ( s * LocalSegment ) LoadIndex ( ctx context . Context , indexInfo * querypb . FieldIndexInfo , fieldType schemapb . DataType ) error {
2024-04-10 07:15:19 +00:00
log := log . Ctx ( ctx ) . With (
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
zap . Int64 ( "fieldID" , indexInfo . GetFieldID ( ) ) ,
zap . Int64 ( "indexID" , indexInfo . GetIndexID ( ) ) ,
)
2024-02-27 12:58:40 +00:00
old := s . GetIndex ( indexInfo . GetFieldID ( ) )
// the index loaded
2024-05-06 12:29:30 +00:00
if old != nil && old . IndexInfo . GetIndexID ( ) == indexInfo . GetIndexID ( ) && old . IsLoaded {
2024-02-27 12:58:40 +00:00
log . Warn ( "index already loaded" )
return nil
}
2024-03-19 03:53:05 +00:00
ctx , sp := otel . Tracer ( typeutil . QueryNodeRole ) . Start ( ctx , fmt . Sprintf ( "LoadIndex-%d-%d" , s . ID ( ) , indexInfo . GetFieldID ( ) ) )
2024-01-10 13:58:51 +00:00
defer sp . End ( )
2023-03-26 16:42:00 +00:00
2024-05-06 12:29:30 +00:00
tr := timerecord . NewTimeRecorder ( "loadIndex" )
// 1.
2024-01-11 03:52:49 +00:00
loadIndexInfo , err := newLoadIndexInfo ( ctx )
if err != nil {
return err
}
defer deleteLoadIndexInfo ( loadIndexInfo )
2024-05-31 01:47:47 +00:00
schema , err := typeutil . CreateSchemaHelper ( s . GetCollection ( ) . Schema ( ) )
if err != nil {
return err
}
fieldSchema , err := schema . GetFieldFromID ( indexInfo . GetFieldID ( ) )
if err != nil {
return err
}
indexParams := funcutil . KeyValuePair2Map ( indexInfo . IndexParams )
// as Knowhere reports error if encounter an unknown param, we need to delete it
delete ( indexParams , common . MmapEnabledKey )
// some build params also exist in indexParams, which are useless during loading process
if indexParams [ "index_type" ] == indexparamcheck . IndexDISKANN {
if err := indexparams . SetDiskIndexLoadParams ( paramtable . Get ( ) , indexParams , indexInfo . GetNumRows ( ) ) ; err != nil {
return err
}
}
if err := indexparams . AppendPrepareLoadParams ( paramtable . Get ( ) , indexParams ) ; err != nil {
return err
}
2024-08-20 16:22:54 +00:00
enableMmap := isIndexMmapEnable ( fieldSchema , indexInfo )
2024-05-31 01:47:47 +00:00
indexInfoProto := & cgopb . LoadIndexInfo {
CollectionID : s . Collection ( ) ,
PartitionID : s . Partition ( ) ,
SegmentID : s . ID ( ) ,
Field : fieldSchema ,
2024-08-20 16:22:54 +00:00
EnableMmap : enableMmap ,
2024-05-31 01:47:47 +00:00
MmapDirPath : paramtable . Get ( ) . QueryNodeCfg . MmapDirPath . GetValue ( ) ,
IndexID : indexInfo . GetIndexID ( ) ,
IndexBuildID : indexInfo . GetBuildID ( ) ,
IndexVersion : indexInfo . GetIndexVersion ( ) ,
IndexParams : indexParams ,
IndexFiles : indexInfo . GetIndexFilePaths ( ) ,
IndexEngineVersion : indexInfo . GetCurrentIndexVersion ( ) ,
IndexStoreVersion : indexInfo . GetIndexStoreVersion ( ) ,
}
2024-05-06 12:29:30 +00:00
newLoadIndexInfoSpan := tr . RecordSpan ( )
// 2.
2024-05-31 01:47:47 +00:00
if err := loadIndexInfo . finish ( ctx , indexInfoProto ) ; err != nil {
2023-12-27 08:10:47 +00:00
if loadIndexInfo . cleanLocalData ( ctx ) != nil {
2023-03-26 16:42:00 +00:00
log . Warn ( "failed to clean cached data on disk after append index failed" ,
zap . Int64 ( "buildID" , indexInfo . BuildID ) ,
zap . Int64 ( "index version" , indexInfo . IndexVersion ) )
}
return err
}
if s . Type ( ) != SegmentTypeSealed {
2024-03-19 03:53:05 +00:00
errMsg := fmt . Sprintln ( "updateSegmentIndex failed, illegal segment type " , s . segmentType , "segmentID = " , s . ID ( ) )
2023-03-26 16:42:00 +00:00
return errors . New ( errMsg )
}
2024-05-06 12:29:30 +00:00
appendLoadIndexInfoSpan := tr . RecordSpan ( )
2023-03-26 16:42:00 +00:00
2024-05-06 12:29:30 +00:00
// 3.
2024-01-26 02:09:00 +00:00
err = s . UpdateIndexInfo ( ctx , indexInfo , loadIndexInfo )
2024-01-25 11:55:39 +00:00
if err != nil {
return err
}
2024-05-06 12:29:30 +00:00
updateIndexInfoSpan := tr . RecordSpan ( )
2024-01-25 11:55:39 +00:00
if ! typeutil . IsVectorType ( fieldType ) || s . HasRawData ( indexInfo . GetFieldID ( ) ) {
return nil
}
2024-05-06 12:29:30 +00:00
// 4.
2024-08-25 07:42:58 +00:00
s . WarmupChunkCache ( ctx , indexInfo . GetFieldID ( ) , isDataMmapEnable ( fieldSchema ) )
2024-05-06 12:29:30 +00:00
warmupChunkCacheSpan := tr . RecordSpan ( )
log . Info ( "Finish loading index" ,
zap . Duration ( "newLoadIndexInfoSpan" , newLoadIndexInfoSpan ) ,
zap . Duration ( "appendLoadIndexInfoSpan" , appendLoadIndexInfoSpan ) ,
zap . Duration ( "updateIndexInfoSpan" , updateIndexInfoSpan ) ,
2024-05-27 11:06:11 +00:00
zap . Duration ( "warmupChunkCacheSpan" , warmupChunkCacheSpan ) ,
2024-05-06 12:29:30 +00:00
)
2024-01-25 11:55:39 +00:00
return nil
2023-07-18 02:51:19 +00:00
}
2023-03-26 16:42:00 +00:00
2024-01-26 02:09:00 +00:00
func ( s * LocalSegment ) UpdateIndexInfo ( ctx context . Context , indexInfo * querypb . FieldIndexInfo , info * LoadIndexInfo ) error {
2024-01-05 06:24:49 +00:00
log := log . Ctx ( ctx ) . With (
2023-03-26 16:42:00 +00:00
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
2023-07-18 02:51:19 +00:00
zap . Int64 ( "fieldID" , indexInfo . FieldID ) ,
2023-03-26 16:42:00 +00:00
)
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
2024-03-19 03:53:05 +00:00
return merr . WrapErrSegmentNotLoaded ( s . ID ( ) , "segment released" )
2023-07-18 02:51:19 +00:00
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2023-03-26 16:42:00 +00:00
2023-04-19 15:56:31 +00:00
var status C . CStatus
2024-01-26 02:09:00 +00:00
GetDynamicPool ( ) . Submit ( func ( ) ( any , error ) {
2023-07-18 02:51:19 +00:00
status = C . UpdateSealedSegmentIndex ( s . ptr , info . cLoadIndexInfo )
2023-04-19 15:56:31 +00:00
return nil , nil
} ) . Await ( )
2023-12-27 08:10:47 +00:00
if err := HandleCStatus ( ctx , & status , "UpdateSealedSegmentIndex failed" ,
2023-12-26 06:06:46 +00:00
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
zap . Int64 ( "fieldID" , indexInfo . FieldID ) ) ; err != nil {
2023-03-26 16:42:00 +00:00
return err
}
2024-01-17 07:06:53 +00:00
2024-05-06 12:29:30 +00:00
s . fieldIndexes . Insert ( indexInfo . GetFieldID ( ) , & IndexedFieldInfo {
2024-01-17 07:06:53 +00:00
FieldBinlog : & datapb . FieldBinlog {
FieldID : indexInfo . GetFieldID ( ) ,
} ,
IndexInfo : indexInfo ,
2024-05-06 12:29:30 +00:00
IsLoaded : true ,
2024-01-17 07:06:53 +00:00
} )
2023-07-18 02:51:19 +00:00
log . Info ( "updateSegmentIndex done" )
2023-03-26 16:42:00 +00:00
return nil
}
2023-08-10 06:11:15 +00:00
2024-08-25 07:42:58 +00:00
func ( s * LocalSegment ) WarmupChunkCache ( ctx context . Context , fieldID int64 , mmapEnabled bool ) {
2024-01-25 11:55:39 +00:00
log := log . Ctx ( ctx ) . With (
zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
zap . Int64 ( "fieldID" , fieldID ) ,
2024-08-25 07:42:58 +00:00
zap . Bool ( "mmapEnabled" , mmapEnabled ) ,
2024-01-25 11:55:39 +00:00
)
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
2024-01-25 11:55:39 +00:00
return
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2024-01-25 11:55:39 +00:00
var status C . CStatus
warmingUp := strings . ToLower ( paramtable . Get ( ) . QueryNodeCfg . ChunkCacheWarmingUp . GetValue ( ) )
switch warmingUp {
case "sync" :
2024-05-26 17:25:40 +00:00
GetWarmupPool ( ) . Submit ( func ( ) ( any , error ) {
2024-01-25 11:55:39 +00:00
cFieldID := C . int64_t ( fieldID )
2024-08-25 07:42:58 +00:00
cMmapEnabled := C . bool ( mmapEnabled )
status = C . WarmupChunkCache ( s . ptr , cFieldID , cMmapEnabled )
2024-01-25 11:55:39 +00:00
if err := HandleCStatus ( ctx , & status , "warming up chunk cache failed" ) ; err != nil {
log . Warn ( "warming up chunk cache synchronously failed" , zap . Error ( err ) )
return nil , err
}
log . Info ( "warming up chunk cache synchronously done" )
return nil , nil
} ) . Await ( )
case "async" :
2024-05-26 17:25:40 +00:00
GetWarmupPool ( ) . Submit ( func ( ) ( any , error ) {
2024-06-10 13:59:53 +00:00
// bad implemtation, warmup is async at another goroutine and hold the rlock.
// the state transition of segment in segment loader will blocked.
// add a waiter to avoid it.
s . ptrLock . BlockUntilDataLoadedOrReleased ( )
2024-04-08 09:09:16 +00:00
if ! s . ptrLock . RLockIf ( state . IsNotReleased ) {
2024-01-25 11:55:39 +00:00
return nil , nil
}
2024-04-08 09:09:16 +00:00
defer s . ptrLock . RUnlock ( )
2024-01-25 11:55:39 +00:00
cFieldID := C . int64_t ( fieldID )
2024-08-25 07:42:58 +00:00
cMmapEnabled := C . bool ( mmapEnabled )
status = C . WarmupChunkCache ( s . ptr , cFieldID , cMmapEnabled )
2024-01-25 11:55:39 +00:00
if err := HandleCStatus ( ctx , & status , "" ) ; err != nil {
log . Warn ( "warming up chunk cache asynchronously failed" , zap . Error ( err ) )
return nil , err
}
log . Info ( "warming up chunk cache asynchronously done" )
return nil , nil
} )
default :
// no warming up
}
}
2023-12-27 08:10:47 +00:00
func ( s * LocalSegment ) UpdateFieldRawDataSize ( ctx context . Context , numRows int64 , fieldBinlog * datapb . FieldBinlog ) error {
2023-08-10 06:11:15 +00:00
var status C . CStatus
fieldID := fieldBinlog . FieldID
fieldDataSize := int64 ( 0 )
for _ , binlog := range fieldBinlog . GetBinlogs ( ) {
2024-05-15 04:59:34 +00:00
fieldDataSize += binlog . GetMemorySize ( )
2023-08-10 06:11:15 +00:00
}
GetDynamicPool ( ) . Submit ( func ( ) ( any , error ) {
status = C . UpdateFieldRawDataSize ( s . ptr , C . int64_t ( fieldID ) , C . int64_t ( numRows ) , C . int64_t ( fieldDataSize ) )
return nil , nil
} ) . Await ( )
2023-12-27 08:10:47 +00:00
if err := HandleCStatus ( ctx , & status , "updateFieldRawDataSize failed" ) ; err != nil {
2023-08-10 06:11:15 +00:00
return err
}
2024-01-05 06:24:49 +00:00
log . Ctx ( ctx ) . Info ( "updateFieldRawDataSize done" , zap . Int64 ( "segmentID" , s . ID ( ) ) )
2023-08-10 06:11:15 +00:00
return nil
}
2023-09-09 02:35:16 +00:00
2024-02-27 12:58:40 +00:00
type ReleaseScope int
const (
ReleaseScopeAll ReleaseScope = iota
ReleaseScopeData
)
type releaseOptions struct {
Scope ReleaseScope
}
func newReleaseOptions ( ) * releaseOptions {
return & releaseOptions {
Scope : ReleaseScopeAll ,
}
}
type releaseOption func ( * releaseOptions )
func WithReleaseScope ( scope ReleaseScope ) releaseOption {
return func ( options * releaseOptions ) {
options . Scope = scope
}
}
2024-05-06 12:29:30 +00:00
func ( s * LocalSegment ) Release ( ctx context . Context , opts ... releaseOption ) {
2024-02-27 12:58:40 +00:00
options := newReleaseOptions ( )
for _ , opt := range opts {
opt ( options )
}
2024-04-08 09:09:16 +00:00
stateLockGuard := s . startRelease ( options . Scope )
if stateLockGuard == nil { // release is already done.
return
}
// release will never fail
defer stateLockGuard . Done ( nil )
2023-09-09 02:35:16 +00:00
2024-05-06 12:29:30 +00:00
log := log . Ctx ( ctx ) . With ( zap . Int64 ( "collectionID" , s . Collection ( ) ) ,
2024-04-10 07:15:19 +00:00
zap . Int64 ( "partitionID" , s . Partition ( ) ) ,
zap . Int64 ( "segmentID" , s . ID ( ) ) ,
zap . String ( "segmentType" , s . segmentType . String ( ) ) ,
zap . Int64 ( "insertCount" , s . InsertCount ( ) ) ,
)
2024-03-05 08:33:00 +00:00
// wait all read ops finished
2024-04-08 09:09:16 +00:00
ptr := s . ptr
2024-02-27 12:58:40 +00:00
if options . Scope == ReleaseScopeData {
2024-05-06 12:29:30 +00:00
s . ReleaseSegmentData ( )
log . Info ( "release segment data done and the field indexes info has been set lazy load=true" )
2024-02-27 12:58:40 +00:00
return
}
2023-09-09 02:35:16 +00:00
C . DeleteSegment ( ptr )
2024-01-10 02:00:51 +00:00
2024-02-20 14:14:53 +00:00
localDiskUsage , err := GetLocalUsedSize ( context . Background ( ) , paramtable . Get ( ) . LocalStorageCfg . Path . GetValue ( ) )
// ignore error here, shall not block releasing
if err == nil {
metrics . QueryNodeDiskUsedSize . WithLabelValues ( fmt . Sprint ( paramtable . GetNodeID ( ) ) ) . Set ( float64 ( localDiskUsage ) / 1024 / 1024 ) // in MB
}
2024-04-10 07:15:19 +00:00
log . Info ( "delete segment from memory" )
2023-09-09 02:35:16 +00:00
}
2024-04-08 09:09:16 +00:00
2024-05-06 12:29:30 +00:00
// ReleaseSegmentData releases the segment data.
func ( s * LocalSegment ) ReleaseSegmentData ( ) {
C . ClearSegmentData ( s . ptr )
for _ , indexInfo := range s . Indexes ( ) {
indexInfo . IsLoaded = false
}
}
2024-04-08 09:09:16 +00:00
// StartLoadData starts the loading process of the segment.
func ( s * LocalSegment ) StartLoadData ( ) ( state . LoadStateLockGuard , error ) {
return s . ptrLock . StartLoadData ( )
}
// startRelease starts the releasing process of the segment.
func ( s * LocalSegment ) startRelease ( scope ReleaseScope ) state . LoadStateLockGuard {
switch scope {
case ReleaseScopeData :
return s . ptrLock . StartReleaseData ( )
case ReleaseScopeAll :
return s . ptrLock . StartReleaseAll ( )
default :
panic ( fmt . Sprintf ( "unexpected release scope %d" , scope ) )
}
}
2024-05-06 12:29:30 +00:00
func ( s * LocalSegment ) RemoveFieldFile ( fieldId int64 ) {
C . RemoveFieldFile ( s . ptr , C . int64_t ( fieldId ) )
}
func ( s * LocalSegment ) RemoveUnusedFieldFiles ( ) error {
schema := s . collection . Schema ( )
indexInfos , _ := separateIndexAndBinlog ( s . LoadInfo ( ) )
for _ , indexInfo := range indexInfos {
need , err := s . indexNeedLoadRawData ( schema , indexInfo )
if err != nil {
return err
}
if ! need {
s . RemoveFieldFile ( indexInfo . IndexInfo . FieldID )
}
}
return nil
}
func ( s * LocalSegment ) indexNeedLoadRawData ( schema * schemapb . CollectionSchema , indexInfo * IndexedFieldInfo ) ( bool , error ) {
schemaHelper , err := typeutil . CreateSchemaHelper ( schema )
if err != nil {
return false , err
}
fieldSchema , err := schemaHelper . GetFieldFromID ( indexInfo . IndexInfo . FieldID )
if err != nil {
return false , err
}
return ! typeutil . IsVectorType ( fieldSchema . DataType ) && s . HasRawData ( indexInfo . IndexInfo . FieldID ) , nil
}