mirror of https://github.com/milvus-io/milvus.git
Signed-off-by: zhagnlu <lu.zhang@zilliz.com> Signed-off-by: zhagnlu <lu.zhang@zilliz.com> Co-authored-by: zhagnlu <lu.zhang@zilliz.com>pull/19428/head
parent
91df8f2d6a
commit
59f8e96522
|
@ -255,6 +255,7 @@ dataCoord:
|
|||
|
||||
segment:
|
||||
maxSize: 512 # Maximum size of a segment in MB
|
||||
diskSegmentMaxSize: 2048 # Maximun size of a segment in MB for collection which has Disk index
|
||||
sealProportion: 0.25 # It's the minimum proportion for a segment which can be sealed
|
||||
assignmentExpiration: 2000 # The time of the assignment expiration in ms
|
||||
maxLife: 86400 # The max lifetime of segment in seconds, 24*60*60
|
||||
|
|
|
@ -18,6 +18,7 @@ package datacoord
|
|||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sort"
|
||||
"sync"
|
||||
"time"
|
||||
|
@ -25,7 +26,10 @@ import (
|
|||
"github.com/milvus-io/milvus/api/commonpb"
|
||||
"github.com/milvus-io/milvus/internal/log"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/funcutil"
|
||||
"github.com/milvus-io/milvus/internal/util/indexparamcheck"
|
||||
"github.com/milvus-io/milvus/internal/util/logutil"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
@ -60,16 +64,17 @@ type compactionSignal struct {
|
|||
var _ trigger = (*compactionTrigger)(nil)
|
||||
|
||||
type compactionTrigger struct {
|
||||
meta *meta
|
||||
allocator allocator
|
||||
signals chan *compactionSignal
|
||||
compactionHandler compactionPlanContext
|
||||
globalTrigger *time.Ticker
|
||||
forceMu sync.Mutex
|
||||
quit chan struct{}
|
||||
wg sync.WaitGroup
|
||||
segRefer *SegmentReferenceManager
|
||||
indexCoord types.IndexCoord
|
||||
meta *meta
|
||||
allocator allocator
|
||||
signals chan *compactionSignal
|
||||
compactionHandler compactionPlanContext
|
||||
globalTrigger *time.Ticker
|
||||
forceMu sync.Mutex
|
||||
quit chan struct{}
|
||||
wg sync.WaitGroup
|
||||
segRefer *SegmentReferenceManager
|
||||
indexCoord types.IndexCoord
|
||||
estimateDiskSegmentPolicy calUpperLimitPolicy
|
||||
}
|
||||
|
||||
func newCompactionTrigger(
|
||||
|
@ -80,12 +85,13 @@ func newCompactionTrigger(
|
|||
indexCoord types.IndexCoord,
|
||||
) *compactionTrigger {
|
||||
return &compactionTrigger{
|
||||
meta: meta,
|
||||
allocator: allocator,
|
||||
signals: make(chan *compactionSignal, 100),
|
||||
compactionHandler: compactionHandler,
|
||||
segRefer: segRefer,
|
||||
indexCoord: indexCoord,
|
||||
meta: meta,
|
||||
allocator: allocator,
|
||||
signals: make(chan *compactionSignal, 100),
|
||||
compactionHandler: compactionHandler,
|
||||
segRefer: segRefer,
|
||||
indexCoord: indexCoord,
|
||||
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -228,6 +234,48 @@ func getPlanIDs(plans []*datapb.CompactionPlan) []int64 {
|
|||
return ids
|
||||
}
|
||||
|
||||
func (t *compactionTrigger) estimateDiskSegmentMaxNumOfRows(collectionID UniqueID) (int, error) {
|
||||
collMeta := t.meta.GetCollection(collectionID)
|
||||
if collMeta == nil {
|
||||
return -1, fmt.Errorf("failed to get collection %d", collectionID)
|
||||
}
|
||||
|
||||
return t.estimateDiskSegmentPolicy(collMeta.Schema)
|
||||
}
|
||||
|
||||
func (t *compactionTrigger) updateSegmentMaxSize(segments []*SegmentInfo) error {
|
||||
ctx := context.Background()
|
||||
|
||||
if len(segments) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
collectionID := segments[0].GetCollectionID()
|
||||
resp, err := t.indexCoord.DescribeIndex(ctx, &indexpb.DescribeIndexRequest{
|
||||
CollectionID: collectionID,
|
||||
IndexName: "",
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for _, indexInfo := range resp.IndexInfos {
|
||||
indexParamsMap := funcutil.KeyValuePair2Map(indexInfo.IndexParams)
|
||||
if indexType, ok := indexParamsMap["index_type"]; ok {
|
||||
if indexType == indexparamcheck.IndexDISKANN {
|
||||
diskSegmentMaxRows, err := t.estimateDiskSegmentMaxNumOfRows(collectionID)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, segment := range segments {
|
||||
segment.MaxRowNum = int64(diskSegmentMaxRows)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) {
|
||||
t.forceMu.Lock()
|
||||
defer t.forceMu.Unlock()
|
||||
|
@ -243,8 +291,15 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) {
|
|||
if !signal.isForce && t.compactionHandler.isFull() {
|
||||
break
|
||||
}
|
||||
|
||||
group.segments = FilterInIndexedSegments(t.meta, t.indexCoord, group.segments...)
|
||||
|
||||
err := t.updateSegmentMaxSize(group.segments)
|
||||
if err != nil {
|
||||
log.Warn("failed to update segment max size,", zap.Error(err))
|
||||
continue
|
||||
}
|
||||
|
||||
plans := t.generatePlans(group.segments, signal.isForce, signal.compactTime)
|
||||
for _, plan := range plans {
|
||||
if !signal.isForce && t.compactionHandler.isFull() {
|
||||
|
@ -292,6 +347,12 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
|
|||
channel := segment.GetInsertChannel()
|
||||
partitionID := segment.GetPartitionID()
|
||||
segments := t.getCandidateSegments(channel, partitionID)
|
||||
|
||||
err := t.updateSegmentMaxSize(segments)
|
||||
if err != nil {
|
||||
log.Warn("failed to update segment max size", zap.Error(err))
|
||||
}
|
||||
|
||||
plans := t.generatePlans(segments, signal.isForce, signal.compactTime)
|
||||
for _, plan := range plans {
|
||||
if t.compactionHandler.isFull() {
|
||||
|
|
|
@ -24,11 +24,8 @@ import (
|
|||
|
||||
"github.com/milvus-io/milvus/api/commonpb"
|
||||
"github.com/milvus-io/milvus/api/schemapb"
|
||||
"github.com/milvus-io/milvus/internal/mocks"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/mock"
|
||||
)
|
||||
|
||||
type spyCompactionHandler struct {
|
||||
|
@ -167,6 +164,79 @@ func Test_compactionTrigger_force(t *testing.T) {
|
|||
},
|
||||
},
|
||||
},
|
||||
1000: {
|
||||
ID: 1000,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: vecFieldID,
|
||||
DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "dim",
|
||||
Value: "128",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
// error (has no vector field)
|
||||
2000: {
|
||||
ID: 2000,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: vecFieldID,
|
||||
DataType: schemapb.DataType_Int16,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
// error (has no dim)
|
||||
3000: {
|
||||
ID: 3000,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: vecFieldID,
|
||||
DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
// error (dim parse fail)
|
||||
4000: {
|
||||
ID: 4000,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: vecFieldID,
|
||||
DataType: schemapb.DataType_FloatVector,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "dim",
|
||||
Value: "128error",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
10000: {
|
||||
ID: 10000,
|
||||
Schema: &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{
|
||||
{
|
||||
FieldID: vecFieldID,
|
||||
DataType: schemapb.DataType_FloatVector,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
newMockAllocator(),
|
||||
|
@ -232,27 +302,7 @@ func Test_compactionTrigger_force(t *testing.T) {
|
|||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
indexCoord := mocks.NewMockIndexCoord(t)
|
||||
for _, segment := range tt.fields.meta.segments.GetSegments() {
|
||||
indexCoord.EXPECT().GetIndexInfos(mock.Anything, &indexpb.GetIndexInfoRequest{
|
||||
CollectionID: 2,
|
||||
SegmentIDs: []int64{segment.GetID()},
|
||||
}).Return(&indexpb.GetIndexInfoResponse{
|
||||
Status: &commonpb.Status{},
|
||||
SegmentInfo: map[int64]*indexpb.SegmentInfo{
|
||||
segment.GetID(): {
|
||||
EnableIndex: true,
|
||||
CollectionID: 2,
|
||||
SegmentID: segment.GetID(),
|
||||
IndexInfos: []*indexpb.IndexFilePathInfo{
|
||||
{
|
||||
FieldID: vecFieldID,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, nil)
|
||||
}
|
||||
indexCoord := newMockIndexCoord()
|
||||
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
|
@ -270,6 +320,162 @@ func Test_compactionTrigger_force(t *testing.T) {
|
|||
sortPlanCompactionBinlogs(plan)
|
||||
assert.EqualValues(t, tt.wantPlans[0], plan)
|
||||
})
|
||||
|
||||
t.Run(tt.name+" with DiskANN index", func(t *testing.T) {
|
||||
indexCood := newMockIndexCoord()
|
||||
segmentIDs := make([]int64, 0)
|
||||
for _, segment := range tt.fields.meta.segments.GetSegments() {
|
||||
segmentIDs = append(segmentIDs, segment.GetID())
|
||||
// Collection 1000 means it has DiskANN index
|
||||
segment.CollectionID = 1000
|
||||
}
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
allocator: tt.fields.allocator,
|
||||
signals: tt.fields.signals,
|
||||
compactionHandler: tt.fields.compactionHandler,
|
||||
globalTrigger: tt.fields.globalTrigger,
|
||||
segRefer: tt.fields.segRefer,
|
||||
indexCoord: indexCood,
|
||||
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
|
||||
}
|
||||
tt.args.collectionID = 1000
|
||||
_, err := tr.forceTriggerCompaction(tt.args.collectionID, tt.args.compactTime)
|
||||
assert.Equal(t, tt.wantErr, err != nil)
|
||||
// expect max row num = 2048*1024*1024/(128*4) = 4194304
|
||||
assert.EqualValues(t, 4194304, tt.fields.meta.segments.GetSegments()[0].MaxRowNum)
|
||||
spy := (tt.fields.compactionHandler).(*spyCompactionHandler)
|
||||
<-spy.spyChan
|
||||
})
|
||||
|
||||
t.Run(tt.name+" with no filtered segments", func(t *testing.T) {
|
||||
indexCood := newMockIndexCoord()
|
||||
segmentIDs := make([]int64, 0)
|
||||
for _, segment := range tt.fields.meta.segments.GetSegments() {
|
||||
segmentIDs = append(segmentIDs, segment.GetID())
|
||||
// Collection 2000 means has no vector field
|
||||
segment.CollectionID = 2000
|
||||
}
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
allocator: tt.fields.allocator,
|
||||
signals: tt.fields.signals,
|
||||
compactionHandler: tt.fields.compactionHandler,
|
||||
globalTrigger: tt.fields.globalTrigger,
|
||||
segRefer: tt.fields.segRefer,
|
||||
indexCoord: indexCood,
|
||||
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
|
||||
}
|
||||
tt.args.collectionID = 2000
|
||||
_, err := tr.forceTriggerCompaction(tt.args.collectionID, tt.args.compactTime)
|
||||
assert.Equal(t, tt.wantErr, err != nil)
|
||||
spy := (tt.fields.compactionHandler).(*spyCompactionHandler)
|
||||
hasPlan := true
|
||||
select {
|
||||
case <-spy.spyChan:
|
||||
hasPlan = true
|
||||
case <-time.After(2 * time.Second):
|
||||
hasPlan = false
|
||||
}
|
||||
assert.Equal(t, false, hasPlan)
|
||||
})
|
||||
|
||||
t.Run(tt.name+" with meta error", func(t *testing.T) {
|
||||
indexCood := newMockIndexCoord()
|
||||
segmentIDs := make([]int64, 0)
|
||||
for _, segment := range tt.fields.meta.segments.GetSegments() {
|
||||
segmentIDs = append(segmentIDs, segment.GetID())
|
||||
// Collection 3000 means field has no dim info
|
||||
segment.CollectionID = 3000
|
||||
}
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
allocator: tt.fields.allocator,
|
||||
signals: tt.fields.signals,
|
||||
compactionHandler: tt.fields.compactionHandler,
|
||||
globalTrigger: tt.fields.globalTrigger,
|
||||
segRefer: tt.fields.segRefer,
|
||||
indexCoord: indexCood,
|
||||
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
|
||||
}
|
||||
tt.args.collectionID = 3000
|
||||
_, err := tr.forceTriggerCompaction(tt.args.collectionID, tt.args.compactTime)
|
||||
assert.Equal(t, tt.wantErr, err != nil)
|
||||
spy := (tt.fields.compactionHandler).(*spyCompactionHandler)
|
||||
hasPlan := true
|
||||
select {
|
||||
case <-spy.spyChan:
|
||||
hasPlan = true
|
||||
case <-time.After(2 * time.Second):
|
||||
hasPlan = false
|
||||
}
|
||||
assert.Equal(t, false, hasPlan)
|
||||
})
|
||||
|
||||
t.Run(tt.name+" with meta error", func(t *testing.T) {
|
||||
indexCood := newMockIndexCoord()
|
||||
segmentIDs := make([]int64, 0)
|
||||
for _, segment := range tt.fields.meta.segments.GetSegments() {
|
||||
segmentIDs = append(segmentIDs, segment.GetID())
|
||||
// Collection 4000 means field has dim mistake
|
||||
segment.CollectionID = 4000
|
||||
}
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
allocator: tt.fields.allocator,
|
||||
signals: tt.fields.signals,
|
||||
compactionHandler: tt.fields.compactionHandler,
|
||||
globalTrigger: tt.fields.globalTrigger,
|
||||
segRefer: tt.fields.segRefer,
|
||||
indexCoord: indexCood,
|
||||
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
|
||||
}
|
||||
tt.args.collectionID = 4000
|
||||
_, err := tr.forceTriggerCompaction(tt.args.collectionID, tt.args.compactTime)
|
||||
assert.Equal(t, tt.wantErr, err != nil)
|
||||
spy := (tt.fields.compactionHandler).(*spyCompactionHandler)
|
||||
hasPlan := true
|
||||
select {
|
||||
case <-spy.spyChan:
|
||||
hasPlan = true
|
||||
case <-time.After(2 * time.Second):
|
||||
hasPlan = false
|
||||
}
|
||||
assert.Equal(t, false, hasPlan)
|
||||
})
|
||||
|
||||
t.Run(tt.name+" with index info error", func(t *testing.T) {
|
||||
indexCood := newMockIndexCoord()
|
||||
segmentIDs := make([]int64, 0)
|
||||
for _, segment := range tt.fields.meta.segments.GetSegments() {
|
||||
segmentIDs = append(segmentIDs, segment.GetID())
|
||||
// Collection 10000 means it has DescribeIndexInfo error
|
||||
segment.CollectionID = 10000
|
||||
}
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
allocator: tt.fields.allocator,
|
||||
signals: tt.fields.signals,
|
||||
compactionHandler: tt.fields.compactionHandler,
|
||||
globalTrigger: tt.fields.globalTrigger,
|
||||
segRefer: tt.fields.segRefer,
|
||||
indexCoord: indexCood,
|
||||
}
|
||||
tt.args.collectionID = 10000
|
||||
_, err := tr.forceTriggerCompaction(tt.args.collectionID, tt.args.compactTime)
|
||||
assert.Equal(t, tt.wantErr, err != nil)
|
||||
spy := (tt.fields.compactionHandler).(*spyCompactionHandler)
|
||||
hasPlan := true
|
||||
select {
|
||||
case <-spy.spyChan:
|
||||
hasPlan = true
|
||||
case <-time.After(2 * time.Second):
|
||||
hasPlan = false
|
||||
}
|
||||
|
||||
assert.Equal(t, false, hasPlan)
|
||||
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -409,27 +615,7 @@ func Test_compactionTrigger_force_maxSegmentLimit(t *testing.T) {
|
|||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
indexCoord := mocks.NewMockIndexCoord(t)
|
||||
for _, segment := range tt.fields.meta.segments.GetSegments() {
|
||||
indexCoord.EXPECT().GetIndexInfos(mock.Anything, &indexpb.GetIndexInfoRequest{
|
||||
CollectionID: 2,
|
||||
SegmentIDs: []int64{segment.GetID()},
|
||||
}).Return(&indexpb.GetIndexInfoResponse{
|
||||
Status: &commonpb.Status{},
|
||||
SegmentInfo: map[int64]*indexpb.SegmentInfo{
|
||||
segment.GetID(): {
|
||||
EnableIndex: true,
|
||||
CollectionID: 2,
|
||||
SegmentID: segment.GetID(),
|
||||
IndexInfos: []*indexpb.IndexFilePathInfo{
|
||||
{
|
||||
FieldID: vecFieldID,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, nil)
|
||||
}
|
||||
indexCoord := newMockIndexCoord()
|
||||
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
|
@ -603,27 +789,8 @@ func Test_compactionTrigger_noplan(t *testing.T) {
|
|||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
indexCoord := mocks.NewMockIndexCoord(t)
|
||||
for _, segment := range tt.fields.meta.segments.GetSegments() {
|
||||
indexCoord.EXPECT().GetIndexInfos(mock.Anything, &indexpb.GetIndexInfoRequest{
|
||||
CollectionID: 2,
|
||||
SegmentIDs: []int64{segment.GetID()},
|
||||
}).Return(&indexpb.GetIndexInfoResponse{
|
||||
Status: &commonpb.Status{},
|
||||
SegmentInfo: map[int64]*indexpb.SegmentInfo{
|
||||
segment.GetID(): {
|
||||
EnableIndex: true,
|
||||
CollectionID: 2,
|
||||
SegmentID: segment.GetID(),
|
||||
IndexInfos: []*indexpb.IndexFilePathInfo{
|
||||
{
|
||||
FieldID: vecFieldID,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, nil)
|
||||
}
|
||||
indexCoord := newMockIndexCoord()
|
||||
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
allocator: tt.fields.allocator,
|
||||
|
@ -813,27 +980,8 @@ func Test_compactionTrigger_smallfiles(t *testing.T) {
|
|||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
indexCoord := mocks.NewMockIndexCoord(t)
|
||||
for _, segment := range tt.fields.meta.segments.GetSegments() {
|
||||
indexCoord.EXPECT().GetIndexInfos(mock.Anything, &indexpb.GetIndexInfoRequest{
|
||||
CollectionID: 2,
|
||||
SegmentIDs: []int64{segment.GetID()},
|
||||
}).Return(&indexpb.GetIndexInfoResponse{
|
||||
Status: &commonpb.Status{},
|
||||
SegmentInfo: map[int64]*indexpb.SegmentInfo{
|
||||
segment.GetID(): {
|
||||
EnableIndex: true,
|
||||
CollectionID: 2,
|
||||
SegmentID: segment.GetID(),
|
||||
IndexInfos: []*indexpb.IndexFilePathInfo{
|
||||
{
|
||||
FieldID: vecFieldID,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, nil)
|
||||
}
|
||||
indexCoord := newMockIndexCoord()
|
||||
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
allocator: tt.fields.allocator,
|
||||
|
@ -953,27 +1101,8 @@ func Test_compactionTrigger_noplan_random_size(t *testing.T) {
|
|||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
indexCoord := mocks.NewMockIndexCoord(t)
|
||||
for _, segment := range tt.fields.meta.segments.GetSegments() {
|
||||
indexCoord.EXPECT().GetIndexInfos(mock.Anything, &indexpb.GetIndexInfoRequest{
|
||||
CollectionID: 2,
|
||||
SegmentIDs: []int64{segment.GetID()},
|
||||
}).Return(&indexpb.GetIndexInfoResponse{
|
||||
Status: &commonpb.Status{},
|
||||
SegmentInfo: map[int64]*indexpb.SegmentInfo{
|
||||
segment.GetID(): {
|
||||
EnableIndex: true,
|
||||
CollectionID: 2,
|
||||
SegmentID: segment.GetID(),
|
||||
IndexInfos: []*indexpb.IndexFilePathInfo{
|
||||
{
|
||||
FieldID: vecFieldID,
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, nil)
|
||||
}
|
||||
indexCoord := newMockIndexCoord()
|
||||
|
||||
tr := &compactionTrigger{
|
||||
meta: tt.fields.meta,
|
||||
allocator: tt.fields.allocator,
|
||||
|
@ -1024,8 +1153,9 @@ func Test_compactionTrigger_noplan_random_size(t *testing.T) {
|
|||
func Test_compactionTrigger_shouldDoSingleCompaction(t *testing.T) {
|
||||
Params.Init()
|
||||
|
||||
indexCoord := newMockIndexCoord()
|
||||
trigger := newCompactionTrigger(&meta{}, &compactionPlanHandler{}, newMockAllocator(),
|
||||
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, nil)
|
||||
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, indexCoord)
|
||||
|
||||
// Test too many files.
|
||||
var binlogs []*datapb.FieldBinlog
|
||||
|
@ -1157,8 +1287,9 @@ func Test_newCompactionTrigger(t *testing.T) {
|
|||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
indexCoord := newMockIndexCoord()
|
||||
got := newCompactionTrigger(tt.args.meta, tt.args.compactionHandler, tt.args.allocator,
|
||||
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, nil)
|
||||
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, indexCoord)
|
||||
assert.Equal(t, tt.args.meta, got.meta)
|
||||
assert.Equal(t, tt.args.compactionHandler, got.compactionHandler)
|
||||
assert.Equal(t, tt.args.allocator, got.allocator)
|
||||
|
@ -1168,8 +1299,9 @@ func Test_newCompactionTrigger(t *testing.T) {
|
|||
|
||||
func Test_handleSignal(t *testing.T) {
|
||||
|
||||
indexCoord := newMockIndexCoord()
|
||||
got := newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{}, newMockAllocator(),
|
||||
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, nil)
|
||||
&SegmentReferenceManager{segmentsLock: map[UniqueID]map[UniqueID]*datapb.SegmentReferenceLock{}}, indexCoord)
|
||||
signal := &compactionSignal{
|
||||
segmentID: 1,
|
||||
}
|
||||
|
|
|
@ -28,9 +28,11 @@ import (
|
|||
"github.com/milvus-io/milvus/internal/kv"
|
||||
memkv "github.com/milvus-io/milvus/internal/kv/mem"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/internalpb"
|
||||
"github.com/milvus-io/milvus/internal/proto/proxypb"
|
||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
||||
"github.com/milvus-io/milvus/internal/types"
|
||||
"github.com/milvus-io/milvus/internal/util/metricsinfo"
|
||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
||||
|
@ -725,3 +727,87 @@ func (h *mockHandler) CheckShouldDropChannel(channel string) bool {
|
|||
}
|
||||
|
||||
func (h *mockHandler) FinishDropChannel(channel string) {}
|
||||
|
||||
type mockIndexCoord struct {
|
||||
types.IndexCoord
|
||||
}
|
||||
|
||||
func newMockIndexCoord() *mockIndexCoord {
|
||||
return &mockIndexCoord{}
|
||||
}
|
||||
|
||||
func (m *mockIndexCoord) Init() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *mockIndexCoord) Start() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *mockIndexCoord) DescribeIndex(ctx context.Context, req *indexpb.DescribeIndexRequest) (*indexpb.DescribeIndexResponse, error) {
|
||||
if req.CollectionID == 10000 {
|
||||
return nil, errors.New("server down")
|
||||
}
|
||||
|
||||
// Has diskann index
|
||||
if req.CollectionID == 1000 || req.CollectionID == 2000 ||
|
||||
req.CollectionID == 3000 || req.CollectionID == 4000 {
|
||||
return &indexpb.DescribeIndexResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
},
|
||||
IndexInfos: []*indexpb.IndexInfo{
|
||||
{
|
||||
CollectionID: req.CollectionID,
|
||||
FieldID: 0,
|
||||
IndexName: "DISKANN",
|
||||
IndexID: 0,
|
||||
TypeParams: nil,
|
||||
IndexParams: []*commonpb.KeyValuePair{
|
||||
{
|
||||
Key: "index_type",
|
||||
Value: "DISKANN",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
// Has common index
|
||||
return &indexpb.DescribeIndexResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_Success,
|
||||
},
|
||||
IndexInfos: []*indexpb.IndexInfo{
|
||||
{
|
||||
CollectionID: 1,
|
||||
FieldID: 0,
|
||||
IndexName: "default",
|
||||
IndexID: 0,
|
||||
TypeParams: nil,
|
||||
IndexParams: nil,
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (m *mockIndexCoord) GetIndexInfos(ctx context.Context, req *indexpb.GetIndexInfoRequest) (*indexpb.GetIndexInfoResponse, error) {
|
||||
segmentID := req.GetSegmentIDs()[0]
|
||||
collectionID := req.GetCollectionID()
|
||||
return &indexpb.GetIndexInfoResponse{
|
||||
Status: &commonpb.Status{},
|
||||
SegmentInfo: map[int64]*indexpb.SegmentInfo{
|
||||
segmentID: {
|
||||
EnableIndex: true,
|
||||
CollectionID: collectionID,
|
||||
SegmentID: segmentID,
|
||||
IndexInfos: []*indexpb.IndexFilePathInfo{
|
||||
{
|
||||
FieldID: int64(201),
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
|
|
|
@ -45,6 +45,22 @@ func calBySchemaPolicy(schema *schemapb.CollectionSchema) (int, error) {
|
|||
return int(threshold / float64(sizePerRecord)), nil
|
||||
}
|
||||
|
||||
func calBySchemaPolicyWithDiskIndex(schema *schemapb.CollectionSchema) (int, error) {
|
||||
if schema == nil {
|
||||
return -1, errors.New("nil schema")
|
||||
}
|
||||
sizePerRecord, err := typeutil.EstimateSizePerRecord(schema)
|
||||
if err != nil {
|
||||
return -1, err
|
||||
}
|
||||
// check zero value, preventing panicking
|
||||
if sizePerRecord == 0 {
|
||||
return -1, errors.New("zero size record schema found")
|
||||
}
|
||||
threshold := Params.DataCoordCfg.DiskSegmentMaxSize * 1024 * 1024
|
||||
return int(threshold / float64(sizePerRecord)), nil
|
||||
}
|
||||
|
||||
// AllocatePolicy helper function definition to allocate Segment space
|
||||
type AllocatePolicy func(segments []*SegmentInfo, count int64,
|
||||
maxCountPerSegment int64) ([]*Allocation, []*Allocation)
|
||||
|
|
|
@ -54,6 +54,9 @@ type uploader interface {
|
|||
// errUploadToBlobStorage is returned if ctx is canceled from outside while a uploading is inprogress.
|
||||
// Beware of the ctx here, if no timeout or cancel is applied to this ctx, this uploading may retry forever.
|
||||
upload(ctx context.Context, segID, partID UniqueID, iData []*InsertData, segStats []byte, dData *DeleteData, meta *etcdpb.CollectionMeta) (*segPaths, error)
|
||||
uploadInsertLog(ctx context.Context, segID, partID UniqueID, iData *InsertData, meta *etcdpb.CollectionMeta) (map[UniqueID]*datapb.FieldBinlog, error)
|
||||
uploadStatsLog(ctx context.Context, segID, partID UniqueID, segStats []byte, meta *etcdpb.CollectionMeta) ([]*datapb.FieldBinlog, error)
|
||||
uploadDeltaLog(ctx context.Context, segID, partID UniqueID, dData *DeleteData, meta *etcdpb.CollectionMeta) ([]*datapb.FieldBinlog, error)
|
||||
}
|
||||
|
||||
type binlogIO struct {
|
||||
|
@ -102,6 +105,33 @@ func (b *binlogIO) download(ctx context.Context, paths []string) ([]*Blob, error
|
|||
return rst, nil
|
||||
}
|
||||
|
||||
func (b *binlogIO) uploadSegmentFiles(
|
||||
ctx context.Context,
|
||||
CollectionID UniqueID,
|
||||
segID UniqueID,
|
||||
kvs map[string][]byte) error {
|
||||
var err = errStart
|
||||
for err != nil {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
log.Warn("ctx done when saving kvs to blob storage",
|
||||
zap.Int64("collectionID", CollectionID),
|
||||
zap.Int64("segmentID", segID),
|
||||
zap.Int("number of kvs", len(kvs)))
|
||||
return errUploadToBlobStorage
|
||||
default:
|
||||
if err != errStart {
|
||||
log.Warn("save binlog failed, retry in 50ms",
|
||||
zap.Int64("collectionID", CollectionID),
|
||||
zap.Int64("segmentID", segID))
|
||||
<-time.After(50 * time.Millisecond)
|
||||
}
|
||||
err = b.MultiWrite(kvs)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type segPaths struct {
|
||||
inPaths []*datapb.FieldBinlog
|
||||
statsPaths []*datapb.FieldBinlog
|
||||
|
@ -207,24 +237,9 @@ func (b *binlogIO) upload(
|
|||
})
|
||||
}
|
||||
|
||||
err = errStart
|
||||
for err != nil {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
log.Warn("ctx done when saving kvs to blob storage",
|
||||
zap.Int64("collectionID", meta.GetID()),
|
||||
zap.Int64("segmentID", segID),
|
||||
zap.Int("number of kvs", len(kvs)))
|
||||
return nil, errUploadToBlobStorage
|
||||
default:
|
||||
if err != errStart {
|
||||
log.Warn("save binlog failed, retry in 50ms",
|
||||
zap.Int64("collectionID", meta.GetID()),
|
||||
zap.Int64("segmentID", segID))
|
||||
<-time.After(50 * time.Millisecond)
|
||||
}
|
||||
err = b.MultiWrite(kvs)
|
||||
}
|
||||
err = b.uploadSegmentFiles(ctx, meta.GetID(), segID, kvs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return p, nil
|
||||
}
|
||||
|
@ -310,3 +325,138 @@ func (b *binlogIO) idxGenerator(n int, done <-chan struct{}) (<-chan UniqueID, e
|
|||
|
||||
return rt, nil
|
||||
}
|
||||
|
||||
func (b *binlogIO) uploadInsertLog(
|
||||
ctx context.Context,
|
||||
segID UniqueID,
|
||||
partID UniqueID,
|
||||
iData *InsertData,
|
||||
meta *etcdpb.CollectionMeta) (map[UniqueID]*datapb.FieldBinlog, error) {
|
||||
var (
|
||||
kvs = make(map[string][]byte)
|
||||
|
||||
insertField2Path = make(map[UniqueID]*datapb.FieldBinlog)
|
||||
)
|
||||
|
||||
tf, ok := iData.Data[common.TimeStampField]
|
||||
if !ok || tf.RowNum() == 0 {
|
||||
log.Warn("binlog io uploading empty insert data",
|
||||
zap.Int64("segmentID", segID),
|
||||
zap.Int64("collectionID", meta.GetID()),
|
||||
)
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
kv, inpaths, err := b.genInsertBlobs(iData, partID, segID, meta)
|
||||
if err != nil {
|
||||
log.Warn("generate insert blobs wrong",
|
||||
zap.Int64("collectionID", meta.GetID()),
|
||||
zap.Int64("segmentID", segID),
|
||||
zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
for k, v := range kv {
|
||||
kvs[k] = v
|
||||
}
|
||||
|
||||
for fID, path := range inpaths {
|
||||
tmpBinlog, ok := insertField2Path[fID]
|
||||
if !ok {
|
||||
tmpBinlog = path
|
||||
} else {
|
||||
tmpBinlog.Binlogs = append(tmpBinlog.Binlogs, path.GetBinlogs()...)
|
||||
}
|
||||
insertField2Path[fID] = tmpBinlog
|
||||
}
|
||||
|
||||
err = b.uploadSegmentFiles(ctx, meta.GetID(), segID, kvs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return insertField2Path, nil
|
||||
}
|
||||
|
||||
func (b *binlogIO) uploadStatsLog(
|
||||
ctx context.Context,
|
||||
segID UniqueID,
|
||||
partID UniqueID,
|
||||
segStats []byte,
|
||||
meta *etcdpb.CollectionMeta) ([]*datapb.FieldBinlog, error) {
|
||||
var (
|
||||
statsInfo = make([]*datapb.FieldBinlog, 0)
|
||||
kvs = make(map[string][]byte)
|
||||
)
|
||||
|
||||
pkID := getPKID(meta)
|
||||
if pkID == common.InvalidFieldID {
|
||||
log.Error("get invalid field id when finding pk", zap.Int64("collectionID", meta.GetID()), zap.Any("fields", meta.GetSchema().GetFields()))
|
||||
return nil, errors.New("invalid pk id")
|
||||
}
|
||||
|
||||
logID, err := b.allocID()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
k := JoinIDPath(meta.GetID(), partID, segID, pkID, logID)
|
||||
key := path.Join(b.ChunkManager.RootPath(), common.SegmentStatslogPath, k)
|
||||
fileLen := len(segStats)
|
||||
|
||||
kvs[key] = segStats
|
||||
statsInfo = append(statsInfo, &datapb.FieldBinlog{
|
||||
FieldID: pkID,
|
||||
Binlogs: []*datapb.Binlog{{
|
||||
LogPath: key,
|
||||
LogSize: int64(fileLen),
|
||||
}},
|
||||
})
|
||||
|
||||
err = b.uploadSegmentFiles(ctx, meta.GetID(), segID, kvs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return statsInfo, nil
|
||||
}
|
||||
|
||||
func (b *binlogIO) uploadDeltaLog(
|
||||
ctx context.Context,
|
||||
segID UniqueID,
|
||||
partID UniqueID,
|
||||
dData *DeleteData,
|
||||
meta *etcdpb.CollectionMeta) ([]*datapb.FieldBinlog, error) {
|
||||
var (
|
||||
deltaInfo = make([]*datapb.FieldBinlog, 0)
|
||||
kvs = make(map[string][]byte)
|
||||
)
|
||||
|
||||
if dData.RowCount > 0 {
|
||||
k, v, err := b.genDeltaBlobs(dData, meta.GetID(), partID, segID)
|
||||
if err != nil {
|
||||
log.Warn("generate delta blobs wrong",
|
||||
zap.Int64("collectionID", meta.GetID()),
|
||||
zap.Int64("segmentID", segID),
|
||||
zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
kvs[k] = v
|
||||
deltaInfo = append(deltaInfo, &datapb.FieldBinlog{
|
||||
FieldID: 0, // TODO: Not useful on deltalogs, FieldID shall be ID of primary key field
|
||||
Binlogs: []*datapb.Binlog{{
|
||||
EntriesNum: dData.RowCount,
|
||||
LogPath: k,
|
||||
LogSize: int64(len(v)),
|
||||
}},
|
||||
})
|
||||
} else {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
err := b.uploadSegmentFiles(ctx, meta.GetID(), segID, kvs)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return deltaInfo, nil
|
||||
}
|
||||
|
|
|
@ -44,6 +44,7 @@ func TestBinlogIOInterfaceMethods(t *testing.T) {
|
|||
f := &MetaFactory{}
|
||||
meta := f.GetCollectionMeta(UniqueID(10001), "uploads", schemapb.DataType_Int64)
|
||||
|
||||
//pkFieldID := int64(106)
|
||||
iData := genInsertData()
|
||||
pk := newInt64PrimaryKey(888)
|
||||
dData := &DeleteData{
|
||||
|
@ -69,11 +70,40 @@ func TestBinlogIOInterfaceMethods(t *testing.T) {
|
|||
assert.NotNil(t, p.deltaInfo)
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
|
||||
in, err := b.uploadInsertLog(ctx, 1, 10, iData, meta)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 12, len(in))
|
||||
assert.Equal(t, 1, len(in[0].GetBinlogs()))
|
||||
|
||||
stats, err := b.uploadStatsLog(ctx, 1, 10, []byte{}, meta)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 1, len(stats))
|
||||
assert.Equal(t, 1, len(stats[0].GetBinlogs()))
|
||||
|
||||
deltas, err := b.uploadDeltaLog(ctx, 1, 10, dData, meta)
|
||||
assert.NoError(t, err)
|
||||
assert.NotNil(t, deltas)
|
||||
assert.Equal(t, 1, len(deltas[0].GetBinlogs()))
|
||||
|
||||
cancel()
|
||||
|
||||
p, err = b.upload(ctx, 1, 10, []*InsertData{iData}, []byte{}, dData, meta)
|
||||
assert.EqualError(t, err, errUploadToBlobStorage.Error())
|
||||
assert.Nil(t, p)
|
||||
|
||||
in, err = b.uploadInsertLog(ctx, 1, 10, iData, meta)
|
||||
assert.EqualError(t, err, errUploadToBlobStorage.Error())
|
||||
assert.Nil(t, in)
|
||||
|
||||
stats, err = b.uploadStatsLog(ctx, 1, 10, []byte{}, meta)
|
||||
assert.EqualError(t, err, errUploadToBlobStorage.Error())
|
||||
assert.Nil(t, stats)
|
||||
|
||||
deltas, err = b.uploadDeltaLog(ctx, 1, 10, dData, meta)
|
||||
assert.EqualError(t, err, errUploadToBlobStorage.Error())
|
||||
assert.Nil(t, deltas)
|
||||
|
||||
})
|
||||
|
||||
t.Run("Test upload error", func(t *testing.T) {
|
||||
|
@ -122,11 +152,35 @@ func TestBinlogIOInterfaceMethods(t *testing.T) {
|
|||
assert.Error(t, err)
|
||||
assert.Empty(t, p)
|
||||
|
||||
in, err := b.uploadInsertLog(ctx, 1, 10, iData, meta)
|
||||
assert.Error(t, err)
|
||||
assert.Empty(t, in)
|
||||
|
||||
stats, err := b.uploadStatsLog(ctx, 1, 10, []byte{}, meta)
|
||||
assert.Error(t, err)
|
||||
assert.Empty(t, stats)
|
||||
|
||||
deltas, err := b.uploadDeltaLog(ctx, 1, 10, dData, meta)
|
||||
assert.Error(t, err)
|
||||
assert.Empty(t, deltas)
|
||||
|
||||
alloc.isvalid = false
|
||||
p, err = bin.upload(ctx, 1, 10, []*InsertData{iData}, []byte{}, dData, meta)
|
||||
assert.Error(t, err)
|
||||
assert.Empty(t, p)
|
||||
|
||||
in, err = b.uploadInsertLog(ctx, 1, 10, iData, meta)
|
||||
assert.Error(t, err)
|
||||
assert.Empty(t, in)
|
||||
|
||||
stats, err = b.uploadStatsLog(ctx, 1, 10, []byte{}, meta)
|
||||
assert.Error(t, err)
|
||||
assert.Empty(t, stats)
|
||||
|
||||
deltas, err = b.uploadDeltaLog(ctx, 1, 10, dData, meta)
|
||||
assert.Error(t, err)
|
||||
assert.Empty(t, deltas)
|
||||
|
||||
alloc.isvalid = true
|
||||
for _, field := range meta.GetSchema().GetFields() {
|
||||
field.IsPrimaryKey = false
|
||||
|
@ -135,6 +189,18 @@ func TestBinlogIOInterfaceMethods(t *testing.T) {
|
|||
assert.Error(t, err)
|
||||
assert.Empty(t, p)
|
||||
|
||||
in, err = b.uploadInsertLog(ctx, 1, 10, iData, meta)
|
||||
assert.Error(t, err)
|
||||
assert.Empty(t, in)
|
||||
|
||||
stats, err = b.uploadStatsLog(ctx, 1, 10, []byte{}, meta)
|
||||
assert.Error(t, err)
|
||||
assert.Empty(t, stats)
|
||||
|
||||
deltas, err = b.uploadDeltaLog(ctx, 1, 10, dData, meta)
|
||||
assert.Error(t, err)
|
||||
assert.Empty(t, deltas)
|
||||
|
||||
cancel()
|
||||
})
|
||||
|
||||
|
|
|
@ -187,7 +187,57 @@ func nano2Milli(nano time.Duration) float64 {
|
|||
return float64(nano) / float64(time.Millisecond)
|
||||
}
|
||||
|
||||
func (t *compactionTask) merge(mergeItr iterator, delta map[interface{}]Timestamp, schema *schemapb.CollectionSchema, currentTs Timestamp) ([]*InsertData, *Segment, int64, error) {
|
||||
func (t *compactionTask) uploadSingleInsertLog(
|
||||
ctxTimeout context.Context,
|
||||
targetSegID UniqueID,
|
||||
partID UniqueID,
|
||||
meta *etcdpb.CollectionMeta,
|
||||
segment *Segment,
|
||||
pkID UniqueID,
|
||||
fID2Content map[UniqueID][]interface{},
|
||||
fID2Type map[UniqueID]schemapb.DataType) (map[UniqueID]*datapb.FieldBinlog, error) {
|
||||
iData := &InsertData{
|
||||
Data: make(map[storage.FieldID]storage.FieldData)}
|
||||
|
||||
for fID, content := range fID2Content {
|
||||
tp, ok := fID2Type[fID]
|
||||
if !ok {
|
||||
log.Warn("no field ID in this schema", zap.Int64("fieldID", fID))
|
||||
return nil, errors.New("Unexpected error")
|
||||
}
|
||||
|
||||
fData, err := interface2FieldData(tp, content, int64(len(content)))
|
||||
if err != nil {
|
||||
log.Warn("transfer interface to FieldData wrong", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if fID == pkID {
|
||||
err = segment.updatePKRange(fData)
|
||||
if err != nil {
|
||||
log.Warn("update pk range failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
iData.Data[fID] = fData
|
||||
}
|
||||
|
||||
inPaths, err := t.uploadInsertLog(ctxTimeout, targetSegID, partID, iData, meta)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return inPaths, nil
|
||||
}
|
||||
|
||||
func (t *compactionTask) merge(
|
||||
ctxTimeout context.Context,
|
||||
unMergedInsertlogs [][]string,
|
||||
targetSegID UniqueID,
|
||||
partID UniqueID,
|
||||
meta *etcdpb.CollectionMeta,
|
||||
delta map[interface{}]Timestamp) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, *Segment, int64, error) {
|
||||
log := log.With(zap.Int64("planID", t.getPlanID()))
|
||||
mergeStart := time.Now()
|
||||
|
||||
|
@ -195,16 +245,20 @@ func (t *compactionTask) merge(mergeItr iterator, delta map[interface{}]Timestam
|
|||
dim int // dimension of float/binary vector field
|
||||
maxRowsPerBinlog int // maximum rows populating one binlog
|
||||
numBinlogs int // binlog number
|
||||
numRows int64 // the number of rows uploaded
|
||||
expired int64 // the number of expired entity
|
||||
err error
|
||||
|
||||
// statslog generation
|
||||
segment *Segment // empty segment used for bf generation
|
||||
pkID UniqueID
|
||||
pkType schemapb.DataType
|
||||
|
||||
iDatas = make([]*InsertData, 0)
|
||||
fID2Type = make(map[UniqueID]schemapb.DataType)
|
||||
fID2Content = make(map[UniqueID][]interface{})
|
||||
|
||||
insertField2Path = make(map[UniqueID]*datapb.FieldBinlog)
|
||||
insertPaths = make([]*datapb.FieldBinlog, 0)
|
||||
)
|
||||
|
||||
isDeletedValue := func(v *storage.Value) bool {
|
||||
|
@ -218,11 +272,24 @@ func (t *compactionTask) merge(mergeItr iterator, delta map[interface{}]Timestam
|
|||
segment = &Segment{}
|
||||
t.Replica.initSegmentBloomFilter(segment)
|
||||
|
||||
// get dim
|
||||
for _, fs := range schema.GetFields() {
|
||||
addInsertFieldPath := func(inPaths map[UniqueID]*datapb.FieldBinlog) {
|
||||
for fID, path := range inPaths {
|
||||
tmpBinlog, ok := insertField2Path[fID]
|
||||
if !ok {
|
||||
tmpBinlog = path
|
||||
} else {
|
||||
tmpBinlog.Binlogs = append(tmpBinlog.Binlogs, path.GetBinlogs()...)
|
||||
}
|
||||
insertField2Path[fID] = tmpBinlog
|
||||
}
|
||||
}
|
||||
|
||||
// get pkID, pkType, dim
|
||||
for _, fs := range meta.GetSchema().GetFields() {
|
||||
fID2Type[fs.GetFieldID()] = fs.GetDataType()
|
||||
if fs.GetIsPrimaryKey() {
|
||||
if fs.GetIsPrimaryKey() && fs.GetFieldID() >= 100 && typeutil.IsPrimaryFieldType(fs.GetDataType()) {
|
||||
pkID = fs.GetFieldID()
|
||||
pkType = fs.GetDataType()
|
||||
}
|
||||
if fs.GetDataType() == schemapb.DataType_FloatVector ||
|
||||
fs.GetDataType() == schemapb.DataType_BinaryVector {
|
||||
|
@ -230,7 +297,7 @@ func (t *compactionTask) merge(mergeItr iterator, delta map[interface{}]Timestam
|
|||
if t.Key == "dim" {
|
||||
if dim, err = strconv.Atoi(t.Value); err != nil {
|
||||
log.Warn("strconv wrong on get dim", zap.Error(err))
|
||||
return nil, nil, 0, err
|
||||
return nil, nil, nil, 0, err
|
||||
}
|
||||
break
|
||||
}
|
||||
|
@ -239,90 +306,118 @@ func (t *compactionTask) merge(mergeItr iterator, delta map[interface{}]Timestam
|
|||
}
|
||||
|
||||
expired = 0
|
||||
for mergeItr.HasNext() {
|
||||
// no error if HasNext() returns true
|
||||
vInter, _ := mergeItr.Next()
|
||||
|
||||
v, ok := vInter.(*storage.Value)
|
||||
if !ok {
|
||||
log.Warn("transfer interface to Value wrong")
|
||||
return nil, nil, 0, errors.New("unexpected error")
|
||||
numRows = 0
|
||||
numBinlogs = 0
|
||||
currentTs := t.GetCurrentTime()
|
||||
maxRowsPerBinlog = int(Params.DataNodeCfg.FlushInsertBufferSize / (int64(dim) * 4))
|
||||
currentRows := 0
|
||||
downloadTimeCost := time.Duration(0)
|
||||
uploadInsertTimeCost := time.Duration(0)
|
||||
uploadStatsTimeCost := time.Duration(0)
|
||||
for _, path := range unMergedInsertlogs {
|
||||
downloadStart := time.Now()
|
||||
data, err := t.download(ctxTimeout, path)
|
||||
if err != nil {
|
||||
log.Warn("download insertlogs wrong")
|
||||
return nil, nil, nil, 0, err
|
||||
}
|
||||
downloadTimeCost += time.Since(downloadStart)
|
||||
|
||||
if isDeletedValue(v) {
|
||||
continue
|
||||
iter, err := storage.NewInsertBinlogIterator(data, pkID, pkType)
|
||||
if err != nil {
|
||||
log.Warn("new insert binlogs Itr wrong")
|
||||
return nil, nil, nil, 0, err
|
||||
}
|
||||
|
||||
ts := Timestamp(v.Timestamp)
|
||||
// Filtering expired entity
|
||||
if t.isExpiredEntity(ts, currentTs) {
|
||||
expired++
|
||||
continue
|
||||
}
|
||||
|
||||
row, ok := v.Value.(map[UniqueID]interface{})
|
||||
if !ok {
|
||||
log.Warn("transfer interface to map wrong")
|
||||
return nil, nil, 0, errors.New("unexpected error")
|
||||
}
|
||||
|
||||
for fID, vInter := range row {
|
||||
if _, ok := fID2Content[fID]; !ok {
|
||||
fID2Content[fID] = make([]interface{}, 0)
|
||||
for iter.HasNext() {
|
||||
vInter, _ := iter.Next()
|
||||
v, ok := vInter.(*storage.Value)
|
||||
if !ok {
|
||||
log.Warn("transfer interface to Value wrong")
|
||||
return nil, nil, nil, 0, errors.New("unexpected error")
|
||||
}
|
||||
|
||||
if isDeletedValue(v) {
|
||||
continue
|
||||
}
|
||||
|
||||
ts := Timestamp(v.Timestamp)
|
||||
// Filtering expired entity
|
||||
if t.isExpiredEntity(ts, currentTs) {
|
||||
expired++
|
||||
continue
|
||||
}
|
||||
|
||||
row, ok := v.Value.(map[UniqueID]interface{})
|
||||
if !ok {
|
||||
log.Warn("transfer interface to map wrong")
|
||||
return nil, nil, nil, 0, errors.New("unexpected error")
|
||||
}
|
||||
|
||||
for fID, vInter := range row {
|
||||
if _, ok := fID2Content[fID]; !ok {
|
||||
fID2Content[fID] = make([]interface{}, 0)
|
||||
}
|
||||
fID2Content[fID] = append(fID2Content[fID], vInter)
|
||||
}
|
||||
|
||||
currentRows++
|
||||
|
||||
if currentRows == maxRowsPerBinlog {
|
||||
uploadInsertStart := time.Now()
|
||||
inPaths, err := t.uploadSingleInsertLog(ctxTimeout, targetSegID, partID, meta, segment, pkID, fID2Content, fID2Type)
|
||||
if err != nil {
|
||||
return nil, nil, nil, 0, err
|
||||
}
|
||||
uploadInsertTimeCost += time.Since(uploadInsertStart)
|
||||
addInsertFieldPath(inPaths)
|
||||
|
||||
fID2Content = make(map[int64][]interface{})
|
||||
currentRows = 0
|
||||
numRows += int64(maxRowsPerBinlog)
|
||||
numBinlogs++
|
||||
}
|
||||
fID2Content[fID] = append(fID2Content[fID], vInter)
|
||||
}
|
||||
}
|
||||
if currentRows != 0 {
|
||||
uploadInsertStart := time.Now()
|
||||
inPaths, err := t.uploadSingleInsertLog(ctxTimeout, targetSegID, partID, meta, segment, pkID, fID2Content, fID2Type)
|
||||
if err != nil {
|
||||
return nil, nil, nil, 0, err
|
||||
}
|
||||
uploadInsertTimeCost += time.Since(uploadInsertStart)
|
||||
|
||||
// calculate numRows from rowID field, fieldID 0
|
||||
numRows := int64(len(fID2Content[0]))
|
||||
maxRowsPerBinlog = int(Params.DataNodeCfg.FlushInsertBufferSize / (int64(dim) * 4))
|
||||
numBinlogs = int(numRows) / maxRowsPerBinlog
|
||||
if int(numRows)%maxRowsPerBinlog != 0 {
|
||||
addInsertFieldPath(inPaths)
|
||||
|
||||
numRows += int64(currentRows)
|
||||
numBinlogs++
|
||||
}
|
||||
|
||||
for i := 0; i < numBinlogs; i++ {
|
||||
iDatas = append(iDatas, &InsertData{Data: make(map[storage.FieldID]storage.FieldData)})
|
||||
for _, path := range insertField2Path {
|
||||
insertPaths = append(insertPaths, path)
|
||||
}
|
||||
|
||||
for fID, content := range fID2Content {
|
||||
tp, ok := fID2Type[fID]
|
||||
if !ok {
|
||||
log.Warn("no field ID in this schema", zap.Int64("fieldID", fID))
|
||||
return nil, nil, 0, errors.New("Unexpected error")
|
||||
}
|
||||
|
||||
for i := 0; i < numBinlogs; i++ {
|
||||
var c []interface{}
|
||||
|
||||
if i == numBinlogs-1 {
|
||||
c = content[i*maxRowsPerBinlog:]
|
||||
} else {
|
||||
c = content[i*maxRowsPerBinlog : i*maxRowsPerBinlog+maxRowsPerBinlog]
|
||||
}
|
||||
|
||||
fData, err := interface2FieldData(tp, c, int64(len(c)))
|
||||
|
||||
if err != nil {
|
||||
log.Warn("transfer interface to FieldData wrong", zap.Error(err))
|
||||
return nil, nil, 0, err
|
||||
}
|
||||
if fID == pkID {
|
||||
err = segment.updatePKRange(fData)
|
||||
if err != nil {
|
||||
log.Warn("update pk range failed", zap.Error(err))
|
||||
return nil, nil, 0, err
|
||||
}
|
||||
}
|
||||
iDatas[i].Data[fID] = fData
|
||||
}
|
||||
// marshal segment statslog
|
||||
segStats, err := segment.getSegmentStatslog(pkID, pkType)
|
||||
if err != nil {
|
||||
log.Warn("failed to generate segment statslog", zap.Int64("pkID", pkID), zap.Error(err))
|
||||
return nil, nil, nil, 0, err
|
||||
}
|
||||
|
||||
uploadStatsStart := time.Now()
|
||||
statsPaths, err := t.uploadStatsLog(ctxTimeout, targetSegID, partID, segStats, meta)
|
||||
if err != nil {
|
||||
return nil, nil, nil, 0, err
|
||||
}
|
||||
uploadStatsTimeCost += time.Since(uploadStatsStart)
|
||||
|
||||
log.Debug("merge end", zap.Int64("remaining insert numRows", numRows),
|
||||
zap.Int64("expired entities", expired),
|
||||
zap.Float64("elapse in ms", nano2Milli(time.Since(mergeStart))))
|
||||
return iDatas, segment, numRows, nil
|
||||
zap.Int64("expired entities", expired), zap.Int("binlog file number", numBinlogs),
|
||||
zap.Float64("download insert log elapse in ms", nano2Milli(downloadTimeCost)),
|
||||
zap.Float64("upload insert log elapse in ms", nano2Milli(uploadInsertTimeCost)),
|
||||
zap.Float64("upload stats log elapse in ms", nano2Milli(uploadStatsTimeCost)),
|
||||
zap.Float64("merge elapse in ms", nano2Milli(time.Since(mergeStart))))
|
||||
|
||||
return insertPaths, statsPaths, segment, numRows, nil
|
||||
}
|
||||
|
||||
func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
||||
|
@ -382,25 +477,12 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
|||
}()
|
||||
|
||||
var (
|
||||
iItr = make([]iterator, 0)
|
||||
imu sync.Mutex
|
||||
|
||||
// SegmentID to deltaBlobs
|
||||
dblobs = make(map[UniqueID][]*Blob)
|
||||
dmu sync.Mutex
|
||||
|
||||
PKfieldID UniqueID
|
||||
PkType schemapb.DataType
|
||||
)
|
||||
|
||||
// Get PK fieldID
|
||||
for _, fs := range meta.GetSchema().GetFields() {
|
||||
if fs.GetFieldID() >= 100 && typeutil.IsPrimaryFieldType(fs.GetDataType()) && fs.GetIsPrimaryKey() {
|
||||
PKfieldID = fs.GetFieldID()
|
||||
PkType = fs.GetDataType()
|
||||
break
|
||||
}
|
||||
}
|
||||
allPs := make([][]string, 0)
|
||||
|
||||
downloadStart := time.Now()
|
||||
g, gCtx := errgroup.WithContext(ctxTimeout)
|
||||
|
@ -425,26 +507,7 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
|||
for _, f := range s.GetFieldBinlogs() {
|
||||
ps = append(ps, f.GetBinlogs()[idx].GetLogPath())
|
||||
}
|
||||
|
||||
g.Go(func() error {
|
||||
bs, err := t.download(gCtx, ps)
|
||||
if err != nil {
|
||||
log.Warn("download insertlogs wrong")
|
||||
return err
|
||||
}
|
||||
|
||||
itr, err := storage.NewInsertBinlogIterator(bs, PKfieldID, PkType)
|
||||
if err != nil {
|
||||
log.Warn("new insert binlogs Itr wrong")
|
||||
return err
|
||||
}
|
||||
|
||||
imu.Lock()
|
||||
iItr = append(iItr, itr)
|
||||
imu.Unlock()
|
||||
|
||||
return nil
|
||||
})
|
||||
allPs = append(allPs, ps)
|
||||
}
|
||||
|
||||
segID := s.GetSegmentID()
|
||||
|
@ -471,7 +534,7 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
|||
err = g.Wait()
|
||||
downloadEnd := time.Now()
|
||||
defer func() {
|
||||
log.Debug("download elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Float64("elapse", nano2Milli(downloadEnd.Sub(downloadStart))))
|
||||
log.Debug("download deltalogs elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Float64("elapse", nano2Milli(downloadEnd.Sub(downloadStart))))
|
||||
}()
|
||||
|
||||
if err != nil {
|
||||
|
@ -479,39 +542,26 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
|||
return nil, err
|
||||
}
|
||||
|
||||
mergeItr := storage.NewMergeIterator(iItr)
|
||||
|
||||
deltaPk2Ts, deltaBuf, err := t.mergeDeltalogs(dblobs, t.plan.GetTimetravel())
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
iDatas, segment, numRows, err := t.merge(mergeItr, deltaPk2Ts, meta.GetSchema(), t.GetCurrentTime())
|
||||
inPaths, statsPaths, segment, numRows, err := t.merge(ctxTimeout, allPs, targetSegID, partID, meta, deltaPk2Ts)
|
||||
if err != nil {
|
||||
log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// marshal segment statslog
|
||||
segStats, err := segment.getSegmentStatslog(PKfieldID, PkType)
|
||||
if err != nil {
|
||||
log.Warn("failed to generate segment statslog", zap.Int64("pkID", PKfieldID), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
uploadStart := time.Now()
|
||||
segPaths, err := t.upload(ctxTimeout, targetSegID, partID, iDatas, segStats, deltaBuf.delData, meta)
|
||||
uploadDeltaStart := time.Now()
|
||||
deltaInfo, err := t.uploadDeltaLog(ctxTimeout, targetSegID, partID, deltaBuf.delData, meta)
|
||||
if err != nil {
|
||||
log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
log.Debug("upload delta log elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Float64("elapse", nano2Milli(time.Since(uploadDeltaStart))))
|
||||
|
||||
uploadEnd := time.Now()
|
||||
defer func() {
|
||||
log.Debug("upload elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Float64("elapse", nano2Milli(uploadEnd.Sub(uploadStart))))
|
||||
}()
|
||||
|
||||
for _, fbl := range segPaths.deltaInfo {
|
||||
for _, fbl := range deltaInfo {
|
||||
for _, deltaLogInfo := range fbl.GetBinlogs() {
|
||||
deltaLogInfo.LogSize = deltaBuf.GetLogSize()
|
||||
deltaLogInfo.TimestampFrom = deltaBuf.GetTimestampFrom()
|
||||
|
@ -523,9 +573,9 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
|||
pack := &datapb.CompactionResult{
|
||||
PlanID: t.plan.GetPlanID(),
|
||||
SegmentID: targetSegID,
|
||||
InsertLogs: segPaths.inPaths,
|
||||
Field2StatslogPaths: segPaths.statsPaths,
|
||||
Deltalogs: segPaths.deltaInfo,
|
||||
InsertLogs: inPaths,
|
||||
Field2StatslogPaths: statsPaths,
|
||||
Deltalogs: deltaInfo,
|
||||
NumOfRows: numRows,
|
||||
}
|
||||
|
||||
|
@ -577,9 +627,9 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
|||
log.Info("compaction done",
|
||||
zap.Int64("planID", t.plan.GetPlanID()),
|
||||
zap.Int64("targetSegmentID", targetSegID),
|
||||
zap.Int("num of binlog paths", len(segPaths.inPaths)),
|
||||
zap.Int("num of stats paths", len(segPaths.statsPaths)),
|
||||
zap.Int("num of delta paths", len(segPaths.deltaInfo)),
|
||||
zap.Int("num of binlog paths", len(inPaths)),
|
||||
zap.Int("num of stats paths", len(statsPaths)),
|
||||
zap.Int("num of delta paths", len(deltaInfo)),
|
||||
)
|
||||
|
||||
log.Info("overall elapse in ms", zap.Int64("planID", t.plan.GetPlanID()), zap.Float64("elapse", nano2Milli(time.Since(compactStart))))
|
||||
|
|
|
@ -20,6 +20,8 @@ import (
|
|||
"context"
|
||||
"fmt"
|
||||
"math"
|
||||
|
||||
// "math"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
|
@ -261,32 +263,40 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
|
|||
replica, err := newReplica(context.Background(), rc, nil, collectionID)
|
||||
require.NoError(t, err)
|
||||
t.Run("Merge without expiration", func(t *testing.T) {
|
||||
alloc := NewAllocatorFactory(1)
|
||||
mockbIO := &binlogIO{cm, alloc}
|
||||
Params.CommonCfg.EntityExpirationTTL = 0
|
||||
iData := genInsertDataWithExpiredTS()
|
||||
|
||||
iblobs, err := getInsertBlobs(100, iData, meta)
|
||||
require.NoError(t, err)
|
||||
var allPaths [][]string
|
||||
inpath, err := mockbIO.uploadInsertLog(context.Background(), 1, 0, iData, meta)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 12, len(inpath))
|
||||
binlogNum := len(inpath[0].GetBinlogs())
|
||||
assert.Equal(t, 1, binlogNum)
|
||||
|
||||
iitr, err := storage.NewInsertBinlogIterator(iblobs, 106, schemapb.DataType_Int64)
|
||||
require.NoError(t, err)
|
||||
|
||||
mitr := storage.NewMergeIterator([]iterator{iitr})
|
||||
for idx := 0; idx < binlogNum; idx++ {
|
||||
var ps []string
|
||||
for _, path := range inpath {
|
||||
ps = append(ps, path.GetBinlogs()[idx].GetLogPath())
|
||||
}
|
||||
allPaths = append(allPaths, ps)
|
||||
}
|
||||
|
||||
dm := map[interface{}]Timestamp{
|
||||
1: 10000,
|
||||
}
|
||||
|
||||
ct := &compactionTask{
|
||||
Replica: replica,
|
||||
}
|
||||
idata, segment, numOfRow, err := ct.merge(mitr, dm, meta.GetSchema(), ct.GetCurrentTime())
|
||||
ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO}
|
||||
inPaths, statsPaths, _, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, int64(2), numOfRow)
|
||||
assert.Equal(t, 1, len(idata))
|
||||
assert.NotEmpty(t, idata[0].Data)
|
||||
assert.NotNil(t, segment)
|
||||
assert.Equal(t, 1, len(inPaths[0].GetBinlogs()))
|
||||
assert.Equal(t, 1, len(statsPaths))
|
||||
})
|
||||
t.Run("Merge without expiration2", func(t *testing.T) {
|
||||
alloc := NewAllocatorFactory(1)
|
||||
mockbIO := &binlogIO{cm, alloc}
|
||||
Params.CommonCfg.EntityExpirationTTL = 0
|
||||
flushInsertBufferSize := Params.DataNodeCfg.FlushInsertBufferSize
|
||||
defer func() {
|
||||
|
@ -296,106 +306,137 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
|
|||
iData := genInsertDataWithExpiredTS()
|
||||
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
||||
|
||||
iblobs, err := getInsertBlobs(100, iData, meta)
|
||||
require.NoError(t, err)
|
||||
var allPaths [][]string
|
||||
inpath, err := mockbIO.uploadInsertLog(context.Background(), 1, 0, iData, meta)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 12, len(inpath))
|
||||
binlogNum := len(inpath[0].GetBinlogs())
|
||||
assert.Equal(t, 1, binlogNum)
|
||||
|
||||
iitr, err := storage.NewInsertBinlogIterator(iblobs, 106, schemapb.DataType_Int64)
|
||||
require.NoError(t, err)
|
||||
|
||||
mitr := storage.NewMergeIterator([]iterator{iitr})
|
||||
for idx := 0; idx < binlogNum; idx++ {
|
||||
var ps []string
|
||||
for _, path := range inpath {
|
||||
ps = append(ps, path.GetBinlogs()[idx].GetLogPath())
|
||||
}
|
||||
allPaths = append(allPaths, ps)
|
||||
}
|
||||
|
||||
dm := map[interface{}]Timestamp{}
|
||||
|
||||
ct := &compactionTask{
|
||||
Replica: replica,
|
||||
}
|
||||
idata, segment, numOfRow, err := ct.merge(mitr, dm, meta.GetSchema(), ct.GetCurrentTime())
|
||||
ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO}
|
||||
inPaths, statsPaths, _, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, int64(2), numOfRow)
|
||||
assert.Equal(t, 2, len(idata))
|
||||
assert.NotEmpty(t, idata[0].Data)
|
||||
assert.NotEmpty(t, segment)
|
||||
assert.Equal(t, 2, len(inPaths[0].GetBinlogs()))
|
||||
assert.Equal(t, 1, len(statsPaths))
|
||||
assert.Equal(t, 1, len(statsPaths[0].GetBinlogs()))
|
||||
})
|
||||
|
||||
t.Run("Merge with expiration", func(t *testing.T) {
|
||||
alloc := NewAllocatorFactory(1)
|
||||
mockbIO := &binlogIO{cm, alloc}
|
||||
Params.CommonCfg.EntityExpirationTTL = 864000 // 10 days in seconds
|
||||
iData := genInsertDataWithExpiredTS()
|
||||
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
||||
|
||||
iblobs, err := getInsertBlobs(100, iData, meta)
|
||||
require.NoError(t, err)
|
||||
var allPaths [][]string
|
||||
inpath, err := mockbIO.uploadInsertLog(context.Background(), 1, 0, iData, meta)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 12, len(inpath))
|
||||
binlogNum := len(inpath[0].GetBinlogs())
|
||||
assert.Equal(t, 1, binlogNum)
|
||||
|
||||
iitr, err := storage.NewInsertBinlogIterator(iblobs, 106, schemapb.DataType_Int64)
|
||||
require.NoError(t, err)
|
||||
for idx := 0; idx < binlogNum; idx++ {
|
||||
var ps []string
|
||||
for _, path := range inpath {
|
||||
ps = append(ps, path.GetBinlogs()[idx].GetLogPath())
|
||||
}
|
||||
allPaths = append(allPaths, ps)
|
||||
}
|
||||
|
||||
mitr := storage.NewMergeIterator([]iterator{iitr})
|
||||
dm := map[interface{}]Timestamp{
|
||||
1: 10000,
|
||||
}
|
||||
|
||||
ct := &compactionTask{
|
||||
Replica: replica,
|
||||
}
|
||||
idata, segment, numOfRow, err := ct.merge(mitr, dm, meta.GetSchema(), genTimestamp())
|
||||
ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO}
|
||||
inPaths, statsPaths, _, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, int64(1), numOfRow)
|
||||
assert.Equal(t, 1, len(idata))
|
||||
assert.NotEmpty(t, segment)
|
||||
assert.Equal(t, int64(0), numOfRow)
|
||||
assert.Equal(t, 0, len(inPaths))
|
||||
assert.Equal(t, 1, len(statsPaths))
|
||||
})
|
||||
|
||||
t.Run("Merge with meta error", func(t *testing.T) {
|
||||
alloc := NewAllocatorFactory(1)
|
||||
mockbIO := &binlogIO{cm, alloc}
|
||||
Params.CommonCfg.EntityExpirationTTL = 0
|
||||
iData := genInsertDataWithExpiredTS()
|
||||
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
||||
|
||||
iblobs, err := getInsertBlobs(100, iData, meta)
|
||||
require.NoError(t, err)
|
||||
var allPaths [][]string
|
||||
inpath, err := mockbIO.uploadInsertLog(context.Background(), 1, 0, iData, meta)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 12, len(inpath))
|
||||
binlogNum := len(inpath[0].GetBinlogs())
|
||||
assert.Equal(t, 1, binlogNum)
|
||||
|
||||
iitr, err := storage.NewInsertBinlogIterator(iblobs, 106, schemapb.DataType_Int64)
|
||||
require.NoError(t, err)
|
||||
|
||||
mitr := storage.NewMergeIterator([]iterator{iitr})
|
||||
for idx := 0; idx < binlogNum; idx++ {
|
||||
var ps []string
|
||||
for _, path := range inpath {
|
||||
ps = append(ps, path.GetBinlogs()[idx].GetLogPath())
|
||||
}
|
||||
allPaths = append(allPaths, ps)
|
||||
}
|
||||
|
||||
dm := map[interface{}]Timestamp{
|
||||
1: 10000,
|
||||
}
|
||||
|
||||
ct := &compactionTask{
|
||||
Replica: replica,
|
||||
}
|
||||
_, _, _, err = ct.merge(mitr, dm, &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{
|
||||
{DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: "dim", Value: "64"},
|
||||
ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO}
|
||||
_, _, _, _, err = ct.merge(context.Background(), allPaths, 2, 0, &etcdpb.CollectionMeta{
|
||||
Schema: &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{
|
||||
{DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: "dim", Value: "64"},
|
||||
}},
|
||||
}},
|
||||
}}, ct.GetCurrentTime())
|
||||
}, dm)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
|
||||
t.Run("Merge with meta type param error", func(t *testing.T) {
|
||||
alloc := NewAllocatorFactory(1)
|
||||
mockbIO := &binlogIO{cm, alloc}
|
||||
Params.CommonCfg.EntityExpirationTTL = 0
|
||||
iData := genInsertDataWithExpiredTS()
|
||||
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
||||
|
||||
iblobs, err := getInsertBlobs(100, iData, meta)
|
||||
require.NoError(t, err)
|
||||
var allPaths [][]string
|
||||
inpath, err := mockbIO.uploadInsertLog(context.Background(), 1, 0, iData, meta)
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, 12, len(inpath))
|
||||
binlogNum := len(inpath[0].GetBinlogs())
|
||||
assert.Equal(t, 1, binlogNum)
|
||||
|
||||
iitr, err := storage.NewInsertBinlogIterator(iblobs, 106, schemapb.DataType_Int64)
|
||||
require.NoError(t, err)
|
||||
|
||||
mitr := storage.NewMergeIterator([]iterator{iitr})
|
||||
for idx := 0; idx < binlogNum; idx++ {
|
||||
var ps []string
|
||||
for _, path := range inpath {
|
||||
ps = append(ps, path.GetBinlogs()[idx].GetLogPath())
|
||||
}
|
||||
allPaths = append(allPaths, ps)
|
||||
}
|
||||
|
||||
dm := map[interface{}]Timestamp{
|
||||
1: 10000,
|
||||
}
|
||||
|
||||
ct := &compactionTask{
|
||||
Replica: replica,
|
||||
}
|
||||
_, _, _, err = ct.merge(mitr, dm, &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{
|
||||
{DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: "dim", Value: "dim"},
|
||||
}},
|
||||
}}, ct.GetCurrentTime())
|
||||
ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO}
|
||||
|
||||
_, _, _, _, err = ct.merge(context.Background(), allPaths, 2, 0, &etcdpb.CollectionMeta{
|
||||
Schema: &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{
|
||||
{DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
|
||||
{Key: "dim", Value: "dim"},
|
||||
}},
|
||||
}}}, dm)
|
||||
assert.Error(t, err)
|
||||
})
|
||||
})
|
||||
|
|
|
@ -1048,6 +1048,7 @@ type dataCoordConfig struct {
|
|||
|
||||
// --- SEGMENTS ---
|
||||
SegmentMaxSize float64
|
||||
DiskSegmentMaxSize float64
|
||||
SegmentSealProportion float64
|
||||
SegAssignmentExpiration int64
|
||||
SegmentMaxLifetime time.Duration
|
||||
|
@ -1084,6 +1085,7 @@ func (p *dataCoordConfig) init(base *BaseTable) {
|
|||
p.initChannelWatchPrefix()
|
||||
|
||||
p.initSegmentMaxSize()
|
||||
p.initDiskSegmentMaxSize()
|
||||
p.initSegmentSealProportion()
|
||||
p.initSegAssignmentExpiration()
|
||||
p.initSegmentMaxLifetime()
|
||||
|
@ -1114,6 +1116,10 @@ func (p *dataCoordConfig) initSegmentMaxSize() {
|
|||
p.SegmentMaxSize = p.Base.ParseFloatWithDefault("dataCoord.segment.maxSize", 512.0)
|
||||
}
|
||||
|
||||
func (p *dataCoordConfig) initDiskSegmentMaxSize() {
|
||||
p.DiskSegmentMaxSize = p.Base.ParseFloatWithDefault("dataCoord.segment.diskSegmentMaxSize", 512.0*4)
|
||||
}
|
||||
|
||||
func (p *dataCoordConfig) initSegmentSealProportion() {
|
||||
p.SegmentSealProportion = p.Base.ParseFloatWithDefault("dataCoord.segment.sealProportion", 0.25)
|
||||
}
|
||||
|
|
|
@ -39,7 +39,6 @@ def assert_fail(func, milvus_client, **params):
|
|||
try:
|
||||
methodcaller(func, **params)(milvus_client)
|
||||
except Exception as e:
|
||||
logger.debug("11111111111111111111111111")
|
||||
logger.info(str(e))
|
||||
pass
|
||||
else:
|
||||
|
|
Loading…
Reference in New Issue