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:
|
segment:
|
||||||
maxSize: 512 # Maximum size of a segment in MB
|
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
|
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
|
assignmentExpiration: 2000 # The time of the assignment expiration in ms
|
||||||
maxLife: 86400 # The max lifetime of segment in seconds, 24*60*60
|
maxLife: 86400 # The max lifetime of segment in seconds, 24*60*60
|
||||||
|
|
|
@ -18,6 +18,7 @@ package datacoord
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"fmt"
|
||||||
"sort"
|
"sort"
|
||||||
"sync"
|
"sync"
|
||||||
"time"
|
"time"
|
||||||
|
@ -25,7 +26,10 @@ import (
|
||||||
"github.com/milvus-io/milvus/api/commonpb"
|
"github.com/milvus-io/milvus/api/commonpb"
|
||||||
"github.com/milvus-io/milvus/internal/log"
|
"github.com/milvus-io/milvus/internal/log"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"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/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"
|
"github.com/milvus-io/milvus/internal/util/logutil"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
)
|
)
|
||||||
|
@ -70,6 +74,7 @@ type compactionTrigger struct {
|
||||||
wg sync.WaitGroup
|
wg sync.WaitGroup
|
||||||
segRefer *SegmentReferenceManager
|
segRefer *SegmentReferenceManager
|
||||||
indexCoord types.IndexCoord
|
indexCoord types.IndexCoord
|
||||||
|
estimateDiskSegmentPolicy calUpperLimitPolicy
|
||||||
}
|
}
|
||||||
|
|
||||||
func newCompactionTrigger(
|
func newCompactionTrigger(
|
||||||
|
@ -86,6 +91,7 @@ func newCompactionTrigger(
|
||||||
compactionHandler: compactionHandler,
|
compactionHandler: compactionHandler,
|
||||||
segRefer: segRefer,
|
segRefer: segRefer,
|
||||||
indexCoord: indexCoord,
|
indexCoord: indexCoord,
|
||||||
|
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -228,6 +234,48 @@ func getPlanIDs(plans []*datapb.CompactionPlan) []int64 {
|
||||||
return ids
|
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) {
|
func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) {
|
||||||
t.forceMu.Lock()
|
t.forceMu.Lock()
|
||||||
defer t.forceMu.Unlock()
|
defer t.forceMu.Unlock()
|
||||||
|
@ -243,8 +291,15 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) {
|
||||||
if !signal.isForce && t.compactionHandler.isFull() {
|
if !signal.isForce && t.compactionHandler.isFull() {
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
|
|
||||||
group.segments = FilterInIndexedSegments(t.meta, t.indexCoord, group.segments...)
|
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)
|
plans := t.generatePlans(group.segments, signal.isForce, signal.compactTime)
|
||||||
for _, plan := range plans {
|
for _, plan := range plans {
|
||||||
if !signal.isForce && t.compactionHandler.isFull() {
|
if !signal.isForce && t.compactionHandler.isFull() {
|
||||||
|
@ -292,6 +347,12 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
|
||||||
channel := segment.GetInsertChannel()
|
channel := segment.GetInsertChannel()
|
||||||
partitionID := segment.GetPartitionID()
|
partitionID := segment.GetPartitionID()
|
||||||
segments := t.getCandidateSegments(channel, partitionID)
|
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)
|
plans := t.generatePlans(segments, signal.isForce, signal.compactTime)
|
||||||
for _, plan := range plans {
|
for _, plan := range plans {
|
||||||
if t.compactionHandler.isFull() {
|
if t.compactionHandler.isFull() {
|
||||||
|
|
|
@ -24,11 +24,8 @@ import (
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/api/commonpb"
|
"github.com/milvus-io/milvus/api/commonpb"
|
||||||
"github.com/milvus-io/milvus/api/schemapb"
|
"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/datapb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/indexpb"
|
|
||||||
"github.com/stretchr/testify/assert"
|
"github.com/stretchr/testify/assert"
|
||||||
"github.com/stretchr/testify/mock"
|
|
||||||
)
|
)
|
||||||
|
|
||||||
type spyCompactionHandler struct {
|
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(),
|
newMockAllocator(),
|
||||||
|
@ -232,27 +302,7 @@ func Test_compactionTrigger_force(t *testing.T) {
|
||||||
}
|
}
|
||||||
for _, tt := range tests {
|
for _, tt := range tests {
|
||||||
t.Run(tt.name, func(t *testing.T) {
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
indexCoord := mocks.NewMockIndexCoord(t)
|
indexCoord := newMockIndexCoord()
|
||||||
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)
|
|
||||||
}
|
|
||||||
|
|
||||||
tr := &compactionTrigger{
|
tr := &compactionTrigger{
|
||||||
meta: tt.fields.meta,
|
meta: tt.fields.meta,
|
||||||
|
@ -270,6 +320,162 @@ func Test_compactionTrigger_force(t *testing.T) {
|
||||||
sortPlanCompactionBinlogs(plan)
|
sortPlanCompactionBinlogs(plan)
|
||||||
assert.EqualValues(t, tt.wantPlans[0], 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 {
|
for _, tt := range tests {
|
||||||
t.Run(tt.name, func(t *testing.T) {
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
indexCoord := mocks.NewMockIndexCoord(t)
|
indexCoord := newMockIndexCoord()
|
||||||
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)
|
|
||||||
}
|
|
||||||
|
|
||||||
tr := &compactionTrigger{
|
tr := &compactionTrigger{
|
||||||
meta: tt.fields.meta,
|
meta: tt.fields.meta,
|
||||||
|
@ -603,27 +789,8 @@ func Test_compactionTrigger_noplan(t *testing.T) {
|
||||||
}
|
}
|
||||||
for _, tt := range tests {
|
for _, tt := range tests {
|
||||||
t.Run(tt.name, func(t *testing.T) {
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
indexCoord := mocks.NewMockIndexCoord(t)
|
indexCoord := newMockIndexCoord()
|
||||||
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)
|
|
||||||
}
|
|
||||||
tr := &compactionTrigger{
|
tr := &compactionTrigger{
|
||||||
meta: tt.fields.meta,
|
meta: tt.fields.meta,
|
||||||
allocator: tt.fields.allocator,
|
allocator: tt.fields.allocator,
|
||||||
|
@ -813,27 +980,8 @@ func Test_compactionTrigger_smallfiles(t *testing.T) {
|
||||||
}
|
}
|
||||||
for _, tt := range tests {
|
for _, tt := range tests {
|
||||||
t.Run(tt.name, func(t *testing.T) {
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
indexCoord := mocks.NewMockIndexCoord(t)
|
indexCoord := newMockIndexCoord()
|
||||||
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)
|
|
||||||
}
|
|
||||||
tr := &compactionTrigger{
|
tr := &compactionTrigger{
|
||||||
meta: tt.fields.meta,
|
meta: tt.fields.meta,
|
||||||
allocator: tt.fields.allocator,
|
allocator: tt.fields.allocator,
|
||||||
|
@ -953,27 +1101,8 @@ func Test_compactionTrigger_noplan_random_size(t *testing.T) {
|
||||||
}
|
}
|
||||||
for _, tt := range tests {
|
for _, tt := range tests {
|
||||||
t.Run(tt.name, func(t *testing.T) {
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
indexCoord := mocks.NewMockIndexCoord(t)
|
indexCoord := newMockIndexCoord()
|
||||||
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)
|
|
||||||
}
|
|
||||||
tr := &compactionTrigger{
|
tr := &compactionTrigger{
|
||||||
meta: tt.fields.meta,
|
meta: tt.fields.meta,
|
||||||
allocator: tt.fields.allocator,
|
allocator: tt.fields.allocator,
|
||||||
|
@ -1024,8 +1153,9 @@ func Test_compactionTrigger_noplan_random_size(t *testing.T) {
|
||||||
func Test_compactionTrigger_shouldDoSingleCompaction(t *testing.T) {
|
func Test_compactionTrigger_shouldDoSingleCompaction(t *testing.T) {
|
||||||
Params.Init()
|
Params.Init()
|
||||||
|
|
||||||
|
indexCoord := newMockIndexCoord()
|
||||||
trigger := newCompactionTrigger(&meta{}, &compactionPlanHandler{}, newMockAllocator(),
|
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.
|
// Test too many files.
|
||||||
var binlogs []*datapb.FieldBinlog
|
var binlogs []*datapb.FieldBinlog
|
||||||
|
@ -1157,8 +1287,9 @@ func Test_newCompactionTrigger(t *testing.T) {
|
||||||
}
|
}
|
||||||
for _, tt := range tests {
|
for _, tt := range tests {
|
||||||
t.Run(tt.name, func(t *testing.T) {
|
t.Run(tt.name, func(t *testing.T) {
|
||||||
|
indexCoord := newMockIndexCoord()
|
||||||
got := newCompactionTrigger(tt.args.meta, tt.args.compactionHandler, tt.args.allocator,
|
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.meta, got.meta)
|
||||||
assert.Equal(t, tt.args.compactionHandler, got.compactionHandler)
|
assert.Equal(t, tt.args.compactionHandler, got.compactionHandler)
|
||||||
assert.Equal(t, tt.args.allocator, got.allocator)
|
assert.Equal(t, tt.args.allocator, got.allocator)
|
||||||
|
@ -1168,8 +1299,9 @@ func Test_newCompactionTrigger(t *testing.T) {
|
||||||
|
|
||||||
func Test_handleSignal(t *testing.T) {
|
func Test_handleSignal(t *testing.T) {
|
||||||
|
|
||||||
|
indexCoord := newMockIndexCoord()
|
||||||
got := newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{}, newMockAllocator(),
|
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{
|
signal := &compactionSignal{
|
||||||
segmentID: 1,
|
segmentID: 1,
|
||||||
}
|
}
|
||||||
|
|
|
@ -28,9 +28,11 @@ import (
|
||||||
"github.com/milvus-io/milvus/internal/kv"
|
"github.com/milvus-io/milvus/internal/kv"
|
||||||
memkv "github.com/milvus-io/milvus/internal/kv/mem"
|
memkv "github.com/milvus-io/milvus/internal/kv/mem"
|
||||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
"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/internalpb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/proxypb"
|
"github.com/milvus-io/milvus/internal/proto/proxypb"
|
||||||
"github.com/milvus-io/milvus/internal/proto/rootcoordpb"
|
"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/metricsinfo"
|
||||||
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
"github.com/milvus-io/milvus/internal/util/tsoutil"
|
||||||
"github.com/milvus-io/milvus/internal/util/typeutil"
|
"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) {}
|
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
|
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
|
// AllocatePolicy helper function definition to allocate Segment space
|
||||||
type AllocatePolicy func(segments []*SegmentInfo, count int64,
|
type AllocatePolicy func(segments []*SegmentInfo, count int64,
|
||||||
maxCountPerSegment int64) ([]*Allocation, []*Allocation)
|
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.
|
// 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.
|
// 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)
|
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 {
|
type binlogIO struct {
|
||||||
|
@ -102,6 +105,33 @@ func (b *binlogIO) download(ctx context.Context, paths []string) ([]*Blob, error
|
||||||
return rst, nil
|
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 {
|
type segPaths struct {
|
||||||
inPaths []*datapb.FieldBinlog
|
inPaths []*datapb.FieldBinlog
|
||||||
statsPaths []*datapb.FieldBinlog
|
statsPaths []*datapb.FieldBinlog
|
||||||
|
@ -207,24 +237,9 @@ func (b *binlogIO) upload(
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
err = errStart
|
err = b.uploadSegmentFiles(ctx, meta.GetID(), segID, kvs)
|
||||||
for err != nil {
|
if err != nil {
|
||||||
select {
|
return nil, err
|
||||||
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)
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
return p, nil
|
return p, nil
|
||||||
}
|
}
|
||||||
|
@ -310,3 +325,138 @@ func (b *binlogIO) idxGenerator(n int, done <-chan struct{}) (<-chan UniqueID, e
|
||||||
|
|
||||||
return rt, nil
|
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{}
|
f := &MetaFactory{}
|
||||||
meta := f.GetCollectionMeta(UniqueID(10001), "uploads", schemapb.DataType_Int64)
|
meta := f.GetCollectionMeta(UniqueID(10001), "uploads", schemapb.DataType_Int64)
|
||||||
|
|
||||||
|
//pkFieldID := int64(106)
|
||||||
iData := genInsertData()
|
iData := genInsertData()
|
||||||
pk := newInt64PrimaryKey(888)
|
pk := newInt64PrimaryKey(888)
|
||||||
dData := &DeleteData{
|
dData := &DeleteData{
|
||||||
|
@ -69,11 +70,40 @@ func TestBinlogIOInterfaceMethods(t *testing.T) {
|
||||||
assert.NotNil(t, p.deltaInfo)
|
assert.NotNil(t, p.deltaInfo)
|
||||||
|
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
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()
|
cancel()
|
||||||
|
|
||||||
p, err = b.upload(ctx, 1, 10, []*InsertData{iData}, []byte{}, dData, meta)
|
p, err = b.upload(ctx, 1, 10, []*InsertData{iData}, []byte{}, dData, meta)
|
||||||
assert.EqualError(t, err, errUploadToBlobStorage.Error())
|
assert.EqualError(t, err, errUploadToBlobStorage.Error())
|
||||||
assert.Nil(t, p)
|
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) {
|
t.Run("Test upload error", func(t *testing.T) {
|
||||||
|
@ -122,11 +152,35 @@ func TestBinlogIOInterfaceMethods(t *testing.T) {
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
assert.Empty(t, p)
|
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
|
alloc.isvalid = false
|
||||||
p, err = bin.upload(ctx, 1, 10, []*InsertData{iData}, []byte{}, dData, meta)
|
p, err = bin.upload(ctx, 1, 10, []*InsertData{iData}, []byte{}, dData, meta)
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
assert.Empty(t, p)
|
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
|
alloc.isvalid = true
|
||||||
for _, field := range meta.GetSchema().GetFields() {
|
for _, field := range meta.GetSchema().GetFields() {
|
||||||
field.IsPrimaryKey = false
|
field.IsPrimaryKey = false
|
||||||
|
@ -135,6 +189,18 @@ func TestBinlogIOInterfaceMethods(t *testing.T) {
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
assert.Empty(t, p)
|
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()
|
cancel()
|
||||||
})
|
})
|
||||||
|
|
||||||
|
|
|
@ -187,7 +187,57 @@ func nano2Milli(nano time.Duration) float64 {
|
||||||
return float64(nano) / float64(time.Millisecond)
|
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()))
|
log := log.With(zap.Int64("planID", t.getPlanID()))
|
||||||
mergeStart := time.Now()
|
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
|
dim int // dimension of float/binary vector field
|
||||||
maxRowsPerBinlog int // maximum rows populating one binlog
|
maxRowsPerBinlog int // maximum rows populating one binlog
|
||||||
numBinlogs int // binlog number
|
numBinlogs int // binlog number
|
||||||
|
numRows int64 // the number of rows uploaded
|
||||||
expired int64 // the number of expired entity
|
expired int64 // the number of expired entity
|
||||||
err error
|
err error
|
||||||
|
|
||||||
// statslog generation
|
// statslog generation
|
||||||
segment *Segment // empty segment used for bf generation
|
segment *Segment // empty segment used for bf generation
|
||||||
pkID UniqueID
|
pkID UniqueID
|
||||||
|
pkType schemapb.DataType
|
||||||
|
|
||||||
iDatas = make([]*InsertData, 0)
|
|
||||||
fID2Type = make(map[UniqueID]schemapb.DataType)
|
fID2Type = make(map[UniqueID]schemapb.DataType)
|
||||||
fID2Content = make(map[UniqueID][]interface{})
|
fID2Content = make(map[UniqueID][]interface{})
|
||||||
|
|
||||||
|
insertField2Path = make(map[UniqueID]*datapb.FieldBinlog)
|
||||||
|
insertPaths = make([]*datapb.FieldBinlog, 0)
|
||||||
)
|
)
|
||||||
|
|
||||||
isDeletedValue := func(v *storage.Value) bool {
|
isDeletedValue := func(v *storage.Value) bool {
|
||||||
|
@ -218,11 +272,24 @@ func (t *compactionTask) merge(mergeItr iterator, delta map[interface{}]Timestam
|
||||||
segment = &Segment{}
|
segment = &Segment{}
|
||||||
t.Replica.initSegmentBloomFilter(segment)
|
t.Replica.initSegmentBloomFilter(segment)
|
||||||
|
|
||||||
// get dim
|
addInsertFieldPath := func(inPaths map[UniqueID]*datapb.FieldBinlog) {
|
||||||
for _, fs := range schema.GetFields() {
|
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()
|
fID2Type[fs.GetFieldID()] = fs.GetDataType()
|
||||||
if fs.GetIsPrimaryKey() {
|
if fs.GetIsPrimaryKey() && fs.GetFieldID() >= 100 && typeutil.IsPrimaryFieldType(fs.GetDataType()) {
|
||||||
pkID = fs.GetFieldID()
|
pkID = fs.GetFieldID()
|
||||||
|
pkType = fs.GetDataType()
|
||||||
}
|
}
|
||||||
if fs.GetDataType() == schemapb.DataType_FloatVector ||
|
if fs.GetDataType() == schemapb.DataType_FloatVector ||
|
||||||
fs.GetDataType() == schemapb.DataType_BinaryVector {
|
fs.GetDataType() == schemapb.DataType_BinaryVector {
|
||||||
|
@ -230,7 +297,7 @@ func (t *compactionTask) merge(mergeItr iterator, delta map[interface{}]Timestam
|
||||||
if t.Key == "dim" {
|
if t.Key == "dim" {
|
||||||
if dim, err = strconv.Atoi(t.Value); err != nil {
|
if dim, err = strconv.Atoi(t.Value); err != nil {
|
||||||
log.Warn("strconv wrong on get dim", zap.Error(err))
|
log.Warn("strconv wrong on get dim", zap.Error(err))
|
||||||
return nil, nil, 0, err
|
return nil, nil, nil, 0, err
|
||||||
}
|
}
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
|
@ -239,14 +306,34 @@ func (t *compactionTask) merge(mergeItr iterator, delta map[interface{}]Timestam
|
||||||
}
|
}
|
||||||
|
|
||||||
expired = 0
|
expired = 0
|
||||||
for mergeItr.HasNext() {
|
numRows = 0
|
||||||
// no error if HasNext() returns true
|
numBinlogs = 0
|
||||||
vInter, _ := mergeItr.Next()
|
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)
|
||||||
|
|
||||||
|
iter, err := storage.NewInsertBinlogIterator(data, pkID, pkType)
|
||||||
|
if err != nil {
|
||||||
|
log.Warn("new insert binlogs Itr wrong")
|
||||||
|
return nil, nil, nil, 0, err
|
||||||
|
}
|
||||||
|
for iter.HasNext() {
|
||||||
|
vInter, _ := iter.Next()
|
||||||
v, ok := vInter.(*storage.Value)
|
v, ok := vInter.(*storage.Value)
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Warn("transfer interface to Value wrong")
|
log.Warn("transfer interface to Value wrong")
|
||||||
return nil, nil, 0, errors.New("unexpected error")
|
return nil, nil, nil, 0, errors.New("unexpected error")
|
||||||
}
|
}
|
||||||
|
|
||||||
if isDeletedValue(v) {
|
if isDeletedValue(v) {
|
||||||
|
@ -263,7 +350,7 @@ func (t *compactionTask) merge(mergeItr iterator, delta map[interface{}]Timestam
|
||||||
row, ok := v.Value.(map[UniqueID]interface{})
|
row, ok := v.Value.(map[UniqueID]interface{})
|
||||||
if !ok {
|
if !ok {
|
||||||
log.Warn("transfer interface to map wrong")
|
log.Warn("transfer interface to map wrong")
|
||||||
return nil, nil, 0, errors.New("unexpected error")
|
return nil, nil, nil, 0, errors.New("unexpected error")
|
||||||
}
|
}
|
||||||
|
|
||||||
for fID, vInter := range row {
|
for fID, vInter := range row {
|
||||||
|
@ -272,57 +359,65 @@ func (t *compactionTask) merge(mergeItr iterator, delta map[interface{}]Timestam
|
||||||
}
|
}
|
||||||
fID2Content[fID] = append(fID2Content[fID], vInter)
|
fID2Content[fID] = append(fID2Content[fID], vInter)
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
// calculate numRows from rowID field, fieldID 0
|
currentRows++
|
||||||
numRows := int64(len(fID2Content[0]))
|
|
||||||
maxRowsPerBinlog = int(Params.DataNodeCfg.FlushInsertBufferSize / (int64(dim) * 4))
|
if currentRows == maxRowsPerBinlog {
|
||||||
numBinlogs = int(numRows) / maxRowsPerBinlog
|
uploadInsertStart := time.Now()
|
||||||
if int(numRows)%maxRowsPerBinlog != 0 {
|
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++
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
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)
|
||||||
|
|
||||||
|
addInsertFieldPath(inPaths)
|
||||||
|
|
||||||
|
numRows += int64(currentRows)
|
||||||
numBinlogs++
|
numBinlogs++
|
||||||
}
|
}
|
||||||
|
|
||||||
for i := 0; i < numBinlogs; i++ {
|
for _, path := range insertField2Path {
|
||||||
iDatas = append(iDatas, &InsertData{Data: make(map[storage.FieldID]storage.FieldData)})
|
insertPaths = append(insertPaths, path)
|
||||||
}
|
}
|
||||||
|
|
||||||
for fID, content := range fID2Content {
|
// marshal segment statslog
|
||||||
tp, ok := fID2Type[fID]
|
segStats, err := segment.getSegmentStatslog(pkID, pkType)
|
||||||
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 {
|
if err != nil {
|
||||||
log.Warn("transfer interface to FieldData wrong", zap.Error(err))
|
log.Warn("failed to generate segment statslog", zap.Int64("pkID", pkID), zap.Error(err))
|
||||||
return nil, nil, 0, err
|
return nil, nil, nil, 0, err
|
||||||
}
|
}
|
||||||
if fID == pkID {
|
|
||||||
err = segment.updatePKRange(fData)
|
uploadStatsStart := time.Now()
|
||||||
|
statsPaths, err := t.uploadStatsLog(ctxTimeout, targetSegID, partID, segStats, meta)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Warn("update pk range failed", zap.Error(err))
|
return nil, nil, nil, 0, err
|
||||||
return nil, nil, 0, err
|
|
||||||
}
|
|
||||||
}
|
|
||||||
iDatas[i].Data[fID] = fData
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
uploadStatsTimeCost += time.Since(uploadStatsStart)
|
||||||
|
|
||||||
log.Debug("merge end", zap.Int64("remaining insert numRows", numRows),
|
log.Debug("merge end", zap.Int64("remaining insert numRows", numRows),
|
||||||
zap.Int64("expired entities", expired),
|
zap.Int64("expired entities", expired), zap.Int("binlog file number", numBinlogs),
|
||||||
zap.Float64("elapse in ms", nano2Milli(time.Since(mergeStart))))
|
zap.Float64("download insert log elapse in ms", nano2Milli(downloadTimeCost)),
|
||||||
return iDatas, segment, numRows, nil
|
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) {
|
func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
||||||
|
@ -382,25 +477,12 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
||||||
}()
|
}()
|
||||||
|
|
||||||
var (
|
var (
|
||||||
iItr = make([]iterator, 0)
|
|
||||||
imu sync.Mutex
|
|
||||||
|
|
||||||
// SegmentID to deltaBlobs
|
// SegmentID to deltaBlobs
|
||||||
dblobs = make(map[UniqueID][]*Blob)
|
dblobs = make(map[UniqueID][]*Blob)
|
||||||
dmu sync.Mutex
|
dmu sync.Mutex
|
||||||
|
|
||||||
PKfieldID UniqueID
|
|
||||||
PkType schemapb.DataType
|
|
||||||
)
|
)
|
||||||
|
|
||||||
// Get PK fieldID
|
allPs := make([][]string, 0)
|
||||||
for _, fs := range meta.GetSchema().GetFields() {
|
|
||||||
if fs.GetFieldID() >= 100 && typeutil.IsPrimaryFieldType(fs.GetDataType()) && fs.GetIsPrimaryKey() {
|
|
||||||
PKfieldID = fs.GetFieldID()
|
|
||||||
PkType = fs.GetDataType()
|
|
||||||
break
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
downloadStart := time.Now()
|
downloadStart := time.Now()
|
||||||
g, gCtx := errgroup.WithContext(ctxTimeout)
|
g, gCtx := errgroup.WithContext(ctxTimeout)
|
||||||
|
@ -425,26 +507,7 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
||||||
for _, f := range s.GetFieldBinlogs() {
|
for _, f := range s.GetFieldBinlogs() {
|
||||||
ps = append(ps, f.GetBinlogs()[idx].GetLogPath())
|
ps = append(ps, f.GetBinlogs()[idx].GetLogPath())
|
||||||
}
|
}
|
||||||
|
allPs = append(allPs, ps)
|
||||||
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
|
|
||||||
})
|
|
||||||
}
|
}
|
||||||
|
|
||||||
segID := s.GetSegmentID()
|
segID := s.GetSegmentID()
|
||||||
|
@ -471,7 +534,7 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
||||||
err = g.Wait()
|
err = g.Wait()
|
||||||
downloadEnd := time.Now()
|
downloadEnd := time.Now()
|
||||||
defer func() {
|
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 {
|
if err != nil {
|
||||||
|
@ -479,39 +542,26 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
mergeItr := storage.NewMergeIterator(iItr)
|
|
||||||
|
|
||||||
deltaPk2Ts, deltaBuf, err := t.mergeDeltalogs(dblobs, t.plan.GetTimetravel())
|
deltaPk2Ts, deltaBuf, err := t.mergeDeltalogs(dblobs, t.plan.GetTimetravel())
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
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 {
|
if err != nil {
|
||||||
log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
|
log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
// marshal segment statslog
|
uploadDeltaStart := time.Now()
|
||||||
segStats, err := segment.getSegmentStatslog(PKfieldID, PkType)
|
deltaInfo, err := t.uploadDeltaLog(ctxTimeout, targetSegID, partID, deltaBuf.delData, meta)
|
||||||
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)
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
|
log.Error("compact wrong", zap.Int64("planID", t.plan.GetPlanID()), zap.Error(err))
|
||||||
return nil, 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()
|
for _, fbl := range deltaInfo {
|
||||||
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 _, deltaLogInfo := range fbl.GetBinlogs() {
|
for _, deltaLogInfo := range fbl.GetBinlogs() {
|
||||||
deltaLogInfo.LogSize = deltaBuf.GetLogSize()
|
deltaLogInfo.LogSize = deltaBuf.GetLogSize()
|
||||||
deltaLogInfo.TimestampFrom = deltaBuf.GetTimestampFrom()
|
deltaLogInfo.TimestampFrom = deltaBuf.GetTimestampFrom()
|
||||||
|
@ -523,9 +573,9 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
||||||
pack := &datapb.CompactionResult{
|
pack := &datapb.CompactionResult{
|
||||||
PlanID: t.plan.GetPlanID(),
|
PlanID: t.plan.GetPlanID(),
|
||||||
SegmentID: targetSegID,
|
SegmentID: targetSegID,
|
||||||
InsertLogs: segPaths.inPaths,
|
InsertLogs: inPaths,
|
||||||
Field2StatslogPaths: segPaths.statsPaths,
|
Field2StatslogPaths: statsPaths,
|
||||||
Deltalogs: segPaths.deltaInfo,
|
Deltalogs: deltaInfo,
|
||||||
NumOfRows: numRows,
|
NumOfRows: numRows,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -577,9 +627,9 @@ func (t *compactionTask) compact() (*datapb.CompactionResult, error) {
|
||||||
log.Info("compaction done",
|
log.Info("compaction done",
|
||||||
zap.Int64("planID", t.plan.GetPlanID()),
|
zap.Int64("planID", t.plan.GetPlanID()),
|
||||||
zap.Int64("targetSegmentID", targetSegID),
|
zap.Int64("targetSegmentID", targetSegID),
|
||||||
zap.Int("num of binlog paths", len(segPaths.inPaths)),
|
zap.Int("num of binlog paths", len(inPaths)),
|
||||||
zap.Int("num of stats paths", len(segPaths.statsPaths)),
|
zap.Int("num of stats paths", len(statsPaths)),
|
||||||
zap.Int("num of delta paths", len(segPaths.deltaInfo)),
|
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))))
|
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"
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"math"
|
"math"
|
||||||
|
|
||||||
|
// "math"
|
||||||
"testing"
|
"testing"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
|
@ -261,32 +263,40 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
|
||||||
replica, err := newReplica(context.Background(), rc, nil, collectionID)
|
replica, err := newReplica(context.Background(), rc, nil, collectionID)
|
||||||
require.NoError(t, err)
|
require.NoError(t, err)
|
||||||
t.Run("Merge without expiration", func(t *testing.T) {
|
t.Run("Merge without expiration", func(t *testing.T) {
|
||||||
|
alloc := NewAllocatorFactory(1)
|
||||||
|
mockbIO := &binlogIO{cm, alloc}
|
||||||
Params.CommonCfg.EntityExpirationTTL = 0
|
Params.CommonCfg.EntityExpirationTTL = 0
|
||||||
iData := genInsertDataWithExpiredTS()
|
iData := genInsertDataWithExpiredTS()
|
||||||
|
|
||||||
iblobs, err := getInsertBlobs(100, iData, meta)
|
var allPaths [][]string
|
||||||
require.NoError(t, err)
|
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)
|
for idx := 0; idx < binlogNum; idx++ {
|
||||||
require.NoError(t, err)
|
var ps []string
|
||||||
|
for _, path := range inpath {
|
||||||
mitr := storage.NewMergeIterator([]iterator{iitr})
|
ps = append(ps, path.GetBinlogs()[idx].GetLogPath())
|
||||||
|
}
|
||||||
|
allPaths = append(allPaths, ps)
|
||||||
|
}
|
||||||
|
|
||||||
dm := map[interface{}]Timestamp{
|
dm := map[interface{}]Timestamp{
|
||||||
1: 10000,
|
1: 10000,
|
||||||
}
|
}
|
||||||
|
|
||||||
ct := &compactionTask{
|
ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO}
|
||||||
Replica: replica,
|
inPaths, statsPaths, _, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm)
|
||||||
}
|
|
||||||
idata, segment, numOfRow, err := ct.merge(mitr, dm, meta.GetSchema(), ct.GetCurrentTime())
|
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
assert.Equal(t, int64(2), numOfRow)
|
assert.Equal(t, int64(2), numOfRow)
|
||||||
assert.Equal(t, 1, len(idata))
|
assert.Equal(t, 1, len(inPaths[0].GetBinlogs()))
|
||||||
assert.NotEmpty(t, idata[0].Data)
|
assert.Equal(t, 1, len(statsPaths))
|
||||||
assert.NotNil(t, segment)
|
|
||||||
})
|
})
|
||||||
t.Run("Merge without expiration2", func(t *testing.T) {
|
t.Run("Merge without expiration2", func(t *testing.T) {
|
||||||
|
alloc := NewAllocatorFactory(1)
|
||||||
|
mockbIO := &binlogIO{cm, alloc}
|
||||||
Params.CommonCfg.EntityExpirationTTL = 0
|
Params.CommonCfg.EntityExpirationTTL = 0
|
||||||
flushInsertBufferSize := Params.DataNodeCfg.FlushInsertBufferSize
|
flushInsertBufferSize := Params.DataNodeCfg.FlushInsertBufferSize
|
||||||
defer func() {
|
defer func() {
|
||||||
|
@ -296,106 +306,137 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
|
||||||
iData := genInsertDataWithExpiredTS()
|
iData := genInsertDataWithExpiredTS()
|
||||||
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
||||||
|
|
||||||
iblobs, err := getInsertBlobs(100, iData, meta)
|
var allPaths [][]string
|
||||||
require.NoError(t, err)
|
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)
|
for idx := 0; idx < binlogNum; idx++ {
|
||||||
require.NoError(t, err)
|
var ps []string
|
||||||
|
for _, path := range inpath {
|
||||||
mitr := storage.NewMergeIterator([]iterator{iitr})
|
ps = append(ps, path.GetBinlogs()[idx].GetLogPath())
|
||||||
|
}
|
||||||
|
allPaths = append(allPaths, ps)
|
||||||
|
}
|
||||||
|
|
||||||
dm := map[interface{}]Timestamp{}
|
dm := map[interface{}]Timestamp{}
|
||||||
|
|
||||||
ct := &compactionTask{
|
ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO}
|
||||||
Replica: replica,
|
inPaths, statsPaths, _, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm)
|
||||||
}
|
|
||||||
idata, segment, numOfRow, err := ct.merge(mitr, dm, meta.GetSchema(), ct.GetCurrentTime())
|
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
assert.Equal(t, int64(2), numOfRow)
|
assert.Equal(t, int64(2), numOfRow)
|
||||||
assert.Equal(t, 2, len(idata))
|
assert.Equal(t, 2, len(inPaths[0].GetBinlogs()))
|
||||||
assert.NotEmpty(t, idata[0].Data)
|
assert.Equal(t, 1, len(statsPaths))
|
||||||
assert.NotEmpty(t, segment)
|
assert.Equal(t, 1, len(statsPaths[0].GetBinlogs()))
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("Merge with expiration", func(t *testing.T) {
|
t.Run("Merge with expiration", func(t *testing.T) {
|
||||||
|
alloc := NewAllocatorFactory(1)
|
||||||
|
mockbIO := &binlogIO{cm, alloc}
|
||||||
Params.CommonCfg.EntityExpirationTTL = 864000 // 10 days in seconds
|
Params.CommonCfg.EntityExpirationTTL = 864000 // 10 days in seconds
|
||||||
iData := genInsertDataWithExpiredTS()
|
iData := genInsertDataWithExpiredTS()
|
||||||
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
||||||
|
|
||||||
iblobs, err := getInsertBlobs(100, iData, meta)
|
var allPaths [][]string
|
||||||
require.NoError(t, err)
|
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)
|
for idx := 0; idx < binlogNum; idx++ {
|
||||||
require.NoError(t, err)
|
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{
|
dm := map[interface{}]Timestamp{
|
||||||
1: 10000,
|
1: 10000,
|
||||||
}
|
}
|
||||||
|
|
||||||
ct := &compactionTask{
|
ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO}
|
||||||
Replica: replica,
|
inPaths, statsPaths, _, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm)
|
||||||
}
|
|
||||||
idata, segment, numOfRow, err := ct.merge(mitr, dm, meta.GetSchema(), genTimestamp())
|
|
||||||
assert.NoError(t, err)
|
assert.NoError(t, err)
|
||||||
assert.Equal(t, int64(1), numOfRow)
|
assert.Equal(t, int64(0), numOfRow)
|
||||||
assert.Equal(t, 1, len(idata))
|
assert.Equal(t, 0, len(inPaths))
|
||||||
assert.NotEmpty(t, segment)
|
assert.Equal(t, 1, len(statsPaths))
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("Merge with meta error", func(t *testing.T) {
|
t.Run("Merge with meta error", func(t *testing.T) {
|
||||||
|
alloc := NewAllocatorFactory(1)
|
||||||
|
mockbIO := &binlogIO{cm, alloc}
|
||||||
Params.CommonCfg.EntityExpirationTTL = 0
|
Params.CommonCfg.EntityExpirationTTL = 0
|
||||||
iData := genInsertDataWithExpiredTS()
|
iData := genInsertDataWithExpiredTS()
|
||||||
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
||||||
|
|
||||||
iblobs, err := getInsertBlobs(100, iData, meta)
|
var allPaths [][]string
|
||||||
require.NoError(t, err)
|
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)
|
for idx := 0; idx < binlogNum; idx++ {
|
||||||
require.NoError(t, err)
|
var ps []string
|
||||||
|
for _, path := range inpath {
|
||||||
mitr := storage.NewMergeIterator([]iterator{iitr})
|
ps = append(ps, path.GetBinlogs()[idx].GetLogPath())
|
||||||
|
}
|
||||||
|
allPaths = append(allPaths, ps)
|
||||||
|
}
|
||||||
|
|
||||||
dm := map[interface{}]Timestamp{
|
dm := map[interface{}]Timestamp{
|
||||||
1: 10000,
|
1: 10000,
|
||||||
}
|
}
|
||||||
|
|
||||||
ct := &compactionTask{
|
ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO}
|
||||||
Replica: replica,
|
_, _, _, _, err = ct.merge(context.Background(), allPaths, 2, 0, &etcdpb.CollectionMeta{
|
||||||
}
|
Schema: &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{
|
||||||
_, _, _, err = ct.merge(mitr, dm, &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{
|
|
||||||
{DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
|
{DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
|
||||||
{Key: "dim", Value: "64"},
|
{Key: "dim", Value: "64"},
|
||||||
}},
|
}},
|
||||||
}}, ct.GetCurrentTime())
|
}},
|
||||||
|
}, dm)
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
})
|
})
|
||||||
|
|
||||||
t.Run("Merge with meta type param error", func(t *testing.T) {
|
t.Run("Merge with meta type param error", func(t *testing.T) {
|
||||||
|
alloc := NewAllocatorFactory(1)
|
||||||
|
mockbIO := &binlogIO{cm, alloc}
|
||||||
Params.CommonCfg.EntityExpirationTTL = 0
|
Params.CommonCfg.EntityExpirationTTL = 0
|
||||||
iData := genInsertDataWithExpiredTS()
|
iData := genInsertDataWithExpiredTS()
|
||||||
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64)
|
||||||
|
|
||||||
iblobs, err := getInsertBlobs(100, iData, meta)
|
var allPaths [][]string
|
||||||
require.NoError(t, err)
|
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)
|
for idx := 0; idx < binlogNum; idx++ {
|
||||||
require.NoError(t, err)
|
var ps []string
|
||||||
|
for _, path := range inpath {
|
||||||
mitr := storage.NewMergeIterator([]iterator{iitr})
|
ps = append(ps, path.GetBinlogs()[idx].GetLogPath())
|
||||||
|
}
|
||||||
|
allPaths = append(allPaths, ps)
|
||||||
|
}
|
||||||
|
|
||||||
dm := map[interface{}]Timestamp{
|
dm := map[interface{}]Timestamp{
|
||||||
1: 10000,
|
1: 10000,
|
||||||
}
|
}
|
||||||
|
|
||||||
ct := &compactionTask{
|
ct := &compactionTask{Replica: replica, downloader: mockbIO, uploader: mockbIO}
|
||||||
Replica: replica,
|
|
||||||
}
|
_, _, _, _, err = ct.merge(context.Background(), allPaths, 2, 0, &etcdpb.CollectionMeta{
|
||||||
_, _, _, err = ct.merge(mitr, dm, &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{
|
Schema: &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{
|
||||||
{DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
|
{DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{
|
||||||
{Key: "dim", Value: "dim"},
|
{Key: "dim", Value: "dim"},
|
||||||
}},
|
}},
|
||||||
}}, ct.GetCurrentTime())
|
}}}, dm)
|
||||||
assert.Error(t, err)
|
assert.Error(t, err)
|
||||||
})
|
})
|
||||||
})
|
})
|
||||||
|
|
|
@ -1048,6 +1048,7 @@ type dataCoordConfig struct {
|
||||||
|
|
||||||
// --- SEGMENTS ---
|
// --- SEGMENTS ---
|
||||||
SegmentMaxSize float64
|
SegmentMaxSize float64
|
||||||
|
DiskSegmentMaxSize float64
|
||||||
SegmentSealProportion float64
|
SegmentSealProportion float64
|
||||||
SegAssignmentExpiration int64
|
SegAssignmentExpiration int64
|
||||||
SegmentMaxLifetime time.Duration
|
SegmentMaxLifetime time.Duration
|
||||||
|
@ -1084,6 +1085,7 @@ func (p *dataCoordConfig) init(base *BaseTable) {
|
||||||
p.initChannelWatchPrefix()
|
p.initChannelWatchPrefix()
|
||||||
|
|
||||||
p.initSegmentMaxSize()
|
p.initSegmentMaxSize()
|
||||||
|
p.initDiskSegmentMaxSize()
|
||||||
p.initSegmentSealProportion()
|
p.initSegmentSealProportion()
|
||||||
p.initSegAssignmentExpiration()
|
p.initSegAssignmentExpiration()
|
||||||
p.initSegmentMaxLifetime()
|
p.initSegmentMaxLifetime()
|
||||||
|
@ -1114,6 +1116,10 @@ func (p *dataCoordConfig) initSegmentMaxSize() {
|
||||||
p.SegmentMaxSize = p.Base.ParseFloatWithDefault("dataCoord.segment.maxSize", 512.0)
|
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() {
|
func (p *dataCoordConfig) initSegmentSealProportion() {
|
||||||
p.SegmentSealProportion = p.Base.ParseFloatWithDefault("dataCoord.segment.sealProportion", 0.25)
|
p.SegmentSealProportion = p.Base.ParseFloatWithDefault("dataCoord.segment.sealProportion", 0.25)
|
||||||
}
|
}
|
||||||
|
|
|
@ -39,7 +39,6 @@ def assert_fail(func, milvus_client, **params):
|
||||||
try:
|
try:
|
||||||
methodcaller(func, **params)(milvus_client)
|
methodcaller(func, **params)(milvus_client)
|
||||||
except Exception as e:
|
except Exception as e:
|
||||||
logger.debug("11111111111111111111111111")
|
|
||||||
logger.info(str(e))
|
logger.info(str(e))
|
||||||
pass
|
pass
|
||||||
else:
|
else:
|
||||||
|
|
Loading…
Reference in New Issue