2021-04-19 11:35:38 +08:00
|
|
|
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
|
|
|
|
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
|
|
|
|
// with the License. You may obtain a copy of the License at
|
|
|
|
//
|
|
|
|
// http://www.apache.org/licenses/LICENSE-2.0
|
|
|
|
//
|
|
|
|
// Unless required by applicable law or agreed to in writing, software distributed under the License
|
|
|
|
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
|
|
|
|
// or implied. See the License for the specific language governing permissions and limitations under the License.
|
2021-06-08 19:25:37 +08:00
|
|
|
|
2021-06-22 10:42:07 +08:00
|
|
|
package datacoord
|
2021-01-15 17:09:41 +08:00
|
|
|
|
|
|
|
import (
|
2021-03-23 16:57:59 +08:00
|
|
|
"context"
|
2021-01-15 17:09:41 +08:00
|
|
|
"fmt"
|
|
|
|
"sync"
|
|
|
|
"time"
|
|
|
|
|
2021-03-27 11:35:42 +08:00
|
|
|
"go.uber.org/zap"
|
|
|
|
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/log"
|
|
|
|
"github.com/milvus-io/milvus/internal/proto/commonpb"
|
2021-06-03 19:06:33 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
2021-09-27 20:34:03 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/rootcoord"
|
2021-04-12 16:35:51 +08:00
|
|
|
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/util/trace"
|
|
|
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
2021-01-15 17:09:41 +08:00
|
|
|
|
2021-04-22 14:45:57 +08:00
|
|
|
"github.com/milvus-io/milvus/internal/proto/datapb"
|
2021-01-15 17:09:41 +08:00
|
|
|
)
|
|
|
|
|
2021-09-07 13:59:58 +08:00
|
|
|
var (
|
2021-09-16 19:55:49 +08:00
|
|
|
// allocPool pool of Allocation, to reduce allocation of Allocation
|
2021-09-07 13:59:58 +08:00
|
|
|
allocPool = sync.Pool{
|
|
|
|
New: func() interface{} {
|
|
|
|
return &Allocation{}
|
|
|
|
},
|
|
|
|
}
|
|
|
|
)
|
|
|
|
|
|
|
|
// getAllocation unified way to retrieve allocation struct
|
|
|
|
func getAllocation(numOfRows int64) *Allocation {
|
|
|
|
v := allocPool.Get()
|
|
|
|
a, ok := v.(*Allocation)
|
|
|
|
if !ok {
|
|
|
|
a = &Allocation{}
|
|
|
|
}
|
|
|
|
if a == nil {
|
|
|
|
return &Allocation{
|
|
|
|
NumOfRows: numOfRows,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
a.NumOfRows = numOfRows
|
|
|
|
a.ExpireTime = 0
|
|
|
|
a.SegmentID = 0
|
|
|
|
return a
|
|
|
|
}
|
|
|
|
|
|
|
|
// putAllocation put allocation for recycling
|
|
|
|
func putAllocation(a *Allocation) {
|
|
|
|
allocPool.Put(a)
|
|
|
|
}
|
|
|
|
|
2021-09-16 19:55:49 +08:00
|
|
|
// segmentMaxLifetime default segment max lifetime value
|
|
|
|
// TODO needs to be configurable
|
2021-08-20 15:42:12 +08:00
|
|
|
const segmentMaxLifetime = 24 * time.Hour
|
|
|
|
|
2021-06-03 19:06:33 +08:00
|
|
|
// Manager manage segment related operations.
|
|
|
|
type Manager interface {
|
2021-10-08 19:05:25 +08:00
|
|
|
// AllocSegment allocates rows and record the allocation.
|
2021-07-23 21:58:33 +08:00
|
|
|
AllocSegment(ctx context.Context, collectionID, partitionID UniqueID, channelName string, requestRows int64) ([]*Allocation, error)
|
2021-10-08 19:05:25 +08:00
|
|
|
// DropSegment drops the segment from manager.
|
2021-03-23 16:57:59 +08:00
|
|
|
DropSegment(ctx context.Context, segmentID UniqueID)
|
2021-10-08 19:05:25 +08:00
|
|
|
// SealAllSegments seals all segments of collection with collectionID and return sealed segments
|
2021-06-23 16:56:11 +08:00
|
|
|
SealAllSegments(ctx context.Context, collectionID UniqueID) ([]UniqueID, error)
|
2021-10-08 19:05:25 +08:00
|
|
|
// GetFlushableSegments returns flushable segment ids
|
2021-05-25 15:35:37 +08:00
|
|
|
GetFlushableSegments(ctx context.Context, channel string, ts Timestamp) ([]UniqueID, error)
|
2021-10-08 19:05:25 +08:00
|
|
|
// ExpireAllocations notifies segment status to expire old allocations
|
2021-06-24 14:20:10 +08:00
|
|
|
ExpireAllocations(channel string, ts Timestamp) error
|
2021-01-15 17:09:41 +08:00
|
|
|
}
|
|
|
|
|
2021-09-14 17:27:55 +08:00
|
|
|
// Allocation records the allocation info
|
2021-07-12 17:24:25 +08:00
|
|
|
type Allocation struct {
|
2021-07-23 21:58:33 +08:00
|
|
|
SegmentID UniqueID
|
|
|
|
NumOfRows int64
|
|
|
|
ExpireTime Timestamp
|
2021-06-03 19:06:33 +08:00
|
|
|
}
|
|
|
|
|
2021-10-02 14:45:56 +08:00
|
|
|
// make sure SegmentManager implements Manager
|
|
|
|
var _ Manager = (*SegmentManager)(nil)
|
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// SegmentManager handles segment related logic
|
2021-06-03 19:06:33 +08:00
|
|
|
type SegmentManager struct {
|
2021-07-12 17:24:25 +08:00
|
|
|
meta *meta
|
|
|
|
mu sync.RWMutex
|
|
|
|
allocator allocator
|
|
|
|
helper allocHelper
|
|
|
|
segments []UniqueID
|
|
|
|
estimatePolicy calUpperLimitPolicy
|
2021-07-23 21:58:33 +08:00
|
|
|
allocPolicy AllocatePolicy
|
2021-06-08 19:25:37 +08:00
|
|
|
segmentSealPolicies []segmentSealPolicy
|
|
|
|
channelSealPolicies []channelSealPolicy
|
|
|
|
flushPolicy flushPolicy
|
2021-04-12 16:35:51 +08:00
|
|
|
}
|
|
|
|
|
2021-05-21 18:30:41 +08:00
|
|
|
type allocHelper struct {
|
|
|
|
afterCreateSegment func(segment *datapb.SegmentInfo) error
|
|
|
|
}
|
2021-05-28 09:55:21 +08:00
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// allocOption allction option applies to `SegmentManager`
|
|
|
|
type allocOption interface {
|
|
|
|
apply(manager *SegmentManager)
|
|
|
|
}
|
|
|
|
|
|
|
|
// allocFunc function shortcut for allocOption
|
|
|
|
type allocFunc func(manager *SegmentManager)
|
|
|
|
|
|
|
|
// implement allocOption
|
|
|
|
func (f allocFunc) apply(manager *SegmentManager) {
|
|
|
|
f(manager)
|
2021-03-05 16:52:45 +08:00
|
|
|
}
|
2021-01-15 17:09:41 +08:00
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// get allocOption with allocHelper setting
|
2021-05-21 18:30:41 +08:00
|
|
|
func withAllocHelper(helper allocHelper) allocOption {
|
2021-06-24 14:20:10 +08:00
|
|
|
return allocFunc(func(manager *SegmentManager) { manager.helper = helper })
|
2021-04-12 16:35:51 +08:00
|
|
|
}
|
2021-05-21 18:30:41 +08:00
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// get default allocHelper, which does nothing
|
2021-05-21 18:30:41 +08:00
|
|
|
func defaultAllocHelper() allocHelper {
|
|
|
|
return allocHelper{
|
|
|
|
afterCreateSegment: func(segment *datapb.SegmentInfo) error { return nil },
|
2021-01-15 17:09:41 +08:00
|
|
|
}
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// get allocOption with estimatePolicy
|
2021-05-21 18:30:41 +08:00
|
|
|
func withCalUpperLimitPolicy(policy calUpperLimitPolicy) allocOption {
|
2021-06-24 14:20:10 +08:00
|
|
|
return allocFunc(func(manager *SegmentManager) { manager.estimatePolicy = policy })
|
2021-01-15 17:09:41 +08:00
|
|
|
}
|
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// get allocOption with allocPolicy
|
2021-07-23 21:58:33 +08:00
|
|
|
func withAllocPolicy(policy AllocatePolicy) allocOption {
|
2021-06-24 14:20:10 +08:00
|
|
|
return allocFunc(func(manager *SegmentManager) { manager.allocPolicy = policy })
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// get allocOption with segmentSealPolicies
|
2021-06-08 19:25:37 +08:00
|
|
|
func withSegmentSealPolices(policies ...segmentSealPolicy) allocOption {
|
2021-06-24 14:20:10 +08:00
|
|
|
return allocFunc(func(manager *SegmentManager) {
|
|
|
|
// do override instead of append, to override default options
|
|
|
|
manager.segmentSealPolicies = policies
|
|
|
|
})
|
2021-06-08 19:25:37 +08:00
|
|
|
}
|
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// get allocOption with channelSealPolicies
|
2021-06-08 19:25:37 +08:00
|
|
|
func withChannelSealPolices(policies ...channelSealPolicy) allocOption {
|
2021-06-24 14:20:10 +08:00
|
|
|
return allocFunc(func(manager *SegmentManager) {
|
|
|
|
// do override instead of append, to override default options
|
|
|
|
manager.channelSealPolicies = policies
|
|
|
|
})
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// get allocOption with flushPolicy
|
2021-05-21 18:30:41 +08:00
|
|
|
func withFlushPolicy(policy flushPolicy) allocOption {
|
2021-06-24 14:20:10 +08:00
|
|
|
return allocFunc(func(manager *SegmentManager) { manager.flushPolicy = policy })
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
func defaultCalUpperLimitPolicy() calUpperLimitPolicy {
|
2021-07-12 17:24:25 +08:00
|
|
|
return calBySchemaPolicy
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
|
|
|
|
2021-07-23 21:58:33 +08:00
|
|
|
func defaultAlocatePolicy() AllocatePolicy {
|
|
|
|
return AllocatePolicyV1
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
|
|
|
|
2021-08-20 15:42:12 +08:00
|
|
|
func defaultSegmentSealPolicy() []segmentSealPolicy {
|
|
|
|
return []segmentSealPolicy{
|
|
|
|
sealByLifetimePolicy(segmentMaxLifetime),
|
|
|
|
getSegmentCapacityPolicy(Params.SegmentSealProportion),
|
|
|
|
}
|
2021-06-08 19:25:37 +08:00
|
|
|
}
|
|
|
|
|
2021-05-21 18:30:41 +08:00
|
|
|
func defaultFlushPolicy() flushPolicy {
|
2021-07-12 17:24:25 +08:00
|
|
|
return flushPolicyV1
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// newSegmentManager should be the only way to retrieve SegmentManager
|
2021-06-03 19:06:33 +08:00
|
|
|
func newSegmentManager(meta *meta, allocator allocator, opts ...allocOption) *SegmentManager {
|
|
|
|
manager := &SegmentManager{
|
2021-07-12 17:24:25 +08:00
|
|
|
meta: meta,
|
|
|
|
allocator: allocator,
|
|
|
|
helper: defaultAllocHelper(),
|
|
|
|
segments: make([]UniqueID, 0),
|
2021-06-08 19:25:37 +08:00
|
|
|
estimatePolicy: defaultCalUpperLimitPolicy(),
|
|
|
|
allocPolicy: defaultAlocatePolicy(),
|
2021-08-20 15:42:12 +08:00
|
|
|
segmentSealPolicies: defaultSegmentSealPolicy(), // default only segment size policy
|
|
|
|
channelSealPolicies: []channelSealPolicy{}, // no default channel seal policy
|
2021-06-08 19:25:37 +08:00
|
|
|
flushPolicy: defaultFlushPolicy(),
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
|
|
|
for _, opt := range opts {
|
2021-06-03 19:06:33 +08:00
|
|
|
opt.apply(manager)
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
2021-06-03 19:06:33 +08:00
|
|
|
manager.loadSegmentsFromMeta()
|
|
|
|
return manager
|
2021-01-15 17:09:41 +08:00
|
|
|
}
|
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// loadSegmentsFromMeta generate corresponding segment status for each segment from meta
|
2021-06-03 19:06:33 +08:00
|
|
|
func (s *SegmentManager) loadSegmentsFromMeta() {
|
|
|
|
segments := s.meta.GetUnFlushedSegments()
|
2021-07-12 17:24:25 +08:00
|
|
|
segmentsID := make([]UniqueID, 0, len(segments))
|
|
|
|
for _, segment := range segments {
|
|
|
|
segmentsID = append(segmentsID, segment.GetID())
|
2021-06-03 19:06:33 +08:00
|
|
|
}
|
2021-07-12 17:24:25 +08:00
|
|
|
s.segments = segmentsID
|
2021-06-03 19:06:33 +08:00
|
|
|
}
|
2021-06-24 14:20:10 +08:00
|
|
|
|
|
|
|
// AllocSegment allocate segment per request collcation, partication, channel and rows
|
2021-06-03 19:06:33 +08:00
|
|
|
func (s *SegmentManager) AllocSegment(ctx context.Context, collectionID UniqueID,
|
2021-07-23 21:58:33 +08:00
|
|
|
partitionID UniqueID, channelName string, requestRows int64) ([]*Allocation, error) {
|
2021-03-23 16:57:59 +08:00
|
|
|
sp, _ := trace.StartSpanFromContext(ctx)
|
|
|
|
defer sp.Finish()
|
2021-04-25 09:51:57 +08:00
|
|
|
s.mu.Lock()
|
|
|
|
defer s.mu.Unlock()
|
2021-01-15 17:09:41 +08:00
|
|
|
|
2021-07-23 21:58:33 +08:00
|
|
|
// filter segments
|
|
|
|
segments := make([]*SegmentInfo, 0)
|
2021-07-12 17:24:25 +08:00
|
|
|
for _, segmentID := range s.segments {
|
2021-07-23 21:58:33 +08:00
|
|
|
segment := s.meta.GetSegment(segmentID)
|
2021-07-12 17:24:25 +08:00
|
|
|
if segment == nil {
|
2021-07-23 21:58:33 +08:00
|
|
|
log.Warn("Failed to get seginfo from meta", zap.Int64("id", segmentID))
|
2021-06-03 19:06:33 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-07-12 17:24:25 +08:00
|
|
|
if segment.State == commonpb.SegmentState_Sealed || segment.CollectionID != collectionID ||
|
|
|
|
segment.PartitionID != partitionID || segment.InsertChannel != channelName {
|
2021-06-24 14:20:10 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-07-23 21:58:33 +08:00
|
|
|
segments = append(segments, segment)
|
2021-04-12 16:35:51 +08:00
|
|
|
}
|
|
|
|
|
2021-07-23 21:58:33 +08:00
|
|
|
// apply allocate policy
|
|
|
|
maxCountPerSegment, err := s.estimateMaxNumOfRows(collectionID)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
2021-01-15 17:09:41 +08:00
|
|
|
}
|
2021-07-23 21:58:33 +08:00
|
|
|
newSegmentAllocations, existedSegmentAllocations := s.allocPolicy(segments,
|
|
|
|
requestRows, int64(maxCountPerSegment))
|
2021-01-15 17:09:41 +08:00
|
|
|
|
2021-07-23 21:58:33 +08:00
|
|
|
// create new segments and add allocations
|
2021-08-23 17:59:51 +08:00
|
|
|
expireTs, err := s.genExpireTs(ctx)
|
2021-01-15 17:09:41 +08:00
|
|
|
if err != nil {
|
2021-07-12 17:24:25 +08:00
|
|
|
return nil, err
|
2021-01-15 17:09:41 +08:00
|
|
|
}
|
2021-07-23 21:58:33 +08:00
|
|
|
for _, allocation := range newSegmentAllocations {
|
|
|
|
segment, err := s.openNewSegment(ctx, collectionID, partitionID, channelName)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
allocation.ExpireTime = expireTs
|
|
|
|
allocation.SegmentID = segment.GetID()
|
|
|
|
if err := s.meta.AddAllocation(segment.GetID(), allocation); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, allocation := range existedSegmentAllocations {
|
|
|
|
allocation.ExpireTime = expireTs
|
|
|
|
if err := s.meta.AddAllocation(allocation.SegmentID, allocation); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
}
|
2021-06-24 14:20:10 +08:00
|
|
|
|
2021-07-23 21:58:33 +08:00
|
|
|
allocations := append(newSegmentAllocations, existedSegmentAllocations...)
|
|
|
|
return allocations, nil
|
2021-01-15 17:09:41 +08:00
|
|
|
}
|
|
|
|
|
2021-08-23 17:59:51 +08:00
|
|
|
func (s *SegmentManager) genExpireTs(ctx context.Context) (Timestamp, error) {
|
|
|
|
ts, err := s.allocator.allocTimestamp(ctx)
|
2021-05-21 18:30:41 +08:00
|
|
|
if err != nil {
|
|
|
|
return 0, err
|
|
|
|
}
|
|
|
|
physicalTs, logicalTs := tsoutil.ParseTS(ts)
|
2021-06-22 16:18:21 +08:00
|
|
|
expirePhysicalTs := physicalTs.Add(time.Duration(Params.SegAssignmentExpiration) * time.Millisecond)
|
2021-05-21 18:30:41 +08:00
|
|
|
expireTs := tsoutil.ComposeTS(expirePhysicalTs.UnixNano()/int64(time.Millisecond), int64(logicalTs))
|
|
|
|
return expireTs, nil
|
|
|
|
}
|
|
|
|
|
2021-07-12 17:24:25 +08:00
|
|
|
func (s *SegmentManager) openNewSegment(ctx context.Context, collectionID UniqueID, partitionID UniqueID, channelName string) (*SegmentInfo, error) {
|
2021-04-12 16:35:51 +08:00
|
|
|
sp, _ := trace.StartSpanFromContext(ctx)
|
|
|
|
defer sp.Finish()
|
2021-08-23 17:59:51 +08:00
|
|
|
id, err := s.allocator.allocID(ctx)
|
2021-04-12 16:35:51 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2021-06-22 18:24:08 +08:00
|
|
|
maxNumOfRows, err := s.estimateMaxNumOfRows(collectionID)
|
2021-04-12 16:35:51 +08:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2021-06-04 11:45:45 +08:00
|
|
|
|
2021-09-27 20:34:03 +08:00
|
|
|
startPosition := []byte{} // default start position
|
|
|
|
coll := s.meta.GetCollection(collectionID)
|
|
|
|
for _, pair := range coll.GetStartPositions() {
|
|
|
|
if pair.Key == rootcoord.ToPhysicalChannel(channelName) { // pchan or vchan
|
|
|
|
startPosition = pair.Data
|
|
|
|
break
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-05-21 18:30:41 +08:00
|
|
|
segmentInfo := &datapb.SegmentInfo{
|
2021-05-31 18:47:32 +08:00
|
|
|
ID: id,
|
|
|
|
CollectionID: collectionID,
|
|
|
|
PartitionID: partitionID,
|
|
|
|
InsertChannel: channelName,
|
2021-06-04 11:45:45 +08:00
|
|
|
NumOfRows: 0,
|
2021-05-31 18:47:32 +08:00
|
|
|
State: commonpb.SegmentState_Growing,
|
2021-06-22 18:24:08 +08:00
|
|
|
MaxRowNum: int64(maxNumOfRows),
|
2021-05-21 18:30:41 +08:00
|
|
|
LastExpireTime: 0,
|
2021-06-15 11:06:42 +08:00
|
|
|
StartPosition: &internalpb.MsgPosition{
|
|
|
|
ChannelName: channelName,
|
2021-09-27 20:34:03 +08:00
|
|
|
MsgID: startPosition,
|
2021-06-15 11:06:42 +08:00
|
|
|
MsgGroup: "",
|
|
|
|
Timestamp: 0,
|
|
|
|
},
|
2021-04-12 16:35:51 +08:00
|
|
|
}
|
2021-07-12 17:24:25 +08:00
|
|
|
segment := NewSegmentInfo(segmentInfo)
|
|
|
|
if err := s.meta.AddSegment(segment); err != nil {
|
2021-05-21 18:30:41 +08:00
|
|
|
return nil, err
|
2021-04-12 16:35:51 +08:00
|
|
|
}
|
2021-07-12 17:24:25 +08:00
|
|
|
s.segments = append(s.segments, id)
|
2021-06-21 18:22:13 +08:00
|
|
|
log.Debug("datacoord: estimateTotalRows: ",
|
2021-05-21 18:30:41 +08:00
|
|
|
zap.Int64("CollectionID", segmentInfo.CollectionID),
|
|
|
|
zap.Int64("SegmentID", segmentInfo.ID),
|
2021-06-22 18:24:08 +08:00
|
|
|
zap.Int("Rows", maxNumOfRows),
|
|
|
|
zap.String("Channel", segmentInfo.InsertChannel))
|
2021-05-21 18:30:41 +08:00
|
|
|
|
2021-09-27 22:46:12 +08:00
|
|
|
return segment, s.helper.afterCreateSegment(segmentInfo)
|
2021-04-12 16:35:51 +08:00
|
|
|
}
|
|
|
|
|
2021-06-22 18:24:08 +08:00
|
|
|
func (s *SegmentManager) estimateMaxNumOfRows(collectionID UniqueID) (int, error) {
|
2021-07-07 14:02:01 +08:00
|
|
|
collMeta := s.meta.GetCollection(collectionID)
|
|
|
|
if collMeta == nil {
|
2021-09-14 17:33:48 +08:00
|
|
|
return -1, fmt.Errorf("failed to get collection %d", collectionID)
|
2021-01-15 17:09:41 +08:00
|
|
|
}
|
2021-07-12 17:24:25 +08:00
|
|
|
return s.estimatePolicy(collMeta.Schema)
|
2021-01-15 17:09:41 +08:00
|
|
|
}
|
|
|
|
|
2021-10-08 19:05:25 +08:00
|
|
|
// DropSegment drop the segment from manager.
|
2021-06-03 19:06:33 +08:00
|
|
|
func (s *SegmentManager) DropSegment(ctx context.Context, segmentID UniqueID) {
|
2021-03-23 16:57:59 +08:00
|
|
|
sp, _ := trace.StartSpanFromContext(ctx)
|
|
|
|
defer sp.Finish()
|
2021-04-25 09:51:57 +08:00
|
|
|
s.mu.Lock()
|
|
|
|
defer s.mu.Unlock()
|
2021-07-12 17:24:25 +08:00
|
|
|
for i, id := range s.segments {
|
|
|
|
if id == segmentID {
|
|
|
|
s.segments = append(s.segments[:i], s.segments[i+1:]...)
|
|
|
|
break
|
2021-06-24 14:20:10 +08:00
|
|
|
}
|
|
|
|
}
|
2021-07-12 17:24:25 +08:00
|
|
|
segment := s.meta.GetSegment(segmentID)
|
|
|
|
if segment == nil {
|
|
|
|
log.Warn("failed to get segment", zap.Int64("id", segmentID))
|
2021-10-08 19:05:25 +08:00
|
|
|
return
|
2021-07-12 17:24:25 +08:00
|
|
|
}
|
|
|
|
s.meta.SetAllocations(segmentID, []*Allocation{})
|
|
|
|
for _, allocation := range segment.allocations {
|
2021-09-07 13:59:58 +08:00
|
|
|
putAllocation(allocation)
|
2021-07-12 17:24:25 +08:00
|
|
|
}
|
2021-01-15 17:09:41 +08:00
|
|
|
}
|
|
|
|
|
2021-10-08 19:05:25 +08:00
|
|
|
// SealAllSegments seals all segmetns of collection with collectionID and return sealed segments
|
2021-06-23 16:56:11 +08:00
|
|
|
func (s *SegmentManager) SealAllSegments(ctx context.Context, collectionID UniqueID) ([]UniqueID, error) {
|
2021-04-23 10:15:30 +08:00
|
|
|
sp, _ := trace.StartSpanFromContext(ctx)
|
|
|
|
defer sp.Finish()
|
2021-04-25 09:51:57 +08:00
|
|
|
s.mu.Lock()
|
|
|
|
defer s.mu.Unlock()
|
2021-06-23 16:56:11 +08:00
|
|
|
ret := make([]UniqueID, 0)
|
2021-07-12 17:24:25 +08:00
|
|
|
for _, id := range s.segments {
|
|
|
|
info := s.meta.GetSegment(id)
|
2021-07-07 14:02:01 +08:00
|
|
|
if info == nil {
|
2021-07-12 17:24:25 +08:00
|
|
|
log.Warn("Failed to get seg info from meta", zap.Int64("id", id))
|
2021-06-23 16:56:11 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-06-24 14:20:10 +08:00
|
|
|
if info.CollectionID != collectionID {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
if info.State == commonpb.SegmentState_Sealed {
|
2021-07-12 17:24:25 +08:00
|
|
|
ret = append(ret, id)
|
2021-06-08 19:25:37 +08:00
|
|
|
continue
|
2021-06-03 19:06:33 +08:00
|
|
|
}
|
2021-07-12 17:24:25 +08:00
|
|
|
if err := s.meta.SetState(id, commonpb.SegmentState_Sealed); err != nil {
|
2021-06-23 16:56:11 +08:00
|
|
|
return nil, err
|
2021-06-08 19:25:37 +08:00
|
|
|
}
|
2021-07-12 17:24:25 +08:00
|
|
|
ret = append(ret, id)
|
2021-06-03 19:06:33 +08:00
|
|
|
}
|
2021-06-23 16:56:11 +08:00
|
|
|
return ret, nil
|
2021-04-25 09:51:57 +08:00
|
|
|
}
|
|
|
|
|
2021-09-16 19:55:49 +08:00
|
|
|
// GetFlushableSegments get segment ids with Sealed State and flushable (meets flushPolicy)
|
2021-10-15 11:46:33 +08:00
|
|
|
func (s *SegmentManager) GetFlushableSegments(ctx context.Context, channel string, t Timestamp) ([]UniqueID, error) {
|
2021-04-25 09:51:57 +08:00
|
|
|
s.mu.Lock()
|
|
|
|
defer s.mu.Unlock()
|
2021-03-23 16:57:59 +08:00
|
|
|
sp, _ := trace.StartSpanFromContext(ctx)
|
|
|
|
defer sp.Finish()
|
2021-07-27 14:34:48 +08:00
|
|
|
if err := s.tryToSealSegment(t, channel); err != nil {
|
2021-05-21 18:30:41 +08:00
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2021-07-12 17:24:25 +08:00
|
|
|
ret := make([]UniqueID, 0, len(s.segments))
|
|
|
|
for _, id := range s.segments {
|
|
|
|
info := s.meta.GetSegment(id)
|
2021-07-27 14:34:48 +08:00
|
|
|
if info == nil || info.InsertChannel != channel {
|
2021-05-25 15:35:37 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-07-12 17:24:25 +08:00
|
|
|
if s.flushPolicy(info, t) {
|
|
|
|
ret = append(ret, id)
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return ret, nil
|
|
|
|
}
|
|
|
|
|
2021-06-24 14:20:10 +08:00
|
|
|
// ExpireAllocations notify segment status to expire old allocations
|
|
|
|
func (s *SegmentManager) ExpireAllocations(channel string, ts Timestamp) error {
|
|
|
|
s.mu.Lock()
|
|
|
|
defer s.mu.Unlock()
|
2021-07-12 17:24:25 +08:00
|
|
|
for _, id := range s.segments {
|
|
|
|
segment := s.meta.GetSegment(id)
|
2021-07-27 14:34:48 +08:00
|
|
|
if segment == nil || segment.InsertChannel != channel {
|
2021-06-24 14:20:10 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-08-12 10:48:08 +08:00
|
|
|
allocations := make([]*Allocation, 0, len(segment.allocations))
|
2021-07-12 17:24:25 +08:00
|
|
|
for i := 0; i < len(segment.allocations); i++ {
|
2021-07-23 21:58:33 +08:00
|
|
|
if segment.allocations[i].ExpireTime <= ts {
|
2021-07-12 17:24:25 +08:00
|
|
|
a := segment.allocations[i]
|
2021-09-07 13:59:58 +08:00
|
|
|
putAllocation(a)
|
2021-08-12 10:48:08 +08:00
|
|
|
} else {
|
|
|
|
allocations = append(allocations, segment.allocations[i])
|
2021-06-24 14:20:10 +08:00
|
|
|
}
|
|
|
|
}
|
2021-08-12 10:48:08 +08:00
|
|
|
s.meta.SetAllocations(segment.GetID(), allocations)
|
2021-06-24 14:20:10 +08:00
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-06-08 19:25:37 +08:00
|
|
|
// tryToSealSegment applies segment & channel seal policies
|
2021-07-27 14:34:48 +08:00
|
|
|
func (s *SegmentManager) tryToSealSegment(ts Timestamp, channel string) error {
|
2021-07-12 17:24:25 +08:00
|
|
|
channelInfo := make(map[string][]*SegmentInfo)
|
|
|
|
for _, id := range s.segments {
|
|
|
|
info := s.meta.GetSegment(id)
|
2021-07-27 14:34:48 +08:00
|
|
|
if info == nil || info.InsertChannel != channel {
|
2021-06-24 14:20:10 +08:00
|
|
|
continue
|
|
|
|
}
|
|
|
|
channelInfo[info.InsertChannel] = append(channelInfo[info.InsertChannel], info)
|
|
|
|
if info.State == commonpb.SegmentState_Sealed {
|
2021-05-21 18:30:41 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-06-08 19:25:37 +08:00
|
|
|
// change shouldSeal to segment seal policy logic
|
|
|
|
for _, policy := range s.segmentSealPolicies {
|
2021-07-12 17:24:25 +08:00
|
|
|
if policy(info, ts) {
|
|
|
|
if err := s.meta.SetState(id, commonpb.SegmentState_Sealed); err != nil {
|
2021-06-08 19:25:37 +08:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
break
|
|
|
|
}
|
2021-05-21 18:30:41 +08:00
|
|
|
}
|
2021-06-08 19:25:37 +08:00
|
|
|
}
|
2021-06-24 14:20:10 +08:00
|
|
|
for channel, segmentInfos := range channelInfo {
|
2021-06-08 19:25:37 +08:00
|
|
|
for _, policy := range s.channelSealPolicies {
|
2021-06-24 14:20:10 +08:00
|
|
|
vs := policy(channel, segmentInfos, ts)
|
|
|
|
for _, info := range vs {
|
|
|
|
if info.State == commonpb.SegmentState_Sealed {
|
2021-06-08 19:25:37 +08:00
|
|
|
continue
|
|
|
|
}
|
2021-07-07 14:02:01 +08:00
|
|
|
if err := s.meta.SetState(info.GetID(), commonpb.SegmentState_Sealed); err != nil {
|
2021-06-08 19:25:37 +08:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
2021-01-15 17:09:41 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|