milvus/internal/datanode/index/task_stats_test.go
aoiasd ee216877bb
enhance: support compaction with file resource in ref mode (#46399)
Add support for DataNode compaction using file resources in ref mode.
SortCompation and StatsJobs will build text indexes, which may use file
resources.
relate: https://github.com/milvus-io/milvus/issues/43687

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: file resources (analyzer binaries/metadata) are only
fetched, downloaded and used when the node is configured in Ref mode
(fileresource.IsRefMode via CommonCfg.QNFileResourceMode /
DNFileResourceMode); Sync now carries a version and managers track
per-resource versions/resource IDs so newer resource sets win and older
entries are pruned (RefManager/SynchManager resource maps).
- Logic removed / simplified: component-specific FileResourceMode flags
and an indirection through a long-lived BinlogIO wrapper were
consolidated — file-resource mode moved to CommonCfg, Sync/Download APIs
became version- and context-aware, and compaction/index tasks accept a
ChunkManager directly (binlog IO wrapper creation inlined). This
eliminates duplicated config checks and wrapper indirection while
preserving the same chunk/IO semantics.
- Why no data loss or behavior regression: all file-resource code paths
are gated by the configured mode (default remains "sync"); when not in
ref-mode or when no resources exist, compaction and stats flows follow
existing code paths unchanged. Versioned Sync + resourceID maps ensure
newly synced sets replace older ones and RefManager prunes stale files;
GetFileResources returns an error if requested IDs are missing (prevents
silent use of wrong resources). Analyzer naming/parameter changes add
analyzer_extra_info but default-callers pass "" so existing analyzers
and index contents remain unchanged.
- New capability: DataNode compaction and StatsJobs can now build text
indexes using external file resources in Ref mode — DataCoord exposes
GetFileResources and populates CompactionPlan.file_resources;
SortCompaction/StatsTask download resources via fileresource.Manager,
produce an analyzer_extra_info JSON (storage + resource->id map) via
analyzer.BuildExtraResourceInfo, and propagate analyzer_extra_info into
BuildIndexInfo so the tantivy bindings can load custom analyzers during
text index creation.
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: aoiasd <zhicheng.yue@zilliz.com>
2026-01-06 16:31:31 +08:00

278 lines
8.5 KiB
Go

// 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 index
import (
"context"
"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/compactor"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/mocks/flushcommon/mock_util"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexcgopb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/proto/workerpb"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/tsoutil"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
func TestTaskStatsSuite(t *testing.T) {
suite.Run(t, new(TaskStatsSuite))
}
type TaskStatsSuite struct {
suite.Suite
collectionID int64
partitionID int64
clusterID string
schema *schemapb.CollectionSchema
mockBinlogIO *mock_util.MockBinlogIO
mockChunkManager *mocks.ChunkManager
segWriter *compactor.SegmentWriter
}
func (s *TaskStatsSuite) SetupSuite() {
s.collectionID = 100
s.partitionID = 101
s.clusterID = "102"
}
func (s *TaskStatsSuite) SetupSubTest() {
paramtable.Init()
s.mockBinlogIO = mock_util.NewMockBinlogIO(s.T())
s.mockChunkManager = mocks.NewChunkManager(s.T())
}
func (s *TaskStatsSuite) GenSegmentWriterWithBM25(magic int64) {
segWriter, err := compactor.NewSegmentWriter(s.schema, 100, statsBatchSize, magic, s.partitionID, s.collectionID, []int64{102})
s.Require().NoError(err)
v := storage.Value{
PK: storage.NewInt64PrimaryKey(magic),
Timestamp: int64(tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)),
Value: genRowWithBM25(magic),
}
err = segWriter.Write(&v)
s.Require().NoError(err)
segWriter.FlushAndIsFull()
s.segWriter = segWriter
}
func (s *TaskStatsSuite) TestSortSegmentWithBM25() {
s.Run("normal case", func() {
s.schema = genCollectionSchemaWithBM25()
s.GenSegmentWriterWithBM25(0)
_, kvs, fBinlogs, err := serializeWrite(context.TODO(), "root_path", 0, s.segWriter)
s.NoError(err)
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, paths []string) ([][]byte, error) {
result := make([][]byte, len(paths))
for i, path := range paths {
result[i] = kvs[path]
}
return result, nil
})
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(nil)
ctx, cancel := context.WithCancel(context.Background())
testTaskKey := Key{ClusterID: s.clusterID, TaskID: 100}
manager := NewTaskManager(ctx)
manager.LoadOrStoreStatsTask(s.clusterID, testTaskKey.TaskID, &StatsTaskInfo{SegID: 1})
task := NewStatsTask(ctx, cancel, &workerpb.CreateStatsRequest{
CollectionID: s.collectionID,
PartitionID: s.partitionID,
ClusterID: s.clusterID,
TaskID: testTaskKey.TaskID,
TargetSegmentID: 1,
InsertLogs: lo.Values(fBinlogs),
Schema: s.schema,
NumRows: 1,
StartLogID: 0,
EndLogID: 7,
BinlogMaxSize: 64 * 1024 * 1024,
StorageConfig: &indexpb.StorageConfig{
RootPath: "root_path",
},
}, manager, s.mockChunkManager)
task.binlogIO = s.mockBinlogIO
err = task.PreExecute(ctx)
s.Require().NoError(err)
binlog, err := task.sort(ctx)
s.Require().NoError(err)
s.Equal(5, len(binlog))
// check bm25 log
s.Equal(1, len(manager.statsTasks))
for key, task := range manager.statsTasks {
s.Equal(testTaskKey.ClusterID, key.ClusterID)
s.Equal(testTaskKey.TaskID, key.TaskID)
s.Equal(1, len(task.Bm25Logs))
}
})
s.Run("upload bm25 binlog failed", func() {
s.schema = genCollectionSchemaWithBM25()
s.GenSegmentWriterWithBM25(0)
_, kvs, fBinlogs, err := serializeWrite(context.TODO(), "root_path", 0, s.segWriter)
s.NoError(err)
s.mockBinlogIO.EXPECT().Download(mock.Anything, mock.Anything).RunAndReturn(func(ctx context.Context, paths []string) ([][]byte, error) {
result := make([][]byte, len(paths))
for i, path := range paths {
result[i] = kvs[path]
}
return result, nil
})
s.mockBinlogIO.EXPECT().Upload(mock.Anything, mock.Anything).Return(errors.New("mock error")).Once()
ctx, cancel := context.WithCancel(context.Background())
testTaskKey := Key{ClusterID: s.clusterID, TaskID: 100}
manager := NewTaskManager(ctx)
manager.LoadOrStoreStatsTask(s.clusterID, testTaskKey.TaskID, &StatsTaskInfo{SegID: 1})
task := NewStatsTask(ctx, cancel, &workerpb.CreateStatsRequest{
CollectionID: s.collectionID,
PartitionID: s.partitionID,
ClusterID: s.clusterID,
TaskID: testTaskKey.TaskID,
TargetSegmentID: 1,
InsertLogs: lo.Values(fBinlogs),
Schema: s.schema,
NumRows: 1,
StartLogID: 0,
EndLogID: 7,
BinlogMaxSize: 64 * 1024 * 1024,
StorageConfig: &indexpb.StorageConfig{
RootPath: "root_path",
},
}, manager, s.mockChunkManager)
task.binlogIO = s.mockBinlogIO
err = task.PreExecute(ctx)
s.Require().NoError(err)
_, err = task.sort(ctx)
s.Error(err)
})
}
func (s *TaskStatsSuite) TestBuildIndexParams() {
s.Run("test storage v2 index params", func() {
req := &workerpb.CreateStatsRequest{
TaskID: 1,
CollectionID: 2,
PartitionID: 3,
TargetSegmentID: 4,
TaskVersion: 5,
CurrentScalarIndexVersion: int32(1),
StorageVersion: storage.StorageV2,
InsertLogs: []*datapb.FieldBinlog{},
StorageConfig: &indexpb.StorageConfig{RootPath: "/test/path"},
}
options := &BuildIndexOptions{
TantivyMemory: 0,
JsonStatsMaxShreddingColumns: 256,
JsonStatsShreddingRatio: 0.3,
JsonStatsWriteBatchSize: 81920,
}
params := buildIndexParams(req, []string{"file1", "file2"}, nil, &indexcgopb.StorageConfig{}, options)
s.Equal(storage.StorageV2, params.StorageVersion)
s.NotNil(params.SegmentInsertFiles)
})
}
func genCollectionSchemaWithBM25() *schemapb.CollectionSchema {
return &schemapb.CollectionSchema{
Name: "schema",
Description: "schema",
Fields: []*schemapb.FieldSchema{
{
FieldID: common.RowIDField,
Name: "row_id",
DataType: schemapb.DataType_Int64,
},
{
FieldID: common.TimeStampField,
Name: "Timestamp",
DataType: schemapb.DataType_Int64,
},
{
FieldID: 100,
Name: "pk",
DataType: schemapb.DataType_Int64,
IsPrimaryKey: true,
},
{
FieldID: 101,
Name: "text",
DataType: schemapb.DataType_VarChar,
TypeParams: []*commonpb.KeyValuePair{
{
Key: common.MaxLengthKey,
Value: "8",
},
},
},
{
FieldID: 102,
Name: "sparse",
DataType: schemapb.DataType_SparseFloatVector,
},
},
Functions: []*schemapb.FunctionSchema{{
Name: "BM25",
Id: 100,
Type: schemapb.FunctionType_BM25,
InputFieldNames: []string{"text"},
InputFieldIds: []int64{101},
OutputFieldNames: []string{"sparse"},
OutputFieldIds: []int64{102},
}},
}
}
func genRowWithBM25(magic int64) map[int64]interface{} {
ts := tsoutil.ComposeTSByTime(getMilvusBirthday(), 0)
return map[int64]interface{}{
common.RowIDField: magic,
common.TimeStampField: int64(ts),
100: magic,
101: "varchar",
102: typeutil.CreateAndSortSparseFloatRow(map[uint32]float32{1: 1}),
}
}
func getMilvusBirthday() time.Time {
return time.Date(2019, time.Month(5), 30, 0, 0, 0, 0, time.UTC)
}