From d5c2dbb92b774fc8710fd7de866c326e4d3a880c Mon Sep 17 00:00:00 2001 From: XuanYang-cn Date: Thu, 23 May 2024 09:53:40 +0800 Subject: [PATCH 1/3] enhance: [cherry-pick] Move compactor into sub package This PR consists of the following commits: - enhance: Tidy compactor and remove dup codes (#32198) - fix: Fix l0 compactor may cause DN from OOM (#33554) - enhance: Add deltaRowCount in l0 compaction (#33997) - enhance: enable stream writer in compactions (#32612) See also: #32451, #33547, #33998, #31679 pr: #32198, #33554, #33997, #32612 Signed-off-by: yangxuan --- Makefile | 3 +- internal/datacoord/compaction_view.go | 3 +- internal/datacoord/mock_session_manager.go | 3 +- internal/datanode/binlog_io.go | 254 ---- internal/datanode/binlog_io_test.go | 407 ------ internal/datanode/compaction/compactor.go | 32 + .../datanode/{ => compaction}/l0_compactor.go | 44 +- .../{ => compaction}/l0_compactor_test.go | 21 +- internal/datanode/compaction/mix_compactor.go | 568 ++++++++ .../datanode/compaction/mix_compactor_test.go | 805 ++++++++++++ .../datanode/compaction/mock_compactor.go | 307 +++++ .../datanode/compaction/segment_writer.go | 237 ++++ internal/datanode/compaction_executor.go | 55 +- internal/datanode/compaction_executor_test.go | 137 +- internal/datanode/compactor.go | 842 ------------ internal/datanode/compactor_test.go | 1162 ----------------- internal/datanode/data_node.go | 1 - internal/datanode/data_sync_service.go | 148 +-- internal/datanode/io/binlog_io.go | 13 +- internal/datanode/io/binlog_io_test.go | 26 +- internal/datanode/io/io_pool.go | 51 + internal/datanode/io/io_pool_test.go | 29 + internal/datanode/io/mock_binlogio.go | 55 - internal/datanode/io_pool.go | 59 - internal/datanode/io_pool_test.go | 37 - internal/datanode/mock_test.go | 51 - internal/datanode/segment_writer.go | 81 -- internal/datanode/services.go | 18 +- internal/datanode/services_test.go | 20 +- internal/datanode/util/load_stats.go | 166 +++ .../datanode/writebuffer/insert_buffer.go | 11 +- internal/mocks/mock_datanode.go | 3 +- internal/mocks/mock_datanode_client.go | 3 +- internal/storage/serde.go | 36 +- internal/storage/serde_test.go | 7 +- 35 files changed, 2380 insertions(+), 3315 deletions(-) delete mode 100644 internal/datanode/binlog_io.go delete mode 100644 internal/datanode/binlog_io_test.go create mode 100644 internal/datanode/compaction/compactor.go rename internal/datanode/{ => compaction}/l0_compactor.go (91%) rename internal/datanode/{ => compaction}/l0_compactor_test.go (96%) create mode 100644 internal/datanode/compaction/mix_compactor.go create mode 100644 internal/datanode/compaction/mix_compactor_test.go create mode 100644 internal/datanode/compaction/mock_compactor.go create mode 100644 internal/datanode/compaction/segment_writer.go delete mode 100644 internal/datanode/compactor.go delete mode 100644 internal/datanode/compactor_test.go delete mode 100644 internal/datanode/io_pool.go delete mode 100644 internal/datanode/io_pool_test.go delete mode 100644 internal/datanode/segment_writer.go create mode 100644 internal/datanode/util/load_stats.go diff --git a/Makefile b/Makefile index 2bb8ca7728002..f41bb53387f78 100644 --- a/Makefile +++ b/Makefile @@ -475,6 +475,7 @@ generate-mockery-datanode: getdeps $(INSTALL_PATH)/mockery --name=BinlogIO --dir=$(PWD)/internal/datanode/io --output=$(PWD)/internal/datanode/io --filename=mock_binlogio.go --with-expecter --structname=MockBinlogIO --outpkg=io --inpackage $(INSTALL_PATH)/mockery --name=FlowgraphManager --dir=$(PWD)/internal/datanode --output=$(PWD)/internal/datanode --filename=mock_fgmanager.go --with-expecter --structname=MockFlowgraphManager --outpkg=datanode --inpackage $(INSTALL_PATH)/mockery --name=ChannelManager --dir=$(PWD)/internal/datanode --output=$(PWD)/internal/datanode --filename=mock_channelmanager.go --with-expecter --structname=MockChannelManager --outpkg=datanode --inpackage + $(INSTALL_PATH)/mockery --name=Compactor --dir=$(PWD)/internal/datanode/compaction --output=$(PWD)/internal/datanode/compaction --filename=mock_compactor.go --with-expecter --structname=MockCompactor --outpkg=compaction --inpackage generate-mockery-metastore: getdeps $(INSTALL_PATH)/mockery --name=RootCoordCatalog --dir=$(PWD)/internal/metastore --output=$(PWD)/internal/metastore/mocks --filename=mock_rootcoord_catalog.go --with-expecter --structname=RootCoordCatalog --outpkg=mocks @@ -517,4 +518,4 @@ mmap-migration: @source $(PWD)/scripts/setenv.sh && \ mkdir -p $(INSTALL_PATH) && go env -w CGO_ENABLED="1" && \ GO111MODULE=on $(GO) build -pgo=$(PGO_PATH)/default.pgo -ldflags="-r $${RPATH} -X '$(OBJPREFIX).BuildTags=$(BUILD_TAGS)' -X '$(OBJPREFIX).BuildTime=$(BUILD_TIME)' -X '$(OBJPREFIX).GitCommit=$(GIT_COMMIT)' -X '$(OBJPREFIX).GoVersion=$(GO_VERSION)'" \ - -tags dynamic -o $(INSTALL_PATH)/mmap-migration $(MMAP_MIGRATION_PATH)/main.go 1>/dev/null \ No newline at end of file + -tags dynamic -o $(INSTALL_PATH)/mmap-migration $(MMAP_MIGRATION_PATH)/main.go 1>/dev/null diff --git a/internal/datacoord/compaction_view.go b/internal/datacoord/compaction_view.go index 0e7a25d334c7f..82647870a38d9 100644 --- a/internal/datacoord/compaction_view.go +++ b/internal/datacoord/compaction_view.go @@ -148,7 +148,8 @@ func (v *SegmentView) Equal(other *SegmentView) bool { v.DeltaSize == other.DeltaSize && v.BinlogCount == other.BinlogCount && v.StatslogCount == other.StatslogCount && - v.DeltalogCount == other.DeltalogCount + v.DeltalogCount == other.DeltalogCount && + v.DeltaRowCount == other.DeltaRowCount } func (v *SegmentView) String() string { diff --git a/internal/datacoord/mock_session_manager.go b/internal/datacoord/mock_session_manager.go index b35ead232a23d..16288bf490df2 100644 --- a/internal/datacoord/mock_session_manager.go +++ b/internal/datacoord/mock_session_manager.go @@ -813,7 +813,8 @@ func (_c *MockSessionManager_SyncSegments_Call) RunAndReturn(run func(int64, *da func NewMockSessionManager(t interface { mock.TestingT Cleanup(func()) -}) *MockSessionManager { +}, +) *MockSessionManager { mock := &MockSessionManager{} mock.Mock.Test(t) diff --git a/internal/datanode/binlog_io.go b/internal/datanode/binlog_io.go deleted file mode 100644 index c6ff5425f6d57..0000000000000 --- a/internal/datanode/binlog_io.go +++ /dev/null @@ -1,254 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package datanode - -import ( - "context" - "strconv" - - "github.com/cockroachdb/errors" - "go.opentelemetry.io/otel" - "go.uber.org/zap" - - "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/log" - "github.com/milvus-io/milvus/pkg/util/metautil" - "github.com/milvus-io/milvus/pkg/util/typeutil" -) - -var ( - errUploadToBlobStorage = errors.New("upload to blob storage wrong") - errDownloadFromBlobStorage = errors.New("download from blob storage wrong") - // errStart used for retry start - errStart = errors.New("start") -) - -func downloadBlobs(ctx context.Context, b io.BinlogIO, paths []string) ([]*Blob, error) { - ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "downloadBlobs") - defer span.End() - log.Debug("down load", zap.Strings("path", paths)) - bytes, err := b.Download(ctx, paths) - if err != nil { - log.Warn("ctx done when downloading kvs from blob storage", zap.Strings("paths", paths)) - return nil, errDownloadFromBlobStorage - } - resp := make([]*Blob, len(paths)) - if len(paths) == 0 { - return resp, nil - } - for i := range bytes { - resp[i] = &Blob{Key: paths[i], Value: bytes[i]} - } - return resp, nil -} - -// genDeltaBlobs returns key, value -func genDeltaBlobs(b io.BinlogIO, allocator allocator.Allocator, data *DeleteData, collID, partID, segID UniqueID) (string, []byte, error) { - dCodec := storage.NewDeleteCodec() - - blob, err := dCodec.Serialize(collID, partID, segID, data) - if err != nil { - return "", nil, err - } - - idx, err := allocator.AllocOne() - if err != nil { - return "", nil, err - } - k := metautil.JoinIDPath(collID, partID, segID, idx) - key := b.JoinFullPath(common.SegmentDeltaLogPath, k) - - return key, blob.GetValue(), nil -} - -// genInsertBlobs returns insert-paths and save blob to kvs -func genInsertBlobs(b io.BinlogIO, allocator allocator.Allocator, data *InsertData, collectionID, partID, segID UniqueID, iCodec *storage.InsertCodec, kvs map[string][]byte) (map[UniqueID]*datapb.FieldBinlog, error) { - inlogs, err := iCodec.Serialize(partID, segID, data) - if err != nil { - return nil, err - } - - inpaths := make(map[UniqueID]*datapb.FieldBinlog) - notifyGenIdx := make(chan struct{}) - defer close(notifyGenIdx) - - generator, err := allocator.GetGenerator(len(inlogs), notifyGenIdx) - if err != nil { - return nil, err - } - - for _, blob := range inlogs { - // Blob Key is generated by Serialize from int64 fieldID in collection schema, which won't raise error in ParseInt - fID, _ := strconv.ParseInt(blob.GetKey(), 10, 64) - k := metautil.JoinIDPath(collectionID, partID, segID, fID, <-generator) - key := b.JoinFullPath(common.SegmentInsertLogPath, k) - value := blob.GetValue() - fileLen := len(value) - - kvs[key] = value - inpaths[fID] = &datapb.FieldBinlog{ - FieldID: fID, - Binlogs: []*datapb.Binlog{{LogSize: int64(fileLen), LogPath: key, EntriesNum: blob.RowNum, MemorySize: blob.GetMemorySize()}}, - } - } - - return inpaths, nil -} - -// genStatBlobs return stats log paths and save blob to kvs -func genStatBlobs(b io.BinlogIO, allocator allocator.Allocator, stats *storage.PrimaryKeyStats, collectionID, partID, segID UniqueID, iCodec *storage.InsertCodec, kvs map[string][]byte, totRows int64) (map[UniqueID]*datapb.FieldBinlog, error) { - statBlob, err := iCodec.SerializePkStats(stats, totRows) - if err != nil { - return nil, err - } - statPaths := make(map[UniqueID]*datapb.FieldBinlog) - - idx, err := allocator.AllocOne() - if err != nil { - return nil, err - } - fID, _ := strconv.ParseInt(statBlob.GetKey(), 10, 64) - k := metautil.JoinIDPath(collectionID, partID, segID, fID, idx) - key := b.JoinFullPath(common.SegmentStatslogPath, k) - value := statBlob.GetValue() - fileLen := len(value) - - kvs[key] = value - - statPaths[fID] = &datapb.FieldBinlog{ - FieldID: fID, - Binlogs: []*datapb.Binlog{{LogSize: int64(fileLen), LogPath: key, EntriesNum: totRows, MemorySize: int64(fileLen)}}, - } - return statPaths, nil -} - -// update stats log -// also update with insert data if not nil -func uploadStatsLog( - ctx context.Context, - b io.BinlogIO, - allocator allocator.Allocator, - collectionID UniqueID, - partID UniqueID, - segID UniqueID, - stats *storage.PrimaryKeyStats, - totRows int64, - iCodec *storage.InsertCodec, -) (map[UniqueID]*datapb.FieldBinlog, error) { - ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "UploadStatslog") - defer span.End() - kvs := make(map[string][]byte) - - statPaths, err := genStatBlobs(b, allocator, stats, collectionID, partID, segID, iCodec, kvs, totRows) - if err != nil { - return nil, err - } - - err = b.Upload(ctx, kvs) - if err != nil { - return nil, err - } - - return statPaths, nil -} - -func uploadInsertLog( - ctx context.Context, - b io.BinlogIO, - allocator allocator.Allocator, - collectionID UniqueID, - partID UniqueID, - segID UniqueID, - iData *InsertData, - iCodec *storage.InsertCodec, -) (map[UniqueID]*datapb.FieldBinlog, error) { - ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "UploadInsertLog") - defer span.End() - kvs := make(map[string][]byte) - - if iData.IsEmpty() { - log.Warn("binlog io uploading empty insert data", - zap.Int64("segmentID", segID), - zap.Int64("collectionID", iCodec.Schema.GetID()), - ) - return nil, nil - } - - inpaths, err := genInsertBlobs(b, allocator, iData, collectionID, partID, segID, iCodec, kvs) - if err != nil { - return nil, err - } - - err = b.Upload(ctx, kvs) - if err != nil { - return nil, err - } - - return inpaths, nil -} - -func uploadDeltaLog( - ctx context.Context, - b io.BinlogIO, - allocator allocator.Allocator, - collectionID UniqueID, - partID UniqueID, - segID UniqueID, - dData *DeleteData, -) ([]*datapb.FieldBinlog, error) { - ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "UploadDeltaLog") - defer span.End() - var ( - deltaInfo = make([]*datapb.FieldBinlog, 0) - kvs = make(map[string][]byte) - ) - - if dData.RowCount > 0 { - k, v, err := genDeltaBlobs(b, allocator, dData, collectionID, partID, segID) - if err != nil { - log.Warn("generate delta blobs wrong", - zap.Int64("collectionID", collectionID), - 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)), - MemorySize: dData.Size(), - }}, - }) - } else { - return nil, nil - } - - err := b.Upload(ctx, kvs) - if err != nil { - return nil, err - } - - return deltaInfo, nil -} diff --git a/internal/datanode/binlog_io_test.go b/internal/datanode/binlog_io_test.go deleted file mode 100644 index eea1b18291e81..0000000000000 --- a/internal/datanode/binlog_io_test.go +++ /dev/null @@ -1,407 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package datanode - -import ( - "context" - "fmt" - "path" - "testing" - "time" - - "github.com/cockroachdb/errors" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - "go.uber.org/zap" - - "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" - "github.com/milvus-io/milvus/internal/datanode/allocator" - "github.com/milvus-io/milvus/internal/datanode/io" - "github.com/milvus-io/milvus/internal/storage" - "github.com/milvus-io/milvus/pkg/common" - "github.com/milvus-io/milvus/pkg/log" -) - -var binlogTestDir = "/tmp/milvus_test/test_binlog_io" - -var validGeneratorFn = func(count int, done <-chan struct{}) <-chan UniqueID { - ret := make(chan UniqueID, count) - for i := 0; i < count; i++ { - ret <- int64(100 + i) - } - return ret -} - -func TestBinlogIOInterfaceMethods(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cm := storage.NewLocalChunkManager(storage.RootPath(binlogTestDir)) - defer cm.RemoveWithPrefix(ctx, cm.RootPath()) - - t.Run("Test download", func(t *testing.T) { - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - tests := []struct { - isvalid bool - ks []string // for preparation - - inctx context.Context - - description string - }{ - {true, []string{"a", "b", "c"}, context.TODO(), "valid input"}, - {false, nil, context.Background(), "cancel by context"}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - if test.isvalid { - inkeys := []string{} - for _, k := range test.ks { - blob, key, err := prepareBlob(cm, k) - require.NoError(t, err) - assert.NotEmpty(t, blob) - inkeys = append(inkeys, key) - - loaded, err := downloadBlobs(test.inctx, binlogIO, []string{key}) - assert.NoError(t, err) - assert.ElementsMatch(t, blob, loaded[0].GetValue()) - } - - loaded, err := downloadBlobs(test.inctx, binlogIO, inkeys) - assert.NoError(t, err) - assert.Equal(t, len(test.ks), len(loaded)) - } else { - ctx, cancel := context.WithCancel(test.inctx) - cancel() - - _, err := downloadBlobs(ctx, binlogIO, []string{"test"}) - assert.EqualError(t, err, errDownloadFromBlobStorage.Error()) - } - }) - } - }) - - t.Run("Test download twice", func(t *testing.T) { - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - - ctx, cancel := context.WithTimeout(context.TODO(), time.Millisecond*20) - blobs, err := downloadBlobs(ctx, binlogIO, []string{"a"}) - assert.Error(t, err) - assert.Empty(t, blobs) - cancel() - }) - - t.Run("Test upload stats log err", func(t *testing.T) { - f := &MetaFactory{} - meta := f.GetCollectionMeta(UniqueID(10001), "test_gen_blobs", schemapb.DataType_Int64) - - t.Run("gen insert blob failed", func(t *testing.T) { - alloc := allocator.NewMockAllocator(t) - alloc.EXPECT().AllocOne().Call.Return(int64(0), fmt.Errorf("mock AllocOne error")) - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - _, err := uploadStatsLog(context.Background(), binlogIO, alloc, meta.GetID(), 10, 1, genTestStat(meta), 10, iCodec) - assert.Error(t, err) - }) - }) - - t.Run("Test upload insert log err", func(t *testing.T) { - f := &MetaFactory{} - meta := f.GetCollectionMeta(UniqueID(10001), "test_gen_blobs", schemapb.DataType_Int64) - - t.Run("empty insert", func(t *testing.T) { - alloc := allocator.NewMockAllocator(t) - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - paths, err := uploadInsertLog(context.Background(), binlogIO, alloc, meta.GetID(), 10, 1, genEmptyInsertData(), iCodec) - assert.NoError(t, err) - assert.Nil(t, paths) - }) - - t.Run("gen insert blob failed", func(t *testing.T) { - alloc := allocator.NewMockAllocator(t) - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - alloc.EXPECT().GetGenerator(mock.Anything, mock.Anything).Call.Return(nil, fmt.Errorf("mock err")) - _, err := uploadInsertLog(context.Background(), binlogIO, alloc, meta.GetID(), 10, 1, genInsertData(2), iCodec) - assert.Error(t, err) - }) - - t.Run("upload failed", func(t *testing.T) { - mkc := &mockCm{errRead: true, errSave: true} - alloc := allocator.NewMockAllocator(t) - binlogIO := io.NewBinlogIO(mkc, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - - alloc.EXPECT().GetGenerator(mock.Anything, mock.Anything).Call.Return(validGeneratorFn, nil) - - ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) - defer cancel() - - _, err := uploadInsertLog(ctx, binlogIO, alloc, meta.GetID(), 1, 10, genInsertData(2), iCodec) - assert.Error(t, err) - }) - }) -} - -func prepareBlob(cm storage.ChunkManager, key string) ([]byte, string, error) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - k := path.Join(cm.RootPath(), "test_prepare_blob", key) - blob := []byte{1, 2, 3, 255, 188} - - err := cm.Write(ctx, k, blob[:]) - if err != nil { - return nil, "", err - } - return blob, k, nil -} - -func TestBinlogIOInnerMethods(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cm := storage.NewLocalChunkManager(storage.RootPath(binlogTestDir)) - defer cm.RemoveWithPrefix(ctx, cm.RootPath()) - - t.Run("Test genDeltaBlobs", func(t *testing.T) { - alloc := allocator.NewMockAllocator(t) - alloc.EXPECT().AllocOne().Call.Return(int64(11111), nil) - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - f := &MetaFactory{} - meta := f.GetCollectionMeta(UniqueID(10002), "test_gen_blobs", schemapb.DataType_Int64) - - tests := []struct { - isvalid bool - deletepk storage.PrimaryKey - ts uint64 - - description string - }{ - {true, storage.NewInt64PrimaryKey(1), 1111111, "valid input"}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - if test.isvalid { - k, v, err := genDeltaBlobs(binlogIO, alloc, &DeleteData{ - Pks: []storage.PrimaryKey{test.deletepk}, - Tss: []uint64{test.ts}, - }, meta.GetID(), 10, 1) - - assert.NoError(t, err) - assert.NotEmpty(t, k) - assert.NotEmpty(t, v) - - log.Debug("genDeltaBlobs returns", zap.String("key", k)) - } - }) - } - }) - - t.Run("Test genDeltaBlobs error", func(t *testing.T) { - pk := storage.NewInt64PrimaryKey(1) - - t.Run("Test serialize error", func(t *testing.T) { - alloc := allocator.NewMockAllocator(t) - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - k, v, err := genDeltaBlobs(binlogIO, alloc, &DeleteData{Pks: []storage.PrimaryKey{pk}, Tss: []uint64{}}, 1, 1, 1) - assert.Error(t, err) - assert.Empty(t, k) - assert.Empty(t, v) - }) - - t.Run("Test AllocOne error", func(t *testing.T) { - alloc := allocator.NewMockAllocator(t) - alloc.EXPECT().AllocOne().Call.Return(int64(0), fmt.Errorf("mock AllocOne error")) - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - k, v, err := genDeltaBlobs(binlogIO, alloc, &DeleteData{Pks: []storage.PrimaryKey{pk}, Tss: []uint64{1}}, 1, 1, 1) - assert.Error(t, err) - assert.Empty(t, k) - assert.Empty(t, v) - }) - }) - - t.Run("Test genInsertBlobs", func(t *testing.T) { - f := &MetaFactory{} - alloc := allocator.NewMockAllocator(t) - alloc.EXPECT().GetGenerator(mock.Anything, mock.Anything).Call.Return(validGeneratorFn, nil) - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - - tests := []struct { - pkType schemapb.DataType - description string - expectError bool - }{ - {schemapb.DataType_Int64, "int64PrimaryField", false}, - {schemapb.DataType_VarChar, "varCharPrimaryField", false}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - meta := f.GetCollectionMeta(UniqueID(10001), "test_gen_blobs", test.pkType) - iCodec := storage.NewInsertCodecWithSchema(meta) - - kvs := make(map[string][]byte) - pin, err := genInsertBlobs(binlogIO, alloc, genInsertData(2), meta.GetID(), 10, 1, iCodec, kvs) - - assert.NoError(t, err) - assert.Equal(t, 12, len(pin)) - assert.Equal(t, 12, len(kvs)) - - log.Debug("test paths", - zap.Int("kvs no.", len(kvs)), - zap.String("insert paths field0", pin[common.TimeStampField].GetBinlogs()[0].GetLogPath())) - }) - } - }) - - t.Run("Test genInsertBlobs error", func(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cm := storage.NewLocalChunkManager(storage.RootPath(binlogTestDir)) - defer cm.RemoveWithPrefix(ctx, cm.RootPath()) - - t.Run("serialize error", func(t *testing.T) { - iCodec := storage.NewInsertCodecWithSchema(nil) - - alloc := allocator.NewMockAllocator(t) - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - kvs := make(map[string][]byte) - pin, err := genInsertBlobs(binlogIO, alloc, genEmptyInsertData(), 0, 10, 1, iCodec, kvs) - - assert.Error(t, err) - assert.Empty(t, kvs) - assert.Empty(t, pin) - }) - - t.Run("GetGenerator error", func(t *testing.T) { - f := &MetaFactory{} - meta := f.GetCollectionMeta(UniqueID(10001), "test_gen_blobs", schemapb.DataType_Int64) - iCodec := storage.NewInsertCodecWithSchema(meta) - - alloc := allocator.NewMockAllocator(t) - alloc.EXPECT().GetGenerator(mock.Anything, mock.Anything).Return(nil, fmt.Errorf("mock GetGenerator error")) - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - kvs := make(map[string][]byte) - - pin, err := genInsertBlobs(binlogIO, alloc, genInsertData(2), meta.GetID(), 10, 1, iCodec, kvs) - - assert.Error(t, err) - assert.Empty(t, kvs) - assert.Empty(t, pin) - }) - }) - - t.Run("Test genStatsBlob", func(t *testing.T) { - f := &MetaFactory{} - alloc := allocator.NewMockAllocator(t) - alloc.EXPECT().AllocOne().Return(0, nil) - - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - - tests := []struct { - pkType schemapb.DataType - description string - expectError bool - }{ - {schemapb.DataType_Int64, "int64PrimaryField", false}, - {schemapb.DataType_VarChar, "varCharPrimaryField", false}, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - meta := f.GetCollectionMeta(UniqueID(10001), "test_gen_stat_blobs", test.pkType) - iCodec := storage.NewInsertCodecWithSchema(meta) - - kvs := make(map[string][]byte) - stat, err := genStatBlobs(binlogIO, alloc, genTestStat(meta), meta.GetID(), 10, 1, iCodec, kvs, 0) - - assert.NoError(t, err) - assert.Equal(t, 1, len(stat)) - assert.Equal(t, 1, len(kvs)) - }) - } - }) - - t.Run("Test genStatsBlob error", func(t *testing.T) { - f := &MetaFactory{} - alloc := allocator.NewMockAllocator(t) - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - - t.Run("serialize error", func(t *testing.T) { - meta := f.GetCollectionMeta(UniqueID(10001), "test_gen_stat_blobs_error", schemapb.DataType_Int64) - iCodec := storage.NewInsertCodecWithSchema(meta) - - kvs := make(map[string][]byte) - _, err := genStatBlobs(binlogIO, alloc, nil, meta.GetID(), 10, 1, iCodec, kvs, 0) - assert.Error(t, err) - }) - }) -} - -type mockCm struct { - storage.ChunkManager - errRead bool - errSave bool - MultiReadReturn [][]byte - ReadReturn []byte -} - -var _ storage.ChunkManager = (*mockCm)(nil) - -func (mk *mockCm) RootPath() string { - return "mock_test" -} - -func (mk *mockCm) Write(ctx context.Context, filePath string, content []byte) error { - if mk.errSave { - return errors.New("mockKv save error") - } - return nil -} - -func (mk *mockCm) MultiWrite(ctx context.Context, contents map[string][]byte) error { - if mk.errSave { - return errors.New("mockKv save error") - } - return nil -} - -func (mk *mockCm) Read(ctx context.Context, filePath string) ([]byte, error) { - if mk.errRead { - return nil, errors.New("mockKv read error") - } - return mk.ReadReturn, nil -} - -func (mk *mockCm) MultiRead(ctx context.Context, filePaths []string) ([][]byte, error) { - if mk.MultiReadReturn != nil { - return mk.MultiReadReturn, nil - } - return [][]byte{[]byte("a")}, nil -} - -func (mk *mockCm) ReadWithPrefix(ctx context.Context, prefix string) ([]string, [][]byte, error) { - return nil, nil, nil -} - -func (mk *mockCm) Remove(ctx context.Context, key string) error { return nil } -func (mk *mockCm) MultiRemove(ctx context.Context, keys []string) error { return nil } -func (mk *mockCm) RemoveWithPrefix(ctx context.Context, key string) error { return nil } -func (mk *mockCm) Close() {} diff --git a/internal/datanode/compaction/compactor.go b/internal/datanode/compaction/compactor.go new file mode 100644 index 0000000000000..da57562d93e28 --- /dev/null +++ b/internal/datanode/compaction/compactor.go @@ -0,0 +1,32 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package compaction + +import ( + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +type Compactor interface { + Complete() + Compact() (*datapb.CompactionPlanResult, error) + InjectDone() + Stop() + GetPlanID() typeutil.UniqueID + GetCollection() typeutil.UniqueID + GetChannelName() string +} diff --git a/internal/datanode/l0_compactor.go b/internal/datanode/compaction/l0_compactor.go similarity index 91% rename from internal/datanode/l0_compactor.go rename to internal/datanode/compaction/l0_compactor.go index a9840415cb0ce..409abdbb7ca8b 100644 --- a/internal/datanode/l0_compactor.go +++ b/internal/datanode/compaction/l0_compactor.go @@ -14,13 +14,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package compaction import ( "context" "fmt" "math" + "github.com/cockroachdb/errors" "github.com/samber/lo" "go.opentelemetry.io/otel" "go.uber.org/zap" @@ -43,8 +44,7 @@ import ( "github.com/milvus-io/milvus/pkg/util/typeutil" ) -type levelZeroCompactionTask struct { - compactor +type LevelZeroCompactionTask struct { io.BinlogIO allocator allocator.Allocator @@ -60,16 +60,16 @@ type levelZeroCompactionTask struct { tr *timerecord.TimeRecorder } -func newLevelZeroCompactionTask( +func NewLevelZeroCompactionTask( ctx context.Context, binlogIO io.BinlogIO, alloc allocator.Allocator, metaCache metacache.MetaCache, syncmgr syncmgr.SyncManager, plan *datapb.CompactionPlan, -) *levelZeroCompactionTask { +) *LevelZeroCompactionTask { ctx, cancel := context.WithCancel(ctx) - return &levelZeroCompactionTask{ + return &LevelZeroCompactionTask{ ctx: ctx, cancel: cancel, @@ -83,31 +83,31 @@ func newLevelZeroCompactionTask( } } -func (t *levelZeroCompactionTask) complete() { +func (t *LevelZeroCompactionTask) Complete() { t.done <- struct{}{} } -func (t *levelZeroCompactionTask) stop() { +func (t *LevelZeroCompactionTask) Stop() { t.cancel() <-t.done } -func (t *levelZeroCompactionTask) getPlanID() UniqueID { +func (t *LevelZeroCompactionTask) GetPlanID() int64 { return t.plan.GetPlanID() } -func (t *levelZeroCompactionTask) getChannelName() string { +func (t *LevelZeroCompactionTask) GetChannelName() string { return t.plan.GetChannel() } -func (t *levelZeroCompactionTask) getCollection() int64 { +func (t *LevelZeroCompactionTask) GetCollection() int64 { return t.metacache.Collection() } // Do nothing for levelzero compaction -func (t *levelZeroCompactionTask) injectDone() {} +func (t *LevelZeroCompactionTask) InjectDone() {} -func (t *levelZeroCompactionTask) compact() (*datapb.CompactionPlanResult, error) { +func (t *LevelZeroCompactionTask) Compact() (*datapb.CompactionPlanResult, error) { ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, "L0Compact") defer span.End() log := log.Ctx(t.ctx).With(zap.Int64("planID", t.plan.GetPlanID()), zap.String("type", t.plan.GetType().String())) @@ -115,7 +115,7 @@ func (t *levelZeroCompactionTask) compact() (*datapb.CompactionPlanResult, error if !funcutil.CheckCtxValid(ctx) { log.Warn("compact wrong, task context done or timeout") - return nil, errContext + return nil, ctx.Err() } l0Segments := lo.Filter(t.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) bool { @@ -130,7 +130,7 @@ func (t *levelZeroCompactionTask) compact() (*datapb.CompactionPlanResult, error }) if len(targetSegIDs) == 0 { log.Warn("compact wrong, not target sealed segments") - return nil, errIllegalCompactionPlan + return nil, errors.New("illegal compaction plan with empty target segments") } err := binlog.DecompressCompactionBinlogs(l0Segments) if err != nil { @@ -140,7 +140,7 @@ func (t *levelZeroCompactionTask) compact() (*datapb.CompactionPlanResult, error var ( totalSize int64 - totalDeltalogs = make(map[UniqueID][]string) + totalDeltalogs = make(map[int64][]string) ) for _, s := range l0Segments { paths := []string{} @@ -187,7 +187,7 @@ func getMaxBatchSize(totalSize int64) int { return max } -func (t *levelZeroCompactionTask) serializeUpload(ctx context.Context, segmentWriters map[int64]*SegmentDeltaWriter) ([]*datapb.CompactionSegment, error) { +func (t *LevelZeroCompactionTask) serializeUpload(ctx context.Context, segmentWriters map[int64]*SegmentDeltaWriter) ([]*datapb.CompactionSegment, error) { traceCtx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "L0Compact serializeUpload") defer span.End() allBlobs := make(map[string][]byte) @@ -237,7 +237,7 @@ func (t *levelZeroCompactionTask) serializeUpload(ctx context.Context, segmentWr return results, nil } -func (t *levelZeroCompactionTask) splitDelta( +func (t *LevelZeroCompactionTask) splitDelta( ctx context.Context, allDelta *storage.DeleteData, targetSegIDs []int64, @@ -265,7 +265,7 @@ func (t *levelZeroCompactionTask) splitDelta( writer, ok := targetSegBuffer[segmentID] if !ok { segment := allSeg[segmentID] - writer = NewSegmentDeltaWriter(segmentID, segment.GetPartitionID(), t.getCollection()) + writer = NewSegmentDeltaWriter(segmentID, segment.GetPartitionID(), t.GetCollection()) targetSegBuffer[segmentID] = writer } writer.Write(allDelta.Pks[startIdx+i], allDelta.Tss[startIdx+i]) @@ -282,7 +282,7 @@ type BatchApplyRet = struct { Segment2Hits map[int64][]bool } -func (t *levelZeroCompactionTask) applyBFInParallel(ctx context.Context, deltaData *storage.DeleteData, pool *conc.Pool[any], segmentBfs []*metacache.SegmentInfo) *typeutil.ConcurrentMap[int, *BatchApplyRet] { +func (t *LevelZeroCompactionTask) applyBFInParallel(ctx context.Context, deltaData *storage.DeleteData, pool *conc.Pool[any], segmentBfs []*metacache.SegmentInfo) *typeutil.ConcurrentMap[int, *BatchApplyRet] { _, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "L0Compact applyBFInParallel") defer span.End() batchSize := paramtable.Get().CommonCfg.BloomFilterApplyBatchSize.GetAsInt() @@ -325,7 +325,7 @@ func (t *levelZeroCompactionTask) applyBFInParallel(ctx context.Context, deltaDa return retMap } -func (t *levelZeroCompactionTask) process(ctx context.Context, batchSize int, targetSegments []int64, deltaLogs ...[]string) ([]*datapb.CompactionSegment, error) { +func (t *LevelZeroCompactionTask) process(ctx context.Context, batchSize int, targetSegments []int64, deltaLogs ...[]string) ([]*datapb.CompactionSegment, error) { ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "L0Compact process") defer span.End() @@ -370,7 +370,7 @@ func (t *levelZeroCompactionTask) process(ctx context.Context, batchSize int, ta return results, nil } -func (t *levelZeroCompactionTask) loadDelta(ctx context.Context, deltaLogs []string) (*storage.DeleteData, error) { +func (t *LevelZeroCompactionTask) loadDelta(ctx context.Context, deltaLogs []string) (*storage.DeleteData, error) { ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "L0Compact loadDelta") defer span.End() diff --git a/internal/datanode/l0_compactor_test.go b/internal/datanode/compaction/l0_compactor_test.go similarity index 96% rename from internal/datanode/l0_compactor_test.go rename to internal/datanode/compaction/l0_compactor_test.go index e9dffdd7305eb..2b7272996d01b 100644 --- a/internal/datanode/l0_compactor_test.go +++ b/internal/datanode/compaction/l0_compactor_test.go @@ -14,7 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package datanode +package compaction import ( "context" @@ -32,10 +32,13 @@ import ( "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/storage" "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/paramtable" "github.com/milvus-io/milvus/pkg/util/timerecord" + "github.com/milvus-io/milvus/pkg/util/typeutil" ) func TestLevelZeroCompactionTaskSuite(t *testing.T) { + paramtable.Init() suite.Run(t, new(LevelZeroCompactionTaskSuite)) } @@ -45,7 +48,7 @@ type LevelZeroCompactionTaskSuite struct { mockBinlogIO *io.MockBinlogIO mockAlloc *allocator.MockAllocator mockMeta *metacache.MockMetaCache - task *levelZeroCompactionTask + task *LevelZeroCompactionTask dData *storage.DeleteData dBlob []byte @@ -56,7 +59,7 @@ func (s *LevelZeroCompactionTaskSuite) SetupTest() { s.mockBinlogIO = io.NewMockBinlogIO(s.T()) s.mockMeta = metacache.NewMockMetaCache(s.T()) // plan of the task is unset - s.task = newLevelZeroCompactionTask(context.Background(), s.mockBinlogIO, s.mockAlloc, s.mockMeta, nil, nil) + s.task = NewLevelZeroCompactionTask(context.Background(), s.mockBinlogIO, s.mockAlloc, s.mockMeta, nil, nil) pk2ts := map[int64]uint64{ 1: 20000, @@ -64,7 +67,7 @@ func (s *LevelZeroCompactionTaskSuite) SetupTest() { 3: 20002, } - s.dData = storage.NewDeleteData([]storage.PrimaryKey{}, []Timestamp{}) + s.dData = storage.NewDeleteData([]storage.PrimaryKey{}, []typeutil.Timestamp{}) for pk, ts := range pk2ts { s.dData.Append(storage.NewInt64PrimaryKey(pk), ts) } @@ -217,9 +220,9 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() { s.mockMeta.EXPECT().Collection().Return(1) s.mockAlloc.EXPECT().AllocOne().Return(19530, nil).Times(2) - s.Require().Equal(plan.GetPlanID(), s.task.getPlanID()) - s.Require().Equal(plan.GetChannel(), s.task.getChannelName()) - s.Require().EqualValues(1, s.task.getCollection()) + s.Require().Equal(plan.GetPlanID(), s.task.GetPlanID()) + s.Require().Equal(plan.GetChannel(), s.task.GetChannelName()) + s.Require().EqualValues(1, s.task.GetCollection()) l0Segments := lo.Filter(s.task.plan.GetSegmentBinlogs(), func(s *datapb.CompactionSegmentBinlogs, _ int) bool { return s.Level == datapb.SegmentLevel_L0 @@ -231,7 +234,7 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() { } return 0, false }) - totalDeltalogs := make(map[UniqueID][]string) + totalDeltalogs := make(map[int64][]string) for _, s := range l0Segments { paths := []string{} @@ -321,7 +324,7 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactBatch() { return 0, false }) - totalDeltalogs := make(map[UniqueID][]string) + totalDeltalogs := make(map[int64][]string) for _, s := range l0Segments { paths := []string{} for _, d := range s.GetDeltalogs() { diff --git a/internal/datanode/compaction/mix_compactor.go b/internal/datanode/compaction/mix_compactor.go new file mode 100644 index 0000000000000..da18de0f82fa8 --- /dev/null +++ b/internal/datanode/compaction/mix_compactor.go @@ -0,0 +1,568 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package compaction + +import ( + "context" + "fmt" + sio "io" + "strconv" + "sync" + "time" + + "github.com/cockroachdb/errors" + "github.com/samber/lo" + "go.opentelemetry.io/otel" + "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/io" + iter "github.com/milvus-io/milvus/internal/datanode/iterators" + "github.com/milvus-io/milvus/internal/datanode/metacache" + "github.com/milvus-io/milvus/internal/datanode/syncmgr" + "github.com/milvus-io/milvus/internal/metastore/kv/binlog" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/metrics" + "github.com/milvus-io/milvus/pkg/util/funcutil" + "github.com/milvus-io/milvus/pkg/util/merr" + "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/timerecord" + "github.com/milvus-io/milvus/pkg/util/tsoutil" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +// for MixCompaction only +type mixCompactionTask struct { + binlogIO io.BinlogIO + Compactor + metaCache metacache.MetaCache + syncMgr syncmgr.SyncManager + allocator.Allocator + currentTs typeutil.Timestamp + + plan *datapb.CompactionPlan + + ctx context.Context + cancel context.CancelFunc + + injectDoneOnce sync.Once + done chan struct{} + tr *timerecord.TimeRecorder +} + +// make sure compactionTask implements compactor interface +var _ Compactor = (*mixCompactionTask)(nil) + +func NewMixCompactionTask( + ctx context.Context, + binlogIO io.BinlogIO, + metaCache metacache.MetaCache, + syncMgr syncmgr.SyncManager, + alloc allocator.Allocator, + plan *datapb.CompactionPlan, +) *mixCompactionTask { + ctx1, cancel := context.WithCancel(ctx) + return &mixCompactionTask{ + ctx: ctx1, + cancel: cancel, + binlogIO: binlogIO, + syncMgr: syncMgr, + metaCache: metaCache, + Allocator: alloc, + plan: plan, + tr: timerecord.NewTimeRecorder("mix compaction"), + currentTs: tsoutil.GetCurrentTime(), + done: make(chan struct{}, 1), + } +} + +func (t *mixCompactionTask) Complete() { + t.done <- struct{}{} +} + +func (t *mixCompactionTask) Stop() { + t.cancel() + <-t.done + t.InjectDone() +} + +func (t *mixCompactionTask) GetPlanID() typeutil.UniqueID { + return t.plan.GetPlanID() +} + +func (t *mixCompactionTask) GetChannelName() string { + return t.plan.GetChannel() +} + +// return num rows of all segment compaction from +func (t *mixCompactionTask) getNumRows() (int64, error) { + numRows := int64(0) + for _, binlog := range t.plan.SegmentBinlogs { + seg, ok := t.metaCache.GetSegmentByID(binlog.GetSegmentID()) + if !ok { + return 0, merr.WrapErrSegmentNotFound(binlog.GetSegmentID(), "get compaction segments num rows failed") + } + + numRows += seg.NumOfRows() + } + + return numRows, nil +} + +func (t *mixCompactionTask) mergeDeltalogs(ctx context.Context, dpaths map[typeutil.UniqueID][]string) (map[interface{}]typeutil.Timestamp, error) { + t.tr.RecordSpan() + ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "mergeDeltalogs") + defer span.End() + + log := log.With(zap.Int64("planID", t.GetPlanID())) + pk2ts := make(map[interface{}]typeutil.Timestamp) + + if len(dpaths) == 0 { + log.Info("compact with no deltalogs, skip merge deltalogs") + return pk2ts, nil + } + + allIters := make([]*iter.DeltalogIterator, 0) + for segID, paths := range dpaths { + if len(paths) == 0 { + continue + } + blobs, err := t.binlogIO.Download(ctx, paths) + if err != nil { + log.Warn("compact wrong, fail to download deltalogs", + zap.Int64("segment", segID), + zap.Strings("path", paths), + zap.Error(err)) + return nil, err + } + + allIters = append(allIters, iter.NewDeltalogIterator(blobs, nil)) + } + + for _, deltaIter := range allIters { + for deltaIter.HasNext() { + labeled, _ := deltaIter.Next() + ts := labeled.GetTimestamp() + if lastTs, ok := pk2ts[labeled.GetPk().GetValue()]; ok && lastTs > ts { + ts = lastTs + } + pk2ts[labeled.GetPk().GetValue()] = ts + } + } + + log.Info("compact mergeDeltalogs end", + zap.Int("deleted pk counts", len(pk2ts)), + zap.Duration("elapse", t.tr.RecordSpan())) + + return pk2ts, nil +} + +func (t *mixCompactionTask) statSerializeWrite(ctx context.Context, writer *SegmentWriter, finalRowCount int64) (*datapb.FieldBinlog, error) { + ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "statslog serializeWrite") + defer span.End() + sblob, err := writer.Finish(finalRowCount) + if err != nil { + return nil, err + } + + logID, err := t.AllocOne() + if err != nil { + return nil, err + } + + key, _ := binlog.BuildLogPath(storage.StatsBinlog, writer.GetCollectionID(), writer.GetPartitionID(), writer.GetSegmentID(), writer.GetPkID(), logID) + kvs := map[string][]byte{key: sblob.GetValue()} + statFieldLog := &datapb.FieldBinlog{ + FieldID: writer.GetPkID(), + Binlogs: []*datapb.Binlog{ + { + LogSize: int64(len(sblob.GetValue())), + MemorySize: int64(len(sblob.GetValue())), + LogPath: key, + EntriesNum: finalRowCount, + }, + }, + } + if err := t.binlogIO.Upload(ctx, kvs); err != nil { + log.Warn("failed to upload insert log", zap.Error(err)) + return nil, err + } + + return statFieldLog, nil +} + +func (t *mixCompactionTask) serializeWrite(ctx context.Context, writer *SegmentWriter) (kvs map[string][]byte, fieldBinlogs map[int64]*datapb.FieldBinlog, err error) { + _, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "serializeWrite") + defer span.End() + + blobs, tr, err := writer.SerializeYield() + startID, _, err := t.Alloc(uint32(len(blobs))) + if err != nil { + return nil, nil, err + } + + kvs = make(map[string][]byte) + fieldBinlogs = make(map[int64]*datapb.FieldBinlog) + for i := range blobs { + // Blob Key is generated by Serialize from int64 fieldID in collection schema, which won't raise error in ParseInt + fID, _ := strconv.ParseInt(blobs[i].GetKey(), 10, 64) + key, _ := binlog.BuildLogPath(storage.InsertBinlog, writer.GetCollectionID(), writer.GetPartitionID(), writer.GetSegmentID(), fID, startID+int64(i)) + + kvs[key] = blobs[i].GetValue() + fieldBinlogs[fID] = &datapb.FieldBinlog{ + FieldID: fID, + Binlogs: []*datapb.Binlog{ + { + LogSize: int64(len(blobs[i].GetValue())), + MemorySize: blobs[i].GetMemorySize(), + LogPath: key, + EntriesNum: blobs[i].RowNum, + TimestampFrom: tr.GetMinTimestamp(), + TimestampTo: tr.GetMaxTimestamp(), + }, + }, + } + } + + return +} + +func (t *mixCompactionTask) merge( + ctx context.Context, + binlogPaths [][]string, + delta map[interface{}]typeutil.Timestamp, + writer *SegmentWriter, +) (*datapb.CompactionSegment, error) { + _ = t.tr.RecordSpan() + + ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "CompactMerge") + defer span.End() + + log := log.With(zap.Int64("planID", t.GetPlanID()), zap.Int64("compactTo segment", writer.GetSegmentID())) + + var ( + syncBatchCount int // binlog batch count + remainingRowCount int64 // the number of remaining entities + expiredRowCount int64 // the number of expired entities + unflushedRowCount int64 = 0 + + // All binlog meta of a segment + allBinlogs = make(map[typeutil.UniqueID]*datapb.FieldBinlog) + ) + + isValueDeleted := func(v *storage.Value) bool { + ts, ok := delta[v.PK.GetValue()] + // insert task and delete task has the same ts when upsert + // here should be < instead of <= + // to avoid the upsert data to be deleted after compact + if ok && uint64(v.Timestamp) < ts { + return true + } + return false + } + + downloadTimeCost := time.Duration(0) + serWriteTimeCost := time.Duration(0) + uploadTimeCost := time.Duration(0) + + for _, paths := range binlogPaths { + log := log.With(zap.Strings("paths", paths)) + downloadStart := time.Now() + allValues, err := t.binlogIO.Download(ctx, paths) + if err != nil { + log.Warn("compact wrong, fail to download insertLogs", zap.Error(err)) + } + downloadTimeCost += time.Since(downloadStart) + + blobs := lo.Map(allValues, func(v []byte, i int) *storage.Blob { + return &storage.Blob{Key: paths[i], Value: v} + }) + + iter, err := storage.NewBinlogDeserializeReader(blobs, writer.GetPkID()) + if err != nil { + log.Warn("compact wrong, failed to new insert binlogs reader", zap.Error(err)) + return nil, err + } + + for { + err := iter.Next() + if err != nil { + if err == sio.EOF { + break + } else { + log.Warn("compact wrong, failed to iter through data", zap.Error(err)) + return nil, err + } + } + v := iter.Value() + if isValueDeleted(v) { + continue + } + + // Filtering expired entity + if t.isExpiredEntity(typeutil.Timestamp(v.Timestamp)) { + expiredRowCount++ + continue + } + + err = writer.Write(v) + if err != nil { + log.Warn("compact wrong, failed to writer row", zap.Error(err)) + return nil, err + } + unflushedRowCount++ + remainingRowCount++ + + if (unflushedRowCount+1)%100 == 0 && writer.IsFull() { + serWriteStart := time.Now() + kvs, partialBinlogs, err := t.serializeWrite(ctx, writer) + if err != nil { + log.Warn("compact wrong, failed to serialize writer", zap.Error(err)) + return nil, err + } + serWriteTimeCost += time.Since(serWriteStart) + + uploadStart := time.Now() + if err := t.binlogIO.Upload(ctx, kvs); err != nil { + log.Warn("compact wrong, failed to upload kvs", zap.Error(err)) + } + uploadTimeCost += time.Since(uploadStart) + mergeFieldBinlogs(allBinlogs, partialBinlogs) + syncBatchCount++ + unflushedRowCount = 0 + } + } + } + + if !writer.IsEmpty() { + serWriteStart := time.Now() + kvs, partialBinlogs, err := t.serializeWrite(ctx, writer) + if err != nil { + log.Warn("compact wrong, failed to serialize writer", zap.Error(err)) + return nil, err + } + serWriteTimeCost += time.Since(serWriteStart) + + uploadStart := time.Now() + if err := t.binlogIO.Upload(ctx, kvs); err != nil { + log.Warn("compact wrong, failed to upload kvs", zap.Error(err)) + } + uploadTimeCost += time.Since(uploadStart) + + mergeFieldBinlogs(allBinlogs, partialBinlogs) + syncBatchCount++ + } + + serWriteStart := time.Now() + sPath, err := t.statSerializeWrite(ctx, writer, remainingRowCount) + if err != nil { + log.Warn("compact wrong, failed to serialize write segment stats", + zap.Int64("remaining row count", remainingRowCount), zap.Error(err)) + return nil, err + } + serWriteTimeCost += time.Since(serWriteStart) + + pack := &datapb.CompactionSegment{ + SegmentID: writer.GetSegmentID(), + InsertLogs: lo.Values(allBinlogs), + Field2StatslogPaths: []*datapb.FieldBinlog{sPath}, + NumOfRows: remainingRowCount, + Channel: t.plan.GetChannel(), + } + + totalElapse := t.tr.RecordSpan() + + log.Info("compact merge end", + zap.Int64("remaining row count", remainingRowCount), + zap.Int64("expired entities", expiredRowCount), + zap.Int("binlog batch count", syncBatchCount), + zap.Duration("download binlogs elapse", downloadTimeCost), + zap.Duration("upload binlogs elapse", uploadTimeCost), + zap.Duration("serWrite elapse", serWriteTimeCost), + zap.Duration("deRead elapse", totalElapse-serWriteTimeCost-downloadTimeCost-uploadTimeCost), + zap.Duration("total elapse", totalElapse)) + + return pack, nil +} + +func mergeFieldBinlogs(base, paths map[typeutil.UniqueID]*datapb.FieldBinlog) { + for fID, fpath := range paths { + if _, ok := base[fID]; !ok { + base[fID] = &datapb.FieldBinlog{FieldID: fID, Binlogs: make([]*datapb.Binlog, 0)} + } + base[fID].Binlogs = append(base[fID].Binlogs, fpath.GetBinlogs()...) + } +} + +func (t *mixCompactionTask) Compact() (*datapb.CompactionPlanResult, error) { + durInQueue := t.tr.RecordSpan() + compactStart := time.Now() + ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, fmt.Sprintf("MixCompact-%d", t.GetPlanID())) + defer span.End() + + log := log.Ctx(ctx).With(zap.Int64("planID", t.plan.GetPlanID()), zap.Int32("timeout in seconds", t.plan.GetTimeoutInSeconds())) + if ok := funcutil.CheckCtxValid(ctx); !ok { + 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() + + log.Info("compact start") + if len(t.plan.GetSegmentBinlogs()) < 1 { + log.Warn("compact wrong, there's no segments in segment binlogs") + return nil, errors.New("compaction plan is illegal") + } + + targetSegID, err := t.AllocOne() + if err != nil { + log.Warn("compact wrong, unable to allocate segmentID", zap.Error(err)) + return nil, err + } + + previousRowCount, err := t.getNumRows() + if err != nil { + log.Warn("compact wrong, unable to get previous numRows", zap.Error(err)) + return nil, err + } + + partID := t.plan.GetSegmentBinlogs()[0].GetPartitionID() + + writer, err := NewSegmentWriter(t.metaCache.Schema(), previousRowCount, targetSegID, partID, t.metaCache.Collection()) + if err != nil { + log.Warn("compact wrong, unable to init segment writer", zap.Error(err)) + return nil, err + } + + segIDs := lo.Map(t.plan.GetSegmentBinlogs(), func(binlogs *datapb.CompactionSegmentBinlogs, _ int) int64 { + return binlogs.GetSegmentID() + }) + // Inject to stop flush + // when compaction failed, these segments need to be Unblocked by injectDone in compaction_executor + // when compaction succeeded, these segments will be Unblocked by SyncSegments from DataCoord. + for _, segID := range segIDs { + t.syncMgr.Block(segID) + } + + if err := binlog.DecompressCompactionBinlogs(t.plan.GetSegmentBinlogs()); err != nil { + log.Warn("compact wrong, fail to decompress compaction binlogs", zap.Error(err)) + return nil, err + } + + deltaPaths := make(map[typeutil.UniqueID][]string) // segmentID to deltalog paths + allPath := make([][]string, 0) // group by binlog batch + for _, s := range t.plan.GetSegmentBinlogs() { + // Get the batch count of field binlog files from non-empty segment + // each segment might contain different batches + var binlogBatchCount int + for _, b := range s.GetFieldBinlogs() { + if b != nil { + binlogBatchCount = len(b.GetBinlogs()) + break + } + } + if binlogBatchCount == 0 { + log.Warn("compacting empty segment", zap.Int64("segmentID", s.GetSegmentID())) + continue + } + + for idx := 0; idx < binlogBatchCount; idx++ { + var batchPaths []string + for _, f := range s.GetFieldBinlogs() { + batchPaths = append(batchPaths, f.GetBinlogs()[idx].GetLogPath()) + } + allPath = append(allPath, batchPaths) + } + + deltaPaths[s.GetSegmentID()] = []string{} + for _, d := range s.GetDeltalogs() { + for _, l := range d.GetBinlogs() { + deltaPaths[s.GetSegmentID()] = append(deltaPaths[s.GetSegmentID()], l.GetLogPath()) + } + } + } + + // Unable to deal with all empty segments cases, so return error + if len(allPath) == 0 { + log.Warn("compact wrong, all segments' binlogs are empty") + return nil, errors.New("illegal compaction plan") + } + + deltaPk2Ts, err := t.mergeDeltalogs(ctxTimeout, deltaPaths) + if err != nil { + log.Warn("compact wrong, fail to merge deltalogs", zap.Error(err)) + return nil, err + } + + compactToSeg, err := t.merge(ctxTimeout, allPath, deltaPk2Ts, writer) + if err != nil { + log.Warn("compact wrong, fail to merge", zap.Error(err)) + return nil, err + } + + log.Info("compact done", + zap.Int64("compact to segment", targetSegID), + zap.Int64s("compact from segments", segIDs), + zap.Int("num of binlog paths", len(compactToSeg.GetInsertLogs())), + zap.Int("num of stats paths", 1), + zap.Int("num of delta paths", len(compactToSeg.GetDeltalogs())), + zap.Duration("compact elapse", time.Since(compactStart)), + ) + + metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.plan.GetType().String()).Observe(float64(t.tr.ElapseSpan().Milliseconds())) + metrics.DataNodeCompactionLatencyInQueue.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(durInQueue.Milliseconds())) + + planResult := &datapb.CompactionPlanResult{ + State: commonpb.CompactionState_Completed, + PlanID: t.GetPlanID(), + Channel: t.GetChannelName(), + Segments: []*datapb.CompactionSegment{compactToSeg}, + Type: t.plan.GetType(), + } + + return planResult, nil +} + +func (t *mixCompactionTask) InjectDone() { + t.injectDoneOnce.Do(func() { + for _, binlog := range t.plan.SegmentBinlogs { + t.syncMgr.Unblock(binlog.SegmentID) + } + }) +} + +func (t *mixCompactionTask) GetCollection() typeutil.UniqueID { + return t.metaCache.Collection() +} + +func (t *mixCompactionTask) isExpiredEntity(ts typeutil.Timestamp) bool { + now := t.currentTs + + // entity expire is not enabled if duration <= 0 + if t.plan.GetCollectionTtl() <= 0 { + return false + } + + entityT, _ := tsoutil.ParseTS(ts) + nowT, _ := tsoutil.ParseTS(now) + + return entityT.Add(time.Duration(t.plan.GetCollectionTtl())).Before(nowT) +} diff --git a/internal/datanode/compaction/mix_compactor_test.go b/internal/datanode/compaction/mix_compactor_test.go new file mode 100644 index 0000000000000..f308d1ab050da --- /dev/null +++ b/internal/datanode/compaction/mix_compactor_test.go @@ -0,0 +1,805 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package compaction + +import ( + "context" + "math" + "testing" + "time" + + "github.com/cockroachdb/errors" + "github.com/samber/lo" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/suite" + + "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/internal/datanode/allocator" + "github.com/milvus-io/milvus/internal/datanode/io" + "github.com/milvus-io/milvus/internal/datanode/metacache" + "github.com/milvus-io/milvus/internal/datanode/syncmgr" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/etcdpb" + "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" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +var compactTestDir = "/tmp/milvus_test/compact" + +func TestMixCompactionTaskSuite(t *testing.T) { + suite.Run(t, new(MixCompactionTaskSuite)) +} + +type MixCompactionTaskSuite struct { + suite.Suite + + mockBinlogIO *io.MockBinlogIO + mockAlloc *allocator.MockAllocator + mockMeta *metacache.MockMetaCache + mockSyncMgr *syncmgr.MockSyncManager + + meta *etcdpb.CollectionMeta + segWriter *SegmentWriter + + task *mixCompactionTask + plan *datapb.CompactionPlan +} + +func (s *MixCompactionTaskSuite) SetupSuite() { + paramtable.Get().Init(paramtable.NewBaseTable()) +} + +func (s *MixCompactionTaskSuite) SetupTest() { + s.mockBinlogIO = io.NewMockBinlogIO(s.T()) + s.mockAlloc = allocator.NewMockAllocator(s.T()) + s.mockMeta = metacache.NewMockMetaCache(s.T()) + s.mockSyncMgr = syncmgr.NewMockSyncManager(s.T()) + + s.task = NewMixCompactionTask(context.Background(), s.mockBinlogIO, s.mockMeta, s.mockSyncMgr, s.mockAlloc, nil) + + s.meta = genTestCollectionMeta() + + paramtable.Get().Save(paramtable.Get().CommonCfg.EntityExpirationTTL.Key, "0") + + s.plan = &datapb.CompactionPlan{ + PlanID: 999, + SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{{ + SegmentID: 100, + FieldBinlogs: nil, + Field2StatslogPaths: nil, + Deltalogs: nil, + }}, + TimeoutInSeconds: 10, + Type: datapb.CompactionType_MixCompaction, + } + s.task.plan = s.plan +} + +func (s *MixCompactionTaskSuite) SetupSubTest() { + s.SetupTest() +} + +func (s *MixCompactionTaskSuite) TearDownTest() { + paramtable.Get().Reset(paramtable.Get().CommonCfg.EntityExpirationTTL.Key) +} + +func getMilvusBirthday() time.Time { + return time.Date(2019, time.Month(5), 30, 0, 0, 0, 0, time.UTC) +} + +func (s *MixCompactionTaskSuite) TestInjectDone() { + segmentIDs := []int64{100, 200, 300} + s.task.plan.SegmentBinlogs = lo.Map(segmentIDs, func(id int64, _ int) *datapb.CompactionSegmentBinlogs { + return &datapb.CompactionSegmentBinlogs{SegmentID: id} + }) + + for _, segmentID := range segmentIDs { + s.mockSyncMgr.EXPECT().Unblock(segmentID).Return().Once() + } + + s.task.InjectDone() + s.task.InjectDone() +} + +func (s *MixCompactionTaskSuite) TestCompactDupPK() { + // Test merge compactions, two segments with the same pk, one deletion pk=1 + // The merged segment 19530 should remain 3 pk without pk=100 + s.mockAlloc.EXPECT().AllocOne().Return(int64(19530), nil).Twice() + s.mockMeta.EXPECT().Schema().Return(s.meta.GetSchema()).Once() + s.mockMeta.EXPECT().Collection().Return(CollectionID).Once() + segments := []int64{7, 8, 9} + dblobs, err := getInt64DeltaBlobs( + 1, + []int64{100}, + []uint64{tsoutil.ComposeTSByTime(getMilvusBirthday().Add(time.Second), 0)}, + ) + s.Require().NoError(err) + + s.mockBinlogIO.EXPECT().Download(mock.Anything, []string{"1"}). + Return([][]byte{dblobs.GetValue()}, nil).Times(3) + s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(7777777, 8888888, nil) + s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil) + + // clear origial segments + s.task.plan.SegmentBinlogs = make([]*datapb.CompactionSegmentBinlogs, 0) + for _, segID := range segments { + s.initSegBuffer(segID) + row := getRow(100) + v := &storage.Value{ + PK: storage.NewInt64PrimaryKey(100), + Timestamp: int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)), + Value: row, + } + err := s.segWriter.Write(v) + s.Require().NoError(err) + + err = s.segWriter.writer.Flush() + s.Require().NoError(err) + + statistic := &storage.PkStatistics{ + PkFilter: s.segWriter.pkstats.BF, + MinPK: s.segWriter.pkstats.MinPk, + MaxPK: s.segWriter.pkstats.MaxPk, + } + bfs := metacache.NewBloomFilterSet(statistic) + + kvs, fBinlogs, err := s.task.serializeWrite(context.TODO(), s.segWriter) + s.Require().NoError(err) + s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.MatchedBy(func(keys []string) bool { + left, right := lo.Difference(keys, lo.Keys(kvs)) + return len(left) == 0 && len(right) == 0 + })).Return(lo.Values(kvs), nil).Once() + + seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{ + CollectionID: CollectionID, + PartitionID: PartitionID, + ID: segID, + NumOfRows: 1, + }, bfs) + + s.mockMeta.EXPECT().GetSegmentByID(segID).RunAndReturn(func(id int64, filters ...metacache.SegmentFilter) (*metacache.SegmentInfo, bool) { + return seg, true + }) + s.mockSyncMgr.EXPECT().Block(segID).Return().Once() + + s.plan.SegmentBinlogs = append(s.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{ + SegmentID: segID, + FieldBinlogs: lo.Values(fBinlogs), + Deltalogs: []*datapb.FieldBinlog{ + {Binlogs: []*datapb.Binlog{{LogID: 1, LogPath: "1"}}}, + }, + }) + } + result, err := s.task.Compact() + s.NoError(err) + s.NotNil(result) + + s.Equal(s.task.plan.GetPlanID(), result.GetPlanID()) + s.Equal(1, len(result.GetSegments())) + + segment := result.GetSegments()[0] + s.EqualValues(19530, segment.GetSegmentID()) + s.EqualValues(3, segment.GetNumOfRows()) + s.NotEmpty(segment.InsertLogs) + s.NotEmpty(segment.Field2StatslogPaths) + s.Empty(segment.Deltalogs) +} + +func (s *MixCompactionTaskSuite) TestCompactTwoToOne() { + s.mockAlloc.EXPECT().AllocOne().Return(int64(19530), nil).Twice() + s.mockMeta.EXPECT().Schema().Return(s.meta.GetSchema()).Once() + s.mockMeta.EXPECT().Collection().Return(CollectionID).Once() + + segments := []int64{5, 6, 7} + s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(7777777, 8888888, nil) + s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil) + s.task.plan.SegmentBinlogs = make([]*datapb.CompactionSegmentBinlogs, 0) + for _, segID := range segments { + s.initSegBuffer(segID) + statistic := &storage.PkStatistics{ + PkFilter: s.segWriter.pkstats.BF, + MinPK: s.segWriter.pkstats.MinPk, + MaxPK: s.segWriter.pkstats.MaxPk, + } + bfs := metacache.NewBloomFilterSet(statistic) + kvs, fBinlogs, err := s.task.serializeWrite(context.TODO(), s.segWriter) + s.Require().NoError(err) + s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.MatchedBy(func(keys []string) bool { + left, right := lo.Difference(keys, lo.Keys(kvs)) + return len(left) == 0 && len(right) == 0 + })).Return(lo.Values(kvs), nil).Once() + + seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{ + CollectionID: CollectionID, + PartitionID: PartitionID, + ID: segID, + NumOfRows: 1, + }, bfs) + + s.mockMeta.EXPECT().GetSegmentByID(segID).RunAndReturn(func(id int64, filters ...metacache.SegmentFilter) (*metacache.SegmentInfo, bool) { + return seg, true + }) + s.mockSyncMgr.EXPECT().Block(segID).Return().Once() + + s.plan.SegmentBinlogs = append(s.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{ + SegmentID: segID, + FieldBinlogs: lo.Values(fBinlogs), + }) + } + + // append an empty segment + seg := metacache.NewSegmentInfo(&datapb.SegmentInfo{ + CollectionID: CollectionID, + PartitionID: PartitionID, + ID: 99999, + NumOfRows: 0, + }, metacache.NewBloomFilterSet()) + s.mockMeta.EXPECT().GetSegmentByID(seg.SegmentID()).RunAndReturn(func(id int64, filters ...metacache.SegmentFilter) (*metacache.SegmentInfo, bool) { + return seg, true + }) + s.mockSyncMgr.EXPECT().Block(seg.SegmentID()).Return().Once() + s.plan.SegmentBinlogs = append(s.plan.SegmentBinlogs, &datapb.CompactionSegmentBinlogs{ + SegmentID: seg.SegmentID(), + }) + + result, err := s.task.Compact() + s.NoError(err) + s.NotNil(result) + + s.Equal(s.task.plan.GetPlanID(), result.GetPlanID()) + s.Equal(1, len(result.GetSegments())) + + segment := result.GetSegments()[0] + s.EqualValues(19530, segment.GetSegmentID()) + s.EqualValues(3, segment.GetNumOfRows()) + s.NotEmpty(segment.InsertLogs) + s.NotEmpty(segment.Field2StatslogPaths) + s.Empty(segment.Deltalogs) +} + +func (s *MixCompactionTaskSuite) TestMergeBufferFull() { + paramtable.Get().Save(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key, "1") + defer paramtable.Get().Reset(paramtable.Get().DataNodeCfg.BinLogMaxSize.Key) + + s.initSegBuffer(5) + v := storage.Value{ + PK: storage.NewInt64PrimaryKey(100), + Timestamp: int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)), + Value: getRow(100), + } + err := s.segWriter.Write(&v) + s.Require().NoError(err) + + s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(888888, 999999, nil).Times(2) + kvs, _, err := s.task.serializeWrite(context.TODO(), s.segWriter) + s.Require().NoError(err) + + s.mockAlloc.EXPECT().AllocOne().Return(888888, nil) + s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).RunAndReturn( + func(ctx context.Context, paths []string) ([][]byte, error) { + s.Require().Equal(len(paths), len(kvs)) + return lo.Values(kvs), nil + }) + s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Maybe() + + segWriter, err := NewSegmentWriter(s.meta.GetSchema(), 100, 19530, PartitionID, CollectionID) + s.Require().NoError(err) + + compactionSegment, err := s.task.merge(s.task.ctx, [][]string{lo.Keys(kvs)}, nil, segWriter) + s.NoError(err) + s.NotNil(compactionSegment) + s.EqualValues(2, compactionSegment.GetNumOfRows()) +} + +func (s *MixCompactionTaskSuite) TestMergeEntityExpired() { + s.initSegBuffer(3) + // entityTs == tsoutil.ComposeTSByTime(milvusBirthday, 0) + collTTL := 864000 // 10 days + currTs := tsoutil.ComposeTSByTime(getMilvusBirthday().Add(time.Second*(time.Duration(collTTL)+1)), 0) + s.task.currentTs = currTs + s.task.plan.CollectionTtl = int64(collTTL) + s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(888888, 999999, nil) + + kvs, _, err := s.task.serializeWrite(context.TODO(), s.segWriter) + s.Require().NoError(err) + s.mockAlloc.EXPECT().AllocOne().Return(888888, nil) + s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).RunAndReturn( + func(ctx context.Context, paths []string) ([][]byte, error) { + s.Require().Equal(len(paths), len(kvs)) + return lo.Values(kvs), nil + }) + s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Maybe() + + segWriter, err := NewSegmentWriter(s.meta.GetSchema(), 100, 19530, PartitionID, CollectionID) + s.Require().NoError(err) + + compactionSegment, err := s.task.merge(s.task.ctx, [][]string{lo.Keys(kvs)}, nil, segWriter) + s.NoError(err) + s.NotNil(compactionSegment) + s.EqualValues(0, compactionSegment.GetNumOfRows()) +} + +func (s *MixCompactionTaskSuite) TestMergeNoExpiration() { + s.initSegBuffer(4) + deleteTs := tsoutil.ComposeTSByTime(getMilvusBirthday().Add(10*time.Second), 0) + tests := []struct { + description string + deletions map[interface{}]uint64 + expectedRowCount int + }{ + {"no deletion", nil, 1}, + {"mismatch deletion", map[interface{}]uint64{int64(1): deleteTs}, 1}, + {"deleted pk=4", map[interface{}]uint64{int64(4): deleteTs}, 0}, + } + + s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(888888, 999999, nil) + kvs, _, err := s.task.serializeWrite(context.TODO(), s.segWriter) + s.Require().NoError(err) + for _, test := range tests { + s.Run(test.description, func() { + if test.expectedRowCount > 0 { + s.mockAlloc.EXPECT().Alloc(mock.Anything).Return(77777, 99999, nil).Once() + } + s.mockAlloc.EXPECT().AllocOne().Return(888888, nil) + s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).RunAndReturn( + func(ctx context.Context, paths []string) ([][]byte, error) { + s.Require().Equal(len(paths), len(kvs)) + return lo.Values(kvs), nil + }) + s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Maybe() + + segWriter, err := NewSegmentWriter(s.meta.GetSchema(), 100, 19530, PartitionID, CollectionID) + s.Require().NoError(err) + + compactionSegment, err := s.task.merge(s.task.ctx, [][]string{lo.Keys(kvs)}, test.deletions, segWriter) + s.NoError(err) + s.NotNil(compactionSegment) + s.EqualValues(test.expectedRowCount, compactionSegment.GetNumOfRows()) + }) + } +} + +func (s *MixCompactionTaskSuite) TestMergeDeltalogsMultiSegment() { + tests := []struct { + segIDA int64 + dataApk []int64 + dataAts []uint64 + + segIDB int64 + dataBpk []int64 + dataBts []uint64 + + segIDC int64 + dataCpk []int64 + dataCts []uint64 + + expectedpk2ts map[int64]uint64 + description string + }{ + { + 0, nil, nil, + 100, + []int64{1, 2, 3}, + []uint64{20000, 30000, 20005}, + 200, + []int64{4, 5, 6}, + []uint64{50000, 50001, 50002}, + map[int64]uint64{ + 1: 20000, + 2: 30000, + 3: 20005, + 4: 50000, + 5: 50001, + 6: 50002, + }, + "2 segments", + }, + { + 300, + []int64{10, 20}, + []uint64{20001, 40001}, + 100, + []int64{1, 2, 3}, + []uint64{20000, 30000, 20005}, + 200, + []int64{4, 5, 6}, + []uint64{50000, 50001, 50002}, + map[int64]uint64{ + 10: 20001, + 20: 40001, + 1: 20000, + 2: 30000, + 3: 20005, + 4: 50000, + 5: 50001, + 6: 50002, + }, + "3 segments", + }, + } + + for _, test := range tests { + s.Run(test.description, func() { + dValues := make([][]byte, 0) + if test.dataApk != nil { + d, err := getInt64DeltaBlobs(test.segIDA, test.dataApk, test.dataAts) + s.Require().NoError(err) + dValues = append(dValues, d.GetValue()) + } + if test.dataBpk != nil { + d, err := getInt64DeltaBlobs(test.segIDB, test.dataBpk, test.dataBts) + s.Require().NoError(err) + dValues = append(dValues, d.GetValue()) + } + if test.dataCpk != nil { + d, err := getInt64DeltaBlobs(test.segIDC, test.dataCpk, test.dataCts) + s.Require().NoError(err) + dValues = append(dValues, d.GetValue()) + } + + s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything). + Return(dValues, nil) + + got, err := s.task.mergeDeltalogs(s.task.ctx, map[int64][]string{100: {"random"}}) + s.NoError(err) + + s.Equal(len(test.expectedpk2ts), len(got)) + gotKeys := lo.Map(lo.Keys(got), func(k interface{}, _ int) int64 { + res, ok := k.(int64) + s.Require().True(ok) + return res + }) + s.ElementsMatch(gotKeys, lo.Keys(test.expectedpk2ts)) + s.ElementsMatch(lo.Values(got), lo.Values(test.expectedpk2ts)) + }) + } +} + +func (s *MixCompactionTaskSuite) TestMergeDeltalogsOneSegment() { + blob, err := getInt64DeltaBlobs( + 100, + []int64{1, 2, 3, 4, 5, 1, 2}, + []uint64{20000, 20001, 20002, 30000, 50000, 50000, 10000}, + ) + s.Require().NoError(err) + + expectedMap := map[int64]uint64{1: 50000, 2: 20001, 3: 20002, 4: 30000, 5: 50000} + + s.mockBinlogIO.EXPECT().Download(mock.Anything, []string{"a"}). + Return([][]byte{blob.GetValue()}, nil).Once() + s.mockBinlogIO.EXPECT().Download(mock.Anything, []string{"mock_error"}). + Return(nil, errors.New("mock_error")).Once() + + invalidPaths := map[int64][]string{2000: {"mock_error"}} + got, err := s.task.mergeDeltalogs(s.task.ctx, invalidPaths) + s.Error(err) + s.Nil(got) + + dpaths := map[int64][]string{1000: {"a"}} + got, err = s.task.mergeDeltalogs(s.task.ctx, dpaths) + s.NoError(err) + s.NotNil(got) + s.Equal(len(expectedMap), len(got)) + + gotKeys := lo.Map(lo.Keys(got), func(k interface{}, _ int) int64 { + res, ok := k.(int64) + s.Require().True(ok) + return res + }) + s.ElementsMatch(gotKeys, lo.Keys(expectedMap)) + s.ElementsMatch(lo.Values(got), lo.Values(expectedMap)) +} + +func (s *MixCompactionTaskSuite) TestCompactFail() { + s.Run("mock ctx done", func() { + ctx, cancel := context.WithCancel(context.Background()) + cancel() + + s.task.ctx = ctx + s.task.cancel = cancel + _, err := s.task.Compact() + s.Error(err) + s.ErrorIs(err, context.Canceled) + }) + + s.Run("Test compact invalid empty segment binlogs", func() { + s.plan.SegmentBinlogs = nil + + _, err := s.task.Compact() + s.Error(err) + }) + + s.Run("Test compact AllocOnce failed", func() { + s.mockAlloc.EXPECT().AllocOne().Return(0, errors.New("mock alloc one error")).Once() + _, err := s.task.Compact() + s.Error(err) + }) + + s.Run("Test getNumRows error", func() { + s.mockAlloc.EXPECT().AllocOne().Return(19530, nil).Once() + s.mockMeta.EXPECT().GetSegmentByID(mock.Anything).Return(nil, false) + + _, err := s.task.Compact() + s.Error(err) + s.ErrorIs(err, merr.ErrSegmentNotFound) + }) +} + +func (s *MixCompactionTaskSuite) TestIsExpiredEntity() { + milvusBirthdayTs := tsoutil.ComposeTSByTime(getMilvusBirthday(), 0) + + tests := []struct { + description string + collTTL int64 + nowTs uint64 + entityTs uint64 + + expect bool + }{ + {"ttl=maxInt64, nowTs-entityTs=ttl", math.MaxInt64, math.MaxInt64, 0, true}, + {"ttl=maxInt64, nowTs-entityTs < 0", math.MaxInt64, milvusBirthdayTs, 0, false}, + {"ttl=maxInt64, 0ttl v2", math.MaxInt64, math.MaxInt64, milvusBirthdayTs, true}, + // entityTs==currTs will never happen + // {"ttl=maxInt64, curTs-entityTs=0", math.MaxInt64, milvusBirthdayTs, milvusBirthdayTs, true}, + {"ttl=0, nowTs>entityTs", 0, milvusBirthdayTs + 1, milvusBirthdayTs, false}, + {"ttl=0, nowTs==entityTs", 0, milvusBirthdayTs, milvusBirthdayTs, false}, + {"ttl=0, nowTs10days", 864000, milvusBirthdayTs + 864001, milvusBirthdayTs, true}, + {"ttl=10days, nowTs-entityTs==10days", 864000, milvusBirthdayTs + 864000, milvusBirthdayTs, true}, + {"ttl=10days, nowTs-entityTs<10days", 864000, milvusBirthdayTs + 10, milvusBirthdayTs, false}, + } + for _, test := range tests { + s.Run(test.description, func() { + t := &mixCompactionTask{ + plan: &datapb.CompactionPlan{ + CollectionTtl: test.collTTL, + }, + currentTs: test.nowTs, + } + got := t.isExpiredEntity(test.entityTs) + s.Equal(test.expect, got) + }) + } +} + +func getRow(magic int64) map[int64]interface{} { + ts := tsoutil.ComposeTSByTime(getMilvusBirthday(), 0) + return map[int64]interface{}{ + common.RowIDField: magic, + common.TimeStampField: int64(ts), // should be int64 here + BoolField: true, + Int8Field: int8(magic), + Int16Field: int16(magic), + Int32Field: int32(magic), + Int64Field: magic, + FloatField: float32(magic), + DoubleField: float64(magic), + StringField: "str", + VarCharField: "varchar", + BinaryVectorField: []byte{0}, + FloatVectorField: []float32{4, 5, 6, 7}, + Float16VectorField: []byte{0, 0, 0, 0, 255, 255, 255, 255}, + BFloat16VectorField: []byte{0, 0, 0, 0, 255, 255, 255, 255}, + SparseFloatVectorField: typeutil.CreateSparseFloatRow([]uint32{0, 1, 2}, []float32{4, 5, 6}), + ArrayField: &schemapb.ScalarField{ + Data: &schemapb.ScalarField_IntData{ + IntData: &schemapb.IntArray{Data: []int32{1, 2, 3}}, + }, + }, + JSONField: []byte(`{"batch":ok}`), + } +} + +func (s *MixCompactionTaskSuite) initSegBuffer(magic int64) { + segWriter, err := NewSegmentWriter(s.meta.GetSchema(), 100, magic, PartitionID, CollectionID) + s.Require().NoError(err) + + v := storage.Value{ + PK: storage.NewInt64PrimaryKey(magic), + Timestamp: int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)), + Value: getRow(magic), + } + err = segWriter.Write(&v) + s.Require().NoError(err) + segWriter.writer.Flush() + + s.segWriter = segWriter +} + +const ( + CollectionID = 1 + PartitionID = 1 + SegmentID = 1 + BoolField = 100 + Int8Field = 101 + Int16Field = 102 + Int32Field = 103 + Int64Field = 104 + FloatField = 105 + DoubleField = 106 + StringField = 107 + BinaryVectorField = 108 + FloatVectorField = 109 + ArrayField = 110 + JSONField = 111 + Float16VectorField = 112 + BFloat16VectorField = 113 + SparseFloatVectorField = 114 + VarCharField = 115 +) + +func getInt64DeltaBlobs(segID int64, pks []int64, tss []uint64) (*storage.Blob, error) { + primaryKeys := make([]storage.PrimaryKey, len(pks)) + for index, v := range pks { + primaryKeys[index] = storage.NewInt64PrimaryKey(v) + } + deltaData := storage.NewDeleteData(primaryKeys, tss) + + dCodec := storage.NewDeleteCodec() + blob, err := dCodec.Serialize(1, 10, segID, deltaData) + return blob, err +} + +func genTestCollectionMeta() *etcdpb.CollectionMeta { + return &etcdpb.CollectionMeta{ + ID: CollectionID, + PartitionTags: []string{"partition_0", "partition_1"}, + Schema: &schemapb.CollectionSchema{ + Name: "schema", + Description: "schema", + AutoID: true, + Fields: []*schemapb.FieldSchema{ + { + FieldID: common.RowIDField, + Name: "row_id", + DataType: schemapb.DataType_Int64, + }, + { + FieldID: common.TimeStampField, + Name: "Timestamp", + DataType: schemapb.DataType_Int64, + }, + { + FieldID: BoolField, + Name: "field_bool", + DataType: schemapb.DataType_Bool, + }, + { + FieldID: Int8Field, + Name: "field_int8", + DataType: schemapb.DataType_Int8, + }, + { + FieldID: Int16Field, + Name: "field_int16", + DataType: schemapb.DataType_Int16, + }, + { + FieldID: Int32Field, + Name: "field_int32", + DataType: schemapb.DataType_Int32, + }, + { + FieldID: Int64Field, + Name: "field_int64", + IsPrimaryKey: true, + DataType: schemapb.DataType_Int64, + }, + { + FieldID: FloatField, + Name: "field_float", + DataType: schemapb.DataType_Float, + }, + { + FieldID: DoubleField, + Name: "field_double", + DataType: schemapb.DataType_Double, + }, + { + FieldID: StringField, + Name: "field_string", + DataType: schemapb.DataType_String, + }, + { + FieldID: VarCharField, + Name: "field_varchar", + DataType: schemapb.DataType_VarChar, + TypeParams: []*commonpb.KeyValuePair{ + { + Key: common.MaxLengthKey, + Value: "128", + }, + }, + }, + { + FieldID: ArrayField, + Name: "field_int32_array", + Description: "int32 array", + DataType: schemapb.DataType_Array, + ElementType: schemapb.DataType_Int32, + }, + { + FieldID: JSONField, + Name: "field_json", + Description: "json", + DataType: schemapb.DataType_JSON, + }, + { + FieldID: BinaryVectorField, + Name: "field_binary_vector", + Description: "binary_vector", + DataType: schemapb.DataType_BinaryVector, + TypeParams: []*commonpb.KeyValuePair{ + { + Key: common.DimKey, + Value: "8", + }, + }, + }, + { + FieldID: FloatVectorField, + Name: "field_float_vector", + Description: "float_vector", + DataType: schemapb.DataType_FloatVector, + TypeParams: []*commonpb.KeyValuePair{ + { + Key: common.DimKey, + Value: "4", + }, + }, + }, + { + FieldID: Float16VectorField, + Name: "field_float16_vector", + Description: "float16_vector", + DataType: schemapb.DataType_Float16Vector, + TypeParams: []*commonpb.KeyValuePair{ + { + Key: common.DimKey, + Value: "4", + }, + }, + }, + { + FieldID: BFloat16VectorField, + Name: "field_bfloat16_vector", + Description: "bfloat16_vector", + DataType: schemapb.DataType_BFloat16Vector, + TypeParams: []*commonpb.KeyValuePair{ + { + Key: common.DimKey, + Value: "4", + }, + }, + }, + { + FieldID: SparseFloatVectorField, + Name: "field_sparse_float_vector", + Description: "sparse_float_vector", + DataType: schemapb.DataType_SparseFloatVector, + TypeParams: []*commonpb.KeyValuePair{}, + }, + }, + }, + } +} diff --git a/internal/datanode/compaction/mock_compactor.go b/internal/datanode/compaction/mock_compactor.go new file mode 100644 index 0000000000000..99dccea0aa54c --- /dev/null +++ b/internal/datanode/compaction/mock_compactor.go @@ -0,0 +1,307 @@ +// Code generated by mockery v2.32.4. DO NOT EDIT. + +package compaction + +import ( + datapb "github.com/milvus-io/milvus/internal/proto/datapb" + mock "github.com/stretchr/testify/mock" +) + +// MockCompactor is an autogenerated mock type for the Compactor type +type MockCompactor struct { + mock.Mock +} + +type MockCompactor_Expecter struct { + mock *mock.Mock +} + +func (_m *MockCompactor) EXPECT() *MockCompactor_Expecter { + return &MockCompactor_Expecter{mock: &_m.Mock} +} + +// Compact provides a mock function with given fields: +func (_m *MockCompactor) Compact() (*datapb.CompactionPlanResult, error) { + ret := _m.Called() + + var r0 *datapb.CompactionPlanResult + var r1 error + if rf, ok := ret.Get(0).(func() (*datapb.CompactionPlanResult, error)); ok { + return rf() + } + if rf, ok := ret.Get(0).(func() *datapb.CompactionPlanResult); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(*datapb.CompactionPlanResult) + } + } + + if rf, ok := ret.Get(1).(func() error); ok { + r1 = rf() + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// MockCompactor_Compact_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Compact' +type MockCompactor_Compact_Call struct { + *mock.Call +} + +// Compact is a helper method to define mock.On call +func (_e *MockCompactor_Expecter) Compact() *MockCompactor_Compact_Call { + return &MockCompactor_Compact_Call{Call: _e.mock.On("Compact")} +} + +func (_c *MockCompactor_Compact_Call) Run(run func()) *MockCompactor_Compact_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockCompactor_Compact_Call) Return(_a0 *datapb.CompactionPlanResult, _a1 error) *MockCompactor_Compact_Call { + _c.Call.Return(_a0, _a1) + return _c +} + +func (_c *MockCompactor_Compact_Call) RunAndReturn(run func() (*datapb.CompactionPlanResult, error)) *MockCompactor_Compact_Call { + _c.Call.Return(run) + return _c +} + +// Complete provides a mock function with given fields: +func (_m *MockCompactor) Complete() { + _m.Called() +} + +// MockCompactor_Complete_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Complete' +type MockCompactor_Complete_Call struct { + *mock.Call +} + +// Complete is a helper method to define mock.On call +func (_e *MockCompactor_Expecter) Complete() *MockCompactor_Complete_Call { + return &MockCompactor_Complete_Call{Call: _e.mock.On("Complete")} +} + +func (_c *MockCompactor_Complete_Call) Run(run func()) *MockCompactor_Complete_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockCompactor_Complete_Call) Return() *MockCompactor_Complete_Call { + _c.Call.Return() + return _c +} + +func (_c *MockCompactor_Complete_Call) RunAndReturn(run func()) *MockCompactor_Complete_Call { + _c.Call.Return(run) + return _c +} + +// GetChannelName provides a mock function with given fields: +func (_m *MockCompactor) GetChannelName() string { + ret := _m.Called() + + var r0 string + if rf, ok := ret.Get(0).(func() string); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(string) + } + + return r0 +} + +// MockCompactor_GetChannelName_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetChannelName' +type MockCompactor_GetChannelName_Call struct { + *mock.Call +} + +// GetChannelName is a helper method to define mock.On call +func (_e *MockCompactor_Expecter) GetChannelName() *MockCompactor_GetChannelName_Call { + return &MockCompactor_GetChannelName_Call{Call: _e.mock.On("GetChannelName")} +} + +func (_c *MockCompactor_GetChannelName_Call) Run(run func()) *MockCompactor_GetChannelName_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockCompactor_GetChannelName_Call) Return(_a0 string) *MockCompactor_GetChannelName_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockCompactor_GetChannelName_Call) RunAndReturn(run func() string) *MockCompactor_GetChannelName_Call { + _c.Call.Return(run) + return _c +} + +// GetCollection provides a mock function with given fields: +func (_m *MockCompactor) GetCollection() int64 { + ret := _m.Called() + + var r0 int64 + if rf, ok := ret.Get(0).(func() int64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int64) + } + + return r0 +} + +// MockCompactor_GetCollection_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetCollection' +type MockCompactor_GetCollection_Call struct { + *mock.Call +} + +// GetCollection is a helper method to define mock.On call +func (_e *MockCompactor_Expecter) GetCollection() *MockCompactor_GetCollection_Call { + return &MockCompactor_GetCollection_Call{Call: _e.mock.On("GetCollection")} +} + +func (_c *MockCompactor_GetCollection_Call) Run(run func()) *MockCompactor_GetCollection_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockCompactor_GetCollection_Call) Return(_a0 int64) *MockCompactor_GetCollection_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockCompactor_GetCollection_Call) RunAndReturn(run func() int64) *MockCompactor_GetCollection_Call { + _c.Call.Return(run) + return _c +} + +// GetPlanID provides a mock function with given fields: +func (_m *MockCompactor) GetPlanID() int64 { + ret := _m.Called() + + var r0 int64 + if rf, ok := ret.Get(0).(func() int64); ok { + r0 = rf() + } else { + r0 = ret.Get(0).(int64) + } + + return r0 +} + +// MockCompactor_GetPlanID_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetPlanID' +type MockCompactor_GetPlanID_Call struct { + *mock.Call +} + +// GetPlanID is a helper method to define mock.On call +func (_e *MockCompactor_Expecter) GetPlanID() *MockCompactor_GetPlanID_Call { + return &MockCompactor_GetPlanID_Call{Call: _e.mock.On("GetPlanID")} +} + +func (_c *MockCompactor_GetPlanID_Call) Run(run func()) *MockCompactor_GetPlanID_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockCompactor_GetPlanID_Call) Return(_a0 int64) *MockCompactor_GetPlanID_Call { + _c.Call.Return(_a0) + return _c +} + +func (_c *MockCompactor_GetPlanID_Call) RunAndReturn(run func() int64) *MockCompactor_GetPlanID_Call { + _c.Call.Return(run) + return _c +} + +// InjectDone provides a mock function with given fields: +func (_m *MockCompactor) InjectDone() { + _m.Called() +} + +// MockCompactor_InjectDone_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'InjectDone' +type MockCompactor_InjectDone_Call struct { + *mock.Call +} + +// InjectDone is a helper method to define mock.On call +func (_e *MockCompactor_Expecter) InjectDone() *MockCompactor_InjectDone_Call { + return &MockCompactor_InjectDone_Call{Call: _e.mock.On("InjectDone")} +} + +func (_c *MockCompactor_InjectDone_Call) Run(run func()) *MockCompactor_InjectDone_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockCompactor_InjectDone_Call) Return() *MockCompactor_InjectDone_Call { + _c.Call.Return() + return _c +} + +func (_c *MockCompactor_InjectDone_Call) RunAndReturn(run func()) *MockCompactor_InjectDone_Call { + _c.Call.Return(run) + return _c +} + +// Stop provides a mock function with given fields: +func (_m *MockCompactor) Stop() { + _m.Called() +} + +// MockCompactor_Stop_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Stop' +type MockCompactor_Stop_Call struct { + *mock.Call +} + +// Stop is a helper method to define mock.On call +func (_e *MockCompactor_Expecter) Stop() *MockCompactor_Stop_Call { + return &MockCompactor_Stop_Call{Call: _e.mock.On("Stop")} +} + +func (_c *MockCompactor_Stop_Call) Run(run func()) *MockCompactor_Stop_Call { + _c.Call.Run(func(args mock.Arguments) { + run() + }) + return _c +} + +func (_c *MockCompactor_Stop_Call) Return() *MockCompactor_Stop_Call { + _c.Call.Return() + return _c +} + +func (_c *MockCompactor_Stop_Call) RunAndReturn(run func()) *MockCompactor_Stop_Call { + _c.Call.Return(run) + return _c +} + +// NewMockCompactor creates a new instance of MockCompactor. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations. +// The first argument is typically a *testing.T value. +func NewMockCompactor(t interface { + mock.TestingT + Cleanup(func()) +}) *MockCompactor { + mock := &MockCompactor{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/internal/datanode/compaction/segment_writer.go b/internal/datanode/compaction/segment_writer.go new file mode 100644 index 0000000000000..0628fc4d662e5 --- /dev/null +++ b/internal/datanode/compaction/segment_writer.go @@ -0,0 +1,237 @@ +// SegmentInsertBuffer can be reused to buffer all insert data of one segment +// buffer.Serialize will serialize the InsertBuffer and clear it +// pkstats keeps tracking pkstats of the segment until Finish + +package compaction + +import ( + "fmt" + "math" + + "go.uber.org/atomic" + + "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/datanode/writebuffer" + "github.com/milvus-io/milvus/internal/proto/etcdpb" + "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/paramtable" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +func NewSegmentDeltaWriter(segmentID, partitionID, collectionID int64) *SegmentDeltaWriter { + return &SegmentDeltaWriter{ + deleteData: &storage.DeleteData{}, + segmentID: segmentID, + partitionID: partitionID, + collectionID: collectionID, + tsFrom: math.MaxUint64, + tsTo: 0, + } +} + +type SegmentDeltaWriter struct { + deleteData *storage.DeleteData + segmentID int64 + partitionID int64 + collectionID int64 + + tsFrom typeutil.Timestamp + tsTo typeutil.Timestamp +} + +func (w *SegmentDeltaWriter) GetCollectionID() int64 { + return w.collectionID +} + +func (w *SegmentDeltaWriter) GetPartitionID() int64 { + return w.partitionID +} + +func (w *SegmentDeltaWriter) GetSegmentID() int64 { + return w.segmentID +} + +func (w *SegmentDeltaWriter) GetRowNum() int64 { + return w.deleteData.RowCount +} + +func (w *SegmentDeltaWriter) GetTimeRange() *writebuffer.TimeRange { + return writebuffer.NewTimeRange(w.tsFrom, w.tsTo) +} + +func (w *SegmentDeltaWriter) updateRange(ts typeutil.Timestamp) { + if ts < w.tsFrom { + w.tsFrom = ts + } + if ts > w.tsTo { + w.tsTo = ts + } +} + +func (w *SegmentDeltaWriter) Write(pk storage.PrimaryKey, ts typeutil.Timestamp) { + w.deleteData.Append(pk, ts) + w.updateRange(ts) +} + +func (w *SegmentDeltaWriter) WriteBatch(pks []storage.PrimaryKey, tss []typeutil.Timestamp) { + w.deleteData.AppendBatch(pks, tss) + + for _, ts := range tss { + w.updateRange(ts) + } +} + +func (w *SegmentDeltaWriter) Finish() (*storage.Blob, *writebuffer.TimeRange, error) { + blob, err := storage.NewDeleteCodec().Serialize(w.collectionID, w.partitionID, w.segmentID, w.deleteData) + if err != nil { + return nil, nil, err + } + + return blob, w.GetTimeRange(), nil +} + +type SegmentWriter struct { + writer *storage.SerializeWriter[*storage.Value] + closers []func() (*storage.Blob, error) + tsFrom typeutil.Timestamp + tsTo typeutil.Timestamp + + pkstats *storage.PrimaryKeyStats + segmentID int64 + partitionID int64 + collectionID int64 + sch *schemapb.CollectionSchema + rowCount *atomic.Int64 +} + +func (w *SegmentWriter) GetRowNum() int64 { + return w.rowCount.Load() +} + +func (w *SegmentWriter) GetCollectionID() int64 { + return w.collectionID +} + +func (w *SegmentWriter) GetPartitionID() int64 { + return w.partitionID +} + +func (w *SegmentWriter) GetSegmentID() int64 { + return w.segmentID +} + +func (w *SegmentWriter) GetPkID() int64 { + return w.pkstats.FieldID +} + +func (w *SegmentWriter) Write(v *storage.Value) error { + ts := typeutil.Timestamp(v.Timestamp) + if ts < w.tsFrom { + w.tsFrom = ts + } + if ts > w.tsTo { + w.tsTo = ts + } + + w.pkstats.Update(v.PK) + w.rowCount.Inc() + return w.writer.Write(v) +} + +func (w *SegmentWriter) Finish(actualRowCount int64) (*storage.Blob, error) { + w.writer.Flush() + codec := storage.NewInsertCodecWithSchema(&etcdpb.CollectionMeta{ID: w.collectionID, Schema: w.sch}) + return codec.SerializePkStats(w.pkstats, actualRowCount) +} + +func (w *SegmentWriter) IsFull() bool { + w.writer.Flush() + return w.writer.WrittenMemorySize() > paramtable.Get().DataNodeCfg.BinLogMaxSize.GetAsUint64() +} + +func (w *SegmentWriter) IsEmpty() bool { + w.writer.Flush() + return w.writer.WrittenMemorySize() == 0 +} + +func (w *SegmentWriter) GetTimeRange() *writebuffer.TimeRange { + return writebuffer.NewTimeRange(w.tsFrom, w.tsTo) +} + +func (w *SegmentWriter) SerializeYield() ([]*storage.Blob, *writebuffer.TimeRange, error) { + w.writer.Flush() + w.writer.Close() + + fieldData := make([]*storage.Blob, len(w.closers)) + for i, f := range w.closers { + blob, err := f() + if err != nil { + return nil, nil, err + } + fieldData[i] = blob + } + + tr := w.GetTimeRange() + w.clear() + + return fieldData, tr, nil +} + +func (w *SegmentWriter) clear() { + writer, closers, _ := newBinlogWriter(w.collectionID, w.partitionID, w.segmentID, w.sch) + w.writer = writer + w.closers = closers + w.tsFrom = math.MaxUint64 + w.tsTo = 0 +} + +func NewSegmentWriter(sch *schemapb.CollectionSchema, maxCount int64, segID, partID, collID int64) (*SegmentWriter, error) { + writer, closers, err := newBinlogWriter(collID, partID, segID, sch) + if err != nil { + return nil, err + } + + var pkField *schemapb.FieldSchema + for _, fs := range sch.GetFields() { + if fs.GetIsPrimaryKey() && fs.GetFieldID() >= 100 && typeutil.IsPrimaryFieldType(fs.GetDataType()) { + pkField = fs + } + } + if pkField == nil { + log.Warn("failed to get pk field from schema") + return nil, fmt.Errorf("no pk field in schema") + } + + stats, err := storage.NewPrimaryKeyStats(pkField.GetFieldID(), int64(pkField.GetDataType()), maxCount) + if err != nil { + return nil, err + } + + segWriter := SegmentWriter{ + writer: writer, + closers: closers, + tsFrom: math.MaxUint64, + tsTo: 0, + + pkstats: stats, + sch: sch, + segmentID: segID, + partitionID: partID, + collectionID: collID, + rowCount: atomic.NewInt64(0), + } + + return &segWriter, nil +} + +func newBinlogWriter(collID, partID, segID int64, schema *schemapb.CollectionSchema, +) (writer *storage.SerializeWriter[*storage.Value], closers []func() (*storage.Blob, error), err error) { + fieldWriters := storage.NewBinlogStreamWriters(collID, partID, segID, schema.Fields) + closers = make([]func() (*storage.Blob, error), 0, len(fieldWriters)) + for _, w := range fieldWriters { + closers = append(closers, w.Finalize) + } + writer, err = storage.NewBinlogSerializeWriter(schema, partID, segID, fieldWriters, 1024) + return +} diff --git a/internal/datanode/compaction_executor.go b/internal/datanode/compaction_executor.go index bbcfbbb8279d6..938d1b5db0fd7 100644 --- a/internal/datanode/compaction_executor.go +++ b/internal/datanode/compaction_executor.go @@ -24,6 +24,7 @@ import ( "go.uber.org/zap" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/pkg/log" "github.com/milvus-io/milvus/pkg/util/typeutil" @@ -34,10 +35,10 @@ const ( ) type compactionExecutor struct { - executing *typeutil.ConcurrentMap[int64, compactor] // planID to compactor - completedCompactor *typeutil.ConcurrentMap[int64, compactor] // planID to compactor + executing *typeutil.ConcurrentMap[int64, compaction.Compactor] // planID to compactor + completedCompactor *typeutil.ConcurrentMap[int64, compaction.Compactor] // planID to compactor completed *typeutil.ConcurrentMap[int64, *datapb.CompactionPlanResult] // planID to CompactionPlanResult - taskCh chan compactor + taskCh chan compaction.Compactor dropped *typeutil.ConcurrentSet[string] // vchannel dropped // To prevent concurrency of release channel and compaction get results @@ -47,39 +48,39 @@ type compactionExecutor struct { func newCompactionExecutor() *compactionExecutor { return &compactionExecutor{ - executing: typeutil.NewConcurrentMap[int64, compactor](), - completedCompactor: typeutil.NewConcurrentMap[int64, compactor](), + executing: typeutil.NewConcurrentMap[int64, compaction.Compactor](), + completedCompactor: typeutil.NewConcurrentMap[int64, compaction.Compactor](), completed: typeutil.NewConcurrentMap[int64, *datapb.CompactionPlanResult](), - taskCh: make(chan compactor, maxTaskNum), + taskCh: make(chan compaction.Compactor, maxTaskNum), dropped: typeutil.NewConcurrentSet[string](), } } -func (c *compactionExecutor) execute(task compactor) { +func (c *compactionExecutor) execute(task compaction.Compactor) { c.taskCh <- task c.toExecutingState(task) } -func (c *compactionExecutor) toExecutingState(task compactor) { - c.executing.Insert(task.getPlanID(), task) +func (c *compactionExecutor) toExecutingState(task compaction.Compactor) { + c.executing.Insert(task.GetPlanID(), task) } -func (c *compactionExecutor) toCompleteState(task compactor) { - task.complete() - c.executing.GetAndRemove(task.getPlanID()) +func (c *compactionExecutor) toCompleteState(task compaction.Compactor) { + task.Complete() + c.executing.GetAndRemove(task.GetPlanID()) } func (c *compactionExecutor) injectDone(planID UniqueID) { c.completed.GetAndRemove(planID) task, loaded := c.completedCompactor.GetAndRemove(planID) if loaded { - log.Info("Compaction task inject done", zap.Int64("planID", planID), zap.String("channel", task.getChannelName())) - task.injectDone() + log.Info("Compaction task inject done", zap.Int64("planID", planID), zap.String("channel", task.GetChannelName())) + task.InjectDone() } } // These two func are bounded for waitGroup -func (c *compactionExecutor) executeWithState(task compactor) { +func (c *compactionExecutor) executeWithState(task compaction.Compactor) { go c.executeTask(task) } @@ -94,11 +95,11 @@ func (c *compactionExecutor) start(ctx context.Context) { } } -func (c *compactionExecutor) executeTask(task compactor) { +func (c *compactionExecutor) executeTask(task compaction.Compactor) { log := log.With( - zap.Int64("planID", task.getPlanID()), - zap.Int64("Collection", task.getCollection()), - zap.String("channel", task.getChannelName()), + zap.Int64("planID", task.GetPlanID()), + zap.Int64("Collection", task.GetCollection()), + zap.String("channel", task.GetChannelName()), ) defer func() { @@ -107,23 +108,23 @@ func (c *compactionExecutor) executeTask(task compactor) { log.Info("start to execute compaction") - result, err := task.compact() + result, err := task.Compact() if err != nil { - task.injectDone() + task.InjectDone() log.Warn("compaction task failed", zap.Error(err)) } else { c.completed.Insert(result.GetPlanID(), result) c.completedCompactor.Insert(result.GetPlanID(), task) } - log.Info("end to execute compaction", zap.Int64("planID", task.getPlanID())) + log.Info("end to execute compaction") } func (c *compactionExecutor) stopTask(planID UniqueID) { task, loaded := c.executing.GetAndRemove(planID) if loaded { - log.Warn("compaction executor stop task", zap.Int64("planID", planID), zap.String("vChannelName", task.getChannelName())) - task.stop() + log.Warn("compaction executor stop task", zap.Int64("planID", planID), zap.String("vChannelName", task.GetChannelName())) + task.Stop() } } @@ -141,8 +142,8 @@ func (c *compactionExecutor) discardPlan(channel string) { c.resultGuard.Lock() defer c.resultGuard.Unlock() - c.executing.Range(func(planID int64, task compactor) bool { - if task.getChannelName() == channel { + c.executing.Range(func(planID int64, task compaction.Compactor) bool { + if task.GetChannelName() == channel { c.stopTask(planID) } return true @@ -170,7 +171,7 @@ func (c *compactionExecutor) getAllCompactionResults() []*datapb.CompactionPlanR ) results := make([]*datapb.CompactionPlanResult, 0) // get executing results - c.executing.Range(func(planID int64, task compactor) bool { + c.executing.Range(func(planID int64, task compaction.Compactor) bool { executing = append(executing, planID) results = append(results, &datapb.CompactionPlanResult{ State: commonpb.CompactionState_Executing, diff --git a/internal/datanode/compaction_executor_test.go b/internal/datanode/compaction_executor_test.go index 68eb61c531e57..fd6fba2e6f0a7 100644 --- a/internal/datanode/compaction_executor_test.go +++ b/internal/datanode/compaction_executor_test.go @@ -20,28 +20,29 @@ import ( "context" "testing" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" + "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/proto/datapb" ) func TestCompactionExecutor(t *testing.T) { t.Run("Test execute", func(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - ex := newCompactionExecutor() - go ex.start(ctx) - ex.execute(newMockCompactor(true)) - - cancel() - }) - - t.Run("Test stopTask", func(t *testing.T) { - ex := newCompactionExecutor() - mc := newMockCompactor(true) - ex.executeWithState(mc) - ex.stopTask(UniqueID(1)) + planID := int64(1) + mockC := compaction.NewMockCompactor(t) + mockC.EXPECT().GetPlanID().Return(planID).Once() + mockC.EXPECT().GetChannelName().Return("ch1").Once() + executor := newCompactionExecutor() + executor.execute(mockC) + + assert.EqualValues(t, 1, len(executor.taskCh)) + assert.EqualValues(t, 1, executor.executing.Len()) + + mockC.EXPECT().Stop().Return().Once() + executor.stopTask(planID) }) t.Run("Test start", func(t *testing.T) { @@ -57,19 +58,36 @@ func TestCompactionExecutor(t *testing.T) { description string }{ - {true, "compact return nil"}, + {true, "compact success"}, {false, "compact return error"}, } ex := newCompactionExecutor() for _, test := range tests { t.Run(test.description, func(t *testing.T) { + mockC := compaction.NewMockCompactor(t) + mockC.EXPECT().GetPlanID().Return(int64(1)) + mockC.EXPECT().GetCollection().Return(int64(1)) + mockC.EXPECT().GetChannelName().Return("ch1") + mockC.EXPECT().Complete().Return().Maybe() + signal := make(chan struct{}) if test.isvalid { - validTask := newMockCompactor(true) - ex.executeWithState(validTask) + mockC.EXPECT().Compact().RunAndReturn( + func() (*datapb.CompactionPlanResult, error) { + signal <- struct{}{} + return &datapb.CompactionPlanResult{PlanID: 1}, nil + }).Once() + ex.executeWithState(mockC) + <-signal } else { - invalidTask := newMockCompactor(false) - ex.executeWithState(invalidTask) + mockC.EXPECT().InjectDone().Return().Maybe() + mockC.EXPECT().Compact().RunAndReturn( + func() (*datapb.CompactionPlanResult, error) { + signal <- struct{}{} + return nil, errors.New("mock error") + }).Once() + ex.executeWithState(mockC) + <-signal } }) } @@ -95,33 +113,25 @@ func TestCompactionExecutor(t *testing.T) { t.Run("test stop vchannel tasks", func(t *testing.T) { ex := newCompactionExecutor() - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - go ex.start(ctx) - mc := newMockCompactor(true) - mc.alwaysWorking = true + mc := compaction.NewMockCompactor(t) + mc.EXPECT().GetPlanID().Return(int64(1)) + mc.EXPECT().GetChannelName().Return("mock") + mc.EXPECT().Compact().Return(&datapb.CompactionPlanResult{PlanID: 1}, nil).Maybe() + mc.EXPECT().Stop().Return().Once() ex.execute(mc) - // wait for task enqueued - found := false - for !found { - found = ex.executing.Contain(mc.getPlanID()) - } + require.True(t, ex.executing.Contain(int64(1))) ex.discardByDroppedChannel("mock") - - select { - case <-mc.ctx.Done(): - default: - t.FailNow() - } + assert.True(t, ex.dropped.Contain("mock")) + assert.False(t, ex.executing.Contain(int64(1))) }) t.Run("test getAllCompactionResults", func(t *testing.T) { ex := newCompactionExecutor() - mockC := newMockCompactor(true) + mockC := compaction.NewMockCompactor(t) ex.executing.Insert(int64(1), mockC) ex.completedCompactor.Insert(int64(2), mockC) @@ -158,60 +168,3 @@ func TestCompactionExecutor(t *testing.T) { require.Equal(t, 1, ex.executing.Len()) }) } - -func newMockCompactor(isvalid bool) *mockCompactor { - ctx, cancel := context.WithCancel(context.TODO()) - return &mockCompactor{ - ctx: ctx, - cancel: cancel, - isvalid: isvalid, - done: make(chan struct{}, 1), - } -} - -type mockCompactor struct { - ctx context.Context - cancel context.CancelFunc - isvalid bool - alwaysWorking bool - - done chan struct{} -} - -var _ compactor = (*mockCompactor)(nil) - -func (mc *mockCompactor) complete() { - mc.done <- struct{}{} -} - -func (mc *mockCompactor) injectDone() {} - -func (mc *mockCompactor) compact() (*datapb.CompactionPlanResult, error) { - if !mc.isvalid { - return nil, errStart - } - if mc.alwaysWorking { - <-mc.ctx.Done() - return nil, mc.ctx.Err() - } - return nil, nil -} - -func (mc *mockCompactor) getPlanID() UniqueID { - return 1 -} - -func (mc *mockCompactor) stop() { - if mc.cancel != nil { - mc.cancel() - <-mc.done - } -} - -func (mc *mockCompactor) getCollection() UniqueID { - return 1 -} - -func (mc *mockCompactor) getChannelName() string { - return "mock" -} diff --git a/internal/datanode/compactor.go b/internal/datanode/compactor.go deleted file mode 100644 index 44e292bc76da1..0000000000000 --- a/internal/datanode/compactor.go +++ /dev/null @@ -1,842 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package datanode - -import ( - "context" - "fmt" - sio "io" - "sync" - "time" - - "github.com/cockroachdb/errors" - "github.com/samber/lo" - "go.opentelemetry.io/otel" - "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/internal/datanode/allocator" - "github.com/milvus-io/milvus/internal/datanode/io" - "github.com/milvus-io/milvus/internal/datanode/metacache" - "github.com/milvus-io/milvus/internal/datanode/syncmgr" - "github.com/milvus-io/milvus/internal/metastore/kv/binlog" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/storage" - "github.com/milvus-io/milvus/pkg/log" - "github.com/milvus-io/milvus/pkg/metrics" - "github.com/milvus-io/milvus/pkg/util/funcutil" - "github.com/milvus-io/milvus/pkg/util/merr" - "github.com/milvus-io/milvus/pkg/util/paramtable" - "github.com/milvus-io/milvus/pkg/util/timerecord" - "github.com/milvus-io/milvus/pkg/util/tsoutil" - "github.com/milvus-io/milvus/pkg/util/typeutil" -) - -var ( - errIllegalCompactionPlan = errors.New("compaction plan illegal") - errTransferType = errors.New("transfer intferface to type wrong") - errUnknownDataType = errors.New("unknown shema DataType") - errContext = errors.New("context done or timeout") -) - -type iterator = storage.Iterator - -type compactor interface { - complete() - compact() (*datapb.CompactionPlanResult, error) - injectDone() - stop() - getPlanID() UniqueID - getCollection() UniqueID - getChannelName() string -} - -// make sure compactionTask implements compactor interface -var _ compactor = (*compactionTask)(nil) - -// for MixCompaction only -type compactionTask struct { - binlogIO io.BinlogIO - compactor - metaCache metacache.MetaCache - syncMgr syncmgr.SyncManager - allocator.Allocator - - plan *datapb.CompactionPlan - - ctx context.Context - cancel context.CancelFunc - - injectDoneOnce sync.Once - done chan struct{} - tr *timerecord.TimeRecorder -} - -func newCompactionTask( - ctx context.Context, - binlogIO io.BinlogIO, - metaCache metacache.MetaCache, - syncMgr syncmgr.SyncManager, - alloc allocator.Allocator, - plan *datapb.CompactionPlan, -) *compactionTask { - ctx1, cancel := context.WithCancel(ctx) - return &compactionTask{ - ctx: ctx1, - cancel: cancel, - binlogIO: binlogIO, - syncMgr: syncMgr, - metaCache: metaCache, - Allocator: alloc, - plan: plan, - tr: timerecord.NewTimeRecorder("levelone compaction"), - done: make(chan struct{}, 1), - } -} - -func (t *compactionTask) complete() { - t.done <- struct{}{} -} - -func (t *compactionTask) stop() { - t.cancel() - <-t.done - t.injectDone() -} - -func (t *compactionTask) getPlanID() UniqueID { - return t.plan.GetPlanID() -} - -func (t *compactionTask) getChannelName() string { - return t.plan.GetChannel() -} - -// return num rows of all segment compaction from -func (t *compactionTask) getNumRows() (int64, error) { - numRows := int64(0) - for _, binlog := range t.plan.SegmentBinlogs { - seg, ok := t.metaCache.GetSegmentByID(binlog.GetSegmentID()) - if !ok { - return 0, merr.WrapErrSegmentNotFound(binlog.GetSegmentID(), "get compaction segments num rows failed") - } - - numRows += seg.NumOfRows() - } - - return numRows, nil -} - -func (t *compactionTask) mergeDeltalogs(dBlobs map[UniqueID][]*Blob) (map[interface{}]Timestamp, error) { - log := log.With(zap.Int64("planID", t.getPlanID())) - mergeStart := time.Now() - dCodec := storage.NewDeleteCodec() - - pk2ts := make(map[interface{}]Timestamp) - - for _, blobs := range dBlobs { - _, _, dData, err := dCodec.Deserialize(blobs) - if err != nil { - log.Warn("merge deltalogs wrong", zap.Error(err)) - return nil, err - } - - for i := int64(0); i < dData.RowCount; i++ { - pk := dData.Pks[i] - ts := dData.Tss[i] - if lastTS, ok := pk2ts[pk.GetValue()]; ok && lastTS > ts { - ts = lastTS - } - pk2ts[pk.GetValue()] = ts - } - } - - log.Info("mergeDeltalogs end", - zap.Int("number of deleted pks to compact in insert logs", len(pk2ts)), - zap.Duration("elapse", time.Since(mergeStart))) - - return pk2ts, nil -} - -func (t *compactionTask) uploadRemainLog( - ctxTimeout context.Context, - targetSegID UniqueID, - partID UniqueID, - meta *etcdpb.CollectionMeta, - stats *storage.PrimaryKeyStats, - totRows int64, - writeBuffer *storage.InsertData, -) (map[UniqueID]*datapb.FieldBinlog, map[UniqueID]*datapb.FieldBinlog, error) { - iCodec := storage.NewInsertCodecWithSchema(meta) - inPaths := make(map[int64]*datapb.FieldBinlog, 0) - var err error - if !writeBuffer.IsEmpty() { - inPaths, err = uploadInsertLog(ctxTimeout, t.binlogIO, t.Allocator, meta.GetID(), partID, targetSegID, writeBuffer, iCodec) - if err != nil { - return nil, nil, err - } - } - - statPaths, err := uploadStatsLog(ctxTimeout, t.binlogIO, t.Allocator, meta.GetID(), partID, targetSegID, stats, totRows, iCodec) - if err != nil { - return nil, nil, err - } - - return inPaths, statPaths, nil -} - -func (t *compactionTask) uploadSingleInsertLog( - ctxTimeout context.Context, - targetSegID UniqueID, - partID UniqueID, - meta *etcdpb.CollectionMeta, - writeBuffer *storage.InsertData, -) (map[UniqueID]*datapb.FieldBinlog, error) { - iCodec := storage.NewInsertCodecWithSchema(meta) - - inPaths, err := uploadInsertLog(ctxTimeout, t.binlogIO, t.Allocator, meta.GetID(), partID, targetSegID, writeBuffer, iCodec) - if err != nil { - return nil, err - } - - return inPaths, nil -} - -func (t *compactionTask) merge( - ctx context.Context, - unMergedInsertlogs [][]string, - targetSegID UniqueID, - partID UniqueID, - meta *etcdpb.CollectionMeta, - delta map[interface{}]Timestamp, -) ([]*datapb.FieldBinlog, []*datapb.FieldBinlog, int64, error) { - ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, fmt.Sprintf("CompactMerge-%d", t.getPlanID())) - defer span.End() - log := log.With(zap.Int64("planID", t.getPlanID())) - mergeStart := time.Now() - - var ( - numBinlogs int // binlog number - numRows int64 // the number of rows uploaded - expired int64 // the number of expired entity - - insertField2Path = make(map[UniqueID]*datapb.FieldBinlog) - insertPaths = make([]*datapb.FieldBinlog, 0) - - statField2Path = make(map[UniqueID]*datapb.FieldBinlog) - statPaths = make([]*datapb.FieldBinlog, 0) - ) - writeBuffer, err := storage.NewInsertData(meta.GetSchema()) - if err != nil { - return nil, nil, -1, err - } - - isDeletedValue := func(v *storage.Value) bool { - ts, ok := delta[v.PK.GetValue()] - // insert task and delete task has the same ts when upsert - // here should be < instead of <= - // to avoid the upsert data to be deleted after compact - if ok && uint64(v.Timestamp) < ts { - return true - } - return false - } - - addInsertFieldPath := func(inPaths map[UniqueID]*datapb.FieldBinlog, timestampFrom, timestampTo int64) { - for fID, path := range inPaths { - for _, binlog := range path.GetBinlogs() { - binlog.TimestampTo = uint64(timestampTo) - binlog.TimestampFrom = uint64(timestampFrom) - } - tmpBinlog, ok := insertField2Path[fID] - if !ok { - tmpBinlog = path - } else { - tmpBinlog.Binlogs = append(tmpBinlog.Binlogs, path.GetBinlogs()...) - } - insertField2Path[fID] = tmpBinlog - } - } - - addStatFieldPath := func(statPaths map[UniqueID]*datapb.FieldBinlog) { - for fID, path := range statPaths { - tmpBinlog, ok := statField2Path[fID] - if !ok { - tmpBinlog = path - } else { - tmpBinlog.Binlogs = append(tmpBinlog.Binlogs, path.GetBinlogs()...) - } - statField2Path[fID] = tmpBinlog - } - } - - // get pkID, pkType, dim - var pkField *schemapb.FieldSchema - for _, fs := range meta.GetSchema().GetFields() { - if fs.GetIsPrimaryKey() && fs.GetFieldID() >= 100 && typeutil.IsPrimaryFieldType(fs.GetDataType()) { - pkField = fs - } - } - - if pkField == nil { - log.Warn("failed to get pk field from schema") - return nil, nil, 0, fmt.Errorf("no pk field in schema") - } - - pkID := pkField.GetFieldID() - pkType := pkField.GetDataType() - - expired = 0 - numRows = 0 - numBinlogs = 0 - currentTs := t.GetCurrentTime() - currentRows := 0 - downloadTimeCost := time.Duration(0) - uploadInsertTimeCost := time.Duration(0) - - oldRowNums, err := t.getNumRows() - if err != nil { - return nil, nil, 0, err - } - - stats, err := storage.NewPrimaryKeyStats(pkID, int64(pkType), oldRowNums) - if err != nil { - return nil, nil, 0, err - } - // initial timestampFrom, timestampTo = -1, -1 is an illegal value, only to mark initial state - var ( - timestampTo int64 = -1 - timestampFrom int64 = -1 - ) - - for _, path := range unMergedInsertlogs { - downloadStart := time.Now() - data, err := downloadBlobs(ctx, t.binlogIO, path) - if err != nil { - log.Warn("download insertlogs wrong", zap.Strings("path", path), zap.Error(err)) - return nil, nil, 0, err - } - downloadTimeCost += time.Since(downloadStart) - - iter, err := storage.NewBinlogDeserializeReader(data, pkID) - if err != nil { - log.Warn("new insert binlogs reader wrong", zap.Strings("path", path), zap.Error(err)) - return nil, nil, 0, err - } - - for { - err := iter.Next() - if err != nil { - if err == sio.EOF { - break - } else { - log.Warn("transfer interface to Value wrong", zap.Strings("path", path)) - return nil, nil, 0, errors.New("unexpected error") - } - } - v := iter.Value() - if isDeletedValue(v) { - continue - } - - ts := Timestamp(v.Timestamp) - // Filtering expired entity - if t.isExpiredEntity(ts, currentTs) { - expired++ - continue - } - - // Update timestampFrom, timestampTo - if v.Timestamp < timestampFrom || timestampFrom == -1 { - timestampFrom = v.Timestamp - } - if v.Timestamp > timestampTo || timestampFrom == -1 { - timestampTo = v.Timestamp - } - - row, ok := v.Value.(map[UniqueID]interface{}) - if !ok { - log.Warn("transfer interface to map wrong", zap.Strings("path", path)) - return nil, nil, 0, errors.New("unexpected error") - } - - err = writeBuffer.Append(row) - if err != nil { - return nil, nil, 0, err - } - - currentRows++ - stats.Update(v.PK) - - // check size every 100 rows in case of too many `GetMemorySize` call - if (currentRows+1)%100 == 0 && writeBuffer.GetMemorySize() > paramtable.Get().DataNodeCfg.BinLogMaxSize.GetAsInt() { - numRows += int64(writeBuffer.GetRowNum()) - uploadInsertStart := time.Now() - inPaths, err := t.uploadSingleInsertLog(ctx, targetSegID, partID, meta, writeBuffer) - if err != nil { - log.Warn("failed to upload single insert log", zap.Error(err)) - return nil, nil, 0, err - } - uploadInsertTimeCost += time.Since(uploadInsertStart) - addInsertFieldPath(inPaths, timestampFrom, timestampTo) - timestampFrom = -1 - timestampTo = -1 - - writeBuffer, _ = storage.NewInsertData(meta.GetSchema()) - currentRows = 0 - numBinlogs++ - } - } - } - - // upload stats log and remain insert rows - if writeBuffer.GetRowNum() > 0 || numRows > 0 { - numRows += int64(writeBuffer.GetRowNum()) - uploadStart := time.Now() - inPaths, statsPaths, err := t.uploadRemainLog(ctx, targetSegID, partID, meta, - stats, numRows+int64(currentRows), writeBuffer) - if err != nil { - return nil, nil, 0, err - } - - uploadInsertTimeCost += time.Since(uploadStart) - addInsertFieldPath(inPaths, timestampFrom, timestampTo) - addStatFieldPath(statsPaths) - numBinlogs += len(inPaths) - } - - for _, path := range insertField2Path { - insertPaths = append(insertPaths, path) - } - - for _, path := range statField2Path { - statPaths = append(statPaths, path) - } - - log.Info("compact merge end", - zap.Int64("remaining insert numRows", numRows), - zap.Int64("expired entities", expired), - zap.Int("binlog file number", numBinlogs), - zap.Duration("download insert log elapse", downloadTimeCost), - zap.Duration("upload insert log elapse", uploadInsertTimeCost), - zap.Duration("merge elapse", time.Since(mergeStart))) - - return insertPaths, statPaths, numRows, nil -} - -func (t *compactionTask) compact() (*datapb.CompactionPlanResult, error) { - ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, fmt.Sprintf("Compact-%d", t.getPlanID())) - defer span.End() - - log := log.Ctx(ctx).With(zap.Int64("planID", t.plan.GetPlanID()), zap.Int32("timeout in seconds", t.plan.GetTimeoutInSeconds())) - if ok := funcutil.CheckCtxValid(ctx); !ok { - log.Warn("compact wrong, task context done or timeout") - return nil, errContext - } - - ctxTimeout, cancelAll := context.WithTimeout(ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second) - defer cancelAll() - - compactStart := time.Now() - durInQueue := t.tr.RecordSpan() - log.Info("compact start") - if len(t.plan.GetSegmentBinlogs()) < 1 { - log.Warn("compact wrong, there's no segments in segment binlogs") - return nil, errIllegalCompactionPlan - } - - targetSegID, err := t.AllocOne() - if err != nil { - log.Warn("compact wrong, unable to allocate segmentID", zap.Error(err)) - return nil, err - } - - segIDs := lo.Map(t.plan.GetSegmentBinlogs(), func(binlogs *datapb.CompactionSegmentBinlogs, _ int) int64 { - return binlogs.GetSegmentID() - }) - - // Inject to stop flush - // when compaction failed, these segments need to be Unblocked by injectDone in compaction_executor - // when compaction succeeded, these segments will be Unblocked by SyncSegments from DataCoord. - for _, segID := range segIDs { - t.syncMgr.Block(segID) - } - log.Info("compact finsh injection", zap.Duration("elapse", t.tr.RecordSpan())) - - if err := binlog.DecompressCompactionBinlogs(t.plan.GetSegmentBinlogs()); err != nil { - log.Warn("compact wrong, fail to decompress compaction binlogs", zap.Error(err)) - return nil, err - } - - dblobs := make(map[UniqueID][]*Blob) - allPath := make([][]string, 0) - - for _, s := range t.plan.GetSegmentBinlogs() { - log := log.With(zap.Int64("segmentID", s.GetSegmentID())) - // Get the batch count of field binlog files - var binlogBatch int - for _, b := range s.GetFieldBinlogs() { - if b != nil { - binlogBatch = len(b.GetBinlogs()) - break - } - } - if binlogBatch == 0 { - log.Warn("compacting empty segment") - continue - } - - for idx := 0; idx < binlogBatch; idx++ { - var ps []string - for _, f := range s.GetFieldBinlogs() { - ps = append(ps, f.GetBinlogs()[idx].GetLogPath()) - } - allPath = append(allPath, ps) - } - - paths := make([]string, 0) - for _, d := range s.GetDeltalogs() { - for _, l := range d.GetBinlogs() { - path := l.GetLogPath() - paths = append(paths, path) - } - } - - if len(paths) != 0 { - bs, err := downloadBlobs(ctxTimeout, t.binlogIO, paths) - if err != nil { - log.Warn("compact wrong, fail to download deltalogs", zap.Strings("path", paths), zap.Error(err)) - return nil, err - } - dblobs[s.GetSegmentID()] = append(dblobs[s.GetSegmentID()], bs...) - } - } - - // Unable to deal with all empty segments cases, so return error - if len(allPath) == 0 { - log.Warn("compact wrong, all segments are empty") - return nil, errIllegalCompactionPlan - } - - log.Info("compact download deltalogs elapse", zap.Duration("elapse", t.tr.RecordSpan())) - - if err != nil { - log.Warn("compact IO wrong", zap.Error(err)) - return nil, err - } - - deltaPk2Ts, err := t.mergeDeltalogs(dblobs) - if err != nil { - log.Warn("compact wrong, fail to merge deltalogs", zap.Error(err)) - return nil, err - } - - segmentBinlog := t.plan.GetSegmentBinlogs()[0] - partID := segmentBinlog.GetPartitionID() - meta := &etcdpb.CollectionMeta{ID: t.metaCache.Collection(), Schema: t.metaCache.Schema()} - - inPaths, statsPaths, numRows, err := t.merge(ctxTimeout, allPath, targetSegID, partID, meta, deltaPk2Ts) - if err != nil { - log.Warn("compact wrong, fail to merge", zap.Error(err)) - return nil, err - } - - pack := &datapb.CompactionSegment{ - SegmentID: targetSegID, - InsertLogs: inPaths, - Field2StatslogPaths: statsPaths, - NumOfRows: numRows, - Channel: t.plan.GetChannel(), - } - - log.Info("compact done", - zap.Int64("targetSegmentID", targetSegID), - zap.Int64s("compactedFrom", segIDs), - zap.Int("num of binlog paths", len(inPaths)), - zap.Int("num of stats paths", len(statsPaths)), - zap.Int("num of delta paths", len(pack.GetDeltalogs())), - zap.Duration("elapse", time.Since(compactStart)), - ) - - metrics.DataNodeCompactionLatency.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.plan.GetType().String()).Observe(float64(t.tr.ElapseSpan().Milliseconds())) - metrics.DataNodeCompactionLatencyInQueue.WithLabelValues(fmt.Sprint(paramtable.GetNodeID())).Observe(float64(durInQueue.Milliseconds())) - - planResult := &datapb.CompactionPlanResult{ - State: commonpb.CompactionState_Completed, - PlanID: t.getPlanID(), - Channel: t.plan.GetChannel(), - Segments: []*datapb.CompactionSegment{pack}, - Type: t.plan.GetType(), - } - - return planResult, nil -} - -func (t *compactionTask) injectDone() { - t.injectDoneOnce.Do(func() { - for _, binlog := range t.plan.SegmentBinlogs { - t.syncMgr.Unblock(binlog.SegmentID) - } - }) -} - -// TODO copy maybe expensive, but this seems to be the only convinent way. -func interface2FieldData(schemaDataType schemapb.DataType, content []interface{}, numRows int64) (storage.FieldData, error) { - var rst storage.FieldData - switch schemaDataType { - case schemapb.DataType_Bool: - data := &storage.BoolFieldData{ - Data: make([]bool, 0, len(content)), - } - - for _, c := range content { - r, ok := c.(bool) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r) - } - rst = data - - case schemapb.DataType_Int8: - data := &storage.Int8FieldData{ - Data: make([]int8, 0, len(content)), - } - - for _, c := range content { - r, ok := c.(int8) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r) - } - rst = data - - case schemapb.DataType_Int16: - data := &storage.Int16FieldData{ - Data: make([]int16, 0, len(content)), - } - - for _, c := range content { - r, ok := c.(int16) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r) - } - rst = data - - case schemapb.DataType_Int32: - data := &storage.Int32FieldData{ - Data: make([]int32, 0, len(content)), - } - - for _, c := range content { - r, ok := c.(int32) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r) - } - rst = data - - case schemapb.DataType_Int64: - data := &storage.Int64FieldData{ - Data: make([]int64, 0, len(content)), - } - - for _, c := range content { - r, ok := c.(int64) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r) - } - rst = data - - case schemapb.DataType_Float: - data := &storage.FloatFieldData{ - Data: make([]float32, 0, len(content)), - } - - for _, c := range content { - r, ok := c.(float32) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r) - } - rst = data - - case schemapb.DataType_Double: - data := &storage.DoubleFieldData{ - Data: make([]float64, 0, len(content)), - } - - for _, c := range content { - r, ok := c.(float64) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r) - } - rst = data - - case schemapb.DataType_String, schemapb.DataType_VarChar: - data := &storage.StringFieldData{ - Data: make([]string, 0, len(content)), - } - - for _, c := range content { - r, ok := c.(string) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r) - } - rst = data - - case schemapb.DataType_JSON: - data := &storage.JSONFieldData{ - Data: make([][]byte, 0, len(content)), - } - - for _, c := range content { - r, ok := c.([]byte) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r) - } - rst = data - - case schemapb.DataType_Array: - data := &storage.ArrayFieldData{ - Data: make([]*schemapb.ScalarField, 0, len(content)), - } - - for _, c := range content { - r, ok := c.(*schemapb.ScalarField) - if !ok { - return nil, errTransferType - } - data.ElementType = r.GetArrayData().GetElementType() - data.Data = append(data.Data, r) - } - rst = data - - case schemapb.DataType_FloatVector: - data := &storage.FloatVectorFieldData{ - Data: []float32{}, - } - - for _, c := range content { - r, ok := c.([]float32) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r...) - } - - data.Dim = len(data.Data) / int(numRows) - rst = data - - case schemapb.DataType_Float16Vector: - data := &storage.Float16VectorFieldData{ - Data: []byte{}, - } - - for _, c := range content { - r, ok := c.([]byte) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r...) - } - - data.Dim = len(data.Data) / 2 / int(numRows) - rst = data - - case schemapb.DataType_BFloat16Vector: - data := &storage.BFloat16VectorFieldData{ - Data: []byte{}, - } - - for _, c := range content { - r, ok := c.([]byte) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r...) - } - - data.Dim = len(data.Data) / 2 / int(numRows) - rst = data - - case schemapb.DataType_BinaryVector: - data := &storage.BinaryVectorFieldData{ - Data: []byte{}, - } - - for _, c := range content { - r, ok := c.([]byte) - if !ok { - return nil, errTransferType - } - data.Data = append(data.Data, r...) - } - - data.Dim = len(data.Data) * 8 / int(numRows) - rst = data - - case schemapb.DataType_SparseFloatVector: - data := &storage.SparseFloatVectorFieldData{} - for _, c := range content { - if err := data.AppendRow(c); err != nil { - return nil, fmt.Errorf("failed to append row: %v, %w", err, errTransferType) - } - } - rst = data - - default: - return nil, errUnknownDataType - } - - return rst, nil -} - -func (t *compactionTask) getCollection() UniqueID { - return t.metaCache.Collection() -} - -func (t *compactionTask) GetCurrentTime() typeutil.Timestamp { - return tsoutil.GetCurrentTime() -} - -func (t *compactionTask) isExpiredEntity(ts, now Timestamp) bool { - // entity expire is not enabled if duration <= 0 - if t.plan.GetCollectionTtl() <= 0 { - return false - } - - pts, _ := tsoutil.ParseTS(ts) - pnow, _ := tsoutil.ParseTS(now) - expireTime := pts.Add(time.Duration(t.plan.GetCollectionTtl())) - return expireTime.Before(pnow) -} diff --git a/internal/datanode/compactor_test.go b/internal/datanode/compactor_test.go deleted file mode 100644 index ccae34bebae2a..0000000000000 --- a/internal/datanode/compactor_test.go +++ /dev/null @@ -1,1162 +0,0 @@ -// Licensed to the LF AI & Data foundation under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package datanode - -import ( - "context" - "fmt" - "math" - "testing" - "time" - - "github.com/cockroachdb/errors" - "github.com/samber/lo" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - - "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/internal/datanode/allocator" - "github.com/milvus-io/milvus/internal/datanode/io" - "github.com/milvus-io/milvus/internal/datanode/metacache" - "github.com/milvus-io/milvus/internal/datanode/syncmgr" - memkv "github.com/milvus-io/milvus/internal/kv/mem" - "github.com/milvus-io/milvus/internal/proto/datapb" - "github.com/milvus-io/milvus/internal/proto/etcdpb" - "github.com/milvus-io/milvus/internal/storage" - "github.com/milvus-io/milvus/pkg/common" - "github.com/milvus-io/milvus/pkg/util/paramtable" - "github.com/milvus-io/milvus/pkg/util/timerecord" - "github.com/milvus-io/milvus/pkg/util/typeutil" -) - -var compactTestDir = "/tmp/milvus_test/compact" - -func TestCompactionTaskInnerMethods(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cm := storage.NewLocalChunkManager(storage.RootPath(compactTestDir)) - defer cm.RemoveWithPrefix(ctx, cm.RootPath()) - t.Run("Test.interface2FieldData", func(t *testing.T) { - tests := []struct { - isvalid bool - - tp schemapb.DataType - content []interface{} - - description string - }{ - {true, schemapb.DataType_Bool, []interface{}{true, false}, "valid bool"}, - {true, schemapb.DataType_Int8, []interface{}{int8(1), int8(2)}, "valid int8"}, - {true, schemapb.DataType_Int16, []interface{}{int16(1), int16(2)}, "valid int16"}, - {true, schemapb.DataType_Int32, []interface{}{int32(1), int32(2)}, "valid int32"}, - {true, schemapb.DataType_Int64, []interface{}{int64(1), int64(2)}, "valid int64"}, - {true, schemapb.DataType_Float, []interface{}{float32(1), float32(2)}, "valid float32"}, - {true, schemapb.DataType_Double, []interface{}{float64(1), float64(2)}, "valid float64"}, - {true, schemapb.DataType_VarChar, []interface{}{"test1", "test2"}, "valid varChar"}, - {true, schemapb.DataType_JSON, []interface{}{[]byte("{\"key\":\"value\"}"), []byte("{\"hello\":\"world\"}")}, "valid json"}, - {true, schemapb.DataType_Array, []interface{}{ - &schemapb.ScalarField{ - Data: &schemapb.ScalarField_IntData{ - IntData: &schemapb.IntArray{ - Data: []int32{1, 2}, - }, - }, - }, - &schemapb.ScalarField{ - Data: &schemapb.ScalarField_IntData{ - IntData: &schemapb.IntArray{ - Data: []int32{3, 4}, - }, - }, - }, - }, "valid array"}, - {true, schemapb.DataType_FloatVector, []interface{}{[]float32{1.0, 2.0}}, "valid floatvector"}, - {true, schemapb.DataType_BinaryVector, []interface{}{[]byte{255}}, "valid binaryvector"}, - {true, schemapb.DataType_Float16Vector, []interface{}{[]byte{255, 255, 255, 255}}, "valid float16vector"}, - {true, schemapb.DataType_BFloat16Vector, []interface{}{[]byte{255, 255, 255, 255}}, "valid bfloat16vector"}, - - {false, schemapb.DataType_Bool, []interface{}{1, 2}, "invalid bool"}, - {false, schemapb.DataType_Int8, []interface{}{nil, nil}, "invalid int8"}, - {false, schemapb.DataType_Int16, []interface{}{nil, nil}, "invalid int16"}, - {false, schemapb.DataType_Int32, []interface{}{nil, nil}, "invalid int32"}, - {false, schemapb.DataType_Int64, []interface{}{nil, nil}, "invalid int64"}, - {false, schemapb.DataType_Float, []interface{}{nil, nil}, "invalid float32"}, - {false, schemapb.DataType_Double, []interface{}{nil, nil}, "invalid float64"}, - {false, schemapb.DataType_VarChar, []interface{}{nil, nil}, "invalid varChar"}, - {false, schemapb.DataType_JSON, []interface{}{nil, nil}, "invalid json"}, - {false, schemapb.DataType_FloatVector, []interface{}{nil, nil}, "invalid floatvector"}, - {false, schemapb.DataType_BinaryVector, []interface{}{nil, nil}, "invalid binaryvector"}, - {false, schemapb.DataType_Float16Vector, []interface{}{nil, nil}, "invalid float16vector"}, - {false, schemapb.DataType_BFloat16Vector, []interface{}{nil, nil}, "invalid bfloat16vector"}, - - {false, schemapb.DataType_SparseFloatVector, []interface{}{nil, nil}, "invalid sparsefloatvector"}, - {false, schemapb.DataType_SparseFloatVector, []interface{}{[]byte{255}, []byte{15}}, "invalid sparsefloatvector"}, - {true, schemapb.DataType_SparseFloatVector, []interface{}{ - typeutil.CreateSparseFloatRow([]uint32{1, 2}, []float32{1.0, 2.0}), - typeutil.CreateSparseFloatRow([]uint32{3, 4}, []float32{1.0, 2.0}), - }, "valid sparsefloatvector"}, - } - - // make sure all new data types missed to handle would throw unexpected error - for typeName, typeValue := range schemapb.DataType_value { - tests = append(tests, struct { - isvalid bool - - tp schemapb.DataType - content []interface{} - - description string - }{false, schemapb.DataType(typeValue), []interface{}{nil, nil}, "invalid " + typeName}) - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - if test.isvalid { - fd, err := interface2FieldData(test.tp, test.content, 2) - assert.NoError(t, err) - assert.Equal(t, 2, fd.RowNum()) - } else { - fd, err := interface2FieldData(test.tp, test.content, 2) - assert.True(t, errors.Is(err, errTransferType) || errors.Is(err, errUnknownDataType)) - assert.Nil(t, fd) - } - }) - } - }) - - t.Run("Test mergeDeltalogs", func(t *testing.T) { - t.Run("One segment", func(t *testing.T) { - invalidBlobs := map[UniqueID][]*Blob{ - 1: {}, - } - - blobs, err := getInt64DeltaBlobs( - 100, - []UniqueID{ - 1, - 2, - 3, - 4, - 5, - 1, - 2, - }, - []Timestamp{ - 20000, - 20001, - 20002, - 30000, - 50000, - 50000, - 10000, - }) - require.NoError(t, err) - - validBlobs := map[UniqueID][]*Blob{ - 100: blobs, - } - - tests := []struct { - isvalid bool - - dBlobs map[UniqueID][]*Blob - - description string - }{ - {false, invalidBlobs, "invalid dBlobs"}, - {true, validBlobs, "valid blobs"}, - } - - for _, test := range tests { - task := &compactionTask{ - done: make(chan struct{}, 1), - } - t.Run(test.description, func(t *testing.T) { - pk2ts, err := task.mergeDeltalogs(test.dBlobs) - if test.isvalid { - assert.NoError(t, err) - assert.Equal(t, 5, len(pk2ts)) - assert.EqualValues(t, 20001, pk2ts[UniqueID(2)]) - } else { - assert.Error(t, err) - assert.Nil(t, pk2ts) - } - }) - } - }) - - t.Run("Multiple segments", func(t *testing.T) { - tests := []struct { - segIDA UniqueID - dataApk []UniqueID - dataAts []Timestamp - - segIDB UniqueID - dataBpk []UniqueID - dataBts []Timestamp - - segIDC UniqueID - dataCpk []UniqueID - dataCts []Timestamp - - expectedpk2ts int - description string - }{ - { - 0, nil, nil, - 100, - []UniqueID{1, 2, 3}, - []Timestamp{20000, 30000, 20005}, - 200, - []UniqueID{4, 5, 6}, - []Timestamp{50000, 50001, 50002}, - 6, "2 segments", - }, - { - 300, - []UniqueID{10, 20}, - []Timestamp{20001, 40001}, - 100, - []UniqueID{1, 2, 3}, - []Timestamp{20000, 30000, 20005}, - 200, - []UniqueID{4, 5, 6}, - []Timestamp{50000, 50001, 50002}, - 8, "3 segments", - }, - } - - for _, test := range tests { - t.Run(test.description, func(t *testing.T) { - dBlobs := make(map[UniqueID][]*Blob) - if test.segIDA != UniqueID(0) { - d, err := getInt64DeltaBlobs(test.segIDA, test.dataApk, test.dataAts) - require.NoError(t, err) - dBlobs[test.segIDA] = d - } - if test.segIDB != UniqueID(0) { - d, err := getInt64DeltaBlobs(test.segIDB, test.dataBpk, test.dataBts) - require.NoError(t, err) - dBlobs[test.segIDB] = d - } - if test.segIDC != UniqueID(0) { - d, err := getInt64DeltaBlobs(test.segIDC, test.dataCpk, test.dataCts) - require.NoError(t, err) - dBlobs[test.segIDC] = d - } - - task := &compactionTask{ - done: make(chan struct{}, 1), - } - pk2ts, err := task.mergeDeltalogs(dBlobs) - assert.NoError(t, err) - assert.Equal(t, test.expectedpk2ts, len(pk2ts)) - }) - } - }) - }) - - t.Run("Test merge", func(t *testing.T) { - collectionID := int64(1) - meta := NewMetaFactory().GetCollectionMeta(collectionID, "test", schemapb.DataType_Int64) - - metaCache := metacache.NewMockMetaCache(t) - metaCache.EXPECT().Schema().Return(meta.GetSchema()).Maybe() - metaCache.EXPECT().GetSegmentByID(mock.Anything).RunAndReturn(func(id int64, filters ...metacache.SegmentFilter) (*metacache.SegmentInfo, bool) { - segment := metacache.NewSegmentInfo(&datapb.SegmentInfo{ - CollectionID: 1, - PartitionID: 0, - ID: id, - NumOfRows: 10, - }, nil) - return segment, true - }) - - alloc := allocator.NewMockAllocator(t) - alloc.EXPECT().GetGenerator(mock.Anything, mock.Anything).Call.Return(validGeneratorFn, nil) - alloc.EXPECT().AllocOne().Return(0, nil) - t.Run("Merge without expiration", func(t *testing.T) { - mockbIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0") - iData := genInsertDataWithExpiredTS() - iCodec := storage.NewInsertCodecWithSchema(meta) - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), 0, 1, iData, iCodec) - assert.NoError(t, err) - assert.Equal(t, 12, len(inpath)) - binlogNum := len(inpath[0].GetBinlogs()) - assert.Equal(t, 1, binlogNum) - - for idx := 0; idx < binlogNum; idx++ { - var ps []string - for _, path := range inpath { - ps = append(ps, path.GetBinlogs()[idx].GetLogPath()) - } - allPaths = append(allPaths, ps) - } - - dm := map[interface{}]Timestamp{ - 1: 10000, - } - - ct := &compactionTask{ - metaCache: metaCache, - binlogIO: mockbIO, - Allocator: alloc, - done: make(chan struct{}, 1), - plan: &datapb.CompactionPlan{ - SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ - {SegmentID: 1}, - }, - }, - } - inPaths, statsPaths, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm) - assert.NoError(t, err) - assert.Equal(t, int64(2), numOfRow) - assert.Equal(t, 1, len(inPaths[0].GetBinlogs())) - assert.Equal(t, 1, len(statsPaths)) - assert.NotEqual(t, -1, inPaths[0].GetBinlogs()[0].GetTimestampFrom()) - assert.NotEqual(t, -1, inPaths[0].GetBinlogs()[0].GetTimestampTo()) - }) - t.Run("Merge without expiration2", func(t *testing.T) { - mockbIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0") - BinLogMaxSize := Params.DataNodeCfg.BinLogMaxSize.GetValue() - defer func() { - Params.Save(Params.DataNodeCfg.BinLogMaxSize.Key, BinLogMaxSize) - }() - paramtable.Get().Save(Params.DataNodeCfg.BinLogMaxSize.Key, "64") - iData := genInsertDataWithExpiredTS() - meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) - - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), 0, 1, iData, iCodec) - assert.NoError(t, err) - assert.Equal(t, 12, len(inpath)) - binlogNum := len(inpath[0].GetBinlogs()) - assert.Equal(t, 1, binlogNum) - - for idx := 0; idx < binlogNum; idx++ { - var ps []string - for _, path := range inpath { - ps = append(ps, path.GetBinlogs()[idx].GetLogPath()) - } - allPaths = append(allPaths, ps) - } - - dm := map[interface{}]Timestamp{} - - ct := &compactionTask{ - metaCache: metaCache, - binlogIO: mockbIO, - Allocator: alloc, - done: make(chan struct{}, 1), - plan: &datapb.CompactionPlan{ - SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ - {SegmentID: 1}, - }, - }, - } - inPaths, statsPaths, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm) - assert.NoError(t, err) - assert.Equal(t, int64(2), numOfRow) - assert.Equal(t, 1, len(inPaths[0].GetBinlogs())) - assert.Equal(t, 1, len(statsPaths)) - assert.Equal(t, 1, len(statsPaths[0].GetBinlogs())) - assert.NotEqual(t, -1, inPaths[0].GetBinlogs()[0].GetTimestampFrom()) - assert.NotEqual(t, -1, inPaths[0].GetBinlogs()[0].GetTimestampTo()) - }) - // set Params.DataNodeCfg.BinLogMaxSize.Key = 1 to generate multi binlogs, each has only one row - t.Run("merge_with_more_than_100rows", func(t *testing.T) { - mockbIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0") - BinLogMaxSize := Params.DataNodeCfg.BinLogMaxSize.GetAsInt() - defer func() { - paramtable.Get().Save(Params.DataNodeCfg.BinLogMaxSize.Key, fmt.Sprintf("%d", BinLogMaxSize)) - }() - paramtable.Get().Save(Params.DataNodeCfg.BinLogMaxSize.Key, "1") - iData := genInsertData(101) - - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), 0, 1, iData, iCodec) - assert.NoError(t, err) - assert.Equal(t, 12, len(inpath)) - binlogNum := len(inpath[0].GetBinlogs()) - assert.Equal(t, 1, binlogNum) - - for idx := 0; idx < binlogNum; idx++ { - var ps []string - for _, path := range inpath { - ps = append(ps, path.GetBinlogs()[idx].GetLogPath()) - } - allPaths = append(allPaths, ps) - } - - dm := map[interface{}]Timestamp{ - 1: 10000, - } - - ct := &compactionTask{ - metaCache: metaCache, - binlogIO: mockbIO, - Allocator: alloc, - done: make(chan struct{}, 1), - plan: &datapb.CompactionPlan{ - SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ - {SegmentID: 1}, - }, - }, - } - inPaths, statsPaths, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm) - assert.NoError(t, err) - assert.Equal(t, int64(101), numOfRow) - assert.Equal(t, 2, len(inPaths[0].GetBinlogs())) - assert.Equal(t, 1, len(statsPaths)) - for _, inpath := range inPaths { - assert.NotEqual(t, -1, inpath.GetBinlogs()[0].GetTimestampFrom()) - assert.NotEqual(t, -1, inpath.GetBinlogs()[0].GetTimestampTo()) - } - }) - - t.Run("Merge with expiration", func(t *testing.T) { - mockbIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - iData := genInsertDataWithExpiredTS() - meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) - - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), 0, 1, iData, iCodec) - assert.NoError(t, err) - assert.Equal(t, 12, len(inpath)) - binlogNum := len(inpath[0].GetBinlogs()) - assert.Equal(t, 1, binlogNum) - - for idx := 0; idx < binlogNum; idx++ { - var ps []string - for _, path := range inpath { - ps = append(ps, path.GetBinlogs()[idx].GetLogPath()) - } - allPaths = append(allPaths, ps) - } - - dm := map[interface{}]Timestamp{ - 1: 10000, - } - - // 10 days in seconds - ct := &compactionTask{ - metaCache: metaCache, - binlogIO: mockbIO, - Allocator: alloc, - plan: &datapb.CompactionPlan{ - CollectionTtl: 864000, - SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ - {SegmentID: 1}, - }, - }, - done: make(chan struct{}, 1), - } - inPaths, statsPaths, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm) - assert.NoError(t, err) - assert.Equal(t, int64(0), numOfRow) - assert.Equal(t, 0, len(inPaths)) - assert.Equal(t, 0, len(statsPaths)) - }) - - t.Run("merge_with_rownum_zero", func(t *testing.T) { - mockbIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iData := genInsertDataWithExpiredTS() - iCodec := storage.NewInsertCodecWithSchema(meta) - meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) - metaCache := metacache.NewMockMetaCache(t) - metaCache.EXPECT().Schema().Return(meta.GetSchema()).Maybe() - metaCache.EXPECT().GetSegmentByID(mock.Anything).RunAndReturn(func(id int64, filters ...metacache.SegmentFilter) (*metacache.SegmentInfo, bool) { - segment := metacache.NewSegmentInfo(&datapb.SegmentInfo{ - CollectionID: 1, - PartitionID: 0, - ID: id, - NumOfRows: 0, - }, nil) - return segment, true - }) - - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), 0, 1, iData, iCodec) - assert.NoError(t, err) - assert.Equal(t, 12, len(inpath)) - binlogNum := len(inpath[0].GetBinlogs()) - assert.Equal(t, 1, binlogNum) - - for idx := 0; idx < binlogNum; idx++ { - var ps []string - for _, path := range inpath { - ps = append(ps, path.GetBinlogs()[idx].GetLogPath()) - } - allPaths = append(allPaths, ps) - } - - dm := map[interface{}]Timestamp{ - 1: 10000, - } - - ct := &compactionTask{ - metaCache: metaCache, - binlogIO: mockbIO, - Allocator: alloc, - done: make(chan struct{}, 1), - plan: &datapb.CompactionPlan{ - SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ - {SegmentID: 1}, - }, - }, - } - _, _, _, err = ct.merge(context.Background(), allPaths, 2, 0, &etcdpb.CollectionMeta{ - Schema: meta.GetSchema(), - }, dm) - assert.Error(t, err) - }) - - t.Run("Merge with meta error", func(t *testing.T) { - mockbIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0") - iData := genInsertDataWithExpiredTS() - meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) - - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), 0, 1, iData, iCodec) - assert.NoError(t, err) - assert.Equal(t, 12, len(inpath)) - binlogNum := len(inpath[0].GetBinlogs()) - assert.Equal(t, 1, binlogNum) - - for idx := 0; idx < binlogNum; idx++ { - var ps []string - for _, path := range inpath { - ps = append(ps, path.GetBinlogs()[idx].GetLogPath()) - } - allPaths = append(allPaths, ps) - } - - dm := map[interface{}]Timestamp{ - 1: 10000, - } - - ct := &compactionTask{ - metaCache: metaCache, - binlogIO: mockbIO, - Allocator: alloc, - done: make(chan struct{}, 1), - plan: &datapb.CompactionPlan{ - SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ - {SegmentID: 1}, - }, - }, - } - _, _, _, err = ct.merge(context.Background(), allPaths, 2, 0, &etcdpb.CollectionMeta{ - Schema: &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{ - {DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{ - {Key: common.DimKey, Value: "64"}, - }}, - }}, - }, dm) - assert.Error(t, err) - }) - - t.Run("Merge with meta type param error", func(t *testing.T) { - mockbIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0") - iData := genInsertDataWithExpiredTS() - meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) - - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), 0, 1, iData, iCodec) - assert.NoError(t, err) - assert.Equal(t, 12, len(inpath)) - binlogNum := len(inpath[0].GetBinlogs()) - assert.Equal(t, 1, binlogNum) - - for idx := 0; idx < binlogNum; idx++ { - var ps []string - for _, path := range inpath { - ps = append(ps, path.GetBinlogs()[idx].GetLogPath()) - } - allPaths = append(allPaths, ps) - } - - dm := map[interface{}]Timestamp{ - 1: 10000, - } - - ct := &compactionTask{ - metaCache: metaCache, - binlogIO: mockbIO, - Allocator: alloc, - done: make(chan struct{}, 1), - } - - _, _, _, err = ct.merge(context.Background(), allPaths, 2, 0, &etcdpb.CollectionMeta{ - Schema: &schemapb.CollectionSchema{Fields: []*schemapb.FieldSchema{ - {DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{ - {Key: common.DimKey, Value: "bad_dim"}, - }}, - }}, - }, dm) - assert.Error(t, err) - }) - }) - t.Run("Test isExpiredEntity", func(t *testing.T) { - t.Run("When CompactionEntityExpiration is set math.MaxInt64", func(t *testing.T) { - ct := &compactionTask{ - plan: &datapb.CompactionPlan{ - CollectionTtl: math.MaxInt64, - }, - done: make(chan struct{}, 1), - } - - res := ct.isExpiredEntity(0, genTimestamp()) - assert.Equal(t, false, res) - - res = ct.isExpiredEntity(math.MaxInt64, genTimestamp()) - assert.Equal(t, false, res) - - res = ct.isExpiredEntity(0, math.MaxInt64) - assert.Equal(t, true, res) - - res = ct.isExpiredEntity(math.MaxInt64, math.MaxInt64) - assert.Equal(t, false, res) - - res = ct.isExpiredEntity(math.MaxInt64, 0) - assert.Equal(t, false, res) - }) - t.Run("When CompactionEntityExpiration is set MAX_ENTITY_EXPIRATION = 0", func(t *testing.T) { - // 0 means expiration is not enabled - ct := &compactionTask{ - plan: &datapb.CompactionPlan{ - CollectionTtl: 0, - }, - done: make(chan struct{}, 1), - } - res := ct.isExpiredEntity(0, genTimestamp()) - assert.Equal(t, false, res) - - res = ct.isExpiredEntity(math.MaxInt64, genTimestamp()) - assert.Equal(t, false, res) - - res = ct.isExpiredEntity(0, math.MaxInt64) - assert.Equal(t, false, res) - - res = ct.isExpiredEntity(math.MaxInt64, math.MaxInt64) - assert.Equal(t, false, res) - - res = ct.isExpiredEntity(math.MaxInt64, 0) - assert.Equal(t, false, res) - }) - t.Run("When CompactionEntityExpiration is set 10 days", func(t *testing.T) { - // 10 days in seconds - ct := &compactionTask{ - plan: &datapb.CompactionPlan{ - CollectionTtl: 864000, - }, - done: make(chan struct{}, 1), - } - res := ct.isExpiredEntity(0, genTimestamp()) - assert.Equal(t, true, res) - - res = ct.isExpiredEntity(math.MaxInt64, genTimestamp()) - assert.Equal(t, false, res) - - res = ct.isExpiredEntity(0, math.MaxInt64) - assert.Equal(t, true, res) - - res = ct.isExpiredEntity(math.MaxInt64, math.MaxInt64) - assert.Equal(t, false, res) - - res = ct.isExpiredEntity(math.MaxInt64, 0) - assert.Equal(t, false, res) - }) - }) - - t.Run("Test getNumRows error", func(t *testing.T) { - metaCache := metacache.NewMockMetaCache(t) - metaCache.EXPECT().GetSegmentByID(mock.Anything).Return(nil, false) - ct := &compactionTask{ - metaCache: metaCache, - plan: &datapb.CompactionPlan{ - SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ - { - SegmentID: 1, - }, - }, - }, - done: make(chan struct{}, 1), - } - - _, err := ct.getNumRows() - assert.Error(t, err, "segment not found") - }) - - t.Run("Test uploadRemainLog error", func(t *testing.T) { - f := &MetaFactory{} - - t.Run("upload failed", func(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) - defer cancel() - - alloc := allocator.NewMockAllocator(t) - alloc.EXPECT().AllocOne().Call.Return(int64(11111), nil) - - meta := f.GetCollectionMeta(UniqueID(10001), "test_upload_remain_log", schemapb.DataType_Int64) - stats, err := storage.NewPrimaryKeyStats(106, int64(schemapb.DataType_Int64), 10) - - require.NoError(t, err) - - ct := &compactionTask{ - binlogIO: io.NewBinlogIO(&mockCm{errSave: true}, getOrCreateIOPool()), - Allocator: alloc, - done: make(chan struct{}, 1), - } - - _, _, err = ct.uploadRemainLog(ctx, 1, 2, meta, stats, 10, nil) - assert.Error(t, err) - }) - }) -} - -func getInt64DeltaBlobs(segID UniqueID, pks []UniqueID, tss []Timestamp) ([]*Blob, error) { - primaryKeys := make([]storage.PrimaryKey, len(pks)) - for index, v := range pks { - primaryKeys[index] = storage.NewInt64PrimaryKey(v) - } - deltaData := &DeleteData{ - Pks: primaryKeys, - Tss: tss, - RowCount: int64(len(pks)), - } - - dCodec := storage.NewDeleteCodec() - blob, err := dCodec.Serialize(1, 10, segID, deltaData) - return []*Blob{blob}, err -} - -func TestCompactorInterfaceMethods(t *testing.T) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cm := storage.NewLocalChunkManager(storage.RootPath(compactTestDir)) - defer cm.RemoveWithPrefix(ctx, cm.RootPath()) - notEmptySegmentBinlogs := []*datapb.CompactionSegmentBinlogs{{ - SegmentID: 100, - FieldBinlogs: nil, - Field2StatslogPaths: nil, - Deltalogs: nil, - }} - paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0") // Turn off auto expiration - t.Run("Test compact with all segment empty", func(t *testing.T) { - alloc := allocator.NewMockAllocator(t) - alloc.EXPECT().AllocOne().Call.Return(int64(11111), nil) - ctx, cancel := context.WithCancel(context.TODO()) - - mockSyncmgr := syncmgr.NewMockSyncManager(t) - mockSyncmgr.EXPECT().Block(mock.Anything).Return() - task := &compactionTask{ - ctx: ctx, - cancel: cancel, - Allocator: alloc, - done: make(chan struct{}, 1), - tr: timerecord.NewTimeRecorder("test"), - syncMgr: mockSyncmgr, - plan: &datapb.CompactionPlan{ - PlanID: 999, - SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{{SegmentID: 100}}, - TimeoutInSeconds: 10, - Type: datapb.CompactionType_MixCompaction, - }, - } - - _, err := task.compact() - assert.ErrorIs(t, errIllegalCompactionPlan, err) - }) - - t.Run("Test compact invalid empty segment binlogs", func(t *testing.T) { - plan := &datapb.CompactionPlan{ - PlanID: 999, - SegmentBinlogs: nil, - TimeoutInSeconds: 10, - Type: datapb.CompactionType_MixCompaction, - } - ctx, cancel := context.WithCancel(context.Background()) - emptyTask := &compactionTask{ - ctx: ctx, - cancel: cancel, - tr: timerecord.NewTimeRecorder("test"), - - done: make(chan struct{}, 1), - plan: plan, - } - - _, err := emptyTask.compact() - assert.Error(t, err) - assert.ErrorIs(t, err, errIllegalCompactionPlan) - - emptyTask.complete() - emptyTask.stop() - }) - - t.Run("Test compact invalid AllocOnce failed", func(t *testing.T) { - mockAlloc := allocator.NewMockAllocator(t) - mockAlloc.EXPECT().AllocOne().Call.Return(int64(0), errors.New("mock allocone error")).Once() - plan := &datapb.CompactionPlan{ - PlanID: 999, - SegmentBinlogs: notEmptySegmentBinlogs, - TimeoutInSeconds: 10, - Type: datapb.CompactionType_MixCompaction, - } - task := &compactionTask{ - ctx: context.Background(), - tr: timerecord.NewTimeRecorder("test"), - Allocator: mockAlloc, - plan: plan, - } - - _, err := task.compact() - assert.Error(t, err) - }) - - t.Run("Test typeII compact valid", func(t *testing.T) { - alloc := allocator.NewMockAllocator(t) - alloc.EXPECT().GetGenerator(mock.Anything, mock.Anything).Call.Return(validGeneratorFn, nil) - alloc.EXPECT().AllocOne().Call.Return(int64(19530), nil) - type testCase struct { - pkType schemapb.DataType - iData1 storage.FieldData - iData2 storage.FieldData - pks1 [2]storage.PrimaryKey - pks2 [2]storage.PrimaryKey - colID UniqueID - parID UniqueID - segID1 UniqueID - segID2 UniqueID - } - cases := []testCase{ - { - pkType: schemapb.DataType_Int64, - iData1: &storage.Int64FieldData{Data: []UniqueID{1}}, - iData2: &storage.Int64FieldData{Data: []UniqueID{9}}, - pks1: [2]storage.PrimaryKey{storage.NewInt64PrimaryKey(1), storage.NewInt64PrimaryKey(2)}, - pks2: [2]storage.PrimaryKey{storage.NewInt64PrimaryKey(9), storage.NewInt64PrimaryKey(10)}, - colID: 1, - parID: 10, - segID1: 100, - segID2: 101, - }, - { - pkType: schemapb.DataType_VarChar, - iData1: &storage.StringFieldData{Data: []string{"aaaa"}}, - iData2: &storage.StringFieldData{Data: []string{"milvus"}}, - pks1: [2]storage.PrimaryKey{storage.NewVarCharPrimaryKey("aaaa"), storage.NewVarCharPrimaryKey("bbbb")}, - pks2: [2]storage.PrimaryKey{storage.NewVarCharPrimaryKey("milvus"), storage.NewVarCharPrimaryKey("mmmm")}, - colID: 2, - parID: 11, - segID1: 102, - segID2: 103, - }, - } - - for _, c := range cases { - collName := "test_compact_coll_name" - meta := NewMetaFactory().GetCollectionMeta(c.colID, collName, c.pkType) - - mockbIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - mockKv := memkv.NewMemoryKV() - metaCache := metacache.NewMockMetaCache(t) - metaCache.EXPECT().Collection().Return(c.colID) - metaCache.EXPECT().Schema().Return(meta.GetSchema()) - syncMgr := syncmgr.NewMockSyncManager(t) - syncMgr.EXPECT().Block(mock.Anything).Return() - - bfs := metacache.NewBloomFilterSet() - bfs.UpdatePKRange(c.iData1) - seg1 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ - CollectionID: c.colID, - PartitionID: c.parID, - ID: c.segID1, - NumOfRows: 2, - }, bfs) - bfs = metacache.NewBloomFilterSet() - bfs.UpdatePKRange(c.iData2) - seg2 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ - CollectionID: c.colID, - PartitionID: c.parID, - ID: c.segID2, - NumOfRows: 2, - }, bfs) - - bfs = metacache.NewBloomFilterSet() - seg3 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ - CollectionID: c.colID, - PartitionID: c.parID, - ID: 99999, - }, bfs) - - metaCache.EXPECT().GetSegmentByID(c.segID1).Return(seg1, true) - metaCache.EXPECT().GetSegmentByID(c.segID2).Return(seg2, true) - metaCache.EXPECT().GetSegmentByID(seg3.SegmentID()).Return(seg3, true) - metaCache.EXPECT().GetSegmentByID(mock.Anything).Return(nil, false) - - iData1 := genInsertDataWithPKs(c.pks1, c.pkType) - dData1 := &DeleteData{ - Pks: []storage.PrimaryKey{c.pks1[0]}, - Tss: []Timestamp{20000}, - RowCount: 1, - } - iData2 := genInsertDataWithPKs(c.pks2, c.pkType) - dData2 := &DeleteData{ - Pks: []storage.PrimaryKey{c.pks2[0]}, - Tss: []Timestamp{30000}, - RowCount: 1, - } - - stats1, err := storage.NewPrimaryKeyStats(1, int64(c.pkType), 1) - require.NoError(t, err) - iPaths1, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), c.parID, c.segID1, iData1, iCodec) - require.NoError(t, err) - sPaths1, err := uploadStatsLog(context.Background(), mockbIO, alloc, meta.GetID(), c.parID, c.segID1, stats1, 2, iCodec) - require.NoError(t, err) - dPaths1, err := uploadDeltaLog(context.TODO(), mockbIO, alloc, meta.GetID(), c.parID, c.segID1, dData1) - require.NoError(t, err) - require.Equal(t, 12, len(iPaths1)) - - stats2, err := storage.NewPrimaryKeyStats(1, int64(c.pkType), 1) - require.NoError(t, err) - iPaths2, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), c.parID, c.segID2, iData2, iCodec) - require.NoError(t, err) - sPaths2, err := uploadStatsLog(context.Background(), mockbIO, alloc, meta.GetID(), c.parID, c.segID2, stats2, 2, iCodec) - require.NoError(t, err) - dPaths2, err := uploadDeltaLog(context.TODO(), mockbIO, alloc, meta.GetID(), c.parID, c.segID2, dData2) - require.NoError(t, err) - require.Equal(t, 12, len(iPaths2)) - - plan := &datapb.CompactionPlan{ - PlanID: 10080, - SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ - { - SegmentID: c.segID1, - FieldBinlogs: lo.Values(iPaths1), - Field2StatslogPaths: lo.Values(sPaths1), - Deltalogs: dPaths1, - }, - { - SegmentID: c.segID2, - FieldBinlogs: lo.Values(iPaths2), - Field2StatslogPaths: lo.Values(sPaths2), - Deltalogs: dPaths2, - }, - { - SegmentID: seg3.SegmentID(), // empty segment - }, - }, - StartTime: 0, - TimeoutInSeconds: 10, - Type: datapb.CompactionType_MergeCompaction, - Channel: "channelname", - } - - task := newCompactionTask(context.TODO(), mockbIO, metaCache, syncMgr, alloc, plan) - result, err := task.compact() - assert.NoError(t, err) - assert.NotNil(t, result) - - assert.Equal(t, plan.GetPlanID(), result.GetPlanID()) - assert.Equal(t, 1, len(result.GetSegments())) - - segment := result.GetSegments()[0] - assert.EqualValues(t, 19530, segment.GetSegmentID()) - assert.EqualValues(t, 2, segment.GetNumOfRows()) - assert.NotEmpty(t, segment.InsertLogs) - assert.NotEmpty(t, segment.Field2StatslogPaths) - - // New test, remove all the binlogs in memkv - err = mockKv.RemoveWithPrefix("/") - require.NoError(t, err) - plan.PlanID++ - - result, err = task.compact() - assert.NoError(t, err) - assert.NotNil(t, result) - - assert.Equal(t, plan.GetPlanID(), result.GetPlanID()) - assert.Equal(t, 1, len(result.GetSegments())) - - segment = result.GetSegments()[0] - assert.EqualValues(t, 19530, segment.GetSegmentID()) - assert.EqualValues(t, 2, segment.GetNumOfRows()) - assert.NotEmpty(t, segment.InsertLogs) - assert.NotEmpty(t, segment.Field2StatslogPaths) - } - }) - - t.Run("Test typeII compact 2 segments with the same pk", func(t *testing.T) { - // Test merge compactions, two segments with the same pk, one deletion pk=1 - // The merged segment 19530 should only contain 2 rows and both pk=2 - // Both pk = 1 rows of the two segments are compacted. - var collID, partID, segID1, segID2 UniqueID = 1, 10, 200, 201 - - alloc := allocator.NewMockAllocator(t) - alloc.EXPECT().AllocOne().Call.Return(int64(19530), nil) - alloc.EXPECT().GetGenerator(mock.Anything, mock.Anything).Call.Return(validGeneratorFn, nil) - - meta := NewMetaFactory().GetCollectionMeta(collID, "test_compact_coll_name", schemapb.DataType_Int64) - - mockbIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - - metaCache := metacache.NewMockMetaCache(t) - metaCache.EXPECT().Collection().Return(collID) - metaCache.EXPECT().Schema().Return(meta.GetSchema()) - syncMgr := syncmgr.NewMockSyncManager(t) - syncMgr.EXPECT().Block(mock.Anything).Return() - - bfs := metacache.NewBloomFilterSet() - bfs.UpdatePKRange(&storage.Int64FieldData{Data: []UniqueID{1}}) - seg1 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ - CollectionID: collID, - PartitionID: partID, - ID: segID1, - NumOfRows: 2, - }, bfs) - bfs = metacache.NewBloomFilterSet() - bfs.UpdatePKRange(&storage.Int64FieldData{Data: []UniqueID{1}}) - seg2 := metacache.NewSegmentInfo(&datapb.SegmentInfo{ - CollectionID: collID, - PartitionID: partID, - ID: segID2, - NumOfRows: 2, - }, bfs) - - metaCache.EXPECT().GetSegmentByID(mock.Anything).RunAndReturn(func(id int64, filters ...metacache.SegmentFilter) (*metacache.SegmentInfo, bool) { - switch id { - case segID1: - return seg1, true - case segID2: - return seg2, true - default: - return nil, false - } - }) - - // the same pk for segmentI and segmentII - pks := [2]storage.PrimaryKey{storage.NewInt64PrimaryKey(1), storage.NewInt64PrimaryKey(2)} - iData1 := genInsertDataWithPKs(pks, schemapb.DataType_Int64) - iData2 := genInsertDataWithPKs(pks, schemapb.DataType_Int64) - - pk1 := storage.NewInt64PrimaryKey(1) - dData1 := &DeleteData{ - Pks: []storage.PrimaryKey{pk1}, - Tss: []Timestamp{20000}, - RowCount: 1, - } - // empty dData2 - dData2 := &DeleteData{ - Pks: []storage.PrimaryKey{}, - Tss: []Timestamp{}, - RowCount: 0, - } - - stats1, err := storage.NewPrimaryKeyStats(1, int64(schemapb.DataType_Int64), 1) - require.NoError(t, err) - iPaths1, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), partID, segID1, iData1, iCodec) - require.NoError(t, err) - sPaths1, err := uploadStatsLog(context.Background(), mockbIO, alloc, meta.GetID(), partID, segID1, stats1, 1, iCodec) - require.NoError(t, err) - dPaths1, err := uploadDeltaLog(context.TODO(), mockbIO, alloc, meta.GetID(), partID, segID1, dData1) - require.NoError(t, err) - require.Equal(t, 12, len(iPaths1)) - - stats2, err := storage.NewPrimaryKeyStats(1, int64(schemapb.DataType_Int64), 1) - require.NoError(t, err) - iPaths2, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), partID, segID2, iData2, iCodec) - require.NoError(t, err) - sPaths2, err := uploadStatsLog(context.Background(), mockbIO, alloc, meta.GetID(), partID, segID2, stats2, 1, iCodec) - require.NoError(t, err) - dPaths2, err := uploadDeltaLog(context.TODO(), mockbIO, alloc, meta.GetID(), partID, segID2, dData2) - require.NoError(t, err) - require.Equal(t, 12, len(iPaths2)) - - plan := &datapb.CompactionPlan{ - PlanID: 20080, - SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ - { - SegmentID: segID1, - FieldBinlogs: lo.Values(iPaths1), - Field2StatslogPaths: lo.Values(sPaths1), - Deltalogs: dPaths1, - }, - { - SegmentID: segID2, - FieldBinlogs: lo.Values(iPaths2), - Field2StatslogPaths: lo.Values(sPaths2), - Deltalogs: dPaths2, - }, - }, - StartTime: 0, - TimeoutInSeconds: 10, - Type: datapb.CompactionType_MergeCompaction, - Channel: "channelname", - } - - task := newCompactionTask(context.TODO(), mockbIO, metaCache, syncMgr, alloc, plan) - result, err := task.compact() - assert.NoError(t, err) - assert.NotNil(t, result) - - assert.Equal(t, plan.GetPlanID(), result.GetPlanID()) - assert.Equal(t, 1, len(result.GetSegments())) - - segment := result.GetSegments()[0] - assert.EqualValues(t, 19530, segment.GetSegmentID()) - assert.EqualValues(t, 2, segment.GetNumOfRows()) - assert.NotEmpty(t, segment.InsertLogs) - assert.NotEmpty(t, segment.Field2StatslogPaths) - }) -} - -func TestInjectDone(t *testing.T) { - syncMgr := syncmgr.NewMockSyncManager(t) - - segmentIDs := []int64{100, 200, 300} - task := &compactionTask{ - plan: &datapb.CompactionPlan{ - SegmentBinlogs: lo.Map(segmentIDs, func(id int64, _ int) *datapb.CompactionSegmentBinlogs { - return &datapb.CompactionSegmentBinlogs{SegmentID: id} - }), - }, - syncMgr: syncMgr, - } - - for _, segmentID := range segmentIDs { - syncMgr.EXPECT().Unblock(segmentID).Return().Once() - } - - task.injectDone() - task.injectDone() -} diff --git a/internal/datanode/data_node.go b/internal/datanode/data_node.go index 2ac347cb452ec..89734ec8ea9d7 100644 --- a/internal/datanode/data_node.go +++ b/internal/datanode/data_node.go @@ -297,7 +297,6 @@ func (node *DataNode) Init() error { } else { node.eventManager = NewEventManager() } - log.Info("init datanode done", zap.String("Address", node.address)) }) return initError diff --git a/internal/datanode/data_sync_service.go b/internal/datanode/data_sync_service.go index 1620d25734ae5..73dcd166e5433 100644 --- a/internal/datanode/data_sync_service.go +++ b/internal/datanode/data_sync_service.go @@ -19,17 +19,16 @@ package datanode import ( "context" "fmt" - "path" "sync" - "time" "go.uber.org/zap" - "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" "github.com/milvus-io/milvus/internal/datanode/allocator" "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/datanode/io" "github.com/milvus-io/milvus/internal/datanode/metacache" "github.com/milvus-io/milvus/internal/datanode/syncmgr" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/datanode/writebuffer" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/querycoordv2/params" @@ -157,13 +156,13 @@ func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2C ) segment := item - future := getOrCreateIOPool().Submit(func() (any, error) { + future := io.GetOrCreateStatsPool().Submit(func() (any, error) { var stats []*storage.PkStatistics var err error if params.Params.CommonCfg.EnableStorageV2.GetAsBool() { - stats, err = loadStatsV2(storageV2Cache, segment, info.GetSchema()) + stats, err = util.LoadStatsV2(storageV2Cache, segment, info.GetSchema()) } else { - stats, err = loadStats(initCtx, chunkManager, info.GetSchema(), segment.GetID(), segment.GetStatslogs()) + stats, err = util.LoadStats(initCtx, chunkManager, info.GetSchema(), segment.GetID(), segment.GetStatslogs()) } if err != nil { return nil, err @@ -198,143 +197,6 @@ func initMetaCache(initCtx context.Context, storageV2Cache *metacache.StorageV2C return metacache, nil } -func loadStatsV2(storageCache *metacache.StorageV2Cache, segment *datapb.SegmentInfo, schema *schemapb.CollectionSchema) ([]*storage.PkStatistics, error) { - space, err := storageCache.GetOrCreateSpace(segment.ID, syncmgr.SpaceCreatorFunc(segment.ID, schema, storageCache.ArrowSchema())) - if err != nil { - return nil, err - } - - getResult := func(stats []*storage.PrimaryKeyStats) []*storage.PkStatistics { - result := make([]*storage.PkStatistics, 0, len(stats)) - for _, stat := range stats { - pkStat := &storage.PkStatistics{ - PkFilter: stat.BF, - MinPK: stat.MinPk, - MaxPK: stat.MaxPk, - } - result = append(result, pkStat) - } - return result - } - - blobs := space.StatisticsBlobs() - deserBlobs := make([]*Blob, 0) - for _, b := range blobs { - if b.Name == storage.CompoundStatsType.LogIdx() { - blobData := make([]byte, b.Size) - _, err = space.ReadBlob(b.Name, blobData) - if err != nil { - return nil, err - } - stats, err := storage.DeserializeStatsList(&Blob{Value: blobData}) - if err != nil { - return nil, err - } - return getResult(stats), nil - } - } - - for _, b := range blobs { - blobData := make([]byte, b.Size) - _, err = space.ReadBlob(b.Name, blobData) - if err != nil { - return nil, err - } - deserBlobs = append(deserBlobs, &Blob{Value: blobData}) - } - stats, err := storage.DeserializeStats(deserBlobs) - if err != nil { - return nil, err - } - return getResult(stats), nil -} - -func loadStats(ctx context.Context, chunkManager storage.ChunkManager, schema *schemapb.CollectionSchema, segmentID int64, statsBinlogs []*datapb.FieldBinlog) ([]*storage.PkStatistics, error) { - startTs := time.Now() - log := log.With(zap.Int64("segmentID", segmentID)) - log.Info("begin to init pk bloom filter", zap.Int("statsBinLogsLen", len(statsBinlogs))) - - // get pkfield id - pkField := int64(-1) - for _, field := range schema.Fields { - if field.IsPrimaryKey { - pkField = field.FieldID - break - } - } - - // filter stats binlog files which is pk field stats log - bloomFilterFiles := []string{} - logType := storage.DefaultStatsType - - for _, binlog := range statsBinlogs { - if binlog.FieldID != pkField { - continue - } - Loop: - for _, log := range binlog.GetBinlogs() { - _, logidx := path.Split(log.GetLogPath()) - // if special status log exist - // only load one file - switch logidx { - case storage.CompoundStatsType.LogIdx(): - bloomFilterFiles = []string{log.GetLogPath()} - logType = storage.CompoundStatsType - break Loop - default: - bloomFilterFiles = append(bloomFilterFiles, log.GetLogPath()) - } - } - } - - // no stats log to parse, initialize a new BF - if len(bloomFilterFiles) == 0 { - log.Warn("no stats files to load") - return nil, nil - } - - // read historical PK filter - values, err := chunkManager.MultiRead(ctx, bloomFilterFiles) - if err != nil { - log.Warn("failed to load bloom filter files", zap.Error(err)) - return nil, err - } - blobs := make([]*Blob, 0) - for i := 0; i < len(values); i++ { - blobs = append(blobs, &Blob{Value: values[i]}) - } - - var stats []*storage.PrimaryKeyStats - if logType == storage.CompoundStatsType { - stats, err = storage.DeserializeStatsList(blobs[0]) - if err != nil { - log.Warn("failed to deserialize stats list", zap.Error(err)) - return nil, err - } - } else { - stats, err = storage.DeserializeStats(blobs) - if err != nil { - log.Warn("failed to deserialize stats", zap.Error(err)) - return nil, err - } - } - - var size uint - result := make([]*storage.PkStatistics, 0, len(stats)) - for _, stat := range stats { - pkStat := &storage.PkStatistics{ - PkFilter: stat.BF, - MinPK: stat.MinPk, - MaxPK: stat.MaxPk, - } - size += stat.BF.Cap() - result = append(result, pkStat) - } - - log.Info("Successfully load pk stats", zap.Any("time", time.Since(startTs)), zap.Uint("size", size)) - return result, nil -} - func getServiceWithChannel(initCtx context.Context, node *DataNode, info *datapb.ChannelWatchInfo, metacache metacache.MetaCache, storageV2Cache *metacache.StorageV2Cache, unflushed, flushed []*datapb.SegmentInfo) (*dataSyncService, error) { var ( channelName = info.GetVchan().GetChannelName() diff --git a/internal/datanode/io/binlog_io.go b/internal/datanode/io/binlog_io.go index bd470d21e6b9f..61a2ccf97615f 100644 --- a/internal/datanode/io/binlog_io.go +++ b/internal/datanode/io/binlog_io.go @@ -18,7 +18,6 @@ package io import ( "context" - "path" "github.com/samber/lo" "go.opentelemetry.io/otel" @@ -34,8 +33,6 @@ import ( type BinlogIO interface { Download(ctx context.Context, paths []string) ([][]byte, error) Upload(ctx context.Context, kvs map[string][]byte) error - // JoinFullPath returns the full path by join the paths with the chunkmanager's rootpath - JoinFullPath(paths ...string) string } type BinlogIoImpl struct { @@ -43,8 +40,8 @@ type BinlogIoImpl struct { pool *conc.Pool[any] } -func NewBinlogIO(cm storage.ChunkManager, ioPool *conc.Pool[any]) BinlogIO { - return &BinlogIoImpl{cm, ioPool} +func NewBinlogIO(cm storage.ChunkManager) BinlogIO { + return &BinlogIoImpl{cm, GetOrCreateIOPool()} } func (b *BinlogIoImpl) Download(ctx context.Context, paths []string) ([][]byte, error) { @@ -99,16 +96,10 @@ func (b *BinlogIoImpl) Upload(ctx context.Context, kvs map[string][]byte) error } return err }) - return struct{}{}, err }) - futures = append(futures, future) } return conc.AwaitAll(futures...) } - -func (b *BinlogIoImpl) JoinFullPath(paths ...string) string { - return path.Join(b.ChunkManager.RootPath(), path.Join(paths...)) -} diff --git a/internal/datanode/io/binlog_io_test.go b/internal/datanode/io/binlog_io_test.go index 70ad89b69b5fc..df5cc6fbe1605 100644 --- a/internal/datanode/io/binlog_io_test.go +++ b/internal/datanode/io/binlog_io_test.go @@ -9,7 +9,7 @@ import ( "golang.org/x/net/context" "github.com/milvus-io/milvus/internal/storage" - "github.com/milvus-io/milvus/pkg/util/conc" + "github.com/milvus-io/milvus/pkg/util/paramtable" ) const binlogIOTestDir = "/tmp/milvus_test/binlog_io" @@ -26,11 +26,10 @@ type BinlogIOSuite struct { } func (s *BinlogIOSuite) SetupTest() { - pool := conc.NewDefaultPool[any]() - + paramtable.Init() s.cm = storage.NewLocalChunkManager(storage.RootPath(binlogIOTestDir)) - s.b = NewBinlogIO(s.cm, pool) + s.b = NewBinlogIO(s.cm) } func (s *BinlogIOSuite) TeardownTest() { @@ -52,22 +51,3 @@ func (s *BinlogIOSuite) TestUploadDownload() { s.NoError(err) s.ElementsMatch(lo.Values(kvs), vs) } - -func (s *BinlogIOSuite) TestJoinFullPath() { - tests := []struct { - description string - inPaths []string - outPath string - }{ - {"no input", nil, path.Join(binlogIOTestDir)}, - {"input one", []string{"a"}, path.Join(binlogIOTestDir, "a")}, - {"input two", []string{"a", "b"}, path.Join(binlogIOTestDir, "a/b")}, - } - - for _, test := range tests { - s.Run(test.description, func() { - out := s.b.JoinFullPath(test.inPaths...) - s.Equal(test.outPath, out) - }) - } -} diff --git a/internal/datanode/io/io_pool.go b/internal/datanode/io/io_pool.go index 3ac18a32d6d27..c961de82917dc 100644 --- a/internal/datanode/io/io_pool.go +++ b/internal/datanode/io/io_pool.go @@ -65,3 +65,54 @@ func GetBFApplyPool() *conc.Pool[any] { initBFApplyPool() return bfApplyPool.Load() } + +var ( + ioPool *conc.Pool[any] + ioPoolInitOnce sync.Once +) + +var ( + statsPool *conc.Pool[any] + statsPoolInitOnce sync.Once +) + +func initIOPool() { + capacity := paramtable.Get().DataNodeCfg.IOConcurrency.GetAsInt() + if capacity > 32 { + capacity = 32 + } + // error only happens with negative expiry duration or with negative pre-alloc size. + ioPool = conc.NewPool[any](capacity) +} + +func GetOrCreateIOPool() *conc.Pool[any] { + ioPoolInitOnce.Do(initIOPool) + return ioPool +} + +func initStatsPool() { + poolSize := paramtable.Get().DataNodeCfg.ChannelWorkPoolSize.GetAsInt() + if poolSize <= 0 { + poolSize = hardware.GetCPUNum() + } + statsPool = conc.NewPool[any](poolSize, conc.WithPreAlloc(false), conc.WithNonBlocking(false)) +} + +func GetOrCreateStatsPool() *conc.Pool[any] { + statsPoolInitOnce.Do(initStatsPool) + return statsPool +} + +func initMultiReadPool() { + capacity := paramtable.Get().DataNodeCfg.FileReadConcurrency.GetAsInt() + if capacity > hardware.GetCPUNum() { + capacity = hardware.GetCPUNum() + } + // error only happens with negative expiry duration or with negative pre-alloc size. + ioPool = conc.NewPool[any](capacity) +} + +func getMultiReadPool() *conc.Pool[any] { + ioPoolInitOnce.Do(initMultiReadPool) + return ioPool +} diff --git a/internal/datanode/io/io_pool_test.go b/internal/datanode/io/io_pool_test.go index 763a1a63ab7c2..c1b57a5ea568e 100644 --- a/internal/datanode/io/io_pool_test.go +++ b/internal/datanode/io/io_pool_test.go @@ -2,11 +2,13 @@ package io import ( "strconv" + "sync" "testing" "github.com/stretchr/testify/assert" "github.com/milvus-io/milvus/pkg/config" + "github.com/milvus-io/milvus/pkg/util/conc" "github.com/milvus-io/milvus/pkg/util/hardware" "github.com/milvus-io/milvus/pkg/util/paramtable" ) @@ -40,3 +42,30 @@ func TestResizePools(t *testing.T) { assert.Equal(t, expectedCap, GetBFApplyPool().Cap()) }) } + +func TestGetOrCreateIOPool(t *testing.T) { + paramtable.Init() + ioConcurrency := paramtable.Get().DataNodeCfg.IOConcurrency.GetValue() + paramtable.Get().Save(paramtable.Get().DataNodeCfg.IOConcurrency.Key, "64") + defer func() { paramtable.Get().Save(paramtable.Get().DataNodeCfg.IOConcurrency.Key, ioConcurrency) }() + nP := 10 + nTask := 10 + wg := sync.WaitGroup{} + for i := 0; i < nP; i++ { + wg.Add(1) + go func() { + defer wg.Done() + p := GetOrCreateIOPool() + futures := make([]*conc.Future[any], 0, nTask) + for j := 0; j < nTask; j++ { + future := p.Submit(func() (interface{}, error) { + return nil, nil + }) + futures = append(futures, future) + } + err := conc.AwaitAll(futures...) + assert.NoError(t, err) + }() + } + wg.Wait() +} diff --git a/internal/datanode/io/mock_binlogio.go b/internal/datanode/io/mock_binlogio.go index 4202a7ed55679..b0132f16299a7 100644 --- a/internal/datanode/io/mock_binlogio.go +++ b/internal/datanode/io/mock_binlogio.go @@ -76,61 +76,6 @@ func (_c *MockBinlogIO_Download_Call) RunAndReturn(run func(context.Context, []s return _c } -// JoinFullPath provides a mock function with given fields: paths -func (_m *MockBinlogIO) JoinFullPath(paths ...string) string { - _va := make([]interface{}, len(paths)) - for _i := range paths { - _va[_i] = paths[_i] - } - var _ca []interface{} - _ca = append(_ca, _va...) - ret := _m.Called(_ca...) - - var r0 string - if rf, ok := ret.Get(0).(func(...string) string); ok { - r0 = rf(paths...) - } else { - r0 = ret.Get(0).(string) - } - - return r0 -} - -// MockBinlogIO_JoinFullPath_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'JoinFullPath' -type MockBinlogIO_JoinFullPath_Call struct { - *mock.Call -} - -// JoinFullPath is a helper method to define mock.On call -// - paths ...string -func (_e *MockBinlogIO_Expecter) JoinFullPath(paths ...interface{}) *MockBinlogIO_JoinFullPath_Call { - return &MockBinlogIO_JoinFullPath_Call{Call: _e.mock.On("JoinFullPath", - append([]interface{}{}, paths...)...)} -} - -func (_c *MockBinlogIO_JoinFullPath_Call) Run(run func(paths ...string)) *MockBinlogIO_JoinFullPath_Call { - _c.Call.Run(func(args mock.Arguments) { - variadicArgs := make([]string, len(args)-0) - for i, a := range args[0:] { - if a != nil { - variadicArgs[i] = a.(string) - } - } - run(variadicArgs...) - }) - return _c -} - -func (_c *MockBinlogIO_JoinFullPath_Call) Return(_a0 string) *MockBinlogIO_JoinFullPath_Call { - _c.Call.Return(_a0) - return _c -} - -func (_c *MockBinlogIO_JoinFullPath_Call) RunAndReturn(run func(...string) string) *MockBinlogIO_JoinFullPath_Call { - _c.Call.Return(run) - return _c -} - // Upload provides a mock function with given fields: ctx, kvs func (_m *MockBinlogIO) Upload(ctx context.Context, kvs map[string][]byte) error { ret := _m.Called(ctx, kvs) diff --git a/internal/datanode/io_pool.go b/internal/datanode/io_pool.go deleted file mode 100644 index 892012a0d975e..0000000000000 --- a/internal/datanode/io_pool.go +++ /dev/null @@ -1,59 +0,0 @@ -package datanode - -import ( - "sync" - - "github.com/milvus-io/milvus/pkg/util/conc" - "github.com/milvus-io/milvus/pkg/util/hardware" -) - -var ( - ioPool *conc.Pool[any] - ioPoolInitOnce sync.Once -) - -var ( - statsPool *conc.Pool[any] - statsPoolInitOnce sync.Once -) - -func initIOPool() { - capacity := Params.DataNodeCfg.IOConcurrency.GetAsInt() - if capacity > 32 { - capacity = 32 - } - // error only happens with negative expiry duration or with negative pre-alloc size. - ioPool = conc.NewPool[any](capacity) -} - -func getOrCreateIOPool() *conc.Pool[any] { - ioPoolInitOnce.Do(initIOPool) - return ioPool -} - -func initStatsPool() { - poolSize := Params.DataNodeCfg.ChannelWorkPoolSize.GetAsInt() - if poolSize <= 0 { - poolSize = hardware.GetCPUNum() - } - statsPool = conc.NewPool[any](poolSize, conc.WithPreAlloc(false), conc.WithNonBlocking(false)) -} - -func getOrCreateStatsPool() *conc.Pool[any] { - statsPoolInitOnce.Do(initStatsPool) - return statsPool -} - -func initMultiReadPool() { - capacity := Params.DataNodeCfg.FileReadConcurrency.GetAsInt() - if capacity > hardware.GetCPUNum() { - capacity = hardware.GetCPUNum() - } - // error only happens with negative expiry duration or with negative pre-alloc size. - ioPool = conc.NewPool[any](capacity) -} - -func getMultiReadPool() *conc.Pool[any] { - ioPoolInitOnce.Do(initMultiReadPool) - return ioPool -} diff --git a/internal/datanode/io_pool_test.go b/internal/datanode/io_pool_test.go deleted file mode 100644 index 20abbcbeca070..0000000000000 --- a/internal/datanode/io_pool_test.go +++ /dev/null @@ -1,37 +0,0 @@ -package datanode - -import ( - "sync" - "testing" - - "github.com/stretchr/testify/assert" - - "github.com/milvus-io/milvus/pkg/util/conc" - "github.com/milvus-io/milvus/pkg/util/paramtable" -) - -func Test_getOrCreateIOPool(t *testing.T) { - ioConcurrency := Params.DataNodeCfg.IOConcurrency.GetValue() - paramtable.Get().Save(Params.DataNodeCfg.IOConcurrency.Key, "64") - defer func() { Params.Save(Params.DataNodeCfg.IOConcurrency.Key, ioConcurrency) }() - nP := 10 - nTask := 10 - wg := sync.WaitGroup{} - for i := 0; i < nP; i++ { - wg.Add(1) - go func() { - defer wg.Done() - p := getOrCreateIOPool() - futures := make([]*conc.Future[any], 0, nTask) - for j := 0; j < nTask; j++ { - future := p.Submit(func() (interface{}, error) { - return nil, nil - }) - futures = append(futures, future) - } - err := conc.AwaitAll(futures...) - assert.NoError(t, err) - }() - } - wg.Wait() -} diff --git a/internal/datanode/mock_test.go b/internal/datanode/mock_test.go index ab9a99ad8f2f3..92286c4bbfcb0 100644 --- a/internal/datanode/mock_test.go +++ b/internal/datanode/mock_test.go @@ -1188,57 +1188,6 @@ func genEmptyInsertData() *InsertData { } } -func genInsertDataWithExpiredTS() *InsertData { - return &InsertData{ - Data: map[int64]storage.FieldData{ - 0: &storage.Int64FieldData{ - Data: []int64{11, 22}, - }, - 1: &storage.Int64FieldData{ - Data: []int64{329749364736000000, 329500223078400000}, // 2009-11-10 23:00:00 +0000 UTC, 2009-10-31 23:00:00 +0000 UTC - }, - 100: &storage.FloatVectorFieldData{ - Data: []float32{1.0, 6.0, 7.0, 8.0}, - Dim: 2, - }, - 101: &storage.BinaryVectorFieldData{ - Data: []byte{0, 255, 255, 255, 128, 128, 128, 0}, - Dim: 32, - }, - 102: &storage.BoolFieldData{ - Data: []bool{true, false}, - }, - 103: &storage.Int8FieldData{ - Data: []int8{5, 6}, - }, - 104: &storage.Int16FieldData{ - Data: []int16{7, 8}, - }, - 105: &storage.Int32FieldData{ - Data: []int32{9, 10}, - }, - 106: &storage.Int64FieldData{ - Data: []int64{1, 2}, - }, - 107: &storage.FloatFieldData{ - Data: []float32{2.333, 2.334}, - }, - 108: &storage.DoubleFieldData{ - Data: []float64{3.333, 3.334}, - }, - 109: &storage.StringFieldData{ - Data: []string{"test1", "test2"}, - }, - }, - } -} - -func genTimestamp() typeutil.Timestamp { - // Generate birthday of Golang - gb := time.Date(2009, time.Month(11), 10, 23, 0, 0, 0, time.UTC) - return tsoutil.ComposeTSByTime(gb, 0) -} - func genTestTickler() *etcdTickler { return newEtcdTickler(0, "", nil, nil, 0) } diff --git a/internal/datanode/segment_writer.go b/internal/datanode/segment_writer.go deleted file mode 100644 index a2b971a2677bd..0000000000000 --- a/internal/datanode/segment_writer.go +++ /dev/null @@ -1,81 +0,0 @@ -package datanode - -import ( - "math" - - "github.com/milvus-io/milvus/internal/datanode/writebuffer" - "github.com/milvus-io/milvus/internal/storage" - "github.com/milvus-io/milvus/pkg/util/typeutil" -) - -func NewSegmentDeltaWriter(segmentID, partitionID, collectionID int64) *SegmentDeltaWriter { - return &SegmentDeltaWriter{ - deleteData: &storage.DeleteData{}, - segmentID: segmentID, - partitionID: partitionID, - collectionID: collectionID, - tsFrom: math.MaxUint64, - tsTo: 0, - } -} - -type SegmentDeltaWriter struct { - deleteData *storage.DeleteData - segmentID int64 - partitionID int64 - collectionID int64 - - tsFrom typeutil.Timestamp - tsTo typeutil.Timestamp -} - -func (w *SegmentDeltaWriter) GetCollectionID() int64 { - return w.collectionID -} - -func (w *SegmentDeltaWriter) GetPartitionID() int64 { - return w.partitionID -} - -func (w *SegmentDeltaWriter) GetSegmentID() int64 { - return w.segmentID -} - -func (w *SegmentDeltaWriter) GetRowNum() int64 { - return w.deleteData.RowCount -} - -func (w *SegmentDeltaWriter) GetTimeRange() *writebuffer.TimeRange { - return writebuffer.NewTimeRange(w.tsFrom, w.tsTo) -} - -func (w *SegmentDeltaWriter) updateRange(ts typeutil.Timestamp) { - if ts < w.tsFrom { - w.tsFrom = ts - } - if ts > w.tsTo { - w.tsTo = ts - } -} - -func (w *SegmentDeltaWriter) Write(pk storage.PrimaryKey, ts typeutil.Timestamp) { - w.deleteData.Append(pk, ts) - w.updateRange(ts) -} - -func (w *SegmentDeltaWriter) WriteBatch(pks []storage.PrimaryKey, tss []typeutil.Timestamp) { - w.deleteData.AppendBatch(pks, tss) - - for _, ts := range tss { - w.updateRange(ts) - } -} - -func (w *SegmentDeltaWriter) Finish() (*storage.Blob, *writebuffer.TimeRange, error) { - blob, err := storage.NewDeleteCodec().Serialize(w.collectionID, w.partitionID, w.segmentID, w.deleteData) - if err != nil { - return nil, nil, err - } - - return blob, w.GetTimeRange(), nil -} diff --git a/internal/datanode/services.go b/internal/datanode/services.go index cce3e04042162..6ae4154cd56ce 100644 --- a/internal/datanode/services.go +++ b/internal/datanode/services.go @@ -27,9 +27,11 @@ import ( "github.com/milvus-io/milvus-proto/go-api/v2/commonpb" "github.com/milvus-io/milvus-proto/go-api/v2/milvuspb" + "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/datanode/importv2" "github.com/milvus-io/milvus/internal/datanode/io" "github.com/milvus-io/milvus/internal/datanode/metacache" + "github.com/milvus-io/milvus/internal/datanode/util" "github.com/milvus-io/milvus/internal/metastore/kv/binlog" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" @@ -236,11 +238,11 @@ func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan taskCtx := trace.ContextWithSpanContext(node.ctx, spanCtx)*/ taskCtx := tracer.Propagate(ctx, node.ctx) - var task compactor + var task compaction.Compactor + binlogIO := io.NewBinlogIO(node.chunkManager) switch req.GetType() { case datapb.CompactionType_Level0DeleteCompaction: - binlogIO := io.NewBinlogIO(node.chunkManager, getOrCreateIOPool()) - task = newLevelZeroCompactionTask( + task = compaction.NewLevelZeroCompactionTask( taskCtx, binlogIO, node.allocator, @@ -249,8 +251,7 @@ func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan req, ) case datapb.CompactionType_MixCompaction: - binlogIO := io.NewBinlogIO(node.chunkManager, getOrCreateIOPool()) - task = newCompactionTask( + task = compaction.NewMixCompactionTask( taskCtx, binlogIO, ds.metacache, @@ -314,16 +315,19 @@ func (node *DataNode) SyncSegments(ctx context.Context, req *datapb.SyncSegments log.Warn("failed to sync segments", zap.Error(err)) return merr.Status(err), nil } + err := binlog.DecompressBinLog(storage.StatsBinlog, req.GetCollectionId(), req.GetPartitionId(), req.GetCompactedTo(), req.GetStatsLogs()) if err != nil { log.Warn("failed to DecompressBinLog", zap.Error(err)) return merr.Status(err), nil } - pks, err := loadStats(ctx, node.chunkManager, ds.metacache.Schema(), req.GetCompactedTo(), req.GetStatsLogs()) + + pks, err := util.LoadStats(ctx, node.chunkManager, ds.metacache.Schema(), req.GetCompactedTo(), req.GetStatsLogs()) if err != nil { - log.Warn("failed to load segment statslog", zap.Error(err)) + log.Warn("failed to load segment stats log", zap.Error(err)) return merr.Status(err), nil } + bfs := metacache.NewBloomFilterSet(pks...) ds.metacache.CompactSegments(req.GetCompactedTo(), req.GetPartitionId(), req.GetNumOfRows(), bfs, req.GetCompactedFrom()...) node.compactionExecutor.injectDone(req.GetPlanID()) diff --git a/internal/datanode/services_test.go b/internal/datanode/services_test.go index 00a803642a675..3b89d409865de 100644 --- a/internal/datanode/services_test.go +++ b/internal/datanode/services_test.go @@ -34,6 +34,7 @@ import ( allocator2 "github.com/milvus-io/milvus/internal/allocator" "github.com/milvus-io/milvus/internal/datanode/allocator" "github.com/milvus-io/milvus/internal/datanode/broker" + "github.com/milvus-io/milvus/internal/datanode/compaction" "github.com/milvus-io/milvus/internal/datanode/metacache" "github.com/milvus-io/milvus/internal/proto/datapb" "github.com/milvus-io/milvus/internal/proto/internalpb" @@ -159,8 +160,12 @@ func (s *DataNodeServicesSuite) TestGetComponentStates() { func (s *DataNodeServicesSuite) TestGetCompactionState() { s.Run("success", func() { - s.node.compactionExecutor.executing.Insert(int64(3), newMockCompactor(true)) - s.node.compactionExecutor.executing.Insert(int64(2), newMockCompactor(true)) + mockC := compaction.NewMockCompactor(s.T()) + s.node.compactionExecutor.executing.Insert(int64(3), mockC) + + mockC2 := compaction.NewMockCompactor(s.T()) + s.node.compactionExecutor.executing.Insert(int64(2), mockC2) + s.node.compactionExecutor.completed.Insert(int64(1), &datapb.CompactionPlanResult{ PlanID: 1, State: commonpb.CompactionState_Completed, @@ -168,9 +173,16 @@ func (s *DataNodeServicesSuite) TestGetCompactionState() { {SegmentID: 10}, }, }) + + s.node.compactionExecutor.completed.Insert(int64(4), &datapb.CompactionPlanResult{ + PlanID: 4, + Type: datapb.CompactionType_Level0DeleteCompaction, + State: commonpb.CompactionState_Completed, + }) + stat, err := s.node.GetCompactionState(s.ctx, nil) s.Assert().NoError(err) - s.Assert().Equal(3, len(stat.GetResults())) + s.Assert().Equal(4, len(stat.GetResults())) var mu sync.RWMutex cnt := 0 @@ -182,7 +194,7 @@ func (s *DataNodeServicesSuite) TestGetCompactionState() { } } mu.Lock() - s.Assert().Equal(1, cnt) + s.Assert().Equal(2, cnt) mu.Unlock() s.Assert().Equal(1, s.node.compactionExecutor.completed.Len()) diff --git a/internal/datanode/util/load_stats.go b/internal/datanode/util/load_stats.go new file mode 100644 index 0000000000000..f0932329b86c6 --- /dev/null +++ b/internal/datanode/util/load_stats.go @@ -0,0 +1,166 @@ +// Licensed to the LF AI & Data foundation under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package util + +import ( + "context" + "path" + "time" + + "go.uber.org/zap" + + "github.com/milvus-io/milvus-proto/go-api/v2/schemapb" + "github.com/milvus-io/milvus/internal/datanode/metacache" + "github.com/milvus-io/milvus/internal/datanode/syncmgr" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/storage" + "github.com/milvus-io/milvus/pkg/log" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +func LoadStats(ctx context.Context, chunkManager storage.ChunkManager, schema *schemapb.CollectionSchema, segmentID int64, statsBinlogs []*datapb.FieldBinlog) ([]*storage.PkStatistics, error) { + startTs := time.Now() + log := log.With(zap.Int64("segmentID", segmentID)) + log.Info("begin to init pk bloom filter", zap.Int("statsBinLogsLen", len(statsBinlogs))) + + pkField, err := typeutil.GetPrimaryFieldSchema(schema) + if err != nil { + return nil, err + } + + // filter stats binlog files which is pk field stats log + bloomFilterFiles := []string{} + logType := storage.DefaultStatsType + + for _, binlog := range statsBinlogs { + if binlog.FieldID != pkField.GetFieldID() { + continue + } + Loop: + for _, log := range binlog.GetBinlogs() { + _, logidx := path.Split(log.GetLogPath()) + // if special status log exist + // only load one file + switch logidx { + case storage.CompoundStatsType.LogIdx(): + bloomFilterFiles = []string{log.GetLogPath()} + logType = storage.CompoundStatsType + break Loop + default: + bloomFilterFiles = append(bloomFilterFiles, log.GetLogPath()) + } + } + } + + // no stats log to parse, initialize a new BF + if len(bloomFilterFiles) == 0 { + log.Warn("no stats files to load") + return nil, nil + } + + // read historical PK filter + values, err := chunkManager.MultiRead(ctx, bloomFilterFiles) + if err != nil { + log.Warn("failed to load bloom filter files", zap.Error(err)) + return nil, err + } + blobs := make([]*storage.Blob, 0) + for i := 0; i < len(values); i++ { + blobs = append(blobs, &storage.Blob{Value: values[i]}) + } + + var stats []*storage.PrimaryKeyStats + if logType == storage.CompoundStatsType { + stats, err = storage.DeserializeStatsList(blobs[0]) + if err != nil { + log.Warn("failed to deserialize stats list", zap.Error(err)) + return nil, err + } + } else { + stats, err = storage.DeserializeStats(blobs) + if err != nil { + log.Warn("failed to deserialize stats", zap.Error(err)) + return nil, err + } + } + + var size uint + result := make([]*storage.PkStatistics, 0, len(stats)) + for _, stat := range stats { + pkStat := &storage.PkStatistics{ + PkFilter: stat.BF, + MinPK: stat.MinPk, + MaxPK: stat.MaxPk, + } + size += stat.BF.Cap() + result = append(result, pkStat) + } + + log.Info("Successfully load pk stats", zap.Any("time", time.Since(startTs)), zap.Uint("size", size)) + return result, nil +} + +func LoadStatsV2(storageCache *metacache.StorageV2Cache, segment *datapb.SegmentInfo, schema *schemapb.CollectionSchema) ([]*storage.PkStatistics, error) { + space, err := storageCache.GetOrCreateSpace(segment.ID, syncmgr.SpaceCreatorFunc(segment.ID, schema, storageCache.ArrowSchema())) + if err != nil { + return nil, err + } + + getResult := func(stats []*storage.PrimaryKeyStats) []*storage.PkStatistics { + result := make([]*storage.PkStatistics, 0, len(stats)) + for _, stat := range stats { + pkStat := &storage.PkStatistics{ + PkFilter: stat.BF, + MinPK: stat.MinPk, + MaxPK: stat.MaxPk, + } + result = append(result, pkStat) + } + return result + } + + blobs := space.StatisticsBlobs() + deserBlobs := make([]*storage.Blob, 0) + for _, b := range blobs { + if b.Name == storage.CompoundStatsType.LogIdx() { + blobData := make([]byte, b.Size) + _, err = space.ReadBlob(b.Name, blobData) + if err != nil { + return nil, err + } + stats, err := storage.DeserializeStatsList(&storage.Blob{Value: blobData}) + if err != nil { + return nil, err + } + return getResult(stats), nil + } + } + + for _, b := range blobs { + blobData := make([]byte, b.Size) + _, err = space.ReadBlob(b.Name, blobData) + if err != nil { + return nil, err + } + deserBlobs = append(deserBlobs, &storage.Blob{Value: blobData}) + } + stats, err := storage.DeserializeStats(deserBlobs) + if err != nil { + return nil, err + } + return getResult(stats), nil +} diff --git a/internal/datanode/writebuffer/insert_buffer.go b/internal/datanode/writebuffer/insert_buffer.go index adc052d0013b2..417c258b34b44 100644 --- a/internal/datanode/writebuffer/insert_buffer.go +++ b/internal/datanode/writebuffer/insert_buffer.go @@ -67,10 +67,7 @@ func (b *BufferBase) MinTimestamp() typeutil.Timestamp { } func (b *BufferBase) GetTimeRange() *TimeRange { - return &TimeRange{ - timestampMin: b.TimestampFrom, - timestampMax: b.TimestampTo, - } + return NewTimeRange(b.TimestampFrom, b.TimestampTo) } type InsertBuffer struct { @@ -117,16 +114,16 @@ func (ib *InsertBuffer) Yield() *storage.InsertData { } func (ib *InsertBuffer) Buffer(inData *inData, startPos, endPos *msgpb.MsgPosition) int64 { - totalMemSize := int64(0) + bufferedSize := int64(0) for idx, data := range inData.data { storage.MergeInsertData(ib.buffer, data) tsData := inData.tsField[idx] // update buffer size ib.UpdateStatistics(int64(data.GetRowNum()), int64(data.GetMemorySize()), ib.getTimestampRange(tsData), startPos, endPos) - totalMemSize += int64(data.GetMemorySize()) + bufferedSize += int64(data.GetMemorySize()) } - return totalMemSize + return bufferedSize } func (ib *InsertBuffer) getTimestampRange(tsData *storage.Int64FieldData) TimeRange { diff --git a/internal/mocks/mock_datanode.go b/internal/mocks/mock_datanode.go index 1a6b281c933b4..6adcf0e559e7c 100644 --- a/internal/mocks/mock_datanode.go +++ b/internal/mocks/mock_datanode.go @@ -1497,7 +1497,8 @@ func (_c *MockDataNode_WatchDmChannels_Call) RunAndReturn(run func(context.Conte func NewMockDataNode(t interface { mock.TestingT Cleanup(func()) -}) *MockDataNode { +}, +) *MockDataNode { mock := &MockDataNode{} mock.Mock.Test(t) diff --git a/internal/mocks/mock_datanode_client.go b/internal/mocks/mock_datanode_client.go index ead0d9136b850..5c8ed2391775b 100644 --- a/internal/mocks/mock_datanode_client.go +++ b/internal/mocks/mock_datanode_client.go @@ -1337,7 +1337,8 @@ func (_c *MockDataNodeClient_WatchDmChannels_Call) RunAndReturn(run func(context func NewMockDataNodeClient(t interface { mock.TestingT Cleanup(func()) -}) *MockDataNodeClient { +}, +) *MockDataNodeClient { mock := &MockDataNodeClient{} mock.Mock.Test(t) diff --git a/internal/storage/serde.go b/internal/storage/serde.go index 8b8f77ca53f9a..02aac9dd961ed 100644 --- a/internal/storage/serde.go +++ b/internal/storage/serde.go @@ -28,6 +28,7 @@ import ( "github.com/apache/arrow/go/v12/arrow" "github.com/apache/arrow/go/v12/arrow/array" "github.com/apache/arrow/go/v12/arrow/memory" + "github.com/apache/arrow/go/v12/parquet" "github.com/apache/arrow/go/v12/parquet/pqarrow" "github.com/cockroachdb/errors" "github.com/golang/protobuf/proto" @@ -748,18 +749,17 @@ var _ RecordWriter = (*singleFieldRecordWriter)(nil) type singleFieldRecordWriter struct { fw *pqarrow.FileWriter fieldId FieldID + schema *arrow.Schema - grouped bool + numRows int } func (sfw *singleFieldRecordWriter) Write(r Record) error { - if !sfw.grouped { - sfw.grouped = true - sfw.fw.NewRowGroup() - } - // TODO: adding row group support by calling fw.NewRowGroup() + sfw.numRows += r.Len() a := r.Column(sfw.fieldId) - return sfw.fw.WriteColumnData(a) + rec := array.NewRecord(sfw.schema, []arrow.Array{a}, int64(r.Len())) + defer rec.Release() + return sfw.fw.WriteBuffered(rec) } func (sfw *singleFieldRecordWriter) Close() { @@ -768,13 +768,16 @@ func (sfw *singleFieldRecordWriter) Close() { func newSingleFieldRecordWriter(fieldId FieldID, field arrow.Field, writer io.Writer) (*singleFieldRecordWriter, error) { schema := arrow.NewSchema([]arrow.Field{field}, nil) - fw, err := pqarrow.NewFileWriter(schema, writer, nil, pqarrow.DefaultWriterProps()) + fw, err := pqarrow.NewFileWriter(schema, writer, + parquet.NewWriterProperties(parquet.WithMaxRowGroupLength(math.MaxInt64)), // No additional grouping for now. + pqarrow.DefaultWriterProps()) if err != nil { return nil, err } return &singleFieldRecordWriter{ fw: fw, fieldId: fieldId, + schema: schema, }, nil } @@ -789,15 +792,18 @@ type SerializeWriter[T any] struct { } func (sw *SerializeWriter[T]) Flush() error { + if sw.pos == 0 { + return nil + } buf := sw.buffer[:sw.pos] r, size, err := sw.serializer(buf) if err != nil { return err } + defer r.Release() if err := sw.rw.Write(r); err != nil { return err } - r.Release() sw.pos = 0 sw.writtenMemorySize += size return nil @@ -822,8 +828,11 @@ func (sw *SerializeWriter[T]) WrittenMemorySize() uint64 { } func (sw *SerializeWriter[T]) Close() error { + if err := sw.Flush(); err != nil { + return err + } sw.rw.Close() - return sw.Flush() + return nil } func NewSerializeRecordWriter[T any](rw RecordWriter, serializer Serializer[T], batchSize int) *SerializeWriter[T] { @@ -880,7 +889,7 @@ type BinlogStreamWriter struct { memorySize int // To be updated on the fly buf bytes.Buffer - rw RecordWriter + rw *singleFieldRecordWriter } func (bsw *BinlogStreamWriter) GetRecordWriter() (RecordWriter, error) { @@ -915,8 +924,9 @@ func (bsw *BinlogStreamWriter) Finalize() (*Blob, error) { return nil, err } return &Blob{ - Key: strconv.Itoa(int(bsw.fieldSchema.FieldID)), - Value: b.Bytes(), + Key: strconv.Itoa(int(bsw.fieldSchema.FieldID)), + Value: b.Bytes(), + RowNum: int64(bsw.rw.numRows), }, nil } diff --git a/internal/storage/serde_test.go b/internal/storage/serde_test.go index 17a10e3a2104e..21a871cb5e606 100644 --- a/internal/storage/serde_test.go +++ b/internal/storage/serde_test.go @@ -124,7 +124,7 @@ func TestBinlogSerializeWriter(t *testing.T) { }) t.Run("test serialize", func(t *testing.T) { - size := 3 + size := 16 blobs, err := generateTestData(size) assert.NoError(t, err) reader, err := NewBinlogDeserializeReader(blobs, common.RowIDField) @@ -134,7 +134,7 @@ func TestBinlogSerializeWriter(t *testing.T) { schema := generateTestSchema() // Copy write the generated data writers := NewBinlogStreamWriters(0, 0, 0, schema.Fields) - writer, err := NewBinlogSerializeWriter(schema, 0, 0, writers, 1024) + writer, err := NewBinlogSerializeWriter(schema, 0, 0, writers, 7) assert.NoError(t, err) for i := 1; i <= size; i++ { @@ -143,7 +143,8 @@ func TestBinlogSerializeWriter(t *testing.T) { value := reader.Value() assertTestData(t, i, value) - writer.Write(value) + err := writer.Write(value) + assert.NoError(t, err) } err = reader.Next() From c0f348ef091ab02d489371252706bfbdd9eef11a Mon Sep 17 00:00:00 2001 From: Ted Xu Date: Fri, 24 May 2024 10:33:40 +0800 Subject: [PATCH 2/3] fix: adding blob memory size in binlog serde (#33324) See: #33280 Signed-off-by: Ted Xu --- internal/storage/serde.go | 8 +++++--- internal/storage/serde_test.go | 1 + 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/internal/storage/serde.go b/internal/storage/serde.go index 02aac9dd961ed..c75bf5aabc0b6 100644 --- a/internal/storage/serde.go +++ b/internal/storage/serde.go @@ -924,9 +924,10 @@ func (bsw *BinlogStreamWriter) Finalize() (*Blob, error) { return nil, err } return &Blob{ - Key: strconv.Itoa(int(bsw.fieldSchema.FieldID)), - Value: b.Bytes(), - RowNum: int64(bsw.rw.numRows), + Key: strconv.Itoa(int(bsw.fieldSchema.FieldID)), + Value: b.Bytes(), + RowNum: int64(bsw.rw.numRows), + MemorySize: int64(bsw.memorySize), }, nil } @@ -1015,6 +1016,7 @@ func NewBinlogSerializeWriter(schema *schemapb.CollectionSchema, partitionID, se if !ok { return nil, 0, errors.New(fmt.Sprintf("serialize error on type %s", types[fid])) } + writers[fid].memorySize += int(typeEntry.sizeof(e)) memorySize += typeEntry.sizeof(e) } } diff --git a/internal/storage/serde_test.go b/internal/storage/serde_test.go index 21a871cb5e606..0d9306069d775 100644 --- a/internal/storage/serde_test.go +++ b/internal/storage/serde_test.go @@ -160,6 +160,7 @@ func TestBinlogSerializeWriter(t *testing.T) { blob, err := w.Finalize() assert.NoError(t, err) assert.NotNil(t, blob) + assert.True(t, blob.MemorySize > 0) newblobs[i] = blob i++ } From b1dbbd4dbe7f6a6c8aac5f7aa6b7aca36e9a6eac Mon Sep 17 00:00:00 2001 From: Ted Xu Date: Mon, 24 Jun 2024 10:08:02 +0800 Subject: [PATCH 3/3] fix: turn on compression on stream writers (#34067) See #31679 Signed-off-by: Ted Xu --- internal/storage/serde.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/internal/storage/serde.go b/internal/storage/serde.go index c75bf5aabc0b6..1474fca5a5795 100644 --- a/internal/storage/serde.go +++ b/internal/storage/serde.go @@ -29,6 +29,7 @@ import ( "github.com/apache/arrow/go/v12/arrow/array" "github.com/apache/arrow/go/v12/arrow/memory" "github.com/apache/arrow/go/v12/parquet" + "github.com/apache/arrow/go/v12/parquet/compress" "github.com/apache/arrow/go/v12/parquet/pqarrow" "github.com/cockroachdb/errors" "github.com/golang/protobuf/proto" @@ -769,7 +770,10 @@ func (sfw *singleFieldRecordWriter) Close() { func newSingleFieldRecordWriter(fieldId FieldID, field arrow.Field, writer io.Writer) (*singleFieldRecordWriter, error) { schema := arrow.NewSchema([]arrow.Field{field}, nil) fw, err := pqarrow.NewFileWriter(schema, writer, - parquet.NewWriterProperties(parquet.WithMaxRowGroupLength(math.MaxInt64)), // No additional grouping for now. + parquet.NewWriterProperties( + parquet.WithMaxRowGroupLength(math.MaxInt64), // No additional grouping for now. + parquet.WithCompression(compress.Codecs.Zstd), + parquet.WithCompressionLevel(3)), pqarrow.DefaultWriterProps()) if err != nil { return nil, err