mirror of https://github.com/milvus-io/milvus.git
enhance: add ut for clustering_compactor (#34852)
issue: #34792 Signed-off-by: wayblink <anyang.wang@zilliz.com>pull/34865/head
parent
c2b8b5fe84
commit
d294fdd060
|
@ -132,14 +132,10 @@ func NewClusteringCompactionTask(
|
|||
plan *datapb.CompactionPlan,
|
||||
) *clusteringCompactionTask {
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
logIDAlloc := allocator.NewLocalAllocator(plan.GetBeginLogID(), math.MaxInt64)
|
||||
segIDAlloc := allocator.NewLocalAllocator(plan.GetPreAllocatedSegments().GetBegin(), plan.GetPreAllocatedSegments().GetEnd())
|
||||
return &clusteringCompactionTask{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
binlogIO: binlogIO,
|
||||
logIDAlloc: logIDAlloc,
|
||||
segIDAlloc: segIDAlloc,
|
||||
plan: plan,
|
||||
tr: timerecord.NewTimeRecorder("clustering_compaction"),
|
||||
done: make(chan struct{}, 1),
|
||||
|
@ -179,12 +175,23 @@ func (t *clusteringCompactionTask) GetCollection() int64 {
|
|||
}
|
||||
|
||||
func (t *clusteringCompactionTask) init() error {
|
||||
if t.plan.GetType() != datapb.CompactionType_ClusteringCompaction {
|
||||
return merr.WrapErrIllegalCompactionPlan("illegal compaction type")
|
||||
}
|
||||
if len(t.plan.GetSegmentBinlogs()) == 0 {
|
||||
return merr.WrapErrIllegalCompactionPlan("empty segment binlogs")
|
||||
}
|
||||
t.collectionID = t.GetCollection()
|
||||
t.partitionID = t.plan.GetSegmentBinlogs()[0].GetPartitionID()
|
||||
|
||||
logIDAlloc := allocator.NewLocalAllocator(t.plan.GetBeginLogID(), math.MaxInt64)
|
||||
segIDAlloc := allocator.NewLocalAllocator(t.plan.GetPreAllocatedSegments().GetBegin(), t.plan.GetPreAllocatedSegments().GetEnd())
|
||||
t.logIDAlloc = logIDAlloc
|
||||
t.segIDAlloc = segIDAlloc
|
||||
|
||||
var pkField *schemapb.FieldSchema
|
||||
if t.plan.Schema == nil {
|
||||
return errors.New("empty schema in compactionPlan")
|
||||
return merr.WrapErrIllegalCompactionPlan("empty schema in compactionPlan")
|
||||
}
|
||||
for _, field := range t.plan.Schema.Fields {
|
||||
if field.GetIsPrimaryKey() && field.GetFieldID() >= 100 && typeutil.IsPrimaryFieldType(field.GetDataType()) {
|
||||
|
@ -209,22 +216,19 @@ func (t *clusteringCompactionTask) Compact() (*datapb.CompactionPlanResult, erro
|
|||
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, fmt.Sprintf("clusteringCompaction-%d", t.GetPlanID()))
|
||||
defer span.End()
|
||||
log := log.With(zap.Int64("planID", t.plan.GetPlanID()), zap.String("type", t.plan.GetType().String()))
|
||||
if t.plan.GetType() != datapb.CompactionType_ClusteringCompaction {
|
||||
// this shouldn't be reached
|
||||
log.Warn("compact wrong, illegal compaction type")
|
||||
return nil, merr.WrapErrIllegalCompactionPlan()
|
||||
// 0, verify and init
|
||||
err := t.init()
|
||||
if err != nil {
|
||||
log.Error("compaction task init failed", zap.Error(err))
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if !funcutil.CheckCtxValid(ctx) {
|
||||
log.Warn("compact wrong, task context done or timeout")
|
||||
return nil, ctx.Err()
|
||||
}
|
||||
ctxTimeout, cancelAll := context.WithTimeout(ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second)
|
||||
defer cancelAll()
|
||||
|
||||
err := t.init()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
defer t.cleanUp(ctx)
|
||||
|
||||
// 1, download delta logs to build deltaMap
|
||||
|
@ -1031,7 +1035,7 @@ func (t *clusteringCompactionTask) scalarAnalyzeSegment(
|
|||
// Unable to deal with all empty segments cases, so return error
|
||||
if binlogNum == 0 {
|
||||
log.Warn("compact wrong, all segments' binlogs are empty")
|
||||
return nil, merr.WrapErrIllegalCompactionPlan()
|
||||
return nil, merr.WrapErrIllegalCompactionPlan("all segments' binlogs are empty")
|
||||
}
|
||||
log.Debug("binlogNum", zap.Int("binlogNum", binlogNum))
|
||||
for idx := 0; idx < binlogNum; idx++ {
|
||||
|
|
|
@ -19,17 +19,26 @@ package compaction
|
|||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/cockroachdb/errors"
|
||||
"github.com/samber/lo"
|
||||
"github.com/stretchr/testify/mock"
|
||||
"github.com/stretchr/testify/suite"
|
||||
"go.uber.org/atomic"
|
||||
"go.uber.org/zap"
|
||||
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
||||
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
|
||||
"github.com/milvus-io/milvus-storage/go/common/log"
|
||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||
"github.com/milvus-io/milvus/internal/proto/datapb"
|
||||
"github.com/milvus-io/milvus/internal/storage"
|
||||
"github.com/milvus-io/milvus/pkg/common"
|
||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||
"github.com/milvus-io/milvus/pkg/util/tsoutil"
|
||||
)
|
||||
|
||||
func TestClusteringCompactionTaskSuite(t *testing.T) {
|
||||
|
@ -40,6 +49,9 @@ type ClusteringCompactionTaskSuite struct {
|
|||
suite.Suite
|
||||
|
||||
mockBinlogIO *io.MockBinlogIO
|
||||
mockAlloc *allocator.MockAllocator
|
||||
mockID atomic.Int64
|
||||
segWriter *SegmentWriter
|
||||
|
||||
task *clusteringCompactionTask
|
||||
|
||||
|
@ -53,6 +65,21 @@ func (s *ClusteringCompactionTaskSuite) SetupSuite() {
|
|||
func (s *ClusteringCompactionTaskSuite) SetupTest() {
|
||||
s.mockBinlogIO = io.NewMockBinlogIO(s.T())
|
||||
|
||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Maybe()
|
||||
|
||||
s.mockAlloc = allocator.NewMockAllocator(s.T())
|
||||
s.mockID.Store(time.Now().UnixMilli())
|
||||
s.mockAlloc.EXPECT().Alloc(mock.Anything).RunAndReturn(func(x uint32) (int64, int64, error) {
|
||||
start := s.mockID.Load()
|
||||
end := s.mockID.Add(int64(x))
|
||||
log.Info("wayblink", zap.Int64("start", start), zap.Int64("end", end))
|
||||
return start, end, nil
|
||||
}).Maybe()
|
||||
s.mockAlloc.EXPECT().AllocOne().RunAndReturn(func() (int64, error) {
|
||||
end := s.mockID.Add(1)
|
||||
return end, nil
|
||||
}).Maybe()
|
||||
|
||||
s.task = NewClusteringCompactionTask(context.Background(), s.mockBinlogIO, nil)
|
||||
|
||||
paramtable.Get().Save(paramtable.Get().CommonCfg.EntityExpirationTTL.Key, "0")
|
||||
|
@ -98,19 +125,103 @@ func (s *ClusteringCompactionTaskSuite) TestContextDown() {
|
|||
|
||||
func (s *ClusteringCompactionTaskSuite) TestIsVectorClusteringKey() {
|
||||
s.task.plan.Schema = genCollectionSchema()
|
||||
s.task.plan.ClusteringKeyField = Int32Field
|
||||
s.task.plan.ClusteringKeyField = 100
|
||||
s.task.init()
|
||||
s.Equal(false, s.task.isVectorClusteringKey)
|
||||
s.task.plan.ClusteringKeyField = FloatVectorField
|
||||
s.task.plan.ClusteringKeyField = 103
|
||||
s.task.init()
|
||||
s.Equal(true, s.task.isVectorClusteringKey)
|
||||
}
|
||||
|
||||
func (s *ClusteringCompactionTaskSuite) TestGetScalarResult() {
|
||||
func (s *ClusteringCompactionTaskSuite) TestCompactionWithEmptyBinlog() {
|
||||
s.task.plan.Schema = genCollectionSchema()
|
||||
s.task.plan.ClusteringKeyField = Int32Field
|
||||
s.task.plan.ClusteringKeyField = 100
|
||||
_, err := s.task.Compact()
|
||||
s.Require().Error(err)
|
||||
s.Equal(true, errors.Is(err, merr.ErrIllegalCompactionPlan))
|
||||
s.task.plan.SegmentBinlogs = []*datapb.CompactionSegmentBinlogs{}
|
||||
_, err2 := s.task.Compact()
|
||||
s.Require().Error(err2)
|
||||
s.Equal(true, errors.Is(err2, merr.ErrIllegalCompactionPlan))
|
||||
}
|
||||
|
||||
func (s *ClusteringCompactionTaskSuite) TestCompactionWithEmptySchema() {
|
||||
s.task.plan.ClusteringKeyField = 100
|
||||
_, err := s.task.Compact()
|
||||
s.Require().Error(err)
|
||||
s.Equal(true, errors.Is(err, merr.ErrIllegalCompactionPlan))
|
||||
}
|
||||
|
||||
func (s *ClusteringCompactionTaskSuite) TestCompactionInit() {
|
||||
s.task.plan.Schema = genCollectionSchema()
|
||||
s.task.plan.ClusteringKeyField = 100
|
||||
s.task.plan.SegmentBinlogs = []*datapb.CompactionSegmentBinlogs{
|
||||
{
|
||||
SegmentID: 100,
|
||||
},
|
||||
}
|
||||
err := s.task.init()
|
||||
s.Require().NoError(err)
|
||||
s.Equal(s.task.primaryKeyField, s.task.plan.Schema.Fields[2])
|
||||
s.Equal(false, s.task.isVectorClusteringKey)
|
||||
s.Equal(true, s.task.memoryBufferSize > 0)
|
||||
s.Equal(8, s.task.getWorkerPoolSize())
|
||||
s.Equal(8, s.task.mappingPool.Cap())
|
||||
s.Equal(8, s.task.flushPool.Cap())
|
||||
}
|
||||
|
||||
func (s *ClusteringCompactionTaskSuite) TestScalarCompactionNormal() {
|
||||
schema := genCollectionSchema()
|
||||
var segmentID int64 = 1001
|
||||
segWriter, err := NewSegmentWriter(schema, 1000, segmentID, PartitionID, CollectionID)
|
||||
s.Require().NoError(err)
|
||||
|
||||
for i := 0; i < 1000; i++ {
|
||||
v := storage.Value{
|
||||
PK: storage.NewInt64PrimaryKey(int64(i)),
|
||||
Timestamp: int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)),
|
||||
Value: genRow(int64(i)),
|
||||
}
|
||||
err = segWriter.Write(&v)
|
||||
s.Require().NoError(err)
|
||||
}
|
||||
segWriter.writer.Flush()
|
||||
|
||||
kvs, fBinlogs, err := serializeWrite(context.TODO(), s.mockAlloc, segWriter)
|
||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return(lo.Values(kvs), nil)
|
||||
|
||||
s.plan.SegmentBinlogs = []*datapb.CompactionSegmentBinlogs{
|
||||
{
|
||||
SegmentID: 100,
|
||||
FieldBinlogs: lo.Values(fBinlogs),
|
||||
},
|
||||
}
|
||||
|
||||
s.task.plan.Schema = genCollectionSchema()
|
||||
s.task.plan.ClusteringKeyField = 100
|
||||
s.task.plan.PreferSegmentRows = 100
|
||||
s.task.plan.MaxSegmentRows = 200
|
||||
s.task.plan.PreAllocatedSegments = &datapb.IDRange{
|
||||
Begin: time.Now().UnixMilli(),
|
||||
End: time.Now().UnixMilli() + 1000,
|
||||
}
|
||||
|
||||
compactionResult, err := s.task.Compact()
|
||||
s.Require().NoError(err)
|
||||
s.Equal(10, len(s.task.clusterBuffers))
|
||||
s.Equal(10, len(compactionResult.GetSegments()))
|
||||
}
|
||||
|
||||
func genRow(magic int64) map[int64]interface{} {
|
||||
ts := tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)
|
||||
return map[int64]interface{}{
|
||||
common.RowIDField: magic,
|
||||
common.TimeStampField: int64(ts),
|
||||
100: magic,
|
||||
101: int32(magic),
|
||||
102: "varchar",
|
||||
103: []float32{4, 5, 6, 7},
|
||||
}
|
||||
}
|
||||
|
||||
func genCollectionSchema() *schemapb.CollectionSchema {
|
||||
|
@ -124,12 +235,23 @@ func genCollectionSchema() *schemapb.CollectionSchema {
|
|||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: Int32Field,
|
||||
FieldID: common.TimeStampField,
|
||||
Name: "Timestamp",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
},
|
||||
{
|
||||
FieldID: 100,
|
||||
Name: "pk",
|
||||
DataType: schemapb.DataType_Int64,
|
||||
IsPrimaryKey: true,
|
||||
},
|
||||
{
|
||||
FieldID: 101,
|
||||
Name: "field_int32",
|
||||
DataType: schemapb.DataType_Int32,
|
||||
},
|
||||
{
|
||||
FieldID: VarCharField,
|
||||
FieldID: 102,
|
||||
Name: "field_varchar",
|
||||
DataType: schemapb.DataType_VarChar,
|
||||
TypeParams: []*commonpb.KeyValuePair{
|
||||
|
@ -140,7 +262,7 @@ func genCollectionSchema() *schemapb.CollectionSchema {
|
|||
},
|
||||
},
|
||||
{
|
||||
FieldID: FloatVectorField,
|
||||
FieldID: 103,
|
||||
Name: "field_float_vector",
|
||||
Description: "float_vector",
|
||||
DataType: schemapb.DataType_FloatVector,
|
||||
|
|
Loading…
Reference in New Issue