mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
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:
parent
b7ee93fc52
commit
ee216877bb
@ -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:
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -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(
|
||||
|
||||
@ -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());
|
||||
|
||||
@ -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);
|
||||
}
|
||||
|
||||
|
||||
@ -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:{}",
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -122,6 +122,7 @@ mod tests {
|
||||
dir.path().to_str().unwrap(),
|
||||
"jieba",
|
||||
¶ms,
|
||||
"",
|
||||
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,
|
||||
|
||||
@ -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);
|
||||
|
||||
@ -287,6 +287,7 @@ mod tests {
|
||||
dir.path().to_str().unwrap(),
|
||||
"default",
|
||||
"",
|
||||
"",
|
||||
1,
|
||||
50_000_000,
|
||||
false,
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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();
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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() {
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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,
|
||||
}
|
||||
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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,
|
||||
},
|
||||
}
|
||||
|
||||
@ -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) {
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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())
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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()
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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())
|
||||
}
|
||||
|
||||
@ -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
@ -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 {
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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() }
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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() }
|
||||
|
||||
@ -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",
|
||||
|
||||
Loading…
x
Reference in New Issue
Block a user