From 6a6673f1b93abc5820f10a1a1a863a8f8649f639 Mon Sep 17 00:00:00 2001 From: groot Date: Tue, 18 Jun 2019 19:31:33 +0800 Subject: [PATCH 1/5] refine code Former-commit-id: af1b487c2c21aba34ac91b8f5df4ce97a5836919 --- cpp/conf/server_config.yaml | 1 - cpp/src/db/DB.cpp | 1 - cpp/src/db/DBImpl.cpp | 169 +++++----- cpp/src/db/DBImpl.h | 40 +-- cpp/src/db/DBMetaImpl.cpp | 299 +++++++++--------- cpp/src/db/Env.cpp | 87 ----- cpp/src/db/Env.h | 56 ---- cpp/src/db/MemManager.cpp | 7 + cpp/src/db/MemManager.h | 2 + cpp/src/db/MetaTypes.h | 22 +- cpp/src/db/Options.cpp | 4 +- cpp/src/db/Options.h | 1 - .../sdk/examples/simple/src/ClientTest.cpp | 6 +- cpp/src/server/RequestTask.cpp | 2 - cpp/src/server/ServerConfig.h | 1 - 15 files changed, 276 insertions(+), 422 deletions(-) delete mode 100644 cpp/src/db/Env.cpp delete mode 100644 cpp/src/db/Env.h diff --git a/cpp/conf/server_config.yaml b/cpp/conf/server_config.yaml index 612d6bb0d8..3d3a67f965 100644 --- a/cpp/conf/server_config.yaml +++ b/cpp/conf/server_config.yaml @@ -8,7 +8,6 @@ server_config: db_config: db_path: /tmp/milvus db_backend_url: http://127.0.0.1 - db_flush_interval: 5 #flush cache data into disk at intervals, unit: second index_building_threshold: 1024 #build index file when raw data file size larger than this value, unit: MB metric_config: diff --git a/cpp/src/db/DB.cpp b/cpp/src/db/DB.cpp index 31f14a90d0..ca08bbd9b4 100644 --- a/cpp/src/db/DB.cpp +++ b/cpp/src/db/DB.cpp @@ -6,7 +6,6 @@ #include "DBImpl.h" #include "DBMetaImpl.h" -#include "Env.h" #include "Factories.h" namespace zilliz { diff --git a/cpp/src/db/DBImpl.cpp b/cpp/src/db/DBImpl.cpp index 69a76981b2..b29d2021e3 100644 --- a/cpp/src/db/DBImpl.cpp +++ b/cpp/src/db/DBImpl.cpp @@ -5,7 +5,6 @@ ******************************************************************************/ #include "DBImpl.h" #include "DBMetaImpl.h" -#include "Env.h" #include "Log.h" #include "EngineFactory.h" #include "metrics/Metrics.h" @@ -125,13 +124,12 @@ void CalcScore(uint64_t vector_count, DBImpl::DBImpl(const Options& options) - : env_(options.env), - options_(options), - bg_compaction_scheduled_(false), + : options_(options), shutting_down_(false), - bg_build_index_started_(false), pMeta_(new meta::DBMetaImpl(options_.meta)), - pMemMgr_(new MemManager(pMeta_, options_)) { + pMemMgr_(new MemManager(pMeta_, options_)), + compact_thread_pool_(1, 1), + index_thread_pool_(1, 1) { StartTimerTasks(options_.memory_sync_interval); } @@ -140,28 +138,10 @@ Status DBImpl::CreateTable(meta::TableSchema& table_schema) { } Status DBImpl::DeleteTable(const std::string& table_id, const meta::DatesT& dates) { - meta::DatePartionedTableFilesSchema files; - auto status = pMeta_->FilesToDelete(table_id, dates, files); - if (!status.ok()) { return status; } - - for (auto &day_files : files) { - for (auto &file : day_files.second) { - boost::filesystem::remove(file.location_); - } - } - //dates empty means delete all files of the table if(dates.empty()) { - meta::TableSchema table_schema; - table_schema.table_id_ = table_id; - status = DescribeTable(table_schema); - - pMeta_->DeleteTable(table_id); - boost::system::error_code ec; - boost::filesystem::remove_all(table_schema.location_, ec); - if(ec.failed()) { - ENGINE_LOG_WARNING << "Failed to remove table folder"; - } + pMemMgr_->EraseMemVector(table_id); //not allow insert + pMeta_->DeleteTable(table_id); //soft delete } return Status::OK(); @@ -405,7 +385,15 @@ void DBImpl::BackgroundTimerTask(int interval) { server::SystemInfo::GetInstance().Init(); while (true) { if (!bg_error_.ok()) break; - if (shutting_down_.load(std::memory_order_acquire)) break; + if (shutting_down_.load(std::memory_order_acquire)){ + for(auto& iter : compact_thread_results_) { + iter.wait(); + } + for(auto& iter : index_thread_results_) { + iter.wait(); + } + break; + } std::this_thread::sleep_for(std::chrono::seconds(interval)); @@ -421,33 +409,34 @@ void DBImpl::BackgroundTimerTask(int interval) { server::Metrics::GetInstance().GPUPercentGaugeSet(); server::Metrics::GetInstance().GPUMemoryUsageGaugeSet(); server::Metrics::GetInstance().OctetsSet(); - TrySchedule(); + + StartCompactionTask(); + StartBuildIndexTask(); } } -void DBImpl::TrySchedule() { - if (bg_compaction_scheduled_) return; - if (!bg_error_.ok()) return; +void DBImpl::StartCompactionTask() { + //serialize memory data + std::vector temp_table_ids; + pMemMgr_->Serialize(temp_table_ids); + for(auto& id : temp_table_ids) { + compact_table_ids_.insert(id); + } - bg_compaction_scheduled_ = true; - env_->Schedule(&DBImpl::BGWork, this); -} + //compactiong has been finished? + if(!compact_thread_results_.empty()) { + std::chrono::milliseconds span(10); + if (compact_thread_results_.back().wait_for(span) == std::future_status::ready) { + compact_thread_results_.pop_back(); + } + } -void DBImpl::BGWork(void* db_) { - reinterpret_cast(db_)->BackgroundCall(); -} - -void DBImpl::BackgroundCall() { - std::lock_guard lock(mutex_); - assert(bg_compaction_scheduled_); - - if (!bg_error_.ok() || shutting_down_.load(std::memory_order_acquire)) - return ; - - BackgroundCompaction(); - - bg_compaction_scheduled_ = false; - bg_work_finish_signal_.notify_all(); + //add new compaction task + if(compact_thread_results_.empty()) { + compact_thread_results_.push_back( + compact_thread_pool_.enqueue(&DBImpl::BackgroundCompaction, this, compact_table_ids_)); + compact_table_ids_.clear(); + } } Status DBImpl::MergeFiles(const std::string& table_id, const meta::DateT& date, @@ -512,7 +501,6 @@ Status DBImpl::BackgroundMergeFiles(const std::string& table_id) { } bool has_merge = false; - for (auto& kv : raw_files) { auto files = kv.second; if (files.size() <= options_.merge_trigger_number) { @@ -520,15 +508,43 @@ Status DBImpl::BackgroundMergeFiles(const std::string& table_id) { } has_merge = true; MergeFiles(table_id, kv.first, kv.second); + + if (shutting_down_.load(std::memory_order_acquire)){ + break; + } + } + + return Status::OK(); +} + +void DBImpl::BackgroundCompaction(std::set table_ids) { + Status status; + for (auto table_id : table_ids) { + status = BackgroundMergeFiles(table_id); + if (!status.ok()) { + bg_error_ = status; + return; + } } pMeta_->Archive(); - - TryBuildIndex(); - pMeta_->CleanUpFilesWithTTL(1); +} - return Status::OK(); +void DBImpl::StartBuildIndexTask() { + //build index has been finished? + if(!index_thread_results_.empty()) { + std::chrono::milliseconds span(10); + if (index_thread_results_.back().wait_for(span) == std::future_status::ready) { + index_thread_results_.pop_back(); + } + } + + //add new build index task + if(index_thread_results_.empty()) { + index_thread_results_.push_back( + index_thread_pool_.enqueue(&DBImpl::BackgroundBuildIndex, this)); + } } Status DBImpl::BuildIndex(const meta::TableFileSchema& file) { @@ -569,8 +585,6 @@ Status DBImpl::BuildIndex(const meta::TableFileSchema& file) { } void DBImpl::BackgroundBuildIndex() { - std::lock_guard lock(build_index_mutex_); - assert(bg_build_index_started_); meta::TableFilesSchema to_index_files; pMeta_->FilesToIndex(to_index_files); Status status; @@ -581,34 +595,12 @@ void DBImpl::BackgroundBuildIndex() { bg_error_ = status; return; } - } - /* LOG(DEBUG) << "All Buiding index Done"; */ - bg_build_index_started_ = false; - bg_build_index_finish_signal_.notify_all(); -} - -Status DBImpl::TryBuildIndex() { - if (bg_build_index_started_) return Status::OK(); - if (shutting_down_.load(std::memory_order_acquire)) return Status::OK(); - bg_build_index_started_ = true; - std::thread build_index_task(&DBImpl::BackgroundBuildIndex, this); - build_index_task.detach(); - return Status::OK(); -} - -void DBImpl::BackgroundCompaction() { - std::vector table_ids; - pMemMgr_->Serialize(table_ids); - - Status status; - for (auto table_id : table_ids) { - status = BackgroundMergeFiles(table_id); - if (!status.ok()) { - bg_error_ = status; - return; + if (shutting_down_.load(std::memory_order_acquire)){ + break; } } + /* LOG(DEBUG) << "All Buiding index Done"; */ } Status DBImpl::DropAll() { @@ -620,23 +612,10 @@ Status DBImpl::Size(uint64_t& result) { } DBImpl::~DBImpl() { - { - std::unique_lock lock(mutex_); - shutting_down_.store(true, std::memory_order_release); - while (bg_compaction_scheduled_) { - bg_work_finish_signal_.wait(lock); - } - } - { - std::unique_lock lock(build_index_mutex_); - while (bg_build_index_started_) { - bg_build_index_finish_signal_.wait(lock); - } - } + shutting_down_.store(true, std::memory_order_release); bg_timer_thread_.join(); std::vector ids; pMemMgr_->Serialize(ids); - env_->Stop(); } } // namespace engine diff --git a/cpp/src/db/DBImpl.h b/cpp/src/db/DBImpl.h index 1a2f80601d..6457ed3e49 100644 --- a/cpp/src/db/DBImpl.h +++ b/cpp/src/db/DBImpl.h @@ -8,12 +8,15 @@ #include "DB.h" #include "MemManager.h" #include "Types.h" +#include "utils/ThreadPool.h" #include #include #include #include #include +#include +#include namespace zilliz { namespace milvus { @@ -62,40 +65,37 @@ private: const float* vectors, const meta::DatesT& dates, QueryResults& results); - void BackgroundBuildIndex(); - Status BuildIndex(const meta::TableFileSchema&); - Status TryBuildIndex(); - Status MergeFiles(const std::string& table_id, - const meta::DateT& date, - const meta::TableFilesSchema& files); - Status BackgroundMergeFiles(const std::string& table_id); - - void TrySchedule(); void StartTimerTasks(int interval); void BackgroundTimerTask(int interval); - static void BGWork(void* db); - void BackgroundCall(); - void BackgroundCompaction(); + void StartCompactionTask(); + Status MergeFiles(const std::string& table_id, + const meta::DateT& date, + const meta::TableFilesSchema& files); + Status BackgroundMergeFiles(const std::string& table_id); + void BackgroundCompaction(std::set table_ids); + + void StartBuildIndexTask(); + void BackgroundBuildIndex(); + Status BuildIndex(const meta::TableFileSchema&); - Env* const env_; const Options options_; - std::mutex mutex_; - std::condition_variable bg_work_finish_signal_; - bool bg_compaction_scheduled_; Status bg_error_; std::atomic shutting_down_; - std::mutex build_index_mutex_; - bool bg_build_index_started_; - std::condition_variable bg_build_index_finish_signal_; - std::thread bg_timer_thread_; MetaPtr pMeta_; MemManagerPtr pMemMgr_; + server::ThreadPool compact_thread_pool_; + std::list> compact_thread_results_; + std::set compact_table_ids_; + + server::ThreadPool index_thread_pool_; + std::list> index_thread_results_; + }; // DBImpl diff --git a/cpp/src/db/DBMetaImpl.cpp b/cpp/src/db/DBMetaImpl.cpp index 7305060c0a..ecad07835e 100644 --- a/cpp/src/db/DBMetaImpl.cpp +++ b/cpp/src/db/DBMetaImpl.cpp @@ -29,24 +29,43 @@ using namespace sqlite_orm; namespace { -void HandleException(std::exception &e) { - ENGINE_LOG_DEBUG << "Engine meta exception: " << e.what(); - throw e; +Status HandleException(const std::string& desc, std::exception &e) { + ENGINE_LOG_ERROR << desc << ": " << e.what(); + return Status::DBTransactionError(desc, e.what()); } +class MetricCollector { +public: + MetricCollector() { + server::Metrics::GetInstance().MetaAccessTotalIncrement(); + start_time_ = METRICS_NOW_TIME; + } + + ~MetricCollector() { + auto end_time = METRICS_NOW_TIME; + auto total_time = METRICS_MICROSECONDS(start_time_, end_time); + server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); + } + +private: + using TIME_POINT = std::chrono::system_clock::time_point; + TIME_POINT start_time_; +}; + } inline auto StoragePrototype(const std::string &path) { return make_storage(path, - make_table("Table", + make_table("Tables", make_column("id", &TableSchema::id_, primary_key()), make_column("table_id", &TableSchema::table_id_, unique()), + make_column("state", &TableSchema::state_), make_column("dimension", &TableSchema::dimension_), make_column("created_on", &TableSchema::created_on_), make_column("files_cnt", &TableSchema::files_cnt_, default_value(0)), make_column("engine_type", &TableSchema::engine_type_), make_column("store_raw_data", &TableSchema::store_raw_data_)), - make_table("TableFile", + make_table("TableFiles", make_column("id", &TableFileSchema::id_, primary_key()), make_column("table_id", &TableFileSchema::table_id_), make_column("engine_type", &TableFileSchema::engine_type_), @@ -109,9 +128,9 @@ Status DBMetaImpl::Initialize() { if (!boost::filesystem::is_directory(options_.path)) { auto ret = boost::filesystem::create_directory(options_.path); if (!ret) { - ENGINE_LOG_ERROR << "Create directory " << options_.path << " Error"; + ENGINE_LOG_ERROR << "Failed to create db directory " << options_.path; + return Status::DBTransactionError("Failed to create db directory", options_.path); } - assert(ret); } ConnectorPtr = std::make_unique(StoragePrototype(options_.path + "/meta.sqlite")); @@ -139,15 +158,15 @@ Status DBMetaImpl::DropPartitionsByDates(const std::string &table_id, return status; } - auto yesterday = GetDateWithDelta(-1); - - for (auto &date : dates) { - if (date >= yesterday) { - return Status::Error("Could not delete partitions with 2 days"); - } - } - try { + auto yesterday = GetDateWithDelta(-1); + + for (auto &date : dates) { + if (date >= yesterday) { + return Status::Error("Could not delete partitions with 2 days"); + } + } + ConnectorPtr->update_all( set( c(&TableFileSchema::file_type_) = (int) TableFileSchema::TO_DELETE @@ -157,40 +176,43 @@ Status DBMetaImpl::DropPartitionsByDates(const std::string &table_id, in(&TableFileSchema::date_, dates) )); } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when drop partition", e); } + return Status::OK(); } Status DBMetaImpl::CreateTable(TableSchema &table_schema) { - server::Metrics::GetInstance().MetaAccessTotalIncrement(); - if (table_schema.table_id_ == "") { - NextTableId(table_schema.table_id_); - } - table_schema.files_cnt_ = 0; - table_schema.id_ = -1; - table_schema.created_on_ = utils::GetMicroSecTimeStamp(); - auto start_time = METRICS_NOW_TIME; - { + try { + MetricCollector metric; + + server::Metrics::GetInstance().MetaAccessTotalIncrement(); + if (table_schema.table_id_ == "") { + NextTableId(table_schema.table_id_); + } + table_schema.files_cnt_ = 0; + table_schema.id_ = -1; + table_schema.created_on_ = utils::GetMicroSecTimeStamp(); + try { auto id = ConnectorPtr->insert(table_schema); table_schema.id_ = id; } catch (...) { return Status::DBTransactionError("Add Table Error"); } - } - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); - auto table_path = GetTablePath(table_schema.table_id_); - table_schema.location_ = table_path; - if (!boost::filesystem::is_directory(table_path)) { - auto ret = boost::filesystem::create_directories(table_path); - if (!ret) { - ENGINE_LOG_ERROR << "Create directory " << table_path << " Error"; + auto table_path = GetTablePath(table_schema.table_id_); + table_schema.location_ = table_path; + if (!boost::filesystem::is_directory(table_path)) { + auto ret = boost::filesystem::create_directories(table_path); + if (!ret) { + ENGINE_LOG_ERROR << "Create directory " << table_path << " Error"; + } + assert(ret); } - assert(ret); + + } catch (std::exception &e) { + return HandleException("Encounter exception when create table", e); } return Status::OK(); @@ -198,14 +220,31 @@ Status DBMetaImpl::CreateTable(TableSchema &table_schema) { Status DBMetaImpl::DeleteTable(const std::string& table_id) { try { - //drop the table from meta - auto tables = ConnectorPtr->select(columns(&TableSchema::id_), + MetricCollector metric; + + //soft delete table + auto tables = ConnectorPtr->select(columns(&TableSchema::id_, + &TableSchema::files_cnt_, + &TableSchema::dimension_, + &TableSchema::engine_type_, + &TableSchema::store_raw_data_, + &TableSchema::created_on_), where(c(&TableSchema::table_id_) == table_id)); for (auto &table : tables) { - ConnectorPtr->remove(std::get<0>(table)); + TableSchema table_schema; + table_schema.table_id_ = table_id; + table_schema.state_ = (int)TableSchema::TO_DELETE; + table_schema.id_ = std::get<0>(table); + table_schema.files_cnt_ = std::get<1>(table); + table_schema.dimension_ = std::get<2>(table); + table_schema.engine_type_ = std::get<3>(table); + table_schema.store_raw_data_ = std::get<4>(table); + table_schema.created_on_ = std::get<5>(table); + + ConnectorPtr->update(table_schema); } } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when delete table", e); } return Status::OK(); @@ -213,19 +252,17 @@ Status DBMetaImpl::DeleteTable(const std::string& table_id) { Status DBMetaImpl::DescribeTable(TableSchema &table_schema) { try { - server::Metrics::GetInstance().MetaAccessTotalIncrement(); - auto start_time = METRICS_NOW_TIME; + MetricCollector metric; + auto groups = ConnectorPtr->select(columns(&TableSchema::id_, &TableSchema::table_id_, &TableSchema::files_cnt_, &TableSchema::dimension_, &TableSchema::engine_type_, &TableSchema::store_raw_data_), - where(c(&TableSchema::table_id_) == table_schema.table_id_)); - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); - assert(groups.size() <= 1); + where(c(&TableSchema::table_id_) == table_schema.table_id_ + and c(&TableSchema::state_) != (int)TableSchema::TO_DELETE)); + if (groups.size() == 1) { table_schema.id_ = std::get<0>(groups[0]); table_schema.files_cnt_ = std::get<2>(groups[0]); @@ -240,47 +277,44 @@ Status DBMetaImpl::DescribeTable(TableSchema &table_schema) { table_schema.location_ = table_path; } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when describe table", e); } return Status::OK(); } Status DBMetaImpl::HasTable(const std::string &table_id, bool &has_or_not) { - try { - server::Metrics::GetInstance().MetaAccessTotalIncrement(); - auto start_time = METRICS_NOW_TIME; + has_or_not = false; + try { + MetricCollector metric; auto tables = ConnectorPtr->select(columns(&TableSchema::id_), - where(c(&TableSchema::table_id_) == table_id)); - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); - assert(tables.size() <= 1); + where(c(&TableSchema::table_id_) == table_id + and c(&TableSchema::state_) != (int)TableSchema::TO_DELETE)); if (tables.size() == 1) { has_or_not = true; } else { has_or_not = false; } + } catch (std::exception &e) { - HandleException(e); + HandleException("Encounter exception when lookup table", e); } + return Status::OK(); } Status DBMetaImpl::AllTables(std::vector& table_schema_array) { try { - server::Metrics::GetInstance().MetaAccessTotalIncrement(); - auto start_time = METRICS_NOW_TIME; + MetricCollector metric; + auto selected = ConnectorPtr->select(columns(&TableSchema::id_, &TableSchema::table_id_, &TableSchema::files_cnt_, &TableSchema::dimension_, &TableSchema::engine_type_, - &TableSchema::store_raw_data_)); - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); + &TableSchema::store_raw_data_), + where(c(&TableSchema::state_) != (int)TableSchema::TO_DELETE)); for (auto &table : selected) { TableSchema schema; schema.id_ = std::get<0>(table); @@ -292,8 +326,9 @@ Status DBMetaImpl::AllTables(std::vector& table_schema_array) { table_schema_array.emplace_back(schema); } + } catch (std::exception &e) { - HandleException(e); + HandleException("Encounter exception when lookup all tables", e); } return Status::OK(); @@ -310,37 +345,33 @@ Status DBMetaImpl::CreateTableFile(TableFileSchema &file_schema) { return status; } - NextFileId(file_schema.file_id_); - file_schema.file_type_ = TableFileSchema::NEW; - file_schema.dimension_ = table_schema.dimension_; - file_schema.size_ = 0; - file_schema.created_on_ = utils::GetMicroSecTimeStamp(); - file_schema.updated_time_ = file_schema.created_on_; - file_schema.engine_type_ = table_schema.engine_type_; - GetTableFilePath(file_schema); - ENGINE_LOG_DEBUG << "CreateTableFile " << file_schema.file_id_; - { - try { - server::Metrics::GetInstance().MetaAccessTotalIncrement(); - auto start_time = METRICS_NOW_TIME; - auto id = ConnectorPtr->insert(file_schema); - file_schema.id_ = id; - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); - } catch (...) { - return Status::DBTransactionError("Add file Error"); - } - } + try { + MetricCollector metric; - auto partition_path = GetTableDatePartitionPath(file_schema.table_id_, file_schema.date_); + NextFileId(file_schema.file_id_); + file_schema.file_type_ = TableFileSchema::NEW; + file_schema.dimension_ = table_schema.dimension_; + file_schema.size_ = 0; + file_schema.created_on_ = utils::GetMicroSecTimeStamp(); + file_schema.updated_time_ = file_schema.created_on_; + file_schema.engine_type_ = table_schema.engine_type_; + GetTableFilePath(file_schema); - if (!boost::filesystem::is_directory(partition_path)) { - auto ret = boost::filesystem::create_directory(partition_path); - if (!ret) { - ENGINE_LOG_ERROR << "Create directory " << partition_path << " Error"; + auto id = ConnectorPtr->insert(file_schema); + file_schema.id_ = id; + + auto partition_path = GetTableDatePartitionPath(file_schema.table_id_, file_schema.date_); + + if (!boost::filesystem::is_directory(partition_path)) { + auto ret = boost::filesystem::create_directory(partition_path); + if (!ret) { + ENGINE_LOG_ERROR << "Create directory " << partition_path << " Error"; + return Status::DBTransactionError("Failed to create partition directory"); + } } - assert(ret); + + } catch (std::exception& ex) { + return HandleException("Encounter exception when create table file", ex); } return Status::OK(); @@ -350,8 +381,8 @@ Status DBMetaImpl::FilesToIndex(TableFilesSchema &files) { files.clear(); try { - server::Metrics::GetInstance().MetaAccessTotalIncrement(); - auto start_time = METRICS_NOW_TIME; + MetricCollector metric; + auto selected = ConnectorPtr->select(columns(&TableFileSchema::id_, &TableFileSchema::table_id_, &TableFileSchema::file_id_, @@ -361,9 +392,6 @@ Status DBMetaImpl::FilesToIndex(TableFilesSchema &files) { &TableFileSchema::engine_type_), where(c(&TableFileSchema::file_type_) == (int) TableFileSchema::TO_INDEX)); - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); std::map groups; TableFileSchema table_file; @@ -391,8 +419,9 @@ Status DBMetaImpl::FilesToIndex(TableFilesSchema &files) { table_file.dimension_ = groups[table_file.table_id_].dimension_; files.push_back(table_file); } + } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when iterate raw files", e); } return Status::OK(); @@ -404,8 +433,8 @@ Status DBMetaImpl::FilesToSearch(const std::string &table_id, files.clear(); try { - server::Metrics::GetInstance().MetaAccessTotalIncrement(); - auto start_time = METRICS_NOW_TIME; + MetricCollector metric; + if (partition.empty()) { auto selected = ConnectorPtr->select(columns(&TableFileSchema::id_, &TableFileSchema::table_id_, @@ -420,9 +449,7 @@ Status DBMetaImpl::FilesToSearch(const std::string &table_id, == (int) TableFileSchema::TO_INDEX or c(&TableFileSchema::file_type_) == (int) TableFileSchema::INDEX))); - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); + TableSchema table_schema; table_schema.table_id_ = table_id; auto status = DescribeTable(table_schema); @@ -464,9 +491,7 @@ Status DBMetaImpl::FilesToSearch(const std::string &table_id, == (int) TableFileSchema::TO_INDEX or c(&TableFileSchema::file_type_) == (int) TableFileSchema::INDEX))); - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); + TableSchema table_schema; table_schema.table_id_ = table_id; auto status = DescribeTable(table_schema); @@ -495,7 +520,7 @@ Status DBMetaImpl::FilesToSearch(const std::string &table_id, } } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when iterate index files", e); } return Status::OK(); @@ -506,8 +531,8 @@ Status DBMetaImpl::FilesToMerge(const std::string &table_id, files.clear(); try { - server::Metrics::GetInstance().MetaAccessTotalIncrement(); - auto start_time = METRICS_NOW_TIME; + MetricCollector metric; + auto selected = ConnectorPtr->select(columns(&TableFileSchema::id_, &TableFileSchema::table_id_, &TableFileSchema::file_id_, @@ -517,9 +542,7 @@ Status DBMetaImpl::FilesToMerge(const std::string &table_id, where(c(&TableFileSchema::file_type_) == (int) TableFileSchema::RAW and c(&TableFileSchema::table_id_) == table_id), order_by(&TableFileSchema::size_).desc()); - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); + TableSchema table_schema; table_schema.table_id_ = table_id; auto status = DescribeTable(table_schema); @@ -545,7 +568,7 @@ Status DBMetaImpl::FilesToMerge(const std::string &table_id, files[table_file.date_].push_back(table_file); } } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when iterate merge files", e); } return Status::OK(); @@ -563,9 +586,7 @@ Status DBMetaImpl::FilesToDelete(const std::string& table_id, &TableFileSchema::file_id_, &TableFileSchema::size_, &TableFileSchema::date_), - where(c(&TableFileSchema::file_type_) != - (int) TableFileSchema::TO_DELETE - and c(&TableFileSchema::table_id_) == table_id)); + where(c(&TableFileSchema::table_id_) == table_id)); //step 2: erase table files from meta for (auto &file : selected) { @@ -592,9 +613,7 @@ Status DBMetaImpl::FilesToDelete(const std::string& table_id, &TableFileSchema::file_id_, &TableFileSchema::size_, &TableFileSchema::date_), - where(c(&TableFileSchema::file_type_) != - (int) TableFileSchema::TO_DELETE - and in(&TableFileSchema::date_, partition) + where(in(&TableFileSchema::date_, partition) and c(&TableFileSchema::table_id_) == table_id)); //step 2: erase table files from meta @@ -617,7 +636,7 @@ Status DBMetaImpl::FilesToDelete(const std::string& table_id, } } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when iterate delete files", e); } return Status::OK(); @@ -648,7 +667,7 @@ Status DBMetaImpl::GetTableFile(TableFileSchema &file_schema) { " File:" + file_schema.file_id_ + " not found"); } } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when lookup table file", e); } return Status::OK(); @@ -677,7 +696,7 @@ Status DBMetaImpl::Archive() { c(&TableFileSchema::file_type_) != (int) TableFileSchema::TO_DELETE )); } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when update table files", e); } } if (criteria == "disk") { @@ -707,7 +726,7 @@ Status DBMetaImpl::Size(uint64_t &result) { result += (uint64_t) (*std::get<0>(sub_query)); } } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when calculte db size", e); } return Status::OK(); @@ -752,7 +771,7 @@ Status DBMetaImpl::DiscardFiles(long to_discard_size) { )); } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when discard table file", e); } return DiscardFiles(to_discard_size); @@ -761,38 +780,33 @@ Status DBMetaImpl::DiscardFiles(long to_discard_size) { Status DBMetaImpl::UpdateTableFile(TableFileSchema &file_schema) { file_schema.updated_time_ = utils::GetMicroSecTimeStamp(); try { - server::Metrics::GetInstance().MetaAccessTotalIncrement(); - auto start_time = METRICS_NOW_TIME; + MetricCollector metric; + ConnectorPtr->update(file_schema); - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); + } catch (std::exception &e) { ENGINE_LOG_DEBUG << "table_id= " << file_schema.table_id_ << " file_id=" << file_schema.file_id_; - HandleException(e); + return HandleException("Encounter exception when update table file", e); } return Status::OK(); } Status DBMetaImpl::UpdateTableFiles(TableFilesSchema &files) { try { - server::Metrics::GetInstance().MetaAccessTotalIncrement(); - auto start_time = METRICS_NOW_TIME; + MetricCollector metric; + auto commited = ConnectorPtr->transaction([&]() mutable { for (auto &file : files) { file.updated_time_ = utils::GetMicroSecTimeStamp(); ConnectorPtr->update(file); } - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); return true; }); if (!commited) { return Status::DBTransactionError("Update files Error"); } } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when update table files", e); } return Status::OK(); } @@ -830,7 +844,7 @@ Status DBMetaImpl::CleanUpFilesWithTTL(uint16_t seconds) { /* LOG(DEBUG) << "Removing deleted id=" << table_file.id << " location=" << table_file.location << std::endl; */ } } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when clean table files", e); } return Status::OK(); @@ -868,7 +882,7 @@ Status DBMetaImpl::CleanUp() { /* LOG(DEBUG) << "Removing id=" << table_file.id << " location=" << table_file.location << std::endl; */ } } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when clean table file", e); } return Status::OK(); @@ -877,9 +891,8 @@ Status DBMetaImpl::CleanUp() { Status DBMetaImpl::Count(const std::string &table_id, uint64_t &result) { try { + MetricCollector metric; - server::Metrics::GetInstance().MetaAccessTotalIncrement(); - auto start_time = METRICS_NOW_TIME; auto selected = ConnectorPtr->select(columns(&TableFileSchema::size_, &TableFileSchema::date_), where((c(&TableFileSchema::file_type_) == (int) TableFileSchema::RAW or @@ -888,9 +901,7 @@ Status DBMetaImpl::Count(const std::string &table_id, uint64_t &result) { c(&TableFileSchema::file_type_) == (int) TableFileSchema::INDEX) and c(&TableFileSchema::table_id_) == table_id)); - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().MetaAccessDurationSecondsHistogramObserve(total_time); + TableSchema table_schema; table_schema.table_id_ = table_id; auto status = DescribeTable(table_schema); @@ -908,7 +919,7 @@ Status DBMetaImpl::Count(const std::string &table_id, uint64_t &result) { result /= sizeof(float); } catch (std::exception &e) { - HandleException(e); + return HandleException("Encounter exception when calculate table file size", e); } return Status::OK(); } diff --git a/cpp/src/db/Env.cpp b/cpp/src/db/Env.cpp deleted file mode 100644 index 2cca7da13b..0000000000 --- a/cpp/src/db/Env.cpp +++ /dev/null @@ -1,87 +0,0 @@ -/******************************************************************************* - * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved - * Unauthorized copying of this file, via any medium is strictly prohibited. - * Proprietary and confidential. - ******************************************************************************/ -#include -#include -#include -#include "Env.h" - -namespace zilliz { -namespace milvus { -namespace engine { - -Env::Env() - : bg_work_started_(false), - shutting_down_(false) { -} - -void Env::Schedule(void (*function)(void* arg), void* arg) { - std::unique_lock lock(bg_work_mutex_); - if (shutting_down_) return; - - if (!bg_work_started_) { - bg_work_started_ = true; - std::thread bg_thread(Env::BackgroundThreadEntryPoint, this); - bg_thread.detach(); - } - - if (bg_work_queue_.empty()) { - bg_work_cv_.notify_one(); - } - - bg_work_queue_.emplace(function, arg); -} - -void Env::BackgroundThreadMain() { - while (!shutting_down_) { - std::unique_lock lock(bg_work_mutex_); - while (bg_work_queue_.empty() && !shutting_down_) { - bg_work_cv_.wait(lock); - } - - if (shutting_down_) break; - - assert(!bg_work_queue_.empty()); - auto bg_function = bg_work_queue_.front().function_; - void* bg_arg = bg_work_queue_.front().arg_; - bg_work_queue_.pop(); - - lock.unlock(); - bg_function(bg_arg); - } - - std::unique_lock lock(bg_work_mutex_); - bg_work_started_ = false; - bg_work_cv_.notify_all(); -} - -void Env::Stop() { - { - std::unique_lock lock(bg_work_mutex_); - if (shutting_down_ || !bg_work_started_) return; - } - shutting_down_ = true; - { - std::unique_lock lock(bg_work_mutex_); - if (bg_work_queue_.empty()) { - bg_work_cv_.notify_one(); - } - while (bg_work_started_) { - bg_work_cv_.wait(lock); - } - } - shutting_down_ = false; -} - -Env::~Env() {} - -Env* Env::Default() { - static Env env; - return &env; -} - -} // namespace engine -} // namespace milvus -} // namespace zilliz diff --git a/cpp/src/db/Env.h b/cpp/src/db/Env.h deleted file mode 100644 index 7818b25ef0..0000000000 --- a/cpp/src/db/Env.h +++ /dev/null @@ -1,56 +0,0 @@ -/******************************************************************************* - * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved - * Unauthorized copying of this file, via any medium is strictly prohibited. - * Proprietary and confidential. - ******************************************************************************/ -#pragma once - -#include -#include -#include -#include -#include - -namespace zilliz { -namespace milvus { -namespace engine { - -class Env { -public: - Env(); - - Env(const Env&) = delete; - Env& operator=(const Env&) = delete; - - void Schedule(void (*function)(void* arg), void* arg); - - virtual void Stop(); - - virtual ~Env(); - - static Env* Default(); - -protected: - void BackgroundThreadMain(); - static void BackgroundThreadEntryPoint(Env* env) { - env->BackgroundThreadMain(); - } - - struct BGWork { - explicit BGWork(void (*function)(void*), void* arg) - : function_(function), arg_(arg) {} - - void (* const function_)(void*); - void* const arg_; - }; - - std::mutex bg_work_mutex_; - std::condition_variable bg_work_cv_; - std::queue bg_work_queue_; - bool bg_work_started_; - std::atomic shutting_down_; -}; // Env - -} // namespace engine -} // namespace milvus -} // namespace zilliz diff --git a/cpp/src/db/MemManager.cpp b/cpp/src/db/MemManager.cpp index c8bd5de23b..31b2070b49 100644 --- a/cpp/src/db/MemManager.cpp +++ b/cpp/src/db/MemManager.cpp @@ -142,6 +142,13 @@ Status MemManager::Serialize(std::vector& table_ids) { return Status::OK(); } +Status MemManager::EraseMemVector(const std::string& table_id) { + std::unique_lock lock(mutex_); + memMap_.erase(table_id); + + return Status::OK(); +} + } // namespace engine } // namespace milvus diff --git a/cpp/src/db/MemManager.h b/cpp/src/db/MemManager.h index 395c8f48cd..1b329f971b 100644 --- a/cpp/src/db/MemManager.h +++ b/cpp/src/db/MemManager.h @@ -75,6 +75,8 @@ public: Status Serialize(std::vector& table_ids); + Status EraseMemVector(const std::string& table_id); + private: Status InsertVectorsNoLock(const std::string& table_id, size_t n, const float* vectors, IDNumbers& vector_ids); diff --git a/cpp/src/db/MetaTypes.h b/cpp/src/db/MetaTypes.h index a284d14a19..b20d3a05d0 100644 --- a/cpp/src/db/MetaTypes.h +++ b/cpp/src/db/MetaTypes.h @@ -21,12 +21,18 @@ const DateT EmptyDate = -1; typedef std::vector DatesT; struct TableSchema { - size_t id_; + typedef enum { + NORMAL, + TO_DELETE, + } TABLE_STATE; + + size_t id_ = 0; std::string table_id_; + int state_ = (int)NORMAL; size_t files_cnt_ = 0; - uint16_t dimension_; + uint16_t dimension_ = 0; std::string location_; - long created_on_; + long created_on_ = 0; int engine_type_ = (int)EngineType::FAISS_IDMAP; bool store_raw_data_ = false; }; // TableSchema @@ -40,17 +46,17 @@ struct TableFileSchema { TO_DELETE, } FILE_TYPE; - size_t id_; + size_t id_ = 0; std::string table_id_; int engine_type_ = (int)EngineType::FAISS_IDMAP; std::string file_id_; int file_type_ = NEW; - size_t size_; + size_t size_ = 0; DateT date_ = EmptyDate; - uint16_t dimension_; + uint16_t dimension_ = 0; std::string location_; - long updated_time_; - long created_on_; + long updated_time_ = 0; + long created_on_ = 0; }; // TableFileSchema typedef std::vector TableFilesSchema; diff --git a/cpp/src/db/Options.cpp b/cpp/src/db/Options.cpp index 74197226fe..ff6404751f 100644 --- a/cpp/src/db/Options.cpp +++ b/cpp/src/db/Options.cpp @@ -9,7 +9,6 @@ #include #include "Options.h" -#include "Env.h" #include "DBMetaImpl.h" #include "Exception.h" @@ -17,8 +16,7 @@ namespace zilliz { namespace milvus { namespace engine { -Options::Options() - : env(Env::Default()) { +Options::Options() { } ArchiveConf::ArchiveConf(const std::string& type, const std::string& criterias) { diff --git a/cpp/src/db/Options.h b/cpp/src/db/Options.h index 2a7740575e..339044d9dd 100644 --- a/cpp/src/db/Options.h +++ b/cpp/src/db/Options.h @@ -47,7 +47,6 @@ struct Options { uint16_t memory_sync_interval = 1; //unit: second uint16_t merge_trigger_number = 2; size_t index_trigger_size = ONE_GB; //unit: byte - Env* env; DBMetaOptions meta; }; // Options diff --git a/cpp/src/sdk/examples/simple/src/ClientTest.cpp b/cpp/src/sdk/examples/simple/src/ClientTest.cpp index 825923b1a4..b2ed9f3b25 100644 --- a/cpp/src/sdk/examples/simple/src/ClientTest.cpp +++ b/cpp/src/sdk/examples/simple/src/ClientTest.cpp @@ -20,7 +20,7 @@ namespace { static constexpr int64_t TOTAL_ROW_COUNT = 100000; static constexpr int64_t TOP_K = 10; static constexpr int64_t SEARCH_TARGET = 5000; //change this value, result is different - static constexpr int64_t ADD_VECTOR_LOOP = 1; + static constexpr int64_t ADD_VECTOR_LOOP = 2; #define BLOCK_SPLITER std::cout << "===========================================" << std::endl; @@ -179,8 +179,8 @@ ClientTest::Test(const std::string& address, const std::string& port) { } {//search vectors - std::cout << "Waiting data persist. Sleep 10 seconds ..." << std::endl; - sleep(10); + std::cout << "Waiting data persist. Sleep 1 seconds ..." << std::endl; + sleep(1); std::vector record_array; BuildVectors(SEARCH_TARGET, SEARCH_TARGET + 10, record_array); diff --git a/cpp/src/server/RequestTask.cpp b/cpp/src/server/RequestTask.cpp index 6d2f5ae4c2..3b90d335ec 100644 --- a/cpp/src/server/RequestTask.cpp +++ b/cpp/src/server/RequestTask.cpp @@ -9,7 +9,6 @@ #include "utils/Log.h" #include "utils/TimeRecorder.h" #include "db/DB.h" -#include "db/Env.h" #include "db/Meta.h" #include "version.h" @@ -34,7 +33,6 @@ namespace { ConfigNode& config = ServerConfig::GetInstance().GetConfig(CONFIG_DB); opt.meta.backend_uri = config.GetValue(CONFIG_DB_URL); std::string db_path = config.GetValue(CONFIG_DB_PATH); - opt.memory_sync_interval = (uint16_t)config.GetInt32Value(CONFIG_DB_FLUSH_INTERVAL, 10); opt.meta.path = db_path + "/db"; int64_t index_size = config.GetInt64Value(CONFIG_DB_INDEX_TRIGGER_SIZE); if(index_size > 0) {//ensure larger than zero, unit is MB diff --git a/cpp/src/server/ServerConfig.h b/cpp/src/server/ServerConfig.h index cd6a89bd70..dd7c9d2966 100644 --- a/cpp/src/server/ServerConfig.h +++ b/cpp/src/server/ServerConfig.h @@ -23,7 +23,6 @@ static const std::string CONFIG_SERVER_MODE = "server_mode"; static const std::string CONFIG_DB = "db_config"; static const std::string CONFIG_DB_URL = "db_backend_url"; static const std::string CONFIG_DB_PATH = "db_path"; -static const std::string CONFIG_DB_FLUSH_INTERVAL = "db_flush_interval"; static const std::string CONFIG_DB_INDEX_TRIGGER_SIZE = "index_building_threshold"; static const std::string CONFIG_LOG = "log_config"; From 54361633baeb3832682f9fc035099a3e3efcf95d Mon Sep 17 00:00:00 2001 From: groot Date: Wed, 19 Jun 2019 12:10:27 +0800 Subject: [PATCH 2/5] refine code Former-commit-id: eae3ec0b3f87dbecf1e44d079d4a4a46794f9b9f --- cpp/src/db/DBImpl.cpp | 103 ++++++++++++------ cpp/src/db/DBImpl.h | 6 +- cpp/src/db/DBMetaImpl.cpp | 16 ++- cpp/src/db/scheduler/SearchScheduler.cpp | 2 +- cpp/src/db/scheduler/SearchScheduler.h | 4 +- .../{SearchTaskQueue.cpp => SearchTask.cpp} | 2 +- .../{SearchTaskQueue.h => SearchTask.h} | 1 - cpp/src/server/RequestTask.cpp | 20 ++-- 8 files changed, 99 insertions(+), 55 deletions(-) rename cpp/src/db/scheduler/{SearchTaskQueue.cpp => SearchTask.cpp} (99%) rename cpp/src/db/scheduler/{SearchTaskQueue.h => SearchTask.h} (92%) diff --git a/cpp/src/db/DBImpl.cpp b/cpp/src/db/DBImpl.cpp index 4e9e4040b7..9b541b8726 100644 --- a/cpp/src/db/DBImpl.cpp +++ b/cpp/src/db/DBImpl.cpp @@ -25,6 +25,10 @@ namespace engine { namespace { +static constexpr uint64_t METRIC_ACTION_INTERVAL = 1; +static constexpr uint64_t COMPACT_ACTION_INTERVAL = 1; +static constexpr uint64_t INDEX_ACTION_INTERVAL = 1; + void CollectInsertMetrics(double total_time, size_t n, bool succeed) { double avg_time = total_time / n; for (int i = 0; i < n; ++i) { @@ -130,7 +134,7 @@ DBImpl::DBImpl(const Options& options) pMemMgr_(new MemManager(pMeta_, options_)), compact_thread_pool_(1, 1), index_thread_pool_(1, 1) { - StartTimerTasks(options_.memory_sync_interval); + StartTimerTasks(); } Status DBImpl::CreateTable(meta::TableSchema& table_schema) { @@ -399,12 +403,11 @@ Status DBImpl::QueryAsync(const std::string& table_id, const meta::TableFilesSch return Status::OK(); } -void DBImpl::StartTimerTasks(int interval) { - bg_timer_thread_ = std::thread(&DBImpl::BackgroundTimerTask, this, interval); +void DBImpl::StartTimerTasks() { + bg_timer_thread_ = std::thread(&DBImpl::BackgroundTimerTask, this); } - -void DBImpl::BackgroundTimerTask(int interval) { +void DBImpl::BackgroundTimerTask() { Status status; server::SystemInfo::GetInstance().Init(); while (true) { @@ -419,27 +422,42 @@ void DBImpl::BackgroundTimerTask(int interval) { break; } - std::this_thread::sleep_for(std::chrono::seconds(interval)); - - server::Metrics::GetInstance().KeepingAliveCounterIncrement(interval); - int64_t cache_usage = cache::CpuCacheMgr::GetInstance()->CacheUsage(); - int64_t cache_total = cache::CpuCacheMgr::GetInstance()->CacheCapacity(); - server::Metrics::GetInstance().CacheUsageGaugeSet(cache_usage*100/cache_total); - uint64_t size; - Size(size); - server::Metrics::GetInstance().DataFileSizeGaugeSet(size); - server::Metrics::GetInstance().CPUUsagePercentSet(); - server::Metrics::GetInstance().RAMUsagePercentSet(); - server::Metrics::GetInstance().GPUPercentGaugeSet(); - server::Metrics::GetInstance().GPUMemoryUsageGaugeSet(); - server::Metrics::GetInstance().OctetsSet(); + std::this_thread::sleep_for(std::chrono::seconds(1)); + StartMetricTask(); StartCompactionTask(); StartBuildIndexTask(); } } +void DBImpl::StartMetricTask() { + static uint64_t metric_clock_tick = 0; + metric_clock_tick++; + if(metric_clock_tick%METRIC_ACTION_INTERVAL != 0) { + return; + } + + server::Metrics::GetInstance().KeepingAliveCounterIncrement(METRIC_ACTION_INTERVAL); + int64_t cache_usage = cache::CpuCacheMgr::GetInstance()->CacheUsage(); + int64_t cache_total = cache::CpuCacheMgr::GetInstance()->CacheCapacity(); + server::Metrics::GetInstance().CacheUsageGaugeSet(cache_usage*100/cache_total); + uint64_t size; + Size(size); + server::Metrics::GetInstance().DataFileSizeGaugeSet(size); + server::Metrics::GetInstance().CPUUsagePercentSet(); + server::Metrics::GetInstance().RAMUsagePercentSet(); + server::Metrics::GetInstance().GPUPercentGaugeSet(); + server::Metrics::GetInstance().GPUMemoryUsageGaugeSet(); + server::Metrics::GetInstance().OctetsSet(); +} + void DBImpl::StartCompactionTask() { + static uint64_t compact_clock_tick = 0; + compact_clock_tick++; + if(compact_clock_tick%COMPACT_ACTION_INTERVAL != 0) { + return; + } + //serialize memory data std::vector temp_table_ids; pMemMgr_->Serialize(temp_table_ids); @@ -556,6 +574,12 @@ void DBImpl::BackgroundCompaction(std::set table_ids) { } void DBImpl::StartBuildIndexTask() { + static uint64_t index_clock_tick = 0; + index_clock_tick++; + if(index_clock_tick%INDEX_ACTION_INTERVAL != 0) { + return; + } + //build index has been finished? if(!index_thread_results_.empty()) { std::chrono::milliseconds span(10); @@ -581,29 +605,36 @@ Status DBImpl::BuildIndex(const meta::TableFileSchema& file) { } ExecutionEnginePtr to_index = EngineFactory::Build(file.dimension_, file.location_, (EngineType)file.engine_type_); + if(to_index == nullptr) { + return Status::Error("Invalid engine type"); + } - to_index->Load(); - auto start_time = METRICS_NOW_TIME; - auto index = to_index->BuildIndex(table_file.location_); - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - server::Metrics::GetInstance().BuildIndexDurationSecondsHistogramObserve(total_time); + try { + to_index->Load(); + auto start_time = METRICS_NOW_TIME; + auto index = to_index->BuildIndex(table_file.location_); + auto end_time = METRICS_NOW_TIME; + auto total_time = METRICS_MICROSECONDS(start_time, end_time); + server::Metrics::GetInstance().BuildIndexDurationSecondsHistogramObserve(total_time); - table_file.file_type_ = meta::TableFileSchema::INDEX; - table_file.size_ = index->Size(); + table_file.file_type_ = meta::TableFileSchema::INDEX; + table_file.size_ = index->Size(); - auto to_remove = file; - to_remove.file_type_ = meta::TableFileSchema::TO_DELETE; + auto to_remove = file; + to_remove.file_type_ = meta::TableFileSchema::TO_DELETE; - meta::TableFilesSchema update_files = {to_remove, table_file}; - pMeta_->UpdateTableFiles(update_files); + meta::TableFilesSchema update_files = {to_remove, table_file}; + pMeta_->UpdateTableFiles(update_files); - LOG(DEBUG) << "New index file " << table_file.file_id_ << " of size " - << index->PhysicalSize()/(1024*1024) << " M" - << " from file " << to_remove.file_id_; + LOG(DEBUG) << "New index file " << table_file.file_id_ << " of size " + << index->PhysicalSize()/(1024*1024) << " M" + << " from file " << to_remove.file_id_; - index->Cache(); - pMeta_->Archive(); + index->Cache(); + + } catch (std::exception& ex) { + return Status::Error("Build index encounter exception", ex.what()); + } return Status::OK(); } diff --git a/cpp/src/db/DBImpl.h b/cpp/src/db/DBImpl.h index 64879a9c9d..177faeb9c1 100644 --- a/cpp/src/db/DBImpl.h +++ b/cpp/src/db/DBImpl.h @@ -70,8 +70,10 @@ private: const meta::DatesT& dates, QueryResults& results); - void StartTimerTasks(int interval); - void BackgroundTimerTask(int interval); + void StartTimerTasks(); + void BackgroundTimerTask(); + + void StartMetricTask(); void StartCompactionTask(); Status MergeFiles(const std::string& table_id, diff --git a/cpp/src/db/DBMetaImpl.cpp b/cpp/src/db/DBMetaImpl.cpp index ecad07835e..3ae34c1c2e 100644 --- a/cpp/src/db/DBMetaImpl.cpp +++ b/cpp/src/db/DBMetaImpl.cpp @@ -186,10 +186,18 @@ Status DBMetaImpl::CreateTable(TableSchema &table_schema) { try { MetricCollector metric; - server::Metrics::GetInstance().MetaAccessTotalIncrement(); if (table_schema.table_id_ == "") { NextTableId(table_schema.table_id_); + } else { + auto table = ConnectorPtr->select(columns(&TableSchema::state_), + where(c(&TableSchema::table_id_) == table_schema.table_id_)); + if (table.size() == 1) { + std::string msg = (TableSchema::TO_DELETE == std::get<0>(table[0])) ? + "Table already exists" : "Table already exists and it is in delete state, please wait a second"; + return Status::Error(msg); + } } + table_schema.files_cnt_ = 0; table_schema.id_ = -1; table_schema.created_on_ = utils::GetMicroSecTimeStamp(); @@ -207,8 +215,8 @@ Status DBMetaImpl::CreateTable(TableSchema &table_schema) { auto ret = boost::filesystem::create_directories(table_path); if (!ret) { ENGINE_LOG_ERROR << "Create directory " << table_path << " Error"; + return Status::Error("Failed to create table path"); } - assert(ret); } } catch (std::exception &e) { @@ -733,10 +741,12 @@ Status DBMetaImpl::Size(uint64_t &result) { } Status DBMetaImpl::DiscardFiles(long to_discard_size) { - LOG(DEBUG) << "About to discard size=" << to_discard_size; if (to_discard_size <= 0) { return Status::OK(); } + + LOG(DEBUG) << "About to discard size=" << to_discard_size; + try { auto selected = ConnectorPtr->select(columns(&TableFileSchema::id_, &TableFileSchema::size_), diff --git a/cpp/src/db/scheduler/SearchScheduler.cpp b/cpp/src/db/scheduler/SearchScheduler.cpp index 756abea4cd..fa0dee88fb 100644 --- a/cpp/src/db/scheduler/SearchScheduler.cpp +++ b/cpp/src/db/scheduler/SearchScheduler.cpp @@ -6,7 +6,7 @@ #include "SearchScheduler.h" #include "IndexLoaderQueue.h" -#include "SearchTaskQueue.h" +#include "SearchTask.h" #include "utils/Log.h" #include "utils/TimeRecorder.h" #include "metrics/Metrics.h" diff --git a/cpp/src/db/scheduler/SearchScheduler.h b/cpp/src/db/scheduler/SearchScheduler.h index 6e84c43684..673d759056 100644 --- a/cpp/src/db/scheduler/SearchScheduler.h +++ b/cpp/src/db/scheduler/SearchScheduler.h @@ -7,7 +7,7 @@ #include "SearchContext.h" #include "IndexLoaderQueue.h" -#include "SearchTaskQueue.h" +#include "SearchTask.h" namespace zilliz { namespace milvus { @@ -35,6 +35,8 @@ private: std::shared_ptr search_thread_; IndexLoaderQueue index_load_queue_; + + using SearchTaskQueue = server::BlockingQueue; SearchTaskQueue search_queue_; bool stopped_ = true; diff --git a/cpp/src/db/scheduler/SearchTaskQueue.cpp b/cpp/src/db/scheduler/SearchTask.cpp similarity index 99% rename from cpp/src/db/scheduler/SearchTaskQueue.cpp rename to cpp/src/db/scheduler/SearchTask.cpp index 819a881f39..45503490d1 100644 --- a/cpp/src/db/scheduler/SearchTaskQueue.cpp +++ b/cpp/src/db/scheduler/SearchTask.cpp @@ -3,7 +3,7 @@ * Unauthorized copying of this file, via any medium is strictly prohibited. * Proprietary and confidential. ******************************************************************************/ -#include "SearchTaskQueue.h" +#include "SearchTask.h" #include "utils/Log.h" #include "utils/TimeRecorder.h" diff --git a/cpp/src/db/scheduler/SearchTaskQueue.h b/cpp/src/db/scheduler/SearchTask.h similarity index 92% rename from cpp/src/db/scheduler/SearchTaskQueue.h rename to cpp/src/db/scheduler/SearchTask.h index e5841cd1d5..3513c15a3e 100644 --- a/cpp/src/db/scheduler/SearchTaskQueue.h +++ b/cpp/src/db/scheduler/SearchTask.h @@ -27,7 +27,6 @@ public: }; using SearchTaskPtr = std::shared_ptr; -using SearchTaskQueue = server::BlockingQueue; } diff --git a/cpp/src/server/RequestTask.cpp b/cpp/src/server/RequestTask.cpp index 47e79c0b85..3133e1cd80 100644 --- a/cpp/src/server/RequestTask.cpp +++ b/cpp/src/server/RequestTask.cpp @@ -221,7 +221,7 @@ ServerError CreateTableTask::OnExecute() { //////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// DescribeTableTask::DescribeTableTask(const std::string &table_name, thrift::TableSchema &schema) - : BaseTask(PING_TASK_GROUP), + : BaseTask(DDL_DML_TASK_GROUP), table_name_(table_name), schema_(schema) { schema_.table_name = table_name_; @@ -329,7 +329,7 @@ ServerError DeleteTableTask::OnExecute() { //////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// ShowTablesTask::ShowTablesTask(std::vector& tables) - : BaseTask(DQL_TASK_GROUP), + : BaseTask(DDL_DML_TASK_GROUP), tables_(tables) { } @@ -451,13 +451,13 @@ SearchVectorTask::SearchVectorTask(const std::string &table_name, const std::vector &query_range_array, const int64_t top_k, std::vector &result_array) - : BaseTask(DQL_TASK_GROUP), - table_name_(table_name), - file_id_array_(file_id_array), - record_array_(query_record_array), - range_array_(query_range_array), - top_k_(top_k), - result_array_(result_array) { + : BaseTask(DQL_TASK_GROUP), + table_name_(table_name), + file_id_array_(file_id_array), + record_array_(query_record_array), + range_array_(query_range_array), + top_k_(top_k), + result_array_(result_array) { } @@ -575,7 +575,7 @@ ServerError SearchVectorTask::OnExecute() { //////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// GetTableRowCountTask::GetTableRowCountTask(const std::string& table_name, int64_t& row_count) -: BaseTask(DQL_TASK_GROUP), +: BaseTask(DDL_DML_TASK_GROUP), table_name_(table_name), row_count_(row_count) { From e050719b2b2f634062109913c8d1442e5eda43fc Mon Sep 17 00:00:00 2001 From: groot Date: Thu, 20 Jun 2019 16:05:44 +0800 Subject: [PATCH 3/5] delete table issue Former-commit-id: d1a8e64086c11081917058623d344ac3c567a74b --- cpp/src/CMakeLists.txt | 10 +- cpp/src/db/DBImpl.cpp | 114 +++--- cpp/src/db/DBImpl.h | 4 +- cpp/src/db/DBMetaImpl.cpp | 326 +++++++++--------- cpp/src/db/DBMetaImpl.h | 8 +- cpp/src/db/FaissExecutionEngine.cpp | 9 +- cpp/src/db/Meta.h | 8 +- cpp/src/db/scheduler/ScheduleStrategy.cpp | 64 ---- cpp/src/db/scheduler/SearchScheduler.cpp | 180 ---------- cpp/src/db/scheduler/SearchScheduler.h | 48 --- ...xLoaderQueue.cpp => TaskDispatchQueue.cpp} | 37 +- ...IndexLoaderQueue.h => TaskDispatchQueue.h} | 31 +- cpp/src/db/scheduler/TaskDispatchStrategy.cpp | 122 +++++++ ...eduleStrategy.h => TaskDispatchStrategy.h} | 12 +- cpp/src/db/scheduler/TaskScheduler.cpp | 117 +++++++ cpp/src/db/scheduler/TaskScheduler.h | 49 +++ .../db/scheduler/context/DeleteContext.cpp | 22 ++ cpp/src/db/scheduler/context/DeleteContext.h | 31 ++ .../db/scheduler/context/IScheduleContext.h | 38 ++ .../scheduler/{ => context}/SearchContext.cpp | 3 +- .../scheduler/{ => context}/SearchContext.h | 3 +- cpp/src/db/scheduler/task/DeleteTask.cpp | 30 ++ .../DeleteTask.h} | 17 +- cpp/src/db/scheduler/task/IScheduleTask.h | 41 +++ cpp/src/db/scheduler/task/IndexLoadTask.cpp | 72 ++++ cpp/src/db/scheduler/task/IndexLoadTask.h | 30 ++ .../db/scheduler/{ => task}/SearchTask.cpp | 38 +- cpp/src/db/scheduler/{ => task}/SearchTask.h | 12 +- .../sdk/examples/simple/src/ClientTest.cpp | 10 +- cpp/src/server/DBWrapper.cpp | 42 +++ cpp/src/server/DBWrapper.h | 34 ++ cpp/src/server/MilvusServer.cpp | 3 + cpp/src/server/RequestTask.cpp | 62 +--- cpp/unittest/db/CMakeLists.txt | 10 +- cpp/unittest/metrics/CMakeLists.txt | 9 +- 35 files changed, 1016 insertions(+), 630 deletions(-) delete mode 100644 cpp/src/db/scheduler/ScheduleStrategy.cpp delete mode 100644 cpp/src/db/scheduler/SearchScheduler.cpp delete mode 100644 cpp/src/db/scheduler/SearchScheduler.h rename cpp/src/db/scheduler/{IndexLoaderQueue.cpp => TaskDispatchQueue.cpp} (76%) rename cpp/src/db/scheduler/{IndexLoaderQueue.h => TaskDispatchQueue.h} (53%) create mode 100644 cpp/src/db/scheduler/TaskDispatchStrategy.cpp rename cpp/src/db/scheduler/{ScheduleStrategy.h => TaskDispatchStrategy.h} (66%) create mode 100644 cpp/src/db/scheduler/TaskScheduler.cpp create mode 100644 cpp/src/db/scheduler/TaskScheduler.h create mode 100644 cpp/src/db/scheduler/context/DeleteContext.cpp create mode 100644 cpp/src/db/scheduler/context/DeleteContext.h create mode 100644 cpp/src/db/scheduler/context/IScheduleContext.h rename cpp/src/db/scheduler/{ => context}/SearchContext.cpp (95%) rename cpp/src/db/scheduler/{ => context}/SearchContext.h (95%) create mode 100644 cpp/src/db/scheduler/task/DeleteTask.cpp rename cpp/src/db/scheduler/{IScheduleStrategy.h => task/DeleteTask.h} (57%) create mode 100644 cpp/src/db/scheduler/task/IScheduleTask.h create mode 100644 cpp/src/db/scheduler/task/IndexLoadTask.cpp create mode 100644 cpp/src/db/scheduler/task/IndexLoadTask.h rename cpp/src/db/scheduler/{ => task}/SearchTask.cpp (82%) rename cpp/src/db/scheduler/{ => task}/SearchTask.h (77%) create mode 100644 cpp/src/server/DBWrapper.cpp create mode 100644 cpp/src/server/DBWrapper.h diff --git a/cpp/src/CMakeLists.txt b/cpp/src/CMakeLists.txt index 4645ffcbe6..807c16f5b3 100644 --- a/cpp/src/CMakeLists.txt +++ b/cpp/src/CMakeLists.txt @@ -10,10 +10,18 @@ aux_source_directory(config config_files) aux_source_directory(server server_files) aux_source_directory(utils utils_files) aux_source_directory(db db_files) -aux_source_directory(db/scheduler db_scheduler_files) aux_source_directory(wrapper wrapper_files) aux_source_directory(metrics metrics_files) +aux_source_directory(db/scheduler scheduler_files) +aux_source_directory(db/scheduler/context scheduler_context_files) +aux_source_directory(db/scheduler/task scheduler_task_files) +set(db_scheduler_files + ${scheduler_files} + ${scheduler_context_files} + ${scheduler_task_files} + ) + set(license_check_files license/LicenseLibrary.cpp license/LicenseCheck.cpp diff --git a/cpp/src/db/DBImpl.cpp b/cpp/src/db/DBImpl.cpp index 9b541b8726..8202c8a7db 100644 --- a/cpp/src/db/DBImpl.cpp +++ b/cpp/src/db/DBImpl.cpp @@ -8,7 +8,9 @@ #include "Log.h" #include "EngineFactory.h" #include "metrics/Metrics.h" -#include "scheduler/SearchScheduler.h" +#include "scheduler/TaskScheduler.h" +#include "scheduler/context/SearchContext.h" +#include "scheduler/context/DeleteContext.h" #include "utils/TimeRecorder.h" #include @@ -130,48 +132,52 @@ void CalcScore(uint64_t vector_count, DBImpl::DBImpl(const Options& options) : options_(options), shutting_down_(false), - pMeta_(new meta::DBMetaImpl(options_.meta)), - pMemMgr_(new MemManager(pMeta_, options_)), + meta_ptr_(new meta::DBMetaImpl(options_.meta)), + mem_mgr_(new MemManager(meta_ptr_, options_)), compact_thread_pool_(1, 1), index_thread_pool_(1, 1) { StartTimerTasks(); } Status DBImpl::CreateTable(meta::TableSchema& table_schema) { - return pMeta_->CreateTable(table_schema); + return meta_ptr_->CreateTable(table_schema); } Status DBImpl::DeleteTable(const std::string& table_id, const meta::DatesT& dates) { - //dates empty means delete all files of the table - if(dates.empty()) { - pMemMgr_->EraseMemVector(table_id); //not allow insert - pMeta_->DeleteTable(table_id); //soft delete - } + //dates partly delete files of the table but currently we don't support + + mem_mgr_->EraseMemVector(table_id); //not allow insert + meta_ptr_->DeleteTable(table_id); //soft delete table + + //scheduler will determine when to delete table files + TaskScheduler& scheduler = TaskScheduler::GetInstance(); + DeleteContextPtr context = std::make_shared(table_id, meta_ptr_); + scheduler.Schedule(context); return Status::OK(); } Status DBImpl::DescribeTable(meta::TableSchema& table_schema) { - return pMeta_->DescribeTable(table_schema); + return meta_ptr_->DescribeTable(table_schema); } Status DBImpl::HasTable(const std::string& table_id, bool& has_or_not) { - return pMeta_->HasTable(table_id, has_or_not); + return meta_ptr_->HasTable(table_id, has_or_not); } Status DBImpl::AllTables(std::vector& table_schema_array) { - return pMeta_->AllTables(table_schema_array); + return meta_ptr_->AllTables(table_schema_array); } Status DBImpl::GetTableRowCount(const std::string& table_id, uint64_t& row_count) { - return pMeta_->Count(table_id, row_count); + return meta_ptr_->Count(table_id, row_count); } Status DBImpl::InsertVectors(const std::string& table_id_, uint64_t n, const float* vectors, IDNumbers& vector_ids_) { auto start_time = METRICS_NOW_TIME; - Status status = pMemMgr_->InsertVectors(table_id_, n, vectors, vector_ids_); + Status status = mem_mgr_->InsertVectors(table_id_, n, vectors, vector_ids_); auto end_time = METRICS_NOW_TIME; double total_time = METRICS_MICROSECONDS(start_time,end_time); // std::chrono::microseconds time_span = std::chrono::duration_cast(end_time - start_time); @@ -203,7 +209,7 @@ Status DBImpl::Query(const std::string& table_id, uint64_t k, uint64_t nq, //get all table files from table meta::DatePartionedTableFilesSchema files; - auto status = pMeta_->FilesToSearch(table_id, dates, files); + auto status = meta_ptr_->FilesToSearch(table_id, dates, files); if (!status.ok()) { return status; } meta::TableFilesSchema file_id_array; @@ -225,7 +231,7 @@ Status DBImpl::Query(const std::string& table_id, const std::vector for (auto &id : file_ids) { meta::TableFileSchema table_file; table_file.table_id_ = id; - auto status = pMeta_->GetTableFile(table_file); + auto status = meta_ptr_->GetTableFile(table_file); if (!status.ok()) { return status; } @@ -238,7 +244,7 @@ Status DBImpl::Query(const std::string& table_id, const std::vector Status DBImpl::QuerySync(const std::string& table_id, uint64_t k, uint64_t nq, const float* vectors, const meta::DatesT& dates, QueryResults& results) { meta::DatePartionedTableFilesSchema files; - auto status = pMeta_->FilesToSearch(table_id, dates, files); + auto status = meta_ptr_->FilesToSearch(table_id, dates, files); if (!status.ok()) { return status; } ENGINE_LOG_DEBUG << "Search DateT Size = " << files.size(); @@ -387,8 +393,8 @@ Status DBImpl::QueryAsync(const std::string& table_id, const meta::TableFilesSch } //step 2: put search task to scheduler - SearchScheduler& scheduler = SearchScheduler::GetInstance(); - scheduler.ScheduleSearchTask(context); + TaskScheduler& scheduler = TaskScheduler::GetInstance(); + scheduler.Schedule(context); context->WaitResult(); @@ -396,7 +402,7 @@ Status DBImpl::QueryAsync(const std::string& table_id, const meta::TableFilesSch auto& context_result = context->GetResult(); meta::TableSchema table_schema; table_schema.table_id_ = table_id; - pMeta_->DescribeTable(table_schema); + meta_ptr_->DescribeTable(table_schema); CalcScore(context->nq(), context->vectors(), table_schema.dimension_, context_result, results); @@ -460,7 +466,7 @@ void DBImpl::StartCompactionTask() { //serialize memory data std::vector temp_table_ids; - pMemMgr_->Serialize(temp_table_ids); + mem_mgr_->Serialize(temp_table_ids); for(auto& id : temp_table_ids) { compact_table_ids_.insert(id); } @@ -486,10 +492,10 @@ Status DBImpl::MergeFiles(const std::string& table_id, const meta::DateT& date, meta::TableFileSchema table_file; table_file.table_id_ = table_id; table_file.date_ = date; - Status status = pMeta_->CreateTableFile(table_file); + Status status = meta_ptr_->CreateTableFile(table_file); if (!status.ok()) { - LOG(INFO) << status.ToString() << std::endl; + ENGINE_LOG_INFO << status.ToString() << std::endl; return status; } @@ -510,7 +516,7 @@ Status DBImpl::MergeFiles(const std::string& table_id, const meta::DateT& date, file_schema.file_type_ = meta::TableFileSchema::TO_DELETE; updated.push_back(file_schema); - LOG(DEBUG) << "Merging file " << file_schema.file_id_; + ENGINE_LOG_DEBUG << "Merging file " << file_schema.file_id_; index_size = index->Size(); if (index_size >= options_.index_trigger_size) break; @@ -526,8 +532,8 @@ Status DBImpl::MergeFiles(const std::string& table_id, const meta::DateT& date, } table_file.size_ = index_size; updated.push_back(table_file); - status = pMeta_->UpdateTableFiles(updated); - LOG(DEBUG) << "New merged file " << table_file.file_id_ << + status = meta_ptr_->UpdateTableFiles(updated); + ENGINE_LOG_DEBUG << "New merged file " << table_file.file_id_ << " of size=" << index->PhysicalSize()/(1024*1024) << " M"; index->Cache(); @@ -537,7 +543,7 @@ Status DBImpl::MergeFiles(const std::string& table_id, const meta::DateT& date, Status DBImpl::BackgroundMergeFiles(const std::string& table_id) { meta::DatePartionedTableFilesSchema raw_files; - auto status = pMeta_->FilesToMerge(table_id, raw_files); + auto status = meta_ptr_->FilesToMerge(table_id, raw_files); if (!status.ok()) { return status; } @@ -569,8 +575,8 @@ void DBImpl::BackgroundCompaction(std::set table_ids) { } } - pMeta_->Archive(); - pMeta_->CleanUpFilesWithTTL(1); + meta_ptr_->Archive(); + meta_ptr_->CleanUpFilesWithTTL(1); } void DBImpl::StartBuildIndexTask() { @@ -596,27 +602,43 @@ void DBImpl::StartBuildIndexTask() { } Status DBImpl::BuildIndex(const meta::TableFileSchema& file) { - meta::TableFileSchema table_file; - table_file.table_id_ = file.table_id_; - table_file.date_ = file.date_; - Status status = pMeta_->CreateTableFile(table_file); - if (!status.ok()) { - return status; - } - ExecutionEnginePtr to_index = EngineFactory::Build(file.dimension_, file.location_, (EngineType)file.engine_type_); if(to_index == nullptr) { return Status::Error("Invalid engine type"); } try { + //step 1: load index to_index->Load(); + + //step 2: create table file + meta::TableFileSchema table_file; + table_file.table_id_ = file.table_id_; + table_file.date_ = file.date_; + Status status = meta_ptr_->CreateTableFile(table_file); + if (!status.ok()) { + return status; + } + + //step 3: build index auto start_time = METRICS_NOW_TIME; auto index = to_index->BuildIndex(table_file.location_); auto end_time = METRICS_NOW_TIME; auto total_time = METRICS_MICROSECONDS(start_time, end_time); server::Metrics::GetInstance().BuildIndexDurationSecondsHistogramObserve(total_time); + //step 4: if table has been deleted, dont save index file + bool has_table = false; + meta_ptr_->HasTable(file.table_id_, has_table); + if(!has_table) { + meta_ptr_->DeleteTableFiles(file.table_id_); + return Status::OK(); + } + + //step 5: save index file + index->Serialize(); + + //step 6: update meta table_file.file_type_ = meta::TableFileSchema::INDEX; table_file.size_ = index->Size(); @@ -624,13 +646,13 @@ Status DBImpl::BuildIndex(const meta::TableFileSchema& file) { to_remove.file_type_ = meta::TableFileSchema::TO_DELETE; meta::TableFilesSchema update_files = {to_remove, table_file}; - pMeta_->UpdateTableFiles(update_files); + meta_ptr_->UpdateTableFiles(update_files); - LOG(DEBUG) << "New index file " << table_file.file_id_ << " of size " + ENGINE_LOG_DEBUG << "New index file " << table_file.file_id_ << " of size " << index->PhysicalSize()/(1024*1024) << " M" << " from file " << to_remove.file_id_; - index->Cache(); + //index->Cache(); } catch (std::exception& ex) { return Status::Error("Build index encounter exception", ex.what()); @@ -641,10 +663,10 @@ Status DBImpl::BuildIndex(const meta::TableFileSchema& file) { void DBImpl::BackgroundBuildIndex() { meta::TableFilesSchema to_index_files; - pMeta_->FilesToIndex(to_index_files); + meta_ptr_->FilesToIndex(to_index_files); Status status; for (auto& file : to_index_files) { - /* LOG(DEBUG) << "Buiding index for " << file.location; */ + /* ENGINE_LOG_DEBUG << "Buiding index for " << file.location; */ status = BuildIndex(file); if (!status.ok()) { bg_error_ = status; @@ -655,22 +677,22 @@ void DBImpl::BackgroundBuildIndex() { break; } } - /* LOG(DEBUG) << "All Buiding index Done"; */ + /* ENGINE_LOG_DEBUG << "All Buiding index Done"; */ } Status DBImpl::DropAll() { - return pMeta_->DropAll(); + return meta_ptr_->DropAll(); } Status DBImpl::Size(uint64_t& result) { - return pMeta_->Size(result); + return meta_ptr_->Size(result); } DBImpl::~DBImpl() { shutting_down_.store(true, std::memory_order_release); bg_timer_thread_.join(); std::vector ids; - pMemMgr_->Serialize(ids); + mem_mgr_->Serialize(ids); } } // namespace engine diff --git a/cpp/src/db/DBImpl.h b/cpp/src/db/DBImpl.h index 177faeb9c1..b4d60a27a9 100644 --- a/cpp/src/db/DBImpl.h +++ b/cpp/src/db/DBImpl.h @@ -93,8 +93,8 @@ private: std::thread bg_timer_thread_; - MetaPtr pMeta_; - MemManagerPtr pMemMgr_; + MetaPtr meta_ptr_; + MemManagerPtr mem_mgr_; server::ThreadPool compact_thread_pool_; std::list> compact_thread_results_; diff --git a/cpp/src/db/DBMetaImpl.cpp b/cpp/src/db/DBMetaImpl.cpp index 3ae34c1c2e..dbbc985067 100644 --- a/cpp/src/db/DBMetaImpl.cpp +++ b/cpp/src/db/DBMetaImpl.cpp @@ -258,6 +258,28 @@ Status DBMetaImpl::DeleteTable(const std::string& table_id) { return Status::OK(); } +Status DBMetaImpl::DeleteTableFiles(const std::string& table_id) { + try { + MetricCollector metric; + + //soft delete table files + ConnectorPtr->update_all( + set( + c(&TableFileSchema::file_type_) = (int) TableFileSchema::TO_DELETE, + c(&TableFileSchema::updated_time_) = utils::GetMicroSecTimeStamp() + ), + where( + c(&TableFileSchema::table_id_) == table_id and + c(&TableFileSchema::file_type_) != (int) TableFileSchema::TO_DELETE + )); + + } catch (std::exception &e) { + return HandleException("Encounter exception when delete table files", e); + } + + return Status::OK(); +} + Status DBMetaImpl::DescribeTable(TableSchema &table_schema) { try { MetricCollector metric; @@ -582,74 +604,6 @@ Status DBMetaImpl::FilesToMerge(const std::string &table_id, return Status::OK(); } -Status DBMetaImpl::FilesToDelete(const std::string& table_id, - const DatesT& partition, - DatePartionedTableFilesSchema& files) { - auto now = utils::GetMicroSecTimeStamp(); - try { - if(partition.empty()) { - //step 1: get table files by dates - auto selected = ConnectorPtr->select(columns(&TableFileSchema::id_, - &TableFileSchema::table_id_, - &TableFileSchema::file_id_, - &TableFileSchema::size_, - &TableFileSchema::date_), - where(c(&TableFileSchema::table_id_) == table_id)); - - //step 2: erase table files from meta - for (auto &file : selected) { - TableFileSchema table_file; - table_file.id_ = std::get<0>(file); - table_file.table_id_ = std::get<1>(file); - table_file.file_id_ = std::get<2>(file); - table_file.size_ = std::get<3>(file); - table_file.date_ = std::get<4>(file); - GetTableFilePath(table_file); - auto dateItr = files.find(table_file.date_); - if (dateItr == files.end()) { - files[table_file.date_] = TableFilesSchema(); - } - files[table_file.date_].push_back(table_file); - - ConnectorPtr->remove(std::get<0>(file)); - } - - } else { - //step 1: get all table files - auto selected = ConnectorPtr->select(columns(&TableFileSchema::id_, - &TableFileSchema::table_id_, - &TableFileSchema::file_id_, - &TableFileSchema::size_, - &TableFileSchema::date_), - where(in(&TableFileSchema::date_, partition) - and c(&TableFileSchema::table_id_) == table_id)); - - //step 2: erase table files from meta - for (auto &file : selected) { - TableFileSchema table_file; - table_file.id_ = std::get<0>(file); - table_file.table_id_ = std::get<1>(file); - table_file.file_id_ = std::get<2>(file); - table_file.size_ = std::get<3>(file); - table_file.date_ = std::get<4>(file); - GetTableFilePath(table_file); - auto dateItr = files.find(table_file.date_); - if (dateItr == files.end()) { - files[table_file.date_] = TableFilesSchema(); - } - files[table_file.date_].push_back(table_file); - - ConnectorPtr->remove(std::get<0>(file)); - } - } - - } catch (std::exception &e) { - return HandleException("Encounter exception when iterate delete files", e); - } - - return Status::OK(); -} - Status DBMetaImpl::GetTableFile(TableFileSchema &file_schema) { try { @@ -745,41 +699,52 @@ Status DBMetaImpl::DiscardFiles(long to_discard_size) { return Status::OK(); } - LOG(DEBUG) << "About to discard size=" << to_discard_size; + ENGINE_LOG_DEBUG << "About to discard size=" << to_discard_size; try { - auto selected = ConnectorPtr->select(columns(&TableFileSchema::id_, - &TableFileSchema::size_), - where(c(&TableFileSchema::file_type_) + MetricCollector metric; + + auto commited = ConnectorPtr->transaction([&]() mutable { + auto selected = ConnectorPtr->select(columns(&TableFileSchema::id_, + &TableFileSchema::size_), + where(c(&TableFileSchema::file_type_) != (int) TableFileSchema::TO_DELETE), - order_by(&TableFileSchema::id_), - limit(10)); + order_by(&TableFileSchema::id_), + limit(10)); - std::vector ids; - TableFileSchema table_file; + std::vector ids; + TableFileSchema table_file; - for (auto &file : selected) { - if (to_discard_size <= 0) break; - table_file.id_ = std::get<0>(file); - table_file.size_ = std::get<1>(file); - ids.push_back(table_file.id_); - ENGINE_LOG_DEBUG << "Discard table_file.id=" << table_file.file_id_ - << " table_file.size=" << table_file.size_; - to_discard_size -= table_file.size_; + for (auto &file : selected) { + if (to_discard_size <= 0) break; + table_file.id_ = std::get<0>(file); + table_file.size_ = std::get<1>(file); + ids.push_back(table_file.id_); + ENGINE_LOG_DEBUG << "Discard table_file.id=" << table_file.file_id_ + << " table_file.size=" << table_file.size_; + to_discard_size -= table_file.size_; + } + + if (ids.size() == 0) { + return true; + } + + ConnectorPtr->update_all( + set( + c(&TableFileSchema::file_type_) = (int) TableFileSchema::TO_DELETE, + c(&TableFileSchema::updated_time_) = utils::GetMicroSecTimeStamp() + ), + where( + in(&TableFileSchema::id_, ids) + )); + + return true; + }); + + if (!commited) { + return Status::DBTransactionError("Update table file error"); } - if (ids.size() == 0) { - return Status::OK(); - } - - ConnectorPtr->update_all( - set( - c(&TableFileSchema::file_type_) = (int) TableFileSchema::TO_DELETE - ), - where( - in(&TableFileSchema::id_, ids) - )); - } catch (std::exception &e) { return HandleException("Encounter exception when discard table file", e); } @@ -792,11 +757,21 @@ Status DBMetaImpl::UpdateTableFile(TableFileSchema &file_schema) { try { MetricCollector metric; + auto tables = ConnectorPtr->select(columns(&TableSchema::state_), + where(c(&TableSchema::table_id_) == file_schema.table_id_)); + + //if the table has been deleted, just mark the table file as TO_DELETE + //clean thread will delete the file later + if(tables.size() < 1 || std::get<0>(tables[0]) == (int)TableSchema::TO_DELETE) { + file_schema.file_type_ = TableFileSchema::TO_DELETE; + } + ConnectorPtr->update(file_schema); } catch (std::exception &e) { - ENGINE_LOG_DEBUG << "table_id= " << file_schema.table_id_ << " file_id=" << file_schema.file_id_; - return HandleException("Encounter exception when update table file", e); + std::string msg = "Exception update table file: table_id = " + file_schema.table_id_ + + " file_id = " + file_schema.file_id_; + return HandleException(msg, e); } return Status::OK(); } @@ -805,16 +780,37 @@ Status DBMetaImpl::UpdateTableFiles(TableFilesSchema &files) { try { MetricCollector metric; + std::map has_tables; + for (auto &file : files) { + if(has_tables.find(file.table_id_) != has_tables.end()) { + continue; + } + auto tables = ConnectorPtr->select(columns(&TableSchema::id_), + where(c(&TableSchema::table_id_) == file.table_id_ + and c(&TableSchema::state_) != (int) TableSchema::TO_DELETE)); + if(tables.size() >= 1) { + has_tables[file.table_id_] = true; + } else { + has_tables[file.table_id_] = false; + } + } + auto commited = ConnectorPtr->transaction([&]() mutable { for (auto &file : files) { + if(!has_tables[file.table_id_]) { + file.file_type_ = TableFileSchema::TO_DELETE; + } + file.updated_time_ = utils::GetMicroSecTimeStamp(); ConnectorPtr->update(file); } return true; }); + if (!commited) { - return Status::DBTransactionError("Update files Error"); + return Status::DBTransactionError("Update table files error"); } + } catch (std::exception &e) { return HandleException("Encounter exception when update table files", e); } @@ -824,35 +820,67 @@ Status DBMetaImpl::UpdateTableFiles(TableFilesSchema &files) { Status DBMetaImpl::CleanUpFilesWithTTL(uint16_t seconds) { auto now = utils::GetMicroSecTimeStamp(); try { - auto selected = ConnectorPtr->select(columns(&TableFileSchema::id_, - &TableFileSchema::table_id_, - &TableFileSchema::file_id_, - &TableFileSchema::file_type_, - &TableFileSchema::size_, - &TableFileSchema::date_), - where( - c(&TableFileSchema::file_type_) == (int) TableFileSchema::TO_DELETE - and - c(&TableFileSchema::updated_time_) - > now - seconds * US_PS)); + MetricCollector metric; - TableFilesSchema updated; - TableFileSchema table_file; + auto files = ConnectorPtr->select(columns(&TableFileSchema::id_, + &TableFileSchema::table_id_, + &TableFileSchema::file_id_, + &TableFileSchema::date_), + where( + c(&TableFileSchema::file_type_) == + (int) TableFileSchema::TO_DELETE + and + c(&TableFileSchema::updated_time_) + < now - seconds * US_PS)); - for (auto &file : selected) { - table_file.id_ = std::get<0>(file); - table_file.table_id_ = std::get<1>(file); - table_file.file_id_ = std::get<2>(file); - table_file.file_type_ = std::get<3>(file); - table_file.size_ = std::get<4>(file); - table_file.date_ = std::get<5>(file); - GetTableFilePath(table_file); - if (table_file.file_type_ == TableFileSchema::TO_DELETE) { + auto commited = ConnectorPtr->transaction([&]() mutable { + TableFileSchema table_file; + for (auto &file : files) { + table_file.id_ = std::get<0>(file); + table_file.table_id_ = std::get<1>(file); + table_file.file_id_ = std::get<2>(file); + table_file.date_ = std::get<3>(file); + GetTableFilePath(table_file); + + ENGINE_LOG_DEBUG << "Removing deleted id =" << table_file.id_ << " location = " << table_file.location_ << std::endl; boost::filesystem::remove(table_file.location_); + ConnectorPtr->remove(table_file.id_); + } - ConnectorPtr->remove(table_file.id_); - /* LOG(DEBUG) << "Removing deleted id=" << table_file.id << " location=" << table_file.location << std::endl; */ + return true; + }); + + if (!commited) { + return Status::DBTransactionError("Clean files error"); } + + } catch (std::exception &e) { + return HandleException("Encounter exception when clean table files", e); + } + + try { + MetricCollector metric; + + auto tables = ConnectorPtr->select(columns(&TableSchema::id_, + &TableSchema::table_id_), + where(c(&TableSchema::state_) == (int) TableSchema::TO_DELETE)); + + auto commited = ConnectorPtr->transaction([&]() mutable { + for (auto &table : tables) { + auto table_path = GetTablePath(std::get<1>(table)); + + ENGINE_LOG_DEBUG << "Remove table folder: " << table_path; + boost::filesystem::remove_all(table_path); + ConnectorPtr->remove(std::get<0>(table)); + } + + return true; + }); + + if (!commited) { + return Status::DBTransactionError("Clean files error"); + } + } catch (std::exception &e) { return HandleException("Encounter exception when clean table files", e); } @@ -862,35 +890,21 @@ Status DBMetaImpl::CleanUpFilesWithTTL(uint16_t seconds) { Status DBMetaImpl::CleanUp() { try { - auto selected = ConnectorPtr->select(columns(&TableFileSchema::id_, - &TableFileSchema::table_id_, - &TableFileSchema::file_id_, - &TableFileSchema::file_type_, - &TableFileSchema::size_, - &TableFileSchema::date_), - where( - c(&TableFileSchema::file_type_) == (int) TableFileSchema::TO_DELETE - or - c(&TableFileSchema::file_type_) - == (int) TableFileSchema::NEW)); + auto files = ConnectorPtr->select(columns(&TableFileSchema::id_), + where(c(&TableFileSchema::file_type_) == (int) TableFileSchema::NEW)); - TableFilesSchema updated; - TableFileSchema table_file; - - for (auto &file : selected) { - table_file.id_ = std::get<0>(file); - table_file.table_id_ = std::get<1>(file); - table_file.file_id_ = std::get<2>(file); - table_file.file_type_ = std::get<3>(file); - table_file.size_ = std::get<4>(file); - table_file.date_ = std::get<5>(file); - GetTableFilePath(table_file); - if (table_file.file_type_ == TableFileSchema::TO_DELETE) { - boost::filesystem::remove(table_file.location_); + auto commited = ConnectorPtr->transaction([&]() mutable { + for (auto &file : files) { + ENGINE_LOG_DEBUG << "Remove table file type as NEW"; + ConnectorPtr->remove(std::get<0>(file)); } - ConnectorPtr->remove(table_file.id_); - /* LOG(DEBUG) << "Removing id=" << table_file.id << " location=" << table_file.location << std::endl; */ + return true; + }); + + if (!commited) { + return Status::DBTransactionError("Clean files error"); } + } catch (std::exception &e) { return HandleException("Encounter exception when clean table file", e); } @@ -903,14 +917,12 @@ Status DBMetaImpl::Count(const std::string &table_id, uint64_t &result) { try { MetricCollector metric; - auto selected = ConnectorPtr->select(columns(&TableFileSchema::size_, - &TableFileSchema::date_), - where((c(&TableFileSchema::file_type_) == (int) TableFileSchema::RAW or - c(&TableFileSchema::file_type_) == (int) TableFileSchema::TO_INDEX - or - c(&TableFileSchema::file_type_) == (int) TableFileSchema::INDEX) - and - c(&TableFileSchema::table_id_) == table_id)); + auto selected = ConnectorPtr->select(columns(&TableFileSchema::size_), + where((c(&TableFileSchema::file_type_) == (int) TableFileSchema::RAW + or + c(&TableFileSchema::file_type_) == (int) TableFileSchema::TO_INDEX + or c(&TableFileSchema::file_type_) == (int) TableFileSchema::INDEX) + and c(&TableFileSchema::table_id_) == table_id)); TableSchema table_schema; table_schema.table_id_ = table_id; diff --git a/cpp/src/db/DBMetaImpl.h b/cpp/src/db/DBMetaImpl.h index dcac77ded8..21ce8d1df6 100644 --- a/cpp/src/db/DBMetaImpl.h +++ b/cpp/src/db/DBMetaImpl.h @@ -20,11 +20,13 @@ public: DBMetaImpl(const DBMetaOptions& options_); virtual Status CreateTable(TableSchema& table_schema) override; - virtual Status DeleteTable(const std::string& table_id) override; virtual Status DescribeTable(TableSchema& group_info_) override; virtual Status HasTable(const std::string& table_id, bool& has_or_not) override; virtual Status AllTables(std::vector& table_schema_array) override; + virtual Status DeleteTable(const std::string& table_id) override; + virtual Status DeleteTableFiles(const std::string& table_id) override; + virtual Status CreateTableFile(TableFileSchema& file_schema) override; virtual Status DropPartitionsByDates(const std::string& table_id, const DatesT& dates) override; @@ -42,10 +44,6 @@ public: virtual Status FilesToMerge(const std::string& table_id, DatePartionedTableFilesSchema& files) override; - virtual Status FilesToDelete(const std::string& table_id, - const DatesT& partition, - DatePartionedTableFilesSchema& files) override; - virtual Status FilesToIndex(TableFilesSchema&) override; virtual Status Archive() override; diff --git a/cpp/src/db/FaissExecutionEngine.cpp b/cpp/src/db/FaissExecutionEngine.cpp index 467ebe31e0..9dfdd978c3 100644 --- a/cpp/src/db/FaissExecutionEngine.cpp +++ b/cpp/src/db/FaissExecutionEngine.cpp @@ -4,8 +4,8 @@ * Proprietary and confidential. ******************************************************************************/ #include "FaissExecutionEngine.h" +#include "Log.h" -#include #include #include #include @@ -74,7 +74,7 @@ Status FaissExecutionEngine::Load() { if (!index) { index = read_index(location_); to_cache = true; - LOG(DEBUG) << "Disk io from: " << location_; + ENGINE_LOG_DEBUG << "Disk io from: " << location_; } pIndex_ = index->data(); @@ -98,6 +98,8 @@ Status FaissExecutionEngine::Merge(const std::string& location) { if (location == location_) { return Status::Error("Cannot Merge Self"); } + ENGINE_LOG_DEBUG << "Merge index file: " << location << " to: " << location_; + auto to_merge = zilliz::milvus::cache::CpuCacheMgr::GetInstance()->GetIndex(location); if (!to_merge) { to_merge = read_index(location); @@ -110,6 +112,8 @@ Status FaissExecutionEngine::Merge(const std::string& location) { ExecutionEnginePtr FaissExecutionEngine::BuildIndex(const std::string& location) { + ENGINE_LOG_DEBUG << "Build index file: " << location << " from: " << location_; + auto opd = std::make_shared(); opd->d = pIndex_->d; opd->index_type = build_index_type_; @@ -122,7 +126,6 @@ FaissExecutionEngine::BuildIndex(const std::string& location) { from_index->id_map.data()); ExecutionEnginePtr new_ee(new FaissExecutionEngine(index->data(), location, build_index_type_, raw_index_type_)); - new_ee->Serialize(); return new_ee; } diff --git a/cpp/src/db/Meta.h b/cpp/src/db/Meta.h index d8242db884..3bd60cc7fe 100644 --- a/cpp/src/db/Meta.h +++ b/cpp/src/db/Meta.h @@ -24,11 +24,13 @@ public: using Ptr = std::shared_ptr; virtual Status CreateTable(TableSchema& table_schema) = 0; - virtual Status DeleteTable(const std::string& table_id) = 0; virtual Status DescribeTable(TableSchema& table_schema) = 0; virtual Status HasTable(const std::string& table_id, bool& has_or_not) = 0; virtual Status AllTables(std::vector& table_schema_array) = 0; + virtual Status DeleteTable(const std::string& table_id) = 0; + virtual Status DeleteTableFiles(const std::string& table_id) = 0; + virtual Status CreateTableFile(TableFileSchema& file_schema) = 0; virtual Status DropPartitionsByDates(const std::string& table_id, const DatesT& dates) = 0; @@ -45,10 +47,6 @@ public: virtual Status FilesToMerge(const std::string& table_id, DatePartionedTableFilesSchema& files) = 0; - virtual Status FilesToDelete(const std::string& table_id, - const DatesT& partition, - DatePartionedTableFilesSchema& files) = 0; - virtual Status Size(uint64_t& result) = 0; virtual Status Archive() = 0; diff --git a/cpp/src/db/scheduler/ScheduleStrategy.cpp b/cpp/src/db/scheduler/ScheduleStrategy.cpp deleted file mode 100644 index f80e01a1f4..0000000000 --- a/cpp/src/db/scheduler/ScheduleStrategy.cpp +++ /dev/null @@ -1,64 +0,0 @@ -/******************************************************************************* - * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved - * Unauthorized copying of this file, via any medium is strictly prohibited. - * Proprietary and confidential. - ******************************************************************************/ - - -#include "ScheduleStrategy.h" -#include "cache/CpuCacheMgr.h" -#include "utils/Error.h" -#include "utils/Log.h" - -namespace zilliz { -namespace milvus { -namespace engine { - -class MemScheduleStrategy : public IScheduleStrategy { -public: - bool Schedule(const SearchContextPtr &search_context, IndexLoaderQueue::LoaderQueue& loader_list) override { - if(search_context == nullptr) { - return false; - } - - SearchContext::Id2IndexMap index_files = search_context->GetIndexMap(); - //some index loader alread exists - for(auto& loader : loader_list) { - if(index_files.find(loader->file_->id_) != index_files.end()){ - SERVER_LOG_INFO << "Append SearchContext to exist IndexLoaderContext"; - index_files.erase(loader->file_->id_); - loader->search_contexts_.push_back(search_context); - } - } - - //index_files still contains some index files, create new loader - for(auto& pair : index_files) { - SERVER_LOG_INFO << "Create new IndexLoaderContext for: " << pair.second->location_; - IndexLoaderContextPtr new_loader = std::make_shared(); - new_loader->search_contexts_.push_back(search_context); - new_loader->file_ = pair.second; - - auto index = zilliz::milvus::cache::CpuCacheMgr::GetInstance()->GetIndex(pair.second->location_); - if(index != nullptr) { - //if the index file has been in memory, increase its priority - loader_list.push_front(new_loader); - } else { - //index file not in memory, put it to tail - loader_list.push_back(new_loader); - } - } - - return true; - } -}; - - -//////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// -ScheduleStrategyPtr StrategyFactory::CreateMemStrategy() { - ScheduleStrategyPtr strategy(new MemScheduleStrategy()); - return strategy; -} - -} -} -} \ No newline at end of file diff --git a/cpp/src/db/scheduler/SearchScheduler.cpp b/cpp/src/db/scheduler/SearchScheduler.cpp deleted file mode 100644 index fa0dee88fb..0000000000 --- a/cpp/src/db/scheduler/SearchScheduler.cpp +++ /dev/null @@ -1,180 +0,0 @@ -/******************************************************************************* - * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved - * Unauthorized copying of this file, via any medium is strictly prohibited. - * Proprietary and confidential. - ******************************************************************************/ - -#include "SearchScheduler.h" -#include "IndexLoaderQueue.h" -#include "SearchTask.h" -#include "utils/Log.h" -#include "utils/TimeRecorder.h" -#include "metrics/Metrics.h" -#include "db/EngineFactory.h" - -namespace zilliz { -namespace milvus { -namespace engine { - -namespace { -void CollectFileMetrics(int file_type, size_t file_size) { - switch(file_type) { - case meta::TableFileSchema::RAW: - case meta::TableFileSchema::TO_INDEX: { - server::Metrics::GetInstance().RawFileSizeHistogramObserve(file_size); - server::Metrics::GetInstance().RawFileSizeTotalIncrement(file_size); - server::Metrics::GetInstance().RawFileSizeGaugeSet(file_size); - break; - } - default: { - server::Metrics::GetInstance().IndexFileSizeHistogramObserve(file_size); - server::Metrics::GetInstance().IndexFileSizeTotalIncrement(file_size); - server::Metrics::GetInstance().IndexFileSizeGaugeSet(file_size); - break; - } - } -} - -void CollectDurationMetrics(int index_type, double total_time) { - switch(index_type) { - case meta::TableFileSchema::RAW: { - server::Metrics::GetInstance().SearchRawDataDurationSecondsHistogramObserve(total_time); - break; - } - case meta::TableFileSchema::TO_INDEX: { - server::Metrics::GetInstance().SearchRawDataDurationSecondsHistogramObserve(total_time); - break; - } - default: { - server::Metrics::GetInstance().SearchIndexDataDurationSecondsHistogramObserve(total_time); - break; - } - } -} - -} - -SearchScheduler::SearchScheduler() - : stopped_(true) { - Start(); -} - -SearchScheduler::~SearchScheduler() { - Stop(); -} - -SearchScheduler& SearchScheduler::GetInstance() { - static SearchScheduler s_instance; - return s_instance; -} - -bool -SearchScheduler::Start() { - if(!stopped_) { - return true; - } - - stopped_ = false; - - search_queue_.SetCapacity(2); - - index_load_thread_ = std::make_shared(&SearchScheduler::IndexLoadWorker, this); - search_thread_ = std::make_shared(&SearchScheduler::SearchWorker, this); - - return true; -} - -bool -SearchScheduler::Stop() { - if(stopped_) { - return true; - } - - if(index_load_thread_) { - index_load_queue_.Put(nullptr); - index_load_thread_->join(); - index_load_thread_ = nullptr; - } - - if(search_thread_) { - search_queue_.Put(nullptr); - search_thread_->join(); - search_thread_ = nullptr; - } - - stopped_ = true; - - return true; -} - -bool -SearchScheduler::ScheduleSearchTask(SearchContextPtr& search_context) { - index_load_queue_.Put(search_context); - - return true; -} - -bool -SearchScheduler::IndexLoadWorker() { - while(true) { - IndexLoaderContextPtr context = index_load_queue_.Take(); - if(context == nullptr) { - SERVER_LOG_INFO << "Stop thread for index loading"; - break;//exit - } - - SERVER_LOG_INFO << "Loading index(" << context->file_->id_ << ") from location: " << context->file_->location_; - - server::TimeRecorder rc("Load index"); - //step 1: load index - ExecutionEnginePtr index_ptr = EngineFactory::Build(context->file_->dimension_, - context->file_->location_, - (EngineType)context->file_->engine_type_); - index_ptr->Load(); - - rc.Record("load index file to memory"); - - size_t file_size = index_ptr->PhysicalSize(); - LOG(DEBUG) << "Index file type " << context->file_->file_type_ << " Of Size: " - << file_size/(1024*1024) << " M"; - - CollectFileMetrics(context->file_->file_type_, file_size); - - //step 2: put search task into another queue - SearchTaskPtr task_ptr = std::make_shared(); - task_ptr->index_id_ = context->file_->id_; - task_ptr->index_type_ = context->file_->file_type_; - task_ptr->index_engine_ = index_ptr; - task_ptr->search_contexts_.swap(context->search_contexts_); - search_queue_.Put(task_ptr); - } - - return true; -} - -bool -SearchScheduler::SearchWorker() { - while(true) { - SearchTaskPtr task_ptr = search_queue_.Take(); - if(task_ptr == nullptr) { - SERVER_LOG_INFO << "Stop thread for searching"; - break;//exit - } - - SERVER_LOG_INFO << "Searching in index(" << task_ptr->index_id_<< ") with " - << task_ptr->search_contexts_.size() << " tasks"; - - //do search - auto start_time = METRICS_NOW_TIME; - task_ptr->DoSearch(); - auto end_time = METRICS_NOW_TIME; - auto total_time = METRICS_MICROSECONDS(start_time, end_time); - CollectDurationMetrics(task_ptr->index_type_, total_time); - } - - return true; -} - -} -} -} \ No newline at end of file diff --git a/cpp/src/db/scheduler/SearchScheduler.h b/cpp/src/db/scheduler/SearchScheduler.h deleted file mode 100644 index 673d759056..0000000000 --- a/cpp/src/db/scheduler/SearchScheduler.h +++ /dev/null @@ -1,48 +0,0 @@ -/******************************************************************************* - * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved - * Unauthorized copying of this file, via any medium is strictly prohibited. - * Proprietary and confidential. - ******************************************************************************/ -#pragma once - -#include "SearchContext.h" -#include "IndexLoaderQueue.h" -#include "SearchTask.h" - -namespace zilliz { -namespace milvus { -namespace engine { - -class SearchScheduler { -private: - SearchScheduler(); - virtual ~SearchScheduler(); - -public: - static SearchScheduler& GetInstance(); - - bool ScheduleSearchTask(SearchContextPtr& search_context); - -private: - bool Start(); - bool Stop(); - - bool IndexLoadWorker(); - bool SearchWorker(); - -private: - std::shared_ptr index_load_thread_; - std::shared_ptr search_thread_; - - IndexLoaderQueue index_load_queue_; - - using SearchTaskQueue = server::BlockingQueue; - SearchTaskQueue search_queue_; - - bool stopped_ = true; -}; - - -} -} -} diff --git a/cpp/src/db/scheduler/IndexLoaderQueue.cpp b/cpp/src/db/scheduler/TaskDispatchQueue.cpp similarity index 76% rename from cpp/src/db/scheduler/IndexLoaderQueue.cpp rename to cpp/src/db/scheduler/TaskDispatchQueue.cpp index 2522815520..2ce0e933b4 100644 --- a/cpp/src/db/scheduler/IndexLoaderQueue.cpp +++ b/cpp/src/db/scheduler/TaskDispatchQueue.cpp @@ -4,8 +4,8 @@ * Proprietary and confidential. ******************************************************************************/ -#include "IndexLoaderQueue.h" -#include "ScheduleStrategy.h" +#include "TaskDispatchQueue.h" +#include "TaskDispatchStrategy.h" #include "utils/Error.h" #include "utils/Log.h" @@ -14,12 +14,12 @@ namespace milvus { namespace engine { void -IndexLoaderQueue::Put(const SearchContextPtr &search_context) { +TaskDispatchQueue::Put(const ScheduleContextPtr &context) { std::unique_lock lock(mtx); full_.wait(lock, [this] { return (queue_.size() < capacity_); }); - if(search_context == nullptr) { - queue_.push_back(nullptr); + if(context == nullptr) { + queue_.push_front(nullptr); empty_.notify_all(); return; } @@ -32,14 +32,13 @@ IndexLoaderQueue::Put(const SearchContextPtr &search_context) { throw server::ServerException(server::SERVER_BLOCKING_QUEUE_EMPTY, error_msg); } - ScheduleStrategyPtr strategy = StrategyFactory::CreateMemStrategy(); - strategy->Schedule(search_context, queue_); + TaskDispatchStrategy::Schedule(context, queue_); empty_.notify_all(); } -IndexLoaderContextPtr -IndexLoaderQueue::Take() { +ScheduleTaskPtr +TaskDispatchQueue::Take() { std::unique_lock lock(mtx); empty_.wait(lock, [this] { return !queue_.empty(); }); @@ -49,20 +48,20 @@ IndexLoaderQueue::Take() { throw server::ServerException(server::SERVER_BLOCKING_QUEUE_EMPTY, error_msg); } - IndexLoaderContextPtr front(queue_.front()); + ScheduleTaskPtr front(queue_.front()); queue_.pop_front(); full_.notify_all(); return front; } size_t -IndexLoaderQueue::Size() { +TaskDispatchQueue::Size() { std::lock_guard lock(mtx); return queue_.size(); } -IndexLoaderContextPtr -IndexLoaderQueue::Front() { +ScheduleTaskPtr +TaskDispatchQueue::Front() { std::unique_lock lock(mtx); empty_.wait(lock, [this] { return !queue_.empty(); }); if (queue_.empty()) { @@ -70,12 +69,12 @@ IndexLoaderQueue::Front() { SERVER_LOG_ERROR << error_msg; throw server::ServerException(server::SERVER_BLOCKING_QUEUE_EMPTY, error_msg); } - IndexLoaderContextPtr front(queue_.front()); + ScheduleTaskPtr front(queue_.front()); return front; } -IndexLoaderContextPtr -IndexLoaderQueue::Back() { +ScheduleTaskPtr +TaskDispatchQueue::Back() { std::unique_lock lock(mtx); empty_.wait(lock, [this] { return !queue_.empty(); }); @@ -85,18 +84,18 @@ IndexLoaderQueue::Back() { throw server::ServerException(server::SERVER_BLOCKING_QUEUE_EMPTY, error_msg); } - IndexLoaderContextPtr back(queue_.back()); + ScheduleTaskPtr back(queue_.back()); return back; } bool -IndexLoaderQueue::Empty() { +TaskDispatchQueue::Empty() { std::unique_lock lock(mtx); return queue_.empty(); } void -IndexLoaderQueue::SetCapacity(const size_t capacity) { +TaskDispatchQueue::SetCapacity(const size_t capacity) { capacity_ = (capacity > 0 ? capacity : capacity_); } diff --git a/cpp/src/db/scheduler/IndexLoaderQueue.h b/cpp/src/db/scheduler/TaskDispatchQueue.h similarity index 53% rename from cpp/src/db/scheduler/IndexLoaderQueue.h rename to cpp/src/db/scheduler/TaskDispatchQueue.h index 4f6dcfcd67..435be4f597 100644 --- a/cpp/src/db/scheduler/IndexLoaderQueue.h +++ b/cpp/src/db/scheduler/TaskDispatchQueue.h @@ -5,7 +5,8 @@ ******************************************************************************/ #pragma once -#include "SearchContext.h" +#include "context/IScheduleContext.h" +#include "task/IScheduleTask.h" #include #include @@ -17,31 +18,23 @@ namespace zilliz { namespace milvus { namespace engine { - -class IndexLoaderContext { +class TaskDispatchQueue { public: - TableFileSchemaPtr file_; - std::vector search_contexts_; -}; -using IndexLoaderContextPtr = std::shared_ptr; + TaskDispatchQueue() : mtx(), full_(), empty_() {} -class IndexLoaderQueue { -public: - IndexLoaderQueue() : mtx(), full_(), empty_() {} + TaskDispatchQueue(const TaskDispatchQueue &rhs) = delete; - IndexLoaderQueue(const IndexLoaderQueue &rhs) = delete; + TaskDispatchQueue &operator=(const TaskDispatchQueue &rhs) = delete; - IndexLoaderQueue &operator=(const IndexLoaderQueue &rhs) = delete; + using TaskList = std::list; - using LoaderQueue = std::list; + void Put(const ScheduleContextPtr &context); - void Put(const SearchContextPtr &search_context); + ScheduleTaskPtr Take(); - IndexLoaderContextPtr Take(); + ScheduleTaskPtr Front(); - IndexLoaderContextPtr Front(); - - IndexLoaderContextPtr Back(); + ScheduleTaskPtr Back(); size_t Size(); @@ -54,7 +47,7 @@ private: std::condition_variable full_; std::condition_variable empty_; - LoaderQueue queue_; + TaskList queue_; size_t capacity_ = 1000000; }; diff --git a/cpp/src/db/scheduler/TaskDispatchStrategy.cpp b/cpp/src/db/scheduler/TaskDispatchStrategy.cpp new file mode 100644 index 0000000000..7200f2584f --- /dev/null +++ b/cpp/src/db/scheduler/TaskDispatchStrategy.cpp @@ -0,0 +1,122 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ +#include "TaskDispatchStrategy.h" +#include "context/SearchContext.h" +#include "context/DeleteContext.h" +#include "task/IndexLoadTask.h" +#include "task/DeleteTask.h" +#include "cache/CpuCacheMgr.h" +#include "utils/Error.h" +#include "db/Log.h" + +namespace zilliz { +namespace milvus { +namespace engine { + +class ReuseCacheIndexStrategy { +public: + bool Schedule(const SearchContextPtr &context, std::list& task_list) { + if(context == nullptr) { + return false; + } + + SearchContext::Id2IndexMap index_files = context->GetIndexMap(); + //some index loader alread exists + for(auto& task : task_list) { + if(task->type() != ScheduleTaskType::kIndexLoad) { + continue; + } + + IndexLoadTaskPtr loader = std::static_pointer_cast(task); + if(index_files.find(loader->file_->id_) != index_files.end()){ + ENGINE_LOG_INFO << "Append SearchContext to exist IndexLoaderContext"; + index_files.erase(loader->file_->id_); + loader->search_contexts_.push_back(context); + } + } + + //index_files still contains some index files, create new loader + for(auto& pair : index_files) { + ENGINE_LOG_INFO << "Create new IndexLoaderContext for: " << pair.second->location_; + IndexLoadTaskPtr new_loader = std::make_shared(); + new_loader->search_contexts_.push_back(context); + new_loader->file_ = pair.second; + + auto index = zilliz::milvus::cache::CpuCacheMgr::GetInstance()->GetIndex(pair.second->location_); + if(index != nullptr) { + //if the index file has been in memory, increase its priority + task_list.push_front(new_loader); + } else { + //index file not in memory, put it to tail + task_list.push_back(new_loader); + } + } + + return true; + } +}; + +//////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// +class DeleteTableStrategy { +public: + bool Schedule(const DeleteContextPtr &context, std::list &task_list) { + if (context == nullptr) { + return false; + } + + DeleteTaskPtr delete_task = std::make_shared(context); + if(task_list.empty()) { + task_list.push_back(delete_task); + return true; + } + + std::string table_id = context->table_id(); + for(auto iter = task_list.begin(); iter != task_list.end(); ++iter) { + if((*iter)->type() != ScheduleTaskType::kIndexLoad) { + continue; + } + + //put delete task to proper position + IndexLoadTaskPtr loader = std::static_pointer_cast(*iter); + if(loader->file_->table_id_ == table_id) { + + task_list.insert(++iter, delete_task); + break; + } + } + + return true; + } +}; + + +//////////////////////////////////////////////////////////////////////////////////////////////////////////////////////// +bool TaskDispatchStrategy::Schedule(const ScheduleContextPtr &context_ptr, + std::list &task_list) { + if(context_ptr == nullptr) { + return false; + } + + switch(context_ptr->type()) { + case ScheduleContextType::kSearch: { + SearchContextPtr search_context = std::static_pointer_cast(context_ptr); + ReuseCacheIndexStrategy strategy; + return strategy.Schedule(search_context, task_list); + } + case ScheduleContextType::kDelete: { + DeleteContextPtr delete_context = std::static_pointer_cast(context_ptr); + DeleteTableStrategy strategy; + return strategy.Schedule(delete_context, task_list); + } + default: + ENGINE_LOG_ERROR << "Invalid schedule task type"; + return false; + } +} + +} +} +} \ No newline at end of file diff --git a/cpp/src/db/scheduler/ScheduleStrategy.h b/cpp/src/db/scheduler/TaskDispatchStrategy.h similarity index 66% rename from cpp/src/db/scheduler/ScheduleStrategy.h rename to cpp/src/db/scheduler/TaskDispatchStrategy.h index 1fc0493e41..d835e3ce86 100644 --- a/cpp/src/db/scheduler/ScheduleStrategy.h +++ b/cpp/src/db/scheduler/TaskDispatchStrategy.h @@ -5,18 +5,18 @@ ******************************************************************************/ #pragma once -#include "IScheduleStrategy.h" +#include "context/IScheduleContext.h" +#include "task/IScheduleTask.h" + +#include namespace zilliz { namespace milvus { namespace engine { -class StrategyFactory { -private: - StrategyFactory() {} - +class TaskDispatchStrategy { public: - static ScheduleStrategyPtr CreateMemStrategy(); + static bool Schedule(const ScheduleContextPtr &context_ptr, std::list& task_list); }; } diff --git a/cpp/src/db/scheduler/TaskScheduler.cpp b/cpp/src/db/scheduler/TaskScheduler.cpp new file mode 100644 index 0000000000..04d6762aae --- /dev/null +++ b/cpp/src/db/scheduler/TaskScheduler.cpp @@ -0,0 +1,117 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ + +#include "TaskScheduler.h" +#include "TaskDispatchQueue.h" +#include "utils/Log.h" +#include "utils/TimeRecorder.h" +#include "db/EngineFactory.h" + +namespace zilliz { +namespace milvus { +namespace engine { + +TaskScheduler::TaskScheduler() + : stopped_(true) { + Start(); +} + +TaskScheduler::~TaskScheduler() { + Stop(); +} + +TaskScheduler& TaskScheduler::GetInstance() { + static TaskScheduler s_instance; + return s_instance; +} + +bool +TaskScheduler::Start() { + if(!stopped_) { + return true; + } + + stopped_ = false; + + task_queue_.SetCapacity(2); + + task_dispatch_thread_ = std::make_shared(&TaskScheduler::TaskDispatchWorker, this); + task_thread_ = std::make_shared(&TaskScheduler::TaskWorker, this); + + return true; +} + +bool +TaskScheduler::Stop() { + if(stopped_) { + return true; + } + + if(task_dispatch_thread_) { + task_dispatch_queue_.Put(nullptr); + task_dispatch_thread_->join(); + task_dispatch_thread_ = nullptr; + } + + if(task_thread_) { + task_queue_.Put(nullptr); + task_thread_->join(); + task_thread_ = nullptr; + } + + stopped_ = true; + + return true; +} + +bool +TaskScheduler::Schedule(ScheduleContextPtr context) { + task_dispatch_queue_.Put(context); + + return true; +} + +bool +TaskScheduler::TaskDispatchWorker() { + while(true) { + ScheduleTaskPtr task_ptr = task_dispatch_queue_.Take(); + if(task_ptr == nullptr) { + SERVER_LOG_INFO << "Stop db task dispatch thread"; + break;//exit + } + + //execute task + ScheduleTaskPtr next_task = task_ptr->Execute(); + if(next_task != nullptr) { + task_queue_.Put(next_task); + } + } + + return true; +} + +bool +TaskScheduler::TaskWorker() { + while(true) { + ScheduleTaskPtr task_ptr = task_queue_.Take(); + if(task_ptr == nullptr) { + SERVER_LOG_INFO << "Stop db task thread"; + break;//exit + } + + //execute task + ScheduleTaskPtr next_task = task_ptr->Execute(); + if(next_task != nullptr) { + task_queue_.Put(next_task); + } + } + + return true; +} + +} +} +} \ No newline at end of file diff --git a/cpp/src/db/scheduler/TaskScheduler.h b/cpp/src/db/scheduler/TaskScheduler.h new file mode 100644 index 0000000000..d03fb858ac --- /dev/null +++ b/cpp/src/db/scheduler/TaskScheduler.h @@ -0,0 +1,49 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ +#pragma once + +#include "context/IScheduleContext.h" +#include "task/IScheduleTask.h" +#include "TaskDispatchQueue.h" +#include "utils/BlockingQueue.h" + +namespace zilliz { +namespace milvus { +namespace engine { + +class TaskScheduler { +private: + TaskScheduler(); + virtual ~TaskScheduler(); + +public: + static TaskScheduler& GetInstance(); + + bool Schedule(ScheduleContextPtr context); + +private: + bool Start(); + bool Stop(); + + bool TaskDispatchWorker(); + bool TaskWorker(); + +private: + std::shared_ptr task_dispatch_thread_; + std::shared_ptr task_thread_; + + TaskDispatchQueue task_dispatch_queue_; + + using TaskQueue = server::BlockingQueue; + TaskQueue task_queue_; + + bool stopped_ = true; +}; + + +} +} +} diff --git a/cpp/src/db/scheduler/context/DeleteContext.cpp b/cpp/src/db/scheduler/context/DeleteContext.cpp new file mode 100644 index 0000000000..bffeb9a134 --- /dev/null +++ b/cpp/src/db/scheduler/context/DeleteContext.cpp @@ -0,0 +1,22 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ + +#include "DeleteContext.h" + +namespace zilliz { +namespace milvus { +namespace engine { + +DeleteContext::DeleteContext(const std::string& table_id, meta::Meta::Ptr& meta_ptr) + : IScheduleContext(ScheduleContextType::kDelete), + table_id_(table_id), + meta_ptr_(meta_ptr) { + +} + +} +} +} \ No newline at end of file diff --git a/cpp/src/db/scheduler/context/DeleteContext.h b/cpp/src/db/scheduler/context/DeleteContext.h new file mode 100644 index 0000000000..b909ecb69e --- /dev/null +++ b/cpp/src/db/scheduler/context/DeleteContext.h @@ -0,0 +1,31 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ +#pragma once + +#include "IScheduleContext.h" +#include "db/Meta.h" + +namespace zilliz { +namespace milvus { +namespace engine { + +class DeleteContext : public IScheduleContext { +public: + DeleteContext(const std::string& table_id, meta::Meta::Ptr& meta_ptr); + + std::string table_id() const { return table_id_; } + meta::Meta::Ptr meta() const { return meta_ptr_; } + +private: + std::string table_id_; + meta::Meta::Ptr meta_ptr_; +}; + +using DeleteContextPtr = std::shared_ptr; + +} +} +} diff --git a/cpp/src/db/scheduler/context/IScheduleContext.h b/cpp/src/db/scheduler/context/IScheduleContext.h new file mode 100644 index 0000000000..6ae56e364e --- /dev/null +++ b/cpp/src/db/scheduler/context/IScheduleContext.h @@ -0,0 +1,38 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ +#pragma once + +#include + +namespace zilliz { +namespace milvus { +namespace engine { + +enum class ScheduleContextType { + kUnknown = 0, + kSearch, + kDelete, +}; + +class IScheduleContext { +public: + IScheduleContext(ScheduleContextType type) + : type_(type) { + } + + virtual ~IScheduleContext() = default; + + ScheduleContextType type() const { return type_; } + +protected: + ScheduleContextType type_; +}; + +using ScheduleContextPtr = std::shared_ptr; + +} +} +} diff --git a/cpp/src/db/scheduler/SearchContext.cpp b/cpp/src/db/scheduler/context/SearchContext.cpp similarity index 95% rename from cpp/src/db/scheduler/SearchContext.cpp rename to cpp/src/db/scheduler/context/SearchContext.cpp index 5bc27f06df..493005f05a 100644 --- a/cpp/src/db/scheduler/SearchContext.cpp +++ b/cpp/src/db/scheduler/context/SearchContext.cpp @@ -14,7 +14,8 @@ namespace milvus { namespace engine { SearchContext::SearchContext(uint64_t topk, uint64_t nq, const float* vectors) - : topk_(topk), + : IScheduleContext(ScheduleContextType::kSearch), + topk_(topk), nq_(nq), vectors_(vectors) { //use current time to identify this context diff --git a/cpp/src/db/scheduler/SearchContext.h b/cpp/src/db/scheduler/context/SearchContext.h similarity index 95% rename from cpp/src/db/scheduler/SearchContext.h rename to cpp/src/db/scheduler/context/SearchContext.h index 72cbfa4f99..1997b80764 100644 --- a/cpp/src/db/scheduler/SearchContext.h +++ b/cpp/src/db/scheduler/context/SearchContext.h @@ -5,6 +5,7 @@ ******************************************************************************/ #pragma once +#include "IScheduleContext.h" #include "db/MetaTypes.h" #include @@ -18,7 +19,7 @@ namespace engine { using TableFileSchemaPtr = std::shared_ptr; -class SearchContext { +class SearchContext : public IScheduleContext { public: SearchContext(uint64_t topk, uint64_t nq, const float* vectors); diff --git a/cpp/src/db/scheduler/task/DeleteTask.cpp b/cpp/src/db/scheduler/task/DeleteTask.cpp new file mode 100644 index 0000000000..a4660fc157 --- /dev/null +++ b/cpp/src/db/scheduler/task/DeleteTask.cpp @@ -0,0 +1,30 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ + +#include "DeleteTask.h" + +namespace zilliz { +namespace milvus { +namespace engine { + +DeleteTask::DeleteTask(const DeleteContextPtr& context) + : IScheduleTask(ScheduleTaskType::kDelete), + context_(context) { + +} + +std::shared_ptr DeleteTask::Execute() { + + if(context_ != nullptr && context_->meta() != nullptr) { + context_->meta()->DeleteTableFiles(context_->table_id()); + } + + return nullptr; +} + +} +} +} \ No newline at end of file diff --git a/cpp/src/db/scheduler/IScheduleStrategy.h b/cpp/src/db/scheduler/task/DeleteTask.h similarity index 57% rename from cpp/src/db/scheduler/IScheduleStrategy.h rename to cpp/src/db/scheduler/task/DeleteTask.h index a619c2a45e..4617a943bb 100644 --- a/cpp/src/db/scheduler/IScheduleStrategy.h +++ b/cpp/src/db/scheduler/task/DeleteTask.h @@ -5,22 +5,25 @@ ******************************************************************************/ #pragma once -#include "IndexLoaderQueue.h" -#include "SearchContext.h" +#include "IScheduleTask.h" +#include "db/scheduler/context/DeleteContext.h" namespace zilliz { namespace milvus { namespace engine { -class IScheduleStrategy { +class DeleteTask : public IScheduleTask { public: - virtual ~IScheduleStrategy() {} + DeleteTask(const DeleteContextPtr& context); - virtual bool Schedule(const SearchContextPtr &search_context, IndexLoaderQueue::LoaderQueue& loader_list) = 0; + virtual std::shared_ptr Execute() override; + +private: + DeleteContextPtr context_; }; -using ScheduleStrategyPtr = std::shared_ptr; +using DeleteTaskPtr = std::shared_ptr; } } -} \ No newline at end of file +} diff --git a/cpp/src/db/scheduler/task/IScheduleTask.h b/cpp/src/db/scheduler/task/IScheduleTask.h new file mode 100644 index 0000000000..652a1739fe --- /dev/null +++ b/cpp/src/db/scheduler/task/IScheduleTask.h @@ -0,0 +1,41 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ +#pragma once + +#include + +namespace zilliz { +namespace milvus { +namespace engine { + +enum class ScheduleTaskType { + kUnknown = 0, + kIndexLoad, + kSearch, + kDelete, +}; + +class IScheduleTask { +public: + IScheduleTask(ScheduleTaskType type) + : type_(type) { + } + + virtual ~IScheduleTask() = default; + + ScheduleTaskType type() const { return type_; } + + virtual std::shared_ptr Execute() = 0; + +protected: + ScheduleTaskType type_; +}; + +using ScheduleTaskPtr = std::shared_ptr; + +} +} +} diff --git a/cpp/src/db/scheduler/task/IndexLoadTask.cpp b/cpp/src/db/scheduler/task/IndexLoadTask.cpp new file mode 100644 index 0000000000..91f0e26577 --- /dev/null +++ b/cpp/src/db/scheduler/task/IndexLoadTask.cpp @@ -0,0 +1,72 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ + +#include "IndexLoadTask.h" +#include "SearchTask.h" +#include "db/Log.h" +#include "db/EngineFactory.h" +#include "utils/TimeRecorder.h" +#include "metrics/Metrics.h" + +namespace zilliz { +namespace milvus { +namespace engine { + +namespace { +void CollectFileMetrics(int file_type, size_t file_size) { + switch(file_type) { + case meta::TableFileSchema::RAW: + case meta::TableFileSchema::TO_INDEX: { + server::Metrics::GetInstance().RawFileSizeHistogramObserve(file_size); + server::Metrics::GetInstance().RawFileSizeTotalIncrement(file_size); + server::Metrics::GetInstance().RawFileSizeGaugeSet(file_size); + break; + } + default: { + server::Metrics::GetInstance().IndexFileSizeHistogramObserve(file_size); + server::Metrics::GetInstance().IndexFileSizeTotalIncrement(file_size); + server::Metrics::GetInstance().IndexFileSizeGaugeSet(file_size); + break; + } + } +} +} + +IndexLoadTask::IndexLoadTask() + : IScheduleTask(ScheduleTaskType::kIndexLoad) { + +} + +std::shared_ptr IndexLoadTask::Execute() { + ENGINE_LOG_INFO << "Loading index(" << file_->id_ << ") from location: " << file_->location_; + + server::TimeRecorder rc("Load index"); + //step 1: load index + ExecutionEnginePtr index_ptr = EngineFactory::Build(file_->dimension_, + file_->location_, + (EngineType)file_->engine_type_); + index_ptr->Load(); + + rc.Record("load index file to memory"); + + size_t file_size = index_ptr->PhysicalSize(); + LOG(DEBUG) << "Index file type " << file_->file_type_ << " Of Size: " + << file_size/(1024*1024) << " M"; + + CollectFileMetrics(file_->file_type_, file_size); + + //step 2: return search task for later execution + SearchTaskPtr task_ptr = std::make_shared(); + task_ptr->index_id_ = file_->id_; + task_ptr->index_type_ = file_->file_type_; + task_ptr->index_engine_ = index_ptr; + task_ptr->search_contexts_.swap(search_contexts_); + return std::static_pointer_cast(task_ptr); +} + +} +} +} \ No newline at end of file diff --git a/cpp/src/db/scheduler/task/IndexLoadTask.h b/cpp/src/db/scheduler/task/IndexLoadTask.h new file mode 100644 index 0000000000..c3548d7311 --- /dev/null +++ b/cpp/src/db/scheduler/task/IndexLoadTask.h @@ -0,0 +1,30 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ +#pragma once + +#include "IScheduleTask.h" +#include "db/scheduler/context/SearchContext.h" + +namespace zilliz { +namespace milvus { +namespace engine { + +class IndexLoadTask : public IScheduleTask { +public: + IndexLoadTask(); + + virtual std::shared_ptr Execute() override; + +public: + TableFileSchemaPtr file_; + std::vector search_contexts_; +}; + +using IndexLoadTaskPtr = std::shared_ptr; + +} +} +} diff --git a/cpp/src/db/scheduler/SearchTask.cpp b/cpp/src/db/scheduler/task/SearchTask.cpp similarity index 82% rename from cpp/src/db/scheduler/SearchTask.cpp rename to cpp/src/db/scheduler/task/SearchTask.cpp index 45503490d1..d8c372699b 100644 --- a/cpp/src/db/scheduler/SearchTask.cpp +++ b/cpp/src/db/scheduler/task/SearchTask.cpp @@ -4,6 +4,7 @@ * Proprietary and confidential. ******************************************************************************/ #include "SearchTask.h" +#include "metrics/Metrics.h" #include "utils/Log.h" #include "utils/TimeRecorder.h" @@ -110,15 +111,42 @@ void TopkResult(SearchContext::ResultSet &result_src, } } +void CollectDurationMetrics(int index_type, double total_time) { + switch(index_type) { + case meta::TableFileSchema::RAW: { + server::Metrics::GetInstance().SearchRawDataDurationSecondsHistogramObserve(total_time); + break; + } + case meta::TableFileSchema::TO_INDEX: { + server::Metrics::GetInstance().SearchRawDataDurationSecondsHistogramObserve(total_time); + break; + } + default: { + server::Metrics::GetInstance().SearchIndexDataDurationSecondsHistogramObserve(total_time); + break; + } + } } -bool SearchTask::DoSearch() { +} + +SearchTask::SearchTask() +: IScheduleTask(ScheduleTaskType::kSearch) { + +} + +std::shared_ptr SearchTask::Execute() { if(index_engine_ == nullptr) { - return false; + return nullptr; } + SERVER_LOG_INFO << "Searching in index(" << index_id_<< ") with " + << search_contexts_.size() << " tasks"; + server::TimeRecorder rc("DoSearch index(" + std::to_string(index_id_) + ")"); + auto start_time = METRICS_NOW_TIME; + std::vector output_ids; std::vector output_distence; for(auto& context : search_contexts_) { @@ -153,9 +181,13 @@ bool SearchTask::DoSearch() { context->IndexSearchDone(index_id_); } + auto end_time = METRICS_NOW_TIME; + auto total_time = METRICS_MICROSECONDS(start_time, end_time); + CollectDurationMetrics(index_type_, total_time); + rc.Elapse("totally cost"); - return true; + return nullptr; } } diff --git a/cpp/src/db/scheduler/SearchTask.h b/cpp/src/db/scheduler/task/SearchTask.h similarity index 77% rename from cpp/src/db/scheduler/SearchTask.h rename to cpp/src/db/scheduler/task/SearchTask.h index 3513c15a3e..0b3a236ce4 100644 --- a/cpp/src/db/scheduler/SearchTask.h +++ b/cpp/src/db/scheduler/task/SearchTask.h @@ -5,19 +5,19 @@ ******************************************************************************/ #pragma once -#include "SearchContext.h" -#include "utils/BlockingQueue.h" +#include "IScheduleTask.h" +#include "db/scheduler/context/SearchContext.h" #include "db/ExecutionEngine.h" -#include - namespace zilliz { namespace milvus { namespace engine { -class SearchTask { +class SearchTask : public IScheduleTask { public: - bool DoSearch(); + SearchTask(); + + virtual std::shared_ptr Execute() override; public: size_t index_id_ = 0; diff --git a/cpp/src/sdk/examples/simple/src/ClientTest.cpp b/cpp/src/sdk/examples/simple/src/ClientTest.cpp index b2ed9f3b25..3aad4e0789 100644 --- a/cpp/src/sdk/examples/simple/src/ClientTest.cpp +++ b/cpp/src/sdk/examples/simple/src/ClientTest.cpp @@ -20,7 +20,7 @@ namespace { static constexpr int64_t TOTAL_ROW_COUNT = 100000; static constexpr int64_t TOP_K = 10; static constexpr int64_t SEARCH_TARGET = 5000; //change this value, result is different - static constexpr int64_t ADD_VECTOR_LOOP = 2; + static constexpr int64_t ADD_VECTOR_LOOP = 10; #define BLOCK_SPLITER std::cout << "===========================================" << std::endl; @@ -195,10 +195,10 @@ ClientTest::Test(const std::string& address, const std::string& port) { PrintSearchResult(topk_query_result_array); } -// {//delete table -// Status stat = conn->DeleteTable(TABLE_NAME); -// std::cout << "DeleteTable function call status: " << stat.ToString() << std::endl; -// } + {//delete table + Status stat = conn->DeleteTable(TABLE_NAME); + std::cout << "DeleteTable function call status: " << stat.ToString() << std::endl; + } {//server status std::string status = conn->ServerStatus(); diff --git a/cpp/src/server/DBWrapper.cpp b/cpp/src/server/DBWrapper.cpp new file mode 100644 index 0000000000..4cae31ea6b --- /dev/null +++ b/cpp/src/server/DBWrapper.cpp @@ -0,0 +1,42 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ + +#include "DBWrapper.h" +#include "ServerConfig.h" +#include "utils/CommonUtil.h" +#include "utils/Log.h" + +namespace zilliz { +namespace milvus { +namespace server { + +DBWrapper::DBWrapper() { + zilliz::milvus::engine::Options opt; + ConfigNode& config = ServerConfig::GetInstance().GetConfig(CONFIG_DB); + opt.meta.backend_uri = config.GetValue(CONFIG_DB_URL); + std::string db_path = config.GetValue(CONFIG_DB_PATH); + opt.meta.path = db_path + "/db"; + int64_t index_size = config.GetInt64Value(CONFIG_DB_INDEX_TRIGGER_SIZE); + if(index_size > 0) {//ensure larger than zero, unit is MB + opt.index_trigger_size = (size_t)index_size * engine::ONE_MB; + } + + CommonUtil::CreateDirectory(opt.meta.path); + + zilliz::milvus::engine::DB::Open(opt, &db_); + if(db_ == nullptr) { + SERVER_LOG_ERROR << "Failed to open db"; + throw ServerException(SERVER_NULL_POINTER, "Failed to open db"); + } +} + +DBWrapper::~DBWrapper() { + delete db_; +} + +} +} +} \ No newline at end of file diff --git a/cpp/src/server/DBWrapper.h b/cpp/src/server/DBWrapper.h new file mode 100644 index 0000000000..5bd09bd0f4 --- /dev/null +++ b/cpp/src/server/DBWrapper.h @@ -0,0 +1,34 @@ +/******************************************************************************* + * Copyright 上海赜睿信息科技有限公司(Zilliz) - All Rights Reserved + * Unauthorized copying of this file, via any medium is strictly prohibited. + * Proprietary and confidential. + ******************************************************************************/ +#pragma once + +#include "db/DB.h" +#include "db/Meta.h" + +namespace zilliz { +namespace milvus { +namespace server { + +class DBWrapper { +private: + DBWrapper(); + ~DBWrapper(); + +public: + static zilliz::milvus::engine::DB* DB() { + static DBWrapper db_wrapper; + return db_wrapper.db(); + } + + zilliz::milvus::engine::DB* db() { return db_; } + +private: + zilliz::milvus::engine::DB* db_ = nullptr; +}; + +} +} +} diff --git a/cpp/src/server/MilvusServer.cpp b/cpp/src/server/MilvusServer.cpp index 2b46627ad0..f738c46711 100644 --- a/cpp/src/server/MilvusServer.cpp +++ b/cpp/src/server/MilvusServer.cpp @@ -7,6 +7,7 @@ #include "RequestHandler.h" #include "ServerConfig.h" #include "ThreadPoolServer.h" +#include "DBWrapper.h" #include "milvus_types.h" #include "milvus_constants.h" @@ -51,6 +52,8 @@ MilvusServer::StartService() { std::string mode = server_config.GetValue(CONFIG_SERVER_MODE, "thread_pool"); try { + DBWrapper::DB();//initialize db + stdcxx::shared_ptr handler(new RequestHandler()); stdcxx::shared_ptr processor(new MilvusServiceProcessor(handler)); stdcxx::shared_ptr server_transport(new TServerSocket(address, port)); diff --git a/cpp/src/server/RequestTask.cpp b/cpp/src/server/RequestTask.cpp index 3133e1cd80..dbe4e6a740 100644 --- a/cpp/src/server/RequestTask.cpp +++ b/cpp/src/server/RequestTask.cpp @@ -8,8 +8,7 @@ #include "utils/CommonUtil.h" #include "utils/Log.h" #include "utils/TimeRecorder.h" -#include "db/DB.h" -#include "db/Meta.h" +#include "DBWrapper.h" #include "version.h" namespace zilliz { @@ -26,43 +25,6 @@ using DB_META = zilliz::milvus::engine::meta::Meta; using DB_DATE = zilliz::milvus::engine::meta::DateT; namespace { - class DBWrapper { - public: - DBWrapper() { - zilliz::milvus::engine::Options opt; - ConfigNode& config = ServerConfig::GetInstance().GetConfig(CONFIG_DB); - opt.meta.backend_uri = config.GetValue(CONFIG_DB_URL); - std::string db_path = config.GetValue(CONFIG_DB_PATH); - opt.meta.path = db_path + "/db"; - int64_t index_size = config.GetInt64Value(CONFIG_DB_INDEX_TRIGGER_SIZE); - if(index_size > 0) {//ensure larger than zero, unit is MB - opt.index_trigger_size = (size_t)index_size * engine::ONE_MB; - } - - CommonUtil::CreateDirectory(opt.meta.path); - - zilliz::milvus::engine::DB::Open(opt, &db_); - if(db_ == nullptr) { - SERVER_LOG_ERROR << "Failed to open db"; - throw ServerException(SERVER_NULL_POINTER, "Failed to open db"); - } - } - - ~DBWrapper() { - delete db_; - } - - zilliz::milvus::engine::DB* DB() { return db_; } - - private: - zilliz::milvus::engine::DB* db_ = nullptr; - }; - - zilliz::milvus::engine::DB* DB() { - static DBWrapper db_wrapper; - return db_wrapper.DB(); - } - engine::EngineType EngineType(int type) { static std::map map_type = { {0, engine::EngineType::INVALID}, @@ -199,7 +161,7 @@ ServerError CreateTableTask::OnExecute() { table_info.store_raw_data_ = schema_.store_raw_vector; //step 3: create table - engine::Status stat = DB()->CreateTable(table_info); + engine::Status stat = DBWrapper::DB()->CreateTable(table_info); if(!stat.ok()) {//table could exist error_code_ = SERVER_UNEXPECTED_ERROR; error_msg_ = "Engine failed: " + stat.ToString(); @@ -246,7 +208,7 @@ ServerError DescribeTableTask::OnExecute() { //step 2: get table info engine::meta::TableSchema table_info; table_info.table_id_ = table_name_; - engine::Status stat = DB()->DescribeTable(table_info); + engine::Status stat = DBWrapper::DB()->DescribeTable(table_info); if(!stat.ok()) { error_code_ = SERVER_TABLE_NOT_EXIST; error_msg_ = "Engine failed: " + stat.ToString(); @@ -297,7 +259,7 @@ ServerError DeleteTableTask::OnExecute() { //step 2: check table existence engine::meta::TableSchema table_info; table_info.table_id_ = table_name_; - engine::Status stat = DB()->DescribeTable(table_info); + engine::Status stat = DBWrapper::DB()->DescribeTable(table_info); if(!stat.ok()) { error_code_ = SERVER_TABLE_NOT_EXIST; error_msg_ = "Engine failed: " + stat.ToString(); @@ -309,7 +271,7 @@ ServerError DeleteTableTask::OnExecute() { //step 3: delete table std::vector dates; - stat = DB()->DeleteTable(table_name_, dates); + stat = DBWrapper::DB()->DeleteTable(table_name_, dates); if(!stat.ok()) { SERVER_LOG_ERROR << "Engine failed: " << stat.ToString(); return SERVER_UNEXPECTED_ERROR; @@ -340,7 +302,7 @@ BaseTaskPtr ShowTablesTask::Create(std::vector& tables) { ServerError ShowTablesTask::OnExecute() { std::vector schema_array; - engine::Status stat = DB()->AllTables(schema_array); + engine::Status stat = DBWrapper::DB()->AllTables(schema_array); if(!stat.ok()) { error_code_ = SERVER_UNEXPECTED_ERROR; error_msg_ = "Engine failed: " + stat.ToString(); @@ -395,7 +357,7 @@ ServerError AddVectorTask::OnExecute() { //step 2: check table existence engine::meta::TableSchema table_info; table_info.table_id_ = table_name_; - engine::Status stat = DB()->DescribeTable(table_info); + engine::Status stat = DBWrapper::DB()->DescribeTable(table_info); if(!stat.ok()) { error_code_ = SERVER_TABLE_NOT_EXIST; error_msg_ = "Engine failed: " + stat.ToString(); @@ -417,7 +379,7 @@ ServerError AddVectorTask::OnExecute() { //step 4: insert vectors uint64_t vec_count = (uint64_t)record_array_.size(); - stat = DB()->InsertVectors(table_name_, vec_count, vec_f.data(), record_ids_); + stat = DBWrapper::DB()->InsertVectors(table_name_, vec_count, vec_f.data(), record_ids_); rc.Record("add vectors to engine"); if(!stat.ok()) { error_code_ = SERVER_UNEXPECTED_ERROR; @@ -493,7 +455,7 @@ ServerError SearchVectorTask::OnExecute() { //step 2: check table existence engine::meta::TableSchema table_info; table_info.table_id_ = table_name_; - engine::Status stat = DB()->DescribeTable(table_info); + engine::Status stat = DBWrapper::DB()->DescribeTable(table_info); if(!stat.ok()) { error_code_ = SERVER_TABLE_NOT_EXIST; error_msg_ = "Engine failed: " + stat.ToString(); @@ -526,9 +488,9 @@ ServerError SearchVectorTask::OnExecute() { uint64_t record_count = (uint64_t)record_array_.size(); if(file_id_array_.empty()) { - stat = DB()->Query(table_name_, (size_t) top_k_, record_count, vec_f.data(), dates, results); + stat = DBWrapper::DB()->Query(table_name_, (size_t) top_k_, record_count, vec_f.data(), dates, results); } else { - stat = DB()->Query(table_name_, file_id_array_, (size_t) top_k_, record_count, vec_f.data(), dates, results); + stat = DBWrapper::DB()->Query(table_name_, file_id_array_, (size_t) top_k_, record_count, vec_f.data(), dates, results); } rc.Record("search vectors from engine"); @@ -599,7 +561,7 @@ ServerError GetTableRowCountTask::OnExecute() { //step 2: get row count uint64_t row_count = 0; - engine::Status stat = DB()->GetTableRowCount(table_name_, row_count); + engine::Status stat = DBWrapper::DB()->GetTableRowCount(table_name_, row_count); if (!stat.ok()) { error_code_ = SERVER_UNEXPECTED_ERROR; error_msg_ = "Engine failed: " + stat.ToString(); diff --git a/cpp/unittest/db/CMakeLists.txt b/cpp/unittest/db/CMakeLists.txt index 67ef5fccec..df47f14c65 100644 --- a/cpp/unittest/db/CMakeLists.txt +++ b/cpp/unittest/db/CMakeLists.txt @@ -4,11 +4,19 @@ # Proprietary and confidential. #------------------------------------------------------------------------------- aux_source_directory(${MILVUS_ENGINE_SRC}/db db_srcs) -aux_source_directory(${MILVUS_ENGINE_SRC}/db/scheduler db_scheduler_srcs) aux_source_directory(${MILVUS_ENGINE_SRC}/config config_files) aux_source_directory(${MILVUS_ENGINE_SRC}/cache cache_srcs) aux_source_directory(${MILVUS_ENGINE_SRC}/wrapper wrapper_src) +aux_source_directory(${MILVUS_ENGINE_SRC}/db/scheduler scheduler_files) +aux_source_directory(${MILVUS_ENGINE_SRC}/db/scheduler/context scheduler_context_files) +aux_source_directory(${MILVUS_ENGINE_SRC}/db/scheduler/task scheduler_task_files) +set(db_scheduler_srcs + ${scheduler_files} + ${scheduler_context_files} + ${scheduler_task_files} + ) + include_directories(/usr/local/cuda/include) link_directories("/usr/local/cuda/lib64") diff --git a/cpp/unittest/metrics/CMakeLists.txt b/cpp/unittest/metrics/CMakeLists.txt index 2f07cbed24..9ae711ce25 100644 --- a/cpp/unittest/metrics/CMakeLists.txt +++ b/cpp/unittest/metrics/CMakeLists.txt @@ -13,12 +13,19 @@ include_directories(../../src) aux_source_directory(../../src/db db_srcs) -aux_source_directory(../../src/db/scheduler db_scheduler_srcs) aux_source_directory(../../src/config config_files) aux_source_directory(../../src/cache cache_srcs) aux_source_directory(../../src/wrapper wrapper_src) aux_source_directory(../../src/metrics metrics_src) +aux_source_directory(${MILVUS_ENGINE_SRC}/db/scheduler scheduler_files) +aux_source_directory(${MILVUS_ENGINE_SRC}/db/scheduler/context scheduler_context_files) +aux_source_directory(${MILVUS_ENGINE_SRC}/db/scheduler/task scheduler_task_files) +set(db_scheduler_srcs + ${scheduler_files} + ${scheduler_context_files} + ${scheduler_task_files} + ) include_directories(/usr/include) include_directories(../../third_party/build/include) From b7bc62d454c308d9972e5c184788a58a159fae2c Mon Sep 17 00:00:00 2001 From: groot Date: Fri, 21 Jun 2019 10:32:46 +0800 Subject: [PATCH 4/5] refine code Former-commit-id: 17c00857221bc167525f7c340c99061697c0c547 --- cpp/src/db/DBImpl.cpp | 2 +- cpp/src/db/scheduler/task/SearchTask.cpp | 4 +- .../sdk/examples/simple/src/ClientTest.cpp | 23 +++++--- cpp/unittest/db/db_tests.cpp | 59 +++++++++---------- 4 files changed, 45 insertions(+), 43 deletions(-) diff --git a/cpp/src/db/DBImpl.cpp b/cpp/src/db/DBImpl.cpp index 8202c8a7db..7642ea37a7 100644 --- a/cpp/src/db/DBImpl.cpp +++ b/cpp/src/db/DBImpl.cpp @@ -652,7 +652,7 @@ Status DBImpl::BuildIndex(const meta::TableFileSchema& file) { << index->PhysicalSize()/(1024*1024) << " M" << " from file " << to_remove.file_id_; - //index->Cache(); + index->Cache(); } catch (std::exception& ex) { return Status::Error("Build index encounter exception", ex.what()); diff --git a/cpp/src/db/scheduler/task/SearchTask.cpp b/cpp/src/db/scheduler/task/SearchTask.cpp index d8c372699b..d04f270331 100644 --- a/cpp/src/db/scheduler/task/SearchTask.cpp +++ b/cpp/src/db/scheduler/task/SearchTask.cpp @@ -55,7 +55,7 @@ void MergeResult(SearchContext::Id2ScoreMap &score_src, while(true) { //all score_src items are merged, if score_merged.size() still less than topk //move items from score_target to score_merged until score_merged.size() equal topk - if(src_index >= src_count - 1) { + if(src_index >= src_count) { for(size_t i = target_index; i < target_count && score_merged.size() < topk; ++i) { score_merged.push_back(score_target[i]); } @@ -64,7 +64,7 @@ void MergeResult(SearchContext::Id2ScoreMap &score_src, //all score_target items are merged, if score_merged.size() still less than topk //move items from score_src to score_merged until score_merged.size() equal topk - if(target_index >= target_count - 1) { + if(target_index >= target_count) { for(size_t i = src_index; i < src_count && score_merged.size() < topk; ++i) { score_merged.push_back(score_src[i]); } diff --git a/cpp/src/sdk/examples/simple/src/ClientTest.cpp b/cpp/src/sdk/examples/simple/src/ClientTest.cpp index 3aad4e0789..78145446a6 100644 --- a/cpp/src/sdk/examples/simple/src/ClientTest.cpp +++ b/cpp/src/sdk/examples/simple/src/ClientTest.cpp @@ -17,10 +17,11 @@ namespace { static const std::string TABLE_NAME = GetTableName(); static constexpr int64_t TABLE_DIMENSION = 512; - static constexpr int64_t TOTAL_ROW_COUNT = 100000; + static constexpr int64_t BATCH_ROW_COUNT = 100000; + static constexpr int64_t NQ = 10; static constexpr int64_t TOP_K = 10; static constexpr int64_t SEARCH_TARGET = 5000; //change this value, result is different - static constexpr int64_t ADD_VECTOR_LOOP = 10; + static constexpr int64_t ADD_VECTOR_LOOP = 5; #define BLOCK_SPLITER std::cout << "===========================================" << std::endl; @@ -96,7 +97,7 @@ namespace { TableSchema BuildTableSchema() { TableSchema tb_schema; tb_schema.table_name = TABLE_NAME; - tb_schema.index_type = IndexType::gpu_ivfflat; + tb_schema.index_type = IndexType::cpu_idmap; tb_schema.dimension = TABLE_DIMENSION; tb_schema.store_raw_vector = true; @@ -110,17 +111,21 @@ namespace { } vector_record_array.clear(); - for (int64_t k = from; k < to; k++) { RowRecord record; record.data.resize(TABLE_DIMENSION); for(int64_t i = 0; i < TABLE_DIMENSION; i++) { - record.data[i] = (float)(i + k); + record.data[i] = (float)(k%(i+1)); } vector_record_array.emplace_back(record); } } + + void Sleep(int seconds) { + std::cout << "Waiting " << seconds << " seconds ..." << std::endl; + sleep(seconds); + } } void @@ -171,7 +176,7 @@ ClientTest::Test(const std::string& address, const std::string& port) { for(int i = 0; i < ADD_VECTOR_LOOP; i++){//add vectors std::vector record_array; - BuildVectors(i*TOTAL_ROW_COUNT, (i+1)*TOTAL_ROW_COUNT, record_array); + BuildVectors(i*BATCH_ROW_COUNT, (i+1)*BATCH_ROW_COUNT, record_array); std::vector record_ids; Status stat = conn->AddVector(TABLE_NAME, record_array, record_ids); std::cout << "AddVector function call status: " << stat.ToString() << std::endl; @@ -179,10 +184,10 @@ ClientTest::Test(const std::string& address, const std::string& port) { } {//search vectors - std::cout << "Waiting data persist. Sleep 1 seconds ..." << std::endl; - sleep(1); + Sleep(2); + std::vector record_array; - BuildVectors(SEARCH_TARGET, SEARCH_TARGET + 10, record_array); + BuildVectors(SEARCH_TARGET, SEARCH_TARGET + NQ, record_array); std::vector query_range_array; Range rg; diff --git a/cpp/unittest/db/db_tests.cpp b/cpp/unittest/db/db_tests.cpp index 00b6c7a9c1..6cfbe9139e 100644 --- a/cpp/unittest/db/db_tests.cpp +++ b/cpp/unittest/db/db_tests.cpp @@ -69,7 +69,7 @@ TEST_F(DBTest2, ARHIVE_DISK_CHECK) { engine::meta::TableSchema group_info; group_info.dimension_ = group_dim; group_info.table_id_ = group_name; - group_info.engine_type_ = (int)engine::EngineType::FAISS_IVFFLAT; + group_info.engine_type_ = (int)engine::EngineType::FAISS_IDMAP; engine::Status stat = db_->CreateTable(group_info); engine::meta::TableSchema group_info_get; @@ -101,30 +101,27 @@ TEST_F(DBTest2, ARHIVE_DISK_CHECK) { db_->Size(size); LOG(DEBUG) << "size=" << size; - ASSERT_TRUE(size < 1 * engine::meta::G); + ASSERT_LT(size, 1 * engine::meta::G); delete [] xb; }; TEST_F(DBTest, DB_TEST) { + static const std::string table_name = "test_group"; + static const int table_dim = 256; + engine::meta::TableSchema table_info; + table_info.dimension_ = table_dim; + table_info.table_id_ = table_name; + table_info.engine_type_ = (int)engine::EngineType::FAISS_IDMAP; + engine::Status stat = db_->CreateTable(table_info); - - static const std::string group_name = "test_group"; - static const int group_dim = 256; - - engine::meta::TableSchema group_info; - group_info.dimension_ = group_dim; - group_info.table_id_ = group_name; - group_info.engine_type_ = (int)engine::EngineType::FAISS_IVFFLAT; - engine::Status stat = db_->CreateTable(group_info); - - engine::meta::TableSchema group_info_get; - group_info_get.table_id_ = group_name; - stat = db_->DescribeTable(group_info_get); + engine::meta::TableSchema table_info_get; + table_info_get.table_id_ = table_name; + stat = db_->DescribeTable(table_info_get); ASSERT_STATS(stat); - ASSERT_EQ(group_info_get.dimension_, group_dim); + ASSERT_EQ(table_info_get.dimension_, table_dim); engine::IDNumbers vector_ids; engine::IDNumbers target_ids; @@ -160,7 +157,7 @@ TEST_F(DBTest, DB_TEST) { prev_count = count; START_TIMER; - stat = db_->Query(group_name, k, qb, qxb, results); + stat = db_->Query(table_name, k, qb, qxb, results); ss << "Search " << j << " With Size " << count/engine::meta::M << " M"; STOP_TIMER(ss.str()); @@ -183,10 +180,10 @@ TEST_F(DBTest, DB_TEST) { for (auto i=0; iInsertVectors(group_name, qb, qxb, target_ids); + db_->InsertVectors(table_name, qb, qxb, target_ids); ASSERT_EQ(target_ids.size(), qb); } else { - db_->InsertVectors(group_name, nb, xb, vector_ids); + db_->InsertVectors(table_name, nb, xb, vector_ids); } std::this_thread::sleep_for(std::chrono::microseconds(1)); } @@ -198,20 +195,20 @@ TEST_F(DBTest, DB_TEST) { }; TEST_F(DBTest, SEARCH_TEST) { - static const std::string group_name = "test_group"; + static const std::string table_name = "test_group"; static const int group_dim = 256; - engine::meta::TableSchema group_info; - group_info.dimension_ = group_dim; - group_info.table_id_ = group_name; - group_info.engine_type_ = (int)engine::EngineType::FAISS_IVFFLAT; - engine::Status stat = db_->CreateTable(group_info); + engine::meta::TableSchema table_info; + table_info.dimension_ = group_dim; + table_info.table_id_ = table_name; + table_info.engine_type_ = (int)engine::EngineType::FAISS_IDMAP; + engine::Status stat = db_->CreateTable(table_info); - engine::meta::TableSchema group_info_get; - group_info_get.table_id_ = group_name; - stat = db_->DescribeTable(group_info_get); + engine::meta::TableSchema table_info_get; + table_info_get.table_id_ = table_name; + stat = db_->DescribeTable(table_info_get); ASSERT_STATS(stat); - ASSERT_EQ(group_info_get.dimension_, group_dim); + ASSERT_EQ(table_info_get.dimension_, group_dim); // prepare raw data size_t nb = 250000; @@ -243,7 +240,7 @@ TEST_F(DBTest, SEARCH_TEST) { // insert data const int batch_size = 100; for (int j = 0; j < nb / batch_size; ++j) { - stat = db_->InsertVectors(group_name, batch_size, xb.data()+batch_size*j*group_dim, ids); + stat = db_->InsertVectors(table_name, batch_size, xb.data()+batch_size*j*group_dim, ids); if (j == 200){ sleep(1);} ASSERT_STATS(stat); } @@ -251,7 +248,7 @@ TEST_F(DBTest, SEARCH_TEST) { sleep(2); // wait until build index finish engine::QueryResults results; - stat = db_->Query(group_name, k, nq, xq.data(), results); + stat = db_->Query(table_name, k, nq, xq.data(), results); ASSERT_STATS(stat); // TODO(linxj): add groundTruth assert From 782c7ada913a516c5c78e5b5db0d5ae31f4b840b Mon Sep 17 00:00:00 2001 From: groot Date: Fri, 21 Jun 2019 11:32:43 +0800 Subject: [PATCH 5/5] add usittest for delete table Former-commit-id: 5219a38c73a349c3f1f8e79d7fee0c4044a81c6c --- cpp/src/db/MemManager.cpp | 2 +- cpp/unittest/db/db_tests.cpp | 192 ++++++++++++++++++++--------------- 2 files changed, 113 insertions(+), 81 deletions(-) diff --git a/cpp/src/db/MemManager.cpp b/cpp/src/db/MemManager.cpp index 31b2070b49..d7a2087ee0 100644 --- a/cpp/src/db/MemManager.cpp +++ b/cpp/src/db/MemManager.cpp @@ -62,7 +62,7 @@ Status MemVectors::Serialize(std::string& table_id) { auto status = pMeta_->UpdateTableFile(schema_); LOG(DEBUG) << "New " << ((schema_.file_type_ == meta::TableFileSchema::RAW) ? "raw" : "to_index") - << " file " << schema_.file_id_ << " of size " << pEE_->Size() / meta::M << " M"; + << " file " << schema_.file_id_ << " of size " << (double)(pEE_->Size()) / (double)meta::M << " M"; pEE_->Cache(); diff --git a/cpp/unittest/db/db_tests.cpp b/cpp/unittest/db/db_tests.cpp index 6cfbe9139e..aa311550ee 100644 --- a/cpp/unittest/db/db_tests.cpp +++ b/cpp/unittest/db/db_tests.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include "utils.h" #include "db/DB.h" @@ -14,6 +15,31 @@ using namespace zilliz::milvus; +namespace { + +static const std::string TABLE_NAME = "test_group"; +static constexpr int64_t TABLE_DIM = 256; + +engine::meta::TableSchema BuildTableSchema() { + engine::meta::TableSchema table_info; + table_info.dimension_ = TABLE_DIM; + table_info.table_id_ = TABLE_NAME; + table_info.engine_type_ = (int)engine::EngineType::FAISS_IDMAP; + return table_info; +} + +void BuildVectors(int64_t n, std::vector& vectors) { + vectors.clear(); + vectors.resize(n*TABLE_DIM); + float* data = vectors.data(); + for(int i = 0; i < n; i++) { + for(int j = 0; j < TABLE_DIM; j++) data[TABLE_DIM * i + j] = drand48(); + data[TABLE_DIM * i] += i / 2000.; + } +} + +} + TEST_F(DBTest, CONFIG_TEST) { { ASSERT_ANY_THROW(engine::ArchiveConf conf("wrong")); @@ -60,52 +86,6 @@ TEST_F(DBTest, CONFIG_TEST) { } } -TEST_F(DBTest2, ARHIVE_DISK_CHECK) { - - static const std::string group_name = "test_group"; - static const int group_dim = 256; - uint64_t size; - - engine::meta::TableSchema group_info; - group_info.dimension_ = group_dim; - group_info.table_id_ = group_name; - group_info.engine_type_ = (int)engine::EngineType::FAISS_IDMAP; - engine::Status stat = db_->CreateTable(group_info); - - engine::meta::TableSchema group_info_get; - group_info_get.table_id_ = group_name; - stat = db_->DescribeTable(group_info_get); - ASSERT_STATS(stat); - ASSERT_EQ(group_info_get.dimension_, group_dim); - - engine::IDNumbers vector_ids; - engine::IDNumbers target_ids; - - db_->Size(size); - int d = 256; - int nb = 20; - float *xb = new float[d * nb]; - for(int i = 0; i < nb; i++) { - for(int j = 0; j < d; j++) xb[d * i + j] = drand48(); - xb[d * i] += i / 2000.; - } - - int loop = 100000; - - for (auto i=0; iInsertVectors(group_name, nb, xb, vector_ids); - std::this_thread::sleep_for(std::chrono::microseconds(1)); - } - - std::this_thread::sleep_for(std::chrono::seconds(1)); - - db_->Size(size); - LOG(DEBUG) << "size=" << size; - ASSERT_LT(size, 1 * engine::meta::G); - - delete [] xb; -}; - TEST_F(DBTest, DB_TEST) { static const std::string table_name = "test_group"; @@ -126,20 +106,13 @@ TEST_F(DBTest, DB_TEST) { engine::IDNumbers vector_ids; engine::IDNumbers target_ids; - int d = 256; - int nb = 50; - float *xb = new float[d * nb]; - for(int i = 0; i < nb; i++) { - for(int j = 0; j < d; j++) xb[d * i + j] = drand48(); - xb[d * i] += i / 2000.; - } + int64_t nb = 50; + std::vector xb; + BuildVectors(nb, xb); - int qb = 5; - float *qxb = new float[d * qb]; - for(int i = 0; i < qb; i++) { - for(int j = 0; j < d; j++) qxb[d * i + j] = drand48(); - qxb[d * i] += i / 2000.; - } + int64_t qb = 5; + std::vector qxb; + BuildVectors(qb, qxb); std::thread search([&]() { engine::QueryResults results; @@ -157,7 +130,7 @@ TEST_F(DBTest, DB_TEST) { prev_count = count; START_TIMER; - stat = db_->Query(table_name, k, qb, qxb, results); + stat = db_->Query(table_name, k, qb, qxb.data(), results); ss << "Search " << j << " With Size " << count/engine::meta::M << " M"; STOP_TIMER(ss.str()); @@ -180,54 +153,45 @@ TEST_F(DBTest, DB_TEST) { for (auto i=0; iInsertVectors(table_name, qb, qxb, target_ids); + db_->InsertVectors(table_name, qb, qxb.data(), target_ids); ASSERT_EQ(target_ids.size(), qb); } else { - db_->InsertVectors(table_name, nb, xb, vector_ids); + db_->InsertVectors(table_name, nb, xb.data(), vector_ids); } std::this_thread::sleep_for(std::chrono::microseconds(1)); } search.join(); - - delete [] xb; - delete [] qxb; }; TEST_F(DBTest, SEARCH_TEST) { - static const std::string table_name = "test_group"; - static const int group_dim = 256; - - engine::meta::TableSchema table_info; - table_info.dimension_ = group_dim; - table_info.table_id_ = table_name; - table_info.engine_type_ = (int)engine::EngineType::FAISS_IDMAP; + engine::meta::TableSchema table_info = BuildTableSchema(); engine::Status stat = db_->CreateTable(table_info); engine::meta::TableSchema table_info_get; - table_info_get.table_id_ = table_name; + table_info_get.table_id_ = TABLE_NAME; stat = db_->DescribeTable(table_info_get); ASSERT_STATS(stat); - ASSERT_EQ(table_info_get.dimension_, group_dim); + ASSERT_EQ(table_info_get.dimension_, TABLE_DIM); // prepare raw data size_t nb = 250000; size_t nq = 10; size_t k = 5; - std::vector xb(nb*group_dim); - std::vector xq(nq*group_dim); + std::vector xb(nb*TABLE_DIM); + std::vector xq(nq*TABLE_DIM); std::vector ids(nb); std::random_device rd; std::mt19937 gen(rd()); std::uniform_real_distribution<> dis_xt(-1.0, 1.0); - for (size_t i = 0; i < nb*group_dim; i++) { + for (size_t i = 0; i < nb*TABLE_DIM; i++) { xb[i] = dis_xt(gen); if (i < nb){ ids[i] = i; } } - for (size_t i = 0; i < nq*group_dim; i++) { + for (size_t i = 0; i < nq*TABLE_DIM; i++) { xq[i] = dis_xt(gen); } @@ -240,7 +204,7 @@ TEST_F(DBTest, SEARCH_TEST) { // insert data const int batch_size = 100; for (int j = 0; j < nb / batch_size; ++j) { - stat = db_->InsertVectors(table_name, batch_size, xb.data()+batch_size*j*group_dim, ids); + stat = db_->InsertVectors(TABLE_NAME, batch_size, xb.data()+batch_size*j*TABLE_DIM, ids); if (j == 200){ sleep(1);} ASSERT_STATS(stat); } @@ -248,9 +212,77 @@ TEST_F(DBTest, SEARCH_TEST) { sleep(2); // wait until build index finish engine::QueryResults results; - stat = db_->Query(table_name, k, nq, xq.data(), results); + stat = db_->Query(TABLE_NAME, k, nq, xq.data(), results); ASSERT_STATS(stat); // TODO(linxj): add groundTruth assert }; +TEST_F(DBTest2, ARHIVE_DISK_CHECK) { + + engine::meta::TableSchema table_info = BuildTableSchema(); + engine::Status stat = db_->CreateTable(table_info); + + engine::meta::TableSchema table_info_get; + table_info_get.table_id_ = TABLE_NAME; + stat = db_->DescribeTable(table_info_get); + ASSERT_STATS(stat); + ASSERT_EQ(table_info_get.dimension_, TABLE_DIM); + + engine::IDNumbers vector_ids; + engine::IDNumbers target_ids; + + uint64_t size; + db_->Size(size); + + int64_t nb = 10; + std::vector xb; + BuildVectors(nb, xb); + + int loop = 100000; + for (auto i=0; iInsertVectors(TABLE_NAME, nb, xb.data(), vector_ids); + std::this_thread::sleep_for(std::chrono::microseconds(1)); + } + + std::this_thread::sleep_for(std::chrono::seconds(1)); + + db_->Size(size); + LOG(DEBUG) << "size=" << size; + ASSERT_LE(size, 1 * engine::meta::G); +}; + +TEST_F(DBTest2, DELETE_TEST) { + + + engine::meta::TableSchema table_info = BuildTableSchema(); + engine::Status stat = db_->CreateTable(table_info); + + engine::meta::TableSchema table_info_get; + table_info_get.table_id_ = TABLE_NAME; + stat = db_->DescribeTable(table_info_get); + ASSERT_STATS(stat); + + ASSERT_TRUE(boost::filesystem::exists(table_info_get.location_)); + + engine::IDNumbers vector_ids; + + uint64_t size; + db_->Size(size); + + int64_t nb = 100000; + std::vector xb; + BuildVectors(nb, xb); + + int loop = 20; + for (auto i=0; iInsertVectors(TABLE_NAME, nb, xb.data(), vector_ids); + std::this_thread::sleep_for(std::chrono::microseconds(1)); + } + + std::vector dates; + stat = db_->DeleteTable(TABLE_NAME, dates); + std::this_thread::sleep_for(std::chrono::seconds(2)); + ASSERT_TRUE(stat.ok()); + ASSERT_FALSE(boost::filesystem::exists(table_info_get.location_)); +};