From 17bf3abafea6caaacbc123d71a32574db5776c91 Mon Sep 17 00:00:00 2001 From: wayblink Date: Mon, 8 Apr 2024 14:30:44 +0800 Subject: [PATCH] Major compaction 1st edition (#31804) Signed-off-by: wayblink Signed-off-by: Cai Zhang Signed-off-by: chasingegg Co-authored-by: chasingegg --- Makefile | 1 + configs/milvus.yaml | 23 + internal/core/src/common/Consts.h | 3 + internal/core/src/indexbuilder/CMakeLists.txt | 2 + internal/core/src/indexbuilder/IndexFactory.h | 33 + .../indexbuilder/KmeansMajorCompaction.cpp | 351 ++++++ .../src/indexbuilder/KmeansMajorCompaction.h | 56 + .../core/src/indexbuilder/MajorCompaction.h | 39 + internal/core/src/indexbuilder/analysis_c.cpp | 307 +++++ internal/core/src/indexbuilder/analysis_c.h | 83 ++ internal/core/src/indexbuilder/type_c.h | 2 + internal/core/src/indexbuilder/types.h | 24 + .../core/src/storage/DiskFileManagerImpl.cpp | 225 +++- .../core/src/storage/DiskFileManagerImpl.h | 31 + internal/core/src/storage/FileManager.h | 20 + internal/core/src/storage/Util.cpp | 55 + internal/core/src/storage/Util.h | 16 + .../core/thirdparty/knowhere/CMakeLists.txt | 4 +- internal/core/unittest/CMakeLists.txt | 1 + .../unittest/test_disk_file_manager_test.cpp | 66 + .../core/unittest/test_major_compaction.cpp | 163 +++ .../unittest/test_utils/storage_test_utils.h | 58 + internal/datacoord/analysis_meta.go | 197 +++ internal/datacoord/analysis_meta_test.go | 255 ++++ internal/datacoord/analysis_scheduler.go | 371 ++++++ internal/datacoord/analysis_scheduler_test.go | 448 +++++++ .../datacoord/clustering_compaction_job.go | 84 ++ .../clustering_compaction_manager.go | 483 ++++++++ internal/datacoord/compaction.go | 79 +- internal/datacoord/compaction_trigger.go | 225 +++- internal/datacoord/compaction_trigger_test.go | 96 +- internal/datacoord/index_builder.go | 124 +- internal/datacoord/index_builder_test.go | 100 +- internal/datacoord/index_meta.go | 8 +- internal/datacoord/index_meta_test.go | 12 +- internal/datacoord/index_service.go | 41 + internal/datacoord/indexnode_manager.go | 92 +- internal/datacoord/indexnode_manager_test.go | 22 +- internal/datacoord/meta.go | 211 +++- internal/datacoord/mock_compaction_meta.go | 84 ++ internal/datacoord/mock_test.go | 6 +- internal/datacoord/mock_worker_manager.go | 324 +++++ internal/datacoord/server.go | 27 +- internal/datacoord/server_test.go | 6 +- internal/datacoord/services.go | 2 +- internal/datanode/clustering_compactor.go | 1085 +++++++++++++++++ internal/datanode/compactor.go | 112 +- internal/datanode/compactor_common.go | 35 + internal/datanode/compactor_test.go | 10 +- internal/datanode/iterators/iterator.go | 24 + internal/datanode/services.go | 11 + internal/datanode/services_test.go | 20 + .../distributed/indexnode/client/client.go | 18 + .../indexnode/client/client_test.go | 18 + internal/distributed/indexnode/service.go | 12 + .../distributed/indexnode/service_test.go | 21 + internal/indexnode/indexnode.go | 24 +- internal/indexnode/indexnode_service.go | 121 +- internal/indexnode/indexnode_service_test.go | 122 ++ internal/indexnode/indexnode_test.go | 6 +- internal/indexnode/meta.go | 1 + internal/indexnode/task.go | 630 ---------- internal/indexnode/task_analysis.go | 207 ++++ internal/indexnode/task_index.go | 641 ++++++++++ internal/indexnode/task_test.go | 444 ++++--- internal/indexnode/taskinfo_ops.go | 170 ++- internal/metastore/catalog.go | 8 + internal/metastore/kv/datacoord/constant.go | 22 +- internal/metastore/kv/datacoord/kv_catalog.go | 75 ++ internal/metastore/kv/datacoord/util.go | 18 + .../metastore/mocks/mock_datacoord_catalog.go | 280 +++++ internal/metastore/model/analysis_stats.go | 110 ++ internal/mocks/mock_indexnode.go | 165 +++ internal/mocks/mock_indexnode_client.go | 210 ++++ internal/proto/data_coord.proto | 35 +- internal/proto/index_coord.proto | 65 + internal/proto/segcore.proto | 11 +- .../querynodev2/delegator/segment_pruner.go | 48 +- internal/storage/binlog_iterator.go | 14 +- internal/storage/field_stats_test.go | 12 + internal/storage/field_value.go | 34 + internal/storage/partition_stats.go | 23 +- internal/util/analysiscgowrapper/analysis.go | 99 ++ .../util/analysiscgowrapper/analysis_info.go | 133 ++ internal/util/analysiscgowrapper/helper.go | 63 + internal/util/clustering/clustering.go | 30 + internal/util/mock/grpc_indexnode_client.go | 12 + internal/util/typeutil/schema.go | 9 - pkg/common/common.go | 6 + pkg/util/merr/errors.go | 6 + pkg/util/merr/utils.go | 32 + pkg/util/paramtable/component_param.go | 153 +++ pkg/util/paramtable/component_param_test.go | 29 + pkg/util/paramtable/param_item.go | 28 + pkg/util/typeutil/schema.go | 10 + pkg/util/typeutil/schema_test.go | 34 + .../clustering/clustering_compaction_test.go | 169 +++ 97 files changed, 9276 insertions(+), 1259 deletions(-) create mode 100644 internal/core/src/indexbuilder/KmeansMajorCompaction.cpp create mode 100644 internal/core/src/indexbuilder/KmeansMajorCompaction.h create mode 100644 internal/core/src/indexbuilder/MajorCompaction.h create mode 100644 internal/core/src/indexbuilder/analysis_c.cpp create mode 100644 internal/core/src/indexbuilder/analysis_c.h create mode 100644 internal/core/unittest/test_major_compaction.cpp create mode 100644 internal/datacoord/analysis_meta.go create mode 100644 internal/datacoord/analysis_meta_test.go create mode 100644 internal/datacoord/analysis_scheduler.go create mode 100644 internal/datacoord/analysis_scheduler_test.go create mode 100644 internal/datacoord/clustering_compaction_job.go create mode 100644 internal/datacoord/clustering_compaction_manager.go create mode 100644 internal/datacoord/mock_worker_manager.go create mode 100644 internal/datanode/clustering_compactor.go create mode 100644 internal/datanode/compactor_common.go create mode 100644 internal/indexnode/meta.go create mode 100644 internal/indexnode/task_analysis.go create mode 100644 internal/indexnode/task_index.go create mode 100644 internal/metastore/model/analysis_stats.go create mode 100644 internal/util/analysiscgowrapper/analysis.go create mode 100644 internal/util/analysiscgowrapper/analysis_info.go create mode 100644 internal/util/analysiscgowrapper/helper.go create mode 100644 tests/integration/clustering/clustering_compaction_test.go diff --git a/Makefile b/Makefile index 6f3a0c1fb2..9b7c3df080 100644 --- a/Makefile +++ b/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 diff --git a/configs/milvus.yaml b/configs/milvus.yaml index 0bd08c6200..414a7e3c63 100644 --- a/configs/milvus.yaml +++ b/configs/milvus.yaml @@ -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: diff --git a/internal/core/src/common/Consts.h b/internal/core/src/common/Consts.h index 65e6795b16..350c92784e 100644 --- a/internal/core/src/common/Consts.h +++ b/internal/core/src/common/Consts.h @@ -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"; diff --git a/internal/core/src/indexbuilder/CMakeLists.txt b/internal/core/src/indexbuilder/CMakeLists.txt index a6fdea6456..40576a5917 100644 --- a/internal/core/src/indexbuilder/CMakeLists.txt +++ b/internal/core/src/indexbuilder/CMakeLists.txt @@ -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") diff --git a/internal/core/src/indexbuilder/IndexFactory.h b/internal/core/src/indexbuilder/IndexFactory.h index cd361499b4..6be49702b9 100644 --- a/internal/core/src/indexbuilder/IndexFactory.h +++ b/internal/core/src/indexbuilder/IndexFactory.h @@ -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>(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 diff --git a/internal/core/src/indexbuilder/KmeansMajorCompaction.cpp b/internal/core/src/indexbuilder/KmeansMajorCompaction.cpp new file mode 100644 index 0000000000..331d485fd5 --- /dev/null +++ b/internal/core/src/indexbuilder/KmeansMajorCompaction.cpp @@ -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 + +namespace milvus::indexbuilder { + +template +KmeansMajorCompaction::KmeansMajorCompaction( + Config& config, const storage::FileManagerContext& file_manager_context) + : config_(config) { + file_manager_ = + std::make_shared(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 +std::unique_ptr +KmeansMajorCompaction::Sample(const std::vector& file_paths, + const std::vector& 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(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(buf.get()) + offset, + file_sizes[i]); + offset += file_sizes[i]; + } + return buf; + } + // random sampling to get train data + std::vector 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(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(buf.get()) + offset, + cur_size); + break; + } else { + selected_size += file_sizes[idx[i]]; + local_chunk_manager->Read( + file_paths[idx[i]], + 0, + reinterpret_cast(buf.get()) + offset, + file_sizes[idx[i]]); + offset += file_sizes[idx[i]]; + } + } + return buf; +} + +template +BinarySet +KmeansMajorCompaction::Upload() { + BinarySet ret; + + std::unordered_map 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 +void +KmeansMajorCompaction::Train() { + if constexpr (!std::is_same_v) { + PanicInfo( + ErrorCode::UnexpectedError, + fmt::format("kmeans major compaction only supports float32 now")); + } + auto insert_files = milvus::index::GetValueFromConfig< + std::map>>(config_, "insert_files"); + AssertInfo(insert_files.has_value(), + "insert file paths is empty when major compaction"); + auto segment_size = + milvus::index::GetValueFromConfig(config_, "segment_size"); + AssertInfo(segment_size.has_value(), + "segment size is empty when major compaction"); + auto train_size = + milvus::index::GetValueFromConfig(config_, "train_size"); + AssertInfo(train_size.has_value(), + "train size is empty when major compaction"); + + std::vector data_files; + std::vector 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(*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(res.value()->GetTensor()); + auto centroid_id_mapping = + reinterpret_cast(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 { + std::vector 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 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(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(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( + *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( + 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; + +} // namespace milvus::indexbuilder diff --git a/internal/core/src/indexbuilder/KmeansMajorCompaction.h b/internal/core/src/indexbuilder/KmeansMajorCompaction.h new file mode 100644 index 0000000000..b60f334cec --- /dev/null +++ b/internal/core/src/indexbuilder/KmeansMajorCompaction.h @@ -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 +#include + +#include "storage/DiskFileManagerImpl.h" +#include "storage/space.h" +#include "indexbuilder/MajorCompaction.h" + +namespace milvus::indexbuilder { + +template +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 + Sample(const std::vector& file_paths, + const std::vector& data_sizes, + uint64_t train_size, + uint64_t total_size); + + std::shared_ptr file_manager_; + Config config_; + std::vector result_files_; +}; + +template +using KmeansMajorCompactionPtr = std::unique_ptr>; +} // namespace milvus::indexbuilder diff --git a/internal/core/src/indexbuilder/MajorCompaction.h b/internal/core/src/indexbuilder/MajorCompaction.h new file mode 100644 index 0000000000..2120945d9c --- /dev/null +++ b/internal/core/src/indexbuilder/MajorCompaction.h @@ -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 +#include + +#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; +} // namespace milvus::indexbuilder \ No newline at end of file diff --git a/internal/core/src/indexbuilder/analysis_c.cpp b/internal/core/src/indexbuilder/analysis_c.cpp new file mode 100644 index 0000000000..765a3a5728 --- /dev/null +++ b/internal/core/src/indexbuilder/analysis_c.cpp @@ -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 + +#ifdef __linux__ +#include +#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( + // 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( + // 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(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(analysis); + auto cAnalysis = + dynamic_cast(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(); + 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(analysis); + auto binary = + std::make_unique(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; +} diff --git a/internal/core/src/indexbuilder/analysis_c.h b/internal/core/src/indexbuilder/analysis_c.h new file mode 100644 index 0000000000..c0161624ab --- /dev/null +++ b/internal/core/src/indexbuilder/analysis_c.h @@ -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 +#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 diff --git a/internal/core/src/indexbuilder/type_c.h b/internal/core/src/indexbuilder/type_c.h index f0d07fb51d..c7fa62d3e9 100644 --- a/internal/core/src/indexbuilder/type_c.h +++ b/internal/core/src/indexbuilder/type_c.h @@ -16,3 +16,5 @@ typedef void* CIndex; typedef void* CIndexQueryResult; typedef void* CBuildIndexInfo; +typedef void* CAnalysis; +typedef void* CAnalysisInfo; diff --git a/internal/core/src/indexbuilder/types.h b/internal/core/src/indexbuilder/types.h index e1c656e25d..b999c7b075 100644 --- a/internal/core/src/indexbuilder/types.h +++ b/internal/core/src/indexbuilder/types.h @@ -14,6 +14,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -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> insert_files; // segment_id->files + // std::vector insert_files; + milvus::storage::StorageConfig storage_config; + milvus::Config config; + std::string field_name; + std::string centroids_store_path; + std::vector 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; +}; diff --git a/internal/core/src/storage/DiskFileManagerImpl.cpp b/internal/core/src/storage/DiskFileManagerImpl.cpp index 9b0080170d..636b6bf3c6 100644 --- a/internal/core/src/storage/DiskFileManagerImpl.cpp +++ b/internal/core/src/storage/DiskFileManagerImpl.cpp @@ -162,6 +162,89 @@ DiskFileManagerImpl::AddFile(const std::string& file) noexcept { return true; } // namespace knowhere +void +DiskFileManagerImpl::AddCompactionResultFiles( + const std::vector& files, + std::unordered_map& map) { + auto local_chunk_manager = + LocalChunkManagerSingleton::GetInstance().GetChunkManager(); + std::vector local_files; + std::vector batch_remote_files; + std::vector 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& local_files, + const std::vector& remote_files, + const std::vector& remote_file_sizes, + std::unordered_map& map) { + auto local_chunk_manager = + LocalChunkManagerSingleton::GetInstance().GetChunkManager(); + auto& pool = ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::HIGH); + + std::vector>> 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 { + auto buf = std::shared_ptr(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> index_datas; + std::vector 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& paths) { }); } +uint64_t +FetchRawDataAndWriteFile(ChunkManagerPtr rcm, + std::string& local_data_path, + std::vector& 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(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>& remote_files, + std::vector& output_files, + std::vector& 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 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 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 remote_files) { SortByPath(remote_files); @@ -466,30 +644,16 @@ DiskFileManagerImpl::CacheRawDataToDisk(std::vector 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(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 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 diff --git a/internal/core/src/storage/DiskFileManagerImpl.h b/internal/core/src/storage/DiskFileManagerImpl.h index 66d5830659..1adeb1b285 100644 --- a/internal/core/src/storage/DiskFileManagerImpl.h +++ b/internal/core/src/storage/DiskFileManagerImpl.h @@ -64,6 +64,12 @@ class DiskFileManagerImpl : public FileManagerImpl { std::string GetLocalRawDataObjectPrefix(); + std::string + GetCompactionRawDataObjectPrefix(); + + std::string + GetCompactionResultObjectPrefix(); + std::map 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& files, + std::unordered_map& map); + + void + AddBatchCompactionResultFiles( + const std::vector& local_files, + const std::vector& remote_files, + const std::vector& remote_file_sizes, + std::unordered_map& map); + void AddBatchIndexFiles(const std::string& local_file_name, const std::vector& local_file_offsets, @@ -102,6 +119,13 @@ class DiskFileManagerImpl : public FileManagerImpl { std::string CacheRawDataToDisk(std::shared_ptr space); + uint64_t + CacheCompactionRawDataToDisk( + const std::map>& insert_files, + std::vector& output_files, + std::vector& 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 local_paths_; diff --git a/internal/core/src/storage/FileManager.h b/internal/core/src/storage/FileManager.h index a0d94cfc58..4049e61178 100644 --- a/internal/core/src/storage/FileManager.h +++ b/internal/core/src/storage/FileManager.h @@ -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_; diff --git a/internal/core/src/storage/Util.cpp b/internal/core/src/storage/Util.cpp index 31d5600515..381f2afa24 100644 --- a/internal/core/src/storage/Util.cpp +++ b/internal/core/src/storage/Util.cpp @@ -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 space, return datas; } +void +PutCompactionResultData(ChunkManager* remote_chunk_manager, + const std::vector& data_slices, + const std::vector& slice_sizes, + const std::vector& slice_names, + std::unordered_map& map) { + auto& pool = ThreadPools::GetThreadPool(milvus::ThreadPoolPriority::MIDDLE); + std::vector>> 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 { + chunk_manager->Write(object_key, buf, batch_size); + return std::make_pair(object_key, batch_size); + }, + remote_chunk_manager, + const_cast(data_slices[i]), + slice_sizes[i], + slice_names[i])); + } + + for (auto& future : futures) { + auto res = future.get(); + map[res.first] = res.second; + } +} std::map PutIndexData(ChunkManager* remote_chunk_manager, const std::vector& data_slices, diff --git a/internal/core/src/storage/Util.h b/internal/core/src/storage/Util.h index acb6d233c0..97190d53ea 100644 --- a/internal/core/src/storage/Util.h +++ b/internal/core/src/storage/Util.h @@ -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& data_slices, + const std::vector& slice_sizes, + const std::vector& slice_names, + std::unordered_map& map); + std::map PutIndexData(std::shared_ptr space, const std::vector& data_slices, diff --git a/internal/core/thirdparty/knowhere/CMakeLists.txt b/internal/core/thirdparty/knowhere/CMakeLists.txt index 5fb93e8711..f228ef58f9 100644 --- a/internal/core/thirdparty/knowhere/CMakeLists.txt +++ b/internal/core/thirdparty/knowhere/CMakeLists.txt @@ -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}") diff --git a/internal/core/unittest/CMakeLists.txt b/internal/core/unittest/CMakeLists.txt index 7a3c507630..83fcda4c93 100644 --- a/internal/core/unittest/CMakeLists.txt +++ b/internal/core/unittest/CMakeLists.txt @@ -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" ) diff --git a/internal/core/unittest/test_disk_file_manager_test.cpp b/internal/core/unittest/test_disk_file_manager_test.cpp index 7fa7a4263e..3421155c98 100644 --- a/internal/core/unittest/test_disk_file_manager_test.cpp +++ b/internal/core/unittest/test_disk_file_manager_test.cpp @@ -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 { @@ -260,6 +262,29 @@ PrepareInsertData() -> std::string { return path; } +auto +PrepareVectorInsertData(int64_t segment_id) -> std::string { + std::vector 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> { @@ -326,6 +351,21 @@ PrepareOptionalField(const std::shared_ptr& 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(&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> 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 output_files; + std::vector 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(); diff --git a/internal/core/unittest/test_major_compaction.cpp b/internal/core/unittest/test_major_compaction.cpp new file mode 100644 index 0000000000..568131aefb --- /dev/null +++ b/internal/core/unittest/test_major_compaction.cpp @@ -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 +#include +#include +#include +#include +#include + +#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 +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 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> 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 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 centroid_id_mapping; + std::vector 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(); +} \ No newline at end of file diff --git a/internal/core/unittest/test_utils/storage_test_utils.h b/internal/core/unittest/test_utils/storage_test_utils.h index 7eca359f30..2cd38ef2db 100644 --- a/internal/core/unittest/test_utils/storage_test_utils.h +++ b/internal/core/unittest/test_utils/storage_test_utils.h @@ -23,6 +23,7 @@ #include "storage/Types.h" #include "storage/InsertData.h" #include "storage/ThreadPools.h" +#include 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 written_; +}; + } // namespace diff --git a/internal/datacoord/analysis_meta.go b/internal/datacoord/analysis_meta.go new file mode 100644 index 0000000000..b95c3a3b86 --- /dev/null +++ b/internal/datacoord/analysis_meta.go @@ -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 +} diff --git a/internal/datacoord/analysis_meta_test.go b/internal/datacoord/analysis_meta_test.go new file mode 100644 index 0000000000..facc1a2edc --- /dev/null +++ b/internal/datacoord/analysis_meta_test.go @@ -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)) +} diff --git a/internal/datacoord/analysis_scheduler.go b/internal/datacoord/analysis_scheduler.go new file mode 100644 index 0000000000..af142dc602 --- /dev/null +++ b/internal/datacoord/analysis_scheduler.go @@ -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 +} diff --git a/internal/datacoord/analysis_scheduler_test.go b/internal/datacoord/analysis_scheduler_test.go new file mode 100644 index 0000000000..28ed000198 --- /dev/null +++ b/internal/datacoord/analysis_scheduler_test.go @@ -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)) +} diff --git a/internal/datacoord/clustering_compaction_job.go b/internal/datacoord/clustering_compaction_job.go new file mode 100644 index 0000000000..97926f2dbf --- /dev/null +++ b/internal/datacoord/clustering_compaction_job.go @@ -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 +} diff --git a/internal/datacoord/clustering_compaction_manager.go b/internal/datacoord/clustering_compaction_manager.go new file mode 100644 index 0000000000..4bb2f2576b --- /dev/null +++ b/internal/datacoord/clustering_compaction_manager.go @@ -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 +} diff --git a/internal/datacoord/compaction.go b/internal/datacoord/compaction.go index 044bcaf07d..2f5a5cef17 100644 --- a/internal/datacoord/compaction.go +++ b/internal/datacoord/compaction.go @@ -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 } diff --git a/internal/datacoord/compaction_trigger.go b/internal/datacoord/compaction_trigger.go index d485a65b90..4e57e86615 100644 --- a/internal/datacoord/compaction_trigger.go +++ b/internal/datacoord/compaction_trigger.go @@ -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(), } diff --git a/internal/datacoord/compaction_trigger_test.go b/internal/datacoord/compaction_trigger_test.go index 125a7eac27..f1e631f45e 100644 --- a/internal/datacoord/compaction_trigger_test.go +++ b/internal/datacoord/compaction_trigger_test.go @@ -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)) } diff --git a/internal/datacoord/index_builder.go b/internal/datacoord/index_builder.go index 4a3f258489..745eabd0a1 100644 --- a/internal/datacoord/index_builder.go +++ b/internal/datacoord/index_builder.go @@ -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 } } } diff --git a/internal/datacoord/index_builder_test.go b/internal/datacoord/index_builder_test.go index b45c4e5e93..d08bb2a7ef 100644 --- a/internal/datacoord/index_builder_test.go +++ b/internal/datacoord/index_builder_test.go @@ -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() diff --git a/internal/datacoord/index_meta.go b/internal/datacoord/index_meta.go index c2a9e1d534..1bd2fd1c6c 100644 --- a/internal/datacoord/index_meta.go +++ b/internal/datacoord/index_meta.go @@ -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}) diff --git a/internal/datacoord/index_meta_test.go b/internal/datacoord/index_meta_test.go index e2c6bcbbbd..709d2e69b3 100644 --- a/internal/datacoord/index_meta_test.go +++ b/internal/datacoord/index_meta_test.go @@ -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) }) } diff --git a/internal/datacoord/index_service.go b/internal/datacoord/index_service.go index a15447de79..318b0f79cd 100644 --- a/internal/datacoord/index_service.go +++ b/internal/datacoord/index_service.go @@ -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 { diff --git a/internal/datacoord/indexnode_manager.go b/internal/datacoord/indexnode_manager.go index 7a6721f72b..f16d516083 100644 --- a/internal/datacoord/indexnode_manager.go +++ b/internal/datacoord/indexnode_manager.go @@ -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 { diff --git a/internal/datacoord/indexnode_manager_test.go b/internal/datacoord/indexnode_manager_test.go index 6abad8c191..6e13bf784c 100644 --- a/internal/datacoord/indexnode_manager_test.go +++ b/internal/datacoord/indexnode_manager_test.go @@ -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) }) } diff --git a/internal/datacoord/meta.go b/internal/datacoord/meta.go index e73de2afb4..1b247365bd 100644 --- a/internal/datacoord/meta.go +++ b/internal/datacoord/meta.go @@ -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 +} diff --git a/internal/datacoord/mock_compaction_meta.go b/internal/datacoord/mock_compaction_meta.go index b76c663839..15dc3ce6f5 100644 --- a/internal/datacoord/mock_compaction_meta.go +++ b/internal/datacoord/mock_compaction_meta.go @@ -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) diff --git a/internal/datacoord/mock_test.go b/internal/datacoord/mock_test.go index c6a16b180f..8b754eb7de 100644 --- a/internal/datacoord/mock_test.go +++ b/internal/datacoord/mock_test.go @@ -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) } diff --git a/internal/datacoord/mock_worker_manager.go b/internal/datacoord/mock_worker_manager.go new file mode 100644 index 0000000000..d192d979b9 --- /dev/null +++ b/internal/datacoord/mock_worker_manager.go @@ -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 +} diff --git a/internal/datacoord/server.go b/internal/datacoord/server.go index 6ee0eb1f1b..5fb4df764c 100644 --- a/internal/datacoord/server.go +++ b/internal/datacoord/server.go @@ -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) diff --git a/internal/datacoord/server_test.go b/internal/datacoord/server_test.go index 1d221b1507..2c5fa33123 100644 --- a/internal/datacoord/server_test.go +++ b/internal/datacoord/server_test.go @@ -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 }, }, diff --git a/internal/datacoord/services.go b/internal/datacoord/services.go index dce783af52..6c6f313427 100644 --- a/internal/datacoord/services.go +++ b/internal/datacoord/services.go @@ -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) diff --git a/internal/datanode/clustering_compactor.go b/internal/datanode/clustering_compactor.go new file mode 100644 index 0000000000..3e8f029ee5 --- /dev/null +++ b/internal/datanode/clustering_compactor.go @@ -0,0 +1,1085 @@ +// 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 ( + "context" + "fmt" + "path" + "sort" + "strconv" + "sync" + "sync/atomic" + "time" + + "github.com/cockroachdb/errors" + "github.com/golang/protobuf/proto" + "github.com/samber/lo" + "go.opentelemetry.io/otel" + "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/datanode/allocator" + "github.com/milvus-io/milvus/internal/datanode/io" + "github.com/milvus-io/milvus/internal/datanode/metacache" + "github.com/milvus-io/milvus/internal/datanode/syncmgr" + "github.com/milvus-io/milvus/internal/metastore/kv/binlog" + "github.com/milvus-io/milvus/internal/proto/datapb" + "github.com/milvus-io/milvus/internal/proto/etcdpb" + "github.com/milvus-io/milvus/internal/proto/segcorepb" + "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/metrics" + "github.com/milvus-io/milvus/pkg/util/conc" + "github.com/milvus-io/milvus/pkg/util/funcutil" + "github.com/milvus-io/milvus/pkg/util/hardware" + "github.com/milvus-io/milvus/pkg/util/lock" + "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" + "github.com/milvus-io/milvus/pkg/util/tsoutil" + "github.com/milvus-io/milvus/pkg/util/typeutil" +) + +type clusteringCompactionTask struct { + compactor + io io.BinlogIO + //stageIO io.BinlogIO + allocator allocator.Allocator + metaCache metacache.MetaCache + syncMgr syncmgr.SyncManager + + ctx context.Context + cancel context.CancelFunc + done chan struct{} + tr *timerecord.TimeRecorder + + plan *datapb.CompactionPlan + + // schedule + totalBufferSize atomic.Int64 + spillChan chan SpillSignal + spillCount atomic.Int64 + pool *conc.Pool[any] + + // inner field + collectionID int64 + partitionID int64 + collectionMeta *etcdpb.CollectionMeta + currentTs typeutil.Timestamp // for TTL + isVectorClusteringKey bool + clusteringKeyField *schemapb.FieldSchema + primaryKeyField *schemapb.FieldSchema + + spillMutex sync.Mutex + memoryBufferSize int64 + clusterBuffers []*ClusterBuffer + clusterBufferLocks *lock.KeyLock[int] + // scalar + keyToBufferFunc func(interface{}) *ClusterBuffer + // vector + segmentIDOffsetMapping map[int64]string + offsetToBufferFunc func(int64, []uint32) *ClusterBuffer +} + +type ClusterBuffer struct { + id int + bufferSize int64 + buffer *InsertData + bufferRowNum int64 + bufferTimeStampFrom int64 + bufferTimeStampTo int64 + + currentSegmentID int64 + currentSpillSize int64 + currentSpillRowNum int64 + currentSpillBinlogs map[UniqueID]*datapb.FieldBinlog + currentPKStats *storage.PrimaryKeyStats + + uploadedSegments []*datapb.CompactionSegment + uploadedSegmentStats map[UniqueID]storage.SegmentStats + + clusteringKeyFieldStats *storage.FieldStats +} + +type SpillSignal struct { + buffer *ClusterBuffer +} + +func newClusteringCompactionTask( + ctx context.Context, + binlogIO io.BinlogIO, + //stagingIO io.BinlogIO, + alloc allocator.Allocator, + metaCache metacache.MetaCache, + syncMgr syncmgr.SyncManager, + plan *datapb.CompactionPlan, +) *clusteringCompactionTask { + ctx, cancel := context.WithCancel(ctx) + return &clusteringCompactionTask{ + ctx: ctx, + cancel: cancel, + io: binlogIO, + //stageIO: stagingIO, + allocator: alloc, + metaCache: metaCache, + syncMgr: syncMgr, + plan: plan, + tr: timerecord.NewTimeRecorder("clustering_compaction"), + done: make(chan struct{}, 1), + totalBufferSize: atomic.Int64{}, + spillChan: make(chan SpillSignal, 100), + pool: conc.NewPool[any](hardware.GetCPUNum() * 2), + clusterBuffers: make([]*ClusterBuffer, 0), + clusterBufferLocks: lock.NewKeyLock[int](), + } +} + +func (t *clusteringCompactionTask) complete() { + t.done <- struct{}{} +} + +func (t *clusteringCompactionTask) stop() { + t.cancel() + <-t.done +} + +func (t *clusteringCompactionTask) getPlanID() UniqueID { + return t.plan.GetPlanID() +} + +func (t *clusteringCompactionTask) getChannelName() string { + return t.plan.GetChannel() +} + +func (t *clusteringCompactionTask) getCollection() int64 { + return t.metaCache.Collection() +} + +// injectDone unlock the segments +func (t *clusteringCompactionTask) injectDone() { + for _, binlog := range t.plan.SegmentBinlogs { + t.syncMgr.Unblock(binlog.SegmentID) + } +} + +func (t *clusteringCompactionTask) init() error { + segIDs := make([]UniqueID, 0, len(t.plan.GetSegmentBinlogs())) + for _, s := range t.plan.GetSegmentBinlogs() { + segIDs = append(segIDs, s.GetSegmentID()) + } + collectionID, partitionID, meta, err := t.getSegmentMeta(segIDs[0]) + if err != nil { + return err + } + t.collectionID = collectionID + t.partitionID = partitionID + t.collectionMeta = meta + + clusteringKeyID := t.plan.GetClusteringKeyId() + var clusteringKeyField *schemapb.FieldSchema + var pkField *schemapb.FieldSchema + for _, field := range meta.Schema.Fields { + if field.FieldID == clusteringKeyID { + clusteringKeyField = field + } + if field.GetIsPrimaryKey() && field.GetFieldID() >= 100 && typeutil.IsPrimaryFieldType(field.GetDataType()) { + pkField = field + } + } + t.clusteringKeyField = clusteringKeyField + t.primaryKeyField = pkField + t.isVectorClusteringKey = typeutil.IsVectorType(t.clusteringKeyField.DataType) + t.currentTs = tsoutil.GetCurrentTime() + t.memoryBufferSize = t.getMemoryBufferSize() + log.Info("clustering compaction memory buffer", zap.Int64("size", t.memoryBufferSize)) + + return nil +} + +func (t *clusteringCompactionTask) compact() (*datapb.CompactionPlanResult, error) { + ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(t.ctx, fmt.Sprintf("L2Compact-%d", t.getPlanID())) + defer span.End() + log := log.With(zap.Int64("planID", t.plan.GetPlanID()), zap.String("type", t.plan.GetType().String())) + if t.plan.GetType() != datapb.CompactionType_ClusteringCompaction { + // this shouldn't be reached + log.Warn("compact wrong, illegal compaction type") + return nil, errIllegalCompactionPlan + } + log.Info("L2 compaction", zap.Duration("wait in queue elapse", t.tr.RecordSpan())) + if !funcutil.CheckCtxValid(ctx) { + log.Warn("compact wrong, task context done or timeout") + return nil, errContext + } + ctxTimeout, cancelAll := context.WithTimeout(ctx, time.Duration(t.plan.GetTimeoutInSeconds())*time.Second) + defer cancelAll() + + err := t.init() + if err != nil { + return nil, err + } + log.Info("compact start", zap.Int32("timeout in seconds", t.plan.GetTimeoutInSeconds())) + + err = binlog.DecompressCompactionBinlogs(t.plan.GetSegmentBinlogs()) + if err != nil { + log.Warn("DecompressCompactionBinlogs fails", zap.Error(err)) + return nil, err + } + segIDs := make([]UniqueID, 0, len(t.plan.GetSegmentBinlogs())) + for _, s := range t.plan.GetSegmentBinlogs() { + segIDs = append(segIDs, s.GetSegmentID()) + } + + // Inject to stop flush + injectStart := time.Now() + for _, segID := range segIDs { + t.syncMgr.Block(segID) + } + log.Info("compact inject elapse", zap.Duration("elapse", time.Since(injectStart))) + defer func() { + if err != nil { + for _, segID := range segIDs { + t.syncMgr.Unblock(segID) + } + } + }() + + // 1, download delta logs to build deltaMap + deltaBlobs, _, err := loadDeltaMap(ctxTimeout, t.io, t.plan.GetSegmentBinlogs()) + if err != nil { + return nil, err + } + deltaPk2Ts, err := MergeDeltalogs(deltaBlobs) + if err != nil { + return nil, err + } + + // 2, final clean up + defer t.cleanUp(ctx) + + // todo move analyze to indexnode Analysis method + if t.isVectorClusteringKey { + analyzeResultPath := t.plan.AnalyzeResultPath + centroidFilePath := t.io.JoinFullPath(common.AnalyzeStatsPath, analyzeResultPath, metautil.JoinIDPath(t.collectionID, t.partitionID, t.clusteringKeyField.FieldID), "centroids") + centroidBytes, err := t.io.Download(ctx, []string{centroidFilePath}) + if err != nil { + return nil, err + } + centroids := &segcorepb.ClusteringCentroidsStats{} + err = proto.Unmarshal(centroidBytes[0], centroids) + if err != nil { + return nil, err + } + log.Debug("read clustering centroids stats", zap.String("path", centroidFilePath), zap.Int("centroidNum", len(centroids.GetCentroids()))) + offsetMappingFiles := make(map[int64]string, 0) + for _, segmentID := range t.plan.AnalyzeSegmentIds { + path := t.io.JoinFullPath(common.AnalyzeStatsPath, analyzeResultPath, metautil.JoinIDPath(t.collectionID, t.partitionID, t.clusteringKeyField.FieldID, segmentID), "offsets_mapping") + offsetMappingFiles[segmentID] = path + log.Debug("read segment offset mapping file", zap.Int64("segmentID", segmentID), zap.String("path", path)) + } + t.segmentIDOffsetMapping = offsetMappingFiles + + for id, centroid := range centroids.GetCentroids() { + fieldStats, err := storage.NewFieldStats(t.clusteringKeyField.FieldID, t.clusteringKeyField.DataType, 0) + if err != nil { + return nil, err + } + fieldStats.SetVectorCentroids(storage.NewVectorFieldValue(t.clusteringKeyField.DataType, centroid)) + clusterBuffer := &ClusterBuffer{ + id: id, + uploadedSegments: make([]*datapb.CompactionSegment, 0), + uploadedSegmentStats: make(map[UniqueID]storage.SegmentStats, 0), + clusteringKeyFieldStats: fieldStats, + } + t.clusterBuffers = append(t.clusterBuffers, clusterBuffer) + } + t.offsetToBufferFunc = func(offset int64, idMapping []uint32) *ClusterBuffer { + return t.clusterBuffers[idMapping[offset]] + } + } else { + analyzeDict, err := t.scalarAnalyze(ctx) + if err != nil { + return nil, err + } + plan := t.scalarPlan(analyzeDict) + scalarToClusterBufferMap := make(map[interface{}]*ClusterBuffer, 0) + for id, bucket := range plan { + fieldStats, err := storage.NewFieldStats(t.clusteringKeyField.FieldID, t.clusteringKeyField.DataType, 0) + if err != nil { + return nil, err + } + for _, key := range bucket { + fieldStats.UpdateMinMax(storage.NewScalarFieldValue(t.clusteringKeyField.DataType, key)) + } + buffer := &ClusterBuffer{ + id: id, + uploadedSegments: make([]*datapb.CompactionSegment, 0), + uploadedSegmentStats: make(map[UniqueID]storage.SegmentStats, 0), + clusteringKeyFieldStats: fieldStats, + } + t.clusterBuffers = append(t.clusterBuffers, buffer) + for _, key := range bucket { + scalarToClusterBufferMap[key] = buffer + } + } + t.keyToBufferFunc = func(key interface{}) *ClusterBuffer { + // todo: if keys are too many, the map will be quite large, we should mark the range of each buffer and select buffer by range + return scalarToClusterBufferMap[key] + } + } + + // 3, mapping + log.Info("L2 compaction start mapping", zap.Int("bufferNum", len(t.clusterBuffers))) + uploadSegments, partitionStats, err := t.mapping(ctx, deltaPk2Ts) + if err != nil { + return nil, err + } + + // 4, collect partition stats + err = t.uploadPartitionStats(ctx, t.collectionID, t.partitionID, partitionStats) + if err != nil { + return nil, err + } + + // 5, assemble CompactionPlanResult + planResult := &datapb.CompactionPlanResult{ + State: commonpb.CompactionState_Completed, + PlanID: t.getPlanID(), + Segments: uploadSegments, + Type: t.plan.GetType(), + Channel: t.plan.GetChannel(), + } + + metrics.DataNodeCompactionLatency. + WithLabelValues(fmt.Sprint(paramtable.GetNodeID()), t.plan.GetType().String()). + Observe(float64(t.tr.ElapseSpan().Milliseconds())) + log.Info("L2 compaction finished", zap.Duration("elapse", t.tr.ElapseSpan())) + + return planResult, nil +} + +// mapping read and split input segments into buffers +func (t *clusteringCompactionTask) mapping(ctx context.Context, + deltaPk2Ts map[interface{}]Timestamp, +) ([]*datapb.CompactionSegment, *storage.PartitionStatsSnapshot, error) { + inputSegments := t.plan.GetSegmentBinlogs() + mapStart := time.Now() + + // start spill goroutine + go t.backgroundSpill(ctx) + + futures := make([]*conc.Future[any], 0, len(inputSegments)) + for _, segment := range inputSegments { + segmentClone := &datapb.CompactionSegmentBinlogs{ + SegmentID: segment.SegmentID, + // only FieldBinlogs needed + FieldBinlogs: segment.FieldBinlogs, + } + future := t.pool.Submit(func() (any, error) { + err := t.mappingSegment(ctx, segmentClone, deltaPk2Ts) + return struct{}{}, err + }) + futures = append(futures, future) + } + if err := conc.AwaitAll(futures...); err != nil { + return nil, nil, err + } + + // force spill all buffers + err := t.spillAll(ctx) + if err != nil { + return nil, nil, err + } + + resultSegments := make([]*datapb.CompactionSegment, 0) + resultPartitionStats := &storage.PartitionStatsSnapshot{ + SegmentStats: make(map[UniqueID]storage.SegmentStats, 0), + } + for _, buffer := range t.clusterBuffers { + for _, seg := range buffer.uploadedSegments { + se := &datapb.CompactionSegment{ + PlanID: seg.GetPlanID(), + SegmentID: seg.GetSegmentID(), + NumOfRows: seg.GetNumOfRows(), + InsertLogs: seg.GetInsertLogs(), + Field2StatslogPaths: seg.GetField2StatslogPaths(), + Deltalogs: seg.GetDeltalogs(), + Channel: seg.GetChannel(), + } + log.Debug("put segment into final compaction result", zap.String("segment", se.String())) + resultSegments = append(resultSegments, se) + } + for segID, segmentStat := range buffer.uploadedSegmentStats { + log.Debug("put segment into final partition stats", zap.Int64("segmentID", segID), zap.Any("stats", segmentStat)) + resultPartitionStats.SegmentStats[segID] = segmentStat + } + } + + log.Info("mapping end", + zap.Int64("collectionID", t.getCollection()), + zap.Int64("partitionID", t.partitionID), + zap.Int("segmentFrom", len(inputSegments)), + zap.Int("segmentTo", len(resultSegments)), + zap.Duration("elapse", time.Since(mapStart))) + + return resultSegments, resultPartitionStats, nil +} + +// read insert log of one segment, mappingSegment it into buckets according to partitionKey. Spill data to file when necessary +func (t *clusteringCompactionTask) mappingSegment( + ctx context.Context, + segment *datapb.CompactionSegmentBinlogs, + delta map[interface{}]Timestamp, +) error { + ctx, span := otel.Tracer(typeutil.DataNodeRole).Start(ctx, fmt.Sprintf("Compact-Map-%d", t.getPlanID())) + defer span.End() + log := log.With(zap.Int64("planID", t.getPlanID()), + zap.Int64("collectionID", t.getCollection()), + zap.Int64("partitionID", t.partitionID), + zap.Int64("segmentID", segment.GetSegmentID())) + log.Info("mapping segment start") + processStart := time.Now() + fieldBinlogPaths := make([][]string, 0) + var ( + expired int64 = 0 + deleted int64 = 0 + remained int64 = 0 + ) + + isDeletedValue := func(v *storage.Value) bool { + ts, ok := delta[v.PK.GetValue()] + // insert task and delete task has the same ts when upsert + // here should be < instead of <= + // to avoid the upsert data to be deleted after compact + if ok && uint64(v.Timestamp) < ts { + return true + } + return false + } + + mappingStats := &segcorepb.ClusteringCentroidIdMappingStats{} + if t.isVectorClusteringKey { + offSetPath := t.segmentIDOffsetMapping[segment.SegmentID] + offsetBytes, err := t.io.Download(ctx, []string{offSetPath}) + if err != nil { + return err + } + err = proto.Unmarshal(offsetBytes[0], mappingStats) + if err != nil { + return err + } + } + + // Get the number of field binlog files from non-empty segment + var binlogNum int + for _, b := range segment.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 errIllegalCompactionPlan + } + for idx := 0; idx < binlogNum; idx++ { + var ps []string + for _, f := range segment.GetFieldBinlogs() { + ps = append(ps, f.GetBinlogs()[idx].GetLogPath()) + } + fieldBinlogPaths = append(fieldBinlogPaths, ps) + } + + for _, path := range fieldBinlogPaths { + bytesArr, err := t.io.Download(ctx, path) + blobs := make([]*Blob, len(bytesArr)) + for i := range bytesArr { + blobs[i] = &Blob{Value: bytesArr[i]} + } + if err != nil { + log.Warn("download insertlogs wrong", zap.Strings("path", path), zap.Error(err)) + return err + } + + pkIter, err := storage.NewInsertBinlogIterator(blobs, t.primaryKeyField.GetFieldID(), t.primaryKeyField.GetDataType()) + if err != nil { + log.Warn("new insert binlogs Itr wrong", zap.Strings("path", path), zap.Error(err)) + return err + } + // approximate row size + rowSize := pkIter.DataSize() / pkIter.RowNum() + + var offset int64 = -1 + for pkIter.HasNext() { + vInter, _ := pkIter.Next() + v, ok := vInter.(*storage.Value) + if !ok { + log.Warn("transfer interface to Value wrong", zap.Strings("path", path)) + return errors.New("unexpected error") + } + offset++ + + // Filtering deleted entity + if isDeletedValue(v) { + deleted++ + continue + } + // Filtering expired entity + ts := Timestamp(v.Timestamp) + if IsExpiredEntity(t.plan.GetCollectionTtl(), ts, t.currentTs) { + expired++ + continue + } + + row, ok := v.Value.(map[UniqueID]interface{}) + if !ok { + log.Warn("transfer interface to map wrong", zap.Strings("path", path)) + return errors.New("unexpected error") + } + + clusteringKey := row[t.plan.GetClusteringKeyId()] + var clusterBuffer *ClusterBuffer + if t.isVectorClusteringKey { + clusterBuffer = t.offsetToBufferFunc(offset, mappingStats.GetCentroidIdMapping()) + } else { + clusterBuffer = t.keyToBufferFunc(clusteringKey) + } + err = t.writeToBuffer(ctx, clusterBuffer, v, rowSize) + if err != nil { + return err + } + remained++ + + // trigger spill + if clusterBuffer.bufferRowNum > t.plan.GetMaxSegmentRows() || + clusterBuffer.bufferSize > int64(Params.DataNodeCfg.BinLogMaxSize.GetAsInt()) { + t.spillChan <- SpillSignal{ + buffer: clusterBuffer, + } + } else if t.totalBufferSize.Load() >= t.memoryBufferSize { + t.spillChan <- SpillSignal{} + } + // block here, wait for memory release by spill + currentSize := t.totalBufferSize.Load() + if currentSize > t.getSpillMemorySizeThreshold() { + loop: + for { + select { + case <-ctx.Done(): + log.Warn("stop waiting for memory buffer release as context done") + return nil + case <-t.done: + log.Warn("stop waiting for memory buffer release as task chan done") + return nil + default: + currentSize := t.totalBufferSize.Load() + if currentSize < t.getSpillMemorySizeThreshold() { + break loop + } + time.Sleep(time.Millisecond * 50) + } + } + } + } + } + + log.Info("mapping segment end", + zap.Int64("remained_entities", remained), + zap.Int64("deleted_entities", deleted), + zap.Int64("expired_entities", expired), + zap.Duration("elapse", time.Since(processStart))) + return nil +} + +func (t *clusteringCompactionTask) writeToBuffer(ctx context.Context, clusterBuffer *ClusterBuffer, value *storage.Value, rowSize int) error { + pk := value.PK + timestamp := value.Timestamp + row, ok := value.Value.(map[UniqueID]interface{}) + if !ok { + log.Warn("transfer interface to map wrong") + return errors.New("unexpected error") + } + + t.clusterBufferLocks.Lock(clusterBuffer.id) + defer t.clusterBufferLocks.Unlock(clusterBuffer.id) + // prepare + if clusterBuffer.currentSegmentID == 0 { + segmentID, err := t.allocator.AllocOne() + if err != nil { + return err + } + clusterBuffer.currentSegmentID = segmentID + clusterBuffer.currentSpillBinlogs = make(map[UniqueID]*datapb.FieldBinlog, 0) + clusterBuffer.bufferTimeStampFrom = -1 + clusterBuffer.bufferTimeStampTo = -1 + } + if clusterBuffer.currentPKStats == nil { + stats, err := storage.NewPrimaryKeyStats(t.primaryKeyField.FieldID, int64(t.primaryKeyField.DataType), 1) + if err != nil { + return err + } + clusterBuffer.currentPKStats = stats + } + if clusterBuffer.buffer == nil { + writeBuffer, err := storage.NewInsertData(t.collectionMeta.GetSchema()) + if err != nil { + return err + } + clusterBuffer.buffer = writeBuffer + } + + // Update timestampFrom, timestampTo + if timestamp < clusterBuffer.bufferTimeStampFrom || clusterBuffer.bufferTimeStampFrom == -1 { + clusterBuffer.bufferTimeStampFrom = timestamp + } + if timestamp > clusterBuffer.bufferTimeStampTo || clusterBuffer.bufferTimeStampFrom == -1 { + clusterBuffer.bufferTimeStampTo = timestamp + } + clusterBuffer.buffer.Append(row) + clusterBuffer.bufferSize = clusterBuffer.bufferSize + int64(rowSize) + clusterBuffer.bufferRowNum = clusterBuffer.bufferRowNum + 1 + clusterBuffer.currentPKStats.Update(pk) + t.totalBufferSize.Add(int64(rowSize)) + return nil +} + +// getMemoryBufferSize return memoryBufferSize +func (t *clusteringCompactionTask) getMemoryBufferSize() int64 { + return int64(float64(hardware.GetMemoryCount()) * Params.DataNodeCfg.ClusteringCompactionMemoryBufferRatio.GetAsFloat()) +} + +func (t *clusteringCompactionTask) getSpillMemorySizeThreshold() int64 { + return int64(float64(t.memoryBufferSize) * 0.8) +} + +func (t *clusteringCompactionTask) backgroundSpill(ctx context.Context) { + for { + select { + case <-ctx.Done(): + log.Info("clustering compaction task context exit") + return + case <-t.done: + log.Info("clustering compaction task done") + return + case signal := <-t.spillChan: + var err error + if signal.buffer == nil { + err = t.spillLargestBuffers(ctx) + } else { + err = func() error { + t.clusterBufferLocks.Lock(signal.buffer.id) + defer t.clusterBufferLocks.Unlock(signal.buffer.id) + return t.spill(ctx, signal.buffer) + }() + } + if err != nil { + log.Warn("fail to spill data", zap.Error(err)) + // todo handle error + } + } + } +} + +func (t *clusteringCompactionTask) spillLargestBuffers(ctx context.Context) error { + // only one spillLargestBuffers or spillAll should do at the same time + t.spillMutex.Lock() + defer t.spillMutex.Unlock() + bufferIDs := make([]int, 0) + for _, buffer := range t.clusterBuffers { + bufferIDs = append(bufferIDs, buffer.id) + t.clusterBufferLocks.Lock(buffer.id) + } + defer func() { + for _, buffer := range t.clusterBuffers { + t.clusterBufferLocks.Unlock(buffer.id) + } + }() + sort.Slice(bufferIDs, func(i, j int) bool { + return t.clusterBuffers[i].bufferSize > t.clusterBuffers[j].bufferSize + }) + for index, id := range bufferIDs { + err := t.spill(ctx, t.clusterBuffers[id]) + if err != nil { + return err + } + if index >= len(bufferIDs) { + break + } + } + return nil +} + +func (t *clusteringCompactionTask) spillAll(ctx context.Context) error { + // only one spillLargestBuffers or spillAll should do at the same time + t.spillMutex.Lock() + defer t.spillMutex.Unlock() + for _, buffer := range t.clusterBuffers { + t.clusterBufferLocks.Lock(buffer.id) + } + defer func() { + for _, buffer := range t.clusterBuffers { + t.clusterBufferLocks.Unlock(buffer.id) + } + }() + for _, buffer := range t.clusterBuffers { + err := func() error { + err := t.spill(ctx, buffer) + if err != nil { + log.Error("spill fail") + return err + } + err = t.packBuffersToSegments(ctx, buffer) + return err + }() + if err != nil { + return err + } + } + t.totalBufferSize.Store(0) + return nil +} + +func (t *clusteringCompactionTask) packBuffersToSegments(ctx context.Context, buffer *ClusterBuffer) error { + if len(buffer.currentSpillBinlogs) == 0 { + return nil + } + insertLogs := make([]*datapb.FieldBinlog, 0) + for _, fieldBinlog := range buffer.currentSpillBinlogs { + insertLogs = append(insertLogs, fieldBinlog) + } + iCodec := storage.NewInsertCodecWithSchema(t.collectionMeta) + statPaths, err := uploadStatsLog(ctx, t.io, t.allocator, t.collectionID, t.partitionID, buffer.currentSegmentID, buffer.currentPKStats, buffer.currentSpillRowNum, iCodec) + if err != nil { + return err + } + statsLogs := make([]*datapb.FieldBinlog, 0) + for _, fieldBinlog := range statPaths { + statsLogs = append(statsLogs, fieldBinlog) + } + + // pack current spill data into a segment + seg := &datapb.CompactionSegment{ + PlanID: t.plan.GetPlanID(), + SegmentID: buffer.currentSegmentID, + NumOfRows: buffer.currentSpillRowNum, + InsertLogs: insertLogs, + Field2StatslogPaths: statsLogs, + Channel: t.plan.GetChannel(), + } + buffer.uploadedSegments = append(buffer.uploadedSegments, seg) + segmentStats := storage.SegmentStats{ + FieldStats: []storage.FieldStats{{ + FieldID: buffer.clusteringKeyFieldStats.FieldID, + Type: buffer.clusteringKeyFieldStats.Type, + Max: buffer.clusteringKeyFieldStats.Max, + Min: buffer.clusteringKeyFieldStats.Min, + BF: buffer.clusteringKeyFieldStats.BF, + Centroids: buffer.clusteringKeyFieldStats.Centroids, + }}, + NumRows: int(buffer.currentSpillRowNum), + } + buffer.uploadedSegmentStats[buffer.currentSegmentID] = segmentStats + // refresh + buffer.currentSpillRowNum = 0 + buffer.currentSpillSize = 0 + buffer.currentPKStats = nil + segmentID, err := t.allocator.AllocOne() + if err != nil { + return err + } + buffer.currentSegmentID = segmentID + buffer.currentSpillBinlogs = make(map[UniqueID]*datapb.FieldBinlog, 0) + log.Debug("finish pack segment", zap.Int64("segID", buffer.currentSegmentID), zap.String("seg", seg.String()), zap.Any("segStats", segmentStats)) + return nil +} + +func (t *clusteringCompactionTask) spill(ctx context.Context, buffer *ClusterBuffer) error { + if buffer.buffer.IsEmpty() { + return nil + } + + iCodec := storage.NewInsertCodecWithSchema(t.collectionMeta) + inPaths, err := uploadInsertLog(ctx, t.io, t.allocator, t.collectionID, t.partitionID, buffer.currentSegmentID, buffer.buffer, iCodec) + if err != nil { + return err + } + + for fID, path := range inPaths { + for _, binlog := range path.GetBinlogs() { + binlog.TimestampTo = uint64(buffer.bufferTimeStampTo) + binlog.TimestampFrom = uint64(buffer.bufferTimeStampFrom) + } + tmpBinlog, ok := buffer.currentSpillBinlogs[fID] + if !ok { + tmpBinlog = path + } else { + tmpBinlog.Binlogs = append(tmpBinlog.Binlogs, path.GetBinlogs()...) + } + buffer.currentSpillBinlogs[fID] = tmpBinlog + } + buffer.currentSpillRowNum = buffer.currentSpillRowNum + buffer.bufferRowNum + + // clean buffer + t.totalBufferSize.Add(-buffer.bufferSize) + buffer.buffer = nil + buffer.bufferSize = 0 + buffer.bufferRowNum = 0 + + if buffer.currentSpillRowNum > t.plan.GetMaxSegmentRows() { + t.packBuffersToSegments(ctx, buffer) + } + return nil +} + +func (t *clusteringCompactionTask) getSegmentMeta(segID UniqueID) (UniqueID, UniqueID, *etcdpb.CollectionMeta, error) { + collID := t.metaCache.Collection() + seg, ok := t.metaCache.GetSegmentByID(segID) + if !ok { + return -1, -1, nil, merr.WrapErrSegmentNotFound(segID) + } + partID := seg.PartitionID() + sch := t.metaCache.Schema() + + meta := &etcdpb.CollectionMeta{ + ID: collID, + Schema: sch, + } + return collID, partID, meta, nil +} + +func (t *clusteringCompactionTask) uploadPartitionStats(ctx context.Context, collectionID, partitionID UniqueID, partitionStats *storage.PartitionStatsSnapshot) error { + // use planID as partitionStats version + version := t.plan.PlanID + partitionStats.Version = version + partitionStatsBytes, err := storage.SerializePartitionStatsSnapshot(partitionStats) + if err != nil { + return err + } + newStatsPath := t.io.JoinFullPath(common.PartitionStatsTempPath, + path.Join(strconv.FormatInt(collectionID, 10), strconv.FormatInt(partitionID, 10), t.plan.GetChannel(), strconv.FormatInt(version, 10))) + kv := map[string][]byte{ + newStatsPath: partitionStatsBytes, + } + err = t.io.Upload(ctx, kv) + if err != nil { + return err + } + log.Info("Finish upload PartitionStats file", zap.String("key", newStatsPath), zap.Int("length", len(partitionStatsBytes))) + return nil +} + +// cleanUp try best to clean all temp datas +func (t *clusteringCompactionTask) cleanUp(ctx context.Context) { + //stagePath := t.stageIO.JoinFullPath(common.CompactionStagePath, metautil.JoinIDPath(t.plan.PlanID)) + //err := t.stageIO.Remove(ctx, stagePath) + //if err != nil { + // log.Warn("Fail to remove staging data", zap.String("key", stagePath), zap.Error(err)) + //} +} + +func (t *clusteringCompactionTask) scalarAnalyze(ctx context.Context) (map[interface{}]int64, error) { + inputSegments := t.plan.GetSegmentBinlogs() + futures := make([]*conc.Future[any], 0, len(inputSegments)) + mapStart := time.Now() + var mutex sync.Mutex + analyzeDict := make(map[interface{}]int64, 0) + for _, segment := range inputSegments { + segmentClone := &datapb.CompactionSegmentBinlogs{ + SegmentID: segment.SegmentID, + FieldBinlogs: segment.FieldBinlogs, + Field2StatslogPaths: segment.Field2StatslogPaths, + Deltalogs: segment.Deltalogs, + InsertChannel: segment.InsertChannel, + Level: segment.Level, + CollectionID: segment.CollectionID, + PartitionID: segment.PartitionID, + } + future := t.pool.Submit(func() (any, error) { + analyzeResult, err := t.scalarAnalyzeSegment(ctx, segmentClone) + mutex.Lock() + defer mutex.Unlock() + for key, v := range analyzeResult { + if _, exist := analyzeDict[key]; exist { + analyzeDict[key] = analyzeDict[key] + v + } else { + analyzeDict[key] = v + } + } + return struct{}{}, err + }) + futures = append(futures, future) + } + if err := conc.AwaitAll(futures...); err != nil { + return nil, err + } + log.Info("analyze end", + zap.Int64("collectionID", t.getCollection()), + zap.Int64("partitionID", t.partitionID), + zap.Int("segments", len(inputSegments)), + zap.Duration("elapse", time.Since(mapStart))) + return analyzeDict, nil +} + +func (t *clusteringCompactionTask) scalarAnalyzeSegment( + ctx context.Context, + segment *datapb.CompactionSegmentBinlogs, +) (map[interface{}]int64, error) { + log := log.With(zap.Int64("planID", t.getPlanID()), zap.Int64("segmentID", segment.GetSegmentID())) + + // vars + processStart := time.Now() + fieldBinlogPaths := make([][]string, 0) + // initial timestampFrom, timestampTo = -1, -1 is an illegal value, only to mark initial state + var ( + timestampTo int64 = -1 + timestampFrom int64 = -1 + expired int64 = 0 + deleted int64 = 0 + remained int64 = 0 + analyzeResult map[interface{}]int64 = make(map[interface{}]int64, 0) + ) + + // Get the number of field binlog files from non-empty segment + var binlogNum int + for _, b := range segment.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 + } + log.Debug("binlogNum", zap.Int("binlogNum", binlogNum)) + for idx := 0; idx < binlogNum; idx++ { + var ps []string + for _, f := range segment.GetFieldBinlogs() { + // todo add a new reader only read one column + if f.FieldID == t.primaryKeyField.GetFieldID() || f.FieldID == t.clusteringKeyField.GetFieldID() || f.FieldID == common.RowIDField || f.FieldID == common.TimeStampField { + ps = append(ps, f.GetBinlogs()[idx].GetLogPath()) + } + } + fieldBinlogPaths = append(fieldBinlogPaths, ps) + } + + for _, path := range fieldBinlogPaths { + bytesArr, err := t.io.Download(ctx, path) + blobs := make([]*Blob, len(bytesArr)) + for i := range bytesArr { + blobs[i] = &Blob{Value: bytesArr[i]} + } + if err != nil { + log.Warn("download insertlogs wrong", zap.Strings("path", path), zap.Error(err)) + return nil, err + } + + pkIter, err := storage.NewInsertBinlogIterator(blobs, t.primaryKeyField.GetFieldID(), t.primaryKeyField.GetDataType()) + if err != nil { + log.Warn("new insert binlogs Itr wrong", zap.Strings("path", path), zap.Error(err)) + return nil, err + } + + // log.Info("pkIter.RowNum()", zap.Int("pkIter.RowNum()", pkIter.RowNum()), zap.Bool("hasNext", pkIter.HasNext())) + for pkIter.HasNext() { + vIter, _ := pkIter.Next() + v, ok := vIter.(*storage.Value) + if !ok { + log.Warn("transfer interface to Value wrong", zap.Strings("path", path)) + return nil, errors.New("unexpected error") + } + + // Filtering expired entity + ts := Timestamp(v.Timestamp) + if IsExpiredEntity(t.plan.GetCollectionTtl(), ts, t.currentTs) { + expired++ + continue + } + + // Update timestampFrom, timestampTo + if v.Timestamp < timestampFrom || timestampFrom == -1 { + timestampFrom = v.Timestamp + } + if v.Timestamp > timestampTo || timestampFrom == -1 { + timestampTo = v.Timestamp + } + // rowValue := vIter.GetData().(*iterators.InsertRow).GetValue() + row, ok := v.Value.(map[UniqueID]interface{}) + if !ok { + log.Warn("transfer interface to map wrong", zap.Strings("path", path)) + return nil, errors.New("unexpected error") + } + key := row[t.clusteringKeyField.GetFieldID()] + if _, exist := analyzeResult[key]; exist { + analyzeResult[key] = analyzeResult[key] + 1 + } else { + analyzeResult[key] = 1 + } + remained++ + } + } + + log.Info("analyze segment end", + zap.Int64("remained entities", remained), + zap.Int64("deleted entities", deleted), + zap.Int64("expired entities", expired), + zap.Duration("map elapse", time.Since(processStart))) + return analyzeResult, nil +} + +func (t *clusteringCompactionTask) scalarPlan(dict map[interface{}]int64) [][]interface{} { + keys := lo.MapToSlice(dict, func(k interface{}, _ int64) interface{} { + return k + }) + sort.Slice(keys, func(i, j int) bool { + return storage.NewScalarFieldValue(t.clusteringKeyField.DataType, keys[i]).LE(storage.NewScalarFieldValue(t.clusteringKeyField.DataType, keys[j])) + }) + + buckets := make([][]interface{}, 0) + currentBucket := make([]interface{}, 0) + var currentBucketSize int64 = 0 + maxRows := t.plan.MaxSegmentRows + preferRows := t.plan.PreferSegmentRows + for _, key := range keys { + // todo can optimize + if dict[key] > preferRows { + buckets = append(buckets, currentBucket) + buckets = append(buckets, []interface{}{key}) + currentBucket = make([]interface{}, 0) + currentBucketSize = 0 + } else if currentBucketSize+dict[key] > maxRows { + buckets = append(buckets, currentBucket) + currentBucket = []interface{}{key} + currentBucketSize = dict[key] + } else if currentBucketSize+dict[key] > preferRows { + currentBucket = append(currentBucket, key) + buckets = append(buckets, currentBucket) + currentBucket = make([]interface{}, 0) + currentBucketSize = 0 + } else { + currentBucket = append(currentBucket, key) + currentBucketSize += dict[key] + } + } + buckets = append(buckets, currentBucket) + return buckets +} diff --git a/internal/datanode/compactor.go b/internal/datanode/compactor.go index 787870d71c..f13ecc17ff 100644 --- a/internal/datanode/compactor.go +++ b/internal/datanode/compactor.go @@ -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 } diff --git a/internal/datanode/compactor_common.go b/internal/datanode/compactor_common.go new file mode 100644 index 0000000000..4ecefb9eb7 --- /dev/null +++ b/internal/datanode/compactor_common.go @@ -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) +} diff --git a/internal/datanode/compactor_test.go b/internal/datanode/compactor_test.go index 8d3b4305d7..856958e726 100644 --- a/internal/datanode/compactor_test.go +++ b/internal/datanode/compactor_test.go @@ -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)) }) diff --git a/internal/datanode/iterators/iterator.go b/internal/datanode/iterators/iterator.go index 158db4ad80..4ca4a60304 100644 --- a/internal/datanode/iterators/iterator.go +++ b/internal/datanode/iterators/iterator.go @@ -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() } diff --git a/internal/datanode/services.go b/internal/datanode/services.go index 67819f42ae..97844ed77c 100644 --- a/internal/datanode/services.go +++ b/internal/datanode/services.go @@ -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 diff --git a/internal/datanode/services_test.go b/internal/datanode/services_test.go index 16d4157089..b6d0676ad0 100644 --- a/internal/datanode/services_test.go +++ b/internal/datanode/services_test.go @@ -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() { diff --git a/internal/distributed/indexnode/client/client.go b/internal/distributed/indexnode/client/client.go index 26505dd96f..1be7d8e8be 100644 --- a/internal/distributed/indexnode/client/client.go +++ b/internal/distributed/indexnode/client/client.go @@ -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) + }) +} diff --git a/internal/distributed/indexnode/client/client_test.go b/internal/distributed/indexnode/client/client_test.go index b88f7a9664..b2d6e17972 100644 --- a/internal/distributed/indexnode/client/client_test.go +++ b/internal/distributed/indexnode/client/client_test.go @@ -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) } diff --git a/internal/distributed/indexnode/service.go b/internal/distributed/indexnode/service.go index c315d52cfc..fae88fecdc 100644 --- a/internal/distributed/indexnode/service.go +++ b/internal/distributed/indexnode/service.go @@ -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) diff --git a/internal/distributed/indexnode/service_test.go b/internal/distributed/indexnode/service_test.go index edfc175423..724dc17e4d 100644 --- a/internal/distributed/indexnode/service_test.go +++ b/internal/distributed/indexnode/service_test.go @@ -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) } diff --git a/internal/indexnode/indexnode.go b/internal/indexnode/indexnode.go index 2dc7ab7580..80e66b8b82 100644 --- a/internal/indexnode/indexnode.go +++ b/internal/indexnode/indexnode.go @@ -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 { diff --git a/internal/indexnode/indexnode_service.go b/internal/indexnode/indexnode_service.go index 70631a5bc4..2f25d82623 100644 --- a/internal/indexnode/indexnode_service.go +++ b/internal/indexnode/indexnode_service.go @@ -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 +} diff --git a/internal/indexnode/indexnode_service_test.go b/internal/indexnode/indexnode_service_test.go index 255551d3e2..5e3021aff0 100644 --- a/internal/indexnode/indexnode_service_test.go +++ b/internal/indexnode/indexnode_service_test.go @@ -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)) +} diff --git a/internal/indexnode/indexnode_test.go b/internal/indexnode/indexnode_test.go index 5c2ff6cceb..aff9cf07b9 100644 --- a/internal/indexnode/indexnode_test.go +++ b/internal/indexnode/indexnode_test.go @@ -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() { diff --git a/internal/indexnode/meta.go b/internal/indexnode/meta.go new file mode 100644 index 0000000000..d67dc18efb --- /dev/null +++ b/internal/indexnode/meta.go @@ -0,0 +1 @@ +package indexnode diff --git a/internal/indexnode/task.go b/internal/indexnode/task.go index b14343900d..f5cd358ff1 100644 --- a/internal/indexnode/task.go +++ b/internal/indexnode/task.go @@ -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 -} diff --git a/internal/indexnode/task_analysis.go b/internal/indexnode/task_analysis.go new file mode 100644 index 0000000000..ebc90a0834 --- /dev/null +++ b/internal/indexnode/task_analysis.go @@ -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 +} diff --git a/internal/indexnode/task_index.go b/internal/indexnode/task_index.go new file mode 100644 index 0000000000..8b2a364473 --- /dev/null +++ b/internal/indexnode/task_index.go @@ -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 +} diff --git a/internal/indexnode/task_test.go b/internal/indexnode/task_test.go index dc30abd800..ab79aa4bd0 100644 --- a/internal/indexnode/task_test.go +++ b/internal/indexnode/task_test.go @@ -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)) +} diff --git a/internal/indexnode/taskinfo_ops.go b/internal/indexnode/taskinfo_ops.go index 166c73436a..7b405d272f 100644 --- a/internal/indexnode/taskinfo_ops.go +++ b/internal/indexnode/taskinfo_ops.go @@ -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)) } diff --git a/internal/metastore/catalog.go b/internal/metastore/catalog.go index ad604a0f5f..7c9dec0d39 100644 --- a/internal/metastore/catalog.go +++ b/internal/metastore/catalog.go @@ -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 { diff --git a/internal/metastore/kv/datacoord/constant.go b/internal/metastore/kv/datacoord/constant.go index a5fc3dcf57..d217ad90c4 100644 --- a/internal/metastore/kv/datacoord/constant.go +++ b/internal/metastore/kv/datacoord/constant.go @@ -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" diff --git a/internal/metastore/kv/datacoord/kv_catalog.go b/internal/metastore/kv/datacoord/kv_catalog.go index f9b9c86ce6..7d653b0e1d 100644 --- a/internal/metastore/kv/datacoord/kv_catalog.go +++ b/internal/metastore/kv/datacoord/kv_catalog.go @@ -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) +} diff --git a/internal/metastore/kv/datacoord/util.go b/internal/metastore/kv/datacoord/util.go index 09a979bafe..47d73b31b7 100644 --- a/internal/metastore/kv/datacoord/util.go +++ b/internal/metastore/kv/datacoord/util.go @@ -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) +} diff --git a/internal/metastore/mocks/mock_datacoord_catalog.go b/internal/metastore/mocks/mock_datacoord_catalog.go index a321655a1e..4d440312df 100644 --- a/internal/metastore/mocks/mock_datacoord_catalog.go +++ b/internal/metastore/mocks/mock_datacoord_catalog.go @@ -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) diff --git a/internal/metastore/model/analysis_stats.go b/internal/metastore/model/analysis_stats.go new file mode 100644 index 0000000000..29ec8e27c3 --- /dev/null +++ b/internal/metastore/model/analysis_stats.go @@ -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, + } +} diff --git a/internal/mocks/mock_indexnode.go b/internal/mocks/mock_indexnode.go index bcd158dc7b..e9a2b089bb 100644 --- a/internal/mocks/mock_indexnode.go +++ b/internal/mocks/mock_indexnode.go @@ -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) diff --git a/internal/mocks/mock_indexnode_client.go b/internal/mocks/mock_indexnode_client.go index 1e30de98ac..97a70130c7 100644 --- a/internal/mocks/mock_indexnode_client.go +++ b/internal/mocks/mock_indexnode_client.go @@ -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)) diff --git a/internal/proto/data_coord.proto b/internal/proto/data_coord.proto index fd6007a0dd..6fc6ba155e 100644 --- a/internal/proto/data_coord.proto +++ b/internal/proto/data_coord.proto @@ -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; +} \ No newline at end of file diff --git a/internal/proto/index_coord.proto b/internal/proto/index_coord.proto index d59452b17d..a49d0143b8 100644 --- a/internal/proto/index_coord.proto +++ b/internal/proto/index_coord.proto @@ -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 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 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 segment_offset_mapping_files = 4; + string fail_reason = 5; +} + +message QueryAnalysisResultResponse { + common.Status status = 1; + string clusterID = 2; + map results = 3; +} + +message DropAnalysisTasksRequest { + string clusterID = 1; + repeated int64 taskIDs = 2; +} diff --git a/internal/proto/segcore.proto b/internal/proto/segcore.proto index ea7697f48c..50aa994e39 100644 --- a/internal/proto/segcore.proto +++ b/internal/proto/segcore.proto @@ -42,4 +42,13 @@ message FieldIndexMeta { message CollectionIndexMeta { int64 maxIndexRowCount = 1; repeated FieldIndexMeta index_metas = 2; -} \ No newline at end of file +} + +message ClusteringCentroidsStats { + repeated schema.VectorField centroids = 1; +} + +message ClusteringCentroidIdMappingStats { + repeated uint32 centroid_id_mapping = 1; + repeated int64 num_in_centroid = 2; +} diff --git a/internal/querynodev2/delegator/segment_pruner.go b/internal/querynodev2/delegator/segment_pruner.go index 93f6c9650d..0577b22b49 100644 --- a/internal/querynodev2/delegator/segment_pruner.go +++ b/internal/querynodev2/delegator/segment_pruner.go @@ -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) diff --git a/internal/storage/binlog_iterator.go b/internal/storage/binlog_iterator.go index f620483982..ca0d4d6a9e 100644 --- a/internal/storage/binlog_iterator.go +++ b/internal/storage/binlog_iterator.go @@ -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 diff --git a/internal/storage/field_stats_test.go b/internal/storage/field_stats_test.go index e169902bf9..d58ffc1bf0 100644 --- a/internal/storage/field_stats_test.go +++ b/internal/storage/field_stats_test.go @@ -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) +} diff --git a/internal/storage/field_value.go b/internal/storage/field_value.go index d9f50cb6e3..07d4ec49be 100644 --- a/internal/storage/field_value.go +++ b/internal/storage/field_value.go @@ -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())) + } +} diff --git a/internal/storage/partition_stats.go b/internal/storage/partition_stats.go index 15173e4457..7a49953eb9 100644 --- a/internal/storage/partition_stats.go +++ b/internal/storage/partition_stats.go @@ -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 +} diff --git a/internal/util/analysiscgowrapper/analysis.go b/internal/util/analysiscgowrapper/analysis.go new file mode 100644 index 0000000000..d1ca1242b8 --- /dev/null +++ b/internal/util/analysiscgowrapper/analysis.go @@ -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 // 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 +} diff --git a/internal/util/analysiscgowrapper/analysis_info.go b/internal/util/analysiscgowrapper/analysis_info.go new file mode 100644 index 0000000000..e6135b16cf --- /dev/null +++ b/internal/util/analysiscgowrapper/analysis_info.go @@ -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 // 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") +} diff --git a/internal/util/analysiscgowrapper/helper.go b/internal/util/analysiscgowrapper/helper.go new file mode 100644 index 0000000000..fe177d952f --- /dev/null +++ b/internal/util/analysiscgowrapper/helper.go @@ -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 // 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 +} diff --git a/internal/util/clustering/clustering.go b/internal/util/clustering/clustering.go index c8b290f185..b985992233 100644 --- a/internal/util/clustering/clustering.go +++ b/internal/util/clustering/clustering.go @@ -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 +} diff --git a/internal/util/mock/grpc_indexnode_client.go b/internal/util/mock/grpc_indexnode_client.go index d8bbbd57c7..a63e3c3749 100644 --- a/internal/util/mock/grpc_indexnode_client.go +++ b/internal/util/mock/grpc_indexnode_client.go @@ -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 } diff --git a/internal/util/typeutil/schema.go b/internal/util/typeutil/schema.go index 399818410a..6854b31149 100644 --- a/internal/util/typeutil/schema.go +++ b/internal/util/typeutil/schema.go @@ -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 -} diff --git a/pkg/common/common.go b/pkg/common/common.go index 3be05d3eb1..71a14c0855 100644 --- a/pkg/common/common.go +++ b/pkg/common/common.go @@ -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 diff --git a/pkg/util/merr/errors.go b/pkg/util/merr/errors.go index 68af2f4230..370b9fc2b3 100644 --- a/pkg/util/merr/errors.go +++ b/pkg/util/merr/errors.go @@ -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 { diff --git a/pkg/util/merr/utils.go b/pkg/util/merr/utils.go index 0b078e73de..ff7f2ea2ff 100644 --- a/pkg/util/merr/utils.go +++ b/pkg/util/merr/utils.go @@ -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 +} diff --git a/pkg/util/paramtable/component_param.go b/pkg/util/paramtable/component_param.go index d6055caac6..97702294fc 100644 --- a/pkg/util/paramtable/component_param.go +++ b/pkg/util/paramtable/component_param.go @@ -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) } // ///////////////////////////////////////////////////////////////////////////// diff --git a/pkg/util/paramtable/component_param_test.go b/pkg/util/paramtable/component_param_test.go index 87b3166330..2852455c8e 100644 --- a/pkg/util/paramtable/component_param_test.go +++ b/pkg/util/paramtable/component_param_test.go @@ -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) { diff --git a/pkg/util/paramtable/param_item.go b/pkg/util/paramtable/param_item.go index d5965be460..50d3aca3cb 100644 --- a/pkg/util/paramtable/param_item.go +++ b/pkg/util/paramtable/param_item.go @@ -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 diff --git a/pkg/util/typeutil/schema.go b/pkg/util/typeutil/schema.go index 696092f566..71c0c492e3 100644 --- a/pkg/util/typeutil/schema.go +++ b/pkg/util/typeutil/schema.go @@ -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 diff --git a/pkg/util/typeutil/schema_test.go b/pkg/util/typeutil/schema_test.go index cb743d873b..c543a716a6 100644 --- a/pkg/util/typeutil/schema_test.go +++ b/pkg/util/typeutil/schema_test.go @@ -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 diff --git a/tests/integration/clustering/clustering_compaction_test.go b/tests/integration/clustering/clustering_compaction_test.go new file mode 100644 index 0000000000..449ea481de --- /dev/null +++ b/tests/integration/clustering/clustering_compaction_test.go @@ -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)) +}