milvus/internal/metastore/model/collection.go
aoiasd 55feb7ded8
feat: set related resource ids in collection schema (#46423)
Support crate analyzer with file resource info, and return used file
resource ids when validate analyzer.
Save the related resource ids in collection schema.
relate: https://github.com/milvus-io/milvus/issues/43687

<!-- This is an auto-generated comment: release notes by coderabbit.ai
-->
- Core invariant: analyzer file-resource resolution is deterministic and
traceable by threading a FileResourcePathHelper (collecting used
resource IDs in a HashSet) through all tokenizer/analyzer construction
and validation paths; validate_analyzer(params, extra_info) returns the
collected Vec<i64) which is propagated through C/Rust/Go layers to
callers (CValidateResult → RustResult::from_vec_i64 → Go []int64 →
querypb.ValidateAnalyzerResponse.ResourceIds →
CollectionSchema.FileResourceIds).

- Logic removed/simplified: ad‑hoc, scattered resource-path lookups and
per-filter file helpers (e.g., read_synonyms_file and other inline
file-reading logic) were consolidated into ResourceInfo +
FileResourcePathHelper and a centralized get_resource_path(helper, ...)
API; filter/tokenizer builder APIs now accept &mut
FileResourcePathHelper so all file path resolution and ID collection use
the same path and bookkeeping logic (redundant duplicated lookups
removed).

- Why no data loss or behavior regression: changes are additive and
default-preserving — existing call sites pass extra_info = "" so
analyzer creation/validation behavior and error paths remain unchanged;
new Collection.FileResourceIds is populated from resp.ResourceIds in
validateSchema and round‑tripped through marshal/unmarshal
(model.Collection ↔ schemapb.CollectionSchema) so schema persistence
uses the new list without overwriting other schema fields; proto change
adds a repeated field (resource_ids) which is wire‑compatible (older
clients ignore extra field). Concrete code paths: analyzer creation
still uses create_analyzer (now with extra_info ""), tokenizer
validation still returns errors as before but now also returns IDs via
CValidateResult/RustResult, and rootcoord.validateSchema assigns
resp.ResourceIds → schema.FileResourceIds.

- New capability added: end‑to‑end discovery, return, and persistence of
file resource IDs used by analyzers — validate flows now return resource
IDs and the system stores them in collection schema (affects tantivy
analyzer binding, canalyzer C bindings, internal/util analyzer APIs,
querynode ValidateAnalyzer response, and rootcoord/create_collection
flow).
<!-- end of auto-generated comment: release notes by coderabbit.ai -->

Signed-off-by: aoiasd <zhicheng.yue@zilliz.com>
2025-12-26 22:49:19 +08:00

