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>
This commit is contained in:
aoiasd 2026-01-06 16:31:31 +08:00 committed by GitHub
parent b7ee93fc52
commit ee216877bb
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
52 changed files with 2172 additions and 1873 deletions

View File

@ -1042,6 +1042,10 @@ common:
enablePosixMode: false # Specifies whether to run in POSIX mode for enhanced file system compatibility
usingJSONShreddingForQuery: true # Indicates whether to use json stats when query
clusterID: 0 # cluster id
fileResource:
mode:
queryNode: sync # File resource mode for query node, options: [sync, close]. Default is sync.
dataNode: sync # File resource mode for data node, options: [sync, ref, close]. Default is sync.
# QuotaConfig, configurations of Milvus quota and limits.
# By default, we enable:

View File

@ -22,7 +22,7 @@
namespace milvus::index {
TextMatchIndex::TextMatchIndex(int64_t commit_interval_in_ms,
const char* unique_id,
const char* tokenizer_name,
const char* analyzer_name,
const char* analyzer_params)
: commit_interval_in_ms_(commit_interval_in_ms),
last_commit_time_(stdclock::now()) {
@ -33,7 +33,7 @@ TextMatchIndex::TextMatchIndex(int64_t commit_interval_in_ms,
"",
TANTIVY_INDEX_LATEST_VERSION /* Growing segment has no reason to use old version index*/
,
tokenizer_name,
analyzer_name,
analyzer_params);
set_is_growing(true);
}
@ -41,7 +41,7 @@ TextMatchIndex::TextMatchIndex(int64_t commit_interval_in_ms,
TextMatchIndex::TextMatchIndex(const std::string& path,
const char* unique_id,
uint32_t tantivy_index_version,
const char* tokenizer_name,
const char* analyzer_name,
const char* analyzer_params)
: commit_interval_in_ms_(std::numeric_limits<int64_t>::max()),
last_commit_time_(stdclock::now()) {
@ -54,14 +54,15 @@ TextMatchIndex::TextMatchIndex(const std::string& path,
false,
path_.c_str(),
tantivy_index_version,
tokenizer_name,
analyzer_name,
analyzer_params);
}
TextMatchIndex::TextMatchIndex(const storage::FileManagerContext& ctx,
uint32_t tantivy_index_version,
const char* tokenizer_name,
const char* analyzer_params)
const char* analyzer_name,
const char* analyzer_params,
const char* analyzer_extra_info)
: commit_interval_in_ms_(std::numeric_limits<int64_t>::max()),
last_commit_time_(stdclock::now()) {
schema_ = ctx.fieldDataMeta.field_schema;
@ -78,8 +79,9 @@ TextMatchIndex::TextMatchIndex(const storage::FileManagerContext& ctx,
false,
path_.c_str(),
tantivy_index_version,
tokenizer_name,
analyzer_params);
analyzer_name,
analyzer_params,
analyzer_extra_info);
}
TextMatchIndex::TextMatchIndex(const storage::FileManagerContext& ctx)
@ -296,9 +298,9 @@ TextMatchIndex::CreateReader(SetBitsetFn set_bitset) {
}
void
TextMatchIndex::RegisterTokenizer(const char* tokenizer_name,
const char* analyzer_params) {
wrapper_->register_tokenizer(tokenizer_name, analyzer_params);
TextMatchIndex::RegisterAnalyzer(const char* analyzer_name,
const char* analyzer_params) {
wrapper_->register_tokenizer(analyzer_name, analyzer_params);
}
TargetBitmap

View File

@ -26,19 +26,20 @@ class TextMatchIndex : public InvertedIndexTantivy<std::string> {
// for growing segment.
explicit TextMatchIndex(int64_t commit_interval_in_ms,
const char* unique_id,
const char* tokenizer_name,
const char* analyzer_name,
const char* analyzer_params);
// for sealed segment to create index from raw data during loading.
explicit TextMatchIndex(const std::string& path,
const char* unique_id,
uint32_t tantivy_index_version,
const char* tokenizer_name,
const char* analyzer_name,
const char* analyzer_params);
// for building index.
explicit TextMatchIndex(const storage::FileManagerContext& ctx,
uint32_t tantivy_index_version,
const char* tokenizer_name,
const char* analyzer_params);
const char* analyzer_name,
const char* analyzer_params,
const char* analyzer_extra_info);
// for loading built index
explicit TextMatchIndex(const storage::FileManagerContext& ctx);
@ -80,7 +81,7 @@ class TextMatchIndex : public InvertedIndexTantivy<std::string> {
CreateReader(SetBitsetFn set_bitset);
void
RegisterTokenizer(const char* tokenizer_name, const char* analyzer_params);
RegisterAnalyzer(const char* analyzer_name, const char* analyzer_params);
TargetBitmap
MatchQuery(const std::string& query, uint32_t min_should_match);

View File

@ -492,11 +492,14 @@ BuildTextIndex(ProtoLayoutInterface result,
auto field_schema =
FieldMeta::ParseFrom(build_index_info->field_schema());
auto index = std::make_unique<index::TextMatchIndex>(
fileManagerContext,
tantivy_index_version,
"milvus_tokenizer",
field_schema.get_analyzer_params().c_str());
field_schema.get_analyzer_params().c_str(),
build_index_info->analyzer_extra_info().c_str());
index->Build(config);
auto create_index_result = index->Upload(config);
create_index_result->SerializeAt(

View File

@ -1857,8 +1857,8 @@ ChunkedSegmentSealedImpl::CreateTextIndex(FieldId field_id) {
index->Reload();
index->RegisterTokenizer("milvus_tokenizer",
field_meta.get_analyzer_params().c_str());
index->RegisterAnalyzer("milvus_tokenizer",
field_meta.get_analyzer_params().c_str());
text_indexes_[field_id] = std::make_shared<index::TextMatchIndexHolder>(
std::move(index), cfg.GetScalarIndexEnableMmap());

View File

@ -1585,8 +1585,8 @@ SegmentGrowingImpl::CreateTextIndex(FieldId field_id) {
field_meta.get_analyzer_params().c_str());
index->Commit();
index->CreateReader(milvus::index::SetBitsetGrowing);
index->RegisterTokenizer("milvus_tokenizer",
field_meta.get_analyzer_params().c_str());
index->RegisterAnalyzer("milvus_tokenizer",
field_meta.get_analyzer_params().c_str());
text_indexes_[field_id] = std::move(index);
}

View File

@ -99,8 +99,8 @@ TextMatchIndexTranslator::get_cells(
},
milvus::ScopedTimer::LogLevel::Info);
index->Load(config_);
index->RegisterTokenizer("milvus_tokenizer",
load_info_.analyzer_params.c_str());
index->RegisterAnalyzer("milvus_tokenizer",
load_info_.analyzer_params.c_str());
}
LOG_INFO("load text match index success for field:{} of segment:{}",

View File

@ -302,7 +302,8 @@ RustResult tantivy_phrase_match_query(void *ptr, const char *query, uint32_t slo
RustResult tantivy_register_tokenizer(void *ptr,
const char *tokenizer_name,
const char *analyzer_params);
const char *analyzer_params,
const char *analyzer_extra_info);
RustResult tantivy_create_index(const char *field_name,
TantivyDataType data_type,
@ -479,8 +480,9 @@ RustResult tantivy_index_add_array_keywords_by_single_segment_writer(void *ptr,
RustResult tantivy_create_text_writer(const char *field_name,
const char *path,
uint32_t tantivy_index_version,
const char *tokenizer_name,
const char *analyzer_name,
const char *analyzer_params,
const char *analyzer_extra_info,
uintptr_t num_threads,
uintptr_t overall_memory_budget_in_bytes,
bool in_ram);

View File

@ -122,6 +122,7 @@ mod tests {
dir.path().to_str().unwrap(),
"jieba",
&params,
"",
1,
50_000_000,
false,
@ -159,6 +160,7 @@ mod tests {
dir.path().to_str().unwrap(),
"default",
"",
"",
1,
50_000_000,
false,
@ -188,6 +190,7 @@ mod tests {
dir.path().to_str().unwrap(),
"default",
"",
"",
1,
50_000_000,
false,

View File

@ -44,12 +44,15 @@ pub extern "C" fn tantivy_register_tokenizer(
ptr: *mut c_void,
tokenizer_name: *const c_char,
analyzer_params: *const c_char,
analyzer_extra_info: *const c_char,
) -> RustResult {
init_log();
let real = ptr as *mut IndexReaderWrapper;
let tokenizer_name = cstr_to_str!(tokenizer_name);
let params = cstr_to_str!(analyzer_params);
let analyzer = create_analyzer(params, "");
let extra_info_str = cstr_to_str!(analyzer_extra_info);
let analyzer = create_analyzer(params, extra_info_str);
match analyzer {
Ok(text_analyzer) => unsafe {
(*real).register_tokenizer(String::from(tokenizer_name), text_analyzer);

View File

@ -287,6 +287,7 @@ mod tests {
dir.path().to_str().unwrap(),
"default",
"",
"",
1,
50_000_000,
false,

View File

@ -9,8 +9,9 @@ impl IndexWriterWrapper {
pub(crate) fn create_text_writer(
field_name: &str,
path: &str,
tokenizer_name: &str,
tokenizer_params: &str,
analyzer_name: &str,
analyzer_params: &str,
analyzer_extra_info: &str,
num_threads: usize,
overall_memory_budget_in_bytes: usize,
in_ram: bool,
@ -21,8 +22,8 @@ impl IndexWriterWrapper {
index_writer_v5::IndexWriterWrapperImpl::create_text_writer(
field_name,
path,
tokenizer_name,
tokenizer_params,
analyzer_name,
analyzer_params,
num_threads,
overall_memory_budget_in_bytes,
in_ram,
@ -32,8 +33,9 @@ impl IndexWriterWrapper {
index_writer_v7::IndexWriterWrapperImpl::create_text_writer(
field_name,
path,
tokenizer_name,
tokenizer_params,
analyzer_name,
analyzer_params,
analyzer_extra_info,
num_threads,
overall_memory_budget_in_bytes,
in_ram,

View File

@ -13,8 +13,9 @@ pub extern "C" fn tantivy_create_text_writer(
field_name: *const c_char,
path: *const c_char,
tantivy_index_version: u32,
tokenizer_name: *const c_char,
analyzer_name: *const c_char,
analyzer_params: *const c_char,
analyzer_extra_info: *const c_char,
num_threads: usize,
overall_memory_budget_in_bytes: usize,
in_ram: bool,
@ -22,9 +23,9 @@ pub extern "C" fn tantivy_create_text_writer(
init_log();
let field_name_str = cstr_to_str!(field_name);
let path_str = cstr_to_str!(path);
let tokenizer_name_str = cstr_to_str!(tokenizer_name);
let name_str = cstr_to_str!(analyzer_name);
let params = cstr_to_str!(analyzer_params);
let extra_info_str = cstr_to_str!(analyzer_extra_info);
let tantivy_index_version = match TantivyIndexVersion::from_u32(tantivy_index_version) {
Ok(v) => v,
Err(e) => return RustResult::from_error(e.to_string()),
@ -33,8 +34,9 @@ pub extern "C" fn tantivy_create_text_writer(
match IndexWriterWrapper::create_text_writer(
field_name_str,
path_str,
tokenizer_name_str,
name_str,
params,
extra_info_str,
num_threads,
overall_memory_budget_in_bytes,
in_ram,

View File

@ -26,8 +26,9 @@ impl IndexWriterWrapperImpl {
pub(crate) fn create_text_writer(
field_name: &str,
path: &str,
tokenizer_name: &str,
tokenizer_params: &str,
analyzer_name: &str,
analyzer_params: &str,
analyzer_extra_info: &str,
num_threads: usize,
overall_memory_budget_in_bytes: usize,
in_ram: bool,
@ -37,15 +38,15 @@ impl IndexWriterWrapperImpl {
field_name
);
let tokenizer = create_analyzer(tokenizer_params, "")?;
let analyzer = create_analyzer(analyzer_params, analyzer_extra_info)?;
let (schema, field) = build_text_schema(field_name, tokenizer_name);
let (schema, field) = build_text_schema(field_name, analyzer_name);
let index = if in_ram {
Index::create_in_ram(schema)
} else {
Index::create_in_dir(path, schema).unwrap()
};
index.tokenizers().register(tokenizer_name, tokenizer);
index.tokenizers().register(analyzer_name, analyzer);
let index_writer = index
.writer_with_num_threads(num_threads, overall_memory_budget_in_bytes)
.unwrap();

View File

@ -18,8 +18,8 @@
namespace milvus::tantivy {
using Map = std::map<std::string, std::string>;
static constexpr const char* DEFAULT_TOKENIZER_NAME = "milvus_tokenizer";
static const char* DEFAULT_analyzer_params = "{}";
static constexpr const char* DEFAULT_ANALYZER_NAME = "milvus_tokenizer";
static const char* DEFAULT_ANALYZER_PARAMS = "{}";
static constexpr uintptr_t DEFAULT_NUM_THREADS =
1; // Every field with index writer will generate a thread, make huge thread amount, wait for refactoring.
static constexpr uintptr_t DEFAULT_OVERALL_MEMORY_BUDGET_IN_BYTES =
@ -134,8 +134,9 @@ struct TantivyIndexWrapper {
bool in_ram,
const char* path,
uint32_t tantivy_index_version,
const char* tokenizer_name = DEFAULT_TOKENIZER_NAME,
const char* analyzer_params = DEFAULT_analyzer_params,
const char* analyzer_name = DEFAULT_ANALYZER_NAME,
const char* analyzer_params = DEFAULT_ANALYZER_PARAMS,
const char* analyzer_extra_info = "",
uintptr_t num_threads = DEFAULT_NUM_THREADS,
uintptr_t overall_memory_budget_in_bytes =
DEFAULT_OVERALL_MEMORY_BUDGET_IN_BYTES) {
@ -143,8 +144,9 @@ struct TantivyIndexWrapper {
tantivy_create_text_writer(field_name,
path,
tantivy_index_version,
tokenizer_name,
analyzer_name,
analyzer_params,
analyzer_extra_info,
num_threads,
overall_memory_budget_in_bytes,
in_ram));
@ -225,12 +227,20 @@ struct TantivyIndexWrapper {
free();
}
void
set_analyzer_extra_info(std::string analyzer_extra_info) {
analyzer_extra_info_ = analyzer_extra_info;
}
void
register_tokenizer(const char* tokenizer_name,
const char* analyzer_params) {
if (reader_ != nullptr) {
auto res = RustResultWrapper(tantivy_register_tokenizer(
reader_, tokenizer_name, analyzer_params));
auto res = RustResultWrapper(
tantivy_register_tokenizer(reader_,
tokenizer_name,
analyzer_params,
analyzer_extra_info_.c_str()));
AssertInfo(res.result_->success,
"failed to register tokenizer: {}",
res.result_->error);
@ -1198,5 +1208,6 @@ struct TantivyIndexWrapper {
IndexReader reader_ = nullptr;
std::string path_;
bool load_in_mmap_ = true;
std::string analyzer_extra_info_ = "";
};
} // namespace milvus::tantivy

View File

@ -14,6 +14,7 @@ import (
"github.com/milvus-io/milvus/internal/compaction"
"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/internal/datacoord/session"
"github.com/milvus-io/milvus/internal/util/fileresource"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/metrics"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
@ -387,6 +388,17 @@ func (t *mixCompactionTask) BuildCompactionRequest() (*datapb.CompactionPlan, er
JsonParams: compactionParams,
CurrentScalarIndexVersion: t.ievm.GetCurrentScalarIndexEngineVersion(),
}
// set analyzer resource for text match index if use ref mode
if fileresource.IsRefMode(paramtable.Get().CommonCfg.DNFileResourceMode.GetValue()) && taskProto.GetType() == datapb.CompactionType_SortCompaction && len(taskProto.GetSchema().GetFileResourceIds()) > 0 {
resources, err := t.meta.GetFileResources(context.Background(), taskProto.GetSchema().GetFileResourceIds()...)
if err != nil {
log.Warn("get file resources for collection failed", zap.Int64("collectionID", taskProto.GetCollectionID()), zap.Error(err))
return nil, errors.Errorf("get file resources for sort compaction failed: %v", err)
}
plan.FileResources = resources
}
segIDMap := make(map[int64][]*datapb.FieldBinlog, len(plan.SegmentBinlogs))
segments := make([]*SegmentInfo, 0, len(taskProto.GetInputSegments()))
for _, segID := range taskProto.GetInputSegments() {

View File

@ -85,7 +85,7 @@ func (m *FileResourceManager) syncLoop() {
}
func (m *FileResourceManager) Start() {
if fileresource.IsSyncMode(paramtable.Get().DataCoordCfg.FileResourceMode.GetValue()) {
if fileresource.IsSyncMode(paramtable.Get().CommonCfg.DNFileResourceMode.GetValue()) {
m.once.Do(func() {
m.notifyCh = make(chan struct{}, 1)
m.wg.Add(1)

View File

@ -63,6 +63,7 @@ func (suite *FileResourceManagerSuite) SetupTest() {
suite.testMeta = &meta{
catalog: suite.mockCatalog,
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
resourceIDMap: make(map[int64]*internalpb.FileResourceInfo),
resourceVersion: 0,
}

View File

@ -79,6 +79,7 @@ type CompactionMeta interface {
GetAnalyzeMeta() *analyzeMeta
GetPartitionStatsMeta() *partitionStatsMeta
GetCompactionTaskMeta() *compactionTaskMeta
GetFileResources(ctx context.Context, resourceIDs ...int64) ([]*internalpb.FileResourceInfo, error)
}
var _ CompactionMeta = (*meta)(nil)
@ -104,6 +105,7 @@ type meta struct {
// File Resource Meta
resourceMeta map[string]*internalpb.FileResourceInfo // name -> info
resourceIDMap map[int64]*internalpb.FileResourceInfo // id -> info
resourceVersion uint64
resourceLock lock.RWMutex
// Snapshot Meta
@ -220,8 +222,9 @@ func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManag
compactionTaskMeta: ctm,
statsTaskMeta: stm,
// externalCollectionTaskMeta: ectm,
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
snapshotMeta: spm,
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
resourceIDMap: make(map[int64]*internalpb.FileResourceInfo),
snapshotMeta: spm,
}
err = mt.reloadFromKV(ctx, broker)
if err != nil {
@ -2459,6 +2462,7 @@ func (m *meta) reloadFileResourceMeta(ctx context.Context) error {
m.resourceMeta = make(map[string]*internalpb.FileResourceInfo)
for _, resource := range resources {
m.resourceMeta[resource.Name] = resource
m.resourceIDMap[resource.Id] = resource
}
m.resourceVersion = version
return nil
@ -2479,6 +2483,7 @@ func (m *meta) AddFileResource(ctx context.Context, resource *internalpb.FileRes
}
m.resourceMeta[resource.Name] = resource
m.resourceIDMap[resource.Id] = resource
m.resourceVersion += 1
return nil
}
@ -2495,12 +2500,28 @@ func (m *meta) RemoveFileResource(ctx context.Context, name string) error {
}
delete(m.resourceMeta, name)
delete(m.resourceIDMap, resource.Id)
m.resourceVersion += 1
}
return nil
}
func (m *meta) GetFileResources(ctx context.Context, resourceIDs ...int64) ([]*internalpb.FileResourceInfo, error) {
m.resourceLock.RLock()
defer m.resourceLock.RUnlock()
resources := make([]*internalpb.FileResourceInfo, 0)
for _, resourceID := range resourceIDs {
if resource, ok := m.resourceIDMap[resourceID]; ok {
resources = append(resources, resource)
} else {
return nil, errors.Errorf("file resource %d not found", resourceID)
}
}
return resources, nil
}
// ListFileResource list file resources from meta
func (m *meta) ListFileResource(ctx context.Context) ([]*internalpb.FileResourceInfo, uint64) {
m.resourceLock.RLock()

View File

@ -6,6 +6,8 @@ import (
context "context"
datapb "github.com/milvus-io/milvus/pkg/v2/proto/datapb"
internalpb "github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
mock "github.com/stretchr/testify/mock"
)
@ -433,6 +435,79 @@ func (_c *MockCompactionMeta_GetCompactionTasksByTriggerID_Call) RunAndReturn(ru
return _c
}
// GetFileResources provides a mock function with given fields: ctx, resourceIDs
func (_m *MockCompactionMeta) GetFileResources(ctx context.Context, resourceIDs ...int64) ([]*internalpb.FileResourceInfo, error) {
_va := make([]interface{}, len(resourceIDs))
for _i := range resourceIDs {
_va[_i] = resourceIDs[_i]
}
var _ca []interface{}
_ca = append(_ca, ctx)
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
if len(ret) == 0 {
panic("no return value specified for GetFileResources")
}
var r0 []*internalpb.FileResourceInfo
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, ...int64) ([]*internalpb.FileResourceInfo, error)); ok {
return rf(ctx, resourceIDs...)
}
if rf, ok := ret.Get(0).(func(context.Context, ...int64) []*internalpb.FileResourceInfo); ok {
r0 = rf(ctx, resourceIDs...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).([]*internalpb.FileResourceInfo)
}
}
if rf, ok := ret.Get(1).(func(context.Context, ...int64) error); ok {
r1 = rf(ctx, resourceIDs...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockCompactionMeta_GetFileResources_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetFileResources'
type MockCompactionMeta_GetFileResources_Call struct {
*mock.Call
}
// GetFileResources is a helper method to define mock.On call
// - ctx context.Context
// - resourceIDs ...int64
func (_e *MockCompactionMeta_Expecter) GetFileResources(ctx interface{}, resourceIDs ...interface{}) *MockCompactionMeta_GetFileResources_Call {
return &MockCompactionMeta_GetFileResources_Call{Call: _e.mock.On("GetFileResources",
append([]interface{}{ctx}, resourceIDs...)...)}
}
func (_c *MockCompactionMeta_GetFileResources_Call) Run(run func(ctx context.Context, resourceIDs ...int64)) *MockCompactionMeta_GetFileResources_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]int64, len(args)-1)
for i, a := range args[1:] {
if a != nil {
variadicArgs[i] = a.(int64)
}
}
run(args[0].(context.Context), variadicArgs...)
})
return _c
}
func (_c *MockCompactionMeta_GetFileResources_Call) Return(_a0 []*internalpb.FileResourceInfo, _a1 error) *MockCompactionMeta_GetFileResources_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockCompactionMeta_GetFileResources_Call) RunAndReturn(run func(context.Context, ...int64) ([]*internalpb.FileResourceInfo, error)) *MockCompactionMeta_GetFileResources_Call {
_c.Call.Return(run)
return _c
}
// GetHealthySegment provides a mock function with given fields: ctx, segID
func (_m *MockCompactionMeta) GetHealthySegment(ctx context.Context, segID int64) *SegmentInfo {
ret := _m.Called(ctx, segID)

View File

@ -1793,8 +1793,9 @@ func TestServer_AddFileResource(t *testing.T) {
idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator),
mixCoord: newMockMixCoord(),
meta: &meta{
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
catalog: mockCatalog,
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
resourceIDMap: make(map[int64]*internalpb.FileResourceInfo),
catalog: mockCatalog,
},
}
server.stateCode.Store(commonpb.StateCode_Healthy)
@ -1836,8 +1837,9 @@ func TestServer_AddFileResource(t *testing.T) {
server := &Server{
idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator),
meta: &meta{
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
catalog: mockCatalog,
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
resourceIDMap: make(map[int64]*internalpb.FileResourceInfo),
catalog: mockCatalog,
},
}
server.stateCode.Store(commonpb.StateCode_Healthy)
@ -1860,8 +1862,9 @@ func TestServer_AddFileResource(t *testing.T) {
server := &Server{
idAllocator: globalIDAllocator.NewTestGlobalIDAllocator(mockAllocator),
meta: &meta{
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
catalog: mockCatalog,
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
resourceIDMap: make(map[int64]*internalpb.FileResourceInfo),
catalog: mockCatalog,
},
}
server.stateCode.Store(commonpb.StateCode_Healthy)
@ -1895,6 +1898,9 @@ func TestServer_AddFileResource(t *testing.T) {
resourceMeta: map[string]*internalpb.FileResourceInfo{
"test_resource": existingResource,
},
resourceIDMap: map[int64]*internalpb.FileResourceInfo{
1: existingResource,
},
catalog: mockCatalog,
},
}
@ -1927,6 +1933,9 @@ func TestServer_RemoveFileResource(t *testing.T) {
resourceMeta: map[string]*internalpb.FileResourceInfo{
"test_resource": existingResource,
},
resourceIDMap: map[int64]*internalpb.FileResourceInfo{
1: existingResource,
},
catalog: mockCatalog,
},
mixCoord: newMockMixCoord(),
@ -1963,8 +1972,9 @@ func TestServer_RemoveFileResource(t *testing.T) {
server := &Server{
meta: &meta{
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
catalog: mockCatalog,
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
resourceIDMap: map[int64]*internalpb.FileResourceInfo{},
catalog: mockCatalog,
},
mixCoord: newMockMixCoord(),
}
@ -1993,6 +2003,9 @@ func TestServer_RemoveFileResource(t *testing.T) {
resourceMeta: map[string]*internalpb.FileResourceInfo{
"test_resource": existingResource,
},
resourceIDMap: map[int64]*internalpb.FileResourceInfo{
1: existingResource,
},
catalog: mockCatalog,
},
}
@ -2016,8 +2029,9 @@ func TestServer_ListFileResources(t *testing.T) {
server := &Server{
meta: &meta{
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
catalog: mockCatalog,
resourceMeta: make(map[string]*internalpb.FileResourceInfo),
resourceIDMap: map[int64]*internalpb.FileResourceInfo{},
catalog: mockCatalog,
},
}
server.stateCode.Store(commonpb.StateCode_Healthy)
@ -2053,6 +2067,10 @@ func TestServer_ListFileResources(t *testing.T) {
"resource1": resource1,
"resource2": resource2,
},
resourceIDMap: map[int64]*internalpb.FileResourceInfo{
1: resource1,
2: resource2,
},
catalog: mockCatalog,
},
}

View File

@ -27,18 +27,21 @@ import (
"github.com/milvus-io/milvus/internal/datacoord/allocator"
"github.com/milvus-io/milvus/internal/datacoord/task"
"github.com/milvus-io/milvus/internal/util/fileresource"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
"github.com/milvus-io/milvus/pkg/v2/util/merr"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
"github.com/milvus-io/milvus/pkg/v2/util/typeutil"
)
type StatsInspector interface {
Start()
Stop()
SubmitStatsTask(originSegmentID, targetSegmentID int64, subJobType indexpb.StatsSubJob, canRecycle bool) error
SubmitStatsTask(originSegmentID, targetSegmentID int64, subJobType indexpb.StatsSubJob, canRecycle bool, resources []*internalpb.FileResourceInfo) error
GetStatsTask(originSegmentID int64, subJobType indexpb.StatsSubJob) *indexpb.StatsTask
DropStatsTask(originSegmentID int64, subJobType indexpb.StatsSubJob) error
}
@ -204,8 +207,18 @@ func (si *statsInspector) triggerTextStatsTask() {
return seg.GetIsSorted() && needDoTextIndex(seg, needTriggerFieldIDs)
}))
resources := []*internalpb.FileResourceInfo{}
var err error
if fileresource.IsRefMode(paramtable.Get().CommonCfg.DNFileResourceMode.GetValue()) && len(collection.Schema.GetFileResourceIds()) > 0 {
resources, err = si.mt.GetFileResources(si.ctx, collection.Schema.GetFileResourceIds()...)
if err != nil {
log.Warn("get file resources for collection failed, wait for retry", zap.Int64("collectionID", collection.ID), zap.Error(err))
continue
}
}
for _, segment := range segments {
if err := si.SubmitStatsTask(segment.GetID(), segment.GetID(), indexpb.StatsSubJob_TextIndexJob, true); err != nil {
if err := si.SubmitStatsTask(segment.GetID(), segment.GetID(), indexpb.StatsSubJob_TextIndexJob, true, resources); err != nil {
log.Warn("create stats task with text index for segment failed, wait for retry",
zap.Int64("segmentID", segment.GetID()), zap.Error(err))
continue
@ -235,7 +248,7 @@ func (si *statsInspector) triggerJsonKeyIndexStatsTask(lastJSONStatsLastTrigger
if maxJSONStatsTaskCount >= Params.DataCoordCfg.JSONStatsTriggerCount.GetAsInt() {
break
}
if err := si.SubmitStatsTask(segment.GetID(), segment.GetID(), indexpb.StatsSubJob_JsonKeyIndexJob, true); err != nil {
if err := si.SubmitStatsTask(segment.GetID(), segment.GetID(), indexpb.StatsSubJob_JsonKeyIndexJob, true, nil); err != nil {
log.Warn("create stats task with json key index for segment failed, wait for retry:",
zap.Int64("segmentID", segment.GetID()), zap.Error(err))
continue
@ -261,7 +274,7 @@ func (si *statsInspector) triggerBM25StatsTask() {
}))
for _, segment := range segments {
if err := si.SubmitStatsTask(segment.GetID(), segment.GetID(), indexpb.StatsSubJob_BM25Job, true); err != nil {
if err := si.SubmitStatsTask(segment.GetID(), segment.GetID(), indexpb.StatsSubJob_BM25Job, true, nil); err != nil {
log.Warn("create stats task with bm25 for segment failed, wait for retry",
zap.Int64("segmentID", segment.GetID()), zap.Error(err))
continue
@ -301,6 +314,7 @@ func (si *statsInspector) cleanupStatsTasksLoop() {
func (si *statsInspector) SubmitStatsTask(originSegmentID, targetSegmentID int64,
subJobType indexpb.StatsSubJob, canRecycle bool,
resources []*internalpb.FileResourceInfo,
) error {
originSegment := si.mt.GetHealthySegment(si.ctx, originSegmentID)
if originSegment == nil {
@ -329,6 +343,7 @@ func (si *statsInspector) SubmitStatsTask(originSegmentID, targetSegmentID int64
TargetSegmentID: targetSegmentID,
SubJobType: subJobType,
CanRecycle: canRecycle,
FileResources: resources,
}
if err = si.mt.statsTaskMeta.AddStatsTask(t); err != nil {
if errors.Is(err, merr.ErrTaskDuplicate) {

View File

@ -222,11 +222,11 @@ func (s *statsInspectorSuite) TestStart() {
func (s *statsInspectorSuite) TestSubmitStatsTask() {
// Test submitting a valid stats task
err := s.inspector.SubmitStatsTask(10, 10, indexpb.StatsSubJob_Sort, true)
err := s.inspector.SubmitStatsTask(10, 10, indexpb.StatsSubJob_Sort, true, nil)
s.NoError(err)
// Test submitting a task for non-existent segment
err = s.inspector.SubmitStatsTask(999, 999, indexpb.StatsSubJob_Sort, true)
err = s.inspector.SubmitStatsTask(999, 999, indexpb.StatsSubJob_Sort, true, nil)
s.Error(err)
s.True(errors.Is(err, merr.ErrSegmentNotFound), "Error should be ErrSegmentNotFound")
@ -242,7 +242,7 @@ func (s *statsInspectorSuite) TestSubmitStatsTask() {
})
// Simulate duplicate task error
err = s.inspector.SubmitStatsTask(10, 10, indexpb.StatsSubJob_Sort, true)
err = s.inspector.SubmitStatsTask(10, 10, indexpb.StatsSubJob_Sort, true, nil)
s.NoError(err) // Duplicate tasks are handled as success
}

View File

@ -37,6 +37,8 @@ import (
"github.com/milvus-io/milvus/internal/flushcommon/io"
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/analyzer"
"github.com/milvus-io/milvus/internal/util/fileresource"
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
@ -50,6 +52,7 @@ import (
type sortCompactionTask struct {
binlogIO io.BinlogIO
cm storage.ChunkManager
currentTime time.Time
plan *datapb.CompactionPlan
@ -80,7 +83,7 @@ var _ Compactor = (*sortCompactionTask)(nil)
func NewSortCompactionTask(
ctx context.Context,
binlogIO io.BinlogIO,
cm storage.ChunkManager,
plan *datapb.CompactionPlan,
compactionParams compaction.Params,
sortByFieldIDs []int64,
@ -89,7 +92,8 @@ func NewSortCompactionTask(
return &sortCompactionTask{
ctx: ctx1,
cancel: cancel,
binlogIO: binlogIO,
binlogIO: io.NewBinlogIO(cm),
cm: cm,
plan: plan,
tr: timerecord.NewTimeRecorder("sort compaction"),
currentTime: time.Now(),
@ -451,6 +455,19 @@ func (t *sortCompactionTask) createTextIndex(ctx context.Context,
eg, egCtx := errgroup.WithContext(ctx)
var analyzerExtraInfo string
if len(t.plan.GetFileResources()) > 0 {
err := fileresource.GlobalFileManager.Download(ctx, t.cm, t.plan.GetFileResources()...)
if err != nil {
return nil, err
}
defer fileresource.GlobalFileManager.Release(t.plan.GetFileResources()...)
analyzerExtraInfo, err = analyzer.BuildExtraResourceInfo(t.compactionParams.StorageConfig.GetRootPath(), t.plan.GetFileResources())
if err != nil {
return nil, err
}
}
for _, field := range t.plan.GetSchema().GetFields() {
field := field
h := typeutil.CreateFieldSchemaHelper(field)
@ -479,6 +496,10 @@ func (t *sortCompactionTask) createTextIndex(ctx context.Context,
Manifest: t.manifest,
}
if len(analyzerExtraInfo) > 0 {
buildIndexParams.AnalyzerExtraInfo = analyzerExtraInfo
}
if t.storageVersion == storage.StorageV2 {
buildIndexParams.SegmentInsertFiles = util.GetSegmentInsertFiles(
insertBinlogs,

View File

@ -29,6 +29,7 @@ import (
"github.com/milvus-io/milvus/internal/allocator"
"github.com/milvus-io/milvus/internal/compaction"
"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/proto/datapb"
@ -45,10 +46,10 @@ func TestSortCompactionTaskSuite(t *testing.T) {
type SortCompactionTaskSuite struct {
suite.Suite
mockBinlogIO *mock_util.MockBinlogIO
meta *etcdpb.CollectionMeta
segWriter *SegmentWriter
mockChunkManager *mocks.ChunkManager
mockBinlogIO *mock_util.MockBinlogIO
meta *etcdpb.CollectionMeta
segWriter *SegmentWriter
task *sortCompactionTask
}
@ -58,8 +59,8 @@ func (s *SortCompactionTaskSuite) SetupSuite() {
}
func (s *SortCompactionTaskSuite) setupTest() {
s.mockChunkManager = mocks.NewChunkManager(s.T())
s.mockBinlogIO = mock_util.NewMockBinlogIO(s.T())
s.meta = genTestCollectionMeta()
params, err := compaction.GenerateJSONParams()
@ -87,7 +88,8 @@ func (s *SortCompactionTaskSuite) setupTest() {
pk, err := typeutil.GetPrimaryFieldSchema(plan.GetSchema())
s.NoError(err)
s.task = NewSortCompactionTask(context.Background(), s.mockBinlogIO, plan, compaction.GenParams(), []int64{pk.GetFieldID()})
s.task = NewSortCompactionTask(context.Background(), s.mockChunkManager, plan, compaction.GenParams(), []int64{pk.GetFieldID()})
s.task.binlogIO = s.mockBinlogIO
}
func (s *SortCompactionTaskSuite) SetupTest() {
@ -108,7 +110,7 @@ func (s *SortCompactionTaskSuite) TestNewSortCompactionTask() {
pk, err := typeutil.GetPrimaryFieldSchema(plan.GetSchema())
s.NoError(err)
task := NewSortCompactionTask(context.Background(), s.mockBinlogIO, plan, compaction.GenParams(), []int64{pk.GetFieldID()})
task := NewSortCompactionTask(context.Background(), s.mockChunkManager, plan, compaction.GenParams(), []int64{pk.GetFieldID()})
s.NotNil(task)
s.Equal(plan.GetPlanID(), task.GetPlanID())
@ -225,6 +227,7 @@ func (s *SortCompactionTaskSuite) TestSortCompactionWithBM25() {
func (s *SortCompactionTaskSuite) setupBM25Test() {
s.mockBinlogIO = mock_util.NewMockBinlogIO(s.T())
s.mockChunkManager = mocks.NewChunkManager(s.T())
s.meta = genTestCollectionMetaWithBM25()
params, err := compaction.GenerateJSONParams()
if err != nil {
@ -251,7 +254,8 @@ func (s *SortCompactionTaskSuite) setupBM25Test() {
pk, err := typeutil.GetPrimaryFieldSchema(plan.GetSchema())
s.NoError(err)
s.task = NewSortCompactionTask(context.Background(), s.mockBinlogIO, plan, compaction.GenParams(), []int64{pk.GetFieldID()})
s.task = NewSortCompactionTask(context.Background(), s.mockChunkManager, plan, compaction.GenParams(), []int64{pk.GetFieldID()})
s.task.binlogIO = s.mockBinlogIO
}
func (s *SortCompactionTaskSuite) prepareSortCompactionWithBM25Task() {
@ -368,7 +372,7 @@ func (s *SortCompactionTaskSuite) initSegBufferWithBM25(seed int64) {
func TestSortCompactionTaskBasic(t *testing.T) {
ctx := context.Background()
mockBinlogIO := mock_util.NewMockBinlogIO(t)
mockChunkManager := mocks.NewChunkManager(t)
plan := &datapb.CompactionPlan{
PlanID: 123,
@ -382,7 +386,7 @@ func TestSortCompactionTaskBasic(t *testing.T) {
pk, err := typeutil.GetPrimaryFieldSchema(plan.GetSchema())
assert.NoError(t, err)
task := NewSortCompactionTask(ctx, mockBinlogIO, plan, compaction.GenParams(), []int64{pk.GetFieldID()})
task := NewSortCompactionTask(ctx, mockChunkManager, plan, compaction.GenParams(), []int64{pk.GetFieldID()})
assert.NotNil(t, task)
assert.Equal(t, int64(123), task.GetPlanID())

View File

@ -41,6 +41,7 @@ import (
"github.com/milvus-io/milvus/internal/datanode/index"
"github.com/milvus-io/milvus/internal/flushcommon/syncmgr"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/internal/util/analyzer"
"github.com/milvus-io/milvus/internal/util/fileresource"
"github.com/milvus-io/milvus/internal/util/sessionutil"
"github.com/milvus-io/milvus/pkg/v2/log"
@ -197,7 +198,7 @@ func (node *DataNode) Init() error {
syncMgr := syncmgr.NewSyncManager(nil)
node.syncMgr = syncMgr
fileMode := fileresource.ParseMode(paramtable.Get().DataCoordCfg.FileResourceMode.GetValue())
fileMode := fileresource.ParseMode(paramtable.Get().CommonCfg.DNFileResourceMode.GetValue())
if fileMode == fileresource.SyncMode {
cm, err := node.storageFactory.NewChunkManager(node.ctx, compaction.CreateStorageConfig())
if err != nil {
@ -217,6 +218,8 @@ func (node *DataNode) Init() error {
if err != nil {
initError = err
}
analyzer.InitOptions()
log.Info("init datanode done", zap.String("Address", node.address))
})
return initError

View File

@ -30,6 +30,7 @@ import (
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/datapb"
"github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
"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/typeutil"
@ -355,6 +356,7 @@ type StatsTaskInfo struct {
TextStatsLogs map[int64]*datapb.TextIndexStats
Bm25Logs []*datapb.FieldBinlog
JSONKeyStatsLogs map[int64]*datapb.JsonKeyStats
FileResources []*internalpb.FileResourceInfo
}
func (s *StatsTaskInfo) Clone() *StatsTaskInfo {
@ -372,6 +374,7 @@ func (s *StatsTaskInfo) Clone() *StatsTaskInfo {
TextStatsLogs: s.CloneTextStatsLogs(),
Bm25Logs: s.CloneBm25Logs(),
JSONKeyStatsLogs: s.CloneJSONKeyStatsLogs(),
FileResources: s.CloneFileResources(),
}
}
@ -393,6 +396,14 @@ func (s *StatsTaskInfo) ToStatsResult(taskID int64) *workerpb.StatsResult {
}
}
func (s *StatsTaskInfo) CloneFileResources() []*internalpb.FileResourceInfo {
clone := make([]*internalpb.FileResourceInfo, len(s.FileResources))
for i, resource := range s.FileResources {
clone[i] = typeutil.Clone(resource)
}
return clone
}
func (s *StatsTaskInfo) CloneInsertLogs() []*datapb.FieldBinlog {
clone := make([]*datapb.FieldBinlog, len(s.InsertLogs))
for i, log := range s.InsertLogs {

View File

@ -39,6 +39,8 @@ import (
"github.com/milvus-io/milvus/internal/flushcommon/io"
"github.com/milvus-io/milvus/internal/metastore/kv/binlog"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/analyzer"
"github.com/milvus-io/milvus/internal/util/fileresource"
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
"github.com/milvus-io/milvus/pkg/v2/common"
"github.com/milvus-io/milvus/pkg/v2/log"
@ -69,6 +71,7 @@ type statsTask struct {
queueDur time.Duration
manager *TaskManager
binlogIO io.BinlogIO
cm storage.ChunkManager
deltaLogs []string
logIDOffset int64
@ -86,7 +89,7 @@ func NewStatsTask(ctx context.Context,
cancel context.CancelFunc,
req *workerpb.CreateStatsRequest,
manager *TaskManager,
binlogIO io.BinlogIO,
cm storage.ChunkManager,
) *statsTask {
return &statsTask{
ident: fmt.Sprintf("%s/%d", req.GetClusterID(), req.GetTaskID()),
@ -94,7 +97,8 @@ func NewStatsTask(ctx context.Context,
cancel: cancel,
req: req,
manager: manager,
binlogIO: binlogIO,
binlogIO: io.NewBinlogIO(cm),
cm: cm,
tr: timerecord.NewTimeRecorder(fmt.Sprintf("ClusterID: %s, TaskID: %d", req.GetClusterID(), req.GetTaskID())),
currentTime: tsoutil.PhysicalTime(req.GetCurrentTs()),
logIDOffset: 0,
@ -481,6 +485,19 @@ func (st *statsTask) createTextIndex(ctx context.Context,
eg, egCtx := errgroup.WithContext(ctx)
var analyzerExtraInfo string
if len(st.req.GetFileResources()) > 0 {
err := fileresource.GlobalFileManager.Download(ctx, st.cm, st.req.GetFileResources()...)
if err != nil {
return err
}
defer fileresource.GlobalFileManager.Release(st.req.GetFileResources()...)
analyzerExtraInfo, err = analyzer.BuildExtraResourceInfo(st.req.GetStorageConfig().GetRootPath(), st.req.GetFileResources())
if err != nil {
return err
}
}
for _, field := range st.req.GetSchema().GetFields() {
field := field
h := typeutil.CreateFieldSchemaHelper(field)
@ -499,6 +516,11 @@ func (st *statsTask) createTextIndex(ctx context.Context,
req.InsertLogs = insertBinlogs
buildIndexParams := buildIndexParams(req, files, field, newStorageConfig, nil)
// set analyzer extra info
if len(analyzerExtraInfo) > 0 {
buildIndexParams.AnalyzerExtraInfo = analyzerExtraInfo
}
uploaded, err := indexcgowrapper.CreateTextIndex(egCtx, buildIndexParams)
if err != nil {
return err

View File

@ -29,6 +29,7 @@ import (
"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"
@ -53,8 +54,9 @@ type TaskStatsSuite struct {
clusterID string
schema *schemapb.CollectionSchema
mockBinlogIO *mock_util.MockBinlogIO
segWriter *compactor.SegmentWriter
mockBinlogIO *mock_util.MockBinlogIO
mockChunkManager *mocks.ChunkManager
segWriter *compactor.SegmentWriter
}
func (s *TaskStatsSuite) SetupSuite() {
@ -66,6 +68,7 @@ func (s *TaskStatsSuite) SetupSuite() {
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) {
@ -119,7 +122,9 @@ func (s *TaskStatsSuite) TestSortSegmentWithBM25() {
StorageConfig: &indexpb.StorageConfig{
RootPath: "root_path",
},
}, manager, s.mockBinlogIO)
}, manager, s.mockChunkManager)
task.binlogIO = s.mockBinlogIO
err = task.PreExecute(ctx)
s.Require().NoError(err)
binlog, err := task.sort(ctx)
@ -169,7 +174,9 @@ func (s *TaskStatsSuite) TestSortSegmentWithBM25() {
StorageConfig: &indexpb.StorageConfig{
RootPath: "root_path",
},
}, manager, s.mockBinlogIO)
}, manager, s.mockChunkManager)
task.binlogIO = s.mockBinlogIO
err = task.PreExecute(ctx)
s.Require().NoError(err)
_, err = task.sort(ctx)

View File

@ -29,7 +29,6 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/datanode/index"
"github.com/milvus-io/milvus/internal/flushcommon/io"
"github.com/milvus-io/milvus/internal/util/hookutil"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/metrics"
@ -412,7 +411,7 @@ func (node *DataNode) createStatsTask(ctx context.Context, req *workerpb.CreateS
return merr.Status(err), nil
}
t := index.NewStatsTask(taskCtx, taskCancel, req, node.taskManager, io.NewBinlogIO(cm))
t := index.NewStatsTask(taskCtx, taskCancel, req, node.taskManager, cm)
ret := merr.Success()
if err := node.taskScheduler.TaskQueue.Enqueue(t); err != nil {
log.Warn("DataNode failed to schedule", zap.Error(err))

View File

@ -220,12 +220,11 @@ func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPl
return merr.Status(err), err
}
var task compactor.Compactor
binlogIO := io.NewBinlogIO(cm)
switch req.GetType() {
case datapb.CompactionType_Level0DeleteCompaction:
task = compactor.NewLevelZeroCompactionTask(
taskCtx,
binlogIO,
io.NewBinlogIO(cm),
cm,
req,
compactionParams,
@ -240,7 +239,7 @@ func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPl
}
task = compactor.NewMixCompactionTask(
taskCtx,
binlogIO,
io.NewBinlogIO(cm),
req,
compactionParams,
[]int64{pk.GetFieldID()},
@ -251,7 +250,7 @@ func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPl
}
task = compactor.NewClusteringCompactionTask(
taskCtx,
binlogIO,
io.NewBinlogIO(cm),
req,
compactionParams,
)
@ -265,7 +264,7 @@ func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPl
}
task = compactor.NewSortCompactionTask(
taskCtx,
binlogIO,
cm,
req,
compactionParams,
[]int64{pk.GetFieldID()},
@ -281,7 +280,7 @@ func (node *DataNode) CompactionV2(ctx context.Context, req *datapb.CompactionPl
}
task = compactor.NewSortCompactionTask(
taskCtx,
binlogIO,
cm,
req,
compactionParams,
[]int64{partitionkey.GetFieldID(), pk.GetFieldID()},
@ -992,7 +991,7 @@ func (node *DataNode) SyncFileResource(ctx context.Context, req *internalpb.Sync
return merr.Status(merr.ErrServiceNotReady), nil
}
err := fileresource.Sync(req.GetResources())
err := fileresource.Sync(req.GetVersion(), req.GetResources())
if err != nil {
return merr.Status(err), nil
}

View File

@ -99,7 +99,7 @@ func (m *FileResourceObserver) syncLoop() {
}
func (m *FileResourceObserver) Start() {
if fileresource.IsSyncMode(paramtable.Get().QueryCoordCfg.FileResourceMode.GetValue()) {
if fileresource.IsSyncMode(paramtable.Get().CommonCfg.QNFileResourceMode.GetValue()) {
m.once.Do(func() {
m.wg.Add(1)
go m.syncLoop()

View File

@ -210,7 +210,7 @@ func (suite *FileResourceObserverSuite) TestSync_NodeSyncError() {
func (suite *FileResourceObserverSuite) TestStart_SyncModeEnabled() {
// Mock paramtable to enable sync mode
paramtable.Get().QueryCoordCfg.FileResourceMode.SwapTempValue("sync")
paramtable.Get().CommonCfg.QNFileResourceMode.SwapTempValue("sync")
// Start observer - real nodeManager starts with empty node list
suite.observer.Start()
@ -224,7 +224,7 @@ func (suite *FileResourceObserverSuite) TestStart_SyncModeEnabled() {
func (suite *FileResourceObserverSuite) TestStart_SyncModeDisabled() {
// Mock paramtable to disable sync mode
paramtable.Get().QueryCoordCfg.FileResourceMode.SwapTempValue("async")
paramtable.Get().CommonCfg.QNFileResourceMode.SwapTempValue("close")
// Start observer - no mocks should be called
suite.observer.Start()

View File

@ -355,7 +355,7 @@ func (node *QueryNode) Init() error {
// init pipeline manager
node.pipelineManager = pipeline.NewManager(node.manager, node.dispClient, node.delegators)
fileresource.InitManager(node.chunkManager, fileresource.ParseMode(paramtable.Get().QueryCoordCfg.FileResourceMode.GetValue()))
fileresource.InitManager(node.chunkManager, fileresource.ParseMode(paramtable.Get().CommonCfg.QNFileResourceMode.GetValue()))
err = initcore.InitQueryNode(node.ctx)
if err != nil {

View File

@ -1521,7 +1521,7 @@ func (node *QueryNode) DeleteBatch(ctx context.Context, req *querypb.DeleteBatch
}
func (node *QueryNode) runAnalyzer(req *querypb.RunAnalyzerRequest) ([]*milvuspb.AnalyzerResult, error) {
tokenizer, err := analyzer.NewAnalyzer(req.GetAnalyzerParams())
tokenizer, err := analyzer.NewAnalyzer(req.GetAnalyzerParams(), "")
if err != nil {
return nil, err
}
@ -1608,7 +1608,7 @@ func (node *QueryNode) ValidateAnalyzer(ctx context.Context, req *querypb.Valida
resourceSet := typeutil.NewSet[int64]()
for _, info := range req.AnalyzerInfos {
ids, err := analyzer.ValidateAnalyzer(info.GetParams())
ids, err := analyzer.ValidateAnalyzer(info.GetParams(), "")
if err != nil {
if info.GetName() != "" {
return &querypb.ValidateAnalyzerResponse{Status: merr.Status(merr.WrapErrParameterInvalidMsg("validate analyzer failed for field: %s, name: %s, error: %v", info.GetField(), info.GetName(), err))}, nil
@ -1725,7 +1725,7 @@ func (node *QueryNode) SyncFileResource(ctx context.Context, req *internalpb.Syn
}
defer node.lifetime.Done()
err := fileresource.Sync(req.GetResources())
err := fileresource.Sync(req.GetVersion(), req.GetResources())
if err != nil {
return merr.Status(err), nil
}

View File

@ -45,7 +45,7 @@ func (s *Server) init() {
s.initService()
// init file resource manager
fileresource.InitManager(resource.Resource().ChunkManager(), fileresource.ParseMode(paramtable.Get().QueryCoordCfg.FileResourceMode.GetValue()))
fileresource.InitManager(resource.Resource().ChunkManager(), fileresource.ParseMode(paramtable.Get().CommonCfg.QNFileResourceMode.GetValue()))
log.Info("init query segcore...")
if err := initcore.InitQueryNode(context.TODO()); err != nil {

View File

@ -3,6 +3,7 @@ package analyzer
import (
"github.com/milvus-io/milvus/internal/util/analyzer/canalyzer"
"github.com/milvus-io/milvus/internal/util/analyzer/interfaces"
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
)
type (
@ -10,18 +11,22 @@ type (
TokenStream interfaces.TokenStream
)
func NewAnalyzer(param string) (Analyzer, error) {
return canalyzer.NewAnalyzer(param, "")
func NewAnalyzer(param string, extraInfo string) (Analyzer, error) {
return canalyzer.NewAnalyzer(param, extraInfo)
}
func ValidateAnalyzer(param string) ([]int64, error) {
return canalyzer.ValidateAnalyzer(param, "")
func ValidateAnalyzer(param string, extraInfo string) ([]int64, error) {
return canalyzer.ValidateAnalyzer(param, extraInfo)
}
func UpdateGlobalResourceInfo(resourceMap map[string]int64) error {
return canalyzer.UpdateGlobalResourceInfo(resourceMap)
}
func BuildExtraResourceInfo(storage string, resources []*internalpb.FileResourceInfo) (string, error) {
return canalyzer.BuildExtraResourceInfo(storage, resources)
}
func InitOptions() {
canalyzer.InitOptions()
}

View File

@ -19,6 +19,7 @@ import (
"github.com/milvus-io/milvus/internal/util/analyzer/interfaces"
"github.com/milvus-io/milvus/internal/util/pathutil"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
"github.com/milvus-io/milvus/pkg/v2/util/paramtable"
)
@ -57,6 +58,22 @@ func UpdateParams() {
}
}
func BuildExtraResourceInfo(storage string, resources []*internalpb.FileResourceInfo) (string, error) {
result := map[string]any{}
result[StorageNameKey] = storage
resultMap := map[string]int64{}
for _, resource := range resources {
resultMap[resource.GetName()] = resource.GetId()
}
result[ResourceMapKey] = resultMap
bytes, err := json.Marshal(result)
if err != nil {
return "", errors.Wrap(err, "marshal extra resource info failed")
}
return string(bytes), nil
}
func UpdateGlobalResourceInfo(resourceMap map[string]int64) error {
bytes, err := json.Marshal(map[string]any{"resource_map": resourceMap})
if err != nil {

View File

@ -30,6 +30,7 @@ import (
"go.uber.org/zap"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/analyzer"
"github.com/milvus-io/milvus/internal/util/pathutil"
"github.com/milvus-io/milvus/pkg/v2/log"
"github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
@ -50,21 +51,21 @@ func InitManager(storage storage.ChunkManager, mode Mode) {
})
}
func Sync(resourceList []*internalpb.FileResourceInfo) error {
func Sync(version uint64, resourceList []*internalpb.FileResourceInfo) error {
if GlobalFileManager == nil {
log.Error("sync file resource to file manager not init")
return nil
}
return GlobalFileManager.Sync(resourceList)
return GlobalFileManager.Sync(version, resourceList)
}
// Manager manage file resource
type Manager interface {
// sync resource to local
Sync(resourceList []*internalpb.FileResourceInfo) error
Sync(version uint64, resourceList []*internalpb.FileResourceInfo) error
Download(downloader storage.ChunkManager, resources ...*internalpb.FileResourceInfo) error
Download(ctx context.Context, downloader storage.ChunkManager, resources ...*internalpb.FileResourceInfo) error
Release(resources ...*internalpb.FileResourceInfo)
Mode() Mode
}
@ -85,8 +86,11 @@ type BaseManager struct {
localPath string
}
func (m *BaseManager) Sync(resourceList []*internalpb.FileResourceInfo) error { return nil }
func (m *BaseManager) Download(downloader storage.ChunkManager, resources ...*internalpb.FileResourceInfo) error {
func (m *BaseManager) Sync(version uint64, resourceList []*internalpb.FileResourceInfo) error {
return nil
}
func (m *BaseManager) Download(ctx context.Context, downloader storage.ChunkManager, resources ...*internalpb.FileResourceInfo) error {
return nil
}
func (m *BaseManager) Release(resources ...*internalpb.FileResourceInfo) {}
@ -94,29 +98,47 @@ func (m *BaseManager) Mode() Mode { retur
// Manager with Sync Mode
// mixcoord should sync all node after add or remove file resource.
// file will download to /<local_resource_path>>/<resource_id>/<file_name>
type SyncManager struct {
BaseManager
sync.RWMutex
downloader storage.ChunkManager
resourceSet map[int64]struct{}
downloader storage.ChunkManager
version uint64
resourceMap map[string]int64 // resource name -> resource id
}
// sync file to local if file mode was Sync
func (m *SyncManager) Sync(resourceList []*internalpb.FileResourceInfo) error {
func (m *SyncManager) Sync(version uint64, resourceList []*internalpb.FileResourceInfo) error {
m.Lock()
defer m.Unlock()
// skip if version is not changed
if version <= m.version {
return nil
}
newResourceMap := make(map[string]int64)
removes := []int64{}
ctx := context.Background()
newSet := make(map[int64]struct{})
for _, resource := range resourceList {
newSet[resource.GetId()] = struct{}{}
if _, ok := m.resourceSet[resource.GetId()]; ok {
continue
newResourceMap[resource.GetName()] = resource.GetId()
if id, ok := m.resourceMap[resource.GetName()]; ok {
if id == resource.GetId() {
continue
}
}
// download new resource
localResourcePath := path.Join(m.localPath, fmt.Sprint(resource.GetId()))
err := os.MkdirAll(localResourcePath, os.ModePerm)
// remove old file if exist
err := os.RemoveAll(localResourcePath)
if err != nil {
log.Warn("remove invalid local resource failed", zap.String("path", localResourcePath), zap.Error(err))
}
err = os.MkdirAll(localResourcePath, os.ModePerm)
if err != nil {
return err
}
@ -139,20 +161,30 @@ func (m *SyncManager) Sync(resourceList []*internalpb.FileResourceInfo) error {
log.Info("download resource failed", zap.String("path", resource.GetPath()), zap.Error(err))
return err
}
log.Info("sync file to local", zap.String("name", fileName))
log.Info("sync file to local", zap.String("name", fileName), zap.Int64("id", resource.GetId()))
}
for resourceId := range m.resourceSet {
if _, ok := newSet[resourceId]; !ok {
err := os.RemoveAll(path.Join(m.localPath, fmt.Sprint(resourceId)))
if err != nil {
log.Warn("remove local resource failed", zap.Error(err))
newSet[resourceId] = struct{}{}
for name, id := range m.resourceMap {
if newId, ok := newResourceMap[name]; ok {
if newId != id {
// remove old resource with same name
removes = append(removes, id)
}
} else {
// remove old resource not exist in new resource list
removes = append(removes, id)
}
}
m.resourceSet = newSet
return nil
for _, resourceID := range removes {
err := os.RemoveAll(path.Join(m.localPath, fmt.Sprint(resourceID)))
if err != nil {
log.Warn("remove local resource failed", zap.Int64("id", resourceID), zap.Error(err))
}
}
m.resourceMap = newResourceMap
m.version = version
return analyzer.UpdateGlobalResourceInfo(newResourceMap)
}
func (m *SyncManager) Mode() Mode { return SyncMode }
@ -161,10 +193,16 @@ func NewSyncManager(downloader storage.ChunkManager) *SyncManager {
return &SyncManager{
BaseManager: BaseManager{localPath: pathutil.GetPath(pathutil.FileResourcePath, paramtable.GetNodeID())},
downloader: downloader,
resourceSet: make(map[int64]struct{}),
resourceMap: make(map[string]int64),
version: 0,
}
}
// RefManager only used for datanode.
// only download file will some one will use it.
// Should Download before use and Release after use.
// file will download to /<local_resource_path>>/<storage_name>/<resource_id>/<file_name>
// and delete file if no one own it for interval times.
type RefManager struct {
BaseManager
sync.RWMutex
@ -174,7 +212,7 @@ type RefManager struct {
sf *conc.Singleflight[interface{}]
}
func (m *RefManager) Download(downloader storage.ChunkManager, resources ...*internalpb.FileResourceInfo) error {
func (m *RefManager) Download(ctx context.Context, downloader storage.ChunkManager, resources ...*internalpb.FileResourceInfo) error {
m.Lock()
// inc ref count and set storage name with storage root path
for _, resource := range resources {
@ -184,7 +222,6 @@ func (m *RefManager) Download(downloader storage.ChunkManager, resources ...*int
}
m.Unlock()
ctx := context.Background()
for _, r := range resources {
resource := r
key := fmt.Sprintf("%s/%d", downloader.RootPath(), resource.GetId())
@ -259,6 +296,10 @@ func (m *RefManager) CleanResource() {
}
}
func (m *RefManager) Start() {
go m.GcLoop()
}
func (m *RefManager) GcLoop() {
ticker := time.NewTicker(15 * time.Minute)
@ -282,6 +323,10 @@ func NewManager(storage storage.ChunkManager, mode Mode) Manager {
return &BaseManager{}
case SyncMode:
return NewSyncManager(storage)
case RefMode:
manager := NewRefManger()
manager.Start()
return manager
default:
panic(fmt.Sprintf("Unknown file resource mananger mod: %v", mode))
}

View File

@ -69,7 +69,7 @@ func (suite *BaseManagerSuite) TestSync() {
{Id: 1, Name: "test.file", Path: "/test/test.file"},
}
err := suite.manager.Sync(resources)
err := suite.manager.Sync(1, resources)
suite.NoError(err)
}
@ -79,7 +79,7 @@ func (suite *BaseManagerSuite) TestDownload() {
{Id: 1, Name: "test.file", Path: "/test/test.file"},
}
err := suite.manager.Download(mockStorage, resources...)
err := suite.manager.Download(context.Background(), mockStorage, resources...)
suite.NoError(err)
}
@ -119,7 +119,8 @@ func (suite *SyncManagerSuite) SetupTest() {
suite.manager = &SyncManager{
BaseManager: BaseManager{localPath: suite.tempDir},
downloader: suite.mockStorage,
resourceSet: make(map[int64]struct{}),
version: 0,
resourceMap: make(map[string]int64),
}
}
@ -139,7 +140,7 @@ func (suite *SyncManagerSuite) TestSync_Success() {
suite.mockStorage.EXPECT().Reader(context.Background(), "/storage/test1.file").Return(newMockReader("test content 1"), nil)
suite.mockStorage.EXPECT().Reader(context.Background(), "/storage/test2.file").Return(newMockReader("test content 2"), nil)
err := suite.manager.Sync(resources)
err := suite.manager.Sync(1, resources)
suite.NoError(err)
// Verify files were created
@ -167,7 +168,7 @@ func (suite *SyncManagerSuite) TestSync_ReaderError() {
// Mock reader to return error
suite.mockStorage.EXPECT().Reader(context.Background(), "/storage/nonexistent.file").Return(nil, io.ErrUnexpectedEOF)
err := suite.manager.Sync(resources)
err := suite.manager.Sync(1, resources)
suite.Error(err)
suite.ErrorIs(err, io.ErrUnexpectedEOF)
}
@ -219,7 +220,7 @@ func (suite *RefManagerSuite) TestNormal() {
suite.mockStorage.EXPECT().RootPath().Return("/test/storage")
suite.mockStorage.EXPECT().Reader(context.Background(), "/storage/test.file").Return(newMockReader("test content"), nil)
err := suite.manager.Download(suite.mockStorage, resources...)
err := suite.manager.Download(context.Background(), suite.mockStorage, resources...)
suite.Require().NoError(err)
// Verify ref count
@ -318,7 +319,7 @@ func (suite *GlobalFunctionsSuite) TestSync_NotInitialized() {
{Id: 1, Name: "test.file", Path: "/test/test.file"},
}
err := Sync(resources)
err := Sync(1, resources)
suite.NoError(err) // Should not error when not initialized
}
@ -329,7 +330,7 @@ func (suite *GlobalFunctionsSuite) TestSync_Initialized() {
{Id: 1, Name: "test.file", Path: "/test/test.file"},
}
err := Sync(resources)
err := Sync(1, resources)
suite.NoError(err)
}

View File

@ -18,13 +18,19 @@
package fileresource
const (
SyncModeStr string = "sync"
RefModeStr string = "ref"
CloseModeStr string = "close"
)
func ParseMode(value string) Mode {
switch value {
case "close":
case CloseModeStr:
return CloseMode
case "sync":
case SyncModeStr:
return SyncMode
case "ref":
case RefModeStr:
return RefMode
default:
return CloseMode
@ -32,5 +38,9 @@ func ParseMode(value string) Mode {
}
func IsSyncMode(value string) bool {
return value == "sync"
return value == SyncModeStr
}
func IsRefMode(value string) bool {
return value == RefModeStr
}

View File

@ -89,7 +89,7 @@ func getAnalyzerParams(field *schemapb.FieldSchema) string {
func NewAnalyzerRunner(field *schemapb.FieldSchema) (Analyzer, error) {
params := getAnalyzerParams(field)
tokenizer, err := analyzer.NewAnalyzer(params)
tokenizer, err := analyzer.NewAnalyzer(params, "")
if err != nil {
return nil, err
}
@ -127,7 +127,7 @@ func NewBM25FunctionRunner(coll *schemapb.CollectionSchema, schema *schemapb.Fun
}
params = getAnalyzerParams(inputField)
tokenizer, err := analyzer.NewAnalyzer(params)
tokenizer, err := analyzer.NewAnalyzer(params, "")
if err != nil {
return nil, err
}

View File

@ -115,7 +115,7 @@ func NewMultiAnalyzerBM25FunctionRunner(coll *schemapb.CollectionSchema, schema
}
for name, param := range analyzersParam {
analyzer, err := analyzer.NewAnalyzer(string(param))
analyzer, err := analyzer.NewAnalyzer(string(param), "")
if err != nil {
return nil, fmt.Errorf("bm25 function create analyzer %s failed with error: %s", name, err.Error())
}

View File

@ -733,6 +733,7 @@ message CompactionPlan {
string json_params = 22;
int32 current_scalar_index_version = 23;
repeated common.KeyValuePair plugin_context = 29;
repeated internal.FileResourceInfo file_resources = 30;
}
message CompactionSegment {
@ -1351,12 +1352,6 @@ message DropCompactionPlanRequest {
int64 planID = 1;
}
message FileResourceInfo {
string name = 1;
string path = 2;
int64 resource_id = 3;
}
message CreateExternalCollectionResponse {
common.Status status = 1;
}

File diff suppressed because it is too large Load Diff

View File

@ -102,6 +102,7 @@ message BuildIndexInfo {
double json_stats_shredding_ratio_threshold = 28;
int64 json_stats_write_batch_size = 29;
string manifest = 30;
string analyzer_extra_info = 31;
}
message StoragePluginContext {

View File

@ -328,6 +328,7 @@ message StatsTask {
int64 target_segmentID = 10;
index.StatsSubJob subJobType = 11;
bool canRecycle = 12;
repeated internal.FileResourceInfo file_resources = 13;
}
message UpdateExternalCollectionTask {

View File

@ -692,6 +692,7 @@ type BuildIndexInfo struct {
JsonStatsShreddingRatioThreshold float64 `protobuf:"fixed64,28,opt,name=json_stats_shredding_ratio_threshold,json=jsonStatsShreddingRatioThreshold,proto3" json:"json_stats_shredding_ratio_threshold,omitempty"`
JsonStatsWriteBatchSize int64 `protobuf:"varint,29,opt,name=json_stats_write_batch_size,json=jsonStatsWriteBatchSize,proto3" json:"json_stats_write_batch_size,omitempty"`
Manifest string `protobuf:"bytes,30,opt,name=manifest,proto3" json:"manifest,omitempty"`
AnalyzerExtraInfo string `protobuf:"bytes,31,opt,name=analyzer_extra_info,json=analyzerExtraInfo,proto3" json:"analyzer_extra_info,omitempty"`
}
func (x *BuildIndexInfo) Reset() {
@ -936,6 +937,13 @@ func (x *BuildIndexInfo) GetManifest() string {
return ""
}
func (x *BuildIndexInfo) GetAnalyzerExtraInfo() string {
if x != nil {
return x.AnalyzerExtraInfo
}
return ""
}
type StoragePluginContext struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -1339,7 +1347,7 @@ var file_index_cgo_msg_proto_rawDesc = []byte{
0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x64, 0x61, 0x74, 0x61, 0x50, 0x61, 0x74, 0x68,
0x73, 0x12, 0x21, 0x0a, 0x0c, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70,
0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0b, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74,
0x54, 0x79, 0x70, 0x65, 0x22, 0x94, 0x0c, 0x0a, 0x0e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x6e,
0x54, 0x79, 0x70, 0x65, 0x22, 0xc4, 0x0c, 0x0a, 0x0e, 0x42, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x6e,
0x64, 0x65, 0x78, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74,
0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73,
0x74, 0x65, 0x72, 0x49, 0x44, 0x12, 0x18, 0x0a, 0x07, 0x62, 0x75, 0x69, 0x6c, 0x64, 0x49, 0x44,
@ -1436,7 +1444,10 @@ var file_index_cgo_msg_proto_rawDesc = []byte{
0x18, 0x1d, 0x20, 0x01, 0x28, 0x03, 0x52, 0x17, 0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74,
0x73, 0x57, 0x72, 0x69, 0x74, 0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12,
0x1a, 0x0a, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x18, 0x1e, 0x20, 0x01, 0x28,
0x09, 0x52, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x22, 0x90, 0x01, 0x0a, 0x14,
0x09, 0x52, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x2e, 0x0a, 0x13, 0x61,
0x6e, 0x61, 0x6c, 0x79, 0x7a, 0x65, 0x72, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x5f, 0x69, 0x6e,
0x66, 0x6f, 0x18, 0x1f, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x61, 0x6e, 0x61, 0x6c, 0x79, 0x7a,
0x65, 0x72, 0x45, 0x78, 0x74, 0x72, 0x61, 0x49, 0x6e, 0x66, 0x6f, 0x22, 0x90, 0x01, 0x0a, 0x14,
0x53, 0x74, 0x6f, 0x72, 0x61, 0x67, 0x65, 0x50, 0x6c, 0x75, 0x67, 0x69, 0x6e, 0x43, 0x6f, 0x6e,
0x74, 0x65, 0x78, 0x74, 0x12, 0x2c, 0x0a, 0x12, 0x65, 0x6e, 0x63, 0x72, 0x79, 0x70, 0x74, 0x69,
0x6f, 0x6e, 0x5f, 0x7a, 0x6f, 0x6e, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03,

View File

@ -2715,18 +2715,19 @@ type StatsTask struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"`
SegmentID int64 `protobuf:"varint,3,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
InsertChannel string `protobuf:"bytes,4,opt,name=insert_channel,json=insertChannel,proto3" json:"insert_channel,omitempty"`
TaskID int64 `protobuf:"varint,5,opt,name=taskID,proto3" json:"taskID,omitempty"`
Version int64 `protobuf:"varint,6,opt,name=version,proto3" json:"version,omitempty"`
NodeID int64 `protobuf:"varint,7,opt,name=nodeID,proto3" json:"nodeID,omitempty"`
State JobState `protobuf:"varint,8,opt,name=state,proto3,enum=milvus.proto.index.JobState" json:"state,omitempty"`
FailReason string `protobuf:"bytes,9,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"`
TargetSegmentID int64 `protobuf:"varint,10,opt,name=target_segmentID,json=targetSegmentID,proto3" json:"target_segmentID,omitempty"`
SubJobType StatsSubJob `protobuf:"varint,11,opt,name=subJobType,proto3,enum=milvus.proto.index.StatsSubJob" json:"subJobType,omitempty"`
CanRecycle bool `protobuf:"varint,12,opt,name=canRecycle,proto3" json:"canRecycle,omitempty"`
CollectionID int64 `protobuf:"varint,1,opt,name=collectionID,proto3" json:"collectionID,omitempty"`
PartitionID int64 `protobuf:"varint,2,opt,name=partitionID,proto3" json:"partitionID,omitempty"`
SegmentID int64 `protobuf:"varint,3,opt,name=segmentID,proto3" json:"segmentID,omitempty"`
InsertChannel string `protobuf:"bytes,4,opt,name=insert_channel,json=insertChannel,proto3" json:"insert_channel,omitempty"`
TaskID int64 `protobuf:"varint,5,opt,name=taskID,proto3" json:"taskID,omitempty"`
Version int64 `protobuf:"varint,6,opt,name=version,proto3" json:"version,omitempty"`
NodeID int64 `protobuf:"varint,7,opt,name=nodeID,proto3" json:"nodeID,omitempty"`
State JobState `protobuf:"varint,8,opt,name=state,proto3,enum=milvus.proto.index.JobState" json:"state,omitempty"`
FailReason string `protobuf:"bytes,9,opt,name=fail_reason,json=failReason,proto3" json:"fail_reason,omitempty"`
TargetSegmentID int64 `protobuf:"varint,10,opt,name=target_segmentID,json=targetSegmentID,proto3" json:"target_segmentID,omitempty"`
SubJobType StatsSubJob `protobuf:"varint,11,opt,name=subJobType,proto3,enum=milvus.proto.index.StatsSubJob" json:"subJobType,omitempty"`
CanRecycle bool `protobuf:"varint,12,opt,name=canRecycle,proto3" json:"canRecycle,omitempty"`
FileResources []*internalpb.FileResourceInfo `protobuf:"bytes,13,rep,name=file_resources,json=fileResources,proto3" json:"file_resources,omitempty"`
}
func (x *StatsTask) Reset() {
@ -2845,6 +2846,13 @@ func (x *StatsTask) GetCanRecycle() bool {
return false
}
func (x *StatsTask) GetFileResources() []*internalpb.FileResourceInfo {
if x != nil {
return x.FileResources
}
return nil
}
type UpdateExternalCollectionTask struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -3405,7 +3413,7 @@ var file_index_coord_proto_rawDesc = []byte{
0x69, 0x65, 0x6c, 0x64, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x66, 0x69,
0x65, 0x6c, 0x64, 0x49, 0x44, 0x12, 0x1d, 0x0a, 0x0a, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x70, 0x61,
0x74, 0x68, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x66, 0x69, 0x6c, 0x65, 0x50,
0x61, 0x74, 0x68, 0x73, 0x22, 0xc1, 0x03, 0x0a, 0x09, 0x53, 0x74, 0x61, 0x74, 0x73, 0x54, 0x61,
0x61, 0x74, 0x68, 0x73, 0x22, 0x91, 0x04, 0x0a, 0x09, 0x53, 0x74, 0x61, 0x74, 0x73, 0x54, 0x61,
0x73, 0x6b, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e,
0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x12, 0x20, 0x0a, 0x0b, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74,
@ -3433,7 +3441,12 @@ var file_index_coord_proto_rawDesc = []byte{
0x2e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x53, 0x75, 0x62, 0x4a, 0x6f, 0x62, 0x52, 0x0a, 0x73, 0x75,
0x62, 0x4a, 0x6f, 0x62, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x63, 0x61, 0x6e, 0x52,
0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x63, 0x61,
0x6e, 0x52, 0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, 0x22, 0xaf, 0x02, 0x0a, 0x1c, 0x55, 0x70, 0x64,
0x6e, 0x52, 0x65, 0x63, 0x79, 0x63, 0x6c, 0x65, 0x12, 0x4e, 0x0a, 0x0e, 0x66, 0x69, 0x6c, 0x65,
0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x0d, 0x20, 0x03, 0x28, 0x0b,
0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e,
0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73,
0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0d, 0x66, 0x69, 0x6c, 0x65, 0x52,
0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x22, 0xaf, 0x02, 0x0a, 0x1c, 0x55, 0x70, 0x64,
0x61, 0x74, 0x65, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x6c, 0x6c, 0x65,
0x63, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x61, 0x73, 0x6b, 0x12, 0x22, 0x0a, 0x0c, 0x63, 0x6f, 0x6c,
0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52,
@ -3617,12 +3630,13 @@ var file_index_coord_proto_goTypes = []interface{}{
(*commonpb.Status)(nil), // 41: milvus.proto.common.Status
(*internalpb.InitParams)(nil), // 42: milvus.proto.internal.InitParams
(schemapb.DataType)(0), // 43: milvus.proto.schema.DataType
(*internalpb.ShowConfigurationsRequest)(nil), // 44: milvus.proto.internal.ShowConfigurationsRequest
(*milvuspb.GetMetricsRequest)(nil), // 45: milvus.proto.milvus.GetMetricsRequest
(*milvuspb.CheckHealthRequest)(nil), // 46: milvus.proto.milvus.CheckHealthRequest
(*internalpb.ShowConfigurationsResponse)(nil), // 47: milvus.proto.internal.ShowConfigurationsResponse
(*milvuspb.GetMetricsResponse)(nil), // 48: milvus.proto.milvus.GetMetricsResponse
(*milvuspb.CheckHealthResponse)(nil), // 49: milvus.proto.milvus.CheckHealthResponse
(*internalpb.FileResourceInfo)(nil), // 44: milvus.proto.internal.FileResourceInfo
(*internalpb.ShowConfigurationsRequest)(nil), // 45: milvus.proto.internal.ShowConfigurationsRequest
(*milvuspb.GetMetricsRequest)(nil), // 46: milvus.proto.milvus.GetMetricsRequest
(*milvuspb.CheckHealthRequest)(nil), // 47: milvus.proto.milvus.CheckHealthRequest
(*internalpb.ShowConfigurationsResponse)(nil), // 48: milvus.proto.internal.ShowConfigurationsResponse
(*milvuspb.GetMetricsResponse)(nil), // 49: milvus.proto.milvus.GetMetricsResponse
(*milvuspb.CheckHealthResponse)(nil), // 50: milvus.proto.milvus.CheckHealthResponse
}
var file_index_coord_proto_depIdxs = []int32{
37, // 0: milvus.proto.index.IndexInfo.type_params:type_name -> milvus.proto.common.KeyValuePair
@ -3660,37 +3674,38 @@ var file_index_coord_proto_depIdxs = []int32{
1, // 32: milvus.proto.index.AnalyzeTask.state:type_name -> milvus.proto.index.JobState
1, // 33: milvus.proto.index.StatsTask.state:type_name -> milvus.proto.index.JobState
2, // 34: milvus.proto.index.StatsTask.subJobType:type_name -> milvus.proto.index.StatsSubJob
1, // 35: milvus.proto.index.UpdateExternalCollectionTask.state:type_name -> milvus.proto.index.JobState
17, // 36: milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry.value:type_name -> milvus.proto.index.SegmentInfo
13, // 37: milvus.proto.index.IndexCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest
14, // 38: milvus.proto.index.IndexCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest
8, // 39: milvus.proto.index.IndexCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest
10, // 40: milvus.proto.index.IndexCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest
15, // 41: milvus.proto.index.IndexCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest
19, // 42: milvus.proto.index.IndexCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest
20, // 43: milvus.proto.index.IndexCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest
27, // 44: milvus.proto.index.IndexCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest
22, // 45: milvus.proto.index.IndexCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest
44, // 46: milvus.proto.index.IndexCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest
45, // 47: milvus.proto.index.IndexCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest
46, // 48: milvus.proto.index.IndexCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest
41, // 49: milvus.proto.index.IndexCoord.CreateIndex:output_type -> milvus.proto.common.Status
41, // 50: milvus.proto.index.IndexCoord.AlterIndex:output_type -> milvus.proto.common.Status
9, // 51: milvus.proto.index.IndexCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse
12, // 52: milvus.proto.index.IndexCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse
18, // 53: milvus.proto.index.IndexCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse
41, // 54: milvus.proto.index.IndexCoord.DropIndex:output_type -> milvus.proto.common.Status
21, // 55: milvus.proto.index.IndexCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse
28, // 56: milvus.proto.index.IndexCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse
23, // 57: milvus.proto.index.IndexCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse
47, // 58: milvus.proto.index.IndexCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
48, // 59: milvus.proto.index.IndexCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
49, // 60: milvus.proto.index.IndexCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse
49, // [49:61] is the sub-list for method output_type
37, // [37:49] is the sub-list for method input_type
37, // [37:37] is the sub-list for extension type_name
37, // [37:37] is the sub-list for extension extendee
0, // [0:37] is the sub-list for field type_name
44, // 35: milvus.proto.index.StatsTask.file_resources:type_name -> milvus.proto.internal.FileResourceInfo
1, // 36: milvus.proto.index.UpdateExternalCollectionTask.state:type_name -> milvus.proto.index.JobState
17, // 37: milvus.proto.index.GetIndexInfoResponse.SegmentInfoEntry.value:type_name -> milvus.proto.index.SegmentInfo
13, // 38: milvus.proto.index.IndexCoord.CreateIndex:input_type -> milvus.proto.index.CreateIndexRequest
14, // 39: milvus.proto.index.IndexCoord.AlterIndex:input_type -> milvus.proto.index.AlterIndexRequest
8, // 40: milvus.proto.index.IndexCoord.GetIndexState:input_type -> milvus.proto.index.GetIndexStateRequest
10, // 41: milvus.proto.index.IndexCoord.GetSegmentIndexState:input_type -> milvus.proto.index.GetSegmentIndexStateRequest
15, // 42: milvus.proto.index.IndexCoord.GetIndexInfos:input_type -> milvus.proto.index.GetIndexInfoRequest
19, // 43: milvus.proto.index.IndexCoord.DropIndex:input_type -> milvus.proto.index.DropIndexRequest
20, // 44: milvus.proto.index.IndexCoord.DescribeIndex:input_type -> milvus.proto.index.DescribeIndexRequest
27, // 45: milvus.proto.index.IndexCoord.GetIndexStatistics:input_type -> milvus.proto.index.GetIndexStatisticsRequest
22, // 46: milvus.proto.index.IndexCoord.GetIndexBuildProgress:input_type -> milvus.proto.index.GetIndexBuildProgressRequest
45, // 47: milvus.proto.index.IndexCoord.ShowConfigurations:input_type -> milvus.proto.internal.ShowConfigurationsRequest
46, // 48: milvus.proto.index.IndexCoord.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest
47, // 49: milvus.proto.index.IndexCoord.CheckHealth:input_type -> milvus.proto.milvus.CheckHealthRequest
41, // 50: milvus.proto.index.IndexCoord.CreateIndex:output_type -> milvus.proto.common.Status
41, // 51: milvus.proto.index.IndexCoord.AlterIndex:output_type -> milvus.proto.common.Status
9, // 52: milvus.proto.index.IndexCoord.GetIndexState:output_type -> milvus.proto.index.GetIndexStateResponse
12, // 53: milvus.proto.index.IndexCoord.GetSegmentIndexState:output_type -> milvus.proto.index.GetSegmentIndexStateResponse
18, // 54: milvus.proto.index.IndexCoord.GetIndexInfos:output_type -> milvus.proto.index.GetIndexInfoResponse
41, // 55: milvus.proto.index.IndexCoord.DropIndex:output_type -> milvus.proto.common.Status
21, // 56: milvus.proto.index.IndexCoord.DescribeIndex:output_type -> milvus.proto.index.DescribeIndexResponse
28, // 57: milvus.proto.index.IndexCoord.GetIndexStatistics:output_type -> milvus.proto.index.GetIndexStatisticsResponse
23, // 58: milvus.proto.index.IndexCoord.GetIndexBuildProgress:output_type -> milvus.proto.index.GetIndexBuildProgressResponse
48, // 59: milvus.proto.index.IndexCoord.ShowConfigurations:output_type -> milvus.proto.internal.ShowConfigurationsResponse
49, // 60: milvus.proto.index.IndexCoord.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
50, // 61: milvus.proto.index.IndexCoord.CheckHealth:output_type -> milvus.proto.milvus.CheckHealthResponse
50, // [50:62] is the sub-list for method output_type
38, // [38:50] is the sub-list for method input_type
38, // [38:38] is the sub-list for extension type_name
38, // [38:38] is the sub-list for extension extendee
0, // [0:38] is the sub-list for field type_name
}
func init() { file_index_coord_proto_init() }

View File

@ -9,6 +9,7 @@ import "schema.proto";
import "data_coord.proto";
import "index_coord.proto";
import "milvus.proto";
import "internal.proto";
service IndexNode {
@ -203,6 +204,7 @@ message CreateStatsRequest {
double json_stats_shredding_ratio_threshold = 29;
int64 json_stats_write_batch_size = 30;
string manifest_path = 31;
repeated internal.FileResourceInfo file_resources = 32;
}
message CreateJobV2Request {

View File

@ -12,6 +12,7 @@ import (
schemapb "github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
datapb "github.com/milvus-io/milvus/pkg/v2/proto/datapb"
indexpb "github.com/milvus-io/milvus/pkg/v2/proto/indexpb"
internalpb "github.com/milvus-io/milvus/pkg/v2/proto/internalpb"
protoreflect "google.golang.org/protobuf/reflect/protoreflect"
protoimpl "google.golang.org/protobuf/runtime/protoimpl"
reflect "reflect"
@ -1096,15 +1097,16 @@ type CreateStatsRequest struct {
JsonKeyStatsTantivyMemory int64 `protobuf:"varint,21,opt,name=json_key_stats_tantivy_memory,json=jsonKeyStatsTantivyMemory,proto3" json:"json_key_stats_tantivy_memory,omitempty"`
JsonKeyStatsDataFormat int64 `protobuf:"varint,22,opt,name=json_key_stats_data_format,json=jsonKeyStatsDataFormat,proto3" json:"json_key_stats_data_format,omitempty"`
// deprecated, the sort logic has been moved into the compaction process.
EnableJsonKeyStatsInSort bool `protobuf:"varint,23,opt,name=enable_json_key_stats_in_sort,json=enableJsonKeyStatsInSort,proto3" json:"enable_json_key_stats_in_sort,omitempty"`
TaskSlot int64 `protobuf:"varint,24,opt,name=task_slot,json=taskSlot,proto3" json:"task_slot,omitempty"`
StorageVersion int64 `protobuf:"varint,25,opt,name=storage_version,json=storageVersion,proto3" json:"storage_version,omitempty"`
CurrentScalarIndexVersion int32 `protobuf:"varint,26,opt,name=current_scalar_index_version,json=currentScalarIndexVersion,proto3" json:"current_scalar_index_version,omitempty"`
PluginContext []*commonpb.KeyValuePair `protobuf:"bytes,27,rep,name=plugin_context,json=pluginContext,proto3" json:"plugin_context,omitempty"`
JsonStatsMaxShreddingColumns int64 `protobuf:"varint,28,opt,name=json_stats_max_shredding_columns,json=jsonStatsMaxShreddingColumns,proto3" json:"json_stats_max_shredding_columns,omitempty"`
JsonStatsShreddingRatioThreshold float64 `protobuf:"fixed64,29,opt,name=json_stats_shredding_ratio_threshold,json=jsonStatsShreddingRatioThreshold,proto3" json:"json_stats_shredding_ratio_threshold,omitempty"`
JsonStatsWriteBatchSize int64 `protobuf:"varint,30,opt,name=json_stats_write_batch_size,json=jsonStatsWriteBatchSize,proto3" json:"json_stats_write_batch_size,omitempty"`
ManifestPath string `protobuf:"bytes,31,opt,name=manifest_path,json=manifestPath,proto3" json:"manifest_path,omitempty"`
EnableJsonKeyStatsInSort bool `protobuf:"varint,23,opt,name=enable_json_key_stats_in_sort,json=enableJsonKeyStatsInSort,proto3" json:"enable_json_key_stats_in_sort,omitempty"`
TaskSlot int64 `protobuf:"varint,24,opt,name=task_slot,json=taskSlot,proto3" json:"task_slot,omitempty"`
StorageVersion int64 `protobuf:"varint,25,opt,name=storage_version,json=storageVersion,proto3" json:"storage_version,omitempty"`
CurrentScalarIndexVersion int32 `protobuf:"varint,26,opt,name=current_scalar_index_version,json=currentScalarIndexVersion,proto3" json:"current_scalar_index_version,omitempty"`
PluginContext []*commonpb.KeyValuePair `protobuf:"bytes,27,rep,name=plugin_context,json=pluginContext,proto3" json:"plugin_context,omitempty"`
JsonStatsMaxShreddingColumns int64 `protobuf:"varint,28,opt,name=json_stats_max_shredding_columns,json=jsonStatsMaxShreddingColumns,proto3" json:"json_stats_max_shredding_columns,omitempty"`
JsonStatsShreddingRatioThreshold float64 `protobuf:"fixed64,29,opt,name=json_stats_shredding_ratio_threshold,json=jsonStatsShreddingRatioThreshold,proto3" json:"json_stats_shredding_ratio_threshold,omitempty"`
JsonStatsWriteBatchSize int64 `protobuf:"varint,30,opt,name=json_stats_write_batch_size,json=jsonStatsWriteBatchSize,proto3" json:"json_stats_write_batch_size,omitempty"`
ManifestPath string `protobuf:"bytes,31,opt,name=manifest_path,json=manifestPath,proto3" json:"manifest_path,omitempty"`
FileResources []*internalpb.FileResourceInfo `protobuf:"bytes,32,rep,name=file_resources,json=fileResources,proto3" json:"file_resources,omitempty"`
}
func (x *CreateStatsRequest) Reset() {
@ -1356,6 +1358,13 @@ func (x *CreateStatsRequest) GetManifestPath() string {
return ""
}
func (x *CreateStatsRequest) GetFileResources() []*internalpb.FileResourceInfo {
if x != nil {
return x.FileResources
}
return nil
}
type CreateJobV2Request struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@ -2188,6 +2197,7 @@ var file_worker_proto_rawDesc = []byte{
0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x1a, 0x11, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x5f, 0x63, 0x6f, 0x6f, 0x72, 0x64, 0x2e, 0x70, 0x72,
0x6f, 0x74, 0x6f, 0x1a, 0x0c, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x1a, 0x0e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x22, 0xc3, 0x01, 0x0a, 0x11, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x61, 0x73, 0x6b,
0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f,
0x61, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61,
@ -2433,7 +2443,7 @@ var file_worker_proto_rawDesc = []byte{
0x28, 0x0b, 0x32, 0x20, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
0x6f, 0x2e, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x2e, 0x53, 0x65, 0x67, 0x6d, 0x65, 0x6e, 0x74, 0x53,
0x74, 0x61, 0x74, 0x73, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22,
0xed, 0x0b, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52,
0xbd, 0x0c, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x53, 0x74, 0x61, 0x74, 0x73, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65,
0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74,
0x65, 0x72, 0x49, 0x44, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x61, 0x73, 0x6b, 0x49, 0x44, 0x18, 0x02,
@ -2527,7 +2537,12 @@ var file_worker_proto_rawDesc = []byte{
0x03, 0x52, 0x17, 0x6a, 0x73, 0x6f, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x73, 0x57, 0x72, 0x69, 0x74,
0x65, 0x42, 0x61, 0x74, 0x63, 0x68, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x6d, 0x61,
0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x5f, 0x70, 0x61, 0x74, 0x68, 0x18, 0x1f, 0x20, 0x01, 0x28,
0x09, 0x52, 0x0c, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x50, 0x61, 0x74, 0x68, 0x22,
0x09, 0x52, 0x0c, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x50, 0x61, 0x74, 0x68, 0x12,
0x4e, 0x0a, 0x0e, 0x66, 0x69, 0x6c, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
0x73, 0x18, 0x20, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x6d, 0x69, 0x6c, 0x76, 0x75, 0x73,
0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e,
0x46, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x6e, 0x66, 0x6f,
0x52, 0x0d, 0x66, 0x69, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x22,
0xf8, 0x02, 0x0a, 0x12, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x4a, 0x6f, 0x62, 0x56, 0x32, 0x52,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74, 0x65,
0x72, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x63, 0x6c, 0x75, 0x73, 0x74,
@ -2817,14 +2832,15 @@ var file_worker_proto_goTypes = []interface{}{
(*indexpb.JobInfo)(nil), // 36: milvus.proto.index.JobInfo
(*schemapb.CollectionSchema)(nil), // 37: milvus.proto.schema.CollectionSchema
(indexpb.StatsSubJob)(0), // 38: milvus.proto.index.StatsSubJob
(indexpb.JobType)(0), // 39: milvus.proto.index.JobType
(commonpb.IndexState)(0), // 40: milvus.proto.common.IndexState
(indexpb.JobState)(0), // 41: milvus.proto.index.JobState
(*indexpb.SegmentStats)(nil), // 42: milvus.proto.index.SegmentStats
(*datapb.TextIndexStats)(nil), // 43: milvus.proto.data.TextIndexStats
(*datapb.JsonKeyStats)(nil), // 44: milvus.proto.data.JsonKeyStats
(*milvuspb.GetMetricsRequest)(nil), // 45: milvus.proto.milvus.GetMetricsRequest
(*milvuspb.GetMetricsResponse)(nil), // 46: milvus.proto.milvus.GetMetricsResponse
(*internalpb.FileResourceInfo)(nil), // 39: milvus.proto.internal.FileResourceInfo
(indexpb.JobType)(0), // 40: milvus.proto.index.JobType
(commonpb.IndexState)(0), // 41: milvus.proto.common.IndexState
(indexpb.JobState)(0), // 42: milvus.proto.index.JobState
(*indexpb.SegmentStats)(nil), // 43: milvus.proto.index.SegmentStats
(*datapb.TextIndexStats)(nil), // 44: milvus.proto.data.TextIndexStats
(*datapb.JsonKeyStats)(nil), // 45: milvus.proto.data.JsonKeyStats
(*milvuspb.GetMetricsRequest)(nil), // 46: milvus.proto.milvus.GetMetricsRequest
(*milvuspb.GetMetricsResponse)(nil), // 47: milvus.proto.milvus.GetMetricsResponse
}
var file_worker_proto_depIdxs = []int32{
22, // 0: milvus.proto.index.CreateTaskRequest.properties:type_name -> milvus.proto.index.CreateTaskRequest.PropertiesEntry
@ -2855,57 +2871,58 @@ var file_worker_proto_depIdxs = []int32{
37, // 25: milvus.proto.index.CreateStatsRequest.schema:type_name -> milvus.proto.schema.CollectionSchema
38, // 26: milvus.proto.index.CreateStatsRequest.subJobType:type_name -> milvus.proto.index.StatsSubJob
31, // 27: milvus.proto.index.CreateStatsRequest.plugin_context:type_name -> milvus.proto.common.KeyValuePair
39, // 28: milvus.proto.index.CreateJobV2Request.job_type:type_name -> milvus.proto.index.JobType
10, // 29: milvus.proto.index.CreateJobV2Request.analyze_request:type_name -> milvus.proto.index.AnalyzeRequest
4, // 30: milvus.proto.index.CreateJobV2Request.index_request:type_name -> milvus.proto.index.CreateJobRequest
11, // 31: milvus.proto.index.CreateJobV2Request.stats_request:type_name -> milvus.proto.index.CreateStatsRequest
39, // 32: milvus.proto.index.QueryJobsV2Request.job_type:type_name -> milvus.proto.index.JobType
40, // 33: milvus.proto.index.IndexTaskInfo.state:type_name -> milvus.proto.common.IndexState
14, // 34: milvus.proto.index.IndexJobResults.results:type_name -> milvus.proto.index.IndexTaskInfo
41, // 35: milvus.proto.index.AnalyzeResult.state:type_name -> milvus.proto.index.JobState
16, // 36: milvus.proto.index.AnalyzeResults.results:type_name -> milvus.proto.index.AnalyzeResult
41, // 37: milvus.proto.index.StatsResult.state:type_name -> milvus.proto.index.JobState
35, // 38: milvus.proto.index.StatsResult.insert_logs:type_name -> milvus.proto.data.FieldBinlog
35, // 39: milvus.proto.index.StatsResult.stats_logs:type_name -> milvus.proto.data.FieldBinlog
27, // 40: milvus.proto.index.StatsResult.text_stats_logs:type_name -> milvus.proto.index.StatsResult.TextStatsLogsEntry
35, // 41: milvus.proto.index.StatsResult.bm25_logs:type_name -> milvus.proto.data.FieldBinlog
28, // 42: milvus.proto.index.StatsResult.json_key_stats_logs:type_name -> milvus.proto.index.StatsResult.JsonKeyStatsLogsEntry
18, // 43: milvus.proto.index.StatsResults.results:type_name -> milvus.proto.index.StatsResult
29, // 44: milvus.proto.index.QueryJobsV2Response.status:type_name -> milvus.proto.common.Status
15, // 45: milvus.proto.index.QueryJobsV2Response.index_job_results:type_name -> milvus.proto.index.IndexJobResults
17, // 46: milvus.proto.index.QueryJobsV2Response.analyze_job_results:type_name -> milvus.proto.index.AnalyzeResults
19, // 47: milvus.proto.index.QueryJobsV2Response.stats_job_results:type_name -> milvus.proto.index.StatsResults
39, // 48: milvus.proto.index.DropJobsV2Request.job_type:type_name -> milvus.proto.index.JobType
42, // 49: milvus.proto.index.AnalyzeRequest.SegmentStatsEntry.value:type_name -> milvus.proto.index.SegmentStats
43, // 50: milvus.proto.index.StatsResult.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats
44, // 51: milvus.proto.index.StatsResult.JsonKeyStatsLogsEntry.value:type_name -> milvus.proto.data.JsonKeyStats
4, // 52: milvus.proto.index.IndexNode.CreateJob:input_type -> milvus.proto.index.CreateJobRequest
5, // 53: milvus.proto.index.IndexNode.QueryJobs:input_type -> milvus.proto.index.QueryJobsRequest
7, // 54: milvus.proto.index.IndexNode.DropJobs:input_type -> milvus.proto.index.DropJobsRequest
8, // 55: milvus.proto.index.IndexNode.GetJobStats:input_type -> milvus.proto.index.GetJobStatsRequest
12, // 56: milvus.proto.index.IndexNode.CreateJobV2:input_type -> milvus.proto.index.CreateJobV2Request
13, // 57: milvus.proto.index.IndexNode.QueryJobsV2:input_type -> milvus.proto.index.QueryJobsV2Request
21, // 58: milvus.proto.index.IndexNode.DropJobsV2:input_type -> milvus.proto.index.DropJobsV2Request
45, // 59: milvus.proto.index.IndexNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest
0, // 60: milvus.proto.index.IndexNode.CreateTask:input_type -> milvus.proto.index.CreateTaskRequest
1, // 61: milvus.proto.index.IndexNode.QueryTask:input_type -> milvus.proto.index.QueryTaskRequest
3, // 62: milvus.proto.index.IndexNode.DropTask:input_type -> milvus.proto.index.DropTaskRequest
29, // 63: milvus.proto.index.IndexNode.CreateJob:output_type -> milvus.proto.common.Status
6, // 64: milvus.proto.index.IndexNode.QueryJobs:output_type -> milvus.proto.index.QueryJobsResponse
29, // 65: milvus.proto.index.IndexNode.DropJobs:output_type -> milvus.proto.common.Status
9, // 66: milvus.proto.index.IndexNode.GetJobStats:output_type -> milvus.proto.index.GetJobStatsResponse
29, // 67: milvus.proto.index.IndexNode.CreateJobV2:output_type -> milvus.proto.common.Status
20, // 68: milvus.proto.index.IndexNode.QueryJobsV2:output_type -> milvus.proto.index.QueryJobsV2Response
29, // 69: milvus.proto.index.IndexNode.DropJobsV2:output_type -> milvus.proto.common.Status
46, // 70: milvus.proto.index.IndexNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
29, // 71: milvus.proto.index.IndexNode.CreateTask:output_type -> milvus.proto.common.Status
2, // 72: milvus.proto.index.IndexNode.QueryTask:output_type -> milvus.proto.index.QueryTaskResponse
29, // 73: milvus.proto.index.IndexNode.DropTask:output_type -> milvus.proto.common.Status
63, // [63:74] is the sub-list for method output_type
52, // [52:63] is the sub-list for method input_type
52, // [52:52] is the sub-list for extension type_name
52, // [52:52] is the sub-list for extension extendee
0, // [0:52] is the sub-list for field type_name
39, // 28: milvus.proto.index.CreateStatsRequest.file_resources:type_name -> milvus.proto.internal.FileResourceInfo
40, // 29: milvus.proto.index.CreateJobV2Request.job_type:type_name -> milvus.proto.index.JobType
10, // 30: milvus.proto.index.CreateJobV2Request.analyze_request:type_name -> milvus.proto.index.AnalyzeRequest
4, // 31: milvus.proto.index.CreateJobV2Request.index_request:type_name -> milvus.proto.index.CreateJobRequest
11, // 32: milvus.proto.index.CreateJobV2Request.stats_request:type_name -> milvus.proto.index.CreateStatsRequest
40, // 33: milvus.proto.index.QueryJobsV2Request.job_type:type_name -> milvus.proto.index.JobType
41, // 34: milvus.proto.index.IndexTaskInfo.state:type_name -> milvus.proto.common.IndexState
14, // 35: milvus.proto.index.IndexJobResults.results:type_name -> milvus.proto.index.IndexTaskInfo
42, // 36: milvus.proto.index.AnalyzeResult.state:type_name -> milvus.proto.index.JobState
16, // 37: milvus.proto.index.AnalyzeResults.results:type_name -> milvus.proto.index.AnalyzeResult
42, // 38: milvus.proto.index.StatsResult.state:type_name -> milvus.proto.index.JobState
35, // 39: milvus.proto.index.StatsResult.insert_logs:type_name -> milvus.proto.data.FieldBinlog
35, // 40: milvus.proto.index.StatsResult.stats_logs:type_name -> milvus.proto.data.FieldBinlog
27, // 41: milvus.proto.index.StatsResult.text_stats_logs:type_name -> milvus.proto.index.StatsResult.TextStatsLogsEntry
35, // 42: milvus.proto.index.StatsResult.bm25_logs:type_name -> milvus.proto.data.FieldBinlog
28, // 43: milvus.proto.index.StatsResult.json_key_stats_logs:type_name -> milvus.proto.index.StatsResult.JsonKeyStatsLogsEntry
18, // 44: milvus.proto.index.StatsResults.results:type_name -> milvus.proto.index.StatsResult
29, // 45: milvus.proto.index.QueryJobsV2Response.status:type_name -> milvus.proto.common.Status
15, // 46: milvus.proto.index.QueryJobsV2Response.index_job_results:type_name -> milvus.proto.index.IndexJobResults
17, // 47: milvus.proto.index.QueryJobsV2Response.analyze_job_results:type_name -> milvus.proto.index.AnalyzeResults
19, // 48: milvus.proto.index.QueryJobsV2Response.stats_job_results:type_name -> milvus.proto.index.StatsResults
40, // 49: milvus.proto.index.DropJobsV2Request.job_type:type_name -> milvus.proto.index.JobType
43, // 50: milvus.proto.index.AnalyzeRequest.SegmentStatsEntry.value:type_name -> milvus.proto.index.SegmentStats
44, // 51: milvus.proto.index.StatsResult.TextStatsLogsEntry.value:type_name -> milvus.proto.data.TextIndexStats
45, // 52: milvus.proto.index.StatsResult.JsonKeyStatsLogsEntry.value:type_name -> milvus.proto.data.JsonKeyStats
4, // 53: milvus.proto.index.IndexNode.CreateJob:input_type -> milvus.proto.index.CreateJobRequest
5, // 54: milvus.proto.index.IndexNode.QueryJobs:input_type -> milvus.proto.index.QueryJobsRequest
7, // 55: milvus.proto.index.IndexNode.DropJobs:input_type -> milvus.proto.index.DropJobsRequest
8, // 56: milvus.proto.index.IndexNode.GetJobStats:input_type -> milvus.proto.index.GetJobStatsRequest
12, // 57: milvus.proto.index.IndexNode.CreateJobV2:input_type -> milvus.proto.index.CreateJobV2Request
13, // 58: milvus.proto.index.IndexNode.QueryJobsV2:input_type -> milvus.proto.index.QueryJobsV2Request
21, // 59: milvus.proto.index.IndexNode.DropJobsV2:input_type -> milvus.proto.index.DropJobsV2Request
46, // 60: milvus.proto.index.IndexNode.GetMetrics:input_type -> milvus.proto.milvus.GetMetricsRequest
0, // 61: milvus.proto.index.IndexNode.CreateTask:input_type -> milvus.proto.index.CreateTaskRequest
1, // 62: milvus.proto.index.IndexNode.QueryTask:input_type -> milvus.proto.index.QueryTaskRequest
3, // 63: milvus.proto.index.IndexNode.DropTask:input_type -> milvus.proto.index.DropTaskRequest
29, // 64: milvus.proto.index.IndexNode.CreateJob:output_type -> milvus.proto.common.Status
6, // 65: milvus.proto.index.IndexNode.QueryJobs:output_type -> milvus.proto.index.QueryJobsResponse
29, // 66: milvus.proto.index.IndexNode.DropJobs:output_type -> milvus.proto.common.Status
9, // 67: milvus.proto.index.IndexNode.GetJobStats:output_type -> milvus.proto.index.GetJobStatsResponse
29, // 68: milvus.proto.index.IndexNode.CreateJobV2:output_type -> milvus.proto.common.Status
20, // 69: milvus.proto.index.IndexNode.QueryJobsV2:output_type -> milvus.proto.index.QueryJobsV2Response
29, // 70: milvus.proto.index.IndexNode.DropJobsV2:output_type -> milvus.proto.common.Status
47, // 71: milvus.proto.index.IndexNode.GetMetrics:output_type -> milvus.proto.milvus.GetMetricsResponse
29, // 72: milvus.proto.index.IndexNode.CreateTask:output_type -> milvus.proto.common.Status
2, // 73: milvus.proto.index.IndexNode.QueryTask:output_type -> milvus.proto.index.QueryTaskResponse
29, // 74: milvus.proto.index.IndexNode.DropTask:output_type -> milvus.proto.common.Status
64, // [64:75] is the sub-list for method output_type
53, // [53:64] is the sub-list for method input_type
53, // [53:53] is the sub-list for extension type_name
53, // [53:53] is the sub-list for extension extendee
0, // [0:53] is the sub-list for field type_name
}
func init() { file_worker_proto_init() }

View File

@ -334,6 +334,8 @@ type commonConfig struct {
ClusterID ParamItem `refreshable:"false"`
HybridSearchRequeryPolicy ParamItem `refreshable:"true"`
QNFileResourceMode ParamItem `refreshable:"true"`
DNFileResourceMode ParamItem `refreshable:"true"`
}
func (p *commonConfig) init(base *BaseTable) {
@ -1289,6 +1291,24 @@ The default value is 1, which is enough for most cases.`,
Export: false,
}
p.HybridSearchRequeryPolicy.Init(base.mgr)
p.QNFileResourceMode = ParamItem{
Key: "common.fileResource.mode.queryNode",
Version: "2.6.8",
DefaultValue: "sync",
Doc: "File resource mode for query node, options: [sync, close]. Default is sync.",
Export: true,
}
p.QNFileResourceMode.Init(base.mgr)
p.DNFileResourceMode = ParamItem{
Key: "common.fileResource.mode.dataNode",
Version: "2.6.8",
DefaultValue: "sync",
Doc: "File resource mode for data node, options: [sync, ref, close]. Default is sync.",
Export: true,
}
p.DNFileResourceMode.Init(base.mgr)
}
type gpuConfig struct {
@ -2512,8 +2532,6 @@ type queryCoordConfig struct {
ResourceExhaustionPenaltyDuration ParamItem `refreshable:"true"`
ResourceExhaustionCleanupInterval ParamItem `refreshable:"true"`
FileResourceMode ParamItem `refreshable:"false"`
UpdateTargetNeedSegmentDataReady ParamItem `refreshable:"true"`
}
@ -2526,13 +2544,6 @@ func (p *queryCoordConfig) init(base *BaseTable) {
}
p.RetryNum.Init(base.mgr)
p.FileResourceMode = ParamItem{
Key: "queryCoord.fileResource.mode",
Version: "2.6.3",
DefaultValue: "sync",
}
p.FileResourceMode.Init(base.mgr)
p.RetryInterval = ParamItem{
Key: "queryCoord.task.retryinterval",
Version: "2.2.0",
@ -4709,7 +4720,6 @@ type dataCoordConfig struct {
JSONStatsWriteBatchSize ParamItem `refreshable:"true"`
RequestTimeoutSeconds ParamItem `refreshable:"true"`
FileResourceMode ParamItem `refreshable:"false"`
}
func (p *dataCoordConfig) init(base *BaseTable) {
@ -4722,13 +4732,6 @@ func (p *dataCoordConfig) init(base *BaseTable) {
}
p.WatchTimeoutInterval.Init(base.mgr)
p.FileResourceMode = ParamItem{
Key: "dataCoord.fileResource.mode",
Version: "2.6.3",
DefaultValue: "sync",
}
p.FileResourceMode.Init(base.mgr)
p.LegacyVersionWithoutRPCWatch = ParamItem{
Key: "dataCoord.channel.legacyVersionWithoutRPCWatch",
Version: "2.4.1",