Major compaction 1st edition (#31804)

Signed-off-by: wayblink <anyang.wang@zilliz.com>
Signed-off-by: Cai Zhang <cai.zhang@zilliz.com>
Signed-off-by: chasingegg <chao.gao@zilliz.com>
Co-authored-by: chasingegg <chao.gao@zilliz.com>
This commit is contained in:
wayblink 2024-04-08 14:30:44 +08:00 committed by GitHub
parent b6fefee0cf
commit 17bf3abafe
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
97 changed files with 9276 additions and 1259 deletions

View File

@ -457,6 +457,7 @@ generate-mockery-datacoord: getdeps
$(INSTALL_PATH)/mockery --name=CompactionMeta --dir=internal/datacoord --filename=mock_compaction_meta.go --output=internal/datacoord --structname=MockCompactionMeta --with-expecter --inpackage
$(INSTALL_PATH)/mockery --name=Scheduler --dir=internal/datacoord --filename=mock_scheduler.go --output=internal/datacoord --structname=MockScheduler --with-expecter --inpackage
$(INSTALL_PATH)/mockery --name=ChannelManager --dir=internal/datacoord --filename=mock_channelmanager.go --output=internal/datacoord --structname=MockChannelManager --with-expecter --inpackage
$(INSTALL_PATH)/mockery --name=WorkerManager --dir=internal/datacoord --filename=mock_worker_manager.go --output=internal/datacoord --structname=MockWorkerManager --with-expecter --inpackage
generate-mockery-datanode: getdeps
$(INSTALL_PATH)/mockery --name=Allocator --dir=$(PWD)/internal/datanode/allocator --output=$(PWD)/internal/datanode/allocator --filename=mock_allocator.go --with-expecter --structname=MockAllocator --outpkg=allocator --inpackage

View File

@ -436,6 +436,23 @@ dataCoord:
maxParallelTaskNum: 10 # max parallel compaction task number
indexBasedCompaction: true
clustering:
enable: true
autoEnable: false
checkInterval: 10
minInterval: 3600
maxInterval: 86400
newDataRatioThreshold: 0.2
newDataSizeThreshold: 256m
timeout: 3600
dropTolerance: 86400
# clustering compaction will try best to distribute data into segments with size range in [preferSegmentSize, maxSegmentSize].
# data will be clustered by preferSegmentSize, if a cluster is larger than maxSegmentSize, will spilt it into multi segment
# buffer between (preferSegmentSize, maxSegmentSize) is left for new data in the same cluster(range), to avoid globally redistribute too often
preferSegmentSize: 64m
maxSegmentSize: 128m
maxTrainSize: 26g # max data size in analyze, if data is larger than it, will down sampling to meet this limit
levelzero:
forceTrigger:
minSize: 8388608 # The minmum size in bytes to force trigger a LevelZero Compaction, default as 8MB
@ -506,6 +523,9 @@ dataNode:
maxConcurrentTaskNum: 16 # The maximum number of import/pre-import tasks allowed to run concurrently on a datanode.
maxImportFileSizeInGB: 16 # The maximum file size (in GB) for an import file, where an import file refers to either a Row-Based file or a set of Column-Based files.
clusteringCompaction:
memoryBufferRatio: 0.1 # The ratio of memory buffer of clustering compaction. Data larger than threshold will be spilled to storage.
# Configures the system log output.
log:
level: info # Only supports debug, info, warn, error, panic, or fatal. Default 'info'.
@ -616,6 +636,9 @@ common:
traceLogMode: 0 # trace request info, 0: none, 1: simple request info, like collection/partition/database name, 2: request detail
bloomFilterSize: 100000
maxBloomFalsePositive: 0.05
usePartitionKeyAsClusteringKey: false
useVectorAsClusteringKey: false
# QuotaConfig, configurations of Milvus quota and limits.
# By default, we enable:

View File

@ -38,6 +38,9 @@ const char INDEX_BUILD_ID_KEY[] = "indexBuildID";
const char INDEX_ROOT_PATH[] = "index_files";
const char RAWDATA_ROOT_PATH[] = "raw_datas";
const char ANALYZE_ROOT_PATH[] = "analyze_stats";
const char COMPACTION_RAWDATA_ROOT_PATH[] = "compaction_raw_datas";
const char COMPACTION_RESULT_ROOT_PATH[] = "compaction_results";
const char VEC_OPT_FIELDS[] = "opt_fields";
const char DEFAULT_PLANNODE_ID[] = "0";

View File

@ -12,9 +12,11 @@
set(INDEXBUILDER_FILES
VecIndexCreator.cpp
analysis_c.cpp
index_c.cpp
init_c.cpp
ScalarIndexCreator.cpp
KmeansMajorCompaction.cpp
)
milvus_add_pkg_config("milvus_indexbuilder")

View File

@ -18,6 +18,8 @@
#include "common/EasyAssert.h"
#include "indexbuilder/IndexCreatorBase.h"
#include "indexbuilder/MajorCompaction.h"
#include "indexbuilder/KmeansMajorCompaction.h"
#include "indexbuilder/ScalarIndexCreator.h"
#include "indexbuilder/VecIndexCreator.h"
#include "indexbuilder/type_c.h"
@ -109,6 +111,37 @@ class IndexFactory {
throw std::invalid_argument(invalid_dtype_msg);
}
}
MajorCompactionBasePtr
CreateCompactionJob(DataType type,
Config& config,
const storage::FileManagerContext& context) {
auto invalid_dtype_msg =
std::string("invalid data type: ") + std::to_string(int(type));
switch (type) {
// currently only support float vector
case DataType::VECTOR_FLOAT:
return std::make_unique<KmeansMajorCompaction<float>>(config,
context);
case DataType::BOOL:
case DataType::INT8:
case DataType::INT16:
case DataType::INT32:
case DataType::INT64:
case DataType::FLOAT:
case DataType::DOUBLE:
case DataType::VARCHAR:
case DataType::STRING:
case DataType::VECTOR_FLOAT16:
case DataType::VECTOR_BFLOAT16:
case DataType::VECTOR_BINARY:
default:
throw SegcoreError(
DataTypeInvalid,
fmt::format("invalid type is {}", invalid_dtype_msg));
}
}
};
} // namespace milvus::indexbuilder

View File

@ -0,0 +1,351 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#include "index/VectorDiskIndex.h"
#include "common/Tracer.h"
#include "common/Types.h"
#include "common/Utils.h"
#include "config/ConfigKnowhere.h"
#include "index/Meta.h"
#include "index/Utils.h"
#include "knowhere/kmeans.h"
#include "indexbuilder/KmeansMajorCompaction.h"
#include "indexbuilder/MajorCompaction.h"
#include "segcore/SegcoreConfig.h"
#include "storage/LocalChunkManagerSingleton.h"
#include "storage/Util.h"
#include "common/Consts.h"
#include "common/RangeSearchHelper.h"
#include "indexbuilder/types.h"
#include <random>
namespace milvus::indexbuilder {
template <typename T>
KmeansMajorCompaction<T>::KmeansMajorCompaction(
Config& config, const storage::FileManagerContext& file_manager_context)
: config_(config) {
file_manager_ =
std::make_shared<storage::DiskFileManagerImpl>(file_manager_context);
AssertInfo(file_manager_ != nullptr, "create file manager failed!");
auto local_chunk_manager =
storage::LocalChunkManagerSingleton::GetInstance().GetChunkManager();
auto local_compaction_raw_data_path_prefix =
file_manager_->GetCompactionRawDataObjectPrefix();
if (local_chunk_manager->Exist(local_compaction_raw_data_path_prefix)) {
LOG_INFO("path {} already exists, delete it",
local_compaction_raw_data_path_prefix);
local_chunk_manager->RemoveDir(local_compaction_raw_data_path_prefix);
}
local_chunk_manager->CreateDir(local_compaction_raw_data_path_prefix);
}
template <typename T>
std::unique_ptr<T[]>
KmeansMajorCompaction<T>::Sample(const std::vector<std::string>& file_paths,
const std::vector<uint64_t>& file_sizes,
uint64_t train_size,
uint64_t total_size) {
auto local_chunk_manager =
storage::LocalChunkManagerSingleton::GetInstance().GetChunkManager();
// train data fits in memory, read by sequence and generate centroids and id_mapping in one pass
if (train_size >= total_size) {
auto buf = std::unique_ptr<T[]>(new T[total_size / sizeof(T)]);
int64_t offset = 0;
for (int i = 0; i < file_paths.size(); i++) {
local_chunk_manager->Read(
file_paths[i],
0,
reinterpret_cast<char*>(buf.get()) + offset,
file_sizes[i]);
offset += file_sizes[i];
}
return buf;
}
// random sampling to get train data
std::vector<int32_t> idx(file_paths.size());
std::iota(idx.begin(), idx.end(), 0);
std::shuffle(idx.begin(), idx.end(), std::mt19937());
int selected_size = 0;
auto buf = std::unique_ptr<T[]>(new T[train_size / sizeof(T)]);
int64_t offset = 0;
for (int i = 0; i < file_paths.size(); i++) {
if (selected_size < train_size &&
selected_size + file_sizes[idx[i]] >= train_size) {
auto cur_size = train_size - selected_size;
local_chunk_manager->Read(
file_paths[idx[i]],
0,
reinterpret_cast<char*>(buf.get()) + offset,
cur_size);
break;
} else {
selected_size += file_sizes[idx[i]];
local_chunk_manager->Read(
file_paths[idx[i]],
0,
reinterpret_cast<char*>(buf.get()) + offset,
file_sizes[idx[i]]);
offset += file_sizes[idx[i]];
}
}
return buf;
}
template <typename T>
BinarySet
KmeansMajorCompaction<T>::Upload() {
BinarySet ret;
std::unordered_map<std::string, int64_t> remote_paths_to_size;
file_manager_->AddCompactionResultFiles(result_files_,
remote_paths_to_size);
for (auto& file : remote_paths_to_size) {
ret.Append(file.first, nullptr, file.second);
}
return ret;
}
void
WritePBFile(google::protobuf::Message& message, std::string& file_path) {
std::ofstream outfile;
outfile.open(file_path.data(), std::ios_base::out | std::ios_base::binary);
if (outfile.fail()) {
std::stringstream err_msg;
err_msg << "Error: open local file '" << file_path << " failed, "
<< strerror(errno);
throw SegcoreError(FileOpenFailed, err_msg.str());
}
outfile.seekp(0, std::ios::beg);
if (!message.SerializeToOstream(&outfile)) {
std::stringstream err_msg;
err_msg << "Error: write local file '" << file_path << " failed, "
<< strerror(errno);
throw SegcoreError(FileWriteFailed, err_msg.str());
}
}
template <typename T>
void
KmeansMajorCompaction<T>::Train() {
if constexpr (!std::is_same_v<T, float>) {
PanicInfo(
ErrorCode::UnexpectedError,
fmt::format("kmeans major compaction only supports float32 now"));
}
auto insert_files = milvus::index::GetValueFromConfig<
std::map<int64_t, std::vector<std::string>>>(config_, "insert_files");
AssertInfo(insert_files.has_value(),
"insert file paths is empty when major compaction");
auto segment_size =
milvus::index::GetValueFromConfig<int64_t>(config_, "segment_size");
AssertInfo(segment_size.has_value(),
"segment size is empty when major compaction");
auto train_size =
milvus::index::GetValueFromConfig<int64_t>(config_, "train_size");
AssertInfo(train_size.has_value(),
"train size is empty when major compaction");
std::vector<std::string> data_files;
std::vector<uint64_t> offsets;
uint32_t dim = 0;
auto data_size = file_manager_->CacheCompactionRawDataToDisk(
insert_files.value(), data_files, offsets, dim);
AssertInfo(data_files.size() == offsets.size(),
fmt::format("file path num {} and file size {} not equal",
data_files.size(),
offsets.size()));
auto data_num = data_size / sizeof(T) / dim;
auto train_num = train_size.value() / sizeof(T) / dim;
// make train num equal to data num
if (train_num >= data_num) {
train_num = data_num;
}
auto train_size_new = train_num * dim * sizeof(T);
auto buf = Sample(data_files, offsets, train_size_new, data_size);
auto dataset = GenDataset(train_num, dim, buf.release());
// get num of clusters by whole data size / segment size
int num_clusters =
DIV_ROUND_UP(data_size, segment_size.value());
auto res =
knowhere::kmeans::ClusteringMajorCompaction<T>(*dataset, num_clusters);
if (!res.has_value()) {
PanicInfo(ErrorCode::UnexpectedError,
fmt::format("failed to kmeans train: {}: {}",
KnowhereStatusString(res.error()),
res.what()));
}
dataset.reset(); // release train data
auto centroids = reinterpret_cast<const T*>(res.value()->GetTensor());
auto centroid_id_mapping =
reinterpret_cast<const uint32_t*>(res.value()->GetCentroidIdMapping());
auto total_num = res.value()->GetRows();
// write centroids and centroid_id_mapping to file
milvus::proto::segcore::ClusteringCentroidsStats stats;
for (int i = 0; i < num_clusters; i++) {
milvus::proto::schema::VectorField* vector_field =
stats.add_centroids();
vector_field->set_dim(dim);
milvus::proto::schema::FloatArray* float_array =
vector_field->mutable_float_vector();
for (int j = 0; j < dim; j++) {
float_array->add_data(float(centroids[i * dim + j]));
}
}
auto output_path = file_manager_->GetCompactionResultObjectPrefix();
auto local_chunk_manager =
storage::LocalChunkManagerSingleton::GetInstance().GetChunkManager();
if (local_chunk_manager->Exist(output_path)) {
LOG_INFO("path {} already exists, delete it", output_path);
local_chunk_manager->RemoveDir(output_path);
}
local_chunk_manager->CreateDir(output_path);
std::string centroid_stats_path = output_path + "centroids";
result_files_.emplace_back(centroid_stats_path);
WritePBFile(stats, centroid_stats_path);
auto compute_num_in_centroid = [&](const uint32_t* centroid_id_mapping,
uint64_t start,
uint64_t end) -> std::vector<int64_t> {
std::vector<int64_t> num_vectors(num_clusters, 0);
for (uint64_t i = start; i < end; ++i) {
num_vectors[centroid_id_mapping[i]]++;
}
return num_vectors;
};
if (train_num >= data_num) { // do not compute id_mapping again
uint64_t i = 0;
uint64_t cur_offset = 0;
for (auto it = insert_files.value().begin();
it != insert_files.value().end();
it++) {
milvus::proto::segcore::ClusteringCentroidIdMappingStats stats;
// write centroid_id_mapping by file sizes
uint64_t num_offset = offsets[i] / sizeof(T) / dim;
for (uint64_t j = 0; j < num_offset; j++) {
stats.add_centroid_id_mapping(
centroid_id_mapping[cur_offset + j]);
}
cur_offset += num_offset;
auto num_vectors =
compute_num_in_centroid(centroid_id_mapping, 0, total_num);
for (uint64_t j = 0; j < num_clusters; j++) {
stats.add_num_in_centroid(num_vectors[j]);
}
std::string id_mapping_path =
output_path + std::to_string(it->first);
result_files_.emplace_back(id_mapping_path);
WritePBFile(stats, id_mapping_path);
i++;
}
} else {
uint64_t i = 0;
uint64_t start = 0;
uint64_t gather_size = 0;
// choose half of train size as a group to compute centroids
uint64_t group_size = train_size_new / 2;
std::vector<int64_t> gather_segment_id;
for (auto it = insert_files.value().begin();
it != insert_files.value().end();
it++) {
gather_segment_id.emplace_back(it->first);
gather_size += offsets[i];
if (gather_size > group_size) {
auto buf = std::unique_ptr<T[]>(new T[gather_size / sizeof(T)]);
uint64_t cur_offset = 0;
for (uint64_t j = start; j <= i; ++j) {
local_chunk_manager->Read(
data_files[j],
0,
reinterpret_cast<char*>(buf.get()) + cur_offset,
offsets[j]);
cur_offset += offsets[j];
}
auto dataset = GenDataset(
gather_size / sizeof(T) / dim, dim, buf.release());
auto res = knowhere::kmeans::ClusteringDataAssign<T>(
*dataset, centroids, num_clusters);
if (!res.has_value()) {
PanicInfo(ErrorCode::UnexpectedError,
fmt::format("failed to kmeans train: {}: {}",
KnowhereStatusString(res.error()),
res.what()));
}
dataset.reset();
auto centroid_id_mapping = reinterpret_cast<const uint32_t*>(
res.value()->GetCentroidIdMapping());
cur_offset = 0;
for (uint64_t j = start; j <= i; ++j) {
uint64_t num_offset = offsets[j] / sizeof(T) / dim;
milvus::proto::segcore::ClusteringCentroidIdMappingStats
stats;
for (uint64_t k = 0; k < num_offset; k++) {
stats.add_centroid_id_mapping(
centroid_id_mapping[cur_offset + k]);
}
auto num_vectors =
compute_num_in_centroid(centroid_id_mapping,
cur_offset,
cur_offset + num_offset);
cur_offset += num_offset;
for (uint64_t k = 0; k < num_clusters; k++) {
stats.add_num_in_centroid(num_vectors[k]);
}
std::string id_mapping_path =
output_path + std::to_string(gather_segment_id[j]);
result_files_.emplace_back(id_mapping_path);
WritePBFile(stats, id_mapping_path);
}
start = i + 1;
gather_size = 0;
}
gather_size += offsets[i];
i++;
}
}
// remove raw data since it is not used anymore
// keep the result file and leave the ownership to golang side
auto local_compaction_raw_data_path_prefix =
file_manager_->GetCompactionRawDataObjectPrefix();
if (local_chunk_manager->Exist(local_compaction_raw_data_path_prefix)) {
LOG_INFO("delete major compaction raw data dir: {}",
local_compaction_raw_data_path_prefix);
local_chunk_manager->RemoveDir(local_compaction_raw_data_path_prefix);
}
}
template class KmeansMajorCompaction<float>;
} // namespace milvus::indexbuilder

View File

@ -0,0 +1,56 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#include <memory>
#include <vector>
#include "storage/DiskFileManagerImpl.h"
#include "storage/space.h"
#include "indexbuilder/MajorCompaction.h"
namespace milvus::indexbuilder {
template <typename T>
class KmeansMajorCompaction : public MajorCompaction {
public:
explicit KmeansMajorCompaction(
Config& config,
const storage::FileManagerContext& file_manager_context =
storage::FileManagerContext());
void
Train() override;
BinarySet
Upload() override;
private:
std::unique_ptr<T[]>
Sample(const std::vector<std::string>& file_paths,
const std::vector<uint64_t>& data_sizes,
uint64_t train_size,
uint64_t total_size);
std::shared_ptr<storage::DiskFileManagerImpl> file_manager_;
Config config_;
std::vector<std::string> result_files_;
};
template <typename T>
using KmeansMajorCompactionPtr = std::unique_ptr<KmeansMajorCompaction<T>>;
} // namespace milvus::indexbuilder

View File

@ -0,0 +1,39 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
#pragma once
#include <memory>
#include <vector>
#include "storage/DiskFileManagerImpl.h"
#include "storage/space.h"
namespace milvus::indexbuilder {
class MajorCompaction {
public:
virtual ~MajorCompaction() = default;
virtual void
Train() = 0;
virtual BinarySet
Upload() = 0;
};
using MajorCompactionBasePtr = std::unique_ptr<MajorCompaction>;
} // namespace milvus::indexbuilder

View File

@ -0,0 +1,307 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include <memory>
#ifdef __linux__
#include <malloc.h>
#endif
#include "analysis_c.h"
#include "common/type_c.h"
#include "type_c.h"
#include "types.h"
#include "index/Utils.h"
#include "index/Meta.h"
#include "storage/Util.h"
#include "indexbuilder/IndexFactory.h"
using namespace milvus;
CStatus
Analysis(CAnalysis* res_analysis, CAnalysisInfo c_analysis_info) {
try {
auto analysis_info = (AnalysisInfo*)c_analysis_info;
auto field_type = analysis_info->field_type;
milvus::index::CreateIndexInfo index_info;
index_info.field_type = analysis_info->field_type;
auto& config = analysis_info->config;
config["insert_files"] = analysis_info->insert_files;
config["segment_size"] = analysis_info->segment_size;
config["train_size"] = analysis_info->train_size;
// get index type
// auto index_type = milvus::index::GetValueFromConfig<std::string>(
// config, "index_type");
// AssertInfo(index_type.has_value(), "index type is empty");
// index_info.index_type = index_type.value();
auto engine_version = analysis_info->index_engine_version;
// index_info.index_engine_version = engine_version;
config[milvus::index::INDEX_ENGINE_VERSION] =
std::to_string(engine_version);
// get metric type
// if (milvus::datatype_is_vector(field_type)) {
// auto metric_type = milvus::index::GetValueFromConfig<std::string>(
// config, "metric_type");
// AssertInfo(metric_type.has_value(), "metric type is empty");
// index_info.metric_type = metric_type.value();
// }
// init file manager
milvus::storage::FieldDataMeta field_meta{analysis_info->collection_id,
analysis_info->partition_id,
0,
analysis_info->field_id};
milvus::storage::IndexMeta index_meta{0,
analysis_info->field_id,
analysis_info->task_id,
analysis_info->version};
auto chunk_manager =
milvus::storage::CreateChunkManager(analysis_info->storage_config);
milvus::storage::FileManagerContext fileManagerContext(
field_meta, index_meta, chunk_manager);
auto compactionJob =
milvus::indexbuilder::IndexFactory::GetInstance()
.CreateCompactionJob(
analysis_info->field_type, config, fileManagerContext);
compactionJob->Train();
*res_analysis = compactionJob.release();
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedError;
status.error_msg = strdup(e.what());
return status;
}
}
CStatus
DeleteAnalysis(CAnalysis analysis) {
auto status = CStatus();
try {
AssertInfo(analysis,
"failed to delete analysis, passed index was null");
auto real_analysis =
reinterpret_cast<milvus::indexbuilder::MajorCompaction*>(analysis);
delete real_analysis;
status.error_code = Success;
status.error_msg = "";
} catch (std::exception& e) {
status.error_code = UnexpectedError;
status.error_msg = strdup(e.what());
}
return status;
}
CStatus
CleanAnalysisLocalData(CAnalysis analysis) {
auto status = CStatus();
try {
AssertInfo(analysis, "failed to build analysis, passed index was null");
auto real_analysis =
reinterpret_cast<milvus::indexbuilder::IndexCreatorBase*>(analysis);
auto cAnalysis =
dynamic_cast<milvus::indexbuilder::VecIndexCreator*>(real_analysis);
cAnalysis->CleanLocalData();
status.error_code = Success;
status.error_msg = "";
} catch (std::exception& e) {
status.error_code = UnexpectedError;
status.error_msg = strdup(e.what());
}
return status;
}
CStatus
NewAnalysisInfo(CAnalysisInfo* c_analysis_info,
CStorageConfig c_storage_config) {
try {
auto analysis_info = std::make_unique<AnalysisInfo>();
auto& storage_config = analysis_info->storage_config;
storage_config.address = std::string(c_storage_config.address);
storage_config.bucket_name = std::string(c_storage_config.bucket_name);
storage_config.access_key_id =
std::string(c_storage_config.access_key_id);
storage_config.access_key_value =
std::string(c_storage_config.access_key_value);
storage_config.root_path = std::string(c_storage_config.root_path);
storage_config.storage_type =
std::string(c_storage_config.storage_type);
storage_config.cloud_provider =
std::string(c_storage_config.cloud_provider);
storage_config.iam_endpoint =
std::string(c_storage_config.iam_endpoint);
storage_config.cloud_provider =
std::string(c_storage_config.cloud_provider);
storage_config.useSSL = c_storage_config.useSSL;
storage_config.useIAM = c_storage_config.useIAM;
storage_config.region = c_storage_config.region;
storage_config.useVirtualHost = c_storage_config.useVirtualHost;
storage_config.requestTimeoutMs = c_storage_config.requestTimeoutMs;
*c_analysis_info = analysis_info.release();
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedError;
status.error_msg = strdup(e.what());
return status;
}
}
void
DeleteAnalysisInfo(CAnalysisInfo c_analysis_info) {
auto info = (AnalysisInfo*)c_analysis_info;
delete info;
}
CStatus
AppendAnalysisFieldMetaInfo(CAnalysisInfo c_analysis_info,
int64_t collection_id,
int64_t partition_id,
int64_t field_id,
const char* field_name,
enum CDataType field_type,
int64_t dim) {
try {
auto analysis_info = (AnalysisInfo*)c_analysis_info;
analysis_info->collection_id = collection_id;
analysis_info->partition_id = partition_id;
analysis_info->field_id = field_id;
analysis_info->field_type = milvus::DataType(field_type);
analysis_info->field_name = field_name;
analysis_info->dim = dim;
return milvus::SuccessCStatus();
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
CStatus
AppendAnalysisInfo(CAnalysisInfo c_analysis_info,
int64_t task_id,
int64_t version) {
try {
auto analysis_info = (AnalysisInfo*)c_analysis_info;
analysis_info->task_id = task_id;
analysis_info->version = version;
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
auto status = CStatus();
status.error_code = UnexpectedError;
status.error_msg = strdup(e.what());
return status;
}
}
CStatus
AppendSegmentID(CAnalysisInfo c_analysis_info, int64_t segment_id) {
try {
auto analysis_info = (AnalysisInfo*)c_analysis_info;
// analysis_info->segment_ids.emplace_back(segment_id);
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
CStatus
AppendSegmentInsertFile(CAnalysisInfo c_analysis_info,
int64_t segID,
const char* c_file_path) {
try {
auto analysis_info = (AnalysisInfo*)c_analysis_info;
std::string insert_file_path(c_file_path);
analysis_info->insert_files[segID].emplace_back(insert_file_path);
// analysis_info->insert_files.emplace_back(insert_file_path);
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
CStatus
AppendSegmentSize(CAnalysisInfo c_analysis_info, int64_t size) {
try {
auto analysis_info = (AnalysisInfo*)c_analysis_info;
analysis_info->segment_size = size;
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
CStatus
AppendTrainSize(CAnalysisInfo c_analysis_info, int64_t size) {
try {
auto analysis_info = (AnalysisInfo*)c_analysis_info;
analysis_info->train_size = size;
auto status = CStatus();
status.error_code = Success;
status.error_msg = "";
return status;
} catch (std::exception& e) {
return milvus::FailureCStatus(&e);
}
}
CStatus
SerializeAnalysisAndUpLoad(CAnalysis analysis) {
auto status = CStatus();
try {
AssertInfo(analysis,
"failed to serialize analysis to binary set, passed index "
"was null");
auto real_analysis =
reinterpret_cast<milvus::indexbuilder::MajorCompaction*>(analysis);
auto binary =
std::make_unique<knowhere::BinarySet>(real_analysis->Upload());
// *c_binary_set = binary.release();
status.error_code = Success;
status.error_msg = "";
} catch (std::exception& e) {
status.error_code = UnexpectedError;
status.error_msg = strdup(e.what());
}
return status;
}

View File

@ -0,0 +1,83 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#pragma once
#ifdef __cplusplus
extern "C" {
#endif
#include <stdint.h>
#include "common/type_c.h"
#include "common/binary_set_c.h"
#include "indexbuilder/type_c.h"
CStatus
Analysis(CAnalysis* res_analysis, CAnalysisInfo c_analysis_info);
//CStatus
//CreateMajorCompaction(CAnalysis* res_analysis,
// CBuildIndexInfo c_build_index_info);
CStatus
DeleteAnalysis(CAnalysis analysis);
CStatus
CleanAnalysisLocalData(CAnalysis analysis);
CStatus
NewAnalysisInfo(CAnalysisInfo* c_analysis_info,
CStorageConfig c_storage_config);
void
DeleteAnalysisInfo(CAnalysisInfo c_analysis_info);
CStatus
AppendAnalysisFieldMetaInfo(CAnalysisInfo c_analysis_info,
int64_t collection_id,
int64_t partition_id,
int64_t field_id,
const char* field_name,
enum CDataType field_type,
int64_t dim);
CStatus
AppendAnalysisInfo(CAnalysisInfo c_analysis_info,
int64_t task_id,
int64_t version);
// no use
CStatus
AppendSegmentID(CAnalysisInfo c_analysis_info, int64_t segment_id);
CStatus
AppendSegmentInsertFile(CAnalysisInfo c_analysis_info,
int64_t segID,
const char* file_path);
CStatus
AppendSegmentSize(CAnalysisInfo c_analysis_info, int64_t size);
CStatus
AppendTrainSize(CAnalysisInfo c_analysis_info, int64_t size);
CStatus
SerializeAnalysisAndUpLoad(CAnalysis analysis);
CStatus
GetCentroidsFile(const char* file);
CStatus
GetSegmentOffsetMapping(int64_t segment_id, const char* file);
#ifdef __cplusplus
};
#endif

View File

@ -16,3 +16,5 @@
typedef void* CIndex;
typedef void* CIndexQueryResult;
typedef void* CBuildIndexInfo;
typedef void* CAnalysis;
typedef void* CAnalysisInfo;

View File

@ -14,6 +14,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
#include <map>
#include <stdint.h>
#include <string>
#include <vector>
@ -41,3 +42,26 @@ struct BuildIndexInfo {
int32_t index_engine_version;
milvus::OptFieldT opt_fields;
};
struct AnalysisInfo {
int64_t collection_id;
int64_t partition_id;
// int64_t segment_id; // no use
int64_t field_id;
milvus::DataType field_type;
int64_t task_id;
int64_t version;
std::map<int64_t, std::vector<std::string>> insert_files; // segment_id->files
// std::vector<std::string> insert_files;
milvus::storage::StorageConfig storage_config;
milvus::Config config;
std::string field_name;
std::string centroids_store_path;
std::vector<std::string> segments_offset_mapping;
int64_t data_store_version;
std::string index_store_path;
int64_t dim;
int32_t index_engine_version;
int64_t segment_size;
int64_t train_size;
};

View File

@ -162,6 +162,89 @@ DiskFileManagerImpl::AddFile(const std::string& file) noexcept {
return true;
} // namespace knowhere
void
DiskFileManagerImpl::AddCompactionResultFiles(
const std::vector<std::string>& files,
std::unordered_map<std::string, int64_t>& map) {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
std::vector<std::string> local_files;
std::vector<std::string> batch_remote_files;
std::vector<int64_t> remote_file_sizes;
for (auto i = 0; i < files.size(); ++i) {
auto file = files[i];
if (!local_chunk_manager->Exist(file)) {
LOG_ERROR("local file {} not exists", file);
std::stringstream err_msg;
err_msg << "Error: open local file '" << file << " failed, "
<< strerror(errno);
throw SegcoreError(FileOpenFailed, err_msg.str());
}
auto fileName = GetFileName(file);
auto fileSize = local_chunk_manager->Size(file);
auto parallel_degree = 16;
if (batch_remote_files.size() >= parallel_degree) {
AddBatchCompactionResultFiles(
local_files, batch_remote_files, remote_file_sizes, map);
batch_remote_files.clear();
remote_file_sizes.clear();
local_files.clear();
}
if (i == 0) { // centroids file
batch_remote_files.emplace_back(GetRemoteCentroidsObjectPrefix() +
"/centroids");
} else {
batch_remote_files.emplace_back(
GetRemoteCentroidIdMappingObjectPrefix(fileName) +
"/offsets_mapping");
}
remote_file_sizes.emplace_back(fileSize);
local_files.emplace_back(file);
}
if (batch_remote_files.size() > 0) {
AddBatchCompactionResultFiles(
local_files, batch_remote_files, remote_file_sizes, map);
}
}
void
DiskFileManagerImpl::AddBatchCompactionResultFiles(
const std::vector<std::string>& local_files,
const std::vector<std::string>& remote_files,
const std::vector<int64_t>& remote_file_sizes,
std::unordered_map<std::string, int64_t>& map) {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
auto& pool = ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::HIGH);
std::vector<std::future<std::shared_ptr<uint8_t[]>>> futures;
futures.reserve(remote_file_sizes.size());
for (int64_t i = 0; i < remote_files.size(); ++i) {
futures.push_back(pool.Submit(
[&](const std::string& file,
const int64_t data_size) -> std::shared_ptr<uint8_t[]> {
auto buf = std::shared_ptr<uint8_t[]>(new uint8_t[data_size]);
local_chunk_manager->Read(file, 0, buf.get(), data_size);
return buf;
},
local_files[i],
remote_file_sizes[i]));
}
std::vector<std::shared_ptr<uint8_t[]>> index_datas;
std::vector<const uint8_t*> data_slices;
for (auto& future : futures) {
auto res = future.get();
index_datas.emplace_back(res);
data_slices.emplace_back(res.get());
}
PutCompactionResultData(
rcm_.get(), data_slices, remote_file_sizes, remote_files, map);
}
void
DiskFileManagerImpl::AddBatchIndexFiles(
const std::string& local_file_name,
@ -442,6 +525,101 @@ SortByPath(std::vector<std::string>& paths) {
});
}
uint64_t
FetchRawDataAndWriteFile(ChunkManagerPtr rcm,
std::string& local_data_path,
std::vector<std::string>& batch_files,
int64_t& write_offset,
uint32_t& num_rows,
uint32_t& dim) {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
auto field_datas = GetObjectData(rcm.get(), batch_files);
int batch_size = batch_files.size();
uint64_t batch_data_size = 0;
for (int i = 0; i < batch_size; ++i) {
auto field_data = field_datas[i].get()->GetFieldData();
num_rows += uint32_t(field_data->get_num_rows());
AssertInfo(dim == 0 || dim == field_data->get_dim(),
"inconsistent dim value in multi binlogs!");
dim = field_data->get_dim();
auto data_size = field_data->get_num_rows() * dim * sizeof(float);
local_chunk_manager->Write(local_data_path,
write_offset,
const_cast<void*>(field_data->Data()),
data_size);
write_offset += data_size;
batch_data_size += data_size;
}
return batch_data_size;
}
// cache raw data for major compaction
uint64_t
DiskFileManagerImpl::CacheCompactionRawDataToDisk(
const std::map<int64_t, std::vector<std::string>>& remote_files,
std::vector<std::string>& output_files,
std::vector<uint64_t>& offsets,
uint32_t& dim) {
auto partition_id = GetFieldDataMeta().partition_id;
auto field_id = GetFieldDataMeta().field_id;
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
auto local_data_path_prefix =
storage::GenCompactionRawDataPathPrefix(
local_chunk_manager, partition_id, field_id) +
"raw_data";
auto next_file_id = 0;
std::vector<std::string> batch_files;
int64_t write_offset = 0;
uint32_t num_rows = 0;
uint64_t whole_size = 0;
auto parallel_degree =
uint64_t(DEFAULT_FIELD_MAX_MEMORY_LIMIT / FILE_SLICE_SIZE);
for (auto& [segment_id, filess] : remote_files) {
std::vector<std::string> files = filess;
SortByPath(files);
auto local_data_path =
local_data_path_prefix + std::to_string(next_file_id);
next_file_id++;
local_chunk_manager->CreateFile(local_data_path);
output_files.emplace_back(local_data_path);
batch_files.clear();
write_offset = 0;
for (auto& file : files) {
if (batch_files.size() >= parallel_degree) {
whole_size += FetchRawDataAndWriteFile(rcm_,
local_data_path,
batch_files,
write_offset,
num_rows,
dim);
batch_files.clear();
}
batch_files.emplace_back(file);
}
if (batch_files.size() > 0) {
whole_size += FetchRawDataAndWriteFile(rcm_,
local_data_path,
batch_files,
write_offset,
num_rows,
dim);
}
offsets.emplace_back(write_offset);
}
return whole_size;
}
std::string
DiskFileManagerImpl::CacheRawDataToDisk(std::vector<std::string> remote_files) {
SortByPath(remote_files);
@ -466,30 +644,16 @@ DiskFileManagerImpl::CacheRawDataToDisk(std::vector<std::string> remote_files) {
uint32_t dim = 0;
int64_t write_offset = sizeof(num_rows) + sizeof(dim);
auto FetchRawData = [&]() {
auto field_datas = GetObjectData(rcm_.get(), batch_files);
int batch_size = batch_files.size();
for (int i = 0; i < batch_size; ++i) {
auto field_data = field_datas[i].get()->GetFieldData();
num_rows += uint32_t(field_data->get_num_rows());
AssertInfo(dim == 0 || dim == field_data->get_dim(),
"inconsistent dim value in multi binlogs!");
dim = field_data->get_dim();
auto data_size = field_data->get_num_rows() * dim * sizeof(float);
local_chunk_manager->Write(local_data_path,
write_offset,
const_cast<void*>(field_data->Data()),
data_size);
write_offset += data_size;
}
};
auto parallel_degree =
uint64_t(DEFAULT_FIELD_MAX_MEMORY_LIMIT / FILE_SLICE_SIZE);
for (auto& file : remote_files) {
if (batch_files.size() >= parallel_degree) {
FetchRawData();
FetchRawDataAndWriteFile(rcm_,
local_data_path,
batch_files,
write_offset,
num_rows,
dim);
batch_files.clear();
}
@ -497,7 +661,8 @@ DiskFileManagerImpl::CacheRawDataToDisk(std::vector<std::string> remote_files) {
}
if (batch_files.size() > 0) {
FetchRawData();
FetchRawDataAndWriteFile(
rcm_, local_data_path, batch_files, write_offset, num_rows, dim);
}
// write num_rows and dim value to file header
@ -805,6 +970,24 @@ DiskFileManagerImpl::GetLocalRawDataObjectPrefix() {
local_chunk_manager, field_meta_.segment_id, field_meta_.field_id);
}
// need to confirm the raw data path, used for train
std::string
DiskFileManagerImpl::GetCompactionRawDataObjectPrefix() {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
return GenCompactionRawDataPathPrefix(
local_chunk_manager, field_meta_.partition_id, field_meta_.field_id);
}
// need to confirm the result path, used for data partition and search
std::string
DiskFileManagerImpl::GetCompactionResultObjectPrefix() {
auto local_chunk_manager =
LocalChunkManagerSingleton::GetInstance().GetChunkManager();
return GenCompactionResultPathPrefix(
local_chunk_manager, index_meta_.build_id, index_meta_.index_version);
}
bool
DiskFileManagerImpl::RemoveFile(const std::string& file) noexcept {
// TODO: implement this interface

View File

@ -64,6 +64,12 @@ class DiskFileManagerImpl : public FileManagerImpl {
std::string
GetLocalRawDataObjectPrefix();
std::string
GetCompactionRawDataObjectPrefix();
std::string
GetCompactionResultObjectPrefix();
std::map<std::string, int64_t>
GetRemotePathsToFileSize() const {
return remote_paths_to_size_;
@ -90,6 +96,17 @@ class DiskFileManagerImpl : public FileManagerImpl {
const std::string& local_file_name,
uint64_t local_file_init_offfset);
void
AddCompactionResultFiles(const std::vector<std::string>& files,
std::unordered_map<std::string, int64_t>& map);
void
AddBatchCompactionResultFiles(
const std::vector<std::string>& local_files,
const std::vector<std::string>& remote_files,
const std::vector<int64_t>& remote_file_sizes,
std::unordered_map<std::string, int64_t>& map);
void
AddBatchIndexFiles(const std::string& local_file_name,
const std::vector<int64_t>& local_file_offsets,
@ -102,6 +119,13 @@ class DiskFileManagerImpl : public FileManagerImpl {
std::string
CacheRawDataToDisk(std::shared_ptr<milvus_storage::Space> space);
uint64_t
CacheCompactionRawDataToDisk(
const std::map<int64_t, std::vector<std::string>>& insert_files,
std::vector<std::string>& output_files,
std::vector<uint64_t>& offsets,
uint32_t& dim);
std::string
CacheOptFieldToDisk(OptFieldT& fields_map);
@ -127,6 +151,13 @@ class DiskFileManagerImpl : public FileManagerImpl {
std::string
GetRemoteIndexPath(const std::string& file_name, int64_t slice_num) const;
std::string
GetCentroidsPath(const std::string& file_name, int64_t slice_num) const;
std::string
GetCentroidIdMappingPath(const std::string& file_name,
int64_t slice_num) const;
private:
// local file path (abs path)
std::vector<std::string> local_paths_;

View File

@ -148,6 +148,26 @@ class FileManagerImpl : public knowhere::FileManager {
std::to_string(field_meta_.segment_id);
}
virtual std::string
GetRemoteCentroidsObjectPrefix() const {
return rcm_->GetRootPath() + "/files" + std::string(ANALYZE_ROOT_PATH) +
"/" + std::to_string(index_meta_.build_id) + "/" +
std::to_string(index_meta_.index_version) + "/" +
std::to_string(field_meta_.collection_id) + "/" +
std::to_string(field_meta_.partition_id) + "/" +
std::to_string(field_meta_.field_id);
}
virtual std::string
GetRemoteCentroidIdMappingObjectPrefix(std::string segment_id) const {
return rcm_->GetRootPath() + "/files" + std::string(ANALYZE_ROOT_PATH) +
"/" + std::to_string(index_meta_.build_id) + "/" +
std::to_string(index_meta_.index_version) + "/" +
std::to_string(field_meta_.collection_id) + "/" +
std::to_string(field_meta_.partition_id) + "/" +
std::to_string(field_meta_.field_id) + "/" + segment_id;
}
protected:
// collection meta
FieldDataMeta field_meta_;

View File

@ -442,6 +442,24 @@ GenFieldRawDataPathPrefix(ChunkManagerPtr cm,
std::to_string(segment_id) + "/" + std::to_string(field_id) + "/";
}
std::string
GenCompactionRawDataPathPrefix(ChunkManagerPtr cm,
int64_t partition_id,
int64_t field_id) {
return cm->GetRootPath() + "/" + std::string(COMPACTION_RAWDATA_ROOT_PATH) +
"/" + std::to_string(partition_id) + "/" + std::to_string(field_id) +
"/";
}
std::string
GenCompactionResultPathPrefix(ChunkManagerPtr cm,
int64_t build_id,
int64_t index_version) {
return cm->GetRootPath() + "/" + std::string(COMPACTION_RESULT_ROOT_PATH) +
"/" + std::to_string(build_id) + "/" +
std::to_string(index_version) + "/";
}
std::string
GetSegmentRawDataPathPrefix(ChunkManagerPtr cm, int64_t segment_id) {
return cm->GetRootPath() + "/" + std::string(RAWDATA_ROOT_PATH) + "/" +
@ -565,6 +583,43 @@ GetObjectData(std::shared_ptr<milvus_storage::Space> space,
return datas;
}
void
PutCompactionResultData(ChunkManager* remote_chunk_manager,
const std::vector<const uint8_t*>& data_slices,
const std::vector<int64_t>& slice_sizes,
const std::vector<std::string>& slice_names,
std::unordered_map<std::string, int64_t>& map) {
auto& pool = ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::MIDDLE);
std::vector<std::future<std::pair<std::string, int64_t>>> futures;
AssertInfo(data_slices.size() == slice_sizes.size(),
"inconsistent data slices size {} with slice sizes {}",
data_slices.size(),
slice_sizes.size());
AssertInfo(data_slices.size() == slice_names.size(),
"inconsistent data slices size {} with slice names size {}",
data_slices.size(),
slice_names.size());
for (int64_t i = 0; i < data_slices.size(); ++i) {
futures.push_back(pool.Submit(
[&](ChunkManager* chunk_manager,
uint8_t* buf,
int64_t batch_size,
std::string object_key) -> std::pair<std::string, int64_t> {
chunk_manager->Write(object_key, buf, batch_size);
return std::make_pair(object_key, batch_size);
},
remote_chunk_manager,
const_cast<uint8_t*>(data_slices[i]),
slice_sizes[i],
slice_names[i]));
}
for (auto& future : futures) {
auto res = future.get();
map[res.first] = res.second;
}
}
std::map<std::string, int64_t>
PutIndexData(ChunkManager* remote_chunk_manager,
const std::vector<const uint8_t*>& data_slices,

View File

@ -82,6 +82,15 @@ GenFieldRawDataPathPrefix(ChunkManagerPtr cm,
int64_t segment_id,
int64_t field_id);
std::string
GenCompactionRawDataPathPrefix(ChunkManagerPtr cm,
int64_t partition_id,
int64_t field_id);
std::string
GenCompactionResultPathPrefix(ChunkManagerPtr cm,
int64_t index_version,
int64_t build_id);
std::string
GetSegmentRawDataPathPrefix(ChunkManagerPtr cm, int64_t segment_id);
@ -132,6 +141,13 @@ PutIndexData(ChunkManager* remote_chunk_manager,
FieldDataMeta& field_meta,
IndexMeta& index_meta);
void
PutCompactionResultData(ChunkManager* remote_chunk_manager,
const std::vector<const uint8_t*>& data_slices,
const std::vector<int64_t>& slice_sizes,
const std::vector<std::string>& slice_names,
std::unordered_map<std::string, int64_t>& map);
std::map<std::string, int64_t>
PutIndexData(std::shared_ptr<milvus_storage::Space> space,
const std::vector<const uint8_t*>& data_slices,

View File

@ -12,8 +12,8 @@
#-------------------------------------------------------------------------------
# Update KNOWHERE_VERSION for the first occurrence
set( KNOWHERE_VERSION e141e8b )
set( GIT_REPOSITORY "https://github.com/zilliztech/knowhere.git")
set( KNOWHERE_VERSION f500ef0a3c146 )
set( GIT_REPOSITORY "https://github.com/chasingegg/Knowhere.git")
message(STATUS "Knowhere repo: ${GIT_REPOSITORY}")
message(STATUS "Knowhere version: ${KNOWHERE_VERSION}")

View File

@ -66,6 +66,7 @@ set(MILVUS_TEST_FILES
test_group_by.cpp
test_regex_query_util.cpp
test_regex_query.cpp
test_major_compaction.cpp
)
if ( BUILD_DISK_ANN STREQUAL "ON" )

View File

@ -214,6 +214,8 @@ const DataType kOptFieldDataType = DataType::INT64;
const FieldDataMeta kOptVecFieldDataMeta = {1, 2, 3, 100};
using OffsetT = uint32_t;
const std::string compactionRawDataPath = "/tmp/diskann/compaction_raw_data/";
auto
CreateFileManager(const ChunkManagerPtr& cm)
-> std::shared_ptr<DiskFileManagerImpl> {
@ -260,6 +262,29 @@ PrepareInsertData() -> std::string {
return path;
}
auto
PrepareVectorInsertData(int64_t segment_id) -> std::string {
std::vector<float> data = {1, 2, 3, 4, 5, 6, 7, 8};
int DIM = 2;
auto field_data =
milvus::storage::CreateFieldData(storage::DataType::VECTOR_FLOAT, DIM);
field_data->FillFieldData(data.data(), data.size() / DIM);
storage::InsertData insert_data(field_data);
storage::FieldDataMeta field_data_meta{100, 101, segment_id, 103};
insert_data.SetFieldDataMeta(field_data_meta);
insert_data.SetTimestamps(0, 100);
auto serialized_data = insert_data.Serialize(storage::StorageType::Remote);
auto chunk_manager =
storage::CreateChunkManager(get_default_local_storage_config());
std::string path = compactionRawDataPath + std::to_string(segment_id);
boost::filesystem::remove_all(path);
chunk_manager->Write(path, serialized_data.data(), serialized_data.size());
return path;
}
auto
PrepareInsertDataSpace()
-> std::pair<std::string, std::shared_ptr<milvus_storage::Space>> {
@ -326,6 +351,21 @@ PrepareOptionalField(const std::shared_ptr<DiskFileManagerImpl>& file_manager,
return opt_field;
}
void
CheckCacheCompactionRawData(const std::string& local_file_path,
int64_t offset,
float expected) {
std::ifstream ifs(local_file_path);
if (!ifs.is_open()) {
FAIL() << "open file failed: " << local_file_path << std::endl;
return;
}
float value;
ifs.seekg(offset, std::ios::beg);
ifs.read(reinterpret_cast<char*>(&value), sizeof(float));
EXPECT_EQ(value, expected);
}
void
CheckOptFieldCorrectness(const std::string& local_file_path) {
std::ifstream ifs(local_file_path);
@ -371,6 +411,32 @@ CheckOptFieldCorrectness(const std::string& local_file_path) {
}
}
TEST_F(DiskAnnFileManagerTest, CacheCompactionRawDataToDisk) {
auto file_manager = CreateFileManager(cm_);
std::map<int64_t, std::vector<std::string>> remote_files;
int64_t segment_id = 1;
int64_t segment_id2 = 2;
remote_files[segment_id] = {PrepareVectorInsertData(segment_id),
PrepareVectorInsertData(segment_id)};
remote_files[segment_id2] = {PrepareVectorInsertData(segment_id2)};
std::vector<std::string> output_files;
std::vector<uint64_t> offsets;
uint32_t dim = 0;
auto whole_size = file_manager->CacheCompactionRawDataToDisk(
remote_files, output_files, offsets, dim);
EXPECT_EQ(whole_size, 96);
EXPECT_EQ(output_files.size(), 2);
EXPECT_EQ(offsets.size(), 2);
EXPECT_EQ(offsets[0], 64);
EXPECT_EQ(offsets[1], 32);
EXPECT_EQ(dim, 2);
CheckCacheCompactionRawData(output_files[0], 0, 1);
CheckCacheCompactionRawData(output_files[0], 40, 3);
CheckCacheCompactionRawData(output_files[0], 48, 5);
CheckCacheCompactionRawData(output_files[1], 4, 2);
}
TEST_F(DiskAnnFileManagerTest, CacheOptFieldToDiskFieldEmpty) {
auto file_manager = CreateFileManager(cm_);
const auto& [insert_file_space_path, space] = PrepareInsertDataSpace();

View File

@ -0,0 +1,163 @@
// Copyright (C) 2019-2020 Zilliz. All rights reserved.
//
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software distributed under the License
// is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express
// or implied. See the License for the specific language governing permissions and limitations under the License
#include <gtest/gtest.h>
#include <functional>
#include <fstream>
#include <boost/filesystem.hpp>
#include <numeric>
#include <unordered_set>
#include "common/Tracer.h"
#include "index/InvertedIndexTantivy.h"
#include "storage/Util.h"
#include "storage/InsertData.h"
#include "indexbuilder/IndexFactory.h"
#include "index/IndexFactory.h"
#include "storage/LocalChunkManagerSingleton.h"
#include "test_utils/indexbuilder_test_utils.h"
#include "test_utils/storage_test_utils.h"
#include "index/Meta.h"
using namespace milvus;
void
ReadPBFile(std::string& file_path, google::protobuf::Message& message) {
std::ifstream infile;
infile.open(file_path.data(), std::ios_base::binary);
if (infile.fail()) {
std::stringstream err_msg;
err_msg << "Error: open local file '" << file_path << " failed, "
<< strerror(errno);
throw SegcoreError(FileOpenFailed, err_msg.str());
}
infile.seekg(0, std::ios::beg);
if (!message.ParseFromIstream(&infile)) {
std::stringstream err_msg;
err_msg << "Error: parse pb file '" << file_path << " failed, "
<< strerror(errno);
throw SegcoreError(FileReadFailed, err_msg.str());
}
}
template <typename T, DataType dtype>
void
test_run() {
int64_t collection_id = 1;
int64_t partition_id = 2;
int64_t segment_id = 3;
int64_t segment_id2 = 4;
int64_t field_id = 101;
int64_t index_build_id = 1000;
int64_t index_version = 10000;
int64_t dim = 100;
int64_t nb = 10000;
auto field_meta =
gen_field_meta(collection_id, partition_id, segment_id, field_id);
auto index_meta =
gen_index_meta(segment_id, field_id, index_build_id, index_version);
std::string root_path = "/tmp/test-major-compaction/";
auto storage_config = gen_local_storage_config(root_path);
auto cm = storage::CreateChunkManager(storage_config);
std::vector<T> data_gen(nb * dim);
for (int64_t i = 0; i < nb * dim; ++i) {
data_gen[i] = rand();
}
auto field_data = storage::CreateFieldData(dtype, dim);
field_data->FillFieldData(data_gen.data(), data_gen.size() / dim);
storage::InsertData insert_data(field_data);
insert_data.SetFieldDataMeta(field_meta);
insert_data.SetTimestamps(0, 100);
auto serialized_bytes = insert_data.Serialize(storage::Remote);
auto get_binlog_path = [=](int64_t log_id) {
return fmt::format("{}/{}/{}/{}/{}",
collection_id,
partition_id,
segment_id,
field_id,
log_id);
};
auto log_path = get_binlog_path(0);
auto cm_w = ChunkManagerWrapper(cm);
cm_w.Write(log_path, serialized_bytes.data(), serialized_bytes.size());
storage::FileManagerContext ctx(field_meta, index_meta, cm);
Config config;
std::map<int64_t, std::vector<std::string>> remote_files;
// two segments
remote_files[segment_id] = {log_path};
remote_files[segment_id2] = {log_path};
config["insert_files"] = remote_files;
config["segment_size"] = 1; // 1MB
config["train_size"] = 26; // 26GB
auto compaction =
indexbuilder::IndexFactory::GetInstance().CreateCompactionJob(
dtype, config, ctx);
compaction->Train();
auto lcm =
storage::LocalChunkManagerSingleton::GetInstance().GetChunkManager();
std::string path_prefix = storage::GenCompactionResultPathPrefix(
lcm, index_meta.build_id, index_meta.index_version);
std::string centroid_path = path_prefix + "centroids";
milvus::proto::segcore::ClusteringCentroidsStats stats;
ReadPBFile(centroid_path, stats);
std::vector<T> centroids;
for (const auto& centroid : stats.centroids()) {
const auto& float_vector = centroid.float_vector();
for (float value : float_vector.data()) {
centroids.emplace_back(T(value));
}
}
int expected_num_clusters = 8;
ASSERT_EQ(centroids.size(), 8 * dim);
std::string centroid_id_mapping_path =
path_prefix + std::to_string(segment_id);
milvus::proto::segcore::ClusteringCentroidIdMappingStats mapping_stats;
std::string centroid_id_mapping_path2 =
path_prefix + std::to_string(segment_id2);
milvus::proto::segcore::ClusteringCentroidIdMappingStats mapping_stats2;
ReadPBFile(centroid_id_mapping_path, mapping_stats);
ReadPBFile(centroid_id_mapping_path2, mapping_stats2);
std::vector<uint32_t> centroid_id_mapping;
std::vector<int64_t> num_in_centroid;
for (const auto id : mapping_stats.centroid_id_mapping()) {
centroid_id_mapping.emplace_back(id);
ASSERT_TRUE(id < expected_num_clusters);
}
ASSERT_EQ(centroid_id_mapping.size(), nb);
for (const auto num : mapping_stats.num_in_centroid()) {
num_in_centroid.emplace_back(num);
}
ASSERT_EQ(
std::accumulate(num_in_centroid.begin(), num_in_centroid.end(), 0),
nb * 2);
// second id mapping should be the same with the first one since the segment data is the same
for (int64_t i = 0; i < mapping_stats2.centroid_id_mapping_size(); i++) {
ASSERT_EQ(mapping_stats2.centroid_id_mapping(i),
centroid_id_mapping[i]);
}
for (int64_t i = 0; i < mapping_stats2.num_in_centroid_size(); i++) {
ASSERT_EQ(mapping_stats2.num_in_centroid(i), num_in_centroid[i]);
}
lcm->RemoveDir(path_prefix);
}
TEST(MajorCompaction, Naive) {
test_run<float, DataType::VECTOR_FLOAT>();
}

View File

@ -23,6 +23,7 @@
#include "storage/Types.h"
#include "storage/InsertData.h"
#include "storage/ThreadPools.h"
#include <boost/filesystem.hpp>
using milvus::DataType;
using milvus::FieldDataPtr;
@ -137,4 +138,61 @@ PutFieldData(milvus::storage::ChunkManager* remote_chunk_manager,
return remote_paths_to_size;
}
auto
gen_field_meta(int64_t collection_id = 1,
int64_t partition_id = 2,
int64_t segment_id = 3,
int64_t field_id = 101) -> milvus::storage::FieldDataMeta {
return milvus::storage::FieldDataMeta{
.collection_id = collection_id,
.partition_id = partition_id,
.segment_id = segment_id,
.field_id = field_id,
};
}
auto
gen_index_meta(int64_t segment_id = 3,
int64_t field_id = 101,
int64_t index_build_id = 1000,
int64_t index_version = 10000) -> milvus::storage::IndexMeta {
return milvus::storage::IndexMeta{
.segment_id = segment_id,
.field_id = field_id,
.build_id = index_build_id,
.index_version = index_version,
};
}
auto
gen_local_storage_config(const std::string& root_path)
-> milvus::storage::StorageConfig {
auto ret = milvus::storage::StorageConfig{};
ret.storage_type = "local";
ret.root_path = root_path;
return ret;
}
struct ChunkManagerWrapper {
ChunkManagerWrapper(milvus::storage::ChunkManagerPtr cm) : cm_(cm) {
}
~ChunkManagerWrapper() {
for (const auto& file : written_) {
cm_->Remove(file);
}
boost::filesystem::remove_all(cm_->GetRootPath());
}
void
Write(const std::string& filepath, void* buf, uint64_t len) {
written_.insert(filepath);
cm_->Write(filepath, buf, len);
}
const milvus::storage::ChunkManagerPtr cm_;
std::unordered_set<std::string> written_;
};
} // namespace

View File

@ -0,0 +1,197 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package datacoord
import (
"context"
"fmt"
"sync"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/metastore"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/timerecord"
)
type analysisMeta struct {
ctx context.Context
lock sync.RWMutex
catalog metastore.DataCoordCatalog
// taskID -> analyzeStats
// TODO: when to mark as dropped?
tasks map[int64]*model.AnalysisTask
}
func newAnalysisMeta(ctx context.Context, catalog metastore.DataCoordCatalog) (*analysisMeta, error) {
mt := &analysisMeta{
ctx: ctx,
lock: sync.RWMutex{},
catalog: catalog,
tasks: make(map[int64]*model.AnalysisTask),
}
if err := mt.reloadFromKV(); err != nil {
return nil, err
}
return mt, nil
}
func (m *analysisMeta) reloadFromKV() error {
record := timerecord.NewTimeRecorder("analysisMeta-reloadFromKV")
// load analysis stats
analysisTasks, err := m.catalog.ListAnalysisTasks(m.ctx)
if err != nil {
log.Warn("analysisMeta reloadFromKV load analysis tasks failed", zap.Error(err))
return err
}
for _, analysisTask := range analysisTasks {
m.tasks[analysisTask.TaskID] = analysisTask
}
log.Info("analysisMeta reloadFromKV done", zap.Duration("duration", record.ElapseSpan()))
return nil
}
func (m *analysisMeta) saveTask(newTask *model.AnalysisTask) error {
if err := m.catalog.SaveAnalysisTask(m.ctx, newTask); err != nil {
return err
}
m.tasks[newTask.TaskID] = newTask
return nil
}
// checkTask is checking and prompting only when creating tasks.
// Please don't use it.
func (m *analysisMeta) checkTask(task *model.AnalysisTask) {
if t := m.tasks[task.TaskID]; t != nil {
log.Warn("task already exist with taskID", zap.Int64("taskID", task.TaskID),
zap.Int64("collectionID", task.CollectionID), zap.Int64("partitionID", task.PartitionID))
}
for _, t := range m.tasks {
if t.CollectionID == task.CollectionID && t.PartitionID == task.PartitionID &&
t.State != commonpb.IndexState_Finished && t.State != commonpb.IndexState_Failed {
log.Warn("there is already exist task with partition and it not finished",
zap.Int64("taskID", task.TaskID),
zap.Int64("collectionID", task.CollectionID), zap.Int64("partitionID", task.PartitionID))
break
}
}
}
func (m *analysisMeta) GetTask(taskID int64) *model.AnalysisTask {
m.lock.RLock()
defer m.lock.RUnlock()
return model.CloneAnalysisTask(m.tasks[taskID])
}
func (m *analysisMeta) AddAnalysisTask(task *model.AnalysisTask) error {
m.lock.Lock()
defer m.lock.Unlock()
m.checkTask(task)
log.Info("add analysis task", zap.Int64("taskID", task.TaskID),
zap.Int64("collectionID", task.CollectionID), zap.Int64("partitionID", task.PartitionID))
return m.saveTask(task)
}
func (m *analysisMeta) DropAnalysisTask(taskID int64) error {
m.lock.Lock()
defer m.lock.Unlock()
log.Info("drop analysis task", zap.Int64("taskID", taskID))
if err := m.catalog.DropAnalysisTask(m.ctx, taskID); err != nil {
log.Warn("drop analysis task by catalog failed", zap.Int64("taskID", taskID),
zap.Error(err))
return err
}
delete(m.tasks, taskID)
return nil
}
func (m *analysisMeta) UpdateVersion(taskID int64) error {
m.lock.Lock()
defer m.lock.Unlock()
t, ok := m.tasks[taskID]
if !ok {
return fmt.Errorf("there is no task with taskID: %d", taskID)
}
cloneT := model.CloneAnalysisTask(t)
cloneT.Version++
log.Info("update task version", zap.Int64("taskID", taskID), zap.Int64("newVersion", cloneT.Version))
return m.saveTask(cloneT)
}
func (m *analysisMeta) BuildingTask(taskID, nodeID int64) error {
m.lock.Lock()
defer m.lock.Unlock()
t, ok := m.tasks[taskID]
if !ok {
return fmt.Errorf("there is no task with taskID: %d", taskID)
}
cloneT := model.CloneAnalysisTask(t)
cloneT.NodeID = nodeID
cloneT.State = commonpb.IndexState_InProgress
log.Info("task will be building", zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID))
return m.saveTask(cloneT)
}
func (m *analysisMeta) FinishTask(taskID int64, result *indexpb.AnalysisResult) error {
m.lock.Lock()
defer m.lock.Unlock()
t, ok := m.tasks[taskID]
if !ok {
return fmt.Errorf("there is no task with taskID: %d", taskID)
}
log.Info("finish task meta...", zap.Int64("taskID", taskID), zap.String("state", result.GetState().String()),
zap.String("centroidsFile", result.GetCentroidsFile()),
zap.Any("segmentOffsetMapping", result.GetSegmentOffsetMappingFiles()),
zap.String("failReason", result.GetFailReason()))
cloneT := model.CloneAnalysisTask(t)
cloneT.State = result.GetState()
cloneT.FailReason = result.GetFailReason()
cloneT.CentroidsFile = result.GetCentroidsFile()
cloneT.SegmentOffsetMappingFiles = result.GetSegmentOffsetMappingFiles()
return m.saveTask(cloneT)
}
func (m *analysisMeta) GetAllTasks() map[int64]*model.AnalysisTask {
m.lock.RLock()
defer m.lock.RUnlock()
tasks := make(map[int64]*model.AnalysisTask)
for taskID, t := range m.tasks {
tasks[taskID] = model.CloneAnalysisTask(t)
}
return tasks
}

View File

@ -0,0 +1,255 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package datacoord
import (
"context"
"testing"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/metastore/mocks"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/indexpb"
)
type AnalysisMetaSuite struct {
suite.Suite
collectionID int64
partitionID int64
fieldID int64
segmentIDs []int64
}
func (s *AnalysisMetaSuite) initParams() {
s.collectionID = 100
s.partitionID = 101
s.fieldID = 102
s.segmentIDs = []int64{1000, 1001, 1002, 1003}
}
func (s *AnalysisMetaSuite) Test_analysisMeta() {
s.initParams()
catalog := mocks.NewDataCoordCatalog(s.T())
catalog.EXPECT().ListAnalysisTasks(mock.Anything).Return([]*model.AnalysisTask{
{
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 1,
State: commonpb.IndexState_IndexStateNone,
},
{
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 2,
State: commonpb.IndexState_Unissued,
},
{
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 3,
State: commonpb.IndexState_InProgress,
},
{
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 4,
State: commonpb.IndexState_Retry,
},
{
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 5,
State: commonpb.IndexState_Finished,
},
{
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 6,
State: commonpb.IndexState_Failed,
},
}, nil)
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(nil)
catalog.EXPECT().DropAnalysisTask(mock.Anything, mock.Anything).Return(nil)
ctx := context.Background()
am, err := newAnalysisMeta(ctx, catalog)
s.NoError(err)
s.Equal(6, len(am.GetAllTasks()))
s.Run("GetTask", func() {
t := am.GetTask(1)
s.NotNil(t)
t = am.GetTask(100)
s.Nil(t)
})
s.Run("AddAnalysisTask", func() {
t := &model.AnalysisTask{
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 7,
}
err := am.AddAnalysisTask(t)
s.NoError(err)
s.Equal(7, len(am.GetAllTasks()))
err = am.AddAnalysisTask(t)
s.NoError(err)
s.Equal(7, len(am.GetAllTasks()))
})
s.Run("DropAnalysisTask", func() {
err := am.DropAnalysisTask(7)
s.NoError(err)
s.Equal(6, len(am.GetAllTasks()))
})
s.Run("UpdateVersion", func() {
err := am.UpdateVersion(1)
s.NoError(err)
s.Equal(int64(1), am.GetTask(1).Version)
})
s.Run("BuildingTask", func() {
err := am.BuildingTask(1, 1)
s.NoError(err)
s.Equal(commonpb.IndexState_InProgress, am.GetTask(1).State)
})
s.Run("FinishTask", func() {
err := am.FinishTask(1, &indexpb.AnalysisResult{
TaskID: 1,
State: commonpb.IndexState_Finished,
CentroidsFile: "a/b/c",
SegmentOffsetMappingFiles: map[int64]string{1000: "1000/a", 1001: "1001/a", 1002: "1002/a", 1003: "1003/a"},
})
s.NoError(err)
s.Equal(commonpb.IndexState_Finished, am.GetTask(1).State)
s.Equal("a/b/c", am.GetTask(1).CentroidsFile)
})
}
func (s *AnalysisMetaSuite) Test_failCase() {
s.initParams()
catalog := mocks.NewDataCoordCatalog(s.T())
catalog.EXPECT().ListAnalysisTasks(mock.Anything).Return(nil, errors.New("error")).Once()
ctx := context.Background()
am, err := newAnalysisMeta(ctx, catalog)
s.Error(err)
s.Nil(am)
catalog.EXPECT().ListAnalysisTasks(mock.Anything).Return([]*model.AnalysisTask{
{
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 1,
State: commonpb.IndexState_Unissued,
},
}, nil)
am, err = newAnalysisMeta(ctx, catalog)
s.NoError(err)
s.NotNil(am)
s.Equal(1, len(am.GetAllTasks()))
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(errors.New("error"))
catalog.EXPECT().DropAnalysisTask(mock.Anything, mock.Anything).Return(errors.New("error"))
s.Run("AddAnalysisTask", func() {
t := &model.AnalysisTask{
TenantID: "",
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 1111,
}
err := am.AddAnalysisTask(t)
s.Error(err)
s.Nil(am.GetTask(1111))
})
s.Run("DropAnalysisTask", func() {
err := am.DropAnalysisTask(1)
s.Error(err)
s.NotNil(am.GetTask(1))
})
s.Run("UpdateVersion", func() {
err := am.UpdateVersion(777)
s.Error(err)
err = am.UpdateVersion(1)
s.Error(err)
s.Equal(int64(0), am.GetTask(1).Version)
})
s.Run("BuildingTask", func() {
err := am.BuildingTask(777, 1)
s.Error(err)
err = am.BuildingTask(1, 1)
s.Error(err)
s.Equal(int64(0), am.GetTask(1).NodeID)
s.Equal(commonpb.IndexState_Unissued, am.GetTask(1).State)
})
s.Run("FinishTask", func() {
err := am.FinishTask(777, nil)
s.Error(err)
err = am.FinishTask(1, &indexpb.AnalysisResult{
TaskID: 1,
State: commonpb.IndexState_Finished,
CentroidsFile: "a/b/c",
SegmentOffsetMappingFiles: map[int64]string{1000: "1000/a", 1001: "1001/a", 1002: "1002/a", 1003: "1003/a"},
})
s.Error(err)
s.Equal(commonpb.IndexState_Unissued, am.GetTask(1).State)
s.Equal("", am.GetTask(1).CentroidsFile)
s.Equal(0, len(am.GetTask(1).SegmentOffsetMappingFiles))
})
}
func TestAnalysisMeta(t *testing.T) {
suite.Run(t, new(AnalysisMetaSuite))
}

View File

@ -0,0 +1,371 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package datacoord
import (
"context"
"sync"
"time"
"github.com/samber/lo"
"go.uber.org/zap"
"golang.org/x/exp/slices"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/merr"
)
type analysisTaskScheduler struct {
ctx context.Context
cancel context.CancelFunc
wg sync.WaitGroup
lock sync.RWMutex
scheduleDuration time.Duration
meta *meta
analysisMeta *analysisMeta
tasks map[int64]taskState
notifyChan chan struct{}
nodeManager WorkerManager
}
func newAnalysisTaskScheduler(
ctx context.Context,
metaTable *meta, at *analysisMeta, nodeManager WorkerManager,
) *analysisTaskScheduler {
ctx, cancel := context.WithCancel(ctx)
ats := &analysisTaskScheduler{
ctx: ctx,
cancel: cancel,
meta: metaTable,
analysisMeta: at,
tasks: make(map[int64]taskState),
notifyChan: make(chan struct{}, 1),
scheduleDuration: Params.DataCoordCfg.IndexTaskSchedulerInterval.GetAsDuration(time.Millisecond),
nodeManager: nodeManager,
}
ats.reloadFromMeta()
log.Info("new analysis task scheduler success")
return ats
}
func (ats *analysisTaskScheduler) reloadFromMeta() {
ats.lock.Lock()
defer ats.lock.Unlock()
allTasks := ats.analysisMeta.GetAllTasks()
for taskID, t := range allTasks {
if t.State == commonpb.IndexState_Finished || t.State == commonpb.IndexState_Failed {
continue
} else if t.State == commonpb.IndexState_InProgress {
ats.tasks[taskID] = taskInProgress
} else {
ats.tasks[taskID] = taskInit
}
}
}
func (ats *analysisTaskScheduler) Start() {
ats.wg.Add(1)
go ats.Schedule()
}
func (ats *analysisTaskScheduler) Stop() {
ats.cancel()
ats.wg.Wait()
}
func (ats *analysisTaskScheduler) Schedule() {
log.Ctx(ats.ctx).Info("analysis task scheduler loop start")
defer ats.wg.Done()
ticker := time.NewTicker(ats.scheduleDuration)
defer ticker.Stop()
for {
select {
case <-ats.ctx.Done():
log.Ctx(ats.ctx).Warn("analysis task scheduler ctx done")
return
case _, ok := <-ats.notifyChan:
if ok {
ats.run()
}
case <-ticker.C:
ats.run()
}
}
}
func (ats *analysisTaskScheduler) enqueue(taskID int64) {
defer func() {
ats.notifyChan <- struct{}{}
}()
ats.lock.Lock()
defer ats.lock.Unlock()
if _, ok := ats.tasks[taskID]; !ok {
ats.tasks[taskID] = taskInit
}
log.Info("analyze task enqueue successfully", zap.Int64("taskID", taskID))
}
func (ats *analysisTaskScheduler) run() {
ats.lock.RLock()
taskIDs := lo.Keys(ats.tasks)
ats.lock.RUnlock()
if len(taskIDs) > 0 {
log.Ctx(ats.ctx).Info("analyze task scheduler start", zap.Int("analyze task num", len(taskIDs)))
}
for _, taskID := range taskIDs {
if !ats.process(taskID) {
log.Ctx(ats.ctx).Info("there is no idle indexing node, wait a minute...")
break
}
}
}
func (ats *analysisTaskScheduler) getTaskState(taskID int64) taskState {
ats.lock.RLock()
defer ats.lock.RUnlock()
return ats.tasks[taskID]
}
func (ats *analysisTaskScheduler) updateTaskState(taskID int64, state taskState) {
ats.lock.Lock()
defer ats.lock.Unlock()
ats.tasks[taskID] = state
}
func (ats *analysisTaskScheduler) deleteTask(taskID int64) {
ats.lock.Lock()
defer ats.lock.Unlock()
delete(ats.tasks, taskID)
}
func (ats *analysisTaskScheduler) process(taskID int64) bool {
state := ats.getTaskState(taskID)
t := ats.analysisMeta.GetTask(taskID)
if t == nil {
log.Ctx(ats.ctx).Info("task is nil, delete it", zap.Int64("taskID", taskID))
ats.deleteTask(taskID)
return true
}
log.Ctx(ats.ctx).Info("process task", zap.Int64("taskID", taskID), zap.String("state", state.String()))
switch state {
case taskInit:
var storageConfig *indexpb.StorageConfig
if Params.CommonCfg.StorageType.GetValue() == "local" {
storageConfig = &indexpb.StorageConfig{
RootPath: Params.LocalStorageCfg.Path.GetValue(),
StorageType: Params.CommonCfg.StorageType.GetValue(),
}
} else {
storageConfig = &indexpb.StorageConfig{
Address: Params.MinioCfg.Address.GetValue(),
AccessKeyID: Params.MinioCfg.AccessKeyID.GetValue(),
SecretAccessKey: Params.MinioCfg.SecretAccessKey.GetValue(),
UseSSL: Params.MinioCfg.UseSSL.GetAsBool(),
BucketName: Params.MinioCfg.BucketName.GetValue(),
RootPath: Params.MinioCfg.RootPath.GetValue(),
UseIAM: Params.MinioCfg.UseIAM.GetAsBool(),
IAMEndpoint: Params.MinioCfg.IAMEndpoint.GetValue(),
StorageType: Params.CommonCfg.StorageType.GetValue(),
Region: Params.MinioCfg.Region.GetValue(),
UseVirtualHost: Params.MinioCfg.UseVirtualHost.GetAsBool(),
CloudProvider: Params.MinioCfg.CloudProvider.GetValue(),
RequestTimeoutMs: Params.MinioCfg.RequestTimeoutMs.GetAsInt64(),
}
}
req := &indexpb.AnalysisRequest{
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
TaskID: taskID,
CollectionID: t.CollectionID,
PartitionID: t.PartitionID,
FieldID: t.FieldID,
FieldName: t.FieldName,
FieldType: t.FieldType,
Dim: t.Dim,
SegmentStats: make(map[int64]*indexpb.SegmentStats),
Version: t.Version + 1,
StorageConfig: storageConfig,
}
// When data analysis occurs, segments must not be discarded. Such as compaction, GC, etc.
segments := ats.meta.SelectSegments(func(info *SegmentInfo) bool {
return isSegmentHealthy(info) && slices.Contains(t.SegmentIDs, info.ID)
})
segmentsMap := lo.SliceToMap(segments, func(t *SegmentInfo) (int64, *SegmentInfo) {
return t.ID, t
})
for _, segID := range t.SegmentIDs {
info := segmentsMap[segID]
if info == nil {
log.Warn("analyze stats task is processing, but segment is nil, delete the task", zap.Int64("taskID", taskID),
zap.Int64("segmentID", segID))
ats.deleteTask(taskID)
return true
}
// get binlogIDs
binlogIDs := getBinLogIds(info, t.FieldID)
req.SegmentStats[segID] = &indexpb.SegmentStats{
ID: segID,
NumRows: info.GetNumOfRows(),
LogIDs: binlogIDs,
}
}
// 1. update task version
if err := ats.analysisMeta.UpdateVersion(taskID); err != nil {
log.Warn("update task version failed", zap.Int64("taskID", taskID), zap.Error(err))
return false
}
assignFunc := func(nodeID int64, client types.IndexNodeClient) error {
if err := ats.analysisMeta.BuildingTask(taskID, nodeID); err != nil {
log.Warn("set task building state failed", zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID),
zap.Error(err))
return err
}
if err := ats.assignTask(client, req); err != nil {
log.Ctx(ats.ctx).Warn("assign analysis task to indexNode failed", zap.Int64("taskID", taskID),
zap.Int64("nodeID", nodeID), zap.Error(err))
ats.updateTaskState(taskID, taskRetry)
return err
}
log.Ctx(ats.ctx).Info("analysis task assigned successfully", zap.Int64("taskID", taskID),
zap.Int64("nodeID", nodeID))
return nil
}
if err := ats.nodeManager.SelectNodeAndAssignTask(assignFunc); err != nil {
log.Ctx(ats.ctx).Info("nodeManager select node or assign task failed", zap.Error(err))
return false
}
ats.updateTaskState(taskID, taskInProgress)
case taskRetry:
if !ats.dropIndexTask(taskID, t.NodeID) {
return true
}
ats.updateTaskState(taskID, taskInit)
case taskDone:
if !ats.dropIndexTask(taskID, t.NodeID) {
return true
}
ats.deleteTask(taskID)
default:
// taskInProgress
ats.updateTaskState(taskID, ats.getTaskResult(taskID, t.NodeID))
}
return true
}
func (ats *analysisTaskScheduler) assignTask(builderClient types.IndexNodeClient, req *indexpb.AnalysisRequest) error {
ctx, cancel := context.WithTimeout(context.Background(), reqTimeoutInterval)
defer cancel()
resp, err := builderClient.Analysis(ctx, req)
if err == nil {
err = merr.Error(resp)
}
return err
}
func (ats *analysisTaskScheduler) dropIndexTask(taskID, nodeID UniqueID) bool {
client, exist := ats.nodeManager.GetClientByID(nodeID)
if exist {
ctx1, cancel := context.WithTimeout(ats.ctx, reqTimeoutInterval)
defer cancel()
status, err := client.DropAnalysisTasks(ctx1, &indexpb.DropAnalysisTasksRequest{
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
TaskIDs: []UniqueID{taskID},
})
if err == nil {
err = merr.Error(status)
}
if err != nil {
log.Ctx(ats.ctx).Warn("indexNode drop the analysis task failed",
zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID), zap.Error(err))
return false
}
log.Ctx(ats.ctx).Info("indexNode drop the analysis task success",
zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID))
return true
}
log.Ctx(ats.ctx).Info("IndexNode no longer exist, no need to drop analysis task",
zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID))
return true
}
func (ats *analysisTaskScheduler) getTaskResult(taskID, nodeID int64) taskState {
client, exist := ats.nodeManager.GetClientByID(nodeID)
if exist {
ctx1, cancel := context.WithTimeout(ats.ctx, reqTimeoutInterval)
defer cancel()
response, err := client.QueryAnalysisResult(ctx1, &indexpb.QueryAnalysisResultRequest{
ClusterID: Params.CommonCfg.ClusterPrefix.GetValue(),
TaskIDs: []int64{taskID},
})
if err == nil {
err = merr.Error(response.GetStatus())
}
if err != nil {
log.Ctx(ats.ctx).Warn("get analysis task result from IndexNode failed",
zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID), zap.Error(err))
return taskRetry
}
// indexInfos length is always one.
result, ok := response.GetResults()[taskID]
if !ok {
log.Ctx(ats.ctx).Info("this analysis task should be retry, indexNode does not have this task",
zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID))
return taskRetry
}
if result.GetState() == commonpb.IndexState_Finished || result.GetState() == commonpb.IndexState_Failed {
log.Ctx(ats.ctx).Info("this analysis task has been finished",
zap.Int64("taskID", taskID), zap.String("state", result.GetState().String()))
if err := ats.analysisMeta.FinishTask(taskID, result); err != nil {
log.Ctx(ats.ctx).Warn("update analysis task state fail",
zap.Int64("taskID", taskID),
zap.String("state", result.GetState().String()), zap.Error(err))
return taskInProgress
}
return taskDone
} else if result.GetState() == commonpb.IndexState_Retry || result.GetState() == commonpb.IndexState_IndexStateNone {
log.Ctx(ats.ctx).Info("this analysis task should be retry", zap.Int64("taskID", taskID),
zap.String("state", result.GetState().String()), zap.String("fail reason", result.GetFailReason()))
return taskRetry
}
return taskInProgress
}
// !exist --> node down
log.Ctx(ats.ctx).Info("this analysis task should be retry, indexNode is no longer exist",
zap.Int64("taskID", taskID), zap.Int64("nodeID", nodeID))
return taskRetry
}

View File

@ -0,0 +1,448 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package datacoord
import (
"context"
"fmt"
"sync"
"testing"
"time"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/mock"
"github.com/stretchr/testify/suite"
"google.golang.org/grpc"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/metastore"
catalogmocks "github.com/milvus-io/milvus/internal/metastore/mocks"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/pkg/util/merr"
)
type AnalysisSchedulerSuite struct {
suite.Suite
collectionID int64
partitionID int64
fieldID int64
segmentIDs []int64
nodeID int64
duration time.Duration
}
func (s *AnalysisSchedulerSuite) initParams() {
s.collectionID = 100
s.partitionID = 101
s.fieldID = 102
s.nodeID = 103
s.segmentIDs = []int64{1000, 1001, 1002}
s.duration = time.Millisecond * 100
}
func (s *AnalysisSchedulerSuite) createAnalysisMeta(catalog metastore.DataCoordCatalog) *analysisMeta {
return &analysisMeta{
ctx: context.Background(),
lock: sync.RWMutex{},
catalog: catalog,
tasks: map[int64]*model.AnalysisTask{
1: {
TenantID: "",
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 1,
State: commonpb.IndexState_Unissued,
},
2: {
TenantID: "",
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 2,
NodeID: s.nodeID,
State: commonpb.IndexState_InProgress,
},
3: {
TenantID: "",
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 3,
NodeID: s.nodeID,
State: commonpb.IndexState_Finished,
},
4: {
TenantID: "",
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 4,
NodeID: s.nodeID,
State: commonpb.IndexState_Failed,
},
5: {
TenantID: "",
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: []int64{1001, 1002},
TaskID: 5,
NodeID: s.nodeID,
State: commonpb.IndexState_Retry,
},
},
}
}
func (s *AnalysisSchedulerSuite) createMeta() *meta {
return &meta{
segments: &SegmentsInfo{
segments: map[UniqueID]*SegmentInfo{
1000: {
SegmentInfo: &datapb.SegmentInfo{
ID: 1000,
CollectionID: s.collectionID,
PartitionID: s.partitionID,
NumOfRows: 3000,
State: commonpb.SegmentState_Flushed,
Binlogs: []*datapb.FieldBinlog{{FieldID: s.fieldID, Binlogs: []*datapb.Binlog{{LogID: 1}, {LogID: 2}, {LogID: 3}}}},
},
},
1001: {
SegmentInfo: &datapb.SegmentInfo{
ID: 1001,
CollectionID: s.collectionID,
PartitionID: s.partitionID,
NumOfRows: 3000,
State: commonpb.SegmentState_Flushed,
Binlogs: []*datapb.FieldBinlog{{FieldID: s.fieldID, Binlogs: []*datapb.Binlog{{LogID: 1}, {LogID: 2}, {LogID: 3}}}},
},
},
1002: {
SegmentInfo: &datapb.SegmentInfo{
ID: 1002,
CollectionID: s.collectionID,
PartitionID: s.partitionID,
NumOfRows: 3000,
State: commonpb.SegmentState_Flushed,
Binlogs: []*datapb.FieldBinlog{{FieldID: s.fieldID, Binlogs: []*datapb.Binlog{{LogID: 1}, {LogID: 2}, {LogID: 3}}}},
},
},
},
},
}
}
func (s *AnalysisSchedulerSuite) Test_analysisScheduler() {
s.initParams()
ctx := context.Background()
catalog := catalogmocks.NewDataCoordCatalog(s.T())
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(nil)
in := mocks.NewMockIndexNodeClient(s.T())
in.EXPECT().Analysis(mock.Anything, mock.Anything).Return(merr.Success(), nil)
in.EXPECT().QueryAnalysisResult(mock.Anything, mock.Anything).RunAndReturn(
func(ctx context.Context, request *indexpb.QueryAnalysisResultRequest, option ...grpc.CallOption) (*indexpb.QueryAnalysisResultResponse, error) {
results := make(map[int64]*indexpb.AnalysisResult)
for _, taskID := range request.GetTaskIDs() {
results[taskID] = &indexpb.AnalysisResult{
TaskID: taskID,
State: commonpb.IndexState_Finished,
CentroidsFile: fmt.Sprintf("%d/stats_file", taskID),
SegmentOffsetMappingFiles: map[int64]string{
1000: "1000/offset_mapping",
1001: "1001/offset_mapping",
1002: "1002/offset_mapping",
},
FailReason: "",
}
}
return &indexpb.QueryAnalysisResultResponse{
Status: merr.Success(),
ClusterID: request.GetClusterID(),
Results: results,
}, nil
})
in.EXPECT().DropAnalysisTasks(mock.Anything, mock.Anything).Return(merr.Success(), nil)
workerManager := NewMockWorkerManager(s.T())
workerManager.EXPECT().SelectNodeAndAssignTask(mock.Anything).RunAndReturn(
func(f func(int64, types.IndexNodeClient) error) error {
return f(s.nodeID, in)
})
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true)
mt := s.createMeta()
at := s.createAnalysisMeta(catalog)
scheduler := newAnalysisTaskScheduler(ctx, mt, at, workerManager)
s.Equal(3, len(scheduler.tasks))
s.Equal(taskInit, scheduler.tasks[1])
s.Equal(taskInProgress, scheduler.tasks[2])
s.Equal(taskInit, scheduler.tasks[5])
scheduler.scheduleDuration = time.Millisecond * 500
scheduler.Start()
s.Run("enqueue", func() {
newTask := &model.AnalysisTask{
CollectionID: s.collectionID,
PartitionID: s.partitionID,
FieldID: s.fieldID,
SegmentIDs: s.segmentIDs,
TaskID: 6,
}
err := scheduler.analysisMeta.AddAnalysisTask(newTask)
s.NoError(err)
scheduler.enqueue(6)
})
for {
scheduler.lock.RLock()
taskNum := len(scheduler.tasks)
scheduler.lock.RUnlock()
if taskNum == 0 {
break
}
time.Sleep(time.Second)
}
scheduler.Stop()
}
func (s *AnalysisSchedulerSuite) Test_failCase() {
s.initParams()
ctx := context.Background()
catalog := catalogmocks.NewDataCoordCatalog(s.T())
catalog.EXPECT().DropAnalysisTask(mock.Anything, mock.Anything).Return(nil)
in := mocks.NewMockIndexNodeClient(s.T())
workerManager := NewMockWorkerManager(s.T())
workerManager.EXPECT().SelectNodeAndAssignTask(mock.Anything).RunAndReturn(
func(f func(int64, types.IndexNodeClient) error) error {
return f(s.nodeID, in)
})
mt := s.createMeta()
at := s.createAnalysisMeta(catalog)
scheduler := newAnalysisTaskScheduler(ctx, mt, at, workerManager)
// remove task in meta
err := scheduler.analysisMeta.DropAnalysisTask(2)
s.NoError(err)
mt.segments.DropSegment(1000)
scheduler.scheduleDuration = s.duration
scheduler.Start()
// 1. update version failed --> state: init
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(errors.New("catalog update version error")).Once()
// 2. update version success, but building fail --> state: init
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(nil).Once()
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(errors.New("catalog update building error")).Once()
// 3. update version success, building success, but assign task fail --> state: retry
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(nil).Twice()
in.EXPECT().Analysis(mock.Anything, mock.Anything).Return(merr.Success(), errors.New("assign task error")).Once()
// 4. drop task success, --> state: init
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().DropAnalysisTasks(mock.Anything, mock.Anything).Return(merr.Success(), nil).Once()
// 5. update version success, building success, assign task success --> state: InProgress
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(nil).Twice()
in.EXPECT().Analysis(mock.Anything, mock.Anything).Return(merr.Success(), nil).Once()
// 6. get task state: InProgress --> state: InProgress
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().QueryAnalysisResult(mock.Anything, mock.Anything).RunAndReturn(
func(ctx context.Context, request *indexpb.QueryAnalysisResultRequest, option ...grpc.CallOption) (*indexpb.QueryAnalysisResultResponse, error) {
results := make(map[int64]*indexpb.AnalysisResult)
for _, taskID := range request.GetTaskIDs() {
results[taskID] = &indexpb.AnalysisResult{
TaskID: taskID,
State: commonpb.IndexState_InProgress,
}
}
return &indexpb.QueryAnalysisResultResponse{
Status: merr.Success(),
ClusterID: request.GetClusterID(),
Results: results,
}, nil
}).Once()
// 7. get task state: Finished, but save meta fail --> state: InProgress
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().QueryAnalysisResult(mock.Anything, mock.Anything).RunAndReturn(
func(ctx context.Context, request *indexpb.QueryAnalysisResultRequest, option ...grpc.CallOption) (*indexpb.QueryAnalysisResultResponse, error) {
results := make(map[int64]*indexpb.AnalysisResult)
for _, taskID := range request.GetTaskIDs() {
results[taskID] = &indexpb.AnalysisResult{
TaskID: taskID,
State: commonpb.IndexState_Finished,
CentroidsFile: fmt.Sprintf("%d/stats_file", taskID),
SegmentOffsetMappingFiles: map[int64]string{
1000: "1000/offset_mapping",
1001: "1001/offset_mapping",
1002: "1002/offset_mapping",
},
FailReason: "",
}
}
return &indexpb.QueryAnalysisResultResponse{
Status: merr.Success(),
ClusterID: request.GetClusterID(),
Results: results,
}, nil
}).Once()
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(errors.New("catalog save finished error")).Once()
// 8. get task state error --> state: retry
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().QueryAnalysisResult(mock.Anything, mock.Anything).Return(nil, errors.New("get task result error")).Once()
// 9. drop task success --> state: init
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().DropAnalysisTasks(mock.Anything, mock.Anything).Return(merr.Success(), nil).Once()
// 10. update version success, building success, assign task success --> state: InProgress
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(nil).Twice()
in.EXPECT().Analysis(mock.Anything, mock.Anything).Return(merr.Success(), nil).Once()
// 11. get task state: retry --> state: retry
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().QueryAnalysisResult(mock.Anything, mock.Anything).RunAndReturn(
func(ctx context.Context, request *indexpb.QueryAnalysisResultRequest, option ...grpc.CallOption) (*indexpb.QueryAnalysisResultResponse, error) {
results := make(map[int64]*indexpb.AnalysisResult)
for _, taskID := range request.GetTaskIDs() {
results[taskID] = &indexpb.AnalysisResult{
TaskID: taskID,
State: commonpb.IndexState_Retry,
FailReason: "state is retry",
}
}
return &indexpb.QueryAnalysisResultResponse{
Status: merr.Success(),
ClusterID: request.GetClusterID(),
Results: results,
}, nil
}).Once()
// 12. drop task success --> state: init
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().DropAnalysisTasks(mock.Anything, mock.Anything).Return(merr.Success(), nil).Once()
// 13. update version success, building success, assign task success --> state: InProgress
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(nil).Twice()
in.EXPECT().Analysis(mock.Anything, mock.Anything).Return(merr.Success(), nil).Once()
// 14. get task state: task not exist in node --> state: retry
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().QueryAnalysisResult(mock.Anything, mock.Anything).RunAndReturn(
func(ctx context.Context, request *indexpb.QueryAnalysisResultRequest, option ...grpc.CallOption) (*indexpb.QueryAnalysisResultResponse, error) {
return &indexpb.QueryAnalysisResultResponse{
Status: merr.Success(),
ClusterID: request.GetClusterID(),
Results: map[int64]*indexpb.AnalysisResult{},
}, nil
}).Once()
// 15. drop task success --> state: init
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().DropAnalysisTasks(mock.Anything, mock.Anything).Return(merr.Success(), nil).Once()
// 16. update version success, building success, assign task success --> state: InProgress
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(nil).Twice()
in.EXPECT().Analysis(mock.Anything, mock.Anything).Return(merr.Success(), nil).Once()
// 17. get task state: node not exist --> retry
workerManager.EXPECT().GetClientByID(mock.Anything).Return(nil, false).Once()
// 18. drop task success --> state: init
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().DropAnalysisTasks(mock.Anything, mock.Anything).Return(merr.Success(), nil).Once()
// 19. update version success, building success, assign task success --> state: InProgress
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(nil).Twice()
in.EXPECT().Analysis(mock.Anything, mock.Anything).Return(merr.Success(), nil)
// 20. get task state: Finished, save success --> state: done
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().QueryAnalysisResult(mock.Anything, mock.Anything).RunAndReturn(
func(ctx context.Context, request *indexpb.QueryAnalysisResultRequest, option ...grpc.CallOption) (*indexpb.QueryAnalysisResultResponse, error) {
results := make(map[int64]*indexpb.AnalysisResult)
for _, taskID := range request.GetTaskIDs() {
results[taskID] = &indexpb.AnalysisResult{
TaskID: taskID,
State: commonpb.IndexState_Finished,
CentroidsFile: fmt.Sprintf("%d/stats_file", taskID),
SegmentOffsetMappingFiles: map[int64]string{
1000: "1000/offset_mapping",
1001: "1001/offset_mapping",
1002: "1002/offset_mapping",
},
FailReason: "",
}
}
return &indexpb.QueryAnalysisResultResponse{
Status: merr.Success(),
ClusterID: request.GetClusterID(),
Results: results,
}, nil
}).Once()
catalog.EXPECT().SaveAnalysisTask(mock.Anything, mock.Anything).Return(nil).Once()
// 21. drop task fail --> state: done
workerManager.EXPECT().GetClientByID(mock.Anything).Return(in, true).Once()
in.EXPECT().DropAnalysisTasks(mock.Anything, mock.Anything).Return(nil, errors.New("drop task error")).Once()
//21. drop task: node not exist --> task done, remove task
workerManager.EXPECT().GetClientByID(mock.Anything).Return(nil, false).Once()
for {
scheduler.lock.RLock()
taskNum := len(scheduler.tasks)
scheduler.lock.RUnlock()
if taskNum == 0 {
break
}
time.Sleep(time.Second)
}
scheduler.Stop()
}
func Test_AnalysisScheduler(t *testing.T) {
suite.Run(t, new(AnalysisSchedulerSuite))
}

View File

@ -0,0 +1,84 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package datacoord
import (
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/proto/datapb"
)
type ClusteringCompactionJob struct {
triggerID UniqueID
collectionID UniqueID
clusteringKeyID UniqueID
clusteringKeyName string
clusteringKeyType schemapb.DataType
// ClusteringCompactionJob life cycle:
// trigger -> pipelining:
// executing:
// completed or failed or timeout
state compactionTaskState
startTime uint64
lastUpdateTime uint64
// todo: only store partial info in meta
pipeliningPlans []*datapb.CompactionPlan
executingPlans []*datapb.CompactionPlan
completedPlans []*datapb.CompactionPlan
failedPlans []*datapb.CompactionPlan
timeoutPlans []*datapb.CompactionPlan
analysisTaskID UniqueID
}
func convertClusteringCompactionJob(info *datapb.ClusteringCompactionInfo) *ClusteringCompactionJob {
job := &ClusteringCompactionJob{
triggerID: info.GetTriggerID(),
collectionID: info.GetCollectionID(),
clusteringKeyID: info.GetClusteringKeyID(),
clusteringKeyName: info.GetClusteringKeyName(),
clusteringKeyType: info.GetClusteringKeyType(),
state: compactionTaskState(info.GetState()),
startTime: info.GetStartTime(),
lastUpdateTime: info.GetLastUpdateTime(),
pipeliningPlans: info.PipeliningPlans,
executingPlans: info.ExecutingPlans,
completedPlans: info.CompletedPlans,
failedPlans: info.FailedPlans,
timeoutPlans: info.TimeoutPlans,
analysisTaskID: info.GetAnalysisTaskID(),
}
return job
}
func convertFromClusteringCompactionJob(job *ClusteringCompactionJob) *datapb.ClusteringCompactionInfo {
info := &datapb.ClusteringCompactionInfo{
TriggerID: job.triggerID,
CollectionID: job.collectionID,
ClusteringKeyID: job.clusteringKeyID,
ClusteringKeyName: job.clusteringKeyName,
ClusteringKeyType: job.clusteringKeyType,
State: datapb.CompactionTaskState(job.state),
StartTime: job.startTime,
LastUpdateTime: job.lastUpdateTime,
PipeliningPlans: job.pipeliningPlans,
ExecutingPlans: job.executingPlans,
CompletedPlans: job.completedPlans,
FailedPlans: job.failedPlans,
TimeoutPlans: job.timeoutPlans,
AnalysisTaskID: job.analysisTaskID,
}
return info
}

View File

@ -0,0 +1,483 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package datacoord
import (
"context"
"path"
"strconv"
"sync"
"time"
"github.com/samber/lo"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/logutil"
"github.com/milvus-io/milvus/pkg/util/metautil"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
type ClusteringCompactionManager struct {
ctx context.Context
meta *meta
allocator allocator
compactionHandler compactionPlanContext
scheduler Scheduler
analysisScheduler *analysisTaskScheduler
forceMu sync.Mutex
quit chan struct{}
wg sync.WaitGroup
signals chan *compactionSignal
ticker *time.Ticker
jobs map[UniqueID]*ClusteringCompactionJob
}
func newClusteringCompactionManager(
ctx context.Context,
meta *meta,
allocator allocator,
compactionHandler compactionPlanContext,
analysisScheduler *analysisTaskScheduler,
) *ClusteringCompactionManager {
return &ClusteringCompactionManager{
ctx: ctx,
meta: meta,
allocator: allocator,
compactionHandler: compactionHandler,
analysisScheduler: analysisScheduler,
}
}
func (t *ClusteringCompactionManager) start() {
t.quit = make(chan struct{})
t.ticker = time.NewTicker(Params.DataCoordCfg.ClusteringCompactionStateCheckInterval.GetAsDuration(time.Second))
t.wg.Add(1)
go t.startJobCheckLoop()
}
func (t *ClusteringCompactionManager) stop() {
close(t.quit)
t.wg.Wait()
}
func (t *ClusteringCompactionManager) submit(job *ClusteringCompactionJob) error {
log.Info("Insert clustering compaction job", zap.Int64("tiggerID", job.triggerID), zap.Int64("collectionID", job.collectionID))
t.saveJob(job)
err := t.runCompactionJob(job)
if err != nil {
job.state = failed
log.Warn("mark clustering compaction job failed", zap.Int64("tiggerID", job.triggerID), zap.Int64("collectionID", job.collectionID))
t.saveJob(job)
}
return nil
}
func (t *ClusteringCompactionManager) startJobCheckLoop() {
defer logutil.LogPanic()
defer t.wg.Done()
for {
select {
case <-t.quit:
t.ticker.Stop()
log.Info("clustering compaction loop exit")
return
case <-t.ticker.C:
err := t.checkAllJobState()
if err != nil {
log.Warn("unable to triggerClusteringCompaction", zap.Error(err))
}
}
}
}
func (t *ClusteringCompactionManager) checkAllJobState() error {
jobs := t.GetAllJobs()
for _, job := range jobs {
err := t.checkJobState(job)
if err != nil {
log.Error("fail to check job state", zap.Error(err))
job.state = failed
log.Warn("mark clustering compaction job failed", zap.Int64("tiggerID", job.triggerID), zap.Int64("collectionID", job.collectionID))
t.saveJob(job)
}
}
return nil
}
func (t *ClusteringCompactionManager) checkJobState(job *ClusteringCompactionJob) error {
if job.state == completed || job.state == failed || job.state == timeout {
if time.Since(tsoutil.PhysicalTime(job.startTime)) > Params.DataCoordCfg.ClusteringCompactionDropTolerance.GetAsDuration(time.Second) {
// skip handle this error, try best to delete meta
t.dropJob(job)
}
return nil
}
pipeliningPlans := make([]*datapb.CompactionPlan, 0)
executingPlans := make([]*datapb.CompactionPlan, 0)
completedPlans := make([]*datapb.CompactionPlan, 0)
failedPlans := make([]*datapb.CompactionPlan, 0)
timeoutPlans := make([]*datapb.CompactionPlan, 0)
checkFunc := func(plans []*datapb.CompactionPlan, lastState compactionTaskState) error {
for _, plan := range plans {
compactionTask := t.compactionHandler.getCompaction(plan.GetPlanID())
// todo: if datacoord crash during clustering compaction, compactTask will lost, we can resubmit these plan
if compactionTask == nil {
// if one compaction task is lost, mark it as failed, and the clustering compaction will be marked failed as well
log.Warn("compaction task lost", zap.Int64("planID", plan.GetPlanID()))
failedPlans = append(failedPlans, plan)
continue
}
switch compactionTask.state {
case pipelining:
pipeliningPlans = append(pipeliningPlans, plan)
case executing:
executingPlans = append(executingPlans, plan)
case failed:
failedPlans = append(failedPlans, plan)
case timeout:
timeoutPlans = append(timeoutPlans, plan)
case completed:
completedPlans = append(completedPlans, plan)
}
if lastState == executing && compactionTask.state == completed {
// new finish task, commit the partitionStats and do cleaning
collectionID := job.collectionID
partitionID := compactionTask.plan.SegmentBinlogs[0].PartitionID
vChannelName := compactionTask.plan.GetChannel()
// read the temp file and write it to formal path
tempPartitionStatsPath := path.Join(t.meta.chunkManager.RootPath(), common.PartitionStatsTempPath, metautil.JoinIDPath(collectionID, partitionID), compactionTask.plan.GetChannel(), strconv.FormatInt(compactionTask.plan.PlanID, 10))
partitionStatsPath := path.Join(t.meta.chunkManager.RootPath(), common.PartitionStatsPath, metautil.JoinIDPath(collectionID, partitionID), compactionTask.plan.GetChannel(), strconv.FormatInt(compactionTask.plan.PlanID, 10))
tempStats, err := t.meta.chunkManager.Read(t.ctx, tempPartitionStatsPath)
if err != nil {
return err
}
err = t.meta.chunkManager.Write(t.ctx, partitionStatsPath, tempStats)
if err != nil {
return err
}
// list the partition stats, normally the files should not be more than two
statsPathPrefix := path.Join(t.meta.chunkManager.RootPath(), common.PartitionStatsPath, metautil.JoinIDPath(collectionID, partitionID), vChannelName)
filePaths, _, err := t.meta.chunkManager.ListWithPrefix(t.ctx, statsPathPrefix, true)
if err != nil {
return err
}
_, maxPartitionStatsPath := storage.FindPartitionStatsMaxVersion(filePaths)
toRemovePaths := make([]string, 0)
for _, filePath := range filePaths {
// keep the newest one, still need it for search before querynode handoff
if filePath != maxPartitionStatsPath {
toRemovePaths = append(toRemovePaths, filePath)
}
}
// remove old partition stats
if len(toRemovePaths) > 0 {
err = t.meta.chunkManager.MultiRemove(t.ctx, toRemovePaths)
if err != nil {
return err
}
}
err = t.meta.chunkManager.Remove(t.ctx, tempPartitionStatsPath)
if err != nil {
return err
}
}
}
return nil
}
checkFunc(job.pipeliningPlans, pipelining)
checkFunc(job.executingPlans, executing)
checkFunc(job.completedPlans, completed)
checkFunc(job.failedPlans, failed)
checkFunc(job.timeoutPlans, timeout)
pipeliningPlans = append(pipeliningPlans, t.generateNewPlans(job)...)
job.pipeliningPlans = pipeliningPlans
job.executingPlans = executingPlans
job.completedPlans = completedPlans
job.failedPlans = failedPlans
job.timeoutPlans = timeoutPlans
if len(job.pipeliningPlans) > 0 {
err := t.runCompactionJob(job)
if err != nil {
return err
}
}
if len(job.pipeliningPlans)+len(job.executingPlans) == 0 {
if len(job.failedPlans) == 0 && len(job.timeoutPlans) == 0 {
job.state = completed
} else if len(job.failedPlans) > 0 {
job.state = failed
} else {
job.state = timeout
}
}
log.Info("Update clustering compaction job", zap.Int64("tiggerID", job.triggerID), zap.Int64("collectionID", job.collectionID), zap.String("state", datapb.CompactionTaskState(job.state).String()))
return t.saveJob(job)
}
func (t *ClusteringCompactionManager) generateNewPlans(job *ClusteringCompactionJob) []*datapb.CompactionPlan {
return nil
}
func (t *ClusteringCompactionManager) runCompactionJob(job *ClusteringCompactionJob) error {
t.forceMu.Lock()
defer t.forceMu.Unlock()
plans := job.pipeliningPlans
for _, plan := range plans {
segIDs := fetchSegIDs(plan.GetSegmentBinlogs())
start := time.Now()
planId, analysisTaskID, err := t.allocator.allocN(2)
if err != nil {
return err
}
plan.PlanID = planId
plan.TimeoutInSeconds = Params.DataCoordCfg.ClusteringCompactionTimeoutInSeconds.GetAsInt32()
// clustering compaction firstly analyze the plan, then decide whether to execute compaction
if typeutil.IsVectorType(job.clusteringKeyType) {
newAnalysisTask := &model.AnalysisTask{
CollectionID: job.collectionID,
PartitionID: plan.SegmentBinlogs[0].PartitionID,
FieldID: job.clusteringKeyID,
FieldName: job.clusteringKeyName,
FieldType: job.clusteringKeyType,
SegmentIDs: segIDs,
TaskID: analysisTaskID,
}
err = t.analysisScheduler.analysisMeta.AddAnalysisTask(newAnalysisTask)
if err != nil {
log.Warn("failed to create analysis task", zap.Int64("planID", plan.PlanID), zap.Error(err))
return err
}
t.analysisScheduler.enqueue(analysisTaskID)
log.Info("submit analysis task", zap.Int64("id", analysisTaskID))
var analysisTask *model.AnalysisTask
analysisFinished := func() bool {
analysisTask = t.analysisScheduler.analysisMeta.GetTask(analysisTaskID)
log.Debug("check analysis task state", zap.Int64("id", analysisTaskID), zap.String("state", analysisTask.State.String()))
if analysisTask.State == commonpb.IndexState_Finished ||
analysisTask.State == commonpb.IndexState_Failed {
return true
}
return false
}
for !analysisFinished() {
// respect context deadline/cancel
select {
case <-t.ctx.Done():
return nil
default:
}
time.Sleep(1 * time.Second)
}
log.Info("get analysisTask", zap.Any("analysisTask", analysisTask))
if analysisTask.State == commonpb.IndexState_Finished {
//version := int64(0) // analysisTask.Version
plan.AnalyzeResultPath = path.Join(metautil.JoinIDPath(analysisTask.TaskID, analysisTask.Version))
offSetSegmentIDs := make([]int64, 0)
for segID, _ := range analysisTask.SegmentOffsetMappingFiles {
offSetSegmentIDs = append(offSetSegmentIDs, segID)
}
plan.AnalyzeSegmentIds = offSetSegmentIDs
}
}
//shouldDo, err := t.shouldDoClusteringCompaction(analyzeResult)
//if err != nil {
// log.Warn("failed to decide whether to execute this compaction plan", zap.Int64("planID", plan.PlanID), zap.Int64s("segmentIDs", segIDs), zap.Error(err))
// continue
//}
//if !shouldDo {
// log.Info("skip execute compaction plan", zap.Int64("planID", plan.PlanID))
// continue
//}
trigger := &compactionSignal{
id: job.triggerID,
collectionID: job.collectionID,
partitionID: plan.SegmentBinlogs[0].PartitionID,
}
err = t.compactionHandler.execCompactionPlan(trigger, plan)
if err != nil {
log.Warn("failed to execute compaction plan", zap.Int64("planID", plan.GetPlanID()), zap.Int64s("segmentIDs", segIDs), zap.Error(err))
continue
}
log.Info("execute clustering compaction plan", zap.Int64("planID", plan.GetPlanID()), zap.Int64s("segmentIDs", segIDs))
segIDMap := make(map[int64][]*datapb.FieldBinlog, len(plan.SegmentBinlogs))
for _, seg := range plan.SegmentBinlogs {
segIDMap[seg.SegmentID] = seg.Deltalogs
}
log.Info("time cost of generating L2 compaction",
zap.Any("segID2DeltaLogs", segIDMap),
zap.Int64("planID", plan.PlanID),
zap.Int64("time cost", time.Since(start).Milliseconds()),
zap.Int64s("segmentIDs", segIDs))
}
if len(plans) > 0 {
job.state = executing
}
log.Info("Update clustering compaction job", zap.Int64("tiggerID", job.triggerID), zap.Int64("collectionID", job.collectionID))
err := t.saveJob(job)
return err
}
//func (t *ClusteringCompactionManager) shouldDoClusteringCompaction(analyzeResult *indexpb.AnalysisResult) (bool, error) {
// return true, nil
//}
// IsClusteringCompacting get clustering compaction info by collection id
func (t *ClusteringCompactionManager) IsClusteringCompacting(collectionID UniqueID) bool {
infos := t.meta.GetClusteringCompactionInfos(collectionID)
executingInfos := lo.Filter(infos, func(info *datapb.ClusteringCompactionInfo, _ int) bool {
return info.State == datapb.CompactionTaskState_analyzing ||
info.State == datapb.CompactionTaskState_executing ||
info.State == datapb.CompactionTaskState_pipelining
})
return len(executingInfos) > 0
}
func (t *ClusteringCompactionManager) setSegmentsCompacting(plan *datapb.CompactionPlan, compacting bool) {
for _, segmentBinlogs := range plan.GetSegmentBinlogs() {
t.meta.SetSegmentCompacting(segmentBinlogs.GetSegmentID(), compacting)
}
}
func (t *ClusteringCompactionManager) fillClusteringCompactionPlans(segments []*SegmentInfo, clusteringKeyId int64, compactTime *compactTime) []*datapb.CompactionPlan {
plan := segmentsToPlan(segments, datapb.CompactionType_ClusteringCompaction, compactTime)
plan.ClusteringKeyId = clusteringKeyId
clusteringMaxSegmentSize := paramtable.Get().DataCoordCfg.ClusteringCompactionMaxSegmentSize.GetAsSize()
clusteringPreferSegmentSize := paramtable.Get().DataCoordCfg.ClusteringCompactionPreferSegmentSize.GetAsSize()
segmentMaxSize := paramtable.Get().DataCoordCfg.SegmentMaxSize.GetAsInt64() * 1024 * 1024
plan.MaxSegmentRows = segments[0].MaxRowNum * clusteringMaxSegmentSize / segmentMaxSize
plan.PreferSegmentRows = segments[0].MaxRowNum * clusteringPreferSegmentSize / segmentMaxSize
return []*datapb.CompactionPlan{
plan,
}
}
// GetAllJobs returns cloned ClusteringCompactionJob from local meta cache
func (t *ClusteringCompactionManager) GetAllJobs() []*ClusteringCompactionJob {
jobs := make([]*ClusteringCompactionJob, 0)
infos := t.meta.GetClonedClusteringCompactionInfos()
for _, info := range infos {
job := convertClusteringCompactionJob(info)
jobs = append(jobs, job)
}
return jobs
}
// dropJob drop clustering compaction job in meta
func (t *ClusteringCompactionManager) dropJob(job *ClusteringCompactionJob) error {
info := convertFromClusteringCompactionJob(job)
return t.meta.DropClusteringCompactionInfo(info)
}
func (t *ClusteringCompactionManager) saveJob(job *ClusteringCompactionJob) error {
info := convertFromClusteringCompactionJob(job)
return t.meta.SaveClusteringCompactionInfo(info)
}
func triggerCompactionPolicy(ctx context.Context, meta *meta, collectionID int64, partitionID int64, channel string, segments []*SegmentInfo) (bool, error) {
log := log.With(zap.Int64("collectionID", collectionID), zap.Int64("partitionID", partitionID))
partitionStatsPrefix := path.Join(meta.chunkManager.RootPath(), common.PartitionStatsPath, strconv.FormatInt(collectionID, 10), strconv.FormatInt(partitionID, 10), channel)
files, _, err := meta.chunkManager.ListWithPrefix(ctx, partitionStatsPrefix, true)
if err != nil {
log.Error("Fail to list partition stats", zap.String("prefix", partitionStatsPrefix), zap.Error(err))
return false, err
}
version, partitionStatsPath := storage.FindPartitionStatsMaxVersion(files)
if version <= 0 {
var newDataSize int64 = 0
for _, seg := range segments {
newDataSize += seg.getSegmentSize()
}
if newDataSize > Params.DataCoordCfg.ClusteringCompactionNewDataSizeThreshold.GetAsSize() {
log.Info("New data is larger than threshold, do compaction", zap.Int64("newDataSize", newDataSize))
return true, nil
}
log.Info("No partition stats and no enough new data, skip compaction")
return false, nil
}
pTime, _ := tsoutil.ParseTS(uint64(version))
if time.Since(pTime) < Params.DataCoordCfg.ClusteringCompactionMinInterval.GetAsDuration(time.Second) {
log.Debug("Too short time before last clustering compaction, skip compaction")
return false, nil
}
if time.Since(pTime) > Params.DataCoordCfg.ClusteringCompactionMaxInterval.GetAsDuration(time.Second) {
log.Debug("It is a long time after last clustering compaction, do compaction")
return true, nil
}
partitionStatsBytes, err := meta.chunkManager.Read(ctx, partitionStatsPath)
if err != nil {
log.Error("Fail to read partition stats", zap.String("path", partitionStatsPath), zap.Error(err))
return false, err
}
partitionStats, err := storage.DeserializePartitionsStatsSnapshot(partitionStatsBytes)
if err != nil {
log.Error("Fail to deserialize partition stats", zap.String("path", partitionStatsPath), zap.Error(err))
return false, err
}
log.Info("Read partition stats", zap.Int64("version", version))
var compactedSegmentSize int64 = 0
var uncompactedSegmentSize int64 = 0
for _, seg := range segments {
if _, ok := partitionStats.SegmentStats[seg.ID]; ok {
compactedSegmentSize += seg.getSegmentSize()
} else {
uncompactedSegmentSize += seg.getSegmentSize()
}
}
// ratio based
//ratio := float64(uncompactedSegmentSize) / float64(compactedSegmentSize)
//if ratio > Params.DataCoordCfg.ClusteringCompactionNewDataRatioThreshold.GetAsFloat() {
// log.Info("New data is larger than threshold, do compaction", zap.Float64("ratio", ratio))
// return true, nil
//}
//log.Info("New data is smaller than threshold, skip compaction", zap.Float64("ratio", ratio))
//return false, nil
// size based
if uncompactedSegmentSize > Params.DataCoordCfg.ClusteringCompactionNewDataSizeThreshold.GetAsSize() {
log.Info("New data is larger than threshold, do compaction", zap.Int64("newDataSize", uncompactedSegmentSize))
return true, nil
}
log.Info("New data is smaller than threshold, skip compaction", zap.Int64("newDataSize", uncompactedSegmentSize))
return false, nil
}

View File

@ -75,17 +75,6 @@ var (
errChannelInBuffer = errors.New("channel is in buffer")
)
type CompactionMeta interface {
SelectSegments(selector SegmentInfoSelector) []*SegmentInfo
GetHealthySegment(segID UniqueID) *SegmentInfo
UpdateSegmentsInfo(operators ...UpdateOperator) error
SetSegmentCompacting(segmentID int64, compacting bool)
CompleteCompactionMutation(plan *datapb.CompactionPlan, result *datapb.CompactionPlanResult) ([]*SegmentInfo, *segMetricMutation, error)
}
var _ CompactionMeta = (*meta)(nil)
type compactionTask struct {
triggerInfo *compactionSignal
plan *datapb.CompactionPlan
@ -283,7 +272,7 @@ func (c *compactionPlanHandler) enqueuePlan(signal *compactionSignal, plan *data
return err
}
log := log.With(zap.Int64("planID", plan.GetPlanID()), zap.Int64("nodeID", nodeID))
log := log.With(zap.Int64("planID", plan.GetPlanID()), zap.Int64("nodeID", nodeID), zap.Int64("collectionID", signal.collectionID), zap.String("type", plan.GetType().String()))
c.setSegmentsCompacting(plan, true)
_, span := otel.Tracer(typeutil.DataCoordRole).Start(context.Background(), fmt.Sprintf("Compaction-%s", plan.GetType()))
@ -442,6 +431,11 @@ func (c *compactionPlanHandler) completeCompaction(result *datapb.CompactionPlan
if err := c.handleL0CompactionResult(plan, result); err != nil {
return err
}
case datapb.CompactionType_ClusteringCompaction:
// todo we may need to create a bew handleMajorCompactionResult method if the logic differs a lot
if err := c.handleMergeCompactionResult(plan, result); err != nil {
return err
}
default:
return errors.New("unknown compaction type")
}
@ -471,48 +465,59 @@ func (c *compactionPlanHandler) handleL0CompactionResult(plan *datapb.Compaction
}
func (c *compactionPlanHandler) handleMergeCompactionResult(plan *datapb.CompactionPlan, result *datapb.CompactionPlanResult) error {
log := log.With(zap.Int64("planID", plan.GetPlanID()))
if len(result.GetSegments()) == 0 || len(result.GetSegments()) > 1 {
// should never happen
log.Warn("illegal compaction results")
return fmt.Errorf("Illegal compaction results: %v", result)
log := log.With(zap.Int64("planID", plan.GetPlanID()), zap.String("type", plan.GetType().String()))
if plan.GetType() == datapb.CompactionType_ClusteringCompaction {
if len(result.GetSegments()) == 0 {
// should never happen
log.Warn("illegal compaction results")
return fmt.Errorf("Illegal compaction results: %v", result)
}
} else {
if len(result.GetSegments()) == 0 || len(result.GetSegments()) > 1 {
// should never happen
log.Warn("illegal compaction results")
return fmt.Errorf("Illegal compaction results: %v", result)
}
}
// Merge compaction has one and only one segment
newSegmentInfo := c.meta.GetHealthySegment(result.GetSegments()[0].SegmentID)
var newSegmentInfos []*SegmentInfo
if newSegmentInfo != nil {
log.Info("meta has already been changed, skip meta change and retry sync segments")
} else {
// Also prepare metric updates.
newSegments, metricMutation, err := c.meta.CompleteCompactionMutation(plan, result)
newSegmentInfos = newSegments
if err != nil {
return err
}
// Apply metrics after successful meta update.
metricMutation.commit()
newSegmentInfo = newSegments[0]
}
nodeID := c.plans[plan.GetPlanID()].dataNodeID
req := &datapb.SyncSegmentsRequest{
PlanID: plan.PlanID,
CompactedTo: newSegmentInfo.GetID(),
CompactedFrom: newSegmentInfo.GetCompactionFrom(),
NumOfRows: newSegmentInfo.GetNumOfRows(),
StatsLogs: newSegmentInfo.GetStatslogs(),
ChannelName: plan.GetChannel(),
PartitionId: newSegmentInfo.GetPartitionID(),
CollectionId: newSegmentInfo.GetCollectionID(),
for _, newSegment := range newSegmentInfos {
nodeID := c.plans[plan.GetPlanID()].dataNodeID
req := &datapb.SyncSegmentsRequest{
PlanID: plan.PlanID,
CompactedTo: newSegment.ID,
CompactedFrom: newSegment.GetCompactionFrom(),
NumOfRows: newSegment.GetNumOfRows(),
StatsLogs: newSegment.GetStatslogs(),
ChannelName: plan.GetChannel(),
PartitionId: newSegment.GetPartitionID(),
CollectionId: newSegment.GetCollectionID(),
}
log.Info("handleCompactionResult: syncing segments with node",
zap.Int64("nodeID", nodeID), zap.Int64("id", newSegment.ID))
if err := c.sessions.SyncSegments(nodeID, req); err != nil {
log.Warn("handleCompactionResult: fail to sync segments with node",
zap.Int64("nodeID", nodeID), zap.Error(err))
return err
}
}
log.Info("handleCompactionResult: syncing segments with node", zap.Int64("nodeID", nodeID))
if err := c.sessions.SyncSegments(nodeID, req); err != nil {
log.Warn("handleCompactionResult: fail to sync segments with node",
zap.Int64("nodeID", nodeID), zap.Error(err))
return err
}
log.Info("handleCompactionResult: success to handle merge compaction result")
log.Info("handleCompactionResult: success to handle compaction result")
return nil
}

View File

@ -32,6 +32,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/util/clustering"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
"github.com/milvus-io/milvus/pkg/util/logutil"
@ -53,14 +54,15 @@ type trigger interface {
triggerCompaction() error
// triggerSingleCompaction triggers a compaction bundled with collection-partition-channel-segment
triggerSingleCompaction(collectionID, partitionID, segmentID int64, channel string, blockToSendSignal bool) error
// forceTriggerCompaction force to start a compaction
forceTriggerCompaction(collectionID int64) (UniqueID, error)
// triggerManualCompaction force to start a compaction
triggerManualCompaction(collectionID int64, clusteringCompaction bool) (UniqueID, error)
}
type compactionSignal struct {
id UniqueID
isForce bool
isGlobal bool
isClustering bool
collectionID UniqueID
partitionID UniqueID
channel string
@ -71,6 +73,7 @@ type compactionSignal struct {
var _ trigger = (*compactionTrigger)(nil)
type compactionTrigger struct {
ctx context.Context
handler Handler
meta *meta
allocator allocator
@ -81,6 +84,9 @@ type compactionTrigger struct {
quit chan struct{}
wg sync.WaitGroup
clusteringCompactionTicker *time.Ticker
clusteringCompactionManager *ClusteringCompactionManager
indexEngineVersionManager IndexEngineVersionManager
estimateNonDiskSegmentPolicy calUpperLimitPolicy
@ -91,13 +97,16 @@ type compactionTrigger struct {
}
func newCompactionTrigger(
ctx context.Context,
meta *meta,
compactionHandler compactionPlanContext,
allocator allocator,
handler Handler,
indexVersionManager IndexEngineVersionManager,
clusteringCompactionManager *ClusteringCompactionManager,
) *compactionTrigger {
return &compactionTrigger{
ctx: ctx,
meta: meta,
allocator: allocator,
signals: make(chan *compactionSignal, 100),
@ -106,13 +115,15 @@ func newCompactionTrigger(
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
handler: handler,
clusteringCompactionManager: clusteringCompactionManager,
}
}
func (t *compactionTrigger) start() {
t.quit = make(chan struct{})
t.globalTrigger = time.NewTicker(Params.DataCoordCfg.GlobalCompactionInterval.GetAsDuration(time.Second))
t.wg.Add(2)
t.clusteringCompactionTicker = time.NewTicker(Params.DataCoordCfg.ClusteringCompactionInterval.GetAsDuration(time.Second))
t.wg.Add(3)
go func() {
defer logutil.LogPanic()
defer t.wg.Done()
@ -124,6 +135,11 @@ func (t *compactionTrigger) start() {
return
case signal := <-t.signals:
switch {
case signal.isClustering:
err := t.handleClusteringCompactionSignal(signal)
if err != nil {
log.Warn("unable to handleClusteringCompactionSignal", zap.Error(err))
}
case signal.isGlobal:
// ManualCompaction also use use handleGlobalSignal
// so throw err here
@ -142,6 +158,7 @@ func (t *compactionTrigger) start() {
}()
go t.startGlobalCompactionLoop()
go t.startClusteringCompactionLoop()
}
func (t *compactionTrigger) startGlobalCompactionLoop() {
@ -168,7 +185,30 @@ func (t *compactionTrigger) startGlobalCompactionLoop() {
}
}
func (t *compactionTrigger) startClusteringCompactionLoop() {
defer logutil.LogPanic()
defer t.wg.Done()
t.clusteringCompactionManager.start()
for {
select {
case <-t.quit:
t.clusteringCompactionTicker.Stop()
log.Info("clustering compaction loop exit")
return
case <-t.clusteringCompactionTicker.C:
err := t.triggerClusteringCompaction()
if err != nil {
log.Warn("unable to triggerClusteringCompaction", zap.Error(err))
}
}
}
}
func (t *compactionTrigger) stop() {
if t.clusteringCompactionManager != nil {
t.clusteringCompactionManager.stop()
}
close(t.quit)
t.wg.Wait()
}
@ -218,7 +258,7 @@ func (t *compactionTrigger) isChannelCheckpointHealthy(vchanName string) bool {
return time.Since(cpTime) < paramtable.Get().DataCoordCfg.ChannelCheckpointMaxLag.GetAsDuration(time.Second)
}
func (t *compactionTrigger) getCompactTime(ts Timestamp, coll *collectionInfo) (*compactTime, error) {
func getCompactTime(ts Timestamp, coll *collectionInfo) (*compactTime, error) {
collectionTTL, err := getCollectionTTL(coll.Properties)
if err != nil {
return nil, err
@ -251,7 +291,35 @@ func (t *compactionTrigger) triggerCompaction() error {
return nil
}
// triggerSingleCompaction triger a compaction bundled with collection-partition-channel-segment
// triggerClusteringCompaction trigger clustering compaction.
func (t *compactionTrigger) triggerClusteringCompaction() error {
if Params.DataCoordCfg.ClusteringCompactionEnable.GetAsBool() &&
Params.DataCoordCfg.ClusteringCompactionAutoEnable.GetAsBool() {
collections := t.meta.GetCollections()
isStart, _, err := t.allocator.allocN(int64(len(collections)))
if err != nil {
return err
}
id := isStart
for _, collection := range collections {
clusteringKeyField := clustering.GetClusteringKeyField(collection.Schema)
if clusteringKeyField != nil {
signal := &compactionSignal{
id: id,
isForce: false,
isGlobal: true,
isClustering: true,
collectionID: collection.ID,
}
t.signals <- signal
id++
}
}
}
return nil
}
// triggerSingleCompaction trigger a compaction bundled with collection-partition-channel-segment
func (t *compactionTrigger) triggerSingleCompaction(collectionID, partitionID, segmentID int64, channel string, blockToSendSignal bool) error {
// If AutoCompaction disabled, flush request will not trigger compaction
if !Params.DataCoordCfg.EnableAutoCompaction.GetAsBool() {
@ -284,9 +352,9 @@ func (t *compactionTrigger) triggerSingleCompaction(collectionID, partitionID, s
return nil
}
// forceTriggerCompaction force to start a compaction
// triggerManualCompaction force to start a compaction
// invoked by user `ManualCompaction` operation
func (t *compactionTrigger) forceTriggerCompaction(collectionID int64) (UniqueID, error) {
func (t *compactionTrigger) triggerManualCompaction(collectionID int64, clusteringCompaction bool) (UniqueID, error) {
id, err := t.allocSignalID()
if err != nil {
return -1, err
@ -295,12 +363,17 @@ func (t *compactionTrigger) forceTriggerCompaction(collectionID int64) (UniqueID
id: id,
isForce: true,
isGlobal: true,
isClustering: clusteringCompaction,
collectionID: collectionID,
}
err = t.handleGlobalSignal(signal)
if clusteringCompaction {
err = t.handleClusteringCompactionSignal(signal)
} else {
err = t.handleGlobalSignal(signal)
}
if err != nil {
log.Warn("unable to handleGlobalSignal", zap.Error(err))
log.Warn("unable to handle compaction signal", zap.Error(err))
return -1, err
}
@ -411,7 +484,8 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
isFlush(segment) &&
!segment.isCompacting && // not compacting now
!segment.GetIsImporting() && // not importing now
segment.GetLevel() != datapb.SegmentLevel_L0 // ignore level zero segments
segment.GetLevel() != datapb.SegmentLevel_L0 && // ignore level zero segments
segment.GetLevel() != datapb.SegmentLevel_L2 // ignore l2 segment
}) // m is list of chanPartSegments, which is channel-partition organized segments
if len(m) == 0 {
@ -470,7 +544,7 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
return nil
}
ct, err := t.getCompactTime(ts, coll)
ct, err := getCompactTime(ts, coll)
if err != nil {
log.Warn("get compact time failed, skip to handle compaction",
zap.Int64("collectionID", group.collectionID),
@ -519,6 +593,123 @@ func (t *compactionTrigger) handleGlobalSignal(signal *compactionSignal) error {
return nil
}
func (t *compactionTrigger) handleClusteringCompactionSignal(signal *compactionSignal) error {
t.forceMu.Lock()
defer t.forceMu.Unlock()
if !Params.DataCoordCfg.ClusteringCompactionEnable.GetAsBool() {
err := merr.WrapErrClusteringCompactionClusterNotSupport()
log.Warn(err.Error())
return err
}
log := log.With(zap.Int64("compactionID", signal.id), zap.Int64("collectionID", signal.collectionID))
coll, err := t.getCollection(signal.collectionID)
if err != nil {
log.Warn("get collection info failed, skip handling compaction", zap.Error(err))
return err
}
clusteringKeyField := clustering.GetClusteringKeyField(coll.Schema)
if clusteringKeyField == nil {
err := merr.WrapErrClusteringCompactionCollectionNotSupport(fmt.Sprint(signal.collectionID))
log.Debug(err.Error())
return err
}
compacting := t.clusteringCompactionManager.IsClusteringCompacting(coll.ID)
if compacting {
err := merr.WrapErrClusteringCompactionCollectionIsCompacting(fmt.Sprint(signal.collectionID))
log.Debug(err.Error())
// only return error if it is a manual compaction
if signal.isForce {
return err
}
return nil
}
partSegments := t.meta.GetSegmentsChanPart(func(segment *SegmentInfo) bool {
return (signal.collectionID == 0 || segment.CollectionID == signal.collectionID) &&
isSegmentHealthy(segment) &&
isFlush(segment) &&
!segment.isCompacting && // not compacting now
!segment.GetIsImporting() && // not importing now
segment.GetLevel() != datapb.SegmentLevel_L0 // ignore level zero segments
}) // partSegments is list of chanPartSegments, which is channel-partition organized segments
if len(partSegments) == 0 {
log.Info("the length of SegmentsChanPart is 0, skip to handle compaction")
return nil
}
ts, err := t.allocTs()
if err != nil {
log.Warn("allocate ts failed, skip to handle compaction")
return err
}
clusteringCompactionJob := &ClusteringCompactionJob{
triggerID: signal.id,
collectionID: signal.collectionID,
clusteringKeyID: clusteringKeyField.FieldID,
clusteringKeyName: clusteringKeyField.Name,
clusteringKeyType: clusteringKeyField.DataType,
startTime: ts,
state: pipelining,
pipeliningPlans: make([]*datapb.CompactionPlan, 0),
}
for _, group := range partSegments {
log := log.With(zap.Int64("collectionID", group.collectionID),
zap.Int64("partitionID", group.partitionID),
zap.String("channel", group.channelName))
//if Params.DataCoordCfg.IndexBasedCompaction.GetAsBool() {
// group.segments = FilterInIndexedSegments(t.handler, t.meta, group.segments...)
//}
ct, err := getCompactTime(ts, coll)
if err != nil {
log.Warn("get compact time failed, skip to handle compaction")
return err
}
if len(group.segments) == 0 {
log.Info("the length of SegmentsChanPart is 0, skip to handle compaction")
continue
}
if !signal.isForce {
execute, err := triggerCompactionPolicy(t.ctx, t.meta, group.collectionID, group.partitionID, group.channelName, group.segments)
if err != nil {
log.Warn("failed to trigger clustering compaction", zap.Error(err))
continue
}
if !execute {
continue
}
}
plans := t.clusteringCompactionManager.fillClusteringCompactionPlans(group.segments, clusteringKeyField.FieldID, ct)
// mark all segments prepare for clustering compaction
// todo for now, no need to set compacting = false, as they will be set after compaction done or failed
// however, if we split clustering compaction into multi sub compaction task and support retry fail sub task,
// we need to manage compacting state correctly
t.setSegmentsCompacting(plans, true)
clusteringCompactionJob.pipeliningPlans = append(clusteringCompactionJob.pipeliningPlans, plans...)
}
if len(clusteringCompactionJob.pipeliningPlans) > 0 {
t.clusteringCompactionManager.submit(clusteringCompactionJob)
}
return nil
}
func (t *compactionTrigger) setSegmentsCompacting(plans []*datapb.CompactionPlan, compacting bool) {
for _, plan := range plans {
for _, segmentBinlogs := range plan.GetSegmentBinlogs() {
t.meta.SetSegmentCompacting(segmentBinlogs.GetSegmentID(), compacting)
}
}
}
// handleSignal processes segment flush caused partition-chan level compaction signal
func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
t.forceMu.Lock()
@ -582,7 +773,7 @@ func (t *compactionTrigger) handleSignal(signal *compactionSignal) {
return
}
ct, err := t.getCompactTime(ts, coll)
ct, err := getCompactTime(ts, coll)
if err != nil {
log.Warn("get compact time failed, skip to handle compaction", zap.Int64("collectionID", segment.GetCollectionID()),
zap.Int64("partitionID", partitionID), zap.String("channel", channel))
@ -689,7 +880,7 @@ func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, force bool, i
}
}
// since this is priority compaction, we will execute even if there is only segment
plan := segmentsToPlan(bucket, compactTime)
plan := segmentsToPlan(bucket, datapb.CompactionType_MixCompaction, compactTime)
var size int64
var row int64
for _, s := range bucket {
@ -734,7 +925,7 @@ func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, force bool, i
// only merge if candidate number is large than MinSegmentToMerge or if target row is large enough
if len(bucket) >= Params.DataCoordCfg.MinSegmentToMerge.GetAsInt() ||
len(bucket) > 1 && t.isCompactableSegment(targetRow, segment) {
plan := segmentsToPlan(bucket, compactTime)
plan := segmentsToPlan(bucket, datapb.CompactionType_MixCompaction, compactTime)
log.Info("generate a plan for small candidates",
zap.Int64s("plan segmentIDs", lo.Map(bucket, getSegmentIDs)),
zap.Int64("target segment row", targetRow),
@ -789,7 +980,7 @@ func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, force bool, i
}
}
if len(bucket) > 1 {
plan := segmentsToPlan(bucket, compactTime)
plan := segmentsToPlan(bucket, datapb.CompactionType_MixCompaction, compactTime)
plans = append(plans, plan)
log.Info("generate a plan for to squeeze small candidates into non-planned segment",
zap.Int64s("plan segmentIDs", lo.Map(bucket, getSegmentIDs)),
@ -800,9 +991,9 @@ func (t *compactionTrigger) generatePlans(segments []*SegmentInfo, force bool, i
return plans
}
func segmentsToPlan(segments []*SegmentInfo, compactTime *compactTime) *datapb.CompactionPlan {
func segmentsToPlan(segments []*SegmentInfo, compactionType datapb.CompactionType, compactTime *compactTime) *datapb.CompactionPlan {
plan := &datapb.CompactionPlan{
Type: datapb.CompactionType_MixCompaction,
Type: compactionType,
Channel: segments[0].GetInsertChannel(),
CollectionTtl: compactTime.collectionTTL.Nanoseconds(),
}

View File

@ -37,6 +37,7 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
)
@ -485,7 +486,7 @@ func Test_compactionTrigger_force(t *testing.T) {
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
}
_, err := tr.forceTriggerCompaction(tt.collectionID)
_, err := tr.triggerManualCompaction(tt.collectionID, false)
assert.Equal(t, tt.wantErr, err != nil)
spy := (tt.fields.compactionHandler).(*spyCompactionHandler)
plan := <-spy.spyChan
@ -510,7 +511,7 @@ func Test_compactionTrigger_force(t *testing.T) {
testingOnly: true,
}
tt.collectionID = 1000
_, err := tr.forceTriggerCompaction(tt.collectionID)
_, err := tr.triggerManualCompaction(tt.collectionID, false)
assert.Equal(t, tt.wantErr, err != nil)
// expect max row num = 2048*1024*1024/(128*4) = 4194304
assert.EqualValues(t, 4194304, tt.fields.meta.segments.GetSegments()[0].MaxRowNum)
@ -827,7 +828,7 @@ func Test_compactionTrigger_force_maxSegmentLimit(t *testing.T) {
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
}
_, err := tr.forceTriggerCompaction(tt.args.collectionID)
_, err := tr.triggerManualCompaction(tt.args.collectionID, false)
assert.Equal(t, tt.wantErr, err != nil)
spy := (tt.fields.compactionHandler).(*spyCompactionHandler)
@ -1759,10 +1760,10 @@ func Test_compactionTrigger_noplan_random_size(t *testing.T) {
// Test shouldDoSingleCompaction
func Test_compactionTrigger_shouldDoSingleCompaction(t *testing.T) {
indexMeta := newSegmentIndexMeta(nil)
trigger := newCompactionTrigger(&meta{
trigger := newCompactionTrigger(context.TODO(), &meta{
indexMeta: indexMeta,
channelCPs: newChannelCps(),
}, &compactionPlanHandler{}, newMockAllocator(), newMockHandler(), newIndexEngineVersionManager())
}, &compactionPlanHandler{}, newMockAllocator(), newMockHandler(), newIndexEngineVersionManager(), nil)
// Test too many deltalogs.
var binlogs []*datapb.FieldBinlog
@ -2009,7 +2010,7 @@ func Test_compactionTrigger_new(t *testing.T) {
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
got := newCompactionTrigger(tt.args.meta, tt.args.compactionHandler, tt.args.allocator, newMockHandler(), newMockVersionManager())
got := newCompactionTrigger(context.TODO(), tt.args.meta, tt.args.compactionHandler, tt.args.allocator, newMockHandler(), newMockVersionManager(), nil)
assert.Equal(t, tt.args.meta, got.meta)
assert.Equal(t, tt.args.compactionHandler, got.compactionHandler)
assert.Equal(t, tt.args.allocator, got.allocator)
@ -2018,44 +2019,18 @@ func Test_compactionTrigger_new(t *testing.T) {
}
func Test_compactionTrigger_allocTs(t *testing.T) {
got := newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{scheduler: NewCompactionScheduler()}, newMockAllocator(), newMockHandler(), newMockVersionManager())
got := newCompactionTrigger(context.TODO(), &meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{scheduler: NewCompactionScheduler()}, newMockAllocator(), newMockHandler(), newMockVersionManager(), nil)
ts, err := got.allocTs()
assert.NoError(t, err)
assert.True(t, ts > 0)
got = newCompactionTrigger(&meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{scheduler: NewCompactionScheduler()}, &FailsAllocator{}, newMockHandler(), newMockVersionManager())
got = newCompactionTrigger(context.TODO(), &meta{segments: NewSegmentsInfo()}, &compactionPlanHandler{scheduler: NewCompactionScheduler()}, &FailsAllocator{}, newMockHandler(), newMockVersionManager(), nil)
ts, err = got.allocTs()
assert.Error(t, err)
assert.Equal(t, uint64(0), ts)
}
func Test_compactionTrigger_getCompactTime(t *testing.T) {
collections := map[UniqueID]*collectionInfo{
1: {
ID: 1,
Schema: newTestSchema(),
Partitions: []UniqueID{1},
Properties: map[string]string{
common.CollectionTTLConfigKey: "10",
},
},
2: {
ID: 2,
Schema: newTestSchema(),
Partitions: []UniqueID{1},
Properties: map[string]string{
common.CollectionTTLConfigKey: "error",
},
},
}
m := &meta{segments: NewSegmentsInfo(), collections: collections}
got := newCompactionTrigger(m, &compactionPlanHandler{scheduler: NewCompactionScheduler()}, newMockAllocator(),
&ServerHandler{
&Server{
meta: m,
},
}, newMockVersionManager())
coll := &collectionInfo{
ID: 1,
Schema: newTestSchema(),
@ -2065,7 +2040,7 @@ func Test_compactionTrigger_getCompactTime(t *testing.T) {
},
}
now := tsoutil.GetCurrentTime()
ct, err := got.getCompactTime(now, coll)
ct, err := getCompactTime(now, coll)
assert.NoError(t, err)
assert.NotNil(t, ct)
}
@ -2080,12 +2055,12 @@ func Test_triggerSingleCompaction(t *testing.T) {
channelCPs: newChannelCps(),
segments: NewSegmentsInfo(), collections: make(map[UniqueID]*collectionInfo),
}
got := newCompactionTrigger(m, &compactionPlanHandler{}, newMockAllocator(),
got := newCompactionTrigger(context.TODO(), m, &compactionPlanHandler{}, newMockAllocator(),
&ServerHandler{
&Server{
meta: m,
},
}, newMockVersionManager())
}, newMockVersionManager(), nil)
got.signals = make(chan *compactionSignal, 1)
{
err := got.triggerSingleCompaction(1, 1, 1, "a", false)
@ -2299,11 +2274,13 @@ func (s *CompactionTriggerSuite) SetupTest() {
s.handler = NewNMockHandler(s.T())
s.versionManager = NewMockVersionManager(s.T())
s.tr = newCompactionTrigger(
context.TODO(),
s.meta,
s.compactionHandler,
s.allocator,
s.handler,
s.versionManager,
nil,
)
s.tr.testingOnly = true
}
@ -2903,6 +2880,51 @@ func Test_compactionTrigger_updateSegmentMaxSize(t *testing.T) {
}
}
func Test_compactionTrigger_clustering(t *testing.T) {
paramtable.Init()
catalog := mocks.NewDataCoordCatalog(t)
catalog.EXPECT().AlterSegments(mock.Anything, mock.Anything).Return(nil).Maybe()
vecFieldID := int64(201)
meta := &meta{
catalog: catalog,
collections: map[int64]*collectionInfo{
1: {
ID: 1,
Schema: &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{
{
FieldID: vecFieldID,
DataType: schemapb.DataType_FloatVector,
TypeParams: []*commonpb.KeyValuePair{
{
Key: common.DimKey,
Value: "128",
},
},
},
},
},
},
},
}
paramtable.Get().Save(paramtable.Get().DataCoordCfg.ClusteringCompactionEnable.Key, "false")
tr := &compactionTrigger{
handler: newMockHandlerWithMeta(meta),
allocator: &MockAllocator0{},
estimateDiskSegmentPolicy: calBySchemaPolicyWithDiskIndex,
estimateNonDiskSegmentPolicy: calBySchemaPolicy,
testingOnly: true,
}
_, err := tr.triggerManualCompaction(1, true)
assert.Error(t, err)
assert.True(t, errors.Is(err, merr.ErrClusteringCompactionClusterNotSupport))
paramtable.Get().Save(paramtable.Get().DataCoordCfg.ClusteringCompactionEnable.Key, "true")
_, err2 := tr.triggerManualCompaction(1, true)
assert.Error(t, err2)
assert.True(t, errors.Is(err2, merr.ErrClusteringCompactionCollectionNotSupport))
}
func TestCompactionTriggerSuite(t *testing.T) {
suite.Run(t, new(CompactionTriggerSuite))
}

View File

@ -38,29 +38,29 @@ import (
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
type indexTaskState int32
type taskState int32
const (
// when we receive a index task
indexTaskInit indexTaskState = iota
taskInit taskState = iota
// we've sent index task to scheduler, and wait for building index.
indexTaskInProgress
taskInProgress
// task done, wait to be cleaned
indexTaskDone
taskDone
// index task need to retry.
indexTaskRetry
taskRetry
reqTimeoutInterval = time.Second * 10
)
var TaskStateNames = map[indexTaskState]string{
var TaskStateNames = map[taskState]string{
0: "Init",
1: "InProgress",
2: "Done",
3: "Retry",
}
func (x indexTaskState) String() string {
func (x taskState) String() string {
ret, ok := TaskStateNames[x]
if !ok {
return "None"
@ -77,13 +77,13 @@ type indexBuilder struct {
scheduleDuration time.Duration
// TODO @xiaocai2333: use priority queue
tasks map[int64]indexTaskState
tasks map[int64]taskState
notifyChan chan struct{}
meta *meta
policy buildIndexPolicy
nodeManager *IndexNodeManager
nodeManager WorkerManager
chunkManager storage.ChunkManager
indexEngineVersionManager IndexEngineVersionManager
handler Handler
@ -91,7 +91,7 @@ type indexBuilder struct {
func newIndexBuilder(
ctx context.Context,
metaTable *meta, nodeManager *IndexNodeManager,
metaTable *meta, nodeManager WorkerManager,
chunkManager storage.ChunkManager,
indexEngineVersionManager IndexEngineVersionManager,
handler Handler,
@ -102,7 +102,7 @@ func newIndexBuilder(
ctx: ctx,
cancel: cancel,
meta: metaTable,
tasks: make(map[int64]indexTaskState),
tasks: make(map[int64]taskState),
notifyChan: make(chan struct{}, 1),
scheduleDuration: Params.DataCoordCfg.IndexTaskSchedulerInterval.GetAsDuration(time.Millisecond),
policy: defaultBuildIndexPolicy,
@ -133,9 +133,9 @@ func (ib *indexBuilder) reloadFromKV() {
continue
}
if segIndex.IndexState == commonpb.IndexState_Unissued {
ib.tasks[segIndex.BuildID] = indexTaskInit
ib.tasks[segIndex.BuildID] = taskInit
} else if segIndex.IndexState == commonpb.IndexState_InProgress {
ib.tasks[segIndex.BuildID] = indexTaskInProgress
ib.tasks[segIndex.BuildID] = taskInProgress
}
}
}
@ -155,7 +155,7 @@ func (ib *indexBuilder) enqueue(buildID UniqueID) {
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
if _, ok := ib.tasks[buildID]; !ok {
ib.tasks[buildID] = indexTaskInit
ib.tasks[buildID] = taskInit
}
log.Info("indexBuilder enqueue task", zap.Int64("buildID", buildID))
}
@ -223,7 +223,7 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
state := ib.tasks[buildID]
ib.taskMutex.RUnlock()
updateStateFunc := func(buildID UniqueID, state indexTaskState) {
updateStateFunc := func(buildID UniqueID, state taskState) {
ib.taskMutex.Lock()
defer ib.taskMutex.Unlock()
ib.tasks[buildID] = state
@ -243,7 +243,7 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
}
switch state {
case indexTaskInit:
case taskInit:
segment := ib.meta.GetSegment(meta.SegmentID)
if !isSegmentHealthy(segment) || !ib.meta.indexMeta.IsIndexExist(meta.CollectionID, meta.IndexID) {
log.Ctx(ib.ctx).Info("task is no need to build index, remove it", zap.Int64("buildID", buildID))
@ -269,22 +269,9 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
log.Ctx(ib.ctx).Warn("IndexCoord update index state fail", zap.Int64("buildID", buildID), zap.Error(err))
return false
}
updateStateFunc(buildID, indexTaskDone)
updateStateFunc(buildID, taskDone)
return true
}
// peek client
// if all IndexNodes are executing task, wait for one of them to finish the task.
nodeID, client := ib.nodeManager.PeekClient(meta)
if client == nil {
log.Ctx(ib.ctx).WithRateGroup("dc.indexBuilder", 1, 60).RatedInfo(5, "index builder peek client error, there is no available")
return false
}
// update version and set nodeID
if err := ib.meta.indexMeta.UpdateVersion(buildID, nodeID); err != nil {
log.Ctx(ib.ctx).Warn("index builder update index version failed", zap.Int64("build", buildID), zap.Error(err))
return false
}
// vector index build needs information of optional scalar fields data
optionalFields := make([]*indexpb.OptionalFieldInfo, 0)
if Params.CommonCfg.EnableMaterializedView.GetAsBool() && isOptionalScalarFieldSupported(indexType) {
@ -338,8 +325,8 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
var err error
indexParams, err = indexparams.UpdateDiskIndexBuildParams(Params, indexParams)
if err != nil {
log.Ctx(ib.ctx).Warn("failed to append index build params", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
log.Ctx(ib.ctx).Warn("failed to append index build params", zap.Int64("buildID", buildID), zap.Error(err))
return false
}
}
var req *indexpb.CreateJobRequest
@ -419,35 +406,48 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
}
}
if err := ib.assignTask(client, req); err != nil {
// need to release lock then reassign, so set task state to retry
log.Ctx(ib.ctx).Warn("index builder assign task to IndexNode failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
updateStateFunc(buildID, indexTaskRetry)
// update version and set nodeID
if err := ib.meta.indexMeta.UpdateVersion(buildID); err != nil {
log.Ctx(ib.ctx).Warn("index builder update index version failed", zap.Int64("build", buildID), zap.Error(err))
return false
}
log.Ctx(ib.ctx).Info("index task assigned successfully", zap.Int64("buildID", buildID),
zap.Int64("segmentID", meta.SegmentID), zap.Int64("nodeID", nodeID))
// update index meta state to InProgress
if err := ib.meta.indexMeta.BuildIndex(buildID); err != nil {
// need to release lock then reassign, so set task state to retry
log.Ctx(ib.ctx).Warn("index builder update index meta to InProgress failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
updateStateFunc(buildID, indexTaskRetry)
return false
}
updateStateFunc(buildID, indexTaskInProgress)
case indexTaskDone:
assignTaskFunc := func(nodeID int64, client types.IndexNodeClient) error {
// update index meta state to InProgress
if err := ib.meta.indexMeta.BuildIndex(buildID, nodeID); err != nil {
// need to release lock then reassign, so set task state to retry
log.Ctx(ib.ctx).Warn("index builder update index meta to InProgress failed", zap.Int64("buildID", buildID), zap.Error(err))
return err
}
if err := ib.assignTask(client, req); err != nil {
// need to release lock then reassign, so set task state to retry
log.Ctx(ib.ctx).Warn("index builder assign task to IndexNode failed", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID), zap.Error(err))
return err
}
log.Ctx(ib.ctx).Info("index task assigned successfully", zap.Int64("buildID", buildID),
zap.Int64("segmentID", meta.SegmentID), zap.Int64("nodeID", nodeID))
return nil
}
if err := ib.nodeManager.SelectNodeAndAssignTask(assignTaskFunc); err != nil {
updateStateFunc(buildID, taskRetry)
return false
}
updateStateFunc(buildID, taskInProgress)
case taskDone:
if !ib.dropIndexTask(buildID, meta.NodeID) {
return true
}
deleteFunc(buildID)
case indexTaskRetry:
case taskRetry:
if !ib.dropIndexTask(buildID, meta.NodeID) {
return true
}
updateStateFunc(buildID, indexTaskInit)
updateStateFunc(buildID, taskInit)
default:
// state: in_progress
@ -456,7 +456,7 @@ func (ib *indexBuilder) process(buildID UniqueID) bool {
return true
}
func (ib *indexBuilder) getTaskState(buildID, nodeID UniqueID) indexTaskState {
func (ib *indexBuilder) getTaskState(buildID, nodeID UniqueID) taskState {
client, exist := ib.nodeManager.GetClientByID(nodeID)
if exist {
ctx1, cancel := context.WithTimeout(ib.ctx, reqTimeoutInterval)
@ -468,12 +468,12 @@ func (ib *indexBuilder) getTaskState(buildID, nodeID UniqueID) indexTaskState {
if err != nil {
log.Ctx(ib.ctx).Warn("IndexCoord get jobs info from IndexNode fail", zap.Int64("nodeID", nodeID),
zap.Error(err))
return indexTaskRetry
return taskRetry
}
if response.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
log.Ctx(ib.ctx).Warn("IndexCoord get jobs info from IndexNode fail", zap.Int64("nodeID", nodeID),
zap.Int64("buildID", buildID), zap.String("fail reason", response.GetStatus().GetReason()))
return indexTaskRetry
return taskRetry
}
// indexInfos length is always one.
@ -485,24 +485,24 @@ func (ib *indexBuilder) getTaskState(buildID, nodeID UniqueID) indexTaskState {
if err := ib.meta.indexMeta.FinishTask(info); err != nil {
log.Ctx(ib.ctx).Warn("IndexCoord update index state fail", zap.Int64("buildID", info.GetBuildID()),
zap.String("index state", info.GetState().String()), zap.Error(err))
return indexTaskInProgress
return taskInProgress
}
return indexTaskDone
return taskDone
} else if info.GetState() == commonpb.IndexState_Retry || info.GetState() == commonpb.IndexState_IndexStateNone {
log.Ctx(ib.ctx).Info("this task should be retry", zap.Int64("buildID", buildID), zap.String("fail reason", info.GetFailReason()))
return indexTaskRetry
return taskRetry
}
return indexTaskInProgress
return taskInProgress
}
}
log.Ctx(ib.ctx).Info("this task should be retry, indexNode does not have this task", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
return indexTaskRetry
return taskRetry
}
// !exist --> node down
log.Ctx(ib.ctx).Info("this task should be retry, indexNode is no longer exist", zap.Int64("buildID", buildID),
zap.Int64("nodeID", nodeID))
return indexTaskRetry
return taskRetry
}
func (ib *indexBuilder) dropIndexTask(buildID, nodeID UniqueID) bool {
@ -559,8 +559,8 @@ func (ib *indexBuilder) nodeDown(nodeID UniqueID) {
defer ib.taskMutex.Unlock()
for _, meta := range metas {
if ib.tasks[meta.BuildID] != indexTaskDone {
ib.tasks[meta.BuildID] = indexTaskRetry
if ib.tasks[meta.BuildID] != taskDone {
ib.tasks[meta.BuildID] = taskRetry
}
}
}

View File

@ -35,7 +35,6 @@ import (
"github.com/milvus-io/milvus/internal/proto/datapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
mclient "github.com/milvus-io/milvus/internal/util/mock"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
"github.com/milvus-io/milvus/pkg/util/merr"
@ -678,13 +677,13 @@ func TestIndexBuilder(t *testing.T) {
ib := newIndexBuilder(ctx, mt, nodeManager, chunkManager, newIndexEngineVersionManager(), nil)
assert.Equal(t, 6, len(ib.tasks))
assert.Equal(t, indexTaskInit, ib.tasks[buildID])
assert.Equal(t, indexTaskInProgress, ib.tasks[buildID+1])
assert.Equal(t, taskInit, ib.tasks[buildID])
assert.Equal(t, taskInProgress, ib.tasks[buildID+1])
// buildID+2 will be filter by isDeleted
assert.Equal(t, indexTaskInit, ib.tasks[buildID+3])
assert.Equal(t, indexTaskInProgress, ib.tasks[buildID+8])
assert.Equal(t, indexTaskInit, ib.tasks[buildID+9])
assert.Equal(t, indexTaskInit, ib.tasks[buildID+10])
assert.Equal(t, taskInit, ib.tasks[buildID+3])
assert.Equal(t, taskInProgress, ib.tasks[buildID+8])
assert.Equal(t, taskInit, ib.tasks[buildID+9])
assert.Equal(t, taskInit, ib.tasks[buildID+10])
ib.scheduleDuration = time.Millisecond * 500
ib.Start()
@ -721,6 +720,7 @@ func TestIndexBuilder(t *testing.T) {
break
}
ib.taskMutex.RUnlock()
time.Sleep(time.Second)
}
ib.Stop()
}
@ -743,8 +743,8 @@ func TestIndexBuilder_Error(t *testing.T) {
chunkManager.EXPECT().RootPath().Return("root")
ib := &indexBuilder{
ctx: context.Background(),
tasks: map[int64]indexTaskState{
buildID: indexTaskInit,
tasks: map[int64]taskState{
buildID: taskInit,
},
meta: createMetaTable(ec),
chunkManager: chunkManager,
@ -752,7 +752,7 @@ func TestIndexBuilder_Error(t *testing.T) {
}
t.Run("meta not exist", func(t *testing.T) {
ib.tasks[buildID+100] = indexTaskInit
ib.tasks[buildID+100] = taskInit
ib.process(buildID + 100)
_, ok := ib.tasks[buildID+100]
@ -760,40 +760,36 @@ func TestIndexBuilder_Error(t *testing.T) {
})
t.Run("finish few rows task fail", func(t *testing.T) {
ib.tasks[buildID+9] = indexTaskInit
ib.tasks[buildID+9] = taskInit
ib.process(buildID + 9)
state, ok := ib.tasks[buildID+9]
assert.True(t, ok)
assert.Equal(t, indexTaskInit, state)
assert.Equal(t, taskInit, state)
})
t.Run("peek client fail", func(t *testing.T) {
ib.tasks[buildID] = indexTaskInit
ib.tasks[buildID] = taskInit
ib.nodeManager = &IndexNodeManager{nodeClients: map[UniqueID]types.IndexNodeClient{}}
ib.process(buildID)
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskInit, state)
assert.Equal(t, taskInit, state)
})
t.Run("update version fail", func(t *testing.T) {
ib.nodeManager = &IndexNodeManager{
ctx: context.Background(),
nodeClients: map[UniqueID]types.IndexNodeClient{1: &mclient.GrpcIndexNodeClient{Err: nil}},
}
ib.process(buildID)
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskInit, state)
assert.Equal(t, taskInit, state)
})
t.Run("no need to build index but update catalog failed", func(t *testing.T) {
ib.meta.catalog = ec
ib.meta.indexMeta.indexes[collID][indexID].IsDeleted = true
ib.tasks[buildID] = indexTaskInit
ib.tasks[buildID] = taskInit
ok := ib.process(buildID)
assert.False(t, ok)
@ -806,7 +802,7 @@ func TestIndexBuilder_Error(t *testing.T) {
ib.meta.catalog = sc
ib.meta.indexMeta.indexes[collID][indexID].IsDeleted = true
ib.tasks[buildID] = indexTaskInit
ib.tasks[buildID] = taskInit
ib.process(buildID)
_, ok := ib.tasks[buildID]
@ -816,7 +812,7 @@ func TestIndexBuilder_Error(t *testing.T) {
t.Run("assign task error", func(t *testing.T) {
paramtable.Get().Save(Params.CommonCfg.StorageType.Key, "local")
ib.tasks[buildID] = indexTaskInit
ib.tasks[buildID] = taskInit
ib.meta.indexMeta.catalog = sc
ib.meta.catalog = sc
@ -837,7 +833,7 @@ func TestIndexBuilder_Error(t *testing.T) {
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskRetry, state)
assert.Equal(t, taskRetry, state)
})
t.Run("assign task fail", func(t *testing.T) {
paramtable.Get().Save(Params.CommonCfg.StorageType.Key, "local")
@ -860,12 +856,12 @@ func TestIndexBuilder_Error(t *testing.T) {
1: ic,
},
}
ib.tasks[buildID] = indexTaskInit
ib.tasks[buildID] = taskInit
ib.process(buildID)
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskRetry, state)
assert.Equal(t, taskRetry, state)
})
t.Run("drop job error", func(t *testing.T) {
@ -882,19 +878,19 @@ func TestIndexBuilder_Error(t *testing.T) {
nodeID: ic,
},
}
ib.tasks[buildID] = indexTaskDone
ib.tasks[buildID] = taskDone
ib.process(buildID)
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskDone, state)
assert.Equal(t, taskDone, state)
ib.tasks[buildID] = indexTaskRetry
ib.tasks[buildID] = taskRetry
ib.process(buildID)
state, ok = ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskRetry, state)
assert.Equal(t, taskRetry, state)
})
t.Run("drop job fail", func(t *testing.T) {
@ -912,19 +908,19 @@ func TestIndexBuilder_Error(t *testing.T) {
nodeID: ic,
},
}
ib.tasks[buildID] = indexTaskDone
ib.tasks[buildID] = taskDone
ib.process(buildID)
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskDone, state)
assert.Equal(t, taskDone, state)
ib.tasks[buildID] = indexTaskRetry
ib.tasks[buildID] = taskRetry
ib.process(buildID)
state, ok = ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskRetry, state)
assert.Equal(t, taskRetry, state)
})
t.Run("get state error", func(t *testing.T) {
@ -941,12 +937,12 @@ func TestIndexBuilder_Error(t *testing.T) {
},
}
ib.tasks[buildID] = indexTaskInProgress
ib.tasks[buildID] = taskInProgress
ib.process(buildID)
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskRetry, state)
assert.Equal(t, taskRetry, state)
})
t.Run("get state fail", func(t *testing.T) {
@ -968,12 +964,12 @@ func TestIndexBuilder_Error(t *testing.T) {
},
}
ib.tasks[buildID] = indexTaskInProgress
ib.tasks[buildID] = taskInProgress
ib.process(buildID)
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskRetry, state)
assert.Equal(t, taskRetry, state)
})
t.Run("finish task fail", func(t *testing.T) {
@ -1002,12 +998,12 @@ func TestIndexBuilder_Error(t *testing.T) {
},
}
ib.tasks[buildID] = indexTaskInProgress
ib.tasks[buildID] = taskInProgress
ib.process(buildID)
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskInProgress, state)
assert.Equal(t, taskInProgress, state)
})
t.Run("task still in progress", func(t *testing.T) {
@ -1034,12 +1030,12 @@ func TestIndexBuilder_Error(t *testing.T) {
},
}
ib.tasks[buildID] = indexTaskInProgress
ib.tasks[buildID] = taskInProgress
ib.process(buildID)
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskInProgress, state)
assert.Equal(t, taskInProgress, state)
})
t.Run("indexNode has no task", func(t *testing.T) {
@ -1057,12 +1053,12 @@ func TestIndexBuilder_Error(t *testing.T) {
},
}
ib.tasks[buildID] = indexTaskInProgress
ib.tasks[buildID] = taskInProgress
ib.process(buildID)
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskRetry, state)
assert.Equal(t, taskRetry, state)
})
t.Run("node not exist", func(t *testing.T) {
@ -1073,12 +1069,12 @@ func TestIndexBuilder_Error(t *testing.T) {
nodeClients: map[UniqueID]types.IndexNodeClient{},
}
ib.tasks[buildID] = indexTaskInProgress
ib.tasks[buildID] = taskInProgress
ib.process(buildID)
state, ok := ib.tasks[buildID]
assert.True(t, ok)
assert.Equal(t, indexTaskRetry, state)
assert.Equal(t, taskRetry, state)
})
}
@ -1169,13 +1165,13 @@ func TestIndexBuilderV2(t *testing.T) {
ib := newIndexBuilder(ctx, mt, nodeManager, chunkManager, newIndexEngineVersionManager(), handler)
assert.Equal(t, 6, len(ib.tasks))
assert.Equal(t, indexTaskInit, ib.tasks[buildID])
assert.Equal(t, indexTaskInProgress, ib.tasks[buildID+1])
assert.Equal(t, taskInit, ib.tasks[buildID])
assert.Equal(t, taskInProgress, ib.tasks[buildID+1])
// buildID+2 will be filter by isDeleted
assert.Equal(t, indexTaskInit, ib.tasks[buildID+3])
assert.Equal(t, indexTaskInProgress, ib.tasks[buildID+8])
assert.Equal(t, indexTaskInit, ib.tasks[buildID+9])
assert.Equal(t, indexTaskInit, ib.tasks[buildID+10])
assert.Equal(t, taskInit, ib.tasks[buildID+3])
assert.Equal(t, taskInProgress, ib.tasks[buildID+8])
assert.Equal(t, taskInit, ib.tasks[buildID+9])
assert.Equal(t, taskInit, ib.tasks[buildID+10])
ib.scheduleDuration = time.Millisecond * 500
ib.Start()
@ -1424,7 +1420,7 @@ func TestVecIndexWithOptionalScalarField(t *testing.T) {
return merr.Success(), nil
}).Once()
assert.Equal(t, 1, len(ib.tasks))
assert.Equal(t, indexTaskInit, ib.tasks[buildID])
assert.Equal(t, taskInit, ib.tasks[buildID])
ib.scheduleDuration = time.Millisecond * 500
ib.Start()

View File

@ -651,18 +651,17 @@ func (m *indexMeta) IsIndexExist(collID, indexID UniqueID) bool {
}
// UpdateVersion updates the version and nodeID of the index meta, whenever the task is built once, the version will be updated once.
func (m *indexMeta) UpdateVersion(buildID UniqueID, nodeID UniqueID) error {
func (m *indexMeta) UpdateVersion(buildID UniqueID) error {
m.Lock()
defer m.Unlock()
log.Debug("IndexCoord metaTable UpdateVersion receive", zap.Int64("buildID", buildID), zap.Int64("nodeID", nodeID))
log.Debug("IndexCoord metaTable UpdateVersion receive", zap.Int64("buildID", buildID))
segIdx, ok := m.buildID2SegmentIndex[buildID]
if !ok {
return fmt.Errorf("there is no index with buildID: %d", buildID)
}
updateFunc := func(segIdx *model.SegmentIndex) error {
segIdx.NodeID = nodeID
segIdx.IndexVersion++
return m.alterSegmentIndexes([]*model.SegmentIndex{segIdx})
}
@ -726,7 +725,7 @@ func (m *indexMeta) DeleteTask(buildID int64) error {
}
// BuildIndex set the index state to be InProgress. It means IndexNode is building the index.
func (m *indexMeta) BuildIndex(buildID UniqueID) error {
func (m *indexMeta) BuildIndex(buildID, nodeID UniqueID) error {
m.Lock()
defer m.Unlock()
@ -736,6 +735,7 @@ func (m *indexMeta) BuildIndex(buildID UniqueID) error {
}
updateFunc := func(segIdx *model.SegmentIndex) error {
segIdx.NodeID = nodeID
segIdx.IndexState = commonpb.IndexState_InProgress
err := m.alterSegmentIndexes([]*model.SegmentIndex{segIdx})

View File

@ -1080,18 +1080,18 @@ func TestMeta_UpdateVersion(t *testing.T) {
).Return(errors.New("fail"))
t.Run("success", func(t *testing.T) {
err := m.UpdateVersion(buildID, nodeID)
err := m.UpdateVersion(buildID)
assert.NoError(t, err)
})
t.Run("fail", func(t *testing.T) {
m.catalog = ec
err := m.UpdateVersion(buildID, nodeID)
err := m.UpdateVersion(buildID)
assert.Error(t, err)
})
t.Run("not exist", func(t *testing.T) {
err := m.UpdateVersion(buildID+1, nodeID)
err := m.UpdateVersion(buildID + 1)
assert.Error(t, err)
})
}
@ -1148,18 +1148,18 @@ func TestMeta_BuildIndex(t *testing.T) {
).Return(errors.New("fail"))
t.Run("success", func(t *testing.T) {
err := m.BuildIndex(buildID)
err := m.BuildIndex(buildID, nodeID)
assert.NoError(t, err)
})
t.Run("fail", func(t *testing.T) {
m.catalog = ec
err := m.BuildIndex(buildID)
err := m.BuildIndex(buildID, nodeID)
assert.Error(t, err)
})
t.Run("not exist", func(t *testing.T) {
err := m.BuildIndex(buildID + 1)
err := m.BuildIndex(buildID+1, nodeID)
assert.Error(t, err)
})
}

View File

@ -185,6 +185,47 @@ func (s *Server) CreateIndex(ctx context.Context, req *indexpb.CreateIndexReques
}
metrics.IndexRequestCounter.WithLabelValues(metrics.TotalLabel).Inc()
/* just for test analysis task
taskID, err := s.allocator.allocID(ctx)
if err != nil {
log.Error("analysis failed", zap.Error(err))
return merr.Status(err), nil
}
segments := s.meta.SelectSegments(func(info *SegmentInfo) bool {
return isFlush(info) && info.GetCollectionID() == req.GetCollectionID()
})
segmentIDs := lo.Map(segments, func(t *SegmentInfo, i int) int64 {
return t.ID
})
task := &model.AnalysisTask{
TenantID: "",
CollectionID: req.GetCollectionID(),
PartitionID: segments[0].GetPartitionID(),
FieldID: req.GetFieldID(),
FieldName: "",
FieldType: schemapb.DataType_FloatVector,
SegmentIDs: segmentIDs,
TaskID: taskID,
Version: 0,
NodeID: 0,
State: commonpb.IndexState_Unissued,
FailReason: "",
CentroidsFile: "",
SegmentOffsetMappingFiles: nil,
Dim: 128,
}
err = s.analysisMeta.AddAnalysisTask(task)
if err != nil {
log.Error("analysis task add failed", zap.Error(err))
return merr.Status(err), nil
}
s.analysisScheduler.enqueue(taskID)
return merr.Success(), nil
*/
if req.GetIndexName() == "" {
indexes := s.meta.indexMeta.GetFieldIndexes(req.GetCollectionID(), req.GetFieldID(), req.GetIndexName())
if len(indexes) == 0 {

View File

@ -18,19 +18,29 @@ package datacoord
import (
"context"
"fmt"
"sync"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
)
type WorkerManager interface {
AddNode(nodeID UniqueID, address string) error
RemoveNode(nodeID UniqueID)
StoppingNode(nodeID UniqueID)
SelectNodeAndAssignTask(assignTask func(selectNodeID int64, client types.IndexNodeClient) error) error
ClientSupportDisk() bool
GetAllClients() map[UniqueID]types.IndexNodeClient
GetClientByID(nodeID UniqueID) (types.IndexNodeClient, bool)
}
// IndexNodeManager is used to manage the client of IndexNode.
type IndexNodeManager struct {
nodeClients map[UniqueID]types.IndexNodeClient
@ -96,60 +106,56 @@ func (nm *IndexNodeManager) AddNode(nodeID UniqueID, address string) error {
return nil
}
// PeekClient peeks the client with the least load.
func (nm *IndexNodeManager) PeekClient(meta *model.SegmentIndex) (UniqueID, types.IndexNodeClient) {
allClients := nm.GetAllClients()
if len(allClients) == 0 {
log.Error("there is no IndexNode online")
return -1, nil
}
func (nm *IndexNodeManager) SelectNodeAndAssignTask(assignTask func(selectNodeID int64, client types.IndexNodeClient) error) error {
nm.lock.Lock()
defer nm.lock.Unlock()
// Note: In order to quickly end other goroutines, an error is returned when the client is successfully selected
ctx, cancel := context.WithCancel(nm.ctx)
var (
peekNodeID = UniqueID(0)
nodeMutex = sync.Mutex{}
wg = sync.WaitGroup{}
selectNodeID = UniqueID(0)
nodeMutex = sync.Mutex{}
wg = sync.WaitGroup{}
)
for nodeID, client := range allClients {
nodeID := nodeID
client := client
wg.Add(1)
go func() {
defer wg.Done()
resp, err := client.GetJobStats(ctx, &indexpb.GetJobStatsRequest{})
if err != nil {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID), zap.Error(err))
return
}
if resp.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID),
zap.String("reason", resp.GetStatus().GetReason()))
return
}
if resp.GetTaskSlots() > 0 {
nodeMutex.Lock()
defer nodeMutex.Unlock()
log.Info("peek client success", zap.Int64("nodeID", nodeID))
if peekNodeID == 0 {
peekNodeID = nodeID
for nodeID, client := range nm.nodeClients {
if _, ok := nm.stoppingNodes[nodeID]; !ok {
nodeID := nodeID
client := client
wg.Add(1)
go func() {
defer wg.Done()
resp, err := client.GetJobStats(ctx, &indexpb.GetJobStatsRequest{})
if err != nil {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID), zap.Error(err))
return
}
cancel()
// Note: In order to quickly end other goroutines, an error is returned when the client is successfully selected
return
}
}()
if resp.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
log.Warn("get IndexNode slots failed", zap.Int64("nodeID", nodeID),
zap.String("reason", resp.GetStatus().GetReason()))
return
}
if resp.GetTaskSlots() > 0 {
nodeMutex.Lock()
defer nodeMutex.Unlock()
if selectNodeID == 0 {
selectNodeID = nodeID
}
cancel()
// Note: In order to quickly end other goroutines, an error is returned when the client is successfully selected
return
}
}()
}
}
wg.Wait()
cancel()
if peekNodeID != 0 {
log.Info("peek client success", zap.Int64("nodeID", peekNodeID))
return peekNodeID, allClients[peekNodeID]
if selectNodeID != 0 {
log.Info("select indexNode success", zap.Int64("nodeID", selectNodeID))
return assignTask(selectNodeID, nm.nodeClients[selectNodeID])
}
log.RatedDebug(5, "peek client fail")
return 0, nil
return fmt.Errorf("there is no idle indexNode")
}
func (nm *IndexNodeManager) ClientSupportDisk() bool {

View File

@ -25,7 +25,6 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/mock"
"github.com/milvus-io/milvus/internal/metastore/model"
"github.com/milvus-io/milvus/internal/mocks"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/types"
@ -34,9 +33,6 @@ import (
func TestIndexNodeManager_AddNode(t *testing.T) {
nm := NewNodeManager(context.Background(), defaultIndexNodeCreatorFunc)
nodeID, client := nm.PeekClient(&model.SegmentIndex{})
assert.Equal(t, int64(-1), nodeID)
assert.Nil(t, client)
t.Run("success", func(t *testing.T) {
err := nm.AddNode(1, "indexnode-1")
@ -49,7 +45,7 @@ func TestIndexNodeManager_AddNode(t *testing.T) {
})
}
func TestIndexNodeManager_PeekClient(t *testing.T) {
func TestIndexNodeManager_SelectNodeAndAssignTask(t *testing.T) {
getMockedGetJobStatsClient := func(resp *indexpb.GetJobStatsResponse, err error) types.IndexNodeClient {
ic := mocks.NewMockIndexNodeClient(t)
ic.EXPECT().GetJobStats(mock.Anything, mock.Anything, mock.Anything).Return(resp, err)
@ -94,9 +90,19 @@ func TestIndexNodeManager_PeekClient(t *testing.T) {
},
}
nodeID, client := nm.PeekClient(&model.SegmentIndex{})
assert.NotNil(t, client)
assert.Contains(t, []UniqueID{8, 9}, nodeID)
err := nm.SelectNodeAndAssignTask(func(selectNodeID int64, client types.IndexNodeClient) error {
assert.NotNil(t, client)
assert.Contains(t, []UniqueID{8, 9}, selectNodeID)
return nil
})
assert.NoError(t, err)
err = nm.SelectNodeAndAssignTask(func(selectNodeID int64, client types.IndexNodeClient) error {
assert.NotNil(t, client)
assert.Contains(t, []UniqueID{8, 9}, selectNodeID)
return errors.New("error")
})
assert.Error(t, err)
})
}

View File

@ -47,14 +47,29 @@ import (
"github.com/milvus-io/milvus/pkg/util/tsoutil"
)
type CompactionMeta interface {
SelectSegments(selector SegmentInfoSelector) []*SegmentInfo
GetHealthySegment(segID UniqueID) *SegmentInfo
UpdateSegmentsInfo(operators ...UpdateOperator) error
SetSegmentCompacting(segmentID int64, compacting bool)
DropClusteringCompactionInfo(info *datapb.ClusteringCompactionInfo) error
SaveClusteringCompactionInfo(info *datapb.ClusteringCompactionInfo) error
CompleteCompactionMutation(plan *datapb.CompactionPlan, result *datapb.CompactionPlanResult) ([]*SegmentInfo, *segMetricMutation, error)
}
var _ CompactionMeta = (*meta)(nil)
type meta struct {
sync.RWMutex
ctx context.Context
catalog metastore.DataCoordCatalog
collections map[UniqueID]*collectionInfo // collection id to collection info
segments *SegmentsInfo // segment id to segment info
channelCPs *channelCPs // vChannel -> channel checkpoint/see position
chunkManager storage.ChunkManager
ctx context.Context
catalog metastore.DataCoordCatalog
collections map[UniqueID]*collectionInfo // collection id to collection info
segments *SegmentsInfo // segment id to segment info
channelCPs *channelCPs // vChannel -> channel checkpoint/see position
chunkManager storage.ChunkManager
clusteringCompactions map[string]*datapb.ClusteringCompactionInfo
indexMeta *indexMeta
}
@ -94,13 +109,14 @@ func newMeta(ctx context.Context, catalog metastore.DataCoordCatalog, chunkManag
}
mt := &meta{
ctx: ctx,
catalog: catalog,
collections: make(map[UniqueID]*collectionInfo),
segments: NewSegmentsInfo(),
channelCPs: newChannelCps(),
indexMeta: indexMeta,
chunkManager: chunkManager,
ctx: ctx,
catalog: catalog,
collections: make(map[UniqueID]*collectionInfo),
segments: NewSegmentsInfo(),
channelCPs: newChannelCps(),
indexMeta: indexMeta,
chunkManager: chunkManager,
clusteringCompactions: make(map[string]*datapb.ClusteringCompactionInfo, 0),
}
err = mt.reloadFromKV()
if err != nil {
@ -156,6 +172,15 @@ func (m *meta) reloadFromKV() error {
pos.ChannelName = vChannel
m.channelCPs.checkpoints[vChannel] = pos
}
compactionInfos, err := m.catalog.ListClusteringCompactionInfos(m.ctx)
if err != nil {
return err
}
for _, info := range compactionInfos {
m.clusteringCompactions[fmt.Sprintf("%d-%d", info.CollectionID, info.TriggerID)] = info
}
log.Info("DataCoord meta reloadFromKV done", zap.Duration("duration", record.ElapseSpan()))
return nil
}
@ -182,6 +207,17 @@ func (m *meta) GetCollection(collectionID UniqueID) *collectionInfo {
return collection
}
// GetCollections returns collections from local cache
func (m *meta) GetCollections() []*collectionInfo {
m.RLock()
defer m.RUnlock()
collections := make([]*collectionInfo, 0)
for _, coll := range m.collections {
collections = append(collections, coll)
}
return collections
}
func (m *meta) GetClonedCollectionInfo(collectionID UniqueID) *collectionInfo {
m.RLock()
defer m.RUnlock()
@ -210,6 +246,7 @@ func (m *meta) GetSegmentsChanPart(selector SegmentInfoSelector) []*chanPartSegm
defer m.RUnlock()
mDimEntry := make(map[string]*chanPartSegments)
log.Debug("GetSegmentsChanPart segment number", zap.Int("length", len(m.segments.GetSegments())))
for _, segmentInfo := range m.segments.segments {
if !selector(segmentInfo) {
continue
@ -1127,6 +1164,81 @@ func (m *meta) CompleteCompactionMutation(plan *datapb.CompactionPlan, result *d
}
}
getMinPosition := func(positions []*msgpb.MsgPosition) *msgpb.MsgPosition {
var minPos *msgpb.MsgPosition
for _, pos := range positions {
if minPos == nil ||
pos != nil && pos.GetTimestamp() < minPos.GetTimestamp() {
minPos = pos
}
}
return minPos
}
if plan.GetType() == datapb.CompactionType_ClusteringCompaction {
newSegments := make([]*SegmentInfo, 0)
for _, seg := range result.GetSegments() {
segmentInfo := &datapb.SegmentInfo{
ID: seg.GetSegmentID(),
CollectionID: latestCompactFromSegments[0].CollectionID,
PartitionID: latestCompactFromSegments[0].PartitionID,
InsertChannel: plan.GetChannel(),
NumOfRows: seg.NumOfRows,
State: commonpb.SegmentState_Flushed,
MaxRowNum: latestCompactFromSegments[0].MaxRowNum,
Binlogs: seg.GetInsertLogs(),
Statslogs: seg.GetField2StatslogPaths(),
CreatedByCompaction: true,
CompactionFrom: compactFromSegIDs,
LastExpireTime: plan.GetStartTime(),
Level: datapb.SegmentLevel_L2,
StartPosition: getMinPosition(lo.Map(latestCompactFromSegments, func(info *SegmentInfo, _ int) *msgpb.MsgPosition {
return info.GetStartPosition()
})),
DmlPosition: getMinPosition(lo.Map(latestCompactFromSegments, func(info *SegmentInfo, _ int) *msgpb.MsgPosition {
return info.GetDmlPosition()
})),
}
segment := NewSegmentInfo(segmentInfo)
newSegments = append(newSegments, segment)
metricMutation.addNewSeg(segment.GetState(), segment.GetLevel(), segment.GetNumOfRows())
}
compactionTo := make([]UniqueID, 0, len(newSegments))
for _, s := range newSegments {
compactionTo = append(compactionTo, s.GetID())
}
log.Info("meta update: prepare for complete compaction mutation - complete",
zap.Int64("collectionID", latestCompactFromSegments[0].CollectionID),
zap.Int64("partitionID", latestCompactFromSegments[0].PartitionID),
zap.Any("compacted from", compactFromSegIDs),
zap.Any("compacted to", compactionTo))
compactFromInfos := lo.Map(latestCompactFromSegments, func(info *SegmentInfo, _ int) *datapb.SegmentInfo {
return info.SegmentInfo
})
newSegmentInfos := lo.Map(newSegments, func(info *SegmentInfo, _ int) *datapb.SegmentInfo {
return info.SegmentInfo
})
binlogs := make([]metastore.BinlogsIncrement, 0)
for _, seg := range newSegmentInfos {
binlogs = append(binlogs, metastore.BinlogsIncrement{Segment: seg})
}
if err := m.catalog.AlterSegments(m.ctx, append(compactFromInfos, newSegmentInfos...), binlogs...); err != nil {
log.Warn("fail to alter segments and new segment", zap.Error(err))
return nil, nil, err
}
lo.ForEach(latestCompactFromSegments, func(info *SegmentInfo, _ int) {
m.segments.SetSegment(info.GetID(), info)
})
lo.ForEach(newSegments, func(info *SegmentInfo, _ int) {
m.segments.SetSegment(info.GetID(), info)
})
return newSegments, metricMutation, nil
}
// MixCompaction / MergeCompaction will generates one and only one segment
compactToSegment := result.GetSegments()[0]
@ -1140,17 +1252,6 @@ func (m *meta) CompleteCompactionMutation(plan *datapb.CompactionPlan, result *d
compactToSegment.Deltalogs = append(compactToSegment.GetDeltalogs(), &datapb.FieldBinlog{Binlogs: newDeltalogs})
}
getMinPosition := func(positions []*msgpb.MsgPosition) *msgpb.MsgPosition {
var minPos *msgpb.MsgPosition
for _, pos := range positions {
if minPos == nil ||
pos != nil && pos.GetTimestamp() < minPos.GetTimestamp() {
minPos = pos
}
}
return minPos
}
compactToSegmentInfo := NewSegmentInfo(
&datapb.SegmentInfo{
ID: compactToSegment.GetSegmentID(),
@ -1467,3 +1568,65 @@ func updateSegStateAndPrepareMetrics(segToUpdate *SegmentInfo, targetState commo
metricMutation.append(segToUpdate.GetState(), targetState, segToUpdate.GetLevel(), segToUpdate.GetNumOfRows())
segToUpdate.State = targetState
}
// GetClonedClusteringCompactionInfos returns cloned ClusteringCompactionInfos from local cache
func (m *meta) GetClonedClusteringCompactionInfos() []*datapb.ClusteringCompactionInfo {
m.RLock()
defer m.RUnlock()
infos := make([]*datapb.ClusteringCompactionInfo, 0)
for _, info := range m.clusteringCompactions {
cloneInfo := &datapb.ClusteringCompactionInfo{
TriggerID: info.GetTriggerID(),
CollectionID: info.GetCollectionID(),
ClusteringKeyID: info.GetClusteringKeyID(),
State: info.GetState(),
StartTime: info.GetStartTime(),
LastUpdateTime: info.GetLastUpdateTime(),
PipeliningPlans: info.GetPipeliningPlans(),
ExecutingPlans: info.GetExecutingPlans(),
CompletedPlans: info.GetCompletedPlans(),
FailedPlans: info.GetFailedPlans(),
TimeoutPlans: info.GetTimeoutPlans(),
AnalysisTaskID: info.GetAnalysisTaskID(),
}
infos = append(infos, cloneInfo)
}
return infos
}
// GetClusteringCompactionInfos get clustering compaction infos by collection id
func (m *meta) GetClusteringCompactionInfos(collectionID UniqueID) []*datapb.ClusteringCompactionInfo {
m.RLock()
defer m.RUnlock()
res := make([]*datapb.ClusteringCompactionInfo, 0)
for _, info := range m.clusteringCompactions {
if info.CollectionID == collectionID {
res = append(res, info)
}
}
return res
}
// DropClusteringCompactionInfo drop clustering compaction info in meta
func (m *meta) DropClusteringCompactionInfo(info *datapb.ClusteringCompactionInfo) error {
m.Lock()
defer m.Unlock()
if err := m.catalog.DropClusteringCompactionInfo(m.ctx, info); err != nil {
log.Error("meta update: drop clustering compaction info fail", zap.Int64("collectionID", info.CollectionID), zap.Error(err))
return err
}
delete(m.clusteringCompactions, fmt.Sprintf("%d-%d", info.CollectionID, info.TriggerID))
return nil
}
// SaveClusteringCompactionInfo update collection compaction plan
func (m *meta) SaveClusteringCompactionInfo(info *datapb.ClusteringCompactionInfo) error {
m.Lock()
defer m.Unlock()
if err := m.catalog.SaveClusteringCompactionInfo(m.ctx, info); err != nil {
log.Error("meta update: update clustering compaction info fail", zap.Error(err))
return err
}
m.clusteringCompactions[fmt.Sprintf("%d-%d", info.CollectionID, info.TriggerID)] = info
return nil
}

View File

@ -84,6 +84,48 @@ func (_c *MockCompactionMeta_CompleteCompactionMutation_Call) RunAndReturn(run f
return _c
}
// DropClusteringCompactionInfo provides a mock function with given fields: info
func (_m *MockCompactionMeta) DropClusteringCompactionInfo(info *datapb.ClusteringCompactionInfo) error {
ret := _m.Called(info)
var r0 error
if rf, ok := ret.Get(0).(func(*datapb.ClusteringCompactionInfo) error); ok {
r0 = rf(info)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockCompactionMeta_DropClusteringCompactionInfo_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'DropClusteringCompactionInfo'
type MockCompactionMeta_DropClusteringCompactionInfo_Call struct {
*mock.Call
}
// DropClusteringCompactionInfo is a helper method to define mock.On call
// - info *datapb.ClusteringCompactionInfo
func (_e *MockCompactionMeta_Expecter) DropClusteringCompactionInfo(info interface{}) *MockCompactionMeta_DropClusteringCompactionInfo_Call {
return &MockCompactionMeta_DropClusteringCompactionInfo_Call{Call: _e.mock.On("DropClusteringCompactionInfo", info)}
}
func (_c *MockCompactionMeta_DropClusteringCompactionInfo_Call) Run(run func(info *datapb.ClusteringCompactionInfo)) *MockCompactionMeta_DropClusteringCompactionInfo_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(*datapb.ClusteringCompactionInfo))
})
return _c
}
func (_c *MockCompactionMeta_DropClusteringCompactionInfo_Call) Return(_a0 error) *MockCompactionMeta_DropClusteringCompactionInfo_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockCompactionMeta_DropClusteringCompactionInfo_Call) RunAndReturn(run func(*datapb.ClusteringCompactionInfo) error) *MockCompactionMeta_DropClusteringCompactionInfo_Call {
_c.Call.Return(run)
return _c
}
// GetHealthySegment provides a mock function with given fields: segID
func (_m *MockCompactionMeta) GetHealthySegment(segID int64) *SegmentInfo {
ret := _m.Called(segID)
@ -128,6 +170,48 @@ func (_c *MockCompactionMeta_GetHealthySegment_Call) RunAndReturn(run func(int64
return _c
}
// SaveClusteringCompactionInfo provides a mock function with given fields: info
func (_m *MockCompactionMeta) SaveClusteringCompactionInfo(info *datapb.ClusteringCompactionInfo) error {
ret := _m.Called(info)
var r0 error
if rf, ok := ret.Get(0).(func(*datapb.ClusteringCompactionInfo) error); ok {
r0 = rf(info)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockCompactionMeta_SaveClusteringCompactionInfo_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SaveClusteringCompactionInfo'
type MockCompactionMeta_SaveClusteringCompactionInfo_Call struct {
*mock.Call
}
// SaveClusteringCompactionInfo is a helper method to define mock.On call
// - info *datapb.ClusteringCompactionInfo
func (_e *MockCompactionMeta_Expecter) SaveClusteringCompactionInfo(info interface{}) *MockCompactionMeta_SaveClusteringCompactionInfo_Call {
return &MockCompactionMeta_SaveClusteringCompactionInfo_Call{Call: _e.mock.On("SaveClusteringCompactionInfo", info)}
}
func (_c *MockCompactionMeta_SaveClusteringCompactionInfo_Call) Run(run func(info *datapb.ClusteringCompactionInfo)) *MockCompactionMeta_SaveClusteringCompactionInfo_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(*datapb.ClusteringCompactionInfo))
})
return _c
}
func (_c *MockCompactionMeta_SaveClusteringCompactionInfo_Call) Return(_a0 error) *MockCompactionMeta_SaveClusteringCompactionInfo_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockCompactionMeta_SaveClusteringCompactionInfo_Call) RunAndReturn(run func(*datapb.ClusteringCompactionInfo) error) *MockCompactionMeta_SaveClusteringCompactionInfo_Call {
_c.Call.Return(run)
return _c
}
// SelectSegments provides a mock function with given fields: selector
func (_m *MockCompactionMeta) SelectSegments(selector SegmentInfoSelector) []*SegmentInfo {
ret := _m.Called(selector)

View File

@ -610,9 +610,9 @@ func (t *mockCompactionTrigger) triggerSingleCompaction(collectionID, partitionI
panic("not implemented")
}
// forceTriggerCompaction force to start a compaction
func (t *mockCompactionTrigger) forceTriggerCompaction(collectionID int64) (UniqueID, error) {
if f, ok := t.methods["forceTriggerCompaction"]; ok {
// triggerManualCompaction force to start a compaction
func (t *mockCompactionTrigger) triggerManualCompaction(collectionID int64, clusteringCompaction bool) (UniqueID, error) {
if f, ok := t.methods["triggerManualCompaction"]; ok {
if ff, ok := f.(func(collectionID int64) (UniqueID, error)); ok {
return ff(collectionID)
}

View File

@ -0,0 +1,324 @@
// Code generated by mockery v2.32.4. DO NOT EDIT.
package datacoord
import (
types "github.com/milvus-io/milvus/internal/types"
mock "github.com/stretchr/testify/mock"
)
// MockWorkerManager is an autogenerated mock type for the WorkerManager type
type MockWorkerManager struct {
mock.Mock
}
type MockWorkerManager_Expecter struct {
mock *mock.Mock
}
func (_m *MockWorkerManager) EXPECT() *MockWorkerManager_Expecter {
return &MockWorkerManager_Expecter{mock: &_m.Mock}
}
// AddNode provides a mock function with given fields: nodeID, address
func (_m *MockWorkerManager) AddNode(nodeID int64, address string) error {
ret := _m.Called(nodeID, address)
var r0 error
if rf, ok := ret.Get(0).(func(int64, string) error); ok {
r0 = rf(nodeID, address)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockWorkerManager_AddNode_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'AddNode'
type MockWorkerManager_AddNode_Call struct {
*mock.Call
}
// AddNode is a helper method to define mock.On call
// - nodeID int64
// - address string
func (_e *MockWorkerManager_Expecter) AddNode(nodeID interface{}, address interface{}) *MockWorkerManager_AddNode_Call {
return &MockWorkerManager_AddNode_Call{Call: _e.mock.On("AddNode", nodeID, address)}
}
func (_c *MockWorkerManager_AddNode_Call) Run(run func(nodeID int64, address string)) *MockWorkerManager_AddNode_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(int64), args[1].(string))
})
return _c
}
func (_c *MockWorkerManager_AddNode_Call) Return(_a0 error) *MockWorkerManager_AddNode_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWorkerManager_AddNode_Call) RunAndReturn(run func(int64, string) error) *MockWorkerManager_AddNode_Call {
_c.Call.Return(run)
return _c
}
// ClientSupportDisk provides a mock function with given fields:
func (_m *MockWorkerManager) ClientSupportDisk() bool {
ret := _m.Called()
var r0 bool
if rf, ok := ret.Get(0).(func() bool); ok {
r0 = rf()
} else {
r0 = ret.Get(0).(bool)
}
return r0
}
// MockWorkerManager_ClientSupportDisk_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ClientSupportDisk'
type MockWorkerManager_ClientSupportDisk_Call struct {
*mock.Call
}
// ClientSupportDisk is a helper method to define mock.On call
func (_e *MockWorkerManager_Expecter) ClientSupportDisk() *MockWorkerManager_ClientSupportDisk_Call {
return &MockWorkerManager_ClientSupportDisk_Call{Call: _e.mock.On("ClientSupportDisk")}
}
func (_c *MockWorkerManager_ClientSupportDisk_Call) Run(run func()) *MockWorkerManager_ClientSupportDisk_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockWorkerManager_ClientSupportDisk_Call) Return(_a0 bool) *MockWorkerManager_ClientSupportDisk_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWorkerManager_ClientSupportDisk_Call) RunAndReturn(run func() bool) *MockWorkerManager_ClientSupportDisk_Call {
_c.Call.Return(run)
return _c
}
// GetAllClients provides a mock function with given fields:
func (_m *MockWorkerManager) GetAllClients() map[int64]types.IndexNodeClient {
ret := _m.Called()
var r0 map[int64]types.IndexNodeClient
if rf, ok := ret.Get(0).(func() map[int64]types.IndexNodeClient); ok {
r0 = rf()
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(map[int64]types.IndexNodeClient)
}
}
return r0
}
// MockWorkerManager_GetAllClients_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetAllClients'
type MockWorkerManager_GetAllClients_Call struct {
*mock.Call
}
// GetAllClients is a helper method to define mock.On call
func (_e *MockWorkerManager_Expecter) GetAllClients() *MockWorkerManager_GetAllClients_Call {
return &MockWorkerManager_GetAllClients_Call{Call: _e.mock.On("GetAllClients")}
}
func (_c *MockWorkerManager_GetAllClients_Call) Run(run func()) *MockWorkerManager_GetAllClients_Call {
_c.Call.Run(func(args mock.Arguments) {
run()
})
return _c
}
func (_c *MockWorkerManager_GetAllClients_Call) Return(_a0 map[int64]types.IndexNodeClient) *MockWorkerManager_GetAllClients_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWorkerManager_GetAllClients_Call) RunAndReturn(run func() map[int64]types.IndexNodeClient) *MockWorkerManager_GetAllClients_Call {
_c.Call.Return(run)
return _c
}
// GetClientByID provides a mock function with given fields: nodeID
func (_m *MockWorkerManager) GetClientByID(nodeID int64) (types.IndexNodeClient, bool) {
ret := _m.Called(nodeID)
var r0 types.IndexNodeClient
var r1 bool
if rf, ok := ret.Get(0).(func(int64) (types.IndexNodeClient, bool)); ok {
return rf(nodeID)
}
if rf, ok := ret.Get(0).(func(int64) types.IndexNodeClient); ok {
r0 = rf(nodeID)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(types.IndexNodeClient)
}
}
if rf, ok := ret.Get(1).(func(int64) bool); ok {
r1 = rf(nodeID)
} else {
r1 = ret.Get(1).(bool)
}
return r0, r1
}
// MockWorkerManager_GetClientByID_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'GetClientByID'
type MockWorkerManager_GetClientByID_Call struct {
*mock.Call
}
// GetClientByID is a helper method to define mock.On call
// - nodeID int64
func (_e *MockWorkerManager_Expecter) GetClientByID(nodeID interface{}) *MockWorkerManager_GetClientByID_Call {
return &MockWorkerManager_GetClientByID_Call{Call: _e.mock.On("GetClientByID", nodeID)}
}
func (_c *MockWorkerManager_GetClientByID_Call) Run(run func(nodeID int64)) *MockWorkerManager_GetClientByID_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(int64))
})
return _c
}
func (_c *MockWorkerManager_GetClientByID_Call) Return(_a0 types.IndexNodeClient, _a1 bool) *MockWorkerManager_GetClientByID_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockWorkerManager_GetClientByID_Call) RunAndReturn(run func(int64) (types.IndexNodeClient, bool)) *MockWorkerManager_GetClientByID_Call {
_c.Call.Return(run)
return _c
}
// RemoveNode provides a mock function with given fields: nodeID
func (_m *MockWorkerManager) RemoveNode(nodeID int64) {
_m.Called(nodeID)
}
// MockWorkerManager_RemoveNode_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'RemoveNode'
type MockWorkerManager_RemoveNode_Call struct {
*mock.Call
}
// RemoveNode is a helper method to define mock.On call
// - nodeID int64
func (_e *MockWorkerManager_Expecter) RemoveNode(nodeID interface{}) *MockWorkerManager_RemoveNode_Call {
return &MockWorkerManager_RemoveNode_Call{Call: _e.mock.On("RemoveNode", nodeID)}
}
func (_c *MockWorkerManager_RemoveNode_Call) Run(run func(nodeID int64)) *MockWorkerManager_RemoveNode_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(int64))
})
return _c
}
func (_c *MockWorkerManager_RemoveNode_Call) Return() *MockWorkerManager_RemoveNode_Call {
_c.Call.Return()
return _c
}
func (_c *MockWorkerManager_RemoveNode_Call) RunAndReturn(run func(int64)) *MockWorkerManager_RemoveNode_Call {
_c.Call.Return(run)
return _c
}
// SelectNodeAndAssignTask provides a mock function with given fields: assignTask
func (_m *MockWorkerManager) SelectNodeAndAssignTask(assignTask func(int64, types.IndexNodeClient) error) error {
ret := _m.Called(assignTask)
var r0 error
if rf, ok := ret.Get(0).(func(func(int64, types.IndexNodeClient) error) error); ok {
r0 = rf(assignTask)
} else {
r0 = ret.Error(0)
}
return r0
}
// MockWorkerManager_SelectNodeAndAssignTask_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SelectNodeAndAssignTask'
type MockWorkerManager_SelectNodeAndAssignTask_Call struct {
*mock.Call
}
// SelectNodeAndAssignTask is a helper method to define mock.On call
// - assignTask func(int64 , types.IndexNodeClient) error
func (_e *MockWorkerManager_Expecter) SelectNodeAndAssignTask(assignTask interface{}) *MockWorkerManager_SelectNodeAndAssignTask_Call {
return &MockWorkerManager_SelectNodeAndAssignTask_Call{Call: _e.mock.On("SelectNodeAndAssignTask", assignTask)}
}
func (_c *MockWorkerManager_SelectNodeAndAssignTask_Call) Run(run func(assignTask func(int64, types.IndexNodeClient) error)) *MockWorkerManager_SelectNodeAndAssignTask_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(func(int64, types.IndexNodeClient) error))
})
return _c
}
func (_c *MockWorkerManager_SelectNodeAndAssignTask_Call) Return(_a0 error) *MockWorkerManager_SelectNodeAndAssignTask_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *MockWorkerManager_SelectNodeAndAssignTask_Call) RunAndReturn(run func(func(int64, types.IndexNodeClient) error) error) *MockWorkerManager_SelectNodeAndAssignTask_Call {
_c.Call.Return(run)
return _c
}
// StoppingNode provides a mock function with given fields: nodeID
func (_m *MockWorkerManager) StoppingNode(nodeID int64) {
_m.Called(nodeID)
}
// MockWorkerManager_StoppingNode_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'StoppingNode'
type MockWorkerManager_StoppingNode_Call struct {
*mock.Call
}
// StoppingNode is a helper method to define mock.On call
// - nodeID int64
func (_e *MockWorkerManager_Expecter) StoppingNode(nodeID interface{}) *MockWorkerManager_StoppingNode_Call {
return &MockWorkerManager_StoppingNode_Call{Call: _e.mock.On("StoppingNode", nodeID)}
}
func (_c *MockWorkerManager_StoppingNode_Call) Run(run func(nodeID int64)) *MockWorkerManager_StoppingNode_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(int64))
})
return _c
}
func (_c *MockWorkerManager_StoppingNode_Call) Return() *MockWorkerManager_StoppingNode_Call {
_c.Call.Return()
return _c
}
func (_c *MockWorkerManager_StoppingNode_Call) RunAndReturn(run func(int64)) *MockWorkerManager_StoppingNode_Call {
_c.Call.Return(run)
return _c
}
// NewMockWorkerManager creates a new instance of MockWorkerManager. It also registers a testing interface on the mock and a cleanup function to assert the mocks expectations.
// The first argument is typically a *testing.T value.
func NewMockWorkerManager(t interface {
mock.TestingT
Cleanup(func())
}) *MockWorkerManager {
mock := &MockWorkerManager{}
mock.Mock.Test(t)
t.Cleanup(func() { mock.AssertExpectations(t) })
return mock
}

View File

@ -126,9 +126,10 @@ type Server struct {
importScheduler ImportScheduler
importChecker ImportChecker
compactionTrigger trigger
compactionHandler compactionPlanContext
compactionViewManager *CompactionViewManager
compactionTrigger trigger
clusteringCompactionManager *ClusteringCompactionManager
compactionHandler compactionPlanContext
compactionViewManager *CompactionViewManager
metricsCacheManager *metricsinfo.MetricsCacheManager
@ -154,9 +155,12 @@ type Server struct {
// segReferManager *SegmentReferenceManager
indexBuilder *indexBuilder
indexNodeManager *IndexNodeManager
indexNodeManager WorkerManager
indexEngineVersionManager IndexEngineVersionManager
analysisMeta *analysisMeta
analysisScheduler *analysisTaskScheduler
// manage ways that data coord access other coord
broker broker.Broker
}
@ -366,6 +370,7 @@ func (s *Server) initDataCoord() error {
}
log.Info("init service discovery done")
s.initAnalysisScheduler()
if Params.DataCoordCfg.EnableCompaction.GetAsBool() {
s.createCompactionHandler()
s.createCompactionTrigger()
@ -412,6 +417,7 @@ func (s *Server) Start() error {
func (s *Server) startDataCoord() {
if Params.DataCoordCfg.EnableCompaction.GetAsBool() {
s.compactionHandler.start()
s.analysisScheduler.Start()
s.compactionTrigger.start()
s.compactionViewManager.Start()
}
@ -514,7 +520,8 @@ func (s *Server) stopCompactionHandler() {
}
func (s *Server) createCompactionTrigger() {
s.compactionTrigger = newCompactionTrigger(s.meta, s.compactionHandler, s.allocator, s.handler, s.indexEngineVersionManager)
s.clusteringCompactionManager = newClusteringCompactionManager(s.ctx, s.meta, s.allocator, s.compactionHandler, s.analysisScheduler)
s.compactionTrigger = newCompactionTrigger(s.ctx, s.meta, s.compactionHandler, s.allocator, s.handler, s.indexEngineVersionManager, s.clusteringCompactionManager)
}
func (s *Server) stopCompactionTrigger() {
@ -644,6 +651,10 @@ func (s *Server) initMeta(chunkManager storage.ChunkManager) error {
if err != nil {
return err
}
s.analysisMeta, err = newAnalysisMeta(s.ctx, catalog)
if err != nil {
return err
}
return nil
}
return retry.Do(s.ctx, reloadEtcdFn, retry.Attempts(connMetaMaxRetryTime))
@ -655,6 +666,12 @@ func (s *Server) initIndexBuilder(manager storage.ChunkManager) {
}
}
func (s *Server) initAnalysisScheduler() {
if s.analysisScheduler == nil {
s.analysisScheduler = newAnalysisTaskScheduler(s.ctx, s.meta, s.analysisMeta, s.indexNodeManager)
}
}
func (s *Server) initIndexNodeManager() {
if s.indexNodeManager == nil {
s.indexNodeManager = NewNodeManager(s.ctx, s.indexNodeCreator)

View File

@ -2472,7 +2472,7 @@ func TestManualCompaction(t *testing.T) {
svr.stateCode.Store(commonpb.StateCode_Healthy)
svr.compactionTrigger = &mockCompactionTrigger{
methods: map[string]interface{}{
"forceTriggerCompaction": func(collectionID int64) (UniqueID, error) {
"triggerManualCompaction": func(collectionID int64) (UniqueID, error) {
return 1, nil
},
},
@ -2500,7 +2500,7 @@ func TestManualCompaction(t *testing.T) {
svr.stateCode.Store(commonpb.StateCode_Healthy)
svr.compactionTrigger = &mockCompactionTrigger{
methods: map[string]interface{}{
"forceTriggerCompaction": func(collectionID int64) (UniqueID, error) {
"triggerManualCompaction": func(collectionID int64) (UniqueID, error) {
return 0, errors.New("mock error")
},
},
@ -2519,7 +2519,7 @@ func TestManualCompaction(t *testing.T) {
svr.stateCode.Store(commonpb.StateCode_Abnormal)
svr.compactionTrigger = &mockCompactionTrigger{
methods: map[string]interface{}{
"forceTriggerCompaction": func(collectionID int64) (UniqueID, error) {
"triggerManualCompaction": func(collectionID int64) (UniqueID, error) {
return 1, nil
},
},

View File

@ -1082,7 +1082,7 @@ func (s *Server) ManualCompaction(ctx context.Context, req *milvuspb.ManualCompa
return resp, nil
}
id, err := s.compactionTrigger.forceTriggerCompaction(req.CollectionID)
id, err := s.compactionTrigger.triggerManualCompaction(req.CollectionID, req.GetMajorCompaction())
if err != nil {
log.Error("failed to trigger manual compaction", zap.Error(err))
resp.Status = merr.Status(err)

File diff suppressed because it is too large Load Diff

View File

@ -143,8 +143,7 @@ func (t *compactionTask) getNumRows() (int64, error) {
return numRows, nil
}
func (t *compactionTask) mergeDeltalogs(dBlobs map[UniqueID][]*Blob) (map[interface{}]Timestamp, error) {
log := log.With(zap.Int64("planID", t.getPlanID()))
func MergeDeltalogs(dBlobs map[UniqueID][]*Blob) (map[interface{}]Timestamp, error) {
mergeStart := time.Now()
dCodec := storage.NewDeleteCodec()
@ -153,8 +152,7 @@ func (t *compactionTask) mergeDeltalogs(dBlobs map[UniqueID][]*Blob) (map[interf
for _, blobs := range dBlobs {
_, _, dData, err := dCodec.Deserialize(blobs)
if err != nil {
log.Warn("merge deltalogs wrong", zap.Error(err))
return nil, err
return nil, merr.WrapErrCompactionReadDeltaLogErr(err.Error())
}
for i := int64(0); i < dData.RowCount; i++ {
@ -484,52 +482,13 @@ func (t *compactionTask) compact() (*datapb.CompactionPlanResult, error) {
return nil, err
}
dblobs := make(map[UniqueID][]*Blob)
allPath := make([][]string, 0)
for _, s := range t.plan.GetSegmentBinlogs() {
// Get the number of field binlog files from non-empty segment
var binlogNum int
for _, b := range s.GetFieldBinlogs() {
if b != nil {
binlogNum = len(b.GetBinlogs())
break
}
}
// Unable to deal with all empty segments cases, so return error
if binlogNum == 0 {
log.Warn("compact wrong, all segments' binlogs are empty")
return nil, errIllegalCompactionPlan
}
for idx := 0; idx < binlogNum; idx++ {
var ps []string
for _, f := range s.GetFieldBinlogs() {
ps = append(ps, f.GetBinlogs()[idx].GetLogPath())
}
allPath = append(allPath, ps)
}
segID := s.GetSegmentID()
paths := make([]string, 0)
for _, d := range s.GetDeltalogs() {
for _, l := range d.GetBinlogs() {
path := l.GetLogPath()
paths = append(paths, path)
}
}
if len(paths) != 0 {
bs, err := downloadBlobs(ctxTimeout, t.binlogIO, paths)
if err != nil {
log.Warn("compact wrong, fail to download deltalogs", zap.Int64("segment", segID), zap.Strings("path", paths), zap.Error(err))
return nil, err
}
dblobs[segID] = append(dblobs[segID], bs...)
}
delatBlobs, allPath, err := loadDeltaMap(ctxTimeout, t.binlogIO, t.plan.GetSegmentBinlogs())
if err != nil {
log.Warn("compact wrong, fail to load deltalogs", zap.Error(err))
return nil, err
}
log.Info("compact download deltalogs done", zap.Duration("elapse", t.tr.RecordSpan()))
deltaPk2Ts, err := t.mergeDeltalogs(dblobs)
deltaPk2Ts, err := MergeDeltalogs(delatBlobs)
if err != nil {
log.Warn("compact wrong, fail to merge deltalogs", zap.Error(err))
return nil, err
@ -821,13 +780,52 @@ func (t *compactionTask) GetCurrentTime() typeutil.Timestamp {
}
func (t *compactionTask) isExpiredEntity(ts, now Timestamp) bool {
// entity expire is not enabled if duration <= 0
if t.plan.GetCollectionTtl() <= 0 {
return false
}
pts, _ := tsoutil.ParseTS(ts)
pnow, _ := tsoutil.ParseTS(now)
expireTime := pts.Add(time.Duration(t.plan.GetCollectionTtl()))
return expireTime.Before(pnow)
return IsExpiredEntity(t.plan.GetCollectionTtl(), ts, now)
}
func loadDeltaMap(ctx context.Context, binlogIO io.BinlogIO, segments []*datapb.CompactionSegmentBinlogs) (map[UniqueID][]*Blob, [][]string, error) {
dblobs := make(map[UniqueID][]*Blob)
allPath := make([][]string, 0)
for _, s := range segments {
// Get the number of field binlog files from non-empty segment
var binlogNum int
for _, b := range s.GetFieldBinlogs() {
if b != nil {
binlogNum = len(b.GetBinlogs())
break
}
}
// Unable to deal with all empty segments cases, so return error
if binlogNum == 0 {
log.Warn("compact wrong, all segments' binlogs are empty")
return nil, nil, errIllegalCompactionPlan
}
for idx := 0; idx < binlogNum; idx++ {
var ps []string
for _, f := range s.GetFieldBinlogs() {
ps = append(ps, f.GetBinlogs()[idx].GetLogPath())
}
allPath = append(allPath, ps)
}
segID := s.GetSegmentID()
paths := make([]string, 0)
for _, d := range s.GetDeltalogs() {
for _, l := range d.GetBinlogs() {
path := l.GetLogPath()
paths = append(paths, path)
}
}
if len(paths) != 0 {
bs, err := downloadBlobs(ctx, binlogIO, paths)
if err != nil {
log.Warn("compact wrong, fail to download deltalogs", zap.Int64("segment", segID), zap.Strings("path", paths), zap.Error(err))
return nil, nil, err
}
dblobs[segID] = append(dblobs[segID], bs...)
}
}
return dblobs, allPath, nil
}

View File

@ -0,0 +1,35 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package datanode
import (
"time"
"github.com/milvus-io/milvus/pkg/util/tsoutil"
)
func IsExpiredEntity(ttl int64, ts, now Timestamp) bool {
// entity expire is not enabled if duration <= 0
if ttl <= 0 {
return false
}
pts, _ := tsoutil.ParseTS(ts)
pnow, _ := tsoutil.ParseTS(now)
expireTime := pts.Add(time.Duration(ttl))
return expireTime.Before(pnow)
}

View File

@ -187,11 +187,8 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
}
for _, test := range tests {
task := &compactionTask{
done: make(chan struct{}, 1),
}
t.Run(test.description, func(t *testing.T) {
pk2ts, err := task.mergeDeltalogs(test.dBlobs)
pk2ts, err := MergeDeltalogs(test.dBlobs)
if test.isvalid {
assert.NoError(t, err)
assert.Equal(t, 5, len(pk2ts))
@ -264,10 +261,7 @@ func TestCompactionTaskInnerMethods(t *testing.T) {
dBlobs[test.segIDC] = d
}
task := &compactionTask{
done: make(chan struct{}, 1),
}
pk2ts, err := task.mergeDeltalogs(dBlobs)
pk2ts, err := MergeDeltalogs(dBlobs)
assert.NoError(t, err)
assert.Equal(t, test.expectedpk2ts, len(pk2ts))
})

View File

@ -1,3 +1,19 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package iterator
import (
@ -36,6 +52,10 @@ func (r *InsertRow) GetTimestamp() uint64 {
return r.Timestamp
}
func (r *InsertRow) GetValue() map[storage.FieldID]interface{} {
return r.Value
}
type DeltalogRow struct {
Pk storage.PrimaryKey
Timestamp typeutil.Timestamp
@ -66,6 +86,10 @@ func (l *LabeledRowData) GetPk() storage.PrimaryKey {
return l.data.GetPk()
}
func (l *LabeledRowData) GetData() Row {
return l.data
}
func (l *LabeledRowData) GetTimestamp() uint64 {
return l.data.GetTimestamp()
}

View File

@ -279,6 +279,17 @@ func (node *DataNode) Compaction(ctx context.Context, req *datapb.CompactionPlan
node.allocator,
req,
)
case datapb.CompactionType_ClusteringCompaction:
binlogIO := io.NewBinlogIO(node.chunkManager, getOrCreateIOPool())
task = newClusteringCompactionTask(
taskCtx,
binlogIO,
//binlogIO,
node.allocator,
ds.metacache,
node.syncMgr,
req,
)
default:
log.Warn("Unknown compaction type", zap.String("type", req.GetType().String()))
return merr.Status(merr.WrapErrParameterInvalidMsg("Unknown compaction type: %v", req.GetType().String())), nil

View File

@ -319,6 +319,26 @@ func (s *DataNodeServicesSuite) TestCompaction() {
s.NoError(err)
s.False(merr.Ok(resp))
})
s.Run("compact_clustering", func() {
node := s.node
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
req := &datapb.CompactionPlan{
PlanID: 1000,
Channel: dmChannelName,
SegmentBinlogs: []*datapb.CompactionSegmentBinlogs{
{SegmentID: 102, Level: datapb.SegmentLevel_L0},
{SegmentID: growingSegmentID, Level: datapb.SegmentLevel_L1},
},
Type: datapb.CompactionType_ClusteringCompaction,
}
resp, err := node.Compaction(ctx, req)
s.NoError(err)
s.False(merr.Ok(resp))
})
}
func (s *DataNodeServicesSuite) TestFlushSegments() {

View File

@ -163,3 +163,21 @@ func (c *Client) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequest
return client.GetMetrics(ctx, req)
})
}
func (c *Client) Analysis(ctx context.Context, req *indexpb.AnalysisRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client indexpb.IndexNodeClient) (*commonpb.Status, error) {
return client.Analysis(ctx, req)
})
}
func (c *Client) QueryAnalysisResult(ctx context.Context, req *indexpb.QueryAnalysisResultRequest, opts ...grpc.CallOption) (*indexpb.QueryAnalysisResultResponse, error) {
return wrapGrpcCall(ctx, c, func(client indexpb.IndexNodeClient) (*indexpb.QueryAnalysisResultResponse, error) {
return client.QueryAnalysisResult(ctx, req)
})
}
func (c *Client) DropAnalysisTasks(ctx context.Context, req *indexpb.DropAnalysisTasksRequest, opt ...grpc.CallOption) (*commonpb.Status, error) {
return wrapGrpcCall(ctx, c, func(client indexpb.IndexNodeClient) (*commonpb.Status, error) {
return client.DropAnalysisTasks(ctx, req)
})
}

View File

@ -164,6 +164,24 @@ func TestIndexNodeClient(t *testing.T) {
assert.NoError(t, err)
})
t.Run("Analysis", func(t *testing.T) {
req := &indexpb.AnalysisRequest{}
_, err := inc.Analysis(ctx, req)
assert.NoError(t, err)
})
t.Run("QueryAnalysisResult", func(t *testing.T) {
req := &indexpb.QueryAnalysisResultRequest{}
_, err := inc.QueryAnalysisResult(ctx, req)
assert.NoError(t, err)
})
t.Run("DropAnalysisTasks", func(t *testing.T) {
req := &indexpb.DropAnalysisTasksRequest{}
_, err := inc.DropAnalysisTasks(ctx, req)
assert.NoError(t, err)
})
err := inc.Close()
assert.NoError(t, err)
}

View File

@ -289,6 +289,18 @@ func (s *Server) GetMetrics(ctx context.Context, request *milvuspb.GetMetricsReq
return s.indexnode.GetMetrics(ctx, request)
}
func (s *Server) Analysis(ctx context.Context, request *indexpb.AnalysisRequest) (*commonpb.Status, error) {
return s.indexnode.Analysis(ctx, request)
}
func (s *Server) QueryAnalysisResult(ctx context.Context, request *indexpb.QueryAnalysisResultRequest) (*indexpb.QueryAnalysisResultResponse, error) {
return s.indexnode.QueryAnalysisResult(ctx, request)
}
func (s *Server) DropAnalysisTasks(ctx context.Context, request *indexpb.DropAnalysisTasksRequest) (*commonpb.Status, error) {
return s.indexnode.DropAnalysisTasks(ctx, request)
}
// NewServer create a new IndexNode grpc server.
func NewServer(ctx context.Context, factory dependency.Factory) (*Server, error) {
ctx1, cancel := context.WithCancel(ctx)

View File

@ -111,6 +111,27 @@ func TestIndexNodeServer(t *testing.T) {
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
})
t.Run("Analysis", func(t *testing.T) {
req := &indexpb.AnalysisRequest{}
resp, err := server.Analysis(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetErrorCode())
})
t.Run("QueryAnalysisResult", func(t *testing.T) {
req := &indexpb.QueryAnalysisResultRequest{}
resp, err := server.QueryAnalysisResult(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetStatus().GetErrorCode())
})
t.Run("DropAnalysisTasks", func(t *testing.T) {
req := &indexpb.DropAnalysisTasksRequest{}
resp, err := server.DropAnalysisTasks(ctx, req)
assert.NoError(t, err)
assert.Equal(t, commonpb.ErrorCode_Success, resp.GetErrorCode())
})
err = server.Stop()
assert.NoError(t, err)
}

View File

@ -105,9 +105,10 @@ type IndexNode struct {
etcdCli *clientv3.Client
address string
initOnce sync.Once
stateLock sync.Mutex
tasks map[taskKey]*taskInfo
initOnce sync.Once
stateLock sync.Mutex
indexTasks map[taskKey]*indexTaskInfo
analysisTasks map[taskKey]*analysisTaskInfo
}
// NewIndexNode creates a new IndexNode component.
@ -120,7 +121,8 @@ func NewIndexNode(ctx context.Context, factory dependency.Factory) *IndexNode {
loopCancel: cancel,
factory: factory,
storageFactory: NewChunkMgrFactory(),
tasks: map[taskKey]*taskInfo{},
indexTasks: make(map[taskKey]*indexTaskInfo),
analysisTasks: make(map[taskKey]*analysisTaskInfo),
lifetime: lifetime.NewLifetime(commonpb.StateCode_Abnormal),
}
sc := NewTaskScheduler(b.loopCtx)
@ -251,10 +253,16 @@ func (i *IndexNode) Stop() error {
i.lifetime.Wait()
log.Info("Index node abnormal")
// cleanup all running tasks
deletedTasks := i.deleteAllTasks()
for _, task := range deletedTasks {
if task.cancel != nil {
task.cancel()
deletedIndexTasks := i.deleteAllIndexTasks()
for _, t := range deletedIndexTasks {
if t.cancel != nil {
t.cancel()
}
}
deletedAnalysisTasks := i.deleteAllAnalysisTasks()
for _, t := range deletedAnalysisTasks {
if t.cancel != nil {
t.cancel()
}
}
if i.sched != nil {

View File

@ -76,7 +76,7 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), metrics.TotalLabel).Inc()
taskCtx, taskCancel := context.WithCancel(i.loopCtx)
if oldInfo := i.loadOrStoreTask(req.GetClusterID(), req.GetBuildID(), &taskInfo{
if oldInfo := i.loadOrStoreIndexTask(req.GetClusterID(), req.GetBuildID(), &indexTaskInfo{
cancel: taskCancel,
state: commonpb.IndexState_InProgress,
}); oldInfo != nil {
@ -91,7 +91,7 @@ func (i *IndexNode) CreateJob(ctx context.Context, req *indexpb.CreateJobRequest
zap.String("accessKey", req.GetStorageConfig().GetAccessKeyID()),
zap.Error(err),
)
i.deleteTaskInfos(ctx, []taskKey{{ClusterID: req.GetClusterID(), BuildID: req.GetBuildID()}})
i.deleteIndexTaskInfos(ctx, []taskKey{{ClusterID: req.GetClusterID(), BuildID: req.GetBuildID()}})
metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.FailLabel).Inc()
return merr.Status(err), nil
}
@ -152,10 +152,10 @@ func (i *IndexNode) QueryJobs(ctx context.Context, req *indexpb.QueryJobsRequest
}, nil
}
defer i.lifetime.Done()
infos := make(map[UniqueID]*taskInfo)
i.foreachTaskInfo(func(ClusterID string, buildID UniqueID, info *taskInfo) {
infos := make(map[UniqueID]*indexTaskInfo)
i.foreachIndexTaskInfo(func(ClusterID string, buildID UniqueID, info *indexTaskInfo) {
if ClusterID == req.GetClusterID() {
infos[buildID] = &taskInfo{
infos[buildID] = &indexTaskInfo{
state: info.state,
fileKeys: common.CloneStringList(info.fileKeys),
serializedSize: info.serializedSize,
@ -208,7 +208,7 @@ func (i *IndexNode) DropJobs(ctx context.Context, req *indexpb.DropJobsRequest)
for _, buildID := range req.GetBuildIDs() {
keys = append(keys, taskKey{ClusterID: req.GetClusterID(), BuildID: buildID})
}
infos := i.deleteTaskInfos(ctx, keys)
infos := i.deleteIndexTaskInfos(ctx, keys)
for _, info := range infos {
if info.cancel != nil {
info.cancel()
@ -229,7 +229,7 @@ func (i *IndexNode) GetJobStats(ctx context.Context, req *indexpb.GetJobStatsReq
defer i.lifetime.Done()
unissued, active := i.sched.IndexBuildQueue.GetTaskNum()
jobInfos := make([]*indexpb.JobInfo, 0)
i.foreachTaskInfo(func(ClusterID string, buildID UniqueID, info *taskInfo) {
i.foreachIndexTaskInfo(func(ClusterID string, buildID UniqueID, info *indexTaskInfo) {
if info.statistic != nil {
jobInfos = append(jobInfos, proto.Clone(info.statistic).(*indexpb.JobInfo))
}
@ -302,3 +302,110 @@ func (i *IndexNode) GetMetrics(ctx context.Context, req *milvuspb.GetMetricsRequ
Status: merr.Status(merr.WrapErrMetricNotFound(metricType)),
}, nil
}
func (i *IndexNode) Analysis(ctx context.Context, req *indexpb.AnalysisRequest) (*commonpb.Status, error) {
log := log.Ctx(ctx).With(
zap.String("clusterID", req.GetClusterID()), zap.Int64("collectionID", req.GetCollectionID()),
zap.Int64("partitionID", req.GetPartitionID()), zap.Int64("fieldID", req.GetFieldID()),
zap.Int64("taskID", req.GetTaskID()), zap.Int64("task version", req.GetVersion()),
)
if err := i.lifetime.Add(merr.IsHealthy); err != nil {
log.Warn("index node not ready",
zap.Error(err),
)
return merr.Status(err), nil
}
defer i.lifetime.Done()
log.Info("IndexNode analyzing data ...", zap.Int("segment num", len(req.GetSegmentStats())))
// Todo @xiaocai2333: add metrics for analysis task
//metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), metrics.TotalLabel).Inc()
taskCtx, taskCancel := context.WithCancel(i.loopCtx)
if oldInfo := i.loadOrStoreAnalysisTask(req.GetClusterID(), req.GetTaskID(), &analysisTaskInfo{
cancel: taskCancel,
state: commonpb.IndexState_InProgress,
}); oldInfo != nil {
err := merr.WrapErrIndexDuplicate("", "analysis task already existed")
log.Warn("duplicated analysis task", zap.Error(err))
//metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.FailLabel).Inc()
return merr.Status(err), nil
}
t := &analysisTask{
ident: fmt.Sprintf("%s/%d", req.GetClusterID(), req.GetTaskID()),
ctx: taskCtx,
cancel: taskCancel,
req: req,
node: i,
tr: timerecord.NewTimeRecorder(fmt.Sprintf("ClusterID: %s, IndexBuildID: %d", req.GetClusterID(), req.GetTaskID())),
}
ret := merr.Success()
if err := i.sched.IndexBuildQueue.Enqueue(t); err != nil {
log.Warn("IndexNode failed to schedule", zap.Error(err))
ret = merr.Status(err)
//metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10), metrics.FailLabel).Inc()
return ret, nil
}
//metrics.IndexNodeBuildIndexTaskCounter.WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), metrics.SuccessLabel).Inc()
log.Info("IndexNode analysis task enqueued successfully")
return ret, nil
}
func (i *IndexNode) QueryAnalysisResult(ctx context.Context, req *indexpb.QueryAnalysisResultRequest) (*indexpb.QueryAnalysisResultResponse, error) {
log := log.Ctx(ctx).With(
zap.String("clusterID", req.GetClusterID()), zap.Int64s("taskIDs", req.GetTaskIDs()),
).WithRateGroup("in.queryAnalysisResult", 1, 60)
if err := i.lifetime.Add(merr.IsHealthyOrStopping); err != nil {
log.Warn("IndexNode not ready", zap.Error(err))
return &indexpb.QueryAnalysisResultResponse{
Status: merr.Status(err),
}, nil
}
defer i.lifetime.Done()
results := make(map[UniqueID]*indexpb.AnalysisResult)
for _, taskID := range req.GetTaskIDs() {
info := i.getAnalysisTaskInfo(req.GetClusterID(), taskID)
if info != nil {
results[taskID] = &indexpb.AnalysisResult{
TaskID: taskID,
State: info.state,
CentroidsFile: info.centroidsFile,
SegmentOffsetMappingFiles: info.segmentsOffsetMapping,
FailReason: info.failReason,
}
}
}
ret := &indexpb.QueryAnalysisResultResponse{
Status: merr.Success(),
ClusterID: req.GetClusterID(),
Results: results,
}
log.Info("query analysis tasks stats done", zap.Any("results", results))
return ret, nil
}
func (i *IndexNode) DropAnalysisTasks(ctx context.Context, req *indexpb.DropAnalysisTasksRequest) (*commonpb.Status, error) {
log.Ctx(ctx).Info("drop analysis tasks",
zap.String("clusterID", req.GetClusterID()),
zap.Int64s("taskIDs", req.GetTaskIDs()),
)
if err := i.lifetime.Add(merr.IsHealthyOrStopping); err != nil {
log.Ctx(ctx).Warn("IndexNode not ready", zap.Error(err), zap.String("clusterID", req.ClusterID))
return merr.Status(err), nil
}
defer i.lifetime.Done()
keys := make([]taskKey, 0, len(req.GetTaskIDs()))
for _, taskID := range req.GetTaskIDs() {
keys = append(keys, taskKey{ClusterID: req.GetClusterID(), BuildID: taskID})
}
infos := i.deleteAnalysisTaskInfos(ctx, keys)
for _, info := range infos {
if info.cancel != nil {
info.cancel()
}
}
log.Ctx(ctx).Info("drop analysis tasks success", zap.String("clusterID", req.GetClusterID()),
zap.Int64s("taskIDs", req.GetTaskIDs()))
return merr.Success(), nil
}

View File

@ -20,6 +20,8 @@ import (
"context"
"testing"
"github.com/stretchr/testify/suite"
"github.com/stretchr/testify/assert"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
@ -100,3 +102,123 @@ func TestMockFieldData(t *testing.T) {
chunkMgr.mockFieldData(100000, 8, 0, 0, 1)
}
type IndexNodeServiceSuite struct {
suite.Suite
cluster string
collectionID int64
partitionID int64
taskID int64
fieldID int64
segmentID int64
}
func (suite *IndexNodeServiceSuite) SetupTest() {
suite.cluster = "test_cluster"
suite.collectionID = 100
suite.partitionID = 102
suite.taskID = 11111
suite.fieldID = 103
suite.segmentID = 104
}
func (suite *IndexNodeServiceSuite) Test_AbnormalIndexNode() {
in, err := NewMockIndexNodeComponent(context.TODO())
suite.NoError(err)
suite.Nil(in.Stop())
ctx := context.TODO()
status, err := in.CreateJob(ctx, &indexpb.CreateJobRequest{})
suite.NoError(err)
suite.ErrorIs(merr.Error(status), merr.ErrServiceNotReady)
qresp, err := in.QueryJobs(ctx, &indexpb.QueryJobsRequest{})
suite.NoError(err)
suite.ErrorIs(merr.Error(qresp.GetStatus()), merr.ErrServiceNotReady)
status, err = in.DropJobs(ctx, &indexpb.DropJobsRequest{})
suite.NoError(err)
suite.ErrorIs(merr.Error(status), merr.ErrServiceNotReady)
jobNumRsp, err := in.GetJobStats(ctx, &indexpb.GetJobStatsRequest{})
suite.NoError(err)
suite.ErrorIs(merr.Error(jobNumRsp.GetStatus()), merr.ErrServiceNotReady)
metricsResp, err := in.GetMetrics(ctx, &milvuspb.GetMetricsRequest{})
err = merr.CheckRPCCall(metricsResp, err)
suite.ErrorIs(err, merr.ErrServiceNotReady)
configurationResp, err := in.ShowConfigurations(ctx, &internalpb.ShowConfigurationsRequest{})
err = merr.CheckRPCCall(configurationResp, err)
suite.ErrorIs(err, merr.ErrServiceNotReady)
status, err = in.Analysis(ctx, &indexpb.AnalysisRequest{})
err = merr.CheckRPCCall(status, err)
suite.ErrorIs(err, merr.ErrServiceNotReady)
queryAnalysisResultResp, err := in.QueryAnalysisResult(ctx, &indexpb.QueryAnalysisResultRequest{})
err = merr.CheckRPCCall(queryAnalysisResultResp, err)
suite.ErrorIs(err, merr.ErrServiceNotReady)
dropAnalysisTasksResp, err := in.DropAnalysisTasks(ctx, &indexpb.DropAnalysisTasksRequest{})
err = merr.CheckRPCCall(dropAnalysisTasksResp, err)
suite.ErrorIs(err, merr.ErrServiceNotReady)
}
func (suite *IndexNodeServiceSuite) Test_Method() {
ctx := context.TODO()
in, err := NewMockIndexNodeComponent(context.TODO())
suite.NoError(err)
suite.NoError(in.Stop())
in.UpdateStateCode(commonpb.StateCode_Healthy)
suite.Run("Analysis", func() {
req := &indexpb.AnalysisRequest{
ClusterID: suite.cluster,
TaskID: suite.taskID,
CollectionID: suite.collectionID,
PartitionID: suite.partitionID,
FieldID: suite.fieldID,
SegmentStats: map[int64]*indexpb.SegmentStats{
suite.segmentID: {
ID: suite.segmentID,
NumRows: 1024,
LogIDs: []int64{1, 2, 3},
}},
Version: 1,
StorageConfig: nil,
}
resp, err := in.Analysis(ctx, req)
err = merr.CheckRPCCall(resp, err)
suite.NoError(err)
})
suite.Run("QueryAnalysisTask", func() {
req := &indexpb.QueryAnalysisResultRequest{
ClusterID: suite.cluster,
TaskIDs: []int64{suite.taskID},
}
resp, err := in.QueryAnalysisResult(ctx, req)
err = merr.CheckRPCCall(resp, err)
suite.NoError(err)
})
suite.Run("DropAnalysisTask", func() {
req := &indexpb.DropAnalysisTasksRequest{
ClusterID: suite.cluster,
TaskIDs: []int64{suite.taskID},
}
resp, err := in.DropAnalysisTasks(ctx, req)
err = merr.CheckRPCCall(resp, err)
suite.NoError(err)
})
}
func Test_IndexNodeServiceSuite(t *testing.T) {
suite.Run(t, new(IndexNodeServiceSuite))
}

View File

@ -529,17 +529,17 @@ func TestIndexTaskWhenStoppingNode(t *testing.T) {
paramtable.Init()
in := NewIndexNode(ctx, factory)
in.loadOrStoreTask("cluster-1", 1, &taskInfo{
in.loadOrStoreIndexTask("cluster-1", 1, &indexTaskInfo{
state: commonpb.IndexState_InProgress,
})
in.loadOrStoreTask("cluster-2", 2, &taskInfo{
in.loadOrStoreIndexTask("cluster-2", 2, &indexTaskInfo{
state: commonpb.IndexState_Finished,
})
assert.True(t, in.hasInProgressTask())
go func() {
time.Sleep(2 * time.Second)
in.storeTaskState("cluster-1", 1, commonpb.IndexState_Finished, "")
in.storeIndexTaskState("cluster-1", 1, commonpb.IndexState_Finished, "")
}()
noTaskChan := make(chan struct{})
go func() {

View File

@ -0,0 +1 @@
package indexnode

View File

@ -18,32 +18,10 @@ package indexnode
import (
"context"
"encoding/json"
"fmt"
"runtime/debug"
"strconv"
"strings"
"time"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/hardware"
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
"github.com/milvus-io/milvus/pkg/util/indexparams"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metautil"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/timerecord"
)
var (
@ -53,19 +31,6 @@ var (
type Blob = storage.Blob
type taskInfo struct {
cancel context.CancelFunc
state commonpb.IndexState
fileKeys []string
serializedSize uint64
failReason string
currentIndexVersion int32
indexStoreVersion int64
// task statistics
statistic *indexpb.JobInfo
}
type task interface {
Ctx() context.Context
Name() string
@ -78,598 +43,3 @@ type task interface {
GetState() commonpb.IndexState
Reset()
}
type indexBuildTaskV2 struct {
*indexBuildTask
}
func (it *indexBuildTaskV2) parseParams(ctx context.Context) error {
it.collectionID = it.req.CollectionID
it.partitionID = it.req.PartitionID
it.segmentID = it.req.SegmentID
it.fieldType = it.req.FieldType
it.fieldID = it.req.FieldID
it.fieldName = it.req.FieldName
return nil
}
func (it *indexBuildTaskV2) BuildIndex(ctx context.Context) error {
err := it.parseParams(ctx)
if err != nil {
log.Ctx(ctx).Warn("parse field meta from binlog failed", zap.Error(err))
return err
}
indexType := it.newIndexParams[common.IndexTypeKey]
if indexType == indexparamcheck.IndexDISKANN {
// check index node support disk index
if !Params.IndexNodeCfg.EnableDisk.GetAsBool() {
log.Ctx(ctx).Warn("IndexNode don't support build disk index",
zap.String("index type", it.newIndexParams[common.IndexTypeKey]),
zap.Bool("enable disk", Params.IndexNodeCfg.EnableDisk.GetAsBool()))
return merr.WrapErrIndexNotSupported("disk index")
}
// check load size and size of field data
localUsedSize, err := indexcgowrapper.GetLocalUsedSize(paramtable.Get().LocalStorageCfg.Path.GetValue())
if err != nil {
log.Ctx(ctx).Warn("IndexNode get local used size failed")
return err
}
fieldDataSize, err := estimateFieldDataSize(it.statistic.Dim, it.req.GetNumRows(), it.fieldType)
if err != nil {
log.Ctx(ctx).Warn("IndexNode get local used size failed")
return err
}
usedLocalSizeWhenBuild := int64(float64(fieldDataSize)*diskUsageRatio) + localUsedSize
maxUsedLocalSize := int64(Params.IndexNodeCfg.DiskCapacityLimit.GetAsFloat() * Params.IndexNodeCfg.MaxDiskUsagePercentage.GetAsFloat())
if usedLocalSizeWhenBuild > maxUsedLocalSize {
log.Ctx(ctx).Warn("IndexNode don't has enough disk size to build disk ann index",
zap.Int64("usedLocalSizeWhenBuild", usedLocalSizeWhenBuild),
zap.Int64("maxUsedLocalSize", maxUsedLocalSize))
return merr.WrapErrServiceDiskLimitExceeded(float32(usedLocalSizeWhenBuild), float32(maxUsedLocalSize))
}
err = indexparams.SetDiskIndexBuildParams(it.newIndexParams, int64(fieldDataSize))
if err != nil {
log.Ctx(ctx).Warn("failed to fill disk index params", zap.Error(err))
return err
}
}
var buildIndexInfo *indexcgowrapper.BuildIndexInfo
buildIndexInfo, err = indexcgowrapper.NewBuildIndexInfo(it.req.GetStorageConfig())
defer indexcgowrapper.DeleteBuildIndexInfo(buildIndexInfo)
if err != nil {
log.Ctx(ctx).Warn("create build index info failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendFieldMetaInfoV2(it.collectionID, it.partitionID, it.segmentID, it.fieldID, it.fieldType, it.fieldName, it.req.Dim)
if err != nil {
log.Ctx(ctx).Warn("append field meta failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendIndexMetaInfo(it.req.IndexID, it.req.BuildID, it.req.IndexVersion)
if err != nil {
log.Ctx(ctx).Warn("append index meta failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendBuildIndexParam(it.newIndexParams)
if err != nil {
log.Ctx(ctx).Warn("append index params failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendIndexStorageInfo(it.req.StorePath, it.req.IndexStorePath, it.req.StoreVersion)
if err != nil {
log.Ctx(ctx).Warn("append storage info failed", zap.Error(err))
return err
}
jsonIndexParams, err := json.Marshal(it.newIndexParams)
if err != nil {
log.Ctx(ctx).Error("failed to json marshal index params", zap.Error(err))
return err
}
log.Ctx(ctx).Info("index params are ready",
zap.Int64("buildID", it.BuildID),
zap.String("index params", string(jsonIndexParams)))
err = buildIndexInfo.AppendBuildTypeParam(it.newTypeParams)
if err != nil {
log.Ctx(ctx).Warn("append type params failed", zap.Error(err))
return err
}
for _, optField := range it.req.GetOptionalScalarFields() {
if err := buildIndexInfo.AppendOptionalField(optField); err != nil {
log.Ctx(ctx).Warn("append optional field failed", zap.Error(err))
return err
}
}
it.index, err = indexcgowrapper.CreateIndexV2(ctx, buildIndexInfo)
if err != nil {
if it.index != nil && it.index.CleanLocalData() != nil {
log.Ctx(ctx).Error("failed to clean cached data on disk after build index failed",
zap.Int64("buildID", it.BuildID),
zap.Int64("index version", it.req.GetIndexVersion()))
}
log.Ctx(ctx).Error("failed to build index", zap.Error(err))
return err
}
buildIndexLatency := it.tr.RecordSpan()
metrics.IndexNodeKnowhereBuildIndexLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(float64(buildIndexLatency.Milliseconds()))
log.Ctx(ctx).Info("Successfully build index", zap.Int64("buildID", it.BuildID), zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID))
return nil
}
func (it *indexBuildTaskV2) SaveIndexFiles(ctx context.Context) error {
gcIndex := func() {
if err := it.index.Delete(); err != nil {
log.Ctx(ctx).Error("IndexNode indexBuildTask Execute CIndexDelete failed", zap.Error(err))
}
}
version, err := it.index.UpLoadV2()
if err != nil {
log.Ctx(ctx).Error("failed to upload index", zap.Error(err))
gcIndex()
return err
}
encodeIndexFileDur := it.tr.Record("index serialize and upload done")
metrics.IndexNodeEncodeIndexFileLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(encodeIndexFileDur.Seconds())
// early release index for gc, and we can ensure that Delete is idempotent.
gcIndex()
// use serialized size before encoding
it.serializedSize = 0
saveFileKeys := make([]string, 0)
it.statistic.EndTime = time.Now().UnixMicro()
it.node.storeIndexFilesAndStatisticV2(it.ClusterID, it.BuildID, saveFileKeys, it.serializedSize, &it.statistic, it.currentIndexVersion, version)
log.Ctx(ctx).Debug("save index files done", zap.Strings("IndexFiles", saveFileKeys))
saveIndexFileDur := it.tr.RecordSpan()
metrics.IndexNodeSaveIndexFileLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(saveIndexFileDur.Seconds())
it.tr.Elapse("index building all done")
log.Ctx(ctx).Info("Successfully save index files", zap.Int64("buildID", it.BuildID), zap.Int64("Collection", it.collectionID),
zap.Int64("partition", it.partitionID), zap.Int64("SegmentId", it.segmentID))
return nil
}
// IndexBuildTask is used to record the information of the index tasks.
type indexBuildTask struct {
ident string
cancel context.CancelFunc
ctx context.Context
cm storage.ChunkManager
index indexcgowrapper.CodecIndex
savePaths []string
req *indexpb.CreateJobRequest
currentIndexVersion int32
BuildID UniqueID
nodeID UniqueID
ClusterID string
collectionID UniqueID
partitionID UniqueID
segmentID UniqueID
fieldID UniqueID
fieldName string
fieldType schemapb.DataType
fieldData storage.FieldData
indexBlobs []*storage.Blob
newTypeParams map[string]string
newIndexParams map[string]string
serializedSize uint64
tr *timerecord.TimeRecorder
queueDur time.Duration
statistic indexpb.JobInfo
node *IndexNode
}
func (it *indexBuildTask) Reset() {
it.ident = ""
it.cancel = nil
it.ctx = nil
it.cm = nil
it.index = nil
it.savePaths = nil
it.req = nil
it.fieldData = nil
it.indexBlobs = nil
it.newTypeParams = nil
it.newIndexParams = nil
it.tr = nil
it.node = nil
}
// Ctx is the context of index tasks.
func (it *indexBuildTask) Ctx() context.Context {
return it.ctx
}
// Name is the name of task to build index.
func (it *indexBuildTask) Name() string {
return it.ident
}
func (it *indexBuildTask) SetState(state commonpb.IndexState, failReason string) {
it.node.storeTaskState(it.ClusterID, it.BuildID, state, failReason)
}
func (it *indexBuildTask) GetState() commonpb.IndexState {
return it.node.loadTaskState(it.ClusterID, it.BuildID)
}
// OnEnqueue enqueues indexing tasks.
func (it *indexBuildTask) OnEnqueue(ctx context.Context) error {
it.queueDur = 0
it.tr.RecordSpan()
it.statistic.StartTime = time.Now().UnixMicro()
it.statistic.PodID = it.node.GetNodeID()
log.Ctx(ctx).Info("IndexNode IndexBuilderTask Enqueue", zap.Int64("buildID", it.BuildID), zap.Int64("segmentID", it.segmentID))
return nil
}
func (it *indexBuildTask) Prepare(ctx context.Context) error {
it.queueDur = it.tr.RecordSpan()
log.Ctx(ctx).Info("Begin to prepare indexBuildTask", zap.Int64("buildID", it.BuildID),
zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID))
typeParams := make(map[string]string)
indexParams := make(map[string]string)
if len(it.req.DataPaths) == 0 {
for _, id := range it.req.GetDataIds() {
path := metautil.BuildInsertLogPath(it.req.GetStorageConfig().RootPath, it.req.GetCollectionID(), it.req.GetPartitionID(), it.req.GetSegmentID(), it.req.GetFieldID(), id)
it.req.DataPaths = append(it.req.DataPaths, path)
}
}
if it.req.OptionalScalarFields != nil {
for _, optFields := range it.req.GetOptionalScalarFields() {
if len(optFields.DataPaths) == 0 {
for _, id := range optFields.DataIds {
path := metautil.BuildInsertLogPath(it.req.GetStorageConfig().RootPath, it.req.GetCollectionID(), it.req.GetPartitionID(), it.req.GetSegmentID(), optFields.FieldID, id)
optFields.DataPaths = append(optFields.DataPaths, path)
}
}
}
}
// type params can be removed
for _, kvPair := range it.req.GetTypeParams() {
key, value := kvPair.GetKey(), kvPair.GetValue()
typeParams[key] = value
indexParams[key] = value
}
for _, kvPair := range it.req.GetIndexParams() {
key, value := kvPair.GetKey(), kvPair.GetValue()
// knowhere would report error if encountered the unknown key,
// so skip this
if key == common.MmapEnabledKey {
continue
}
indexParams[key] = value
}
it.newTypeParams = typeParams
it.newIndexParams = indexParams
it.statistic.IndexParams = it.req.GetIndexParams()
// ugly codes to get dimension
if dimStr, ok := typeParams[common.DimKey]; ok {
var err error
it.statistic.Dim, err = strconv.ParseInt(dimStr, 10, 64)
if err != nil {
log.Ctx(ctx).Error("parse dimesion failed", zap.Error(err))
// ignore error
}
}
log.Ctx(ctx).Info("Successfully prepare indexBuildTask", zap.Int64("buildID", it.BuildID),
zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID))
return nil
}
func (it *indexBuildTask) LoadData(ctx context.Context) error {
getValueByPath := func(path string) ([]byte, error) {
data, err := it.cm.Read(ctx, path)
if err != nil {
if errors.Is(err, merr.ErrIoKeyNotFound) {
return nil, err
}
return nil, err
}
return data, nil
}
getBlobByPath := func(path string) (*Blob, error) {
value, err := getValueByPath(path)
if err != nil {
return nil, err
}
return &Blob{
Key: path,
Value: value,
}, nil
}
toLoadDataPaths := it.req.GetDataPaths()
keys := make([]string, len(toLoadDataPaths))
blobs := make([]*Blob, len(toLoadDataPaths))
loadKey := func(idx int) error {
keys[idx] = toLoadDataPaths[idx]
blob, err := getBlobByPath(toLoadDataPaths[idx])
if err != nil {
return err
}
blobs[idx] = blob
return nil
}
// Use hardware.GetCPUNum() instead of hardware.GetCPUNum()
// to respect CPU quota of container/pod
// gomaxproc will be set by `automaxproc`, passing 0 will just retrieve the value
err := funcutil.ProcessFuncParallel(len(toLoadDataPaths), hardware.GetCPUNum(), loadKey, "loadKey")
if err != nil {
log.Ctx(ctx).Warn("loadKey failed", zap.Error(err))
return err
}
loadFieldDataLatency := it.tr.CtxRecord(ctx, "load field data done")
metrics.IndexNodeLoadFieldLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(loadFieldDataLatency.Seconds())
err = it.decodeBlobs(ctx, blobs)
if err != nil {
log.Ctx(ctx).Info("failed to decode blobs", zap.Int64("buildID", it.BuildID),
zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID), zap.Error(err))
} else {
log.Ctx(ctx).Info("Successfully load data", zap.Int64("buildID", it.BuildID),
zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID))
}
blobs = nil
debug.FreeOSMemory()
return err
}
func (it *indexBuildTask) BuildIndex(ctx context.Context) error {
err := it.parseFieldMetaFromBinlog(ctx)
if err != nil {
log.Ctx(ctx).Warn("parse field meta from binlog failed", zap.Error(err))
return err
}
indexType := it.newIndexParams[common.IndexTypeKey]
if indexType == indexparamcheck.IndexDISKANN {
// check index node support disk index
if !Params.IndexNodeCfg.EnableDisk.GetAsBool() {
log.Ctx(ctx).Warn("IndexNode don't support build disk index",
zap.String("index type", it.newIndexParams[common.IndexTypeKey]),
zap.Bool("enable disk", Params.IndexNodeCfg.EnableDisk.GetAsBool()))
return errors.New("index node don't support build disk index")
}
// check load size and size of field data
localUsedSize, err := indexcgowrapper.GetLocalUsedSize(paramtable.Get().LocalStorageCfg.Path.GetValue())
if err != nil {
log.Ctx(ctx).Warn("IndexNode get local used size failed")
return err
}
fieldDataSize, err := estimateFieldDataSize(it.statistic.Dim, it.req.GetNumRows(), it.fieldType)
if err != nil {
log.Ctx(ctx).Warn("IndexNode get local used size failed")
return err
}
usedLocalSizeWhenBuild := int64(float64(fieldDataSize)*diskUsageRatio) + localUsedSize
maxUsedLocalSize := int64(Params.IndexNodeCfg.DiskCapacityLimit.GetAsFloat() * Params.IndexNodeCfg.MaxDiskUsagePercentage.GetAsFloat())
if usedLocalSizeWhenBuild > maxUsedLocalSize {
log.Ctx(ctx).Warn("IndexNode don't has enough disk size to build disk ann index",
zap.Int64("usedLocalSizeWhenBuild", usedLocalSizeWhenBuild),
zap.Int64("maxUsedLocalSize", maxUsedLocalSize))
return errors.New("index node don't has enough disk size to build disk ann index")
}
err = indexparams.SetDiskIndexBuildParams(it.newIndexParams, int64(fieldDataSize))
if err != nil {
log.Ctx(ctx).Warn("failed to fill disk index params", zap.Error(err))
return err
}
}
var buildIndexInfo *indexcgowrapper.BuildIndexInfo
buildIndexInfo, err = indexcgowrapper.NewBuildIndexInfo(it.req.GetStorageConfig())
defer indexcgowrapper.DeleteBuildIndexInfo(buildIndexInfo)
if err != nil {
log.Ctx(ctx).Warn("create build index info failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendFieldMetaInfo(it.collectionID, it.partitionID, it.segmentID, it.fieldID, it.fieldType)
if err != nil {
log.Ctx(ctx).Warn("append field meta failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendIndexMetaInfo(it.req.IndexID, it.req.BuildID, it.req.IndexVersion)
if err != nil {
log.Ctx(ctx).Warn("append index meta failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendBuildIndexParam(it.newIndexParams)
if err != nil {
log.Ctx(ctx).Warn("append index params failed", zap.Error(err))
return err
}
jsonIndexParams, err := json.Marshal(it.newIndexParams)
if err != nil {
log.Ctx(ctx).Error("failed to json marshal index params", zap.Error(err))
return err
}
log.Ctx(ctx).Info("index params are ready",
zap.Int64("buildID", it.BuildID),
zap.String("index params", string(jsonIndexParams)))
err = buildIndexInfo.AppendBuildTypeParam(it.newTypeParams)
if err != nil {
log.Ctx(ctx).Warn("append type params failed", zap.Error(err))
return err
}
for _, path := range it.req.GetDataPaths() {
err = buildIndexInfo.AppendInsertFile(path)
if err != nil {
log.Ctx(ctx).Warn("append insert binlog path failed", zap.Error(err))
return err
}
}
it.currentIndexVersion = getCurrentIndexVersion(it.req.GetCurrentIndexVersion())
if err := buildIndexInfo.AppendIndexEngineVersion(it.currentIndexVersion); err != nil {
log.Ctx(ctx).Warn("append index engine version failed", zap.Error(err))
return err
}
for _, optField := range it.req.GetOptionalScalarFields() {
if err := buildIndexInfo.AppendOptionalField(optField); err != nil {
log.Ctx(ctx).Warn("append optional field failed", zap.Error(err))
return err
}
}
it.index, err = indexcgowrapper.CreateIndex(ctx, buildIndexInfo)
if err != nil {
if it.index != nil && it.index.CleanLocalData() != nil {
log.Ctx(ctx).Error("failed to clean cached data on disk after build index failed",
zap.Int64("buildID", it.BuildID),
zap.Int64("index version", it.req.GetIndexVersion()))
}
log.Ctx(ctx).Error("failed to build index", zap.Error(err))
return err
}
buildIndexLatency := it.tr.RecordSpan()
metrics.IndexNodeKnowhereBuildIndexLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(buildIndexLatency.Seconds())
log.Ctx(ctx).Info("Successfully build index", zap.Int64("buildID", it.BuildID), zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID), zap.Int32("currentIndexVersion", it.currentIndexVersion))
return nil
}
func (it *indexBuildTask) SaveIndexFiles(ctx context.Context) error {
gcIndex := func() {
if err := it.index.Delete(); err != nil {
log.Ctx(ctx).Error("IndexNode indexBuildTask Execute CIndexDelete failed", zap.Error(err))
}
}
indexFilePath2Size, err := it.index.UpLoad()
if err != nil {
log.Ctx(ctx).Error("failed to upload index", zap.Error(err))
gcIndex()
return err
}
encodeIndexFileDur := it.tr.Record("index serialize and upload done")
metrics.IndexNodeEncodeIndexFileLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(encodeIndexFileDur.Seconds())
// early release index for gc, and we can ensure that Delete is idempotent.
gcIndex()
// use serialized size before encoding
it.serializedSize = 0
saveFileKeys := make([]string, 0)
for filePath, fileSize := range indexFilePath2Size {
it.serializedSize += uint64(fileSize)
parts := strings.Split(filePath, "/")
fileKey := parts[len(parts)-1]
saveFileKeys = append(saveFileKeys, fileKey)
}
it.statistic.EndTime = time.Now().UnixMicro()
it.node.storeIndexFilesAndStatistic(it.ClusterID, it.BuildID, saveFileKeys, it.serializedSize, &it.statistic, it.currentIndexVersion)
log.Ctx(ctx).Debug("save index files done", zap.Strings("IndexFiles", saveFileKeys))
saveIndexFileDur := it.tr.RecordSpan()
metrics.IndexNodeSaveIndexFileLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(saveIndexFileDur.Seconds())
it.tr.Elapse("index building all done")
log.Ctx(ctx).Info("Successfully save index files", zap.Int64("buildID", it.BuildID), zap.Int64("Collection", it.collectionID),
zap.Int64("partition", it.partitionID), zap.Int64("SegmentId", it.segmentID))
return nil
}
func (it *indexBuildTask) parseFieldMetaFromBinlog(ctx context.Context) error {
toLoadDataPaths := it.req.GetDataPaths()
if len(toLoadDataPaths) == 0 {
return merr.WrapErrParameterInvalidMsg("data insert path must be not empty")
}
data, err := it.cm.Read(ctx, toLoadDataPaths[0])
if err != nil {
if errors.Is(err, merr.ErrIoKeyNotFound) {
return err
}
return err
}
var insertCodec storage.InsertCodec
collectionID, partitionID, segmentID, insertData, err := insertCodec.DeserializeAll([]*Blob{{Key: toLoadDataPaths[0], Value: data}})
if err != nil {
return err
}
if len(insertData.Data) != 1 {
return merr.WrapErrParameterInvalidMsg("we expect only one field in deserialized insert data")
}
it.collectionID = collectionID
it.partitionID = partitionID
it.segmentID = segmentID
for fID, value := range insertData.Data {
it.fieldType = value.GetDataType()
it.fieldID = fID
break
}
return nil
}
func (it *indexBuildTask) decodeBlobs(ctx context.Context, blobs []*storage.Blob) error {
var insertCodec storage.InsertCodec
collectionID, partitionID, segmentID, insertData, err2 := insertCodec.DeserializeAll(blobs)
if err2 != nil {
return err2
}
metrics.IndexNodeDecodeFieldLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(it.tr.RecordSpan().Seconds())
if len(insertData.Data) != 1 {
return merr.WrapErrParameterInvalidMsg("we expect only one field in deserialized insert data")
}
it.collectionID = collectionID
it.partitionID = partitionID
it.segmentID = segmentID
deserializeDur := it.tr.RecordSpan()
log.Ctx(ctx).Info("IndexNode deserialize data success",
zap.Int64("index id", it.req.IndexID),
zap.String("index name", it.req.IndexName),
zap.Int64("collectionID", it.collectionID),
zap.Int64("partitionID", it.partitionID),
zap.Int64("segmentID", it.segmentID),
zap.Duration("deserialize duration", deserializeDur))
// we can ensure that there blobs are in one Field
var data storage.FieldData
var fieldID storage.FieldID
for fID, value := range insertData.Data {
data = value
fieldID = fID
break
}
it.statistic.NumRows = int64(data.RowNum())
it.fieldID = fieldID
it.fieldData = data
return nil
}

View File

@ -0,0 +1,207 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package indexnode
import (
"context"
"time"
"github.com/milvus-io/milvus/internal/util/analysiscgowrapper"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/metautil"
"github.com/milvus-io/milvus/pkg/util/timerecord"
)
type analysisTask struct {
ident string
ctx context.Context
cancel context.CancelFunc
req *indexpb.AnalysisRequest
tr *timerecord.TimeRecorder
queueDur time.Duration
node *IndexNode
analysis analysiscgowrapper.CodecAnalysis
segmentIDs []int64
dataPaths map[int64][]string
startTime int64
endTime int64
}
func (at *analysisTask) Ctx() context.Context {
return at.ctx
}
func (at *analysisTask) Name() string {
return at.ident
}
func (at *analysisTask) Prepare(ctx context.Context) error {
at.queueDur = at.tr.RecordSpan()
log := log.Ctx(ctx).With(zap.String("clusterID", at.req.GetClusterID()),
zap.Int64("taskID", at.req.GetTaskID()), zap.Int64("Collection", at.req.GetCollectionID()),
zap.Int64("partitionID", at.req.GetPartitionID()), zap.Int64("fieldID", at.req.GetFieldID()))
log.Info("Begin to prepare analysis task")
at.segmentIDs = make([]int64, 0)
at.dataPaths = make(map[int64][]string)
for segID, stats := range at.req.GetSegmentStats() {
at.segmentIDs = append(at.segmentIDs, segID)
at.dataPaths[segID] = make([]string, 0)
for _, id := range stats.GetLogIDs() {
path := metautil.BuildInsertLogPath(at.req.GetStorageConfig().RootPath,
at.req.GetCollectionID(), at.req.GetPartitionID(), segID, at.req.GetFieldID(), id)
at.dataPaths[segID] = append(at.dataPaths[segID], path)
}
}
log.Info("Successfully prepare analysis task", zap.Any("dataPaths", at.dataPaths))
return nil
}
func (at *analysisTask) LoadData(ctx context.Context) error {
// Load data in segcore
return nil
}
func (at *analysisTask) BuildIndex(ctx context.Context) error {
var err error
var analysisInfo *analysiscgowrapper.AnalysisInfo
log := log.Ctx(ctx).With(zap.String("clusterID", at.req.GetClusterID()),
zap.Int64("taskID", at.req.GetTaskID()), zap.Int64("Collection", at.req.GetCollectionID()),
zap.Int64("partitionID", at.req.GetPartitionID()), zap.Int64("fieldID", at.req.GetFieldID()))
log.Info("Begin to build analysis task")
analysisInfo, err = analysiscgowrapper.NewAnalysisInfo(at.req.GetStorageConfig())
defer analysiscgowrapper.DeleteAnalysisInfo(analysisInfo)
if err != nil {
log.Warn("create analysis info failed", zap.Error(err))
return err
}
err = analysisInfo.AppendAnalysisFieldMetaInfo(at.req.GetCollectionID(), at.req.GetPartitionID(),
at.req.GetFieldID(), at.req.GetFieldType(), at.req.GetFieldName(), at.req.GetDim())
if err != nil {
log.Warn("append field meta failed", zap.Error(err))
return err
}
err = analysisInfo.AppendAnalysisInfo(at.req.GetTaskID(), at.req.GetVersion())
if err != nil {
log.Warn("append index meta failed", zap.Error(err))
return err
}
for segID, paths := range at.dataPaths {
for _, path := range paths {
err = analysisInfo.AppendSegmentInsertFile(segID, path)
if err != nil {
log.Warn("append insert binlog path failed", zap.Error(err))
return err
}
}
}
err = analysisInfo.AppendSegmentSize(Params.DataCoordCfg.ClusteringCompactionPreferSegmentSize.GetAsSize())
if err != nil {
log.Warn("append segment size failed", zap.Error(err))
return err
}
err = analysisInfo.AppendTrainSize(Params.DataCoordCfg.ClusteringCompactionMaxTrainSize.GetAsSize())
if err != nil {
log.Warn("append train size failed", zap.Error(err))
return err
}
at.analysis, err = analysiscgowrapper.Analysis(ctx, analysisInfo)
if err != nil {
if at.analysis != nil && at.analysis.CleanLocalData() != nil {
log.Error("failed to clean cached data on disk after analysis failed",
zap.Int64("buildID", at.req.GetTaskID()),
zap.Int64("index version", at.req.GetVersion()))
}
log.Error("failed to analysis data", zap.Error(err))
return err
}
analysisLatency := at.tr.RecordSpan()
log.Info("analysis done", zap.Int64("analysis cost", analysisLatency.Milliseconds()))
return nil
}
func (at *analysisTask) SaveIndexFiles(ctx context.Context) error {
log := log.Ctx(ctx).With(zap.String("clusterID", at.req.GetClusterID()),
zap.Int64("taskID", at.req.GetTaskID()), zap.Int64("Collection", at.req.GetCollectionID()),
zap.Int64("partitionID", at.req.GetPartitionID()), zap.Int64("fieldID", at.req.GetFieldID()))
gc := func() {
if err := at.analysis.Delete(); err != nil {
log.Error("IndexNode indexBuildTask Execute CIndexDelete failed", zap.Error(err))
}
}
centroidsFile, segmentsOffsetMapping, err := at.analysis.UpLoad(at.segmentIDs)
if err != nil {
log.Error("failed to upload index", zap.Error(err))
gc()
return err
}
//encodeIndexFileDur := at.tr.Record("index serialize and upload done")
//metrics.IndexNodeEncodeIndexFileLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(encodeIndexFileDur.Seconds())
// early release analysis for gc, and we can ensure that Delete is idempotent.
gc()
at.endTime = time.Now().UnixMicro()
at.node.storeAnalysisStatistic(at.req.GetClusterID(), at.req.GetTaskID(), centroidsFile, segmentsOffsetMapping)
//saveIndexFileDur := at.tr.RecordSpan()
//metrics.IndexNodeSaveIndexFileLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(saveIndexFileDur.Seconds())
at.tr.Elapse("index building all done")
log.Info("Successfully save analysis files")
return nil
}
func (at *analysisTask) OnEnqueue(ctx context.Context) error {
at.queueDur = 0
at.tr.RecordSpan()
at.startTime = time.Now().UnixMicro()
log.Ctx(ctx).Info("IndexNode analysisTask enqueued", zap.String("clusterID", at.req.GetClusterID()),
zap.Int64("taskID", at.req.GetTaskID()))
return nil
}
func (at *analysisTask) SetState(state commonpb.IndexState, failReason string) {
at.node.storeAnalysisTaskState(at.req.GetClusterID(), at.req.GetTaskID(), state, failReason)
}
func (at *analysisTask) GetState() commonpb.IndexState {
return at.node.loadAnalysisTaskState(at.req.GetClusterID(), at.req.GetTaskID())
}
func (at *analysisTask) Reset() {
at.ident = ""
at.ctx = nil
at.cancel = nil
at.req = nil
at.tr = nil
at.queueDur = 0
at.node = nil
at.startTime = 0
at.endTime = 0
}

View File

@ -0,0 +1,641 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package indexnode
import (
"context"
"encoding/json"
"runtime/debug"
"strconv"
"strings"
"time"
"github.com/cockroachdb/errors"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/indexcgowrapper"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/metrics"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/hardware"
"github.com/milvus-io/milvus/pkg/util/indexparamcheck"
"github.com/milvus-io/milvus/pkg/util/indexparams"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/metautil"
"github.com/milvus-io/milvus/pkg/util/paramtable"
"github.com/milvus-io/milvus/pkg/util/timerecord"
)
type indexBuildTaskV2 struct {
*indexBuildTask
}
func (it *indexBuildTaskV2) parseParams(ctx context.Context) error {
it.collectionID = it.req.CollectionID
it.partitionID = it.req.PartitionID
it.segmentID = it.req.SegmentID
it.fieldType = it.req.FieldType
it.fieldID = it.req.FieldID
it.fieldName = it.req.FieldName
return nil
}
func (it *indexBuildTaskV2) BuildIndex(ctx context.Context) error {
err := it.parseParams(ctx)
if err != nil {
log.Ctx(ctx).Warn("parse field meta from binlog failed", zap.Error(err))
return err
}
indexType := it.newIndexParams[common.IndexTypeKey]
if indexType == indexparamcheck.IndexDISKANN {
// check index node support disk index
if !Params.IndexNodeCfg.EnableDisk.GetAsBool() {
log.Ctx(ctx).Warn("IndexNode don't support build disk index",
zap.String("index type", it.newIndexParams[common.IndexTypeKey]),
zap.Bool("enable disk", Params.IndexNodeCfg.EnableDisk.GetAsBool()))
return merr.WrapErrIndexNotSupported("disk index")
}
// check load size and size of field data
localUsedSize, err := indexcgowrapper.GetLocalUsedSize(paramtable.Get().LocalStorageCfg.Path.GetValue())
if err != nil {
log.Ctx(ctx).Warn("IndexNode get local used size failed")
return err
}
fieldDataSize, err := estimateFieldDataSize(it.statistic.Dim, it.req.GetNumRows(), it.fieldType)
if err != nil {
log.Ctx(ctx).Warn("IndexNode get local used size failed")
return err
}
usedLocalSizeWhenBuild := int64(float64(fieldDataSize)*diskUsageRatio) + localUsedSize
maxUsedLocalSize := int64(Params.IndexNodeCfg.DiskCapacityLimit.GetAsFloat() * Params.IndexNodeCfg.MaxDiskUsagePercentage.GetAsFloat())
if usedLocalSizeWhenBuild > maxUsedLocalSize {
log.Ctx(ctx).Warn("IndexNode don't has enough disk size to build disk ann index",
zap.Int64("usedLocalSizeWhenBuild", usedLocalSizeWhenBuild),
zap.Int64("maxUsedLocalSize", maxUsedLocalSize))
return merr.WrapErrServiceDiskLimitExceeded(float32(usedLocalSizeWhenBuild), float32(maxUsedLocalSize))
}
err = indexparams.SetDiskIndexBuildParams(it.newIndexParams, int64(fieldDataSize))
if err != nil {
log.Ctx(ctx).Warn("failed to fill disk index params", zap.Error(err))
return err
}
}
var buildIndexInfo *indexcgowrapper.BuildIndexInfo
buildIndexInfo, err = indexcgowrapper.NewBuildIndexInfo(it.req.GetStorageConfig())
defer indexcgowrapper.DeleteBuildIndexInfo(buildIndexInfo)
if err != nil {
log.Ctx(ctx).Warn("create build index info failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendFieldMetaInfoV2(it.collectionID, it.partitionID, it.segmentID, it.fieldID, it.fieldType, it.fieldName, it.req.Dim)
if err != nil {
log.Ctx(ctx).Warn("append field meta failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendIndexMetaInfo(it.req.IndexID, it.req.BuildID, it.req.IndexVersion)
if err != nil {
log.Ctx(ctx).Warn("append index meta failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendBuildIndexParam(it.newIndexParams)
if err != nil {
log.Ctx(ctx).Warn("append index params failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendIndexStorageInfo(it.req.StorePath, it.req.IndexStorePath, it.req.StoreVersion)
if err != nil {
log.Ctx(ctx).Warn("append storage info failed", zap.Error(err))
return err
}
jsonIndexParams, err := json.Marshal(it.newIndexParams)
if err != nil {
log.Ctx(ctx).Error("failed to json marshal index params", zap.Error(err))
return err
}
log.Ctx(ctx).Info("index params are ready",
zap.Int64("buildID", it.BuildID),
zap.String("index params", string(jsonIndexParams)))
err = buildIndexInfo.AppendBuildTypeParam(it.newTypeParams)
if err != nil {
log.Ctx(ctx).Warn("append type params failed", zap.Error(err))
return err
}
for _, optField := range it.req.GetOptionalScalarFields() {
if err := buildIndexInfo.AppendOptionalField(optField); err != nil {
log.Ctx(ctx).Warn("append optional field failed", zap.Error(err))
return err
}
}
it.index, err = indexcgowrapper.CreateIndexV2(ctx, buildIndexInfo)
if err != nil {
if it.index != nil && it.index.CleanLocalData() != nil {
log.Ctx(ctx).Error("failed to clean cached data on disk after build index failed",
zap.Int64("buildID", it.BuildID),
zap.Int64("index version", it.req.GetIndexVersion()))
}
log.Ctx(ctx).Error("failed to build index", zap.Error(err))
return err
}
buildIndexLatency := it.tr.RecordSpan()
metrics.IndexNodeKnowhereBuildIndexLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(float64(buildIndexLatency.Milliseconds()))
log.Ctx(ctx).Info("Successfully build index", zap.Int64("buildID", it.BuildID), zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID))
return nil
}
func (it *indexBuildTaskV2) SaveIndexFiles(ctx context.Context) error {
gcIndex := func() {
if err := it.index.Delete(); err != nil {
log.Ctx(ctx).Error("IndexNode indexBuildTask Execute CIndexDelete failed", zap.Error(err))
}
}
version, err := it.index.UpLoadV2()
if err != nil {
log.Ctx(ctx).Error("failed to upload index", zap.Error(err))
gcIndex()
return err
}
encodeIndexFileDur := it.tr.Record("index serialize and upload done")
metrics.IndexNodeEncodeIndexFileLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(encodeIndexFileDur.Seconds())
// early release index for gc, and we can ensure that Delete is idempotent.
gcIndex()
// use serialized size before encoding
it.serializedSize = 0
saveFileKeys := make([]string, 0)
it.statistic.EndTime = time.Now().UnixMicro()
it.node.storeIndexFilesAndStatisticV2(it.ClusterID, it.BuildID, saveFileKeys, it.serializedSize, &it.statistic, it.currentIndexVersion, version)
log.Ctx(ctx).Debug("save index files done", zap.Strings("IndexFiles", saveFileKeys))
saveIndexFileDur := it.tr.RecordSpan()
metrics.IndexNodeSaveIndexFileLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(saveIndexFileDur.Seconds())
it.tr.Elapse("index building all done")
log.Ctx(ctx).Info("Successfully save index files", zap.Int64("buildID", it.BuildID), zap.Int64("Collection", it.collectionID),
zap.Int64("partition", it.partitionID), zap.Int64("SegmentId", it.segmentID))
return nil
}
// IndexBuildTask is used to record the information of the index tasks.
type indexBuildTask struct {
ident string
cancel context.CancelFunc
ctx context.Context
cm storage.ChunkManager
index indexcgowrapper.CodecIndex
savePaths []string
req *indexpb.CreateJobRequest
currentIndexVersion int32
BuildID UniqueID
nodeID UniqueID
ClusterID string
collectionID UniqueID
partitionID UniqueID
segmentID UniqueID
fieldID UniqueID
fieldName string
fieldType schemapb.DataType
fieldData storage.FieldData
indexBlobs []*storage.Blob
newTypeParams map[string]string
newIndexParams map[string]string
serializedSize uint64
tr *timerecord.TimeRecorder
queueDur time.Duration
statistic indexpb.JobInfo
node *IndexNode
}
func (it *indexBuildTask) Reset() {
it.ident = ""
it.cancel = nil
it.ctx = nil
it.cm = nil
it.index = nil
it.savePaths = nil
it.req = nil
it.fieldData = nil
it.indexBlobs = nil
it.newTypeParams = nil
it.newIndexParams = nil
it.tr = nil
it.node = nil
}
// Ctx is the context of index tasks.
func (it *indexBuildTask) Ctx() context.Context {
return it.ctx
}
// Name is the name of task to build index.
func (it *indexBuildTask) Name() string {
return it.ident
}
func (it *indexBuildTask) SetState(state commonpb.IndexState, failReason string) {
it.node.storeIndexTaskState(it.ClusterID, it.BuildID, state, failReason)
}
func (it *indexBuildTask) GetState() commonpb.IndexState {
return it.node.loadIndexTaskState(it.ClusterID, it.BuildID)
}
// OnEnqueue enqueues indexing tasks.
func (it *indexBuildTask) OnEnqueue(ctx context.Context) error {
it.queueDur = 0
it.tr.RecordSpan()
it.statistic.StartTime = time.Now().UnixMicro()
it.statistic.PodID = it.node.GetNodeID()
log.Ctx(ctx).Info("IndexNode IndexBuilderTask Enqueue", zap.Int64("buildID", it.BuildID), zap.Int64("segmentID", it.segmentID))
return nil
}
func (it *indexBuildTask) Prepare(ctx context.Context) error {
it.queueDur = it.tr.RecordSpan()
log.Ctx(ctx).Info("Begin to prepare indexBuildTask", zap.Int64("buildID", it.BuildID),
zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID))
typeParams := make(map[string]string)
indexParams := make(map[string]string)
if len(it.req.DataPaths) == 0 {
for _, id := range it.req.GetDataIds() {
path := metautil.BuildInsertLogPath(it.req.GetStorageConfig().RootPath, it.req.GetCollectionID(), it.req.GetPartitionID(), it.req.GetSegmentID(), it.req.GetFieldID(), id)
it.req.DataPaths = append(it.req.DataPaths, path)
}
}
if it.req.OptionalScalarFields != nil {
for _, optFields := range it.req.GetOptionalScalarFields() {
if len(optFields.DataPaths) == 0 {
for _, id := range optFields.DataIds {
path := metautil.BuildInsertLogPath(it.req.GetStorageConfig().RootPath, it.req.GetCollectionID(), it.req.GetPartitionID(), it.req.GetSegmentID(), optFields.FieldID, id)
optFields.DataPaths = append(optFields.DataPaths, path)
}
}
}
}
// type params can be removed
for _, kvPair := range it.req.GetTypeParams() {
key, value := kvPair.GetKey(), kvPair.GetValue()
typeParams[key] = value
indexParams[key] = value
}
for _, kvPair := range it.req.GetIndexParams() {
key, value := kvPair.GetKey(), kvPair.GetValue()
// knowhere would report error if encountered the unknown key,
// so skip this
if key == common.MmapEnabledKey {
continue
}
indexParams[key] = value
}
it.newTypeParams = typeParams
it.newIndexParams = indexParams
it.statistic.IndexParams = it.req.GetIndexParams()
// ugly codes to get dimension
if dimStr, ok := typeParams[common.DimKey]; ok {
var err error
it.statistic.Dim, err = strconv.ParseInt(dimStr, 10, 64)
if err != nil {
log.Ctx(ctx).Error("parse dimesion failed", zap.Error(err))
// ignore error
}
}
log.Ctx(ctx).Info("Successfully prepare indexBuildTask", zap.Int64("buildID", it.BuildID),
zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID))
return nil
}
func (it *indexBuildTask) LoadData(ctx context.Context) error {
getValueByPath := func(path string) ([]byte, error) {
data, err := it.cm.Read(ctx, path)
if err != nil {
if errors.Is(err, merr.ErrIoKeyNotFound) {
return nil, err
}
return nil, err
}
return data, nil
}
getBlobByPath := func(path string) (*Blob, error) {
value, err := getValueByPath(path)
if err != nil {
return nil, err
}
return &Blob{
Key: path,
Value: value,
}, nil
}
toLoadDataPaths := it.req.GetDataPaths()
keys := make([]string, len(toLoadDataPaths))
blobs := make([]*Blob, len(toLoadDataPaths))
loadKey := func(idx int) error {
keys[idx] = toLoadDataPaths[idx]
blob, err := getBlobByPath(toLoadDataPaths[idx])
if err != nil {
return err
}
blobs[idx] = blob
return nil
}
// Use hardware.GetCPUNum() instead of hardware.GetCPUNum()
// to respect CPU quota of container/pod
// gomaxproc will be set by `automaxproc`, passing 0 will just retrieve the value
err := funcutil.ProcessFuncParallel(len(toLoadDataPaths), hardware.GetCPUNum(), loadKey, "loadKey")
if err != nil {
log.Ctx(ctx).Warn("loadKey failed", zap.Error(err))
return err
}
loadFieldDataLatency := it.tr.CtxRecord(ctx, "load field data done")
metrics.IndexNodeLoadFieldLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(loadFieldDataLatency.Seconds())
err = it.decodeBlobs(ctx, blobs)
if err != nil {
log.Ctx(ctx).Info("failed to decode blobs", zap.Int64("buildID", it.BuildID),
zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID), zap.Error(err))
} else {
log.Ctx(ctx).Info("Successfully load data", zap.Int64("buildID", it.BuildID),
zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID))
}
blobs = nil
debug.FreeOSMemory()
return err
}
func (it *indexBuildTask) BuildIndex(ctx context.Context) error {
err := it.parseFieldMetaFromBinlog(ctx)
if err != nil {
log.Ctx(ctx).Warn("parse field meta from binlog failed", zap.Error(err))
return err
}
indexType := it.newIndexParams[common.IndexTypeKey]
if indexType == indexparamcheck.IndexDISKANN {
// check index node support disk index
if !Params.IndexNodeCfg.EnableDisk.GetAsBool() {
log.Ctx(ctx).Warn("IndexNode don't support build disk index",
zap.String("index type", it.newIndexParams[common.IndexTypeKey]),
zap.Bool("enable disk", Params.IndexNodeCfg.EnableDisk.GetAsBool()))
return errors.New("index node don't support build disk index")
}
// check load size and size of field data
localUsedSize, err := indexcgowrapper.GetLocalUsedSize(paramtable.Get().LocalStorageCfg.Path.GetValue())
if err != nil {
log.Ctx(ctx).Warn("IndexNode get local used size failed")
return err
}
fieldDataSize, err := estimateFieldDataSize(it.statistic.Dim, it.req.GetNumRows(), it.fieldType)
if err != nil {
log.Ctx(ctx).Warn("IndexNode get local used size failed")
return err
}
usedLocalSizeWhenBuild := int64(float64(fieldDataSize)*diskUsageRatio) + localUsedSize
maxUsedLocalSize := int64(Params.IndexNodeCfg.DiskCapacityLimit.GetAsFloat() * Params.IndexNodeCfg.MaxDiskUsagePercentage.GetAsFloat())
if usedLocalSizeWhenBuild > maxUsedLocalSize {
log.Ctx(ctx).Warn("IndexNode don't has enough disk size to build disk ann index",
zap.Int64("usedLocalSizeWhenBuild", usedLocalSizeWhenBuild),
zap.Int64("maxUsedLocalSize", maxUsedLocalSize))
return errors.New("index node don't has enough disk size to build disk ann index")
}
err = indexparams.SetDiskIndexBuildParams(it.newIndexParams, int64(fieldDataSize))
if err != nil {
log.Ctx(ctx).Warn("failed to fill disk index params", zap.Error(err))
return err
}
}
var buildIndexInfo *indexcgowrapper.BuildIndexInfo
buildIndexInfo, err = indexcgowrapper.NewBuildIndexInfo(it.req.GetStorageConfig())
defer indexcgowrapper.DeleteBuildIndexInfo(buildIndexInfo)
if err != nil {
log.Ctx(ctx).Warn("create build index info failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendFieldMetaInfo(it.collectionID, it.partitionID, it.segmentID, it.fieldID, it.fieldType)
if err != nil {
log.Ctx(ctx).Warn("append field meta failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendIndexMetaInfo(it.req.IndexID, it.req.BuildID, it.req.IndexVersion)
if err != nil {
log.Ctx(ctx).Warn("append index meta failed", zap.Error(err))
return err
}
err = buildIndexInfo.AppendBuildIndexParam(it.newIndexParams)
if err != nil {
log.Ctx(ctx).Warn("append index params failed", zap.Error(err))
return err
}
jsonIndexParams, err := json.Marshal(it.newIndexParams)
if err != nil {
log.Ctx(ctx).Error("failed to json marshal index params", zap.Error(err))
return err
}
log.Ctx(ctx).Info("index params are ready",
zap.Int64("buildID", it.BuildID),
zap.String("index params", string(jsonIndexParams)))
err = buildIndexInfo.AppendBuildTypeParam(it.newTypeParams)
if err != nil {
log.Ctx(ctx).Warn("append type params failed", zap.Error(err))
return err
}
for _, path := range it.req.GetDataPaths() {
err = buildIndexInfo.AppendInsertFile(path)
if err != nil {
log.Ctx(ctx).Warn("append insert binlog path failed", zap.Error(err))
return err
}
}
it.currentIndexVersion = getCurrentIndexVersion(it.req.GetCurrentIndexVersion())
if err := buildIndexInfo.AppendIndexEngineVersion(it.currentIndexVersion); err != nil {
log.Ctx(ctx).Warn("append index engine version failed", zap.Error(err))
return err
}
for _, optField := range it.req.GetOptionalScalarFields() {
if err := buildIndexInfo.AppendOptionalField(optField); err != nil {
log.Ctx(ctx).Warn("append optional field failed", zap.Error(err))
return err
}
}
it.index, err = indexcgowrapper.CreateIndex(ctx, buildIndexInfo)
if err != nil {
if it.index != nil && it.index.CleanLocalData() != nil {
log.Ctx(ctx).Error("failed to clean cached data on disk after build index failed",
zap.Int64("buildID", it.BuildID),
zap.Int64("index version", it.req.GetIndexVersion()))
}
log.Ctx(ctx).Error("failed to build index", zap.Error(err))
return err
}
buildIndexLatency := it.tr.RecordSpan()
metrics.IndexNodeKnowhereBuildIndexLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(buildIndexLatency.Seconds())
log.Ctx(ctx).Info("Successfully build index", zap.Int64("buildID", it.BuildID), zap.Int64("Collection", it.collectionID), zap.Int64("SegmentID", it.segmentID), zap.Int32("currentIndexVersion", it.currentIndexVersion))
return nil
}
func (it *indexBuildTask) SaveIndexFiles(ctx context.Context) error {
gcIndex := func() {
if err := it.index.Delete(); err != nil {
log.Ctx(ctx).Error("IndexNode indexBuildTask Execute CIndexDelete failed", zap.Error(err))
}
}
indexFilePath2Size, err := it.index.UpLoad()
if err != nil {
log.Ctx(ctx).Error("failed to upload index", zap.Error(err))
gcIndex()
return err
}
encodeIndexFileDur := it.tr.Record("index serialize and upload done")
metrics.IndexNodeEncodeIndexFileLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(encodeIndexFileDur.Seconds())
// early release index for gc, and we can ensure that Delete is idempotent.
gcIndex()
// use serialized size before encoding
it.serializedSize = 0
saveFileKeys := make([]string, 0)
for filePath, fileSize := range indexFilePath2Size {
it.serializedSize += uint64(fileSize)
parts := strings.Split(filePath, "/")
fileKey := parts[len(parts)-1]
saveFileKeys = append(saveFileKeys, fileKey)
}
it.statistic.EndTime = time.Now().UnixMicro()
it.node.storeIndexFilesAndStatistic(it.ClusterID, it.BuildID, saveFileKeys, it.serializedSize, &it.statistic, it.currentIndexVersion)
log.Ctx(ctx).Debug("save index files done", zap.Strings("IndexFiles", saveFileKeys))
saveIndexFileDur := it.tr.RecordSpan()
metrics.IndexNodeSaveIndexFileLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(saveIndexFileDur.Seconds())
it.tr.Elapse("index building all done")
log.Ctx(ctx).Info("Successfully save index files", zap.Int64("buildID", it.BuildID), zap.Int64("Collection", it.collectionID),
zap.Int64("partition", it.partitionID), zap.Int64("SegmentId", it.segmentID))
return nil
}
func (it *indexBuildTask) parseFieldMetaFromBinlog(ctx context.Context) error {
toLoadDataPaths := it.req.GetDataPaths()
if len(toLoadDataPaths) == 0 {
return merr.WrapErrParameterInvalidMsg("data insert path must be not empty")
}
data, err := it.cm.Read(ctx, toLoadDataPaths[0])
if err != nil {
if errors.Is(err, merr.ErrIoKeyNotFound) {
return err
}
return err
}
var insertCodec storage.InsertCodec
collectionID, partitionID, segmentID, insertData, err := insertCodec.DeserializeAll([]*Blob{{Key: toLoadDataPaths[0], Value: data}})
if err != nil {
return err
}
if len(insertData.Data) != 1 {
return merr.WrapErrParameterInvalidMsg("we expect only one field in deserialized insert data")
}
it.collectionID = collectionID
it.partitionID = partitionID
it.segmentID = segmentID
for fID, value := range insertData.Data {
it.fieldType = value.GetDataType()
it.fieldID = fID
break
}
return nil
}
func (it *indexBuildTask) decodeBlobs(ctx context.Context, blobs []*storage.Blob) error {
var insertCodec storage.InsertCodec
collectionID, partitionID, segmentID, insertData, err2 := insertCodec.DeserializeAll(blobs)
if err2 != nil {
return err2
}
metrics.IndexNodeDecodeFieldLatency.WithLabelValues(strconv.FormatInt(paramtable.GetNodeID(), 10)).Observe(it.tr.RecordSpan().Seconds())
if len(insertData.Data) != 1 {
return merr.WrapErrParameterInvalidMsg("we expect only one field in deserialized insert data")
}
it.collectionID = collectionID
it.partitionID = partitionID
it.segmentID = segmentID
deserializeDur := it.tr.RecordSpan()
log.Ctx(ctx).Info("IndexNode deserialize data success",
zap.Int64("index id", it.req.IndexID),
zap.String("index name", it.req.IndexName),
zap.Int64("collectionID", it.collectionID),
zap.Int64("partitionID", it.partitionID),
zap.Int64("segmentID", it.segmentID),
zap.Duration("deserialize duration", deserializeDur))
// we can ensure that there blobs are in one Field
var data storage.FieldData
var fieldID storage.FieldID
for fID, value := range insertData.Data {
data = value
fieldID = fID
break
}
it.statistic.NumRows = int64(data.RowNum())
it.fieldID = fieldID
it.fieldData = data
return nil
}

View File

@ -20,6 +20,14 @@ import (
"context"
"testing"
"github.com/milvus-io/milvus/pkg/util/metautil"
"github.com/milvus-io/milvus/pkg/util/indexparams"
"github.com/milvus-io/milvus/internal/proto/etcdpb"
"github.com/milvus-io/milvus/internal/storage"
"github.com/apache/arrow/go/v12/arrow"
"github.com/apache/arrow/go/v12/arrow/array"
"github.com/apache/arrow/go/v12/arrow/memory"
@ -39,180 +47,185 @@ import (
"github.com/milvus-io/milvus/pkg/util/timerecord"
)
// import (
// "context"
// "github.com/cockroachdb/errors"
// "math/rand"
// "path"
// "strconv"
// "testing"
type IndexBuildTaskSuite struct {
suite.Suite
schema *schemapb.CollectionSchema
collectionID int64
partitionID int64
segmentID int64
dataPath string
}
// "github.com/milvus-io/milvus/internal/kv"
func (suite *IndexBuildTaskSuite) SetupSuite() {
paramtable.Init()
suite.collectionID = 1000
suite.partitionID = 1001
suite.segmentID = 1002
suite.dataPath = "/tmp/milvus/data/1000/1001/1002/3/1"
}
// "github.com/golang/protobuf/proto"
// etcdkv "github.com/milvus-io/milvus/internal/kv/etcd"
// "github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
// "github.com/milvus-io/milvus/internal/proto/indexpb"
// "github.com/milvus-io/milvus/internal/storage"
// "github.com/milvus-io/milvus/pkg/util/etcd"
// "github.com/milvus-io/milvus/pkg/util/timerecord"
// "github.com/stretchr/testify/assert"
// )
func (suite *IndexBuildTaskSuite) SetupTest() {
suite.schema = &schemapb.CollectionSchema{
Name: "test",
Description: "test",
AutoID: false,
Fields: []*schemapb.FieldSchema{
{FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64, IsPrimaryKey: true},
{FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64, IsPrimaryKey: true},
{FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true},
{FieldID: 101, Name: "ts", DataType: schemapb.DataType_Int64},
{FieldID: 102, Name: "vec", DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{{Key: "dim", Value: "1"}}},
},
}
}
// func TestIndexBuildTask_saveIndexMeta(t *testing.T) {
// Params.Init()
// etcdCli, err := etcd.GetEtcdClient(&Params.EtcdCfg)
// assert.NoError(t, err)
// assert.NotNil(t, etcdCli)
// etcdKV := etcdkv.NewEtcdKV(etcdCli, Params.EtcdCfg.MetaRootPath)
// assert.NotNil(t, etcdKV)
// indexBuildID := rand.Int63()
// indexMeta := &indexpb.IndexMeta{
// IndexBuildID: indexBuildID,
// State: commonpb.IndexState_InProgress,
// NodeID: 1,
// IndexVersion: 1,
// }
// metaPath := path.Join("indexes", strconv.FormatInt(indexMeta.IndexBuildID, 10))
// metaValue, err := proto.Marshal(indexMeta)
// assert.NoError(t, err)
// err = etcdKV.Save(metaPath, string(metaValue))
// assert.NoError(t, err)
// indexBuildTask := &IndexBuildTask{
// BaseTask: BaseTask{
// internalErr: errors.New("internal err"),
// },
// etcdKV: etcdKV,
// req: &indexpb.CreateIndexRequest{
// IndexBuildID: indexBuildID,
// Version: 1,
// MetaPath: metaPath,
// },
// tr: &timerecord.TimeRecorder{},
// }
// err = indexBuildTask.saveIndexMeta(context.Background())
// assert.NoError(t, err)
func (suite *IndexBuildTaskSuite) serializeData() ([]*storage.Blob, error) {
insertCodec := storage.NewInsertCodecWithSchema(&etcdpb.CollectionMeta{
Schema: suite.schema,
})
return insertCodec.Serialize(suite.partitionID, suite.segmentID, &storage.InsertData{
Data: map[storage.FieldID]storage.FieldData{
0: &storage.Int64FieldData{Data: []int64{0, 1, 2}},
1: &storage.Int64FieldData{Data: []int64{1, 2, 3}},
100: &storage.Int64FieldData{Data: []int64{0, 1, 2}},
101: &storage.Int64FieldData{Data: []int64{0, 1, 2}},
102: &storage.FloatVectorFieldData{Data: []float32{1, 2, 3}, Dim: 1},
},
Infos: []storage.BlobInfo{{3}},
})
}
// indexMeta2, _, err := indexBuildTask.loadIndexMeta(context.Background())
// assert.NoError(t, err)
// assert.NotNil(t, indexMeta2)
// assert.Equal(t, commonpb.IndexState_Unissued, indexMeta2.State)
func (suite *IndexBuildTaskSuite) TestBuildMemoryIndex() {
ctx, cancel := context.WithCancel(context.Background())
req := &indexpb.CreateJobRequest{
BuildID: 1,
IndexVersion: 1,
DataPaths: []string{suite.dataPath},
IndexID: 0,
IndexName: "",
IndexParams: []*commonpb.KeyValuePair{{Key: common.IndexTypeKey, Value: "FLAT"}, {Key: common.MetricTypeKey, Value: metric.L2}, {Key: common.DimKey, Value: "1"}},
TypeParams: []*commonpb.KeyValuePair{{Key: "dim", Value: "1"}},
NumRows: 10,
StorageConfig: &indexpb.StorageConfig{
RootPath: "/tmp/milvus/data",
StorageType: "local",
},
CollectionID: 1,
PartitionID: 1,
SegmentID: 1,
FieldID: 3,
FieldName: "vec",
FieldType: schemapb.DataType_FloatVector,
}
// err = etcdKV.Remove(metaPath)
// assert.NoError(t, err)
// }
cm, err := NewChunkMgrFactory().NewChunkManager(ctx, req.GetStorageConfig())
suite.NoError(err)
blobs, err := suite.serializeData()
suite.NoError(err)
err = cm.Write(ctx, suite.dataPath, blobs[0].Value)
suite.NoError(err)
// type mockChunkManager struct {
// storage.ChunkManager
t := &indexBuildTask{
ident: "",
cancel: cancel,
ctx: ctx,
cm: cm,
req: req,
currentIndexVersion: 0,
BuildID: req.GetBuildID(),
nodeID: 1,
ClusterID: req.GetClusterID(),
collectionID: req.GetCollectionID(),
partitionID: req.GetPartitionID(),
segmentID: req.GetSegmentID(),
fieldID: req.GetFieldID(),
fieldName: req.GetFieldName(),
fieldType: req.GetFieldType(),
tr: timerecord.NewTimeRecorder("test-indexBuildTask"),
queueDur: 0,
statistic: indexpb.JobInfo{},
node: NewIndexNode(context.Background(), dependency.NewDefaultFactory(true)),
}
// read func(key string) ([]byte, error)
// }
err = t.Prepare(context.Background())
suite.NoError(err)
err = t.BuildIndex(context.Background())
suite.NoError(err)
err = t.SaveIndexFiles(context.Background())
suite.NoError(err)
}
// func (mcm *mockChunkManager) Read(key string) ([]byte, error) {
// return mcm.read(key)
// }
func (suite *IndexBuildTaskSuite) Test_BuildDISKANNIndex() {
ctx, cancel := context.WithCancel(context.Background())
req := &indexpb.CreateJobRequest{
BuildID: 1,
IndexVersion: 1,
DataPaths: []string{suite.dataPath},
IndexID: 0,
IndexName: "",
IndexParams: []*commonpb.KeyValuePair{{Key: common.IndexTypeKey, Value: "DISKANN"},
{Key: common.MetricTypeKey, Value: metric.L2},
{Key: indexparams.PQCodeBudgetRatioKey, Value: "0.125"},
{Key: indexparams.NumBuildThreadRatioKey, Value: "1.0"},
{Key: indexparams.SearchCacheBudgetRatioKey, Value: "0.10"},
{Key: indexparams.SearchCacheBudgetRatioKey, Value: "0.10"},
{Key: indexparams.NumLoadThreadRatioKey, Value: "8.0"},
{Key: indexparams.BeamWidthRatioKey, Value: "4.0"},
},
TypeParams: []*commonpb.KeyValuePair{{Key: "dim", Value: "1"}},
NumRows: 10,
StorageConfig: &indexpb.StorageConfig{
RootPath: "/tmp/milvus/data",
StorageType: "local",
},
CollectionID: 1,
PartitionID: 1,
SegmentID: 1,
FieldID: 3,
FieldName: "vec",
FieldType: schemapb.DataType_FloatVector,
}
// func TestIndexBuildTask_Execute(t *testing.T) {
// t.Run("task retry", func(t *testing.T) {
// indexTask := &IndexBuildTask{
// cm: &mockChunkManager{
// read: func(key string) ([]byte, error) {
// return nil, errors.New("error occurred")
// },
// },
// req: &indexpb.CreateIndexRequest{
// IndexBuildID: 1,
// DataPaths: []string{"path1", "path2"},
// },
// }
cm, err := NewChunkMgrFactory().NewChunkManager(ctx, req.GetStorageConfig())
suite.NoError(err)
blobs, err := suite.serializeData()
suite.NoError(err)
err = cm.Write(ctx, suite.dataPath, blobs[0].Value)
suite.NoError(err)
// err := indexTask.Execute(context.Background())
// assert.Error(t, err)
// assert.Equal(t, TaskStateRetry, indexTask.state)
// })
t := &indexBuildTask{
ident: "",
cancel: cancel,
ctx: ctx,
cm: cm,
req: req,
currentIndexVersion: 0,
BuildID: req.GetBuildID(),
nodeID: 1,
ClusterID: req.GetClusterID(),
collectionID: req.GetCollectionID(),
partitionID: req.GetPartitionID(),
segmentID: req.GetSegmentID(),
fieldID: req.GetFieldID(),
fieldName: req.GetFieldName(),
fieldType: req.GetFieldType(),
tr: timerecord.NewTimeRecorder("test-indexBuildTask"),
queueDur: 0,
statistic: indexpb.JobInfo{},
node: NewIndexNode(context.Background(), dependency.NewDefaultFactory(true)),
}
// t.Run("task failed", func(t *testing.T) {
// indexTask := &IndexBuildTask{
// cm: &mockChunkManager{
// read: func(key string) ([]byte, error) {
// return nil, ErrNoSuchKey
// },
// },
// req: &indexpb.CreateIndexRequest{
// IndexBuildID: 1,
// DataPaths: []string{"path1", "path2"},
// },
// }
err = t.Prepare(context.Background())
suite.NoError(err)
err = t.BuildIndex(context.Background())
suite.NoError(err)
err = t.SaveIndexFiles(context.Background())
suite.NoError(err)
}
// err := indexTask.Execute(context.Background())
// assert.ErrorIs(t, err, ErrNoSuchKey)
// assert.Equal(t, TaskStateFailed, indexTask.state)
// })
// }
// type mockETCDKV struct {
// kv.MetaKv
// loadWithPrefix2 func(key string) ([]string, []string, []int64, error)
// }
// func TestIndexBuildTask_loadIndexMeta(t *testing.T) {
// t.Run("load empty meta", func(t *testing.T) {
// indexTask := &IndexBuildTask{
// etcdKV: &mockETCDKV{
// loadWithPrefix2: func(key string) ([]string, []string, []int64, error) {
// return []string{}, []string{}, []int64{}, nil
// },
// },
// req: &indexpb.CreateIndexRequest{
// IndexBuildID: 1,
// DataPaths: []string{"path1", "path2"},
// },
// }
// indexMeta, revision, err := indexTask.loadIndexMeta(context.Background())
// assert.NoError(t, err)
// assert.Equal(t, int64(0), revision)
// assert.Equal(t, TaskStateAbandon, indexTask.GetState())
// indexTask.updateTaskState(indexMeta, nil)
// assert.Equal(t, TaskStateAbandon, indexTask.GetState())
// })
// }
// func TestIndexBuildTask_saveIndex(t *testing.T) {
// t.Run("save index failed", func(t *testing.T) {
// indexTask := &IndexBuildTask{
// etcdKV: &mockETCDKV{
// loadWithPrefix2: func(key string) ([]string, []string, []int64, error) {
// return []string{}, []string{}, []int64{}, errors.New("error")
// },
// },
// partitionID: 1,
// segmentID: 1,
// req: &indexpb.CreateIndexRequest{
// IndexBuildID: 1,
// DataPaths: []string{"path1", "path2"},
// Version: 1,
// },
// }
// blobs := []*storage.Blob{
// {
// Key: "key1",
// Value: []byte("value1"),
// },
// {
// Key: "key2",
// Value: []byte("value2"),
// },
// }
// err := indexTask.saveIndex(context.Background(), blobs)
// assert.Error(t, err)
// })
// }
func TestIndexBuildTask(t *testing.T) {
suite.Run(t, new(IndexBuildTaskSuite))
}
type IndexBuildTaskV2Suite struct {
suite.Suite
@ -322,3 +335,110 @@ func (suite *IndexBuildTaskV2Suite) TestBuildIndex() {
func TestIndexBuildTaskV2Suite(t *testing.T) {
suite.Run(t, new(IndexBuildTaskV2Suite))
}
type AnalysisTaskSuite struct {
suite.Suite
schema *schemapb.CollectionSchema
collectionID int64
partitionID int64
segmentID int64
fieldID int64
taskID int64
}
func (suite *AnalysisTaskSuite) SetupSuite() {
paramtable.Init()
suite.collectionID = 1000
suite.partitionID = 1001
suite.segmentID = 1002
suite.fieldID = 102
suite.taskID = 1004
}
func (suite *AnalysisTaskSuite) SetupTest() {
suite.schema = &schemapb.CollectionSchema{
Name: "test",
Description: "test",
AutoID: false,
Fields: []*schemapb.FieldSchema{
{FieldID: common.RowIDField, Name: common.RowIDFieldName, DataType: schemapb.DataType_Int64, IsPrimaryKey: true},
{FieldID: common.TimeStampField, Name: common.TimeStampFieldName, DataType: schemapb.DataType_Int64, IsPrimaryKey: true},
{FieldID: 100, Name: "pk", DataType: schemapb.DataType_Int64, IsPrimaryKey: true},
{FieldID: 101, Name: "ts", DataType: schemapb.DataType_Int64},
{FieldID: 102, Name: "vec", DataType: schemapb.DataType_FloatVector, TypeParams: []*commonpb.KeyValuePair{{Key: "dim", Value: "1"}}},
},
}
}
func (suite *AnalysisTaskSuite) serializeData() ([]*storage.Blob, error) {
insertCodec := storage.NewInsertCodecWithSchema(&etcdpb.CollectionMeta{
Schema: suite.schema,
})
return insertCodec.Serialize(suite.partitionID, suite.segmentID, &storage.InsertData{
Data: map[storage.FieldID]storage.FieldData{
0: &storage.Int64FieldData{Data: []int64{0, 1, 2}},
1: &storage.Int64FieldData{Data: []int64{1, 2, 3}},
100: &storage.Int64FieldData{Data: []int64{0, 1, 2}},
101: &storage.Int64FieldData{Data: []int64{0, 1, 2}},
102: &storage.FloatVectorFieldData{Data: []float32{1, 2, 3}, Dim: 1},
},
Infos: []storage.BlobInfo{{3}},
})
}
func (suite *AnalysisTaskSuite) TestAnalysis() {
ctx, cancel := context.WithCancel(context.Background())
req := &indexpb.AnalysisRequest{
ClusterID: "test",
TaskID: 1,
CollectionID: suite.collectionID,
PartitionID: suite.partitionID,
FieldID: suite.fieldID,
FieldName: "vec",
FieldType: schemapb.DataType_FloatVector,
SegmentStats: map[int64]*indexpb.SegmentStats{
suite.segmentID: {
ID: suite.segmentID,
NumRows: 1024,
LogIDs: []int64{1},
},
},
Version: 1,
StorageConfig: &indexpb.StorageConfig{
RootPath: "/tmp/milvus/data",
StorageType: "local",
},
Dim: 1,
}
cm, err := NewChunkMgrFactory().NewChunkManager(ctx, req.GetStorageConfig())
suite.NoError(err)
blobs, err := suite.serializeData()
suite.NoError(err)
dataPath := metautil.BuildInsertLogPath(cm.RootPath(), suite.collectionID, suite.partitionID, suite.segmentID,
suite.fieldID, 1)
err = cm.Write(ctx, dataPath, blobs[0].Value)
suite.NoError(err)
t := &analysisTask{
ident: "",
cancel: cancel,
ctx: ctx,
req: req,
tr: timerecord.NewTimeRecorder("test-indexBuildTask"),
queueDur: 0,
node: NewIndexNode(context.Background(), dependency.NewDefaultFactory(true)),
}
err = t.Prepare(context.Background())
suite.NoError(err)
//err = t.BuildIndex(context.Background())
//suite.NoError(err)
//err = t.SaveIndexFiles(context.Background())
//suite.NoError(err)
}
func TestAnalysisTaskSuite(t *testing.T) {
suite.Run(t, new(AnalysisTaskSuite))
}

View File

@ -13,34 +13,47 @@ import (
"github.com/milvus-io/milvus/pkg/log"
)
func (i *IndexNode) loadOrStoreTask(ClusterID string, buildID UniqueID, info *taskInfo) *taskInfo {
type indexTaskInfo struct {
cancel context.CancelFunc
state commonpb.IndexState
fileKeys []string
serializedSize uint64
failReason string
currentIndexVersion int32
indexStoreVersion int64
// task statistics
statistic *indexpb.JobInfo
}
func (i *IndexNode) loadOrStoreIndexTask(ClusterID string, buildID UniqueID, info *indexTaskInfo) *indexTaskInfo {
i.stateLock.Lock()
defer i.stateLock.Unlock()
key := taskKey{ClusterID: ClusterID, BuildID: buildID}
oldInfo, ok := i.tasks[key]
oldInfo, ok := i.indexTasks[key]
if ok {
return oldInfo
}
i.tasks[key] = info
i.indexTasks[key] = info
return nil
}
func (i *IndexNode) loadTaskState(ClusterID string, buildID UniqueID) commonpb.IndexState {
func (i *IndexNode) loadIndexTaskState(ClusterID string, buildID UniqueID) commonpb.IndexState {
key := taskKey{ClusterID: ClusterID, BuildID: buildID}
i.stateLock.Lock()
defer i.stateLock.Unlock()
task, ok := i.tasks[key]
task, ok := i.indexTasks[key]
if !ok {
return commonpb.IndexState_IndexStateNone
}
return task.state
}
func (i *IndexNode) storeTaskState(ClusterID string, buildID UniqueID, state commonpb.IndexState, failReason string) {
func (i *IndexNode) storeIndexTaskState(ClusterID string, buildID UniqueID, state commonpb.IndexState, failReason string) {
key := taskKey{ClusterID: ClusterID, BuildID: buildID}
i.stateLock.Lock()
defer i.stateLock.Unlock()
if task, ok := i.tasks[key]; ok {
if task, ok := i.indexTasks[key]; ok {
log.Debug("IndexNode store task state", zap.String("clusterID", ClusterID), zap.Int64("buildID", buildID),
zap.String("state", state.String()), zap.String("fail reason", failReason))
task.state = state
@ -48,10 +61,10 @@ func (i *IndexNode) storeTaskState(ClusterID string, buildID UniqueID, state com
}
}
func (i *IndexNode) foreachTaskInfo(fn func(ClusterID string, buildID UniqueID, info *taskInfo)) {
func (i *IndexNode) foreachIndexTaskInfo(fn func(ClusterID string, buildID UniqueID, info *indexTaskInfo)) {
i.stateLock.Lock()
defer i.stateLock.Unlock()
for key, info := range i.tasks {
for key, info := range i.indexTasks {
fn(key.ClusterID, key.BuildID, info)
}
}
@ -67,7 +80,7 @@ func (i *IndexNode) storeIndexFilesAndStatistic(
key := taskKey{ClusterID: ClusterID, BuildID: buildID}
i.stateLock.Lock()
defer i.stateLock.Unlock()
if info, ok := i.tasks[key]; ok {
if info, ok := i.indexTasks[key]; ok {
info.fileKeys = common.CloneStringList(fileKeys)
info.serializedSize = serializedSize
info.statistic = proto.Clone(statistic).(*indexpb.JobInfo)
@ -88,7 +101,7 @@ func (i *IndexNode) storeIndexFilesAndStatisticV2(
key := taskKey{ClusterID: ClusterID, BuildID: buildID}
i.stateLock.Lock()
defer i.stateLock.Unlock()
if info, ok := i.tasks[key]; ok {
if info, ok := i.indexTasks[key]; ok {
info.fileKeys = common.CloneStringList(fileKeys)
info.serializedSize = serializedSize
info.statistic = proto.Clone(statistic).(*indexpb.JobInfo)
@ -98,15 +111,15 @@ func (i *IndexNode) storeIndexFilesAndStatisticV2(
}
}
func (i *IndexNode) deleteTaskInfos(ctx context.Context, keys []taskKey) []*taskInfo {
func (i *IndexNode) deleteIndexTaskInfos(ctx context.Context, keys []taskKey) []*indexTaskInfo {
i.stateLock.Lock()
defer i.stateLock.Unlock()
deleted := make([]*taskInfo, 0, len(keys))
deleted := make([]*indexTaskInfo, 0, len(keys))
for _, key := range keys {
info, ok := i.tasks[key]
info, ok := i.indexTasks[key]
if ok {
deleted = append(deleted, info)
delete(i.tasks, key)
delete(i.indexTasks, key)
log.Ctx(ctx).Info("delete task infos",
zap.String("cluster_id", key.ClusterID), zap.Int64("build_id", key.BuildID))
}
@ -114,13 +127,117 @@ func (i *IndexNode) deleteTaskInfos(ctx context.Context, keys []taskKey) []*task
return deleted
}
func (i *IndexNode) deleteAllTasks() []*taskInfo {
func (i *IndexNode) deleteAllIndexTasks() []*indexTaskInfo {
i.stateLock.Lock()
deletedTasks := i.tasks
i.tasks = make(map[taskKey]*taskInfo)
deletedTasks := i.indexTasks
i.indexTasks = make(map[taskKey]*indexTaskInfo)
i.stateLock.Unlock()
deleted := make([]*taskInfo, 0, len(deletedTasks))
deleted := make([]*indexTaskInfo, 0, len(deletedTasks))
for _, info := range deletedTasks {
deleted = append(deleted, info)
}
return deleted
}
type analysisTaskInfo struct {
cancel context.CancelFunc
state commonpb.IndexState
failReason string
centroidsFile string
segmentsOffsetMapping map[int64]string
indexStoreVersion int64
}
func (i *IndexNode) loadOrStoreAnalysisTask(clusterID string, taskID UniqueID, info *analysisTaskInfo) *analysisTaskInfo {
i.stateLock.Lock()
defer i.stateLock.Unlock()
key := taskKey{ClusterID: clusterID, BuildID: taskID}
oldInfo, ok := i.analysisTasks[key]
if ok {
return oldInfo
}
i.analysisTasks[key] = info
return nil
}
func (i *IndexNode) loadAnalysisTaskState(clusterID string, taskID UniqueID) commonpb.IndexState {
key := taskKey{ClusterID: clusterID, BuildID: taskID}
i.stateLock.Lock()
defer i.stateLock.Unlock()
task, ok := i.analysisTasks[key]
if !ok {
return commonpb.IndexState_IndexStateNone
}
return task.state
}
func (i *IndexNode) storeAnalysisTaskState(clusterID string, taskID UniqueID, state commonpb.IndexState, failReason string) {
key := taskKey{ClusterID: clusterID, BuildID: taskID}
i.stateLock.Lock()
defer i.stateLock.Unlock()
if task, ok := i.analysisTasks[key]; ok {
log.Info("IndexNode store analysis task state", zap.String("clusterID", clusterID), zap.Int64("taskID", taskID),
zap.String("state", state.String()), zap.String("fail reason", failReason))
task.state = state
task.failReason = failReason
}
}
func (i *IndexNode) foreachAnalysisTaskInfo(fn func(clusterID string, taskID UniqueID, info *analysisTaskInfo)) {
i.stateLock.Lock()
defer i.stateLock.Unlock()
for key, info := range i.analysisTasks {
fn(key.ClusterID, key.BuildID, info)
}
}
func (i *IndexNode) getAnalysisTaskInfo(clusterID string, taskID UniqueID) *analysisTaskInfo {
i.stateLock.Lock()
defer i.stateLock.Unlock()
return i.analysisTasks[taskKey{ClusterID: clusterID, BuildID: taskID}]
}
func (i *IndexNode) storeAnalysisStatistic(
clusterID string,
taskID UniqueID,
centroidsFile string,
segmentsOffsetMapping map[int64]string,
) {
key := taskKey{ClusterID: clusterID, BuildID: taskID}
i.stateLock.Lock()
defer i.stateLock.Unlock()
if info, ok := i.analysisTasks[key]; ok {
info.centroidsFile = centroidsFile
info.segmentsOffsetMapping = segmentsOffsetMapping
return
}
}
func (i *IndexNode) deleteAnalysisTaskInfos(ctx context.Context, keys []taskKey) []*analysisTaskInfo {
i.stateLock.Lock()
defer i.stateLock.Unlock()
deleted := make([]*analysisTaskInfo, 0, len(keys))
for _, key := range keys {
info, ok := i.analysisTasks[key]
if ok {
deleted = append(deleted, info)
delete(i.analysisTasks, key)
log.Ctx(ctx).Info("delete analysis task infos",
zap.String("clusterID", key.ClusterID), zap.Int64("taskID", key.BuildID))
}
}
return deleted
}
func (i *IndexNode) deleteAllAnalysisTasks() []*analysisTaskInfo {
i.stateLock.Lock()
deletedTasks := i.analysisTasks
i.analysisTasks = make(map[taskKey]*analysisTaskInfo)
i.stateLock.Unlock()
deleted := make([]*analysisTaskInfo, 0, len(deletedTasks))
for _, info := range deletedTasks {
deleted = append(deleted, info)
}
@ -130,7 +247,13 @@ func (i *IndexNode) deleteAllTasks() []*taskInfo {
func (i *IndexNode) hasInProgressTask() bool {
i.stateLock.Lock()
defer i.stateLock.Unlock()
for _, info := range i.tasks {
for _, info := range i.indexTasks {
if info.state == commonpb.IndexState_InProgress {
return true
}
}
for _, info := range i.analysisTasks {
if info.state == commonpb.IndexState_InProgress {
return true
}
@ -157,7 +280,12 @@ func (i *IndexNode) waitTaskFinish() {
}
case <-timeoutCtx.Done():
log.Warn("timeout, the index node has some progress task")
for _, info := range i.tasks {
for _, info := range i.indexTasks {
if info.state == commonpb.IndexState_InProgress {
log.Warn("progress task", zap.Any("info", info))
}
}
for _, info := range i.analysisTasks {
if info.state == commonpb.IndexState_InProgress {
log.Warn("progress task", zap.Any("info", info))
}

View File

@ -148,6 +148,14 @@ type DataCoordCatalog interface {
DropImportTask(taskID int64) error
GcConfirm(ctx context.Context, collectionID, partitionID typeutil.UniqueID) bool
ListClusteringCompactionInfos(ctx context.Context) ([]*datapb.ClusteringCompactionInfo, error)
SaveClusteringCompactionInfo(ctx context.Context, info *datapb.ClusteringCompactionInfo) error
DropClusteringCompactionInfo(ctx context.Context, info *datapb.ClusteringCompactionInfo) error
ListAnalysisTasks(ctx context.Context) ([]*model.AnalysisTask, error)
SaveAnalysisTask(ctx context.Context, task *model.AnalysisTask) error
DropAnalysisTask(ctx context.Context, taskID typeutil.UniqueID) error
}
type QueryCoordCatalog interface {

View File

@ -17,16 +17,18 @@
package datacoord
const (
MetaPrefix = "datacoord-meta"
SegmentPrefix = MetaPrefix + "/s"
SegmentBinlogPathPrefix = MetaPrefix + "/binlog"
SegmentDeltalogPathPrefix = MetaPrefix + "/deltalog"
SegmentStatslogPathPrefix = MetaPrefix + "/statslog"
ChannelRemovePrefix = MetaPrefix + "/channel-removal"
ChannelCheckpointPrefix = MetaPrefix + "/channel-cp"
ImportJobPrefix = MetaPrefix + "/import-job"
ImportTaskPrefix = MetaPrefix + "/import-task"
PreImportTaskPrefix = MetaPrefix + "/preimport-task"
MetaPrefix = "datacoord-meta"
SegmentPrefix = MetaPrefix + "/s"
SegmentBinlogPathPrefix = MetaPrefix + "/binlog"
SegmentDeltalogPathPrefix = MetaPrefix + "/deltalog"
SegmentStatslogPathPrefix = MetaPrefix + "/statslog"
ChannelRemovePrefix = MetaPrefix + "/channel-removal"
ChannelCheckpointPrefix = MetaPrefix + "/channel-cp"
ImportJobPrefix = MetaPrefix + "/import-job"
ImportTaskPrefix = MetaPrefix + "/import-task"
PreImportTaskPrefix = MetaPrefix + "/preimport-task"
ClusteringCompactionInfoPrefix = MetaPrefix + "/clustering-compaction"
AnalysisTaskPrefix = MetaPrefix + "/analysis-task"
NonRemoveFlagTomestone = "non-removed"
RemoveFlagTomestone = "removed"

View File

@ -815,3 +815,78 @@ func (kc *Catalog) GcConfirm(ctx context.Context, collectionID, partitionID type
}
return len(keys) == 0 && len(values) == 0
}
func (kc *Catalog) ListClusteringCompactionInfos(ctx context.Context) ([]*datapb.ClusteringCompactionInfo, error) {
infos := make([]*datapb.ClusteringCompactionInfo, 0)
_, values, err := kc.MetaKv.LoadWithPrefix(ClusteringCompactionInfoPrefix)
if err != nil {
return nil, err
}
for _, value := range values {
info := &datapb.ClusteringCompactionInfo{}
err = proto.Unmarshal([]byte(value), info)
if err != nil {
return nil, err
}
infos = append(infos, info)
}
return infos, nil
}
func (kc *Catalog) SaveClusteringCompactionInfo(ctx context.Context, coll *datapb.ClusteringCompactionInfo) error {
if coll == nil {
return nil
}
cloned := proto.Clone(coll).(*datapb.ClusteringCompactionInfo)
k, v, err := buildCollectionCompactionInfoKv(cloned)
if err != nil {
return err
}
kvs := make(map[string]string)
kvs[k] = v
return kc.SaveByBatch(kvs)
}
func (kc *Catalog) DropClusteringCompactionInfo(ctx context.Context, info *datapb.ClusteringCompactionInfo) error {
key := buildClusteringCompactionInfoPath(info.CollectionID, info.TriggerID)
return kc.MetaKv.Remove(key)
}
func (kc *Catalog) ListAnalysisTasks(ctx context.Context) ([]*model.AnalysisTask, error) {
tasks := make([]*model.AnalysisTask, 0)
_, values, err := kc.MetaKv.LoadWithPrefix(AnalysisTaskPrefix)
if err != nil {
return nil, err
}
for _, value := range values {
task := &indexpb.AnalysisTask{}
err = proto.Unmarshal([]byte(value), task)
if err != nil {
return nil, err
}
tasks = append(tasks, model.UnmarshalAnalysisTask(task))
}
return tasks, nil
}
func (kc *Catalog) SaveAnalysisTask(ctx context.Context, task *model.AnalysisTask) error {
key := buildAnalysisTaskKey(task.TaskID)
value, err := proto.Marshal(model.MarshalAnalysisTask(task))
if err != nil {
return err
}
err = kc.MetaKv.Save(key, string(value))
if err != nil {
return err
}
return nil
}
func (kc *Catalog) DropAnalysisTask(ctx context.Context, taskID typeutil.UniqueID) error {
key := buildAnalysisTaskKey(taskID)
return kc.MetaKv.Remove(key)
}

View File

@ -227,6 +227,20 @@ func buildSegmentKv(segment *datapb.SegmentInfo) (string, string, error) {
return key, segBytes, nil
}
func buildCollectionCompactionInfoKv(info *datapb.ClusteringCompactionInfo) (string, string, error) {
valueBytes, err := proto.Marshal(info)
if err != nil {
return "", "", fmt.Errorf("failed to marshal collection clustering compaction info: %d, err: %w", info.CollectionID, err)
}
key := buildClusteringCompactionInfoPath(info.CollectionID, info.TriggerID)
return key, string(valueBytes), nil
}
// buildClusteringCompactionInfoPath common logic mapping collection id and trigger id to corresponding key in kv store
func buildClusteringCompactionInfoPath(collectionID typeutil.UniqueID, triggerID typeutil.UniqueID) string {
return fmt.Sprintf("%s/%d/%d", ClusteringCompactionInfoPrefix, collectionID, triggerID)
}
// buildSegmentPath common logic mapping segment info to corresponding key in kv store
func buildSegmentPath(collectionID typeutil.UniqueID, partitionID typeutil.UniqueID, segmentID typeutil.UniqueID) string {
return fmt.Sprintf("%s/%d/%d/%d", SegmentPrefix, collectionID, partitionID, segmentID)
@ -294,3 +308,7 @@ func buildImportTaskKey(taskID int64) string {
func buildPreImportTaskKey(taskID int64) string {
return fmt.Sprintf("%s/%d", PreImportTaskPrefix, taskID)
}
func buildAnalysisTaskKey(taskID int64) string {
return fmt.Sprintf("%s/%d", AnalysisTaskPrefix, taskID)
}

View File

@ -344,6 +344,49 @@ func (_c *DataCoordCatalog_CreateSegmentIndex_Call) RunAndReturn(run func(contex
return _c
}
// DropAnalysisTask provides a mock function with given fields: ctx, taskID
func (_m *DataCoordCatalog) DropAnalysisTask(ctx context.Context, taskID int64) error {
ret := _m.Called(ctx, taskID)
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, int64) error); ok {
r0 = rf(ctx, taskID)
} else {
r0 = ret.Error(0)
}
return r0
}
// DataCoordCatalog_DropAnalysisTask_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'DropAnalysisTask'
type DataCoordCatalog_DropAnalysisTask_Call struct {
*mock.Call
}
// DropAnalysisTask is a helper method to define mock.On call
// - ctx context.Context
// - taskID int64
func (_e *DataCoordCatalog_Expecter) DropAnalysisTask(ctx interface{}, taskID interface{}) *DataCoordCatalog_DropAnalysisTask_Call {
return &DataCoordCatalog_DropAnalysisTask_Call{Call: _e.mock.On("DropAnalysisTask", ctx, taskID)}
}
func (_c *DataCoordCatalog_DropAnalysisTask_Call) Run(run func(ctx context.Context, taskID int64)) *DataCoordCatalog_DropAnalysisTask_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(int64))
})
return _c
}
func (_c *DataCoordCatalog_DropAnalysisTask_Call) Return(_a0 error) *DataCoordCatalog_DropAnalysisTask_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *DataCoordCatalog_DropAnalysisTask_Call) RunAndReturn(run func(context.Context, int64) error) *DataCoordCatalog_DropAnalysisTask_Call {
_c.Call.Return(run)
return _c
}
// DropChannel provides a mock function with given fields: ctx, channel
func (_m *DataCoordCatalog) DropChannel(ctx context.Context, channel string) error {
ret := _m.Called(ctx, channel)
@ -430,6 +473,49 @@ func (_c *DataCoordCatalog_DropChannelCheckpoint_Call) RunAndReturn(run func(con
return _c
}
// DropClusteringCompactionInfo provides a mock function with given fields: ctx, info
func (_m *DataCoordCatalog) DropClusteringCompactionInfo(ctx context.Context, info *datapb.ClusteringCompactionInfo) error {
ret := _m.Called(ctx, info)
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ClusteringCompactionInfo) error); ok {
r0 = rf(ctx, info)
} else {
r0 = ret.Error(0)
}
return r0
}
// DataCoordCatalog_DropClusteringCompactionInfo_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'DropClusteringCompactionInfo'
type DataCoordCatalog_DropClusteringCompactionInfo_Call struct {
*mock.Call
}
// DropClusteringCompactionInfo is a helper method to define mock.On call
// - ctx context.Context
// - info *datapb.ClusteringCompactionInfo
func (_e *DataCoordCatalog_Expecter) DropClusteringCompactionInfo(ctx interface{}, info interface{}) *DataCoordCatalog_DropClusteringCompactionInfo_Call {
return &DataCoordCatalog_DropClusteringCompactionInfo_Call{Call: _e.mock.On("DropClusteringCompactionInfo", ctx, info)}
}
func (_c *DataCoordCatalog_DropClusteringCompactionInfo_Call) Run(run func(ctx context.Context, info *datapb.ClusteringCompactionInfo)) *DataCoordCatalog_DropClusteringCompactionInfo_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.ClusteringCompactionInfo))
})
return _c
}
func (_c *DataCoordCatalog_DropClusteringCompactionInfo_Call) Return(_a0 error) *DataCoordCatalog_DropClusteringCompactionInfo_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *DataCoordCatalog_DropClusteringCompactionInfo_Call) RunAndReturn(run func(context.Context, *datapb.ClusteringCompactionInfo) error) *DataCoordCatalog_DropClusteringCompactionInfo_Call {
_c.Call.Return(run)
return _c
}
// DropImportJob provides a mock function with given fields: jobID
func (_m *DataCoordCatalog) DropImportJob(jobID int64) error {
ret := _m.Called(jobID)
@ -733,6 +819,60 @@ func (_c *DataCoordCatalog_GcConfirm_Call) RunAndReturn(run func(context.Context
return _c
}
// ListAnalysisTasks provides a mock function with given fields: ctx
func (_m *DataCoordCatalog) ListAnalysisTasks(ctx context.Context) ([]*model.AnalysisTask, error) {
ret := _m.Called(ctx)
var r0 []*model.AnalysisTask
var r1 error
if rf, ok := ret.Get(0).(func(context.Context) ([]*model.AnalysisTask, error)); ok {
return rf(ctx)
}
if rf, ok := ret.Get(0).(func(context.Context) []*model.AnalysisTask); ok {
r0 = rf(ctx)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).([]*model.AnalysisTask)
}
}
if rf, ok := ret.Get(1).(func(context.Context) error); ok {
r1 = rf(ctx)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// DataCoordCatalog_ListAnalysisTasks_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ListAnalysisTasks'
type DataCoordCatalog_ListAnalysisTasks_Call struct {
*mock.Call
}
// ListAnalysisTasks is a helper method to define mock.On call
// - ctx context.Context
func (_e *DataCoordCatalog_Expecter) ListAnalysisTasks(ctx interface{}) *DataCoordCatalog_ListAnalysisTasks_Call {
return &DataCoordCatalog_ListAnalysisTasks_Call{Call: _e.mock.On("ListAnalysisTasks", ctx)}
}
func (_c *DataCoordCatalog_ListAnalysisTasks_Call) Run(run func(ctx context.Context)) *DataCoordCatalog_ListAnalysisTasks_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context))
})
return _c
}
func (_c *DataCoordCatalog_ListAnalysisTasks_Call) Return(_a0 []*model.AnalysisTask, _a1 error) *DataCoordCatalog_ListAnalysisTasks_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *DataCoordCatalog_ListAnalysisTasks_Call) RunAndReturn(run func(context.Context) ([]*model.AnalysisTask, error)) *DataCoordCatalog_ListAnalysisTasks_Call {
_c.Call.Return(run)
return _c
}
// ListChannelCheckpoint provides a mock function with given fields: ctx
func (_m *DataCoordCatalog) ListChannelCheckpoint(ctx context.Context) (map[string]*msgpb.MsgPosition, error) {
ret := _m.Called(ctx)
@ -787,6 +927,60 @@ func (_c *DataCoordCatalog_ListChannelCheckpoint_Call) RunAndReturn(run func(con
return _c
}
// ListClusteringCompactionInfos provides a mock function with given fields: ctx
func (_m *DataCoordCatalog) ListClusteringCompactionInfos(ctx context.Context) ([]*datapb.ClusteringCompactionInfo, error) {
ret := _m.Called(ctx)
var r0 []*datapb.ClusteringCompactionInfo
var r1 error
if rf, ok := ret.Get(0).(func(context.Context) ([]*datapb.ClusteringCompactionInfo, error)); ok {
return rf(ctx)
}
if rf, ok := ret.Get(0).(func(context.Context) []*datapb.ClusteringCompactionInfo); ok {
r0 = rf(ctx)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).([]*datapb.ClusteringCompactionInfo)
}
}
if rf, ok := ret.Get(1).(func(context.Context) error); ok {
r1 = rf(ctx)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// DataCoordCatalog_ListClusteringCompactionInfos_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'ListClusteringCompactionInfos'
type DataCoordCatalog_ListClusteringCompactionInfos_Call struct {
*mock.Call
}
// ListClusteringCompactionInfos is a helper method to define mock.On call
// - ctx context.Context
func (_e *DataCoordCatalog_Expecter) ListClusteringCompactionInfos(ctx interface{}) *DataCoordCatalog_ListClusteringCompactionInfos_Call {
return &DataCoordCatalog_ListClusteringCompactionInfos_Call{Call: _e.mock.On("ListClusteringCompactionInfos", ctx)}
}
func (_c *DataCoordCatalog_ListClusteringCompactionInfos_Call) Run(run func(ctx context.Context)) *DataCoordCatalog_ListClusteringCompactionInfos_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context))
})
return _c
}
func (_c *DataCoordCatalog_ListClusteringCompactionInfos_Call) Return(_a0 []*datapb.ClusteringCompactionInfo, _a1 error) *DataCoordCatalog_ListClusteringCompactionInfos_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *DataCoordCatalog_ListClusteringCompactionInfos_Call) RunAndReturn(run func(context.Context) ([]*datapb.ClusteringCompactionInfo, error)) *DataCoordCatalog_ListClusteringCompactionInfos_Call {
_c.Call.Return(run)
return _c
}
// ListImportJobs provides a mock function with given fields:
func (_m *DataCoordCatalog) ListImportJobs() ([]*datapb.ImportJob, error) {
ret := _m.Called()
@ -1194,6 +1388,49 @@ func (_c *DataCoordCatalog_MarkChannelDeleted_Call) RunAndReturn(run func(contex
return _c
}
// SaveAnalysisTask provides a mock function with given fields: ctx, task
func (_m *DataCoordCatalog) SaveAnalysisTask(ctx context.Context, task *model.AnalysisTask) error {
ret := _m.Called(ctx, task)
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, *model.AnalysisTask) error); ok {
r0 = rf(ctx, task)
} else {
r0 = ret.Error(0)
}
return r0
}
// DataCoordCatalog_SaveAnalysisTask_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SaveAnalysisTask'
type DataCoordCatalog_SaveAnalysisTask_Call struct {
*mock.Call
}
// SaveAnalysisTask is a helper method to define mock.On call
// - ctx context.Context
// - task *model.AnalysisTask
func (_e *DataCoordCatalog_Expecter) SaveAnalysisTask(ctx interface{}, task interface{}) *DataCoordCatalog_SaveAnalysisTask_Call {
return &DataCoordCatalog_SaveAnalysisTask_Call{Call: _e.mock.On("SaveAnalysisTask", ctx, task)}
}
func (_c *DataCoordCatalog_SaveAnalysisTask_Call) Run(run func(ctx context.Context, task *model.AnalysisTask)) *DataCoordCatalog_SaveAnalysisTask_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*model.AnalysisTask))
})
return _c
}
func (_c *DataCoordCatalog_SaveAnalysisTask_Call) Return(_a0 error) *DataCoordCatalog_SaveAnalysisTask_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *DataCoordCatalog_SaveAnalysisTask_Call) RunAndReturn(run func(context.Context, *model.AnalysisTask) error) *DataCoordCatalog_SaveAnalysisTask_Call {
_c.Call.Return(run)
return _c
}
// SaveChannelCheckpoint provides a mock function with given fields: ctx, vChannel, pos
func (_m *DataCoordCatalog) SaveChannelCheckpoint(ctx context.Context, vChannel string, pos *msgpb.MsgPosition) error {
ret := _m.Called(ctx, vChannel, pos)
@ -1281,6 +1518,49 @@ func (_c *DataCoordCatalog_SaveChannelCheckpoints_Call) RunAndReturn(run func(co
return _c
}
// SaveClusteringCompactionInfo provides a mock function with given fields: ctx, info
func (_m *DataCoordCatalog) SaveClusteringCompactionInfo(ctx context.Context, info *datapb.ClusteringCompactionInfo) error {
ret := _m.Called(ctx, info)
var r0 error
if rf, ok := ret.Get(0).(func(context.Context, *datapb.ClusteringCompactionInfo) error); ok {
r0 = rf(ctx, info)
} else {
r0 = ret.Error(0)
}
return r0
}
// DataCoordCatalog_SaveClusteringCompactionInfo_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'SaveClusteringCompactionInfo'
type DataCoordCatalog_SaveClusteringCompactionInfo_Call struct {
*mock.Call
}
// SaveClusteringCompactionInfo is a helper method to define mock.On call
// - ctx context.Context
// - info *datapb.ClusteringCompactionInfo
func (_e *DataCoordCatalog_Expecter) SaveClusteringCompactionInfo(ctx interface{}, info interface{}) *DataCoordCatalog_SaveClusteringCompactionInfo_Call {
return &DataCoordCatalog_SaveClusteringCompactionInfo_Call{Call: _e.mock.On("SaveClusteringCompactionInfo", ctx, info)}
}
func (_c *DataCoordCatalog_SaveClusteringCompactionInfo_Call) Run(run func(ctx context.Context, info *datapb.ClusteringCompactionInfo)) *DataCoordCatalog_SaveClusteringCompactionInfo_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*datapb.ClusteringCompactionInfo))
})
return _c
}
func (_c *DataCoordCatalog_SaveClusteringCompactionInfo_Call) Return(_a0 error) *DataCoordCatalog_SaveClusteringCompactionInfo_Call {
_c.Call.Return(_a0)
return _c
}
func (_c *DataCoordCatalog_SaveClusteringCompactionInfo_Call) RunAndReturn(run func(context.Context, *datapb.ClusteringCompactionInfo) error) *DataCoordCatalog_SaveClusteringCompactionInfo_Call {
_c.Call.Return(run)
return _c
}
// SaveDroppedSegmentsInBatch provides a mock function with given fields: ctx, segments
func (_m *DataCoordCatalog) SaveDroppedSegmentsInBatch(ctx context.Context, segments []*datapb.SegmentInfo) error {
ret := _m.Called(ctx, segments)

View File

@ -0,0 +1,110 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package model
import (
"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/proto/indexpb"
)
type AnalysisTask struct {
TenantID string
CollectionID int64
PartitionID int64
FieldID int64
FieldName string
FieldType schemapb.DataType
SegmentIDs []int64
TaskID int64
Version int64
NodeID int64
State commonpb.IndexState
FailReason string
CentroidsFile string
SegmentOffsetMappingFiles map[int64]string
Dim int64
}
func UnmarshalAnalysisTask(info *indexpb.AnalysisTask) *AnalysisTask {
if info == nil {
return nil
}
return &AnalysisTask{
TenantID: "",
CollectionID: info.GetCollectionID(),
PartitionID: info.GetPartitionID(),
FieldID: info.GetFieldID(),
FieldName: info.GetFieldName(),
FieldType: info.GetFieldType(),
SegmentIDs: info.GetSegmentIDs(),
TaskID: info.GetTaskID(),
Version: info.GetVersion(),
NodeID: info.GetNodeID(),
State: info.GetState(),
FailReason: info.GetFailReason(),
CentroidsFile: info.GetCentroidsFile(),
SegmentOffsetMappingFiles: info.GetSegmentOffsetMappingFiles(),
Dim: info.GetDim(),
}
}
func MarshalAnalysisTask(t *AnalysisTask) *indexpb.AnalysisTask {
if t == nil {
return nil
}
return &indexpb.AnalysisTask{
CollectionID: t.CollectionID,
PartitionID: t.PartitionID,
FieldID: t.FieldID,
FieldName: t.FieldName,
FieldType: t.FieldType,
TaskID: t.TaskID,
Version: t.Version,
SegmentIDs: t.SegmentIDs,
NodeID: t.NodeID,
State: t.State,
FailReason: t.FailReason,
CentroidsFile: t.CentroidsFile,
SegmentOffsetMappingFiles: t.SegmentOffsetMappingFiles,
Dim: t.Dim,
}
}
func CloneAnalysisTask(t *AnalysisTask) *AnalysisTask {
if t == nil {
return t
}
return &AnalysisTask{
TenantID: t.TenantID,
CollectionID: t.CollectionID,
PartitionID: t.PartitionID,
FieldID: t.FieldID,
FieldName: t.FieldName,
FieldType: t.FieldType,
SegmentIDs: t.SegmentIDs,
TaskID: t.TaskID,
Version: t.Version,
NodeID: t.NodeID,
State: t.State,
FailReason: t.FailReason,
CentroidsFile: t.CentroidsFile,
SegmentOffsetMappingFiles: t.SegmentOffsetMappingFiles,
Dim: t.Dim,
}
}

View File

@ -30,6 +30,61 @@ func (_m *MockIndexNode) EXPECT() *MockIndexNode_Expecter {
return &MockIndexNode_Expecter{mock: &_m.Mock}
}
// Analysis provides a mock function with given fields: _a0, _a1
func (_m *MockIndexNode) Analysis(_a0 context.Context, _a1 *indexpb.AnalysisRequest) (*commonpb.Status, error) {
ret := _m.Called(_a0, _a1)
var r0 *commonpb.Status
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.AnalysisRequest) (*commonpb.Status, error)); ok {
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.AnalysisRequest) *commonpb.Status); ok {
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.AnalysisRequest) error); ok {
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNode_Analysis_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Analysis'
type MockIndexNode_Analysis_Call struct {
*mock.Call
}
// Analysis is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *indexpb.AnalysisRequest
func (_e *MockIndexNode_Expecter) Analysis(_a0 interface{}, _a1 interface{}) *MockIndexNode_Analysis_Call {
return &MockIndexNode_Analysis_Call{Call: _e.mock.On("Analysis", _a0, _a1)}
}
func (_c *MockIndexNode_Analysis_Call) Run(run func(_a0 context.Context, _a1 *indexpb.AnalysisRequest)) *MockIndexNode_Analysis_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*indexpb.AnalysisRequest))
})
return _c
}
func (_c *MockIndexNode_Analysis_Call) Return(_a0 *commonpb.Status, _a1 error) *MockIndexNode_Analysis_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNode_Analysis_Call) RunAndReturn(run func(context.Context, *indexpb.AnalysisRequest) (*commonpb.Status, error)) *MockIndexNode_Analysis_Call {
_c.Call.Return(run)
return _c
}
// CreateJob provides a mock function with given fields: _a0, _a1
func (_m *MockIndexNode) CreateJob(_a0 context.Context, _a1 *indexpb.CreateJobRequest) (*commonpb.Status, error) {
ret := _m.Called(_a0, _a1)
@ -85,6 +140,61 @@ func (_c *MockIndexNode_CreateJob_Call) RunAndReturn(run func(context.Context, *
return _c
}
// DropAnalysisTasks provides a mock function with given fields: _a0, _a1
func (_m *MockIndexNode) DropAnalysisTasks(_a0 context.Context, _a1 *indexpb.DropAnalysisTasksRequest) (*commonpb.Status, error) {
ret := _m.Called(_a0, _a1)
var r0 *commonpb.Status
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.DropAnalysisTasksRequest) (*commonpb.Status, error)); ok {
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.DropAnalysisTasksRequest) *commonpb.Status); ok {
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.DropAnalysisTasksRequest) error); ok {
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNode_DropAnalysisTasks_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'DropAnalysisTasks'
type MockIndexNode_DropAnalysisTasks_Call struct {
*mock.Call
}
// DropAnalysisTasks is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *indexpb.DropAnalysisTasksRequest
func (_e *MockIndexNode_Expecter) DropAnalysisTasks(_a0 interface{}, _a1 interface{}) *MockIndexNode_DropAnalysisTasks_Call {
return &MockIndexNode_DropAnalysisTasks_Call{Call: _e.mock.On("DropAnalysisTasks", _a0, _a1)}
}
func (_c *MockIndexNode_DropAnalysisTasks_Call) Run(run func(_a0 context.Context, _a1 *indexpb.DropAnalysisTasksRequest)) *MockIndexNode_DropAnalysisTasks_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*indexpb.DropAnalysisTasksRequest))
})
return _c
}
func (_c *MockIndexNode_DropAnalysisTasks_Call) Return(_a0 *commonpb.Status, _a1 error) *MockIndexNode_DropAnalysisTasks_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNode_DropAnalysisTasks_Call) RunAndReturn(run func(context.Context, *indexpb.DropAnalysisTasksRequest) (*commonpb.Status, error)) *MockIndexNode_DropAnalysisTasks_Call {
_c.Call.Return(run)
return _c
}
// DropJobs provides a mock function with given fields: _a0, _a1
func (_m *MockIndexNode) DropJobs(_a0 context.Context, _a1 *indexpb.DropJobsRequest) (*commonpb.Status, error) {
ret := _m.Called(_a0, _a1)
@ -442,6 +552,61 @@ func (_c *MockIndexNode_Init_Call) RunAndReturn(run func() error) *MockIndexNode
return _c
}
// QueryAnalysisResult provides a mock function with given fields: _a0, _a1
func (_m *MockIndexNode) QueryAnalysisResult(_a0 context.Context, _a1 *indexpb.QueryAnalysisResultRequest) (*indexpb.QueryAnalysisResultResponse, error) {
ret := _m.Called(_a0, _a1)
var r0 *indexpb.QueryAnalysisResultResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.QueryAnalysisResultRequest) (*indexpb.QueryAnalysisResultResponse, error)); ok {
return rf(_a0, _a1)
}
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.QueryAnalysisResultRequest) *indexpb.QueryAnalysisResultResponse); ok {
r0 = rf(_a0, _a1)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*indexpb.QueryAnalysisResultResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.QueryAnalysisResultRequest) error); ok {
r1 = rf(_a0, _a1)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNode_QueryAnalysisResult_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'QueryAnalysisResult'
type MockIndexNode_QueryAnalysisResult_Call struct {
*mock.Call
}
// QueryAnalysisResult is a helper method to define mock.On call
// - _a0 context.Context
// - _a1 *indexpb.QueryAnalysisResultRequest
func (_e *MockIndexNode_Expecter) QueryAnalysisResult(_a0 interface{}, _a1 interface{}) *MockIndexNode_QueryAnalysisResult_Call {
return &MockIndexNode_QueryAnalysisResult_Call{Call: _e.mock.On("QueryAnalysisResult", _a0, _a1)}
}
func (_c *MockIndexNode_QueryAnalysisResult_Call) Run(run func(_a0 context.Context, _a1 *indexpb.QueryAnalysisResultRequest)) *MockIndexNode_QueryAnalysisResult_Call {
_c.Call.Run(func(args mock.Arguments) {
run(args[0].(context.Context), args[1].(*indexpb.QueryAnalysisResultRequest))
})
return _c
}
func (_c *MockIndexNode_QueryAnalysisResult_Call) Return(_a0 *indexpb.QueryAnalysisResultResponse, _a1 error) *MockIndexNode_QueryAnalysisResult_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNode_QueryAnalysisResult_Call) RunAndReturn(run func(context.Context, *indexpb.QueryAnalysisResultRequest) (*indexpb.QueryAnalysisResultResponse, error)) *MockIndexNode_QueryAnalysisResult_Call {
_c.Call.Return(run)
return _c
}
// QueryJobs provides a mock function with given fields: _a0, _a1
func (_m *MockIndexNode) QueryJobs(_a0 context.Context, _a1 *indexpb.QueryJobsRequest) (*indexpb.QueryJobsResponse, error) {
ret := _m.Called(_a0, _a1)

View File

@ -31,6 +31,76 @@ func (_m *MockIndexNodeClient) EXPECT() *MockIndexNodeClient_Expecter {
return &MockIndexNodeClient_Expecter{mock: &_m.Mock}
}
// Analysis provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) Analysis(ctx context.Context, in *indexpb.AnalysisRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
_va := make([]interface{}, len(opts))
for _i := range opts {
_va[_i] = opts[_i]
}
var _ca []interface{}
_ca = append(_ca, ctx, in)
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
var r0 *commonpb.Status
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.AnalysisRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.AnalysisRequest, ...grpc.CallOption) *commonpb.Status); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.AnalysisRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNodeClient_Analysis_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'Analysis'
type MockIndexNodeClient_Analysis_Call struct {
*mock.Call
}
// Analysis is a helper method to define mock.On call
// - ctx context.Context
// - in *indexpb.AnalysisRequest
// - opts ...grpc.CallOption
func (_e *MockIndexNodeClient_Expecter) Analysis(ctx interface{}, in interface{}, opts ...interface{}) *MockIndexNodeClient_Analysis_Call {
return &MockIndexNodeClient_Analysis_Call{Call: _e.mock.On("Analysis",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockIndexNodeClient_Analysis_Call) Run(run func(ctx context.Context, in *indexpb.AnalysisRequest, opts ...grpc.CallOption)) *MockIndexNodeClient_Analysis_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]grpc.CallOption, len(args)-2)
for i, a := range args[2:] {
if a != nil {
variadicArgs[i] = a.(grpc.CallOption)
}
}
run(args[0].(context.Context), args[1].(*indexpb.AnalysisRequest), variadicArgs...)
})
return _c
}
func (_c *MockIndexNodeClient_Analysis_Call) Return(_a0 *commonpb.Status, _a1 error) *MockIndexNodeClient_Analysis_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNodeClient_Analysis_Call) RunAndReturn(run func(context.Context, *indexpb.AnalysisRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockIndexNodeClient_Analysis_Call {
_c.Call.Return(run)
return _c
}
// Close provides a mock function with given fields:
func (_m *MockIndexNodeClient) Close() error {
ret := _m.Called()
@ -142,6 +212,76 @@ func (_c *MockIndexNodeClient_CreateJob_Call) RunAndReturn(run func(context.Cont
return _c
}
// DropAnalysisTasks provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) DropAnalysisTasks(ctx context.Context, in *indexpb.DropAnalysisTasksRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
_va := make([]interface{}, len(opts))
for _i := range opts {
_va[_i] = opts[_i]
}
var _ca []interface{}
_ca = append(_ca, ctx, in)
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
var r0 *commonpb.Status
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.DropAnalysisTasksRequest, ...grpc.CallOption) (*commonpb.Status, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.DropAnalysisTasksRequest, ...grpc.CallOption) *commonpb.Status); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*commonpb.Status)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.DropAnalysisTasksRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNodeClient_DropAnalysisTasks_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'DropAnalysisTasks'
type MockIndexNodeClient_DropAnalysisTasks_Call struct {
*mock.Call
}
// DropAnalysisTasks is a helper method to define mock.On call
// - ctx context.Context
// - in *indexpb.DropAnalysisTasksRequest
// - opts ...grpc.CallOption
func (_e *MockIndexNodeClient_Expecter) DropAnalysisTasks(ctx interface{}, in interface{}, opts ...interface{}) *MockIndexNodeClient_DropAnalysisTasks_Call {
return &MockIndexNodeClient_DropAnalysisTasks_Call{Call: _e.mock.On("DropAnalysisTasks",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockIndexNodeClient_DropAnalysisTasks_Call) Run(run func(ctx context.Context, in *indexpb.DropAnalysisTasksRequest, opts ...grpc.CallOption)) *MockIndexNodeClient_DropAnalysisTasks_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]grpc.CallOption, len(args)-2)
for i, a := range args[2:] {
if a != nil {
variadicArgs[i] = a.(grpc.CallOption)
}
}
run(args[0].(context.Context), args[1].(*indexpb.DropAnalysisTasksRequest), variadicArgs...)
})
return _c
}
func (_c *MockIndexNodeClient_DropAnalysisTasks_Call) Return(_a0 *commonpb.Status, _a1 error) *MockIndexNodeClient_DropAnalysisTasks_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNodeClient_DropAnalysisTasks_Call) RunAndReturn(run func(context.Context, *indexpb.DropAnalysisTasksRequest, ...grpc.CallOption) (*commonpb.Status, error)) *MockIndexNodeClient_DropAnalysisTasks_Call {
_c.Call.Return(run)
return _c
}
// DropJobs provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) DropJobs(ctx context.Context, in *indexpb.DropJobsRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
_va := make([]interface{}, len(opts))
@ -492,6 +632,76 @@ func (_c *MockIndexNodeClient_GetStatisticsChannel_Call) RunAndReturn(run func(c
return _c
}
// QueryAnalysisResult provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) QueryAnalysisResult(ctx context.Context, in *indexpb.QueryAnalysisResultRequest, opts ...grpc.CallOption) (*indexpb.QueryAnalysisResultResponse, error) {
_va := make([]interface{}, len(opts))
for _i := range opts {
_va[_i] = opts[_i]
}
var _ca []interface{}
_ca = append(_ca, ctx, in)
_ca = append(_ca, _va...)
ret := _m.Called(_ca...)
var r0 *indexpb.QueryAnalysisResultResponse
var r1 error
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.QueryAnalysisResultRequest, ...grpc.CallOption) (*indexpb.QueryAnalysisResultResponse, error)); ok {
return rf(ctx, in, opts...)
}
if rf, ok := ret.Get(0).(func(context.Context, *indexpb.QueryAnalysisResultRequest, ...grpc.CallOption) *indexpb.QueryAnalysisResultResponse); ok {
r0 = rf(ctx, in, opts...)
} else {
if ret.Get(0) != nil {
r0 = ret.Get(0).(*indexpb.QueryAnalysisResultResponse)
}
}
if rf, ok := ret.Get(1).(func(context.Context, *indexpb.QueryAnalysisResultRequest, ...grpc.CallOption) error); ok {
r1 = rf(ctx, in, opts...)
} else {
r1 = ret.Error(1)
}
return r0, r1
}
// MockIndexNodeClient_QueryAnalysisResult_Call is a *mock.Call that shadows Run/Return methods with type explicit version for method 'QueryAnalysisResult'
type MockIndexNodeClient_QueryAnalysisResult_Call struct {
*mock.Call
}
// QueryAnalysisResult is a helper method to define mock.On call
// - ctx context.Context
// - in *indexpb.QueryAnalysisResultRequest
// - opts ...grpc.CallOption
func (_e *MockIndexNodeClient_Expecter) QueryAnalysisResult(ctx interface{}, in interface{}, opts ...interface{}) *MockIndexNodeClient_QueryAnalysisResult_Call {
return &MockIndexNodeClient_QueryAnalysisResult_Call{Call: _e.mock.On("QueryAnalysisResult",
append([]interface{}{ctx, in}, opts...)...)}
}
func (_c *MockIndexNodeClient_QueryAnalysisResult_Call) Run(run func(ctx context.Context, in *indexpb.QueryAnalysisResultRequest, opts ...grpc.CallOption)) *MockIndexNodeClient_QueryAnalysisResult_Call {
_c.Call.Run(func(args mock.Arguments) {
variadicArgs := make([]grpc.CallOption, len(args)-2)
for i, a := range args[2:] {
if a != nil {
variadicArgs[i] = a.(grpc.CallOption)
}
}
run(args[0].(context.Context), args[1].(*indexpb.QueryAnalysisResultRequest), variadicArgs...)
})
return _c
}
func (_c *MockIndexNodeClient_QueryAnalysisResult_Call) Return(_a0 *indexpb.QueryAnalysisResultResponse, _a1 error) *MockIndexNodeClient_QueryAnalysisResult_Call {
_c.Call.Return(_a0, _a1)
return _c
}
func (_c *MockIndexNodeClient_QueryAnalysisResult_Call) RunAndReturn(run func(context.Context, *indexpb.QueryAnalysisResultRequest, ...grpc.CallOption) (*indexpb.QueryAnalysisResultResponse, error)) *MockIndexNodeClient_QueryAnalysisResult_Call {
_c.Call.Return(run)
return _c
}
// QueryJobs provides a mock function with given fields: ctx, in, opts
func (_m *MockIndexNodeClient) QueryJobs(ctx context.Context, in *indexpb.QueryJobsRequest, opts ...grpc.CallOption) (*indexpb.QueryJobsResponse, error) {
_va := make([]interface{}, len(opts))

View File

@ -25,7 +25,7 @@ enum SegmentLevel {
Legacy = 0; // zero value for legacy logic
L0 = 1; // L0 segment, contains delta data for current channel
L1 = 2; // L1 segment, normal segment, with no extra compaction attribute
L2 = 3; // L2 segemnt, segment with extra data distribution info
L2 = 3; // L2 segment, segment with extra data distribution info
}
service DataCoord {
@ -483,6 +483,7 @@ enum CompactionType {
MinorCompaction = 5;
MajorCompaction = 6;
Level0DeleteCompaction = 7;
ClusteringCompaction = 8;
}
message CompactionStateRequest {
@ -521,6 +522,11 @@ message CompactionPlan {
string channel = 7;
int64 collection_ttl = 8;
int64 total_rows = 9;
int64 clustering_key_id = 10;
int64 max_segment_rows = 11;
int64 prefer_segment_rows = 12;
string analyze_result_path = 13;
repeated int64 analyze_segment_ids = 14;
}
message CompactionSegment {
@ -825,3 +831,30 @@ message GcControlRequest {
GcCommand command = 2;
repeated common.KeyValuePair params = 3;
}
enum CompactionTaskState {
undefined = 0;
executing = 1;
pipelining = 2;
completed = 3;
failed = 4;
timeout = 5;
analyzing = 6;
}
message ClusteringCompactionInfo {
int64 triggerID = 1;
int64 collectionID = 2;
int64 clusteringKeyID = 3;
string clusteringKeyName = 4;
schema.DataType clusteringKeyType = 5;
CompactionTaskState state = 6;
uint64 startTime = 7;
uint64 lastUpdateTime = 8;
repeated CompactionPlan pipeliningPlans = 9;
repeated CompactionPlan executingPlans = 10;
repeated CompactionPlan completedPlans = 11;
repeated CompactionPlan failedPlans = 12;
repeated CompactionPlan timeoutPlans = 13;
int64 analysisTaskID = 14;
}

View File

@ -60,6 +60,10 @@ service IndexNode {
rpc GetMetrics(milvus.GetMetricsRequest)
returns (milvus.GetMetricsResponse) {
}
rpc Analysis(AnalysisRequest) returns (common.Status) {}
rpc QueryAnalysisResult(QueryAnalysisResultRequest) returns (QueryAnalysisResultResponse) {}
rpc DropAnalysisTasks(DropAnalysisTasksRequest) returns (common.Status) {}
}
message IndexInfo {
@ -344,3 +348,64 @@ message ListIndexesResponse {
common.Status status = 1;
repeated IndexInfo index_infos = 2;
}
message AnalysisTask {
int64 collectionID = 1;
int64 partitionID = 2;
int64 fieldID = 3;
string field_name = 4;
schema.DataType field_type = 5;
int64 taskID = 6;
int64 version = 7;
repeated int64 segmentIDs = 8;
int64 nodeID = 9;
common.IndexState state = 10;
string fail_reason = 11;
string centroids_file = 12;
map<int64, string> segment_offset_mapping_files = 13;
int64 dim = 14;
}
message SegmentStats {
int64 ID = 1;
int64 num_rows = 2;
repeated int64 logIDs = 3;
}
message AnalysisRequest {
string clusterID = 1;
int64 taskID =2;
int64 collectionID = 3;
int64 partitionID = 4;
int64 fieldID = 5;
string fieldName = 6;
schema.DataType field_type = 7;
map<int64, SegmentStats> segment_stats = 8;
int64 version = 9;
StorageConfig storage_config = 10;
int64 dim = 11;
}
message QueryAnalysisResultRequest {
string clusterID = 1;
repeated int64 taskIDs = 2;
}
message AnalysisResult {
int64 taskID = 1;
common.IndexState state = 2;
string centroids_file = 3;
map<int64, string> segment_offset_mapping_files = 4;
string fail_reason = 5;
}
message QueryAnalysisResultResponse {
common.Status status = 1;
string clusterID = 2;
map<int64, AnalysisResult> results = 3;
}
message DropAnalysisTasksRequest {
string clusterID = 1;
repeated int64 taskIDs = 2;
}

View File

@ -42,4 +42,13 @@ message FieldIndexMeta {
message CollectionIndexMeta {
int64 maxIndexRowCount = 1;
repeated FieldIndexMeta index_metas = 2;
}
}
message ClusteringCentroidsStats {
repeated schema.VectorField centroids = 1;
}
message ClusteringCentroidIdMappingStats {
repeated uint32 centroid_id_mapping = 1;
repeated int64 num_in_centroid = 2;
}

View File

@ -15,12 +15,12 @@ import (
"github.com/milvus-io/milvus/internal/storage"
"github.com/milvus-io/milvus/internal/util/clustering"
"github.com/milvus-io/milvus/internal/util/exprutil"
"github.com/milvus-io/milvus/internal/util/typeutil"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/distance"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/typeutil"
)
const defaultFilterRatio float64 = 0.5
@ -40,11 +40,21 @@ func PruneSegments(ctx context.Context,
log := log.Ctx(ctx)
// 1. calculate filtered segments
filteredSegments := make(map[UniqueID]struct{}, 0)
clusteringKeyField := typeutil.GetClusteringKeyField(schema.Fields)
clusteringKeyField := clustering.GetClusteringKeyField(schema)
if clusteringKeyField == nil {
// no need to prune
return
}
var expr []byte
if searchReq != nil {
expr = searchReq.GetSerializedExprPlan()
} else {
expr = queryReq.GetSerializedExprPlan()
}
// currently we only prune based on one column
if typeutil.IsVectorType(clusteringKeyField.GetDataType()) {
// parse searched vectors
var vectorsHolder commonpb.PlaceholderGroup
err := proto.Unmarshal(searchReq.GetPlaceholderGroup(), &vectorsHolder)
@ -61,14 +71,13 @@ func PruneSegments(ctx context.Context,
if err != nil {
return
}
for _, partID := range searchReq.GetPartitionIDs() {
partStats := partitionStats[partID]
for _, partStats := range partitionStats {
FilterSegmentsByVector(partStats, searchReq, vectorsBytes, dimValue, clusteringKeyField, filteredSegments, info.filterRatio)
}
} else if queryReq != nil {
} else {
// 0. parse expr from plan
plan := planpb.PlanNode{}
err := proto.Unmarshal(queryReq.GetSerializedExprPlan(), &plan)
err := proto.Unmarshal(expr, &plan)
if err != nil {
log.Error("failed to unmarshall serialized expr from bytes, failed the operation")
return
@ -82,20 +91,23 @@ func PruneSegments(ctx context.Context,
if matchALL || targetRanges == nil {
return
}
for _, partID := range queryReq.GetPartitionIDs() {
partStats := partitionStats[partID]
for _, partStats := range partitionStats {
FilterSegmentsOnScalarField(partStats, targetRanges, clusteringKeyField, filteredSegments)
}
}
// 2. remove filtered segments from sealed segment list
if len(filteredSegments) > 0 {
var realFilteredSegments = 0
totalSegNum := 0
for idx, item := range sealedSegments {
newSegments := make([]SegmentEntry, 0)
totalSegNum += len(item.Segments)
for _, segment := range item.Segments {
if _, ok := filteredSegments[segment.SegmentID]; !ok {
_, exist := filteredSegments[segment.SegmentID]
if exist {
realFilteredSegments++
} else {
newSegments = append(newSegments, segment)
}
}
@ -103,7 +115,7 @@ func PruneSegments(ctx context.Context,
sealedSegments[idx] = item
}
log.RatedInfo(30, "Pruned segment for search/query",
zap.Int("filtered_segment_num[excluded]", len(filteredSegments)),
zap.Int("filtered_segment_num[excluded]", realFilteredSegments),
zap.Int("total_segment_num", totalSegNum),
zap.Float32("filtered_rate", float32(len(filteredSegments)/totalSegNum)),
)
@ -152,6 +164,7 @@ func FilterSegmentsByVector(partitionStats *storage.PartitionStatsSnapshot,
}
// currently, we only support float vector and only one center one segment
if disErr != nil {
log.Error("calculate distance error", zap.Error(disErr))
neededSegments[segId] = struct{}{}
break
}
@ -207,10 +220,23 @@ func FilterSegmentsOnScalarField(partitionStats *storage.PartitionStatsSnapshot,
overlap := func(min storage.ScalarFieldValue, max storage.ScalarFieldValue) bool {
for _, tRange := range targetRanges {
switch keyField.DataType {
case schemapb.DataType_Int8, schemapb.DataType_Int16, schemapb.DataType_Int32, schemapb.DataType_Int64:
case schemapb.DataType_Int8:
targetRange := tRange.ToIntRange()
statRange := exprutil.NewIntRange(int64(min.GetValue().(int8)), int64(max.GetValue().(int8)), true, true)
return exprutil.IntRangeOverlap(targetRange, statRange)
case schemapb.DataType_Int16:
targetRange := tRange.ToIntRange()
statRange := exprutil.NewIntRange(int64(min.GetValue().(int16)), int64(max.GetValue().(int16)), true, true)
return exprutil.IntRangeOverlap(targetRange, statRange)
case schemapb.DataType_Int32:
targetRange := tRange.ToIntRange()
statRange := exprutil.NewIntRange(int64(min.GetValue().(int32)), int64(max.GetValue().(int32)), true, true)
return exprutil.IntRangeOverlap(targetRange, statRange)
case schemapb.DataType_Int64:
targetRange := tRange.ToIntRange()
statRange := exprutil.NewIntRange(min.GetValue().(int64), max.GetValue().(int64), true, true)
return exprutil.IntRangeOverlap(targetRange, statRange)
// todo: add float/double pruner
case schemapb.DataType_String, schemapb.DataType_VarChar:
targetRange := tRange.ToStrRange()
statRange := exprutil.NewStrRange(min.GetValue().(string), max.GetValue().(string), true, true)

View File

@ -116,17 +116,21 @@ func (itr *InsertBinlogIterator) Dispose() {
}
func (itr *InsertBinlogIterator) hasNext() bool {
_, ok := itr.data.Data[common.RowIDField]
if !ok {
return false
}
return itr.pos < itr.data.Data[common.RowIDField].RowNum()
return itr.pos < itr.RowNum()
}
func (itr *InsertBinlogIterator) isDisposed() bool {
return atomic.LoadInt32(&itr.dispose) == 1
}
func (itr *InsertBinlogIterator) DataSize() int {
return itr.data.GetMemorySize()
}
func (itr *InsertBinlogIterator) RowNum() int {
return itr.data.GetRowNum()
}
/*
type DeltalogIterator struct {
dispose int32

View File

@ -707,3 +707,15 @@ func TestVectorFieldStatsMarshal(t *testing.T) {
assert.Equal(t, 2, len(stats4.Centroids))
assert.ElementsMatch(t, []VectorFieldValue{centroid, centroid2}, stats4.Centroids)
}
func TestFindMaxVersion(t *testing.T) {
files := []string{"path/1", "path/2", "path/3"}
version, path := FindPartitionStatsMaxVersion(files)
assert.Equal(t, int64(3), version)
assert.Equal(t, "path/3", path)
files2 := []string{}
version2, path2 := FindPartitionStatsMaxVersion(files2)
assert.Equal(t, int64(-1), version2)
assert.Equal(t, "", path2)
}

View File

@ -1013,3 +1013,37 @@ func (ifv *FloatVectorFieldValue) GetValue() interface{} {
func (ifv *FloatVectorFieldValue) Size() int64 {
return int64(len(ifv.Value) * 8)
}
func NewScalarFieldValue(dtype schemapb.DataType, data interface{}) ScalarFieldValue {
switch dtype {
case schemapb.DataType_Int8:
return NewInt8FieldValue(data.(int8))
case schemapb.DataType_Int16:
return NewInt16FieldValue(data.(int16))
case schemapb.DataType_Int32:
return NewInt32FieldValue(data.(int32))
case schemapb.DataType_Int64:
return NewInt64FieldValue(data.(int64))
case schemapb.DataType_Float:
return NewFloatFieldValue(data.(float32))
case schemapb.DataType_Double:
return NewDoubleFieldValue(data.(float64))
case schemapb.DataType_String:
return NewStringFieldValue(data.(string))
case schemapb.DataType_VarChar:
return NewVarCharFieldValue(data.(string))
default:
// should not be reach
panic(fmt.Sprintf("not supported datatype: %s", dtype.String()))
}
}
func NewVectorFieldValue(dtype schemapb.DataType, data *schemapb.VectorField) VectorFieldValue {
switch dtype {
case schemapb.DataType_FloatVector:
return NewFloatVectorFieldValue(data.GetFloatVector().GetData())
default:
// should not be reach
panic(fmt.Sprintf("not supported datatype: %s", dtype.String()))
}
}

View File

@ -16,7 +16,11 @@
package storage
import "encoding/json"
import (
"encoding/json"
"path"
"strconv"
)
type SegmentStats struct {
FieldStats []FieldStats `json:"fieldStats"`
@ -77,3 +81,20 @@ func SerializePartitionStatsSnapshot(partStats *PartitionStatsSnapshot) ([]byte,
}
return partData, nil
}
func FindPartitionStatsMaxVersion(filePaths []string) (int64, string) {
maxVersion := int64(-1)
maxVersionFilePath := ""
for _, filePath := range filePaths {
versionStr := path.Base(filePath)
version, err := strconv.ParseInt(versionStr, 10, 64)
if err != nil {
continue
}
if version > maxVersion {
maxVersion = version
maxVersionFilePath = filePath
}
}
return maxVersion, maxVersionFilePath
}

View File

@ -0,0 +1,99 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package analysiscgowrapper
/*
//libdir=/home/zc/work/milvus/internal/core/output/lib
//includedir=/home/zc/work/milvus/internal/core/output/include
//
//Libs: -L${libdir} -lmilvus_indexbuilder
//Cflags: -I${includedir
#cgo pkg-config: milvus_indexbuilder
#include <stdlib.h> // free
#include "indexbuilder/analysis_c.h"
*/
import "C"
import (
"context"
)
type CodecAnalysis interface {
Delete() error
CleanLocalData() error
UpLoad(segmentIDs []int64) (string, map[int64]string, error)
}
func Analysis(ctx context.Context, analysisInfo *AnalysisInfo) (CodecAnalysis, error) {
var analysisPtr C.CAnalysis
status := C.Analysis(&analysisPtr, analysisInfo.cAnalysisInfo)
if err := HandleCStatus(&status, "failed to analysis task"); err != nil {
return nil, err
}
analysis := &CgoAnalysis{
analysisPtr: analysisPtr,
close: false,
}
return analysis, nil
}
type CgoAnalysis struct {
analysisPtr C.CAnalysis
close bool
}
func (ca *CgoAnalysis) Delete() error {
if ca.close {
return nil
}
// Todo: delete in segcore
status := C.DeleteAnalysis(ca.analysisPtr)
ca.close = true
return HandleCStatus(&status, "failed to delete analysis")
//return nil
}
func (ca *CgoAnalysis) CleanLocalData() error {
status := C.CleanAnalysisLocalData(ca.analysisPtr)
return HandleCStatus(&status, "failed to clean cached data on disk")
}
func (ca *CgoAnalysis) UpLoad(segmentIDs []int64) (string, map[int64]string, error) {
status := C.SerializeAnalysisAndUpLoad(ca.analysisPtr)
if err := HandleCStatus(&status, "failed to upload analysis result"); err != nil {
return "", nil, err
}
centroidsFile, err := GetCentroidsFile()
if err != nil {
return "", nil, err
}
segmentsOffsetMapping := make(map[int64]string)
for _, segID := range segmentIDs {
offsetMappingFile, err := GetSegmentOffsetMapping(segID)
if err != nil {
return "", nil, err
}
segmentsOffsetMapping[segID] = offsetMappingFile
}
return centroidsFile, segmentsOffsetMapping, nil
}

View File

@ -0,0 +1,133 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package analysiscgowrapper
/*
#cgo pkg-config: milvus_indexbuilder
#include <stdlib.h> // free
#include "indexbuilder/analysis_c.h"
*/
import "C"
import (
"unsafe"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/internal/proto/indexpb"
)
type AnalysisInfo struct {
cAnalysisInfo C.CAnalysisInfo
}
func NewAnalysisInfo(config *indexpb.StorageConfig) (*AnalysisInfo, error) {
var cAnalysisInfo C.CAnalysisInfo
cAddress := C.CString(config.Address)
cBucketName := C.CString(config.BucketName)
cAccessKey := C.CString(config.AccessKeyID)
cAccessValue := C.CString(config.SecretAccessKey)
cRootPath := C.CString(config.RootPath)
cStorageType := C.CString(config.StorageType)
cIamEndPoint := C.CString(config.IAMEndpoint)
cRegion := C.CString(config.Region)
cCloudProvider := C.CString(config.CloudProvider)
defer C.free(unsafe.Pointer(cAddress))
defer C.free(unsafe.Pointer(cBucketName))
defer C.free(unsafe.Pointer(cAccessKey))
defer C.free(unsafe.Pointer(cAccessValue))
defer C.free(unsafe.Pointer(cRootPath))
defer C.free(unsafe.Pointer(cStorageType))
defer C.free(unsafe.Pointer(cIamEndPoint))
defer C.free(unsafe.Pointer(cRegion))
defer C.free(unsafe.Pointer(cCloudProvider))
storageConfig := C.CStorageConfig{
address: cAddress,
bucket_name: cBucketName,
access_key_id: cAccessKey,
access_key_value: cAccessValue,
root_path: cRootPath,
storage_type: cStorageType,
iam_endpoint: cIamEndPoint,
cloud_provider: cCloudProvider,
useSSL: C.bool(config.UseSSL),
useIAM: C.bool(config.UseIAM),
region: cRegion,
useVirtualHost: C.bool(config.UseVirtualHost),
requestTimeoutMs: C.int64_t(config.RequestTimeoutMs),
}
status := C.NewAnalysisInfo(&cAnalysisInfo, storageConfig)
if err := HandleCStatus(&status, "NewAnalysisInfo failed"); err != nil {
return nil, err
}
return &AnalysisInfo{cAnalysisInfo: cAnalysisInfo}, nil
}
func DeleteAnalysisInfo(info *AnalysisInfo) {
C.DeleteAnalysisInfo(info.cAnalysisInfo)
}
func (ai *AnalysisInfo) AppendAnalysisFieldMetaInfo(collectionID int64, partitionID int64, fieldID int64, fieldType schemapb.DataType, fieldName string, dim int64) error {
cColID := C.int64_t(collectionID)
cParID := C.int64_t(partitionID)
cFieldID := C.int64_t(fieldID)
cintDType := uint32(fieldType)
cFieldName := C.CString(fieldName)
cDim := C.int64_t(dim)
defer C.free(unsafe.Pointer(cFieldName))
status := C.AppendAnalysisFieldMetaInfo(ai.cAnalysisInfo, cColID, cParID, cFieldID, cFieldName, cintDType, cDim)
return HandleCStatus(&status, "appendFieldMetaInfo failed")
}
func (ai *AnalysisInfo) AppendAnalysisInfo(taskID int64, version int64) error {
cTaskID := C.int64_t(taskID)
cVersion := C.int64_t(version)
status := C.AppendAnalysisInfo(ai.cAnalysisInfo, cTaskID, cVersion)
return HandleCStatus(&status, "appendAnalysisMetaInfo failed")
}
func (ai *AnalysisInfo) AppendSegmentID(segID int64) error {
cSegID := C.int64_t(segID)
status := C.AppendSegmentID(ai.cAnalysisInfo, cSegID)
return HandleCStatus(&status, "appendAnalysisSegmentID failed")
}
func (ai *AnalysisInfo) AppendSegmentInsertFile(segID int64, filePath string) error {
cSegID := C.int64_t(segID)
cInsertFilePath := C.CString(filePath)
defer C.free(unsafe.Pointer(cInsertFilePath))
status := C.AppendSegmentInsertFile(ai.cAnalysisInfo, cSegID, cInsertFilePath)
return HandleCStatus(&status, "appendInsertFile failed")
}
func (ai *AnalysisInfo) AppendSegmentSize(size int64) error {
cSize := C.int64_t(size)
status := C.AppendSegmentSize(ai.cAnalysisInfo, cSize)
return HandleCStatus(&status, "appendSegmentSize failed")
}
func (ai *AnalysisInfo) AppendTrainSize(size int64) error {
cSize := C.int64_t(size)
status := C.AppendTrainSize(ai.cAnalysisInfo, cSize)
return HandleCStatus(&status, "appendTrainSize failed")
}

View File

@ -0,0 +1,63 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package analysiscgowrapper
/*
#cgo pkg-config: milvus_indexbuilder
#include <stdlib.h> // free
#include "indexbuilder/analysis_c.h"
*/
import "C"
import (
"fmt"
"unsafe"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/merr"
)
// HandleCStatus deal with the error returned from CGO
func HandleCStatus(status *C.CStatus, extraInfo string) error {
if status.error_code == 0 {
return nil
}
errorCode := int(status.error_code)
errorMsg := C.GoString(status.error_msg)
defer C.free(unsafe.Pointer(status.error_msg))
logMsg := fmt.Sprintf("%s, C Runtime Exception: %s\n", extraInfo, errorMsg)
log.Warn(logMsg)
return merr.WrapErrSegcore(int32(errorCode), logMsg)
}
func GetCentroidsFile() (string, error) {
// need malloc size
//var path unsafe.Pointer
//return C.GoString((*C.char)(C.GetCentroidsFile(path))), nil
return "centroids", nil
}
func GetSegmentOffsetMapping(segID int64) (string, error) {
// need malloc size
//var path unsafe.Pointer
//cSegID := C.int64_t(segID)
//return C.GoString((*C.char)(C.GetSegmentOffsetMapping(cSegID, path))), nil
return fmt.Sprintf("offsets-mapping-%d", segID), nil
}

View File

@ -7,6 +7,7 @@ import (
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/pkg/util/distance"
"github.com/milvus-io/milvus/pkg/util/merr"
"github.com/milvus-io/milvus/pkg/util/paramtable"
)
func CalcVectorDistance(dim int64, dataType schemapb.DataType, left []byte, right []float32, metric string) ([]float32, error) {
@ -48,3 +49,32 @@ func SerializeFloatVector(fv []float32) []byte {
}
return data
}
func GetClusteringKeyField(collectionSchema *schemapb.CollectionSchema) *schemapb.FieldSchema {
var clusteringKeyField *schemapb.FieldSchema
var partitionKeyField *schemapb.FieldSchema
vectorFields := make([]*schemapb.FieldSchema, 0)
for _, field := range collectionSchema.GetFields() {
if field.IsClusteringKey {
clusteringKeyField = field
}
if field.IsPartitionKey {
partitionKeyField = field
}
// todo support other vector type
// if typeutil.IsVectorType(field.GetDataType()) {
if field.DataType == schemapb.DataType_FloatVector {
vectorFields = append(vectorFields, field)
}
}
// in some server mode, we regard partition key field or vector field as clustering key by default.
// here is the priority: clusteringKey > partitionKey > vector field(only single vector)
if clusteringKeyField != nil {
return clusteringKeyField
} else if paramtable.Get().CommonCfg.UsePartitionKeyAsClusteringKey.GetAsBool() && partitionKeyField != nil {
return partitionKeyField
} else if paramtable.Get().CommonCfg.UseVectorAsClusteringKey.GetAsBool() && len(vectorFields) == 1 {
return vectorFields[0]
}
return nil
}

View File

@ -69,6 +69,18 @@ func (m *GrpcIndexNodeClient) ShowConfigurations(ctx context.Context, in *intern
return &internalpb.ShowConfigurationsResponse{}, m.Err
}
func (m *GrpcIndexNodeClient) Analysis(ctx context.Context, in *indexpb.AnalysisRequest, opt ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}
func (m *GrpcIndexNodeClient) QueryAnalysisResult(ctx context.Context, in *indexpb.QueryAnalysisResultRequest, opt ...grpc.CallOption) (*indexpb.QueryAnalysisResultResponse, error) {
return &indexpb.QueryAnalysisResultResponse{}, m.Err
}
func (m *GrpcIndexNodeClient) DropAnalysisTasks(ctx context.Context, in *indexpb.DropAnalysisTasksRequest, opt ...grpc.CallOption) (*commonpb.Status, error) {
return &commonpb.Status{}, m.Err
}
func (m *GrpcIndexNodeClient) Close() error {
return m.Err
}

View File

@ -128,12 +128,3 @@ func convertToArrowType(dataType schemapb.DataType) (arrow.DataType, error) {
return nil, merr.WrapErrParameterInvalidMsg("unknown type %v", dataType.String())
}
}
func GetClusteringKeyField(fields []*schemapb.FieldSchema) *schemapb.FieldSchema {
for _, field := range fields {
if field.IsClusteringKey {
return field
}
}
return nil
}

View File

@ -91,6 +91,12 @@ const (
// PartitionStatsPath storage path const for partition stats files
PartitionStatsPath = `part_stats`
// PartitionStatsTempPath storage path const for temporary partition stats files
PartitionStatsTempPath = `part_stats_temp`
// AnalyzeStatsPath storage path const for analyze.
AnalyzeStatsPath = `filesanalyze_stats`
)
// Search, Index parameter keys

View File

@ -158,6 +158,12 @@ var (
// Search/Query related
ErrInconsistentRequery = newMilvusError("inconsistent requery result", 2200, true)
// Compaction
ErrCompactionReadDeltaLogErr = newMilvusError("fail to read delta log", 2300, false)
ErrClusteringCompactionClusterNotSupport = newMilvusError("milvus cluster not support clustering compaction", 2301, false)
ErrClusteringCompactionCollectionNotSupport = newMilvusError("collection not support clustering compaction", 2302, false)
ErrClusteringCompactionCollectionIsCompacting = newMilvusError("collection is compacting", 2303, false)
)
type milvusError struct {

View File

@ -959,3 +959,35 @@ func WrapErrInconsistentRequery(msg ...string) error {
}
return err
}
func WrapErrCompactionReadDeltaLogErr(msg ...string) error {
err := error(ErrCompactionReadDeltaLogErr)
if len(msg) > 0 {
err = errors.Wrap(err, strings.Join(msg, "->"))
}
return err
}
func WrapErrClusteringCompactionClusterNotSupport(msg ...string) error {
err := error(ErrClusteringCompactionClusterNotSupport)
if len(msg) > 0 {
err = errors.Wrap(err, strings.Join(msg, "->"))
}
return err
}
func WrapErrClusteringCompactionCollectionNotSupport(msg ...string) error {
err := error(ErrClusteringCompactionCollectionNotSupport)
if len(msg) > 0 {
err = errors.Wrap(err, strings.Join(msg, "->"))
}
return err
}
func WrapErrClusteringCompactionCollectionIsCompacting(msg ...string) error {
err := error(ErrClusteringCompactionCollectionIsCompacting)
if len(msg) > 0 {
err = errors.Wrap(err, strings.Join(msg, "->"))
}
return err
}

View File

@ -244,6 +244,9 @@ type commonConfig struct {
TraceLogMode ParamItem `refreshable:"true"`
BloomFilterSize ParamItem `refreshable:"true"`
MaxBloomFalsePositive ParamItem `refreshable:"true"`
UsePartitionKeyAsClusteringKey ParamItem `refreshable:"true"`
UseVectorAsClusteringKey ParamItem `refreshable:"true"`
}
func (p *commonConfig) init(base *BaseTable) {
@ -717,6 +720,22 @@ like the old password verification when updating the credential`,
Doc: "max false positive rate for bloom filter",
}
p.MaxBloomFalsePositive.Init(base.mgr)
p.UsePartitionKeyAsClusteringKey = ParamItem{
Key: "common.usePartitionKeyAsClusteringKey",
Version: "2.4.0",
Doc: "if true, do clustering compaction and segment prune on partition key field",
DefaultValue: "false",
}
p.UsePartitionKeyAsClusteringKey.Init(base.mgr)
p.UseVectorAsClusteringKey = ParamItem{
Key: "common.useVectorAsClusteringKey",
Version: "2.4.0",
Doc: "if true, do clustering compaction and segment prune on vector field",
DefaultValue: "false",
}
p.UseVectorAsClusteringKey.Init(base.mgr)
}
type gpuConfig struct {
@ -2540,6 +2559,21 @@ type dataCoordConfig struct {
GlobalCompactionInterval ParamItem `refreshable:"false"`
ChannelCheckpointMaxLag ParamItem `refreshable:"true"`
// Clustering Compaction
ClusteringCompactionEnable ParamItem `refreshable:"true"`
ClusteringCompactionAutoEnable ParamItem `refreshable:"true"`
ClusteringCompactionInterval ParamItem `refreshable:"false"`
ClusteringCompactionStateCheckInterval ParamItem `refreshable:"true"`
ClusteringCompactionMinInterval ParamItem `refreshable:"true"`
ClusteringCompactionMaxInterval ParamItem `refreshable:"true"`
ClusteringCompactionNewDataRatioThreshold ParamItem `refreshable:"true"`
ClusteringCompactionNewDataSizeThreshold ParamItem `refreshable:"true"`
ClusteringCompactionDropTolerance ParamItem `refreshable:"true"`
ClusteringCompactionPreferSegmentSize ParamItem `refreshable:"true"`
ClusteringCompactionMaxSegmentSize ParamItem `refreshable:"true"`
ClusteringCompactionMaxTrainSize ParamItem `refreshable:"true"`
ClusteringCompactionTimeoutInSeconds ParamItem `refreshable:"true"`
// LevelZero Segment
EnableLevelZeroSegment ParamItem `refreshable:"false"`
LevelZeroCompactionTriggerMinSize ParamItem `refreshable:"true"`
@ -2901,6 +2935,112 @@ During compaction, the size of segment # of rows is able to exceed segment max #
}
p.LevelZeroCompactionTriggerDeltalogMaxNum.Init(base.mgr)
p.ClusteringCompactionEnable = ParamItem{
Key: "dataCoord.compaction.clustering.enable",
Version: "2.4.0",
DefaultValue: "false",
Doc: "Enable clustering compaction",
Export: true,
}
p.ClusteringCompactionEnable.Init(base.mgr)
p.ClusteringCompactionAutoEnable = ParamItem{
Key: "dataCoord.compaction.clustering.autoEnable",
Version: "2.4.0",
DefaultValue: "false",
Doc: "Enable auto clustering compaction",
Export: true,
}
p.ClusteringCompactionAutoEnable.Init(base.mgr)
p.ClusteringCompactionInterval = ParamItem{
Key: "dataCoord.compaction.clustering.interval",
Version: "2.4.0",
DefaultValue: "600",
}
p.ClusteringCompactionInterval.Init(base.mgr)
p.ClusteringCompactionStateCheckInterval = ParamItem{
Key: "dataCoord.compaction.clustering.stateCheckInterval",
Version: "2.4.0",
DefaultValue: "10",
}
p.ClusteringCompactionStateCheckInterval.Init(base.mgr)
p.ClusteringCompactionMinInterval = ParamItem{
Key: "dataCoord.compaction.clustering.minInterval",
Version: "2.4.0",
Doc: "The minimum interval between clustering compaction executions of one collection, to avoid redundant compaction",
DefaultValue: "3600",
}
p.ClusteringCompactionMinInterval.Init(base.mgr)
p.ClusteringCompactionMaxInterval = ParamItem{
Key: "dataCoord.compaction.clustering.maxInterval",
Version: "2.4.0",
Doc: "If a collection haven't been clustering compacted for longer than maxInterval, force compact",
DefaultValue: "86400",
}
p.ClusteringCompactionMaxInterval.Init(base.mgr)
p.ClusteringCompactionNewDataRatioThreshold = ParamItem{
Key: "dataCoord.compaction.clustering.newDataRatioThreshold",
Version: "2.4.0",
Doc: "If new data ratio is large than newDataRatioThreshold, execute clustering compaction",
DefaultValue: "0.2",
}
p.ClusteringCompactionNewDataRatioThreshold.Init(base.mgr)
p.ClusteringCompactionNewDataSizeThreshold = ParamItem{
Key: "dataCoord.compaction.clustering.newDataSizeThreshold",
Version: "2.4.0",
Doc: "If new data size is large than newDataSizeThreshold, execute clustering compaction",
DefaultValue: "512m",
}
p.ClusteringCompactionNewDataSizeThreshold.Init(base.mgr)
p.ClusteringCompactionTimeoutInSeconds = ParamItem{
Key: "dataCoord.compaction.clustering.timeout",
Version: "2.4.0",
DefaultValue: "3600",
}
p.ClusteringCompactionTimeoutInSeconds.Init(base.mgr)
p.ClusteringCompactionDropTolerance = ParamItem{
Key: "dataCoord.compaction.clustering.dropTolerance",
Version: "2.4.0",
Doc: "If clustering compaction job is finished for a long time, gc it",
DefaultValue: "86400",
}
p.ClusteringCompactionDropTolerance.Init(base.mgr)
p.ClusteringCompactionPreferSegmentSize = ParamItem{
Key: "dataCoord.compaction.clustering.preferSegmentSize",
Version: "2.4.0",
DefaultValue: "64m",
PanicIfEmpty: false,
Export: true,
}
p.ClusteringCompactionPreferSegmentSize.Init(base.mgr)
p.ClusteringCompactionMaxSegmentSize = ParamItem{
Key: "dataCoord.compaction.clustering.maxSegmentSize",
Version: "2.4.0",
DefaultValue: "128m",
PanicIfEmpty: false,
Export: true,
}
p.ClusteringCompactionMaxSegmentSize.Init(base.mgr)
p.ClusteringCompactionMaxTrainSize = ParamItem{
Key: "dataCoord.compaction.clustering.maxTrainSize",
Version: "2.4.0",
DefaultValue: "26g",
Doc: "max data size in Kmeans train, if larger than it, will down sampling to meet this limit",
Export: true,
}
p.ClusteringCompactionMaxTrainSize.Init(base.mgr)
p.EnableGarbageCollection = ParamItem{
Key: "dataCoord.enableGarbageCollection",
Version: "2.0.0",
@ -3190,6 +3330,9 @@ type dataNodeConfig struct {
L0BatchMemoryRatio ParamItem `refreshable:"true"`
GracefulStopTimeout ParamItem `refreshable:"true"`
// clustering compaction
ClusteringCompactionMemoryBufferRatio ParamItem `refreshable:"true"`
}
func (p *dataNodeConfig) init(base *BaseTable) {
@ -3470,6 +3613,16 @@ func (p *dataNodeConfig) init(base *BaseTable) {
Export: true,
}
p.GracefulStopTimeout.Init(base.mgr)
p.ClusteringCompactionMemoryBufferRatio = ParamItem{
Key: "datanode.clusteringCompaction.memoryBufferRatio",
Version: "2.4.0",
Doc: "The ratio of memory buffer of clustering compaction. Data larger than threshold will be spilled to storage.",
DefaultValue: "0.1",
PanicIfEmpty: false,
Export: true,
}
p.ClusteringCompactionMemoryBufferRatio.Init(base.mgr)
}
// /////////////////////////////////////////////////////////////////////////////

View File

@ -387,6 +387,23 @@ func TestComponentParam(t *testing.T) {
params.Save("datacoord.gracefulStopTimeout", "100")
assert.Equal(t, 100*time.Second, Params.GracefulStopTimeout.GetAsDuration(time.Second))
params.Save("dataCoord.compaction.clustering.enable", "true")
assert.Equal(t, true, Params.ClusteringCompactionEnable.GetAsBool())
params.Save("dataCoord.compaction.clustering.newDataSizeThreshold", "10")
assert.Equal(t, int64(10), Params.ClusteringCompactionNewDataSizeThreshold.GetAsSize())
params.Save("dataCoord.compaction.clustering.newDataSizeThreshold", "10k")
assert.Equal(t, int64(10*1024), Params.ClusteringCompactionNewDataSizeThreshold.GetAsSize())
params.Save("dataCoord.compaction.clustering.newDataSizeThreshold", "10m")
assert.Equal(t, int64(10*1024*1024), Params.ClusteringCompactionNewDataSizeThreshold.GetAsSize())
params.Save("dataCoord.compaction.clustering.newDataSizeThreshold", "10g")
assert.Equal(t, int64(10*1024*1024*1024), Params.ClusteringCompactionNewDataSizeThreshold.GetAsSize())
params.Save("dataCoord.compaction.clustering.dropTolerance", "86400")
assert.Equal(t, int64(86400), Params.ClusteringCompactionDropTolerance.GetAsInt64())
params.Save("dataCoord.compaction.clustering.maxSegmentSize", "100m")
assert.Equal(t, int64(100*1024*1024), Params.ClusteringCompactionMaxSegmentSize.GetAsSize())
params.Save("dataCoord.compaction.clustering.preferSegmentSize", "10m")
assert.Equal(t, int64(10*1024*1024), Params.ClusteringCompactionPreferSegmentSize.GetAsSize())
})
t.Run("test dataNodeConfig", func(t *testing.T) {
@ -438,6 +455,10 @@ func TestComponentParam(t *testing.T) {
assert.Equal(t, int64(16), Params.MaxImportFileSizeInGB.GetAsInt64())
params.Save("datanode.gracefulStopTimeout", "100")
assert.Equal(t, 100*time.Second, Params.GracefulStopTimeout.GetAsDuration(time.Second))
// clustering compaction
params.Save("datanode.clusteringCompaction.memoryBufferRatio", "0.1")
assert.Equal(t, 0.1, Params.ClusteringCompactionMemoryBufferRatio.GetAsFloat())
})
t.Run("test indexNodeConfig", func(t *testing.T) {
@ -456,6 +477,14 @@ func TestComponentParam(t *testing.T) {
assert.Equal(t, "by-dev-dml1", Params.RootCoordDml.GetValue())
})
t.Run("clustering compaction config", func(t *testing.T) {
Params := &params.CommonCfg
params.Save("common.usePartitionKeyAsClusteringKey", "true")
assert.Equal(t, true, Params.UsePartitionKeyAsClusteringKey.GetAsBool())
params.Save("common.useVectorAsClusteringKey", "true")
assert.Equal(t, true, Params.UseVectorAsClusteringKey.GetAsBool())
})
}
func TestForbiddenItem(t *testing.T) {

View File

@ -223,6 +223,34 @@ func (pi *ParamItem) GetAsRoleDetails() map[string](map[string]([](map[string]st
return getAndConvert(pi.GetValue(), funcutil.JSONToRoleDetails, nil)
}
func (pi *ParamItem) GetAsSize() int64 {
valueStr := strings.ToLower(pi.GetValue())
if strings.HasSuffix(valueStr, "g") || strings.HasSuffix(valueStr, "gb") {
size, err := strconv.ParseInt(strings.Split(valueStr, "g")[0], 10, 64)
if err != nil {
return 0
}
return size * 1024 * 1024 * 1024
} else if strings.HasSuffix(valueStr, "m") || strings.HasSuffix(valueStr, "mb") {
size, err := strconv.ParseInt(strings.Split(valueStr, "m")[0], 10, 64)
if err != nil {
return 0
}
return size * 1024 * 1024
} else if strings.HasSuffix(valueStr, "k") || strings.HasSuffix(valueStr, "kb") {
size, err := strconv.ParseInt(strings.Split(valueStr, "k")[0], 10, 64)
if err != nil {
return 0
}
return size * 1024
}
size, err := strconv.ParseInt(valueStr, 10, 64)
if err != nil {
return 0
}
return size
}
type CompositeParamItem struct {
Items []*ParamItem
Format func(map[string]string) string

View File

@ -1075,6 +1075,16 @@ func HasPartitionKey(schema *schemapb.CollectionSchema) bool {
return false
}
// HasClusterKey check if a collection schema has ClusterKey field
func HasClusterKey(schema *schemapb.CollectionSchema) bool {
for _, fieldSchema := range schema.Fields {
if fieldSchema.IsClusteringKey {
return true
}
}
return false
}
// GetPrimaryFieldData get primary field data from all field data inserted from sdk
func GetPrimaryFieldData(datas []*schemapb.FieldData, primaryFieldSchema *schemapb.FieldSchema) (*schemapb.FieldData, error) {
primaryFieldID := primaryFieldSchema.FieldID

View File

@ -1046,6 +1046,40 @@ func TestGetPrimaryFieldSchema(t *testing.T) {
assert.True(t, hasPartitionKey2)
}
func TestGetClusterKeyFieldSchema(t *testing.T) {
int64Field := &schemapb.FieldSchema{
FieldID: 1,
Name: "int64Field",
DataType: schemapb.DataType_Int64,
}
clusterKeyfloatField := &schemapb.FieldSchema{
FieldID: 2,
Name: "floatField",
DataType: schemapb.DataType_Float,
IsClusteringKey: true,
}
unClusterKeyfloatField := &schemapb.FieldSchema{
FieldID: 2,
Name: "floatField",
DataType: schemapb.DataType_Float,
IsClusteringKey: false,
}
schema := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{int64Field, clusterKeyfloatField},
}
schema2 := &schemapb.CollectionSchema{
Fields: []*schemapb.FieldSchema{int64Field, unClusterKeyfloatField},
}
hasClusterKey1 := HasClusterKey(schema)
assert.True(t, hasClusterKey1)
hasClusterKey2 := HasClusterKey(schema2)
assert.False(t, hasClusterKey2)
}
func TestGetPK(t *testing.T) {
type args struct {
data *schemapb.IDs

View File

@ -0,0 +1,169 @@
// Licensed to the LF AI & Data foundation under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package clustering
import (
"context"
"fmt"
"testing"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/suite"
"go.uber.org/zap"
"github.com/milvus-io/milvus-proto/go-api/v2/commonpb"
"github.com/milvus-io/milvus-proto/go-api/v2/milvuspb"
"github.com/milvus-io/milvus-proto/go-api/v2/schemapb"
"github.com/milvus-io/milvus/pkg/common"
"github.com/milvus-io/milvus/pkg/log"
"github.com/milvus-io/milvus/pkg/util/funcutil"
"github.com/milvus-io/milvus/pkg/util/metric"
"github.com/milvus-io/milvus/tests/integration"
)
type ClusteringCompactionSuite struct {
integration.MiniClusterSuite
}
func (s *ClusteringCompactionSuite) TestClusteringCompaction() {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
c := s.Cluster
const (
dim = 128
dbName = ""
rowNum = 3000
)
collectionName := "TestClusteringCompaction" + funcutil.GenRandomStr()
schema := integration.ConstructSchema(collectionName, dim, true)
marshaledSchema, err := proto.Marshal(schema)
s.NoError(err)
createCollectionStatus, err := c.Proxy.CreateCollection(ctx, &milvuspb.CreateCollectionRequest{
DbName: dbName,
CollectionName: collectionName,
Schema: marshaledSchema,
ShardsNum: common.DefaultShardsNum,
})
s.NoError(err)
if createCollectionStatus.GetErrorCode() != commonpb.ErrorCode_Success {
log.Warn("createCollectionStatus fail reason", zap.String("reason", createCollectionStatus.GetReason()))
}
s.Equal(createCollectionStatus.GetErrorCode(), commonpb.ErrorCode_Success)
log.Info("CreateCollection result", zap.Any("createCollectionStatus", createCollectionStatus))
showCollectionsResp, err := c.Proxy.ShowCollections(ctx, &milvuspb.ShowCollectionsRequest{})
s.NoError(err)
s.Equal(showCollectionsResp.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success)
log.Info("ShowCollections result", zap.Any("showCollectionsResp", showCollectionsResp))
fVecColumn := integration.NewFloatVectorFieldData(integration.FloatVecField, rowNum, dim)
hashKeys := integration.GenerateHashKeys(rowNum)
insertResult, err := c.Proxy.Insert(ctx, &milvuspb.InsertRequest{
DbName: dbName,
CollectionName: collectionName,
FieldsData: []*schemapb.FieldData{fVecColumn},
HashKeys: hashKeys,
NumRows: uint32(rowNum),
})
s.NoError(err)
s.Equal(insertResult.GetStatus().GetErrorCode(), commonpb.ErrorCode_Success)
// flush
flushResp, err := c.Proxy.Flush(ctx, &milvuspb.FlushRequest{
DbName: dbName,
CollectionNames: []string{collectionName},
})
s.NoError(err)
segmentIDs, has := flushResp.GetCollSegIDs()[collectionName]
ids := segmentIDs.GetData()
s.Require().NotEmpty(segmentIDs)
s.Require().True(has)
flushTs, has := flushResp.GetCollFlushTs()[collectionName]
s.True(has)
segments, err := c.MetaWatcher.ShowSegments()
s.NoError(err)
s.NotEmpty(segments)
for _, segment := range segments {
log.Info("ShowSegments result", zap.String("segment", segment.String()))
}
s.WaitForFlush(ctx, ids, flushTs, dbName, collectionName)
// create index
createIndexStatus, err := c.Proxy.CreateIndex(ctx, &milvuspb.CreateIndexRequest{
CollectionName: collectionName,
FieldName: integration.FloatVecField,
IndexName: "_default",
ExtraParams: integration.ConstructIndexParam(dim, integration.IndexFaissIvfFlat, metric.L2),
})
if createIndexStatus.GetErrorCode() != commonpb.ErrorCode_Success {
log.Warn("createIndexStatus fail reason", zap.String("reason", createIndexStatus.GetReason()))
}
s.NoError(err)
s.Equal(commonpb.ErrorCode_Success, createIndexStatus.GetErrorCode())
s.WaitForIndexBuilt(ctx, collectionName, integration.FloatVecField)
// load
loadStatus, err := c.Proxy.LoadCollection(ctx, &milvuspb.LoadCollectionRequest{
DbName: dbName,
CollectionName: collectionName,
})
s.NoError(err)
if loadStatus.GetErrorCode() != commonpb.ErrorCode_Success {
log.Warn("loadStatus fail reason", zap.String("reason", loadStatus.GetReason()))
}
s.Equal(commonpb.ErrorCode_Success, loadStatus.GetErrorCode())
s.WaitForLoad(ctx, collectionName)
// search
expr := fmt.Sprintf("%s > 0", integration.Int64Field)
nq := 10
topk := 10
roundDecimal := -1
params := integration.GetSearchParams(integration.IndexFaissIvfFlat, metric.L2)
searchReq := integration.ConstructSearchRequest("", collectionName, expr,
integration.FloatVecField, schemapb.DataType_FloatVector, nil, metric.L2, params, nq, dim, topk, roundDecimal)
searchResult, err := c.Proxy.Search(ctx, searchReq)
if searchResult.GetStatus().GetErrorCode() != commonpb.ErrorCode_Success {
log.Warn("searchResult fail reason", zap.String("reason", searchResult.GetStatus().GetReason()))
}
s.NoError(err)
s.Equal(commonpb.ErrorCode_Success, searchResult.GetStatus().GetErrorCode())
compactReq := &milvuspb.ManualCompactionRequest{
CollectionID: showCollectionsResp.CollectionIds[0],
MajorCompaction: true,
}
compactResp, err := c.Proxy.ManualCompaction(ctx, compactReq)
s.NoError(err)
log.Info("comapct", zap.Any("compactResp", compactResp))
log.Info("TestClusteringCompaction succeed")
}
func TestClusteringCompaction(t *testing.T) {
// t.Skip("Skip integration test")
suite.Run(t, new(ClusteringCompactionSuite))
}