From b37d7c0b1cbc440e815428c56e30e014da923cfe Mon Sep 17 00:00:00 2001 From: yangxuan Date: Wed, 10 Apr 2024 17:40:50 +0800 Subject: [PATCH] enhance: Tidy compactor and remove dup codes Add compaction package remove compactor.go and compactor_test.go Signed-off-by: yangxuan --- Makefile | 3 +- internal/datacoord/mock_cluster.go | 48 +- internal/datacoord/mock_session_manager.go | 62 +- internal/datanode/binlog_io.go | 249 ---- internal/datanode/binlog_io_test.go | 404 ------ internal/datanode/compaction/compactor.go | 32 + internal/datanode/compaction/mix_compactor.go | 568 ++++++++ .../datanode/compaction/mix_compactor_test.go | 803 +++++++++++ .../datanode/compaction/mock_compactor.go | 307 ++++ .../datanode/compaction/segment_writer.go | 165 +++ internal/datanode/compaction_executor.go | 55 +- internal/datanode/compaction_executor_test.go | 135 +- internal/datanode/compactor.go | 827 ----------- internal/datanode/compactor_test.go | 1246 ----------------- internal/datanode/io/binlog_io.go | 1 - internal/datanode/l0_compactor.go | 25 +- internal/datanode/l0_compactor_test.go | 8 +- internal/datanode/mock_test.go | 51 - internal/datanode/services.go | 8 +- internal/datanode/services_test.go | 20 +- .../datanode/writebuffer/insert_buffer.go | 11 +- .../datanode/writebuffer/segment_buffer.go | 15 + internal/metastore/kv/binlog/binlog.go | 4 +- internal/mocks/mock_datanode.go | 86 +- internal/mocks/mock_datanode_client.go | 114 +- 25 files changed, 2164 insertions(+), 3083 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 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 diff --git a/Makefile b/Makefile index 6ed5b46d19680..3da031187e584 100644 --- a/Makefile +++ b/Makefile @@ -479,6 +479,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 @@ -521,4 +522,4 @@ mmap-migration: @source $(PWD)/scripts/setenv.sh && \ mkdir -p $(INSTALL_PATH) && go env -w CGO_ENABLED="1" && \ GO111MODULE=on $(GO) build -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/mock_cluster.go b/internal/datacoord/mock_cluster.go index e35f1e1fee0ab..e92ae8ecb3c28 100644 --- a/internal/datacoord/mock_cluster.go +++ b/internal/datacoord/mock_cluster.go @@ -74,8 +74,8 @@ type MockCluster_DropImport_Call struct { } // DropImport is a helper method to define mock.On call -// - nodeID int64 -// - in *datapb.DropImportRequest +// - nodeID int64 +// - in *datapb.DropImportRequest func (_e *MockCluster_Expecter) DropImport(nodeID interface{}, in interface{}) *MockCluster_DropImport_Call { return &MockCluster_DropImport_Call{Call: _e.mock.On("DropImport", nodeID, in)} } @@ -117,10 +117,10 @@ type MockCluster_Flush_Call struct { } // Flush is a helper method to define mock.On call -// - ctx context.Context -// - nodeID int64 -// - channel string -// - segments []*datapb.SegmentInfo +// - ctx context.Context +// - nodeID int64 +// - channel string +// - segments []*datapb.SegmentInfo func (_e *MockCluster_Expecter) Flush(ctx interface{}, nodeID interface{}, channel interface{}, segments interface{}) *MockCluster_Flush_Call { return &MockCluster_Flush_Call{Call: _e.mock.On("Flush", ctx, nodeID, channel, segments)} } @@ -162,10 +162,10 @@ type MockCluster_FlushChannels_Call struct { } // FlushChannels is a helper method to define mock.On call -// - ctx context.Context -// - nodeID int64 -// - flushTs uint64 -// - channels []string +// - ctx context.Context +// - nodeID int64 +// - flushTs uint64 +// - channels []string func (_e *MockCluster_Expecter) FlushChannels(ctx interface{}, nodeID interface{}, flushTs interface{}, channels interface{}) *MockCluster_FlushChannels_Call { return &MockCluster_FlushChannels_Call{Call: _e.mock.On("FlushChannels", ctx, nodeID, flushTs, channels)} } @@ -250,8 +250,8 @@ type MockCluster_ImportV2_Call struct { } // ImportV2 is a helper method to define mock.On call -// - nodeID int64 -// - in *datapb.ImportRequest +// - nodeID int64 +// - in *datapb.ImportRequest func (_e *MockCluster_Expecter) ImportV2(nodeID interface{}, in interface{}) *MockCluster_ImportV2_Call { return &MockCluster_ImportV2_Call{Call: _e.mock.On("ImportV2", nodeID, in)} } @@ -293,8 +293,8 @@ type MockCluster_PreImport_Call struct { } // PreImport is a helper method to define mock.On call -// - nodeID int64 -// - in *datapb.PreImportRequest +// - nodeID int64 +// - in *datapb.PreImportRequest func (_e *MockCluster_Expecter) PreImport(nodeID interface{}, in interface{}) *MockCluster_PreImport_Call { return &MockCluster_PreImport_Call{Call: _e.mock.On("PreImport", nodeID, in)} } @@ -348,8 +348,8 @@ type MockCluster_QueryImport_Call struct { } // QueryImport is a helper method to define mock.On call -// - nodeID int64 -// - in *datapb.QueryImportRequest +// - nodeID int64 +// - in *datapb.QueryImportRequest func (_e *MockCluster_Expecter) QueryImport(nodeID interface{}, in interface{}) *MockCluster_QueryImport_Call { return &MockCluster_QueryImport_Call{Call: _e.mock.On("QueryImport", nodeID, in)} } @@ -403,8 +403,8 @@ type MockCluster_QueryPreImport_Call struct { } // QueryPreImport is a helper method to define mock.On call -// - nodeID int64 -// - in *datapb.QueryPreImportRequest +// - nodeID int64 +// - in *datapb.QueryPreImportRequest func (_e *MockCluster_Expecter) QueryPreImport(nodeID interface{}, in interface{}) *MockCluster_QueryPreImport_Call { return &MockCluster_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", nodeID, in)} } @@ -489,7 +489,7 @@ type MockCluster_Register_Call struct { } // Register is a helper method to define mock.On call -// - node *NodeInfo +// - node *NodeInfo func (_e *MockCluster_Expecter) Register(node interface{}) *MockCluster_Register_Call { return &MockCluster_Register_Call{Call: _e.mock.On("Register", node)} } @@ -531,8 +531,8 @@ type MockCluster_Startup_Call struct { } // Startup is a helper method to define mock.On call -// - ctx context.Context -// - nodes []*NodeInfo +// - ctx context.Context +// - nodes []*NodeInfo func (_e *MockCluster_Expecter) Startup(ctx interface{}, nodes interface{}) *MockCluster_Startup_Call { return &MockCluster_Startup_Call{Call: _e.mock.On("Startup", ctx, nodes)} } @@ -574,7 +574,7 @@ type MockCluster_UnRegister_Call struct { } // UnRegister is a helper method to define mock.On call -// - node *NodeInfo +// - node *NodeInfo func (_e *MockCluster_Expecter) UnRegister(node interface{}) *MockCluster_UnRegister_Call { return &MockCluster_UnRegister_Call{Call: _e.mock.On("UnRegister", node)} } @@ -616,8 +616,8 @@ type MockCluster_Watch_Call struct { } // Watch is a helper method to define mock.On call -// - ctx context.Context -// - ch RWChannel +// - ctx context.Context +// - ch RWChannel func (_e *MockCluster_Expecter) Watch(ctx interface{}, ch interface{}) *MockCluster_Watch_Call { return &MockCluster_Watch_Call{Call: _e.mock.On("Watch", ctx, ch)} } diff --git a/internal/datacoord/mock_session_manager.go b/internal/datacoord/mock_session_manager.go index a7d8e7f679c59..aea14b219ce03 100644 --- a/internal/datacoord/mock_session_manager.go +++ b/internal/datacoord/mock_session_manager.go @@ -35,7 +35,7 @@ type MockSessionManager_AddSession_Call struct { } // AddSession is a helper method to define mock.On call -// - node *NodeInfo +// - node *NodeInfo func (_e *MockSessionManager_Expecter) AddSession(node interface{}) *MockSessionManager_AddSession_Call { return &MockSessionManager_AddSession_Call{Call: _e.mock.On("AddSession", node)} } @@ -89,9 +89,9 @@ type MockSessionManager_CheckChannelOperationProgress_Call struct { } // CheckChannelOperationProgress is a helper method to define mock.On call -// - ctx context.Context -// - nodeID int64 -// - info *datapb.ChannelWatchInfo +// - ctx context.Context +// - nodeID int64 +// - info *datapb.ChannelWatchInfo func (_e *MockSessionManager_Expecter) CheckChannelOperationProgress(ctx interface{}, nodeID interface{}, info interface{}) *MockSessionManager_CheckChannelOperationProgress_Call { return &MockSessionManager_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress", ctx, nodeID, info)} } @@ -133,7 +133,7 @@ type MockSessionManager_CheckHealth_Call struct { } // CheckHealth is a helper method to define mock.On call -// - ctx context.Context +// - ctx context.Context func (_e *MockSessionManager_Expecter) CheckHealth(ctx interface{}) *MockSessionManager_CheckHealth_Call { return &MockSessionManager_CheckHealth_Call{Call: _e.mock.On("CheckHealth", ctx)} } @@ -207,9 +207,9 @@ type MockSessionManager_Compaction_Call struct { } // Compaction is a helper method to define mock.On call -// - ctx context.Context -// - nodeID int64 -// - plan *datapb.CompactionPlan +// - ctx context.Context +// - nodeID int64 +// - plan *datapb.CompactionPlan func (_e *MockSessionManager_Expecter) Compaction(ctx interface{}, nodeID interface{}, plan interface{}) *MockSessionManager_Compaction_Call { return &MockSessionManager_Compaction_Call{Call: _e.mock.On("Compaction", ctx, nodeID, plan)} } @@ -242,7 +242,7 @@ type MockSessionManager_DeleteSession_Call struct { } // DeleteSession is a helper method to define mock.On call -// - node *NodeInfo +// - node *NodeInfo func (_e *MockSessionManager_Expecter) DeleteSession(node interface{}) *MockSessionManager_DeleteSession_Call { return &MockSessionManager_DeleteSession_Call{Call: _e.mock.On("DeleteSession", node)} } @@ -284,8 +284,8 @@ type MockSessionManager_DropImport_Call struct { } // DropImport is a helper method to define mock.On call -// - nodeID int64 -// - in *datapb.DropImportRequest +// - nodeID int64 +// - in *datapb.DropImportRequest func (_e *MockSessionManager_Expecter) DropImport(nodeID interface{}, in interface{}) *MockSessionManager_DropImport_Call { return &MockSessionManager_DropImport_Call{Call: _e.mock.On("DropImport", nodeID, in)} } @@ -318,9 +318,9 @@ type MockSessionManager_Flush_Call struct { } // Flush is a helper method to define mock.On call -// - ctx context.Context -// - nodeID int64 -// - req *datapb.FlushSegmentsRequest +// - ctx context.Context +// - nodeID int64 +// - req *datapb.FlushSegmentsRequest func (_e *MockSessionManager_Expecter) Flush(ctx interface{}, nodeID interface{}, req interface{}) *MockSessionManager_Flush_Call { return &MockSessionManager_Flush_Call{Call: _e.mock.On("Flush", ctx, nodeID, req)} } @@ -362,9 +362,9 @@ type MockSessionManager_FlushChannels_Call struct { } // FlushChannels is a helper method to define mock.On call -// - ctx context.Context -// - nodeID int64 -// - req *datapb.FlushChannelsRequest +// - ctx context.Context +// - nodeID int64 +// - req *datapb.FlushChannelsRequest func (_e *MockSessionManager_Expecter) FlushChannels(ctx interface{}, nodeID interface{}, req interface{}) *MockSessionManager_FlushChannels_Call { return &MockSessionManager_FlushChannels_Call{Call: _e.mock.On("FlushChannels", ctx, nodeID, req)} } @@ -545,8 +545,8 @@ type MockSessionManager_ImportV2_Call struct { } // ImportV2 is a helper method to define mock.On call -// - nodeID int64 -// - in *datapb.ImportRequest +// - nodeID int64 +// - in *datapb.ImportRequest func (_e *MockSessionManager_Expecter) ImportV2(nodeID interface{}, in interface{}) *MockSessionManager_ImportV2_Call { return &MockSessionManager_ImportV2_Call{Call: _e.mock.On("ImportV2", nodeID, in)} } @@ -588,9 +588,9 @@ type MockSessionManager_NotifyChannelOperation_Call struct { } // NotifyChannelOperation is a helper method to define mock.On call -// - ctx context.Context -// - nodeID int64 -// - req *datapb.ChannelOperationsRequest +// - ctx context.Context +// - nodeID int64 +// - req *datapb.ChannelOperationsRequest func (_e *MockSessionManager_Expecter) NotifyChannelOperation(ctx interface{}, nodeID interface{}, req interface{}) *MockSessionManager_NotifyChannelOperation_Call { return &MockSessionManager_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation", ctx, nodeID, req)} } @@ -632,8 +632,8 @@ type MockSessionManager_PreImport_Call struct { } // PreImport is a helper method to define mock.On call -// - nodeID int64 -// - in *datapb.PreImportRequest +// - nodeID int64 +// - in *datapb.PreImportRequest func (_e *MockSessionManager_Expecter) PreImport(nodeID interface{}, in interface{}) *MockSessionManager_PreImport_Call { return &MockSessionManager_PreImport_Call{Call: _e.mock.On("PreImport", nodeID, in)} } @@ -687,8 +687,8 @@ type MockSessionManager_QueryImport_Call struct { } // QueryImport is a helper method to define mock.On call -// - nodeID int64 -// - in *datapb.QueryImportRequest +// - nodeID int64 +// - in *datapb.QueryImportRequest func (_e *MockSessionManager_Expecter) QueryImport(nodeID interface{}, in interface{}) *MockSessionManager_QueryImport_Call { return &MockSessionManager_QueryImport_Call{Call: _e.mock.On("QueryImport", nodeID, in)} } @@ -742,8 +742,8 @@ type MockSessionManager_QueryPreImport_Call struct { } // QueryPreImport is a helper method to define mock.On call -// - nodeID int64 -// - in *datapb.QueryPreImportRequest +// - nodeID int64 +// - in *datapb.QueryPreImportRequest func (_e *MockSessionManager_Expecter) QueryPreImport(nodeID interface{}, in interface{}) *MockSessionManager_QueryPreImport_Call { return &MockSessionManager_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", nodeID, in)} } @@ -797,7 +797,7 @@ type MockSessionManager_QuerySlot_Call struct { } // QuerySlot is a helper method to define mock.On call -// - nodeID int64 +// - nodeID int64 func (_e *MockSessionManager_Expecter) QuerySlot(nodeID interface{}) *MockSessionManager_QuerySlot_Call { return &MockSessionManager_QuerySlot_Call{Call: _e.mock.On("QuerySlot", nodeID)} } @@ -839,8 +839,8 @@ type MockSessionManager_SyncSegments_Call struct { } // SyncSegments is a helper method to define mock.On call -// - nodeID int64 -// - req *datapb.SyncSegmentsRequest +// - nodeID int64 +// - req *datapb.SyncSegmentsRequest func (_e *MockSessionManager_Expecter) SyncSegments(nodeID interface{}, req interface{}) *MockSessionManager_SyncSegments_Call { return &MockSessionManager_SyncSegments_Call{Call: _e.mock.On("SyncSegments", nodeID, req)} } diff --git a/internal/datanode/binlog_io.go b/internal/datanode/binlog_io.go deleted file mode 100644 index 506c614a1c9b9..0000000000000 --- a/internal/datanode/binlog_io.go +++ /dev/null @@ -1,249 +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 []*Blob, collectionID, partID, segID UniqueID, kvs map[string][]byte, -) (map[UniqueID]*datapb.FieldBinlog, error) { - inpaths := make(map[UniqueID]*datapb.FieldBinlog) - notifyGenIdx := make(chan struct{}) - defer close(notifyGenIdx) - - generator, err := allocator.GetGenerator(len(data), notifyGenIdx) - if err != nil { - return nil, err - } - - for _, blob := range data { - // 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, - data []*Blob, -) (map[UniqueID]*datapb.FieldBinlog, error) { - ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, "UploadInsertLog") - defer span.End() - kvs := make(map[string][]byte) - - if len(data) <= 0 || data[0].RowNum <= 0 { - log.Warn("binlog io uploading empty insert data", - zap.Int64("segmentID", segID), - zap.Int64("collectionID", collectionID), - ) - return nil, nil - } - - inpaths, err := genInsertBlobs(b, allocator, data, collectionID, partID, segID, 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 038978ac0464c..0000000000000 --- a/internal/datanode/binlog_io_test.go +++ /dev/null @@ -1,404 +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("gen insert blob failed", func(t *testing.T) { - alloc := allocator.NewMockAllocator(t) - binlogIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - iCodec := storage.NewInsertCodecWithSchema(meta) - var partId int64 = 10 - var segId int64 = 1 - iData := genInsertData(2) - blobs, err := iCodec.Serialize(10, 1, iData) - assert.NoError(t, err) - alloc.EXPECT().GetGenerator(mock.Anything, mock.Anything).Call.Return(nil, fmt.Errorf("mock err")) - _, err = uploadInsertLog(context.Background(), binlogIO, alloc, meta.GetID(), partId, segId, blobs) - 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) - var partId int64 = 1 - var segId int64 = 10 - iData := genInsertData(2) - blobs, err := iCodec.Serialize(10, 1, iData) - assert.NoError(t, err) - - 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(), partId, segId, blobs) - 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) - var partId int64 = 10 - var segId int64 = 1 - iData := genInsertData(2) - blobs, err := iCodec.Serialize(10, 1, iData) - assert.NoError(t, err) - kvs := make(map[string][]byte) - pin, err := genInsertBlobs(binlogIO, alloc, blobs, meta.GetID(), partId, segId, 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("GetGenerator error", func(t *testing.T) { - f := &MetaFactory{} - meta := f.GetCollectionMeta(UniqueID(10001), "test_gen_blobs", schemapb.DataType_Int64) - iCodec := storage.NewInsertCodecWithSchema(meta) - var partId int64 = 10 - var segId int64 = 1 - iData := genInsertData(2) - blobs, err := iCodec.Serialize(partId, segId, iData) - assert.NoError(t, err) - - 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, blobs, meta.GetID(), partId, segId, 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/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..6ca701bedd91e --- /dev/null +++ b/internal/datanode/compaction/mix_compactor_test.go @@ -0,0 +1,803 @@ +// 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.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..3d458aad9974e --- /dev/null +++ b/internal/datanode/compaction/segment_writer.go @@ -0,0 +1,165 @@ +// 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" +) + +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..8e0a53f2bb3f9 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,27 +113,19 @@ 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) { @@ -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 e99642316e6f4..0000000000000 --- a/internal/datanode/compactor.go +++ /dev/null @@ -1,827 +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 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 newBinlogWriter(collectionId, partitionId, segmentId UniqueID, schema *schemapb.CollectionSchema, -) (writer *storage.SerializeWriter[*storage.Value], closers []func() (*Blob, error), err error) { - fieldWriters := storage.NewBinlogStreamWriters(collectionId, partitionId, segmentId, schema.Fields) - closers = make([]func() (*Blob, error), 0, len(fieldWriters)) - for _, w := range fieldWriters { - closers = append(closers, w.Finalize) - } - writer, err = storage.NewBinlogSerializeWriter(schema, partitionId, segmentId, fieldWriters, 1024) - return -} - -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() - - writer, finalizers, err := newBinlogWriter(meta.GetID(), partID, targetSegID, meta.GetSchema()) - if err != nil { - return nil, nil, 0, err - } - - var ( - numBinlogs int // binlog number - numRows uint64 // 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) - ) - - 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() - unflushedRows := 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 - ) - - flush := func() error { - uploadInsertStart := time.Now() - writer.Close() - fieldData := make([]*Blob, len(finalizers)) - - for i, f := range finalizers { - blob, err := f() - if err != nil { - return err - } - fieldData[i] = blob - } - inPaths, err := uploadInsertLog(ctx, t.binlogIO, t.Allocator, meta.ID, partID, targetSegID, fieldData) - if err != nil { - log.Warn("failed to upload single insert log", zap.Error(err)) - return err - } - numBinlogs += len(inPaths) - uploadInsertTimeCost += time.Since(uploadInsertStart) - addInsertFieldPath(inPaths, timestampFrom, timestampTo) - unflushedRows = 0 - return nil - } - - 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 - } - - err = writer.Write(v) - if err != nil { - return nil, nil, 0, err - } - numRows++ - unflushedRows++ - - stats.Update(v.PK) - - // check size every 100 rows in case of too many `GetMemorySize` call - if (unflushedRows+1)%100 == 0 { - writer.Flush() // Flush to update memory size - - if writer.WrittenMemorySize() > paramtable.Get().DataNodeCfg.BinLogMaxSize.GetAsUint64() { - if err := flush(); err != nil { - return nil, nil, 0, err - } - timestampFrom = -1 - timestampTo = -1 - - writer, finalizers, err = newBinlogWriter(meta.ID, targetSegID, partID, meta.Schema) - if err != nil { - return nil, nil, 0, err - } - } - } - } - } - - // final flush if there is unflushed rows - if unflushedRows > 0 { - if err := flush(); err != nil { - return nil, nil, 0, err - } - } - - // upload stats log - if numRows > 0 { - iCodec := storage.NewInsertCodecWithSchema(meta) - statsPaths, err := uploadStatsLog(ctx, t.binlogIO, t.Allocator, meta.GetID(), partID, targetSegID, stats, int64(numRows), iCodec) - if err != nil { - return nil, nil, 0, err - } - addStatFieldPath(statsPaths) - } - - for _, path := range insertField2Path { - insertPaths = append(insertPaths, path) - } - - for _, path := range statField2Path { - statPaths = append(statPaths, path) - } - - log.Info("compact merge end", - zap.Uint64("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, int64(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 efea77b55fbbb..0000000000000 --- a/internal/datanode/compactor_test.go +++ /dev/null @@ -1,1246 +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" - - "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 partId int64 = 0 - var segmentId int64 = 1 - blobs, err := iCodec.Serialize(partId, 0, iData) - assert.NoError(t, err) - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), partId, segmentId, blobs) - 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()) - iData := genInsertDataWithExpiredTS() - iCodec := storage.NewInsertCodecWithSchema(meta) - var partId int64 = 0 - var segmentId int64 = 1 - blobs, err := iCodec.Serialize(partId, 0, iData) - assert.NoError(t, err) - 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") - meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) - - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), partId, segmentId, blobs) - 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 partId int64 = 0 - var segmentId int64 = 1 - blobs, err := iCodec.Serialize(partId, segmentId, iData) - assert.NoError(t, err) - - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), partId, segmentId, blobs) - 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() - var partId int64 = 0 - var segmentId int64 = 1 - blobs, err := iCodec.Serialize(partId, 0, iData) - assert.NoError(t, err) - meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) - - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), partId, segmentId, blobs) - 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) - var partId int64 = 0 - var segmentId int64 = 1 - blobs, err := iCodec.Serialize(partId, 0, iData) - assert.NoError(t, err) - 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(), partId, segmentId, blobs) - 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() - var partId int64 = 0 - var segmentId int64 = 1 - blobs, err := iCodec.Serialize(partId, 0, iData) - assert.NoError(t, err) - meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) - - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), partId, segmentId, blobs) - 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() - var partId int64 = 0 - var segmentId int64 = 1 - blobs, err := iCodec.Serialize(partId, 0, iData) - assert.NoError(t, err) - meta := NewMetaFactory().GetCollectionMeta(1, "test", schemapb.DataType_Int64) - - var allPaths [][]string - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), partId, segmentId, blobs) - 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") - }) -} - -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) - iblobs1, err := iCodec.Serialize(c.parID, 0, iData1) - assert.NoError(t, err) - dData1 := &DeleteData{ - Pks: []storage.PrimaryKey{c.pks1[0]}, - Tss: []Timestamp{20000}, - RowCount: 1, - } - iData2 := genInsertDataWithPKs(c.pks2, c.pkType) - iblobs2, err := iCodec.Serialize(c.parID, 3, iData2) - assert.NoError(t, err) - 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, iblobs1) - 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, iblobs2) - 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) - iblobs1, err := iCodec.Serialize(partID, 0, iData1) - assert.NoError(t, err) - iData2 := genInsertDataWithPKs(pks, schemapb.DataType_Int64) - iblobs2, err := iCodec.Serialize(partID, 1, iData2) - assert.NoError(t, err) - - 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, iblobs1) - 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, iblobs2) - 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() -} - -func BenchmarkCompaction(b *testing.B) { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - cm := storage.NewLocalChunkManager(storage.RootPath(compactTestDir)) - defer cm.RemoveWithPrefix(ctx, cm.RootPath()) - - collectionID := int64(1) - meta := NewMetaFactory().GetCollectionMeta(collectionID, "test", schemapb.DataType_Int64) - mockbIO := io.NewBinlogIO(cm, getOrCreateIOPool()) - paramtable.Get().Save(Params.CommonCfg.EntityExpirationTTL.Key, "0") - iData := genInsertDataWithExpiredTS() - iCodec := storage.NewInsertCodecWithSchema(meta) - var partId int64 = 0 - var segmentId int64 = 1 - blobs, err := iCodec.Serialize(partId, 0, iData) - assert.NoError(b, err) - var allPaths [][]string - alloc := allocator.NewMockAllocator(b) - alloc.EXPECT().GetGenerator(mock.Anything, mock.Anything).Call.Return(validGeneratorFn, nil) - alloc.EXPECT().AllocOne().Call.Return(int64(19530), nil) - inpath, err := uploadInsertLog(context.Background(), mockbIO, alloc, meta.GetID(), partId, segmentId, blobs) - assert.NoError(b, err) - assert.Equal(b, 12, len(inpath)) - binlogNum := len(inpath[0].GetBinlogs()) - assert.Equal(b, 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, - } - - metaCache := metacache.NewMockMetaCache(b) - 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 - }) - - ct := &compactionTask{ - metaCache: metaCache, - binlogIO: mockbIO, - Allocator: alloc, - done: make(chan struct{}, 1), - plan: &datapb.CompactionPlan{ - SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{ - {SegmentID: 1}, - }, - }, - } - - b.ResetTimer() - - for i := 0; i < b.N; i++ { - inPaths, statsPaths, numOfRow, err := ct.merge(context.Background(), allPaths, 2, 0, meta, dm) - assert.NoError(b, err) - assert.Equal(b, int64(2), numOfRow) - assert.Equal(b, 1, len(inPaths[0].GetBinlogs())) - assert.Equal(b, 1, len(statsPaths)) - assert.NotEqual(b, -1, inPaths[0].GetBinlogs()[0].GetTimestampFrom()) - assert.NotEqual(b, -1, inPaths[0].GetBinlogs()[0].GetTimestampTo()) - } -} diff --git a/internal/datanode/io/binlog_io.go b/internal/datanode/io/binlog_io.go index c60af8e992dda..317f267978132 100644 --- a/internal/datanode/io/binlog_io.go +++ b/internal/datanode/io/binlog_io.go @@ -101,7 +101,6 @@ func (b *BinlogIoImpl) Upload(ctx context.Context, kvs map[string][]byte) error }) return struct{}{}, err }) - futures = append(futures, future) } diff --git a/internal/datanode/l0_compactor.go b/internal/datanode/l0_compactor.go index 75bde780323c7..89157c407834b 100644 --- a/internal/datanode/l0_compactor.go +++ b/internal/datanode/l0_compactor.go @@ -23,12 +23,14 @@ import ( "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/compaction" "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" @@ -50,7 +52,7 @@ import ( ) type levelZeroCompactionTask struct { - compactor + compaction.Compactor io.BinlogIO allocator allocator.Allocator @@ -67,6 +69,9 @@ type levelZeroCompactionTask struct { tr *timerecord.TimeRecorder } +// make sure compactionTask implements compactor interface +var _ compaction.Compactor = (*levelZeroCompactionTask)(nil) + func newLevelZeroCompactionTask( ctx context.Context, binlogIO io.BinlogIO, @@ -92,31 +97,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() UniqueID { 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())) @@ -124,7 +129,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() } ctxTimeout, cancelAll := context.WithTimeout(ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second) @@ -139,7 +144,7 @@ func (t *levelZeroCompactionTask) compact() (*datapb.CompactionPlanResult, error }) if len(targetSegments) == 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 { diff --git a/internal/datanode/l0_compactor_test.go b/internal/datanode/l0_compactor_test.go index 59b66086d67fa..08df575433ace 100644 --- a/internal/datanode/l0_compactor_test.go +++ b/internal/datanode/l0_compactor_test.go @@ -274,9 +274,9 @@ func (s *LevelZeroCompactionTaskSuite) TestCompactLinear() { }).Times(2) s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil).Once() - 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 @@ -757,7 +757,7 @@ func (s *LevelZeroCompactionTaskSuite) TestFailed() { s.task.plan = plan - _, err := s.task.compact() + _, err := s.task.Compact() s.Error(err) }) } 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/services.go b/internal/datanode/services.go index ad8cb3039e7ec..a3ce9e8df8d1d 100644 --- a/internal/datanode/services.go +++ b/internal/datanode/services.go @@ -27,6 +27,7 @@ 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" @@ -235,10 +236,10 @@ 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, getOrCreateIOPool()) switch req.GetType() { case datapb.CompactionType_Level0DeleteCompaction: - binlogIO := io.NewBinlogIO(node.chunkManager, getOrCreateIOPool()) task = newLevelZeroCompactionTask( taskCtx, binlogIO, @@ -249,8 +250,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, diff --git a/internal/datanode/services_test.go b/internal/datanode/services_test.go index 94eed7f5193e5..a834b1907d849 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/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/datanode/writebuffer/segment_buffer.go b/internal/datanode/writebuffer/segment_buffer.go index 8e14c3f4f869d..58ec2b4afda61 100644 --- a/internal/datanode/writebuffer/segment_buffer.go +++ b/internal/datanode/writebuffer/segment_buffer.go @@ -76,6 +76,21 @@ type TimeRange struct { timestampMax typeutil.Timestamp } +func NewTimeRange(min, max typeutil.Timestamp) *TimeRange { + return &TimeRange{ + timestampMin: min, + timestampMax: max, + } +} + +func (tr *TimeRange) GetMinTimestamp() typeutil.Timestamp { + return tr.timestampMin +} + +func (tr *TimeRange) GetMaxTimestamp() typeutil.Timestamp { + return tr.timestampMax +} + func (tr *TimeRange) Merge(other *TimeRange) { if other.timestampMin < tr.timestampMin { tr.timestampMin = other.timestampMin diff --git a/internal/metastore/kv/binlog/binlog.go b/internal/metastore/kv/binlog/binlog.go index 94e0c09cc73e6..8b1d47d0e5970 100644 --- a/internal/metastore/kv/binlog/binlog.go +++ b/internal/metastore/kv/binlog/binlog.go @@ -148,7 +148,7 @@ func DecompressBinLog(binlogType storage.BinlogType, collectionID, partitionID, for _, fieldBinlog := range fieldBinlogs { for _, binlog := range fieldBinlog.Binlogs { if binlog.GetLogPath() == "" { - path, err := buildLogPath(binlogType, collectionID, partitionID, + path, err := BuildLogPath(binlogType, collectionID, partitionID, segmentID, fieldBinlog.GetFieldID(), binlog.GetLogID()) if err != nil { return err @@ -161,7 +161,7 @@ func DecompressBinLog(binlogType storage.BinlogType, collectionID, partitionID, } // build a binlog path on the storage by metadata -func buildLogPath(binlogType storage.BinlogType, collectionID, partitionID, segmentID, fieldID, logID typeutil.UniqueID) (string, error) { +func BuildLogPath(binlogType storage.BinlogType, collectionID, partitionID, segmentID, fieldID, logID typeutil.UniqueID) (string, error) { chunkManagerRootPath := paramtable.Get().MinioCfg.RootPath.GetValue() if paramtable.Get().CommonCfg.StorageType.GetValue() == "local" { chunkManagerRootPath = paramtable.Get().LocalStorageCfg.Path.GetValue() diff --git a/internal/mocks/mock_datanode.go b/internal/mocks/mock_datanode.go index 3392028c1bd48..b6dc02ae27de5 100644 --- a/internal/mocks/mock_datanode.go +++ b/internal/mocks/mock_datanode.go @@ -64,8 +64,8 @@ type MockDataNode_CheckChannelOperationProgress_Call struct { } // CheckChannelOperationProgress is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.ChannelWatchInfo +// - _a0 context.Context +// - _a1 *datapb.ChannelWatchInfo func (_e *MockDataNode_Expecter) CheckChannelOperationProgress(_a0 interface{}, _a1 interface{}) *MockDataNode_CheckChannelOperationProgress_Call { return &MockDataNode_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress", _a0, _a1)} } @@ -119,8 +119,8 @@ type MockDataNode_Compaction_Call struct { } // Compaction is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.CompactionPlan +// - _a0 context.Context +// - _a1 *datapb.CompactionPlan func (_e *MockDataNode_Expecter) Compaction(_a0 interface{}, _a1 interface{}) *MockDataNode_Compaction_Call { return &MockDataNode_Compaction_Call{Call: _e.mock.On("Compaction", _a0, _a1)} } @@ -174,8 +174,8 @@ type MockDataNode_DropImport_Call struct { } // DropImport is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.DropImportRequest +// - _a0 context.Context +// - _a1 *datapb.DropImportRequest func (_e *MockDataNode_Expecter) DropImport(_a0 interface{}, _a1 interface{}) *MockDataNode_DropImport_Call { return &MockDataNode_DropImport_Call{Call: _e.mock.On("DropImport", _a0, _a1)} } @@ -229,8 +229,8 @@ type MockDataNode_FlushChannels_Call struct { } // FlushChannels is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.FlushChannelsRequest +// - _a0 context.Context +// - _a1 *datapb.FlushChannelsRequest func (_e *MockDataNode_Expecter) FlushChannels(_a0 interface{}, _a1 interface{}) *MockDataNode_FlushChannels_Call { return &MockDataNode_FlushChannels_Call{Call: _e.mock.On("FlushChannels", _a0, _a1)} } @@ -284,8 +284,8 @@ type MockDataNode_FlushSegments_Call struct { } // FlushSegments is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.FlushSegmentsRequest +// - _a0 context.Context +// - _a1 *datapb.FlushSegmentsRequest func (_e *MockDataNode_Expecter) FlushSegments(_a0 interface{}, _a1 interface{}) *MockDataNode_FlushSegments_Call { return &MockDataNode_FlushSegments_Call{Call: _e.mock.On("FlushSegments", _a0, _a1)} } @@ -380,8 +380,8 @@ type MockDataNode_GetCompactionState_Call struct { } // GetCompactionState is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.CompactionStateRequest +// - _a0 context.Context +// - _a1 *datapb.CompactionStateRequest func (_e *MockDataNode_Expecter) GetCompactionState(_a0 interface{}, _a1 interface{}) *MockDataNode_GetCompactionState_Call { return &MockDataNode_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", _a0, _a1)} } @@ -435,8 +435,8 @@ type MockDataNode_GetComponentStates_Call struct { } // GetComponentStates is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *milvuspb.GetComponentStatesRequest +// - _a0 context.Context +// - _a1 *milvuspb.GetComponentStatesRequest func (_e *MockDataNode_Expecter) GetComponentStates(_a0 interface{}, _a1 interface{}) *MockDataNode_GetComponentStates_Call { return &MockDataNode_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", _a0, _a1)} } @@ -490,8 +490,8 @@ type MockDataNode_GetMetrics_Call struct { } // GetMetrics is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *milvuspb.GetMetricsRequest +// - _a0 context.Context +// - _a1 *milvuspb.GetMetricsRequest func (_e *MockDataNode_Expecter) GetMetrics(_a0 interface{}, _a1 interface{}) *MockDataNode_GetMetrics_Call { return &MockDataNode_GetMetrics_Call{Call: _e.mock.On("GetMetrics", _a0, _a1)} } @@ -627,8 +627,8 @@ type MockDataNode_GetStatisticsChannel_Call struct { } // GetStatisticsChannel is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *internalpb.GetStatisticsChannelRequest +// - _a0 context.Context +// - _a1 *internalpb.GetStatisticsChannelRequest func (_e *MockDataNode_Expecter) GetStatisticsChannel(_a0 interface{}, _a1 interface{}) *MockDataNode_GetStatisticsChannel_Call { return &MockDataNode_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", _a0, _a1)} } @@ -682,8 +682,8 @@ type MockDataNode_ImportV2_Call struct { } // ImportV2 is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.ImportRequest +// - _a0 context.Context +// - _a1 *datapb.ImportRequest func (_e *MockDataNode_Expecter) ImportV2(_a0 interface{}, _a1 interface{}) *MockDataNode_ImportV2_Call { return &MockDataNode_ImportV2_Call{Call: _e.mock.On("ImportV2", _a0, _a1)} } @@ -778,8 +778,8 @@ type MockDataNode_NotifyChannelOperation_Call struct { } // NotifyChannelOperation is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.ChannelOperationsRequest +// - _a0 context.Context +// - _a1 *datapb.ChannelOperationsRequest func (_e *MockDataNode_Expecter) NotifyChannelOperation(_a0 interface{}, _a1 interface{}) *MockDataNode_NotifyChannelOperation_Call { return &MockDataNode_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation", _a0, _a1)} } @@ -833,8 +833,8 @@ type MockDataNode_PreImport_Call struct { } // PreImport is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.PreImportRequest +// - _a0 context.Context +// - _a1 *datapb.PreImportRequest func (_e *MockDataNode_Expecter) PreImport(_a0 interface{}, _a1 interface{}) *MockDataNode_PreImport_Call { return &MockDataNode_PreImport_Call{Call: _e.mock.On("PreImport", _a0, _a1)} } @@ -888,8 +888,8 @@ type MockDataNode_QueryImport_Call struct { } // QueryImport is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.QueryImportRequest +// - _a0 context.Context +// - _a1 *datapb.QueryImportRequest func (_e *MockDataNode_Expecter) QueryImport(_a0 interface{}, _a1 interface{}) *MockDataNode_QueryImport_Call { return &MockDataNode_QueryImport_Call{Call: _e.mock.On("QueryImport", _a0, _a1)} } @@ -943,8 +943,8 @@ type MockDataNode_QueryPreImport_Call struct { } // QueryPreImport is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.QueryPreImportRequest +// - _a0 context.Context +// - _a1 *datapb.QueryPreImportRequest func (_e *MockDataNode_Expecter) QueryPreImport(_a0 interface{}, _a1 interface{}) *MockDataNode_QueryPreImport_Call { return &MockDataNode_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", _a0, _a1)} } @@ -998,8 +998,8 @@ type MockDataNode_QuerySlot_Call struct { } // QuerySlot is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.QuerySlotRequest +// - _a0 context.Context +// - _a1 *datapb.QuerySlotRequest func (_e *MockDataNode_Expecter) QuerySlot(_a0 interface{}, _a1 interface{}) *MockDataNode_QuerySlot_Call { return &MockDataNode_QuerySlot_Call{Call: _e.mock.On("QuerySlot", _a0, _a1)} } @@ -1094,8 +1094,8 @@ type MockDataNode_ResendSegmentStats_Call struct { } // ResendSegmentStats is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.ResendSegmentStatsRequest +// - _a0 context.Context +// - _a1 *datapb.ResendSegmentStatsRequest func (_e *MockDataNode_Expecter) ResendSegmentStats(_a0 interface{}, _a1 interface{}) *MockDataNode_ResendSegmentStats_Call { return &MockDataNode_ResendSegmentStats_Call{Call: _e.mock.On("ResendSegmentStats", _a0, _a1)} } @@ -1128,7 +1128,7 @@ type MockDataNode_SetAddress_Call struct { } // SetAddress is a helper method to define mock.On call -// - address string +// - address string func (_e *MockDataNode_Expecter) SetAddress(address interface{}) *MockDataNode_SetAddress_Call { return &MockDataNode_SetAddress_Call{Call: _e.mock.On("SetAddress", address)} } @@ -1170,7 +1170,7 @@ type MockDataNode_SetDataCoordClient_Call struct { } // SetDataCoordClient is a helper method to define mock.On call -// - dataCoord types.DataCoordClient +// - dataCoord types.DataCoordClient func (_e *MockDataNode_Expecter) SetDataCoordClient(dataCoord interface{}) *MockDataNode_SetDataCoordClient_Call { return &MockDataNode_SetDataCoordClient_Call{Call: _e.mock.On("SetDataCoordClient", dataCoord)} } @@ -1203,7 +1203,7 @@ type MockDataNode_SetEtcdClient_Call struct { } // SetEtcdClient is a helper method to define mock.On call -// - etcdClient *clientv3.Client +// - etcdClient *clientv3.Client func (_e *MockDataNode_Expecter) SetEtcdClient(etcdClient interface{}) *MockDataNode_SetEtcdClient_Call { return &MockDataNode_SetEtcdClient_Call{Call: _e.mock.On("SetEtcdClient", etcdClient)} } @@ -1245,7 +1245,7 @@ type MockDataNode_SetRootCoordClient_Call struct { } // SetRootCoordClient is a helper method to define mock.On call -// - rootCoord types.RootCoordClient +// - rootCoord types.RootCoordClient func (_e *MockDataNode_Expecter) SetRootCoordClient(rootCoord interface{}) *MockDataNode_SetRootCoordClient_Call { return &MockDataNode_SetRootCoordClient_Call{Call: _e.mock.On("SetRootCoordClient", rootCoord)} } @@ -1299,8 +1299,8 @@ type MockDataNode_ShowConfigurations_Call struct { } // ShowConfigurations is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *internalpb.ShowConfigurationsRequest +// - _a0 context.Context +// - _a1 *internalpb.ShowConfigurationsRequest func (_e *MockDataNode_Expecter) ShowConfigurations(_a0 interface{}, _a1 interface{}) *MockDataNode_ShowConfigurations_Call { return &MockDataNode_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", _a0, _a1)} } @@ -1436,8 +1436,8 @@ type MockDataNode_SyncSegments_Call struct { } // SyncSegments is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.SyncSegmentsRequest +// - _a0 context.Context +// - _a1 *datapb.SyncSegmentsRequest func (_e *MockDataNode_Expecter) SyncSegments(_a0 interface{}, _a1 interface{}) *MockDataNode_SyncSegments_Call { return &MockDataNode_SyncSegments_Call{Call: _e.mock.On("SyncSegments", _a0, _a1)} } @@ -1470,7 +1470,7 @@ type MockDataNode_UpdateStateCode_Call struct { } // UpdateStateCode is a helper method to define mock.On call -// - stateCode commonpb.StateCode +// - stateCode commonpb.StateCode func (_e *MockDataNode_Expecter) UpdateStateCode(stateCode interface{}) *MockDataNode_UpdateStateCode_Call { return &MockDataNode_UpdateStateCode_Call{Call: _e.mock.On("UpdateStateCode", stateCode)} } @@ -1524,8 +1524,8 @@ type MockDataNode_WatchDmChannels_Call struct { } // WatchDmChannels is a helper method to define mock.On call -// - _a0 context.Context -// - _a1 *datapb.WatchDmChannelsRequest +// - _a0 context.Context +// - _a1 *datapb.WatchDmChannelsRequest func (_e *MockDataNode_Expecter) WatchDmChannels(_a0 interface{}, _a1 interface{}) *MockDataNode_WatchDmChannels_Call { return &MockDataNode_WatchDmChannels_Call{Call: _e.mock.On("WatchDmChannels", _a0, _a1)} } diff --git a/internal/mocks/mock_datanode_client.go b/internal/mocks/mock_datanode_client.go index 78f7aeec32131..f16ff8d1705bb 100644 --- a/internal/mocks/mock_datanode_client.go +++ b/internal/mocks/mock_datanode_client.go @@ -70,9 +70,9 @@ type MockDataNodeClient_CheckChannelOperationProgress_Call struct { } // CheckChannelOperationProgress is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.ChannelWatchInfo -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.ChannelWatchInfo +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) CheckChannelOperationProgress(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_CheckChannelOperationProgress_Call { return &MockDataNodeClient_CheckChannelOperationProgress_Call{Call: _e.mock.On("CheckChannelOperationProgress", append([]interface{}{ctx, in}, opts...)...)} @@ -181,9 +181,9 @@ type MockDataNodeClient_Compaction_Call struct { } // Compaction is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.CompactionPlan -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.CompactionPlan +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) Compaction(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_Compaction_Call { return &MockDataNodeClient_Compaction_Call{Call: _e.mock.On("Compaction", append([]interface{}{ctx, in}, opts...)...)} @@ -251,9 +251,9 @@ type MockDataNodeClient_DropImport_Call struct { } // DropImport is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.DropImportRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.DropImportRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) DropImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_DropImport_Call { return &MockDataNodeClient_DropImport_Call{Call: _e.mock.On("DropImport", append([]interface{}{ctx, in}, opts...)...)} @@ -321,9 +321,9 @@ type MockDataNodeClient_FlushChannels_Call struct { } // FlushChannels is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.FlushChannelsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.FlushChannelsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) FlushChannels(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_FlushChannels_Call { return &MockDataNodeClient_FlushChannels_Call{Call: _e.mock.On("FlushChannels", append([]interface{}{ctx, in}, opts...)...)} @@ -391,9 +391,9 @@ type MockDataNodeClient_FlushSegments_Call struct { } // FlushSegments is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.FlushSegmentsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.FlushSegmentsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) FlushSegments(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_FlushSegments_Call { return &MockDataNodeClient_FlushSegments_Call{Call: _e.mock.On("FlushSegments", append([]interface{}{ctx, in}, opts...)...)} @@ -461,9 +461,9 @@ type MockDataNodeClient_GetCompactionState_Call struct { } // GetCompactionState is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.CompactionStateRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.CompactionStateRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) GetCompactionState(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetCompactionState_Call { return &MockDataNodeClient_GetCompactionState_Call{Call: _e.mock.On("GetCompactionState", append([]interface{}{ctx, in}, opts...)...)} @@ -531,9 +531,9 @@ type MockDataNodeClient_GetComponentStates_Call struct { } // GetComponentStates is a helper method to define mock.On call -// - ctx context.Context -// - in *milvuspb.GetComponentStatesRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *milvuspb.GetComponentStatesRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) GetComponentStates(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetComponentStates_Call { return &MockDataNodeClient_GetComponentStates_Call{Call: _e.mock.On("GetComponentStates", append([]interface{}{ctx, in}, opts...)...)} @@ -601,9 +601,9 @@ type MockDataNodeClient_GetMetrics_Call struct { } // GetMetrics is a helper method to define mock.On call -// - ctx context.Context -// - in *milvuspb.GetMetricsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *milvuspb.GetMetricsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) GetMetrics(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetMetrics_Call { return &MockDataNodeClient_GetMetrics_Call{Call: _e.mock.On("GetMetrics", append([]interface{}{ctx, in}, opts...)...)} @@ -671,9 +671,9 @@ type MockDataNodeClient_GetStatisticsChannel_Call struct { } // GetStatisticsChannel is a helper method to define mock.On call -// - ctx context.Context -// - in *internalpb.GetStatisticsChannelRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *internalpb.GetStatisticsChannelRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) GetStatisticsChannel(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_GetStatisticsChannel_Call { return &MockDataNodeClient_GetStatisticsChannel_Call{Call: _e.mock.On("GetStatisticsChannel", append([]interface{}{ctx, in}, opts...)...)} @@ -741,9 +741,9 @@ type MockDataNodeClient_ImportV2_Call struct { } // ImportV2 is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.ImportRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.ImportRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) ImportV2(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ImportV2_Call { return &MockDataNodeClient_ImportV2_Call{Call: _e.mock.On("ImportV2", append([]interface{}{ctx, in}, opts...)...)} @@ -811,9 +811,9 @@ type MockDataNodeClient_NotifyChannelOperation_Call struct { } // NotifyChannelOperation is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.ChannelOperationsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.ChannelOperationsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) NotifyChannelOperation(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_NotifyChannelOperation_Call { return &MockDataNodeClient_NotifyChannelOperation_Call{Call: _e.mock.On("NotifyChannelOperation", append([]interface{}{ctx, in}, opts...)...)} @@ -881,9 +881,9 @@ type MockDataNodeClient_PreImport_Call struct { } // PreImport is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.PreImportRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.PreImportRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) PreImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_PreImport_Call { return &MockDataNodeClient_PreImport_Call{Call: _e.mock.On("PreImport", append([]interface{}{ctx, in}, opts...)...)} @@ -951,9 +951,9 @@ type MockDataNodeClient_QueryImport_Call struct { } // QueryImport is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.QueryImportRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.QueryImportRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) QueryImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_QueryImport_Call { return &MockDataNodeClient_QueryImport_Call{Call: _e.mock.On("QueryImport", append([]interface{}{ctx, in}, opts...)...)} @@ -1021,9 +1021,9 @@ type MockDataNodeClient_QueryPreImport_Call struct { } // QueryPreImport is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.QueryPreImportRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.QueryPreImportRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) QueryPreImport(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_QueryPreImport_Call { return &MockDataNodeClient_QueryPreImport_Call{Call: _e.mock.On("QueryPreImport", append([]interface{}{ctx, in}, opts...)...)} @@ -1091,9 +1091,9 @@ type MockDataNodeClient_QuerySlot_Call struct { } // QuerySlot is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.QuerySlotRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.QuerySlotRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) QuerySlot(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_QuerySlot_Call { return &MockDataNodeClient_QuerySlot_Call{Call: _e.mock.On("QuerySlot", append([]interface{}{ctx, in}, opts...)...)} @@ -1161,9 +1161,9 @@ type MockDataNodeClient_ResendSegmentStats_Call struct { } // ResendSegmentStats is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.ResendSegmentStatsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.ResendSegmentStatsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) ResendSegmentStats(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ResendSegmentStats_Call { return &MockDataNodeClient_ResendSegmentStats_Call{Call: _e.mock.On("ResendSegmentStats", append([]interface{}{ctx, in}, opts...)...)} @@ -1231,9 +1231,9 @@ type MockDataNodeClient_ShowConfigurations_Call struct { } // ShowConfigurations is a helper method to define mock.On call -// - ctx context.Context -// - in *internalpb.ShowConfigurationsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *internalpb.ShowConfigurationsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) ShowConfigurations(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_ShowConfigurations_Call { return &MockDataNodeClient_ShowConfigurations_Call{Call: _e.mock.On("ShowConfigurations", append([]interface{}{ctx, in}, opts...)...)} @@ -1301,9 +1301,9 @@ type MockDataNodeClient_SyncSegments_Call struct { } // SyncSegments is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.SyncSegmentsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.SyncSegmentsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) SyncSegments(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_SyncSegments_Call { return &MockDataNodeClient_SyncSegments_Call{Call: _e.mock.On("SyncSegments", append([]interface{}{ctx, in}, opts...)...)} @@ -1371,9 +1371,9 @@ type MockDataNodeClient_WatchDmChannels_Call struct { } // WatchDmChannels is a helper method to define mock.On call -// - ctx context.Context -// - in *datapb.WatchDmChannelsRequest -// - opts ...grpc.CallOption +// - ctx context.Context +// - in *datapb.WatchDmChannelsRequest +// - opts ...grpc.CallOption func (_e *MockDataNodeClient_Expecter) WatchDmChannels(ctx interface{}, in interface{}, opts ...interface{}) *MockDataNodeClient_WatchDmChannels_Call { return &MockDataNodeClient_WatchDmChannels_Call{Call: _e.mock.On("WatchDmChannels", append([]interface{}{ctx, in}, opts...)...)}