351 lines
12 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 model
import (
"slices"
"github.com/samber/lo"
"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/pkg/v2/common"
pb "github.com/milvus-io/milvus/pkg/v2/proto/etcdpb"
"github.com/milvus-io/milvus/pkg/v2/streaming/util/message"
)
// TODO: These collection is dirty implementation and easy to be broken, we should drop it in the future.
type Collection struct {
TenantID string
DBID int64
CollectionID int64
Partitions []*Partition
Name string
DBName string
Description string
AutoID bool
Fields []*Field
StructArrayFields []*StructArrayField
Functions []*Function
VirtualChannelNames []string
PhysicalChannelNames []string
ShardsNum int32
StartPositions []*commonpb.KeyDataPair
CreateTime uint64
ConsistencyLevel commonpb.ConsistencyLevel
Aliases []string // TODO: deprecate this.
Properties []*commonpb.KeyValuePair
State pb.CollectionState
EnableDynamicField bool
UpdateTimestamp uint64
SchemaVersion int32
ShardInfos map[string]*ShardInfo
FileResourceIds []int64
}
type ShardInfo struct {
PChannelName string // the pchannel name of the shard, it is the same with the physical channel name.
VChannelName string // the vchannel name of the shard, it is the same with the virtual channel name.
LastTruncateTimeTick uint64 // the last truncate time tick of the shard, if the shard is not truncated, the value is 0.
}
func (c *Collection) Available() bool {
return c.State == pb.CollectionState_CollectionCreated
}
func (c *Collection) ShallowClone() *Collection {
return &Collection{
TenantID: c.TenantID,
DBID: c.DBID,
CollectionID: c.CollectionID,
Name: c.Name,
DBName: c.DBName,
Description: c.Description,
AutoID: c.AutoID,
Fields: c.Fields,
StructArrayFields: c.StructArrayFields,
Partitions: c.Partitions,
VirtualChannelNames: c.VirtualChannelNames,
PhysicalChannelNames: c.PhysicalChannelNames,
ShardsNum: c.ShardsNum,
ConsistencyLevel: c.ConsistencyLevel,
CreateTime: c.CreateTime,
StartPositions: c.StartPositions,
Aliases: c.Aliases,
Properties: c.Properties,
State: c.State,
EnableDynamicField: c.EnableDynamicField,
Functions: c.Functions,
UpdateTimestamp: c.UpdateTimestamp,
SchemaVersion: c.SchemaVersion,
ShardInfos: c.ShardInfos,
FileResourceIds: c.FileResourceIds,
}
}
func (c *Collection) Clone() *Collection {
shardInfos := make(map[string]*ShardInfo, len(c.ShardInfos))
for channelName, shardInfo := range c.ShardInfos {
shardInfos[channelName] = &ShardInfo{
VChannelName: channelName,
PChannelName: shardInfo.PChannelName,
LastTruncateTimeTick: shardInfo.LastTruncateTimeTick,
}
}
return &Collection{
TenantID: c.TenantID,
DBID: c.DBID,
CollectionID: c.CollectionID,
Name: c.Name,
DBName: c.DBName,
Description: c.Description,
AutoID: c.AutoID,
Fields: CloneFields(c.Fields),
StructArrayFields: CloneStructArrayFields(c.StructArrayFields),
Partitions: ClonePartitions(c.Partitions),
VirtualChannelNames: common.CloneStringList(c.VirtualChannelNames),
PhysicalChannelNames: common.CloneStringList(c.PhysicalChannelNames),
ShardsNum: c.ShardsNum,
ConsistencyLevel: c.ConsistencyLevel,
CreateTime: c.CreateTime,
StartPositions: common.CloneKeyDataPairs(c.StartPositions),
Aliases: common.CloneStringList(c.Aliases),
Properties: common.CloneKeyValuePairs(c.Properties),
State: c.State,
EnableDynamicField: c.EnableDynamicField,
Functions: CloneFunctions(c.Functions),
UpdateTimestamp: c.UpdateTimestamp,
SchemaVersion: c.SchemaVersion,
ShardInfos: shardInfos,
FileResourceIds: slices.Clone(c.FileResourceIds),
}
}
func (c *Collection) GetPartitionNum(filterUnavailable bool) int {
if !filterUnavailable {
return len(c.Partitions)
}
return lo.CountBy(c.Partitions, func(p *Partition) bool { return p.Available() })
}
func (c *Collection) Equal(other Collection) bool {
return c.TenantID == other.TenantID &&
c.DBID == other.DBID &&
CheckPartitionsEqual(c.Partitions, other.Partitions) &&
c.Name == other.Name &&
c.Description == other.Description &&
c.AutoID == other.AutoID &&
CheckFieldsEqual(c.Fields, other.Fields) &&
CheckStructArrayFieldsEqual(c.StructArrayFields, other.StructArrayFields) &&
c.ShardsNum == other.ShardsNum &&
c.ConsistencyLevel == other.ConsistencyLevel &&
checkParamsEqual(c.Properties, other.Properties) &&
c.EnableDynamicField == other.EnableDynamicField
}
func (c *Collection) ApplyUpdates(header *message.AlterCollectionMessageHeader, body *message.AlterCollectionMessageBody) {
updateMask := header.UpdateMask
updates := body.Updates
for _, field := range updateMask.GetPaths() {
switch field {
case message.FieldMaskDB:
c.DBID = updates.DbId
c.DBName = updates.DbName
case message.FieldMaskCollectionName:
c.Name = updates.CollectionName
case message.FieldMaskCollectionDescription:
c.Description = updates.Description
case message.FieldMaskCollectionConsistencyLevel:
c.ConsistencyLevel = updates.ConsistencyLevel
case message.FieldMaskCollectionProperties:
c.Properties = updates.Properties
case message.FieldMaskCollectionSchema:
c.AutoID = updates.Schema.AutoID
c.Fields = UnmarshalFieldModels(updates.Schema.Fields)
c.EnableDynamicField = updates.Schema.EnableDynamicField
c.Functions = UnmarshalFunctionModels(updates.Schema.Functions)
c.StructArrayFields = UnmarshalStructArrayFieldModels(updates.Schema.StructArrayFields)
c.SchemaVersion = updates.Schema.Version
}
}
}
func UnmarshalCollectionModel(coll *pb.CollectionInfo) *Collection {
if coll == nil {
return nil
}
// backward compatible for deprecated fields
partitions := make([]*Partition, len(coll.PartitionIDs))
for idx := range coll.PartitionIDs {
partitions[idx] = &Partition{
PartitionID: coll.PartitionIDs[idx],
PartitionName: coll.PartitionNames[idx],
PartitionCreatedTimestamp: coll.PartitionCreatedTimestamps[idx],
}
}
shardInfos := make(map[string]*ShardInfo, len(coll.VirtualChannelNames))
for idx, channelName := range coll.VirtualChannelNames {
if len(coll.ShardInfos) == 0 {
shardInfos[channelName] = &ShardInfo{
VChannelName: channelName,
PChannelName: coll.PhysicalChannelNames[idx],
LastTruncateTimeTick: 0,
}
} else {
shardInfos[channelName] = &ShardInfo{
VChannelName: channelName,
PChannelName: coll.PhysicalChannelNames[idx],
LastTruncateTimeTick: coll.ShardInfos[idx].LastTruncateTimeTick,
}
}
}
return &Collection{
CollectionID: coll.ID,
DBID: coll.DbId,
Name: coll.Schema.Name,
DBName: coll.Schema.DbName,
Description: coll.Schema.Description,
AutoID: coll.Schema.AutoID,
Fields: UnmarshalFieldModels(coll.GetSchema().GetFields()),
StructArrayFields: UnmarshalStructArrayFieldModels(coll.GetSchema().GetStructArrayFields()),
Partitions: partitions,
VirtualChannelNames: coll.VirtualChannelNames,
PhysicalChannelNames: coll.PhysicalChannelNames,
ShardsNum: coll.ShardsNum,
ConsistencyLevel: coll.ConsistencyLevel,
CreateTime: coll.CreateTime,
StartPositions: coll.StartPositions,
State: coll.State,
Properties: coll.Properties,
EnableDynamicField: coll.Schema.EnableDynamicField,
UpdateTimestamp: coll.UpdateTimestamp,
SchemaVersion: coll.Schema.Version,
ShardInfos: shardInfos,
FileResourceIds: coll.Schema.GetFileResourceIds(),
}
}
// MarshalCollectionModel marshal only collection-related information.
// partitions, aliases and fields won't be marshaled. They should be written to newly path.
func MarshalCollectionModel(coll *Collection) *pb.CollectionInfo {
return marshalCollectionModelWithConfig(coll, newDefaultConfig())
}
type config struct {
withFields bool
withPartitions bool
withStructArrayFields bool
}
type Option func(c *config)
func newDefaultConfig() *config {
return &config{withFields: false, withPartitions: false, withStructArrayFields: false}
}
func WithFields() Option {
return func(c *config) {
c.withFields = true
}
}
func WithPartitions() Option {
return func(c *config) {
c.withPartitions = true
}
}
func WithStructArrayFields() Option {
return func(c *config) {
c.withStructArrayFields = true
}
}
func marshalCollectionModelWithConfig(coll *Collection, c *config) *pb.CollectionInfo {
if coll == nil {
return nil
}
collSchema := &schemapb.CollectionSchema{
Name: coll.Name,
Description: coll.Description,
AutoID: coll.AutoID,
EnableDynamicField: coll.EnableDynamicField,
DbName: coll.DBName,
Version: coll.SchemaVersion,
FileResourceIds: coll.FileResourceIds,
}
if c.withFields {
fields := MarshalFieldModels(coll.Fields)
collSchema.Fields = fields
}
if c.withStructArrayFields {
structArrayFields := MarshalStructArrayFieldModels(coll.StructArrayFields)
collSchema.StructArrayFields = structArrayFields
}
shardInfos := make([]*pb.CollectionShardInfo, len(coll.ShardInfos))
for idx, channelName := range coll.VirtualChannelNames {
if shard, ok := coll.ShardInfos[channelName]; ok {
shardInfos[idx] = &pb.CollectionShardInfo{
LastTruncateTimeTick: shard.LastTruncateTimeTick,
}
} else {
shardInfos[idx] = &pb.CollectionShardInfo{
LastTruncateTimeTick: 0,
}
}
}
collectionPb := &pb.CollectionInfo{
ID: coll.CollectionID,
DbId: coll.DBID,
Schema: collSchema,
CreateTime: coll.CreateTime,
VirtualChannelNames: coll.VirtualChannelNames,
PhysicalChannelNames: coll.PhysicalChannelNames,
ShardsNum: coll.ShardsNum,
ConsistencyLevel: coll.ConsistencyLevel,
StartPositions: coll.StartPositions,
State: coll.State,
Properties: coll.Properties,
UpdateTimestamp: coll.UpdateTimestamp,
ShardInfos: shardInfos,
}
if c.withPartitions {
for _, partition := range coll.Partitions {
collectionPb.PartitionNames = append(collectionPb.PartitionNames, partition.PartitionName)
collectionPb.PartitionIDs = append(collectionPb.PartitionIDs, partition.PartitionID)
collectionPb.PartitionCreatedTimestamps = append(collectionPb.PartitionCreatedTimestamps, partition.PartitionCreatedTimestamp)
}
}
return collectionPb
}
func MarshalCollectionModelWithOption(coll *Collection, opts ...Option) *pb.CollectionInfo {
c := newDefaultConfig()
for _, opt := range opts {
opt(c)
}
return marshalCollectionModelWithConfig(coll, c)
}