mirror of https://github.com/milvus-io/milvus.git
enhance: Speed up L0 compaction (#30410)
This PR changes the following to speed up L0 compaction and prevent OOM: 1. Lower deltabuf limit to 16MB by default, so that each L0 segment would be 4X smaller than before. 2. Add BatchProcess, use it if memory is sufficient 3. Iterator will Deserialize when called HasNext to avoid massive memory peek 4. Add tracing in spiltDelta See also: #30191 --------- Signed-off-by: yangxuan <xuan.yang@zilliz.com>pull/29788/head
parent
6959630652
commit
e6eb6f2c78
|
@ -22,8 +22,10 @@ import (
|
||||||
|
|
||||||
"github.com/samber/lo"
|
"github.com/samber/lo"
|
||||||
"go.opentelemetry.io/otel"
|
"go.opentelemetry.io/otel"
|
||||||
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
"github.com/milvus-io/milvus/pkg/util/conc"
|
"github.com/milvus-io/milvus/pkg/util/conc"
|
||||||
"github.com/milvus-io/milvus/pkg/util/retry"
|
"github.com/milvus-io/milvus/pkg/util/retry"
|
||||||
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
"github.com/milvus-io/milvus/pkg/util/typeutil"
|
||||||
|
@ -56,8 +58,10 @@ func (b *BinlogIoImpl) Download(ctx context.Context, paths []string) ([][]byte,
|
||||||
var val []byte
|
var val []byte
|
||||||
var err error
|
var err error
|
||||||
|
|
||||||
|
log.Debug("BinlogIO download", zap.String("path", path))
|
||||||
err = retry.Do(ctx, func() error {
|
err = retry.Do(ctx, func() error {
|
||||||
val, err = b.Read(ctx, path)
|
val, err = b.Read(ctx, path)
|
||||||
|
log.Warn("BinlogIO fail to download", zap.String("path", path), zap.Error(err))
|
||||||
return err
|
return err
|
||||||
})
|
})
|
||||||
|
|
||||||
|
@ -80,6 +84,7 @@ func (b *BinlogIoImpl) Upload(ctx context.Context, kvs map[string][]byte) error
|
||||||
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "Upload")
|
ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "Upload")
|
||||||
defer span.End()
|
defer span.End()
|
||||||
future := b.pool.Submit(func() (any, error) {
|
future := b.pool.Submit(func() (any, error) {
|
||||||
|
log.Debug("BinlogIO uplaod", zap.Strings("paths", lo.Keys(kvs)))
|
||||||
err := retry.Do(ctx, func() error {
|
err := retry.Do(ctx, func() error {
|
||||||
return b.MultiWrite(ctx, kvs)
|
return b.MultiWrite(ctx, kvs)
|
||||||
})
|
})
|
||||||
|
|
|
@ -4,8 +4,10 @@ import (
|
||||||
"sync"
|
"sync"
|
||||||
|
|
||||||
"go.uber.org/atomic"
|
"go.uber.org/atomic"
|
||||||
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus/internal/storage"
|
"github.com/milvus-io/milvus/internal/storage"
|
||||||
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
)
|
)
|
||||||
|
|
||||||
var _ Iterator = (*DeltalogIterator)(nil)
|
var _ Iterator = (*DeltalogIterator)(nil)
|
||||||
|
@ -16,26 +18,22 @@ type DeltalogIterator struct {
|
||||||
disposed atomic.Bool
|
disposed atomic.Bool
|
||||||
|
|
||||||
data *storage.DeleteData
|
data *storage.DeleteData
|
||||||
|
blobs []*storage.Blob
|
||||||
label *Label
|
label *Label
|
||||||
pos int
|
pos int
|
||||||
}
|
}
|
||||||
|
|
||||||
func NewDeltalogIterator(v [][]byte, label *Label) (*DeltalogIterator, error) {
|
func NewDeltalogIterator(v [][]byte, label *Label) *DeltalogIterator {
|
||||||
blobs := make([]*storage.Blob, len(v))
|
blobs := make([]*storage.Blob, len(v))
|
||||||
for i := range blobs {
|
for i := range blobs {
|
||||||
blobs[i] = &storage.Blob{Value: v[i]}
|
blobs[i] = &storage.Blob{Value: v[i]}
|
||||||
}
|
}
|
||||||
|
|
||||||
reader := storage.NewDeleteCodec()
|
|
||||||
_, _, dData, err := reader.Deserialize(blobs)
|
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
return &DeltalogIterator{
|
return &DeltalogIterator{
|
||||||
disposeCh: make(chan struct{}),
|
disposeCh: make(chan struct{}),
|
||||||
data: dData,
|
blobs: blobs,
|
||||||
label: label,
|
label: label,
|
||||||
}, nil
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func (d *DeltalogIterator) HasNext() bool {
|
func (d *DeltalogIterator) HasNext() bool {
|
||||||
|
@ -68,6 +66,16 @@ func (d *DeltalogIterator) Dispose() {
|
||||||
}
|
}
|
||||||
|
|
||||||
func (d *DeltalogIterator) hasNext() bool {
|
func (d *DeltalogIterator) hasNext() bool {
|
||||||
|
if d.data == nil && d.blobs != nil {
|
||||||
|
reader := storage.NewDeleteCodec()
|
||||||
|
_, _, dData, err := reader.Deserialize(d.blobs)
|
||||||
|
if err != nil {
|
||||||
|
log.Warn("Deltalog iterator failed to deserialize blobs", zap.Error(err))
|
||||||
|
return false
|
||||||
|
}
|
||||||
|
d.data = dData
|
||||||
|
d.blobs = nil
|
||||||
|
}
|
||||||
return int64(d.pos) < d.data.RowCount
|
return int64(d.pos) < d.data.RowCount
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -18,9 +18,10 @@ type DeltalogIteratorSuite struct {
|
||||||
|
|
||||||
func (s *DeltalogIteratorSuite) TestDeltalogIteratorIntPK() {
|
func (s *DeltalogIteratorSuite) TestDeltalogIteratorIntPK() {
|
||||||
s.Run("invalid blobs", func() {
|
s.Run("invalid blobs", func() {
|
||||||
iter, err := NewDeltalogIterator([][]byte{}, nil)
|
iter := NewDeltalogIterator([][]byte{}, nil)
|
||||||
s.Error(err)
|
|
||||||
s.Nil(iter)
|
s.NotNil(iter)
|
||||||
|
s.False(iter.HasNext())
|
||||||
})
|
})
|
||||||
|
|
||||||
testpks := []int64{1, 2, 3, 4}
|
testpks := []int64{1, 2, 3, 4}
|
||||||
|
@ -36,8 +37,8 @@ func (s *DeltalogIteratorSuite) TestDeltalogIteratorIntPK() {
|
||||||
s.Require().NoError(err)
|
s.Require().NoError(err)
|
||||||
value := [][]byte{blob.Value[:]}
|
value := [][]byte{blob.Value[:]}
|
||||||
|
|
||||||
iter, err := NewDeltalogIterator(value, &Label{segmentID: 100})
|
iter := NewDeltalogIterator(value, &Label{segmentID: 100})
|
||||||
s.NoError(err)
|
s.NotNil(iter)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
gotpks = []int64{}
|
gotpks = []int64{}
|
||||||
|
|
|
@ -38,6 +38,7 @@ import (
|
||||||
"github.com/milvus-io/milvus/pkg/log"
|
"github.com/milvus-io/milvus/pkg/log"
|
||||||
"github.com/milvus-io/milvus/pkg/metrics"
|
"github.com/milvus-io/milvus/pkg/metrics"
|
||||||
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
"github.com/milvus-io/milvus/pkg/util/funcutil"
|
||||||
|
"github.com/milvus-io/milvus/pkg/util/hardware"
|
||||||
"github.com/milvus-io/milvus/pkg/util/merr"
|
"github.com/milvus-io/milvus/pkg/util/merr"
|
||||||
"github.com/milvus-io/milvus/pkg/util/metautil"
|
"github.com/milvus-io/milvus/pkg/util/metautil"
|
||||||
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
"github.com/milvus-io/milvus/pkg/util/paramtable"
|
||||||
|
@ -160,71 +161,13 @@ func (t *levelZeroCompactionTask) compact() (*datapb.CompactionPlanResult, error
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO
|
|
||||||
// batchProcess := func() ([]*datapb.CompactionSegment, error) {
|
|
||||||
// resultSegments := make(map[int64]*datapb.CompactionSegment)
|
|
||||||
//
|
|
||||||
// iters, err := t.loadDelta(ctxTimeout, lo.Values(totalDeltalogs)...)
|
|
||||||
// if err != nil {
|
|
||||||
// return nil, err
|
|
||||||
// }
|
|
||||||
// log.Info("Batch L0 compaction load delta into memeory", zap.Duration("elapse", t.tr.RecordSpan()))
|
|
||||||
//
|
|
||||||
// alteredSegments := make(map[int64]*storage.DeleteData)
|
|
||||||
// err = t.splitDelta(iters, alteredSegments, targetSegIDs)
|
|
||||||
// if err != nil {
|
|
||||||
// return nil, err
|
|
||||||
// }
|
|
||||||
// log.Info("Batch L0 compaction split delta into segments", zap.Duration("elapse", t.tr.RecordSpan()))
|
|
||||||
//
|
|
||||||
// err = t.uploadByCheck(ctxTimeout, false, alteredSegments, resultSegments)
|
|
||||||
// log.Info("Batch L0 compaction upload all", zap.Duration("elapse", t.tr.RecordSpan()))
|
|
||||||
//
|
|
||||||
// return lo.Values(resultSegments), nil
|
|
||||||
// }
|
|
||||||
|
|
||||||
linearProcess := func() ([]*datapb.CompactionSegment, error) {
|
|
||||||
var (
|
|
||||||
resultSegments = make(map[int64]*datapb.CompactionSegment)
|
|
||||||
alteredSegments = make(map[int64]*storage.DeleteData)
|
|
||||||
)
|
|
||||||
for segID, deltaLogs := range totalDeltalogs {
|
|
||||||
log := log.With(zap.Int64("levelzero segment", segID))
|
|
||||||
log.Info("Linear L0 compaction processing segment", zap.Int("target segment count", len(targetSegIDs)))
|
|
||||||
|
|
||||||
allIters, err := t.loadDelta(ctxTimeout, deltaLogs)
|
|
||||||
if err != nil {
|
|
||||||
log.Warn("Linear L0 compaction loadDelta fail", zap.Error(err))
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
err = t.splitDelta(allIters, alteredSegments, targetSegIDs)
|
|
||||||
if err != nil {
|
|
||||||
log.Warn("Linear L0 compaction splitDelta fail", zap.Error(err))
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
err = t.uploadByCheck(ctxTimeout, true, alteredSegments, resultSegments)
|
|
||||||
if err != nil {
|
|
||||||
log.Warn("Linear L0 compaction upload buffer fail", zap.Error(err))
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
err := t.uploadByCheck(ctxTimeout, false, alteredSegments, resultSegments)
|
|
||||||
if err != nil {
|
|
||||||
log.Warn("Linear L0 compaction upload all buffer fail", zap.Error(err))
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
log.Info("Linear L0 compaction finished", zap.Duration("elapse", t.tr.RecordSpan()))
|
|
||||||
return lo.Values(resultSegments), nil
|
|
||||||
}
|
|
||||||
|
|
||||||
var resultSegments []*datapb.CompactionSegment
|
var resultSegments []*datapb.CompactionSegment
|
||||||
// if totalSize*3 < int64(hardware.GetFreeMemoryCount()) {
|
// TODO
|
||||||
// resultSegments, err = batchProcess()
|
if hardware.GetFreeMemoryCount() < uint64(totalSize)*20 {
|
||||||
// }
|
resultSegments, err = t.linearProcess(ctxTimeout, targetSegIDs, totalDeltalogs)
|
||||||
resultSegments, err = linearProcess()
|
} else {
|
||||||
|
resultSegments, err = t.batchProcess(ctxTimeout, targetSegIDs, lo.Values(totalDeltalogs)...)
|
||||||
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
@ -244,29 +187,94 @@ func (t *levelZeroCompactionTask) compact() (*datapb.CompactionPlanResult, error
|
||||||
return result, nil
|
return result, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (t *levelZeroCompactionTask) linearProcess(ctx context.Context, targetSegments []int64, totalDeltalogs map[int64][]string) ([]*datapb.CompactionSegment, error) {
|
||||||
|
log := log.Ctx(t.ctx).With(
|
||||||
|
zap.Int64("planID", t.plan.GetPlanID()),
|
||||||
|
zap.String("type", t.plan.GetType().String()),
|
||||||
|
zap.Int("target segment counts", len(targetSegments)),
|
||||||
|
)
|
||||||
|
var (
|
||||||
|
resultSegments = make(map[int64]*datapb.CompactionSegment)
|
||||||
|
alteredSegments = make(map[int64]*storage.DeleteData)
|
||||||
|
)
|
||||||
|
for segID, deltaLogs := range totalDeltalogs {
|
||||||
|
log := log.With(zap.Int64("levelzero segment", segID))
|
||||||
|
|
||||||
|
log.Info("Linear L0 compaction start processing segment")
|
||||||
|
allIters, err := t.loadDelta(ctx, deltaLogs)
|
||||||
|
if err != nil {
|
||||||
|
log.Warn("Linear L0 compaction loadDelta fail", zap.Int64s("target segments", targetSegments), zap.Error(err))
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
t.splitDelta(ctx, allIters, alteredSegments, targetSegments)
|
||||||
|
|
||||||
|
err = t.uploadByCheck(ctx, true, alteredSegments, resultSegments)
|
||||||
|
if err != nil {
|
||||||
|
log.Warn("Linear L0 compaction upload buffer fail", zap.Int64s("target segments", targetSegments), zap.Error(err))
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
err := t.uploadByCheck(ctx, false, alteredSegments, resultSegments)
|
||||||
|
if err != nil {
|
||||||
|
log.Warn("Linear L0 compaction upload all buffer fail", zap.Int64s("target segment", targetSegments), zap.Error(err))
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
log.Info("Linear L0 compaction finished", zap.Duration("elapse", t.tr.RecordSpan()))
|
||||||
|
return lo.Values(resultSegments), nil
|
||||||
|
}
|
||||||
|
|
||||||
|
func (t *levelZeroCompactionTask) batchProcess(ctx context.Context, targetSegments []int64, deltaLogs ...[]string) ([]*datapb.CompactionSegment, error) {
|
||||||
|
log := log.Ctx(t.ctx).With(
|
||||||
|
zap.Int64("planID", t.plan.GetPlanID()),
|
||||||
|
zap.String("type", t.plan.GetType().String()),
|
||||||
|
zap.Int("target segment counts", len(targetSegments)),
|
||||||
|
)
|
||||||
|
log.Info("Batch L0 compaction start processing")
|
||||||
|
resultSegments := make(map[int64]*datapb.CompactionSegment)
|
||||||
|
|
||||||
|
iters, err := t.loadDelta(ctx, lo.Flatten(deltaLogs))
|
||||||
|
if err != nil {
|
||||||
|
log.Warn("Batch L0 compaction loadDelta fail", zap.Int64s("target segments", targetSegments), zap.Error(err))
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
|
||||||
|
alteredSegments := make(map[int64]*storage.DeleteData)
|
||||||
|
t.splitDelta(ctx, iters, alteredSegments, targetSegments)
|
||||||
|
|
||||||
|
err = t.uploadByCheck(ctx, false, alteredSegments, resultSegments)
|
||||||
|
if err != nil {
|
||||||
|
log.Warn("Batch L0 compaction upload fail", zap.Int64s("target segments", targetSegments), zap.Error(err))
|
||||||
|
return nil, err
|
||||||
|
}
|
||||||
|
log.Info("Batch L0 compaction finished", zap.Duration("elapse", t.tr.RecordSpan()))
|
||||||
|
return lo.Values(resultSegments), nil
|
||||||
|
}
|
||||||
|
|
||||||
func (t *levelZeroCompactionTask) loadDelta(ctx context.Context, deltaLogs ...[]string) ([]*iter.DeltalogIterator, error) {
|
func (t *levelZeroCompactionTask) loadDelta(ctx context.Context, deltaLogs ...[]string) ([]*iter.DeltalogIterator, error) {
|
||||||
allIters := make([]*iter.DeltalogIterator, 0)
|
allIters := make([]*iter.DeltalogIterator, 0)
|
||||||
|
|
||||||
for _, paths := range deltaLogs {
|
for _, paths := range deltaLogs {
|
||||||
blobs, err := t.Download(ctx, paths)
|
blobs, err := t.Download(ctx, paths)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return nil, err
|
return nil, err
|
||||||
}
|
}
|
||||||
|
|
||||||
deltaIter, err := iter.NewDeltalogIterator(blobs, nil)
|
allIters = append(allIters, iter.NewDeltalogIterator(blobs, nil))
|
||||||
if err != nil {
|
|
||||||
return nil, err
|
|
||||||
}
|
|
||||||
|
|
||||||
allIters = append(allIters, deltaIter)
|
|
||||||
}
|
}
|
||||||
return allIters, nil
|
return allIters, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (t *levelZeroCompactionTask) splitDelta(
|
func (t *levelZeroCompactionTask) splitDelta(
|
||||||
|
ctx context.Context,
|
||||||
allIters []*iter.DeltalogIterator,
|
allIters []*iter.DeltalogIterator,
|
||||||
targetSegBuffer map[int64]*storage.DeleteData,
|
targetSegBuffer map[int64]*storage.DeleteData,
|
||||||
targetSegIDs []int64,
|
targetSegIDs []int64,
|
||||||
) error {
|
) {
|
||||||
|
_, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "L0Compact splitDelta")
|
||||||
|
defer span.End()
|
||||||
|
|
||||||
// segments shall be safe to read outside
|
// segments shall be safe to read outside
|
||||||
segments := t.metacache.GetSegmentsBy(metacache.WithSegmentIDs(targetSegIDs...))
|
segments := t.metacache.GetSegmentsBy(metacache.WithSegmentIDs(targetSegIDs...))
|
||||||
split := func(pk storage.PrimaryKey) []int64 {
|
split := func(pk storage.PrimaryKey) []int64 {
|
||||||
|
@ -278,10 +286,8 @@ func (t *levelZeroCompactionTask) splitDelta(
|
||||||
// spilt all delete data to segments
|
// spilt all delete data to segments
|
||||||
for _, deltaIter := range allIters {
|
for _, deltaIter := range allIters {
|
||||||
for deltaIter.HasNext() {
|
for deltaIter.HasNext() {
|
||||||
labeled, err := deltaIter.Next()
|
// checked by HasNext, no error here
|
||||||
if err != nil {
|
labeled, _ := deltaIter.Next()
|
||||||
return err
|
|
||||||
}
|
|
||||||
|
|
||||||
predicted := split(labeled.GetPk())
|
predicted := split(labeled.GetPk())
|
||||||
|
|
||||||
|
@ -296,7 +302,6 @@ func (t *levelZeroCompactionTask) splitDelta(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return nil
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (t *levelZeroCompactionTask) composeDeltalog(segmentID int64, dData *storage.DeleteData) (map[string][]byte, *datapb.Binlog, error) {
|
func (t *levelZeroCompactionTask) composeDeltalog(segmentID int64, dData *storage.DeleteData) (map[string][]byte, *datapb.Binlog, error) {
|
||||||
|
|
|
@ -27,7 +27,6 @@ import (
|
||||||
"github.com/stretchr/testify/suite"
|
"github.com/stretchr/testify/suite"
|
||||||
"go.uber.org/zap"
|
"go.uber.org/zap"
|
||||||
|
|
||||||
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
|
|
||||||
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
"github.com/milvus-io/milvus/internal/datanode/allocator"
|
||||||
"github.com/milvus-io/milvus/internal/datanode/io"
|
"github.com/milvus-io/milvus/internal/datanode/io"
|
||||||
iter "github.com/milvus-io/milvus/internal/datanode/iterators"
|
iter "github.com/milvus-io/milvus/internal/datanode/iterators"
|
||||||
|
@ -81,6 +80,89 @@ func (s *LevelZeroCompactionTaskSuite) SetupTest() {
|
||||||
s.dBlob = blob.GetValue()
|
s.dBlob = blob.GetValue()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
func (s *LevelZeroCompactionTaskSuite) TestLinearBatchLoadDeltaFail() {
|
||||||
|
plan := &datapb.CompactionPlan{
|
||||||
|
PlanID: 19530,
|
||||||
|
Type: datapb.CompactionType_Level0DeleteCompaction,
|
||||||
|
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
|
||||||
|
{
|
||||||
|
SegmentID: 100, Level: datapb.SegmentLevel_L0, Deltalogs: []*datapb.FieldBinlog{
|
||||||
|
{
|
||||||
|
Binlogs: []*datapb.Binlog{
|
||||||
|
{LogPath: "a/b/c1", LogSize: 100},
|
||||||
|
{LogPath: "a/b/c2", LogSize: 100},
|
||||||
|
{LogPath: "a/b/c3", LogSize: 100},
|
||||||
|
{LogPath: "a/b/c4", LogSize: 100},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{SegmentID: 200, Level: datapb.SegmentLevel_L1},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
s.task.plan = plan
|
||||||
|
s.task.tr = timerecord.NewTimeRecorder("test")
|
||||||
|
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return(nil, errors.New("mock download fail")).Twice()
|
||||||
|
|
||||||
|
targetSegments := []int64{200}
|
||||||
|
deltaLogs := map[int64][]string{100: {"a/b/c1"}}
|
||||||
|
|
||||||
|
segments, err := s.task.linearProcess(context.Background(), targetSegments, deltaLogs)
|
||||||
|
s.Error(err)
|
||||||
|
s.Empty(segments)
|
||||||
|
|
||||||
|
segments, err = s.task.batchProcess(context.Background(), targetSegments, lo.Values(deltaLogs)...)
|
||||||
|
s.Error(err)
|
||||||
|
s.Empty(segments)
|
||||||
|
}
|
||||||
|
|
||||||
|
func (s *LevelZeroCompactionTaskSuite) TestLinearBatchUploadByCheckFail() {
|
||||||
|
plan := &datapb.CompactionPlan{
|
||||||
|
PlanID: 19530,
|
||||||
|
Type: datapb.CompactionType_Level0DeleteCompaction,
|
||||||
|
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
|
||||||
|
{
|
||||||
|
SegmentID: 100, Level: datapb.SegmentLevel_L0, Deltalogs: []*datapb.FieldBinlog{
|
||||||
|
{
|
||||||
|
Binlogs: []*datapb.Binlog{
|
||||||
|
{LogPath: "a/b/c1", LogSize: 100},
|
||||||
|
{LogPath: "a/b/c2", LogSize: 100},
|
||||||
|
{LogPath: "a/b/c3", LogSize: 100},
|
||||||
|
{LogPath: "a/b/c4", LogSize: 100},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
},
|
||||||
|
{SegmentID: 200, Level: datapb.SegmentLevel_L1},
|
||||||
|
},
|
||||||
|
}
|
||||||
|
|
||||||
|
s.task.plan = plan
|
||||||
|
s.task.tr = timerecord.NewTimeRecorder("test")
|
||||||
|
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return([][]byte{s.dBlob}, nil).Times(2)
|
||||||
|
s.mockMeta.EXPECT().Collection().Return(1)
|
||||||
|
s.mockMeta.EXPECT().GetSegmentByID(mock.Anything).Return(nil, false).Twice()
|
||||||
|
s.mockMeta.EXPECT().GetSegmentsBy(mock.Anything).RunAndReturn(
|
||||||
|
func(filters ...metacache.SegmentFilter) []*metacache.SegmentInfo {
|
||||||
|
bfs1 := metacache.NewBloomFilterSetWithBatchSize(100)
|
||||||
|
bfs1.UpdatePKRange(&storage.Int64FieldData{Data: []int64{1, 2}})
|
||||||
|
segment1 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 200}, bfs1)
|
||||||
|
return []*metacache.SegmentInfo{segment1}
|
||||||
|
}).Twice()
|
||||||
|
|
||||||
|
targetSegments := []int64{200}
|
||||||
|
deltaLogs := map[int64][]string{100: {"a/b/c1"}}
|
||||||
|
|
||||||
|
segments, err := s.task.linearProcess(context.Background(), targetSegments, deltaLogs)
|
||||||
|
s.Error(err)
|
||||||
|
s.Empty(segments)
|
||||||
|
|
||||||
|
segments, err = s.task.batchProcess(context.Background(), targetSegments, lo.Values(deltaLogs)...)
|
||||||
|
s.Error(err)
|
||||||
|
s.Empty(segments)
|
||||||
|
}
|
||||||
|
|
||||||
func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() {
|
func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() {
|
||||||
plan := &datapb.CompactionPlan{
|
plan := &datapb.CompactionPlan{
|
||||||
PlanID: 19530,
|
PlanID: 19530,
|
||||||
|
@ -144,26 +226,42 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() {
|
||||||
s.Require().Equal(plan.GetChannel(), s.task.getChannelName())
|
s.Require().Equal(plan.GetChannel(), s.task.getChannelName())
|
||||||
s.Require().EqualValues(1, s.task.getCollection())
|
s.Require().EqualValues(1, s.task.getCollection())
|
||||||
|
|
||||||
result, err := s.task.compact()
|
l0Segments := lo.Filter(s.task.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) bool {
|
||||||
|
return s.Level == datapb.SegmentLevel_L0
|
||||||
|
})
|
||||||
|
|
||||||
|
targetSegIDs := lo.FilterMap(s.task.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) (int64, bool) {
|
||||||
|
if s.Level == datapb.SegmentLevel_L1 {
|
||||||
|
return s.GetSegmentID(), true
|
||||||
|
}
|
||||||
|
return 0, false
|
||||||
|
})
|
||||||
|
totalDeltalogs := make(map[UniqueID][]string)
|
||||||
|
|
||||||
|
for _, s := range l0Segments {
|
||||||
|
paths := []string{}
|
||||||
|
for _, d := range s.GetDeltalogs() {
|
||||||
|
for _, l := range d.GetBinlogs() {
|
||||||
|
paths = append(paths, l.GetLogPath())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if len(paths) > 0 {
|
||||||
|
totalDeltalogs[s.GetSegmentID()] = paths
|
||||||
|
}
|
||||||
|
}
|
||||||
|
segments, err := s.task.linearProcess(context.Background(), targetSegIDs, totalDeltalogs)
|
||||||
s.NoError(err)
|
s.NoError(err)
|
||||||
s.NotNil(result)
|
s.NotEmpty(segments)
|
||||||
s.Equal(commonpb.CompactionState_Completed, result.GetState())
|
s.Equal(2, len(segments))
|
||||||
s.Equal(plan.GetChannel(), result.GetChannel())
|
|
||||||
s.Equal(2, len(result.GetSegments()))
|
|
||||||
s.ElementsMatch([]int64{200, 201},
|
s.ElementsMatch([]int64{200, 201},
|
||||||
lo.Map(result.GetSegments(), func(seg *datapb.CompactionSegment, _ int) int64 {
|
lo.Map(segments, func(seg *datapb.CompactionSegment, _ int) int64 {
|
||||||
return seg.GetSegmentID()
|
return seg.GetSegmentID()
|
||||||
}))
|
}))
|
||||||
|
|
||||||
s.EqualValues(plan.GetPlanID(), result.GetPlanID())
|
log.Info("test segment results", zap.Any("result", segments))
|
||||||
log.Info("test segment results", zap.Any("result", result))
|
|
||||||
|
|
||||||
s.task.complete()
|
|
||||||
s.task.stop()
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *LevelZeroCompactionTaskSuite) TestCompactBatch() {
|
func (s *LevelZeroCompactionTaskSuite) TestCompactBatch() {
|
||||||
s.T().Skip()
|
|
||||||
plan := &datapb.CompactionPlan{
|
plan := &datapb.CompactionPlan{
|
||||||
PlanID: 19530,
|
PlanID: 19530,
|
||||||
Type: datapb.CompactionType_Level0DeleteCompaction,
|
Type: datapb.CompactionType_Level0DeleteCompaction,
|
||||||
|
@ -200,8 +298,19 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactBatch() {
|
||||||
s.task.plan = plan
|
s.task.plan = plan
|
||||||
s.task.tr = timerecord.NewTimeRecorder("test")
|
s.task.tr = timerecord.NewTimeRecorder("test")
|
||||||
|
|
||||||
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return([][]byte{s.dBlob}, nil).Times(2)
|
s.mockMeta.EXPECT().GetSegmentsBy(mock.Anything).RunAndReturn(
|
||||||
s.mockMeta.EXPECT().PredictSegments(mock.Anything, mock.Anything).Return([]int64{200, 201}, true)
|
func(filters ...metacache.SegmentFilter) []*metacache.SegmentInfo {
|
||||||
|
bfs1 := metacache.NewBloomFilterSetWithBatchSize(100)
|
||||||
|
bfs1.UpdatePKRange(&storage.Int64FieldData{Data: []int64{1, 2, 3}})
|
||||||
|
segment1 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 200}, bfs1)
|
||||||
|
bfs2 := metacache.NewBloomFilterSetWithBatchSize(100)
|
||||||
|
bfs2.UpdatePKRange(&storage.Int64FieldData{Data: []int64{1, 2, 3}})
|
||||||
|
segment2 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ID: 201}, bfs2)
|
||||||
|
|
||||||
|
return []*metacache.SegmentInfo{segment1, segment2}
|
||||||
|
})
|
||||||
|
|
||||||
|
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).Return([][]byte{s.dBlob}, nil).Once()
|
||||||
s.mockMeta.EXPECT().Collection().Return(1)
|
s.mockMeta.EXPECT().Collection().Return(1)
|
||||||
s.mockMeta.EXPECT().GetSegmentByID(mock.Anything, mock.Anything).
|
s.mockMeta.EXPECT().GetSegmentByID(mock.Anything, mock.Anything).
|
||||||
RunAndReturn(func(id int64, filters ...metacache.SegmentFilter) (*metacache.SegmentInfo, bool) {
|
RunAndReturn(func(id int64, filters ...metacache.SegmentFilter) (*metacache.SegmentInfo, bool) {
|
||||||
|
@ -215,19 +324,39 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactBatch() {
|
||||||
}).Times(2)
|
}).Times(2)
|
||||||
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Times(2)
|
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Times(2)
|
||||||
|
|
||||||
result, err := s.task.compact()
|
l0Segments := lo.Filter(s.task.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) bool {
|
||||||
|
return s.Level == datapb.SegmentLevel_L0
|
||||||
|
})
|
||||||
|
|
||||||
|
targetSegIDs := lo.FilterMap(s.task.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) (int64, bool) {
|
||||||
|
if s.Level == datapb.SegmentLevel_L1 {
|
||||||
|
return s.GetSegmentID(), true
|
||||||
|
}
|
||||||
|
return 0, false
|
||||||
|
})
|
||||||
|
totalDeltalogs := make(map[UniqueID][]string)
|
||||||
|
|
||||||
|
for _, s := range l0Segments {
|
||||||
|
paths := []string{}
|
||||||
|
for _, d := range s.GetDeltalogs() {
|
||||||
|
for _, l := range d.GetBinlogs() {
|
||||||
|
paths = append(paths, l.GetLogPath())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if len(paths) > 0 {
|
||||||
|
totalDeltalogs[s.GetSegmentID()] = paths
|
||||||
|
}
|
||||||
|
}
|
||||||
|
segments, err := s.task.batchProcess(context.TODO(), targetSegIDs, lo.Values(totalDeltalogs)...)
|
||||||
s.NoError(err)
|
s.NoError(err)
|
||||||
s.NotNil(result)
|
s.NotEmpty(segments)
|
||||||
s.Equal(commonpb.CompactionState_Completed, result.GetState())
|
s.Equal(2, len(segments))
|
||||||
s.Equal(plan.GetChannel(), result.GetChannel())
|
|
||||||
s.Equal(2, len(result.GetSegments()))
|
|
||||||
s.ElementsMatch([]int64{200, 201},
|
s.ElementsMatch([]int64{200, 201},
|
||||||
lo.Map(result.GetSegments(), func(seg *datapb.CompactionSegment, _ int) int64 {
|
lo.Map(segments, func(seg *datapb.CompactionSegment, _ int) int64 {
|
||||||
return seg.GetSegmentID()
|
return seg.GetSegmentID()
|
||||||
}))
|
}))
|
||||||
|
|
||||||
s.EqualValues(plan.GetPlanID(), result.GetPlanID())
|
log.Info("test segment results", zap.Any("result", segments))
|
||||||
log.Info("test segment results", zap.Any("result", result))
|
|
||||||
}
|
}
|
||||||
|
|
||||||
func (s *LevelZeroCompactionTaskSuite) TestUploadByCheck() {
|
func (s *LevelZeroCompactionTaskSuite) TestUploadByCheck() {
|
||||||
|
@ -388,14 +517,12 @@ func (s *LevelZeroCompactionTaskSuite) TestSplitDelta() {
|
||||||
predicted := []int64{100, 101, 102}
|
predicted := []int64{100, 101, 102}
|
||||||
s.mockMeta.EXPECT().GetSegmentsBy(mock.Anything).Return([]*metacache.SegmentInfo{segment1, segment2, segment3})
|
s.mockMeta.EXPECT().GetSegmentsBy(mock.Anything).Return([]*metacache.SegmentInfo{segment1, segment2, segment3})
|
||||||
|
|
||||||
diter, err := iter.NewDeltalogIterator([][]byte{s.dBlob}, nil)
|
diter := iter.NewDeltalogIterator([][]byte{s.dBlob}, nil)
|
||||||
s.Require().NoError(err)
|
|
||||||
s.Require().NotNil(diter)
|
s.Require().NotNil(diter)
|
||||||
|
|
||||||
targetSegBuffer := make(map[int64]*storage.DeleteData)
|
targetSegBuffer := make(map[int64]*storage.DeleteData)
|
||||||
targetSegIDs := predicted
|
targetSegIDs := predicted
|
||||||
err = s.task.splitDelta([]*iter.DeltalogIterator{diter}, targetSegBuffer, targetSegIDs)
|
s.task.splitDelta(context.TODO(), []*iter.DeltalogIterator{diter}, targetSegBuffer, targetSegIDs)
|
||||||
s.NoError(err)
|
|
||||||
|
|
||||||
s.NotEmpty(targetSegBuffer)
|
s.NotEmpty(targetSegBuffer)
|
||||||
s.ElementsMatch(predicted, lo.Keys(targetSegBuffer))
|
s.ElementsMatch(predicted, lo.Keys(targetSegBuffer))
|
||||||
|
@ -435,13 +562,19 @@ func (s *LevelZeroCompactionTaskSuite) TestLoadDelta() {
|
||||||
}{
|
}{
|
||||||
{"no error", []string{"correct"}, false, false},
|
{"no error", []string{"correct"}, false, false},
|
||||||
{"download error", []string{"error"}, true, true},
|
{"download error", []string{"error"}, true, true},
|
||||||
{"new iter error", []string{"invalid-blobs"}, true, true},
|
{"new iter error", []string{"invalid-blobs"}, true, false},
|
||||||
}
|
}
|
||||||
|
|
||||||
for _, test := range tests {
|
for _, test := range tests {
|
||||||
iters, err := s.task.loadDelta(ctx, test.paths)
|
iters, err := s.task.loadDelta(ctx, test.paths)
|
||||||
if test.expectNilIter {
|
if test.expectNilIter {
|
||||||
s.Nil(iters)
|
if len(iters) > 0 {
|
||||||
|
for _, iter := range iters {
|
||||||
|
s.False(iter.HasNext())
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
s.Nil(iters)
|
||||||
|
}
|
||||||
} else {
|
} else {
|
||||||
s.NotNil(iters)
|
s.NotNil(iters)
|
||||||
s.Equal(1, len(iters))
|
s.Equal(1, len(iters))
|
||||||
|
|
|
@ -2992,8 +2992,8 @@ func (p *dataNodeConfig) init(base *BaseTable) {
|
||||||
p.FlushDeleteBufferBytes = ParamItem{
|
p.FlushDeleteBufferBytes = ParamItem{
|
||||||
Key: "dataNode.segment.deleteBufBytes",
|
Key: "dataNode.segment.deleteBufBytes",
|
||||||
Version: "2.0.0",
|
Version: "2.0.0",
|
||||||
DefaultValue: "67108864",
|
DefaultValue: "16777216",
|
||||||
Doc: "Max buffer size to flush del for a single channel",
|
Doc: "Max buffer size in bytes to flush del for a single channel, default as 16MB",
|
||||||
Export: true,
|
Export: true,
|
||||||
}
|
}
|
||||||
p.FlushDeleteBufferBytes.Init(base.mgr)
|
p.FlushDeleteBufferBytes.Init(base.mgr)
|
||||||
|
|
Loading…
Reference in New Issue