mirror of
https://gitee.com/milvus-io/milvus.git
synced 2026-01-07 19:31:51 +08:00
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:
parent
b6fefee0cf
commit
17bf3abafe
1
Makefile
1
Makefile
@ -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
|
||||
|
||||
@ -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:
|
||||
|
||||
@ -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";
|
||||
|
||||
@ -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")
|
||||
|
||||
@ -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
|
||||
|
||||
351
internal/core/src/indexbuilder/KmeansMajorCompaction.cpp
Normal file
351
internal/core/src/indexbuilder/KmeansMajorCompaction.cpp
Normal 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
|
||||
56
internal/core/src/indexbuilder/KmeansMajorCompaction.h
Normal file
56
internal/core/src/indexbuilder/KmeansMajorCompaction.h
Normal 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
|
||||
39
internal/core/src/indexbuilder/MajorCompaction.h
Normal file
39
internal/core/src/indexbuilder/MajorCompaction.h
Normal 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
|
||||
307
internal/core/src/indexbuilder/analysis_c.cpp
Normal file
307
internal/core/src/indexbuilder/analysis_c.cpp
Normal 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;
|
||||
}
|
||||
83
internal/core/src/indexbuilder/analysis_c.h
Normal file
83
internal/core/src/indexbuilder/analysis_c.h
Normal 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
|
||||
@ -16,3 +16,5 @@
|
||||
typedef void* CIndex;
|
||||
typedef void* CIndexQueryResult;
|
||||
typedef void* CBuildIndexInfo;
|
||||
typedef void* CAnalysis;
|
||||
typedef void* CAnalysisInfo;
|
||||
|
||||
@ -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;
|
||||
};
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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_;
|
||||
|
||||
@ -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_;
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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,
|
||||
|
||||
@ -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}")
|
||||
|
||||
|
||||
@ -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" )
|
||||
|
||||
@ -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();
|
||||
|
||||
163
internal/core/unittest/test_major_compaction.cpp
Normal file
163
internal/core/unittest/test_major_compaction.cpp
Normal 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>();
|
||||
}
|
||||
@ -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
|
||||
|
||||
197
internal/datacoord/analysis_meta.go
Normal file
197
internal/datacoord/analysis_meta.go
Normal 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
|
||||
}
|
||||
255
internal/datacoord/analysis_meta_test.go
Normal file
255
internal/datacoord/analysis_meta_test.go
Normal 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))
|
||||
}
|
||||
371
internal/datacoord/analysis_scheduler.go
Normal file
371
internal/datacoord/analysis_scheduler.go
Normal 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
|
||||
}
|
||||
448
internal/datacoord/analysis_scheduler_test.go
Normal file
448
internal/datacoord/analysis_scheduler_test.go
Normal 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))
|
||||
}
|
||||
84
internal/datacoord/clustering_compaction_job.go
Normal file
84
internal/datacoord/clustering_compaction_job.go
Normal 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
|
||||
}
|
||||
483
internal/datacoord/clustering_compaction_manager.go
Normal file
483
internal/datacoord/clustering_compaction_manager.go
Normal 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
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
|
||||
|
||||
@ -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(),
|
||||
}
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -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()
|
||||
|
||||
@ -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})
|
||||
|
||||
@ -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)
|
||||
})
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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)
|
||||
})
|
||||
}
|
||||
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
324
internal/datacoord/mock_worker_manager.go
Normal file
324
internal/datacoord/mock_worker_manager.go
Normal 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
|
||||
}
|
||||
@ -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)
|
||||
|
||||
@ -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
|
||||
},
|
||||
},
|
||||
|
||||
@ -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)
|
||||
|
||||
1085
internal/datanode/clustering_compactor.go
Normal file
1085
internal/datanode/clustering_compactor.go
Normal file
File diff suppressed because it is too large
Load Diff
@ -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
|
||||
}
|
||||
|
||||
35
internal/datanode/compactor_common.go
Normal file
35
internal/datanode/compactor_common.go
Normal 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)
|
||||
}
|
||||
@ -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))
|
||||
})
|
||||
|
||||
@ -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()
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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() {
|
||||
|
||||
@ -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)
|
||||
})
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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() {
|
||||
|
||||
1
internal/indexnode/meta.go
Normal file
1
internal/indexnode/meta.go
Normal file
@ -0,0 +1 @@
|
||||
package indexnode
|
||||
@ -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
|
||||
}
|
||||
|
||||
207
internal/indexnode/task_analysis.go
Normal file
207
internal/indexnode/task_analysis.go
Normal 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
|
||||
}
|
||||
641
internal/indexnode/task_index.go
Normal file
641
internal/indexnode/task_index.go
Normal 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
|
||||
}
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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))
|
||||
}
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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"
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
110
internal/metastore/model/analysis_stats.go
Normal file
110
internal/metastore/model/analysis_stats.go
Normal 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,
|
||||
}
|
||||
}
|
||||
@ -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)
|
||||
|
||||
@ -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))
|
||||
|
||||
@ -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;
|
||||
}
|
||||
@ -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;
|
||||
}
|
||||
|
||||
@ -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;
|
||||
}
|
||||
|
||||
@ -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)
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
@ -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()))
|
||||
}
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
99
internal/util/analysiscgowrapper/analysis.go
Normal file
99
internal/util/analysiscgowrapper/analysis.go
Normal 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
|
||||
}
|
||||
133
internal/util/analysiscgowrapper/analysis_info.go
Normal file
133
internal/util/analysiscgowrapper/analysis_info.go
Normal 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")
|
||||
}
|
||||
63
internal/util/analysiscgowrapper/helper.go
Normal file
63
internal/util/analysiscgowrapper/helper.go
Normal 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
|
||||
}
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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 {
|
||||
|
||||
@ -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
|
||||
}
|
||||
|
||||
@ -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)
|
||||
}
|
||||
|
||||
// /////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
@ -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 := ¶ms.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) {
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
@ -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
|
||||
|
||||
169
tests/integration/clustering/clustering_compaction_test.go
Normal file
169
tests/integration/clustering/clustering_compaction_test.go
Normal 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))
|
||||
}
|
||||
Loading…
x
Reference in New Issue
Block a user