提交 54b85185 编写于 作者: S starlord

refine error code


Former-commit-id: 2919dc7e39aa970522bd99d553bd0f92fcee9345
上级 ce7de094
......@@ -30,7 +30,7 @@ class ConfigMgr {
public:
static ConfigMgr* GetInstance();
virtual ServerError LoadConfigFile(const std::string &filename) = 0;
virtual ErrorCode LoadConfigFile(const std::string &filename) = 0;
virtual void Print() const = 0;//will be deleted
virtual std::string DumpString() const = 0;
......
......@@ -12,7 +12,7 @@ namespace zilliz {
namespace milvus {
namespace server {
ServerError YamlConfigMgr::LoadConfigFile(const std::string &filename) {
ErrorCode YamlConfigMgr::LoadConfigFile(const std::string &filename) {
struct stat directoryStat;
int statOK = stat(filename.c_str(), &directoryStat);
if (statOK != 0) {
......
......@@ -17,7 +17,7 @@ namespace server {
class YamlConfigMgr : public ConfigMgr {
public:
virtual ServerError LoadConfigFile(const std::string &filename);
virtual ErrorCode LoadConfigFile(const std::string &filename);
virtual void Print() const;
virtual std::string DumpString() const;
......
......@@ -41,7 +41,7 @@ ExecutionEngineImpl::ExecutionEngineImpl(uint16_t dimension,
build_cfg["metric_type"] = (metric_type_ == MetricType::IP) ? "IP" : "L2";
AutoGenParams(index_->GetType(), 0, build_cfg);
auto ec = std::static_pointer_cast<BFIndex>(index_)->Build(build_cfg);
if (ec != server::KNOWHERE_SUCCESS) { throw Exception("Build index error"); }
if (ec != KNOWHERE_SUCCESS) { throw Exception("Build index error"); }
}
ExecutionEngineImpl::ExecutionEngineImpl(VecIndexPtr index,
......@@ -85,7 +85,7 @@ VecIndexPtr ExecutionEngineImpl::CreatetVecIndex(EngineType type) {
Status ExecutionEngineImpl::AddWithIds(long n, const float *xdata, const long *xids) {
auto ec = index_->Add(n, xdata, xids);
if (ec != server::KNOWHERE_SUCCESS) {
if (ec != KNOWHERE_SUCCESS) {
return Status::Error("Add error");
}
return Status::OK();
......@@ -117,7 +117,7 @@ size_t ExecutionEngineImpl::PhysicalSize() const {
Status ExecutionEngineImpl::Serialize() {
auto ec = write_index(index_, location_);
if (ec != server::KNOWHERE_SUCCESS) {
if (ec != KNOWHERE_SUCCESS) {
return Status::Error("Serialize: write to disk error");
}
return Status::OK();
......@@ -248,7 +248,7 @@ Status ExecutionEngineImpl::Merge(const std::string &location) {
if (auto file_index = std::dynamic_pointer_cast<BFIndex>(to_merge)) {
auto ec = index_->Add(file_index->Count(), file_index->GetRawVectors(), file_index->GetRawIds());
if (ec != server::KNOWHERE_SUCCESS) {
if (ec != KNOWHERE_SUCCESS) {
ENGINE_LOG_ERROR << "Merge: Add Error";
return Status::Error("Merge: Add Error");
}
......@@ -284,7 +284,7 @@ ExecutionEngineImpl::BuildIndex(const std::string &location, EngineType engine_t
from_index->GetRawVectors(),
from_index->GetRawIds(),
build_cfg);
if (ec != server::KNOWHERE_SUCCESS) { throw Exception("Build index error"); }
if (ec != KNOWHERE_SUCCESS) { throw Exception("Build index error"); }
return std::make_shared<ExecutionEngineImpl>(to_index, location, engine_type, metric_type_, nlist_);
}
......@@ -303,7 +303,7 @@ Status ExecutionEngineImpl::Search(long n,
ENGINE_LOG_DEBUG << "Search Params: [k] " << k << " [nprobe] " << nprobe;
auto cfg = Config::object{{"k", k}, {"nprobe", nprobe}};
auto ec = index_->Search(n, data, distances, labels, cfg);
if (ec != server::KNOWHERE_SUCCESS) {
if (ec != KNOWHERE_SUCCESS) {
ENGINE_LOG_ERROR << "Search error";
return Status::Error("Search: Search Error");
}
......
......@@ -654,8 +654,12 @@ Status SqliteMetaImpl::FilesToSearch(const std::string &table_id,
auto match_tableid = c(&TableFileSchema::table_id_) == table_id;
std::vector<int> file_type = {(int) TableFileSchema::RAW, (int) TableFileSchema::TO_INDEX, (int) TableFileSchema::INDEX};
auto match_type = in(&TableFileSchema::file_type_, file_type);
std::vector<int> file_types = {
(int) TableFileSchema::RAW,
(int) TableFileSchema::TO_INDEX,
(int) TableFileSchema::INDEX
};
auto match_type = in(&TableFileSchema::file_type_, file_types);
TableSchema table_schema;
table_schema.table_id_ = table_id;
......@@ -1152,8 +1156,12 @@ Status SqliteMetaImpl::CleanUp() {
//multi-threads call sqlite update may get exception('bad logic', etc), so we add a lock here
std::lock_guard<std::mutex> meta_lock(meta_mutex_);
std::vector<int> file_type = {(int) TableFileSchema::NEW, (int) TableFileSchema::NEW_INDEX, (int) TableFileSchema::NEW_MERGE};
auto files = ConnectorPtr->select(columns(&TableFileSchema::id_), where(in(&TableFileSchema::file_type_, file_type)));
std::vector<int> file_types = {
(int) TableFileSchema::NEW,
(int) TableFileSchema::NEW_INDEX,
(int) TableFileSchema::NEW_MERGE
};
auto files = ConnectorPtr->select(columns(&TableFileSchema::id_), where(in(&TableFileSchema::file_type_, file_types)));
auto commited = ConnectorPtr->transaction([&]() mutable {
for (auto &file : files) {
......@@ -1180,9 +1188,13 @@ Status SqliteMetaImpl::Count(const std::string &table_id, uint64_t &result) {
try {
server::MetricCollector metric;
std::vector<int> file_type = {(int) TableFileSchema::RAW, (int) TableFileSchema::TO_INDEX, (int) TableFileSchema::INDEX};
std::vector<int> file_types = {
(int) TableFileSchema::RAW,
(int) TableFileSchema::TO_INDEX,
(int) TableFileSchema::INDEX
};
auto selected = ConnectorPtr->select(columns(&TableFileSchema::row_count_),
where(in(&TableFileSchema::file_type_, file_type)
where(in(&TableFileSchema::file_type_, file_types)
and c(&TableFileSchema::table_id_) == table_id));
TableSchema table_schema;
......
......@@ -53,7 +53,7 @@ TaskDispatchQueue::Front() {
if (queue_.empty()) {
std::string error_msg = "blocking queue empty";
SERVER_LOG_ERROR << error_msg;
throw server::ServerException(server::SERVER_BLOCKING_QUEUE_EMPTY, error_msg);
throw server::ServerException(SERVER_BLOCKING_QUEUE_EMPTY, error_msg);
}
ScheduleTaskPtr front(queue_.front());
return front;
......@@ -67,7 +67,7 @@ TaskDispatchQueue::Back() {
if (queue_.empty()) {
std::string error_msg = "blocking queue empty";
SERVER_LOG_ERROR << error_msg;
throw server::ServerException(server::SERVER_BLOCKING_QUEUE_EMPTY, error_msg);
throw server::ServerException(SERVER_BLOCKING_QUEUE_EMPTY, error_msg);
}
ScheduleTaskPtr back(queue_.back());
......
......@@ -21,7 +21,7 @@ class MetricsBase{
return instance;
}
virtual ServerError Init() {};
virtual ErrorCode Init() {};
virtual void AddVectorsSuccessTotalIncrement(double value = 1) {};
virtual void AddVectorsFailTotalIncrement(double value = 1) {};
......
......@@ -14,7 +14,7 @@ namespace zilliz {
namespace milvus {
namespace server {
ServerError
ErrorCode
PrometheusMetrics::Init() {
try {
ConfigNode &configNode = ServerConfig::GetInstance().GetConfig(CONFIG_METRIC);
......
......@@ -39,8 +39,7 @@ class PrometheusMetrics: public MetricsBase {
return instance;
}
ServerError
Init();
ErrorCode Init();
private:
std::shared_ptr<prometheus::Exposer> exposer_ptr_;
......
......@@ -21,7 +21,7 @@ DBWrapper::DBWrapper() {
}
ServerError DBWrapper::StartService() {
ErrorCode DBWrapper::StartService() {
//db config
zilliz::milvus::engine::Options opt;
ConfigNode& db_config = ServerConfig::GetInstance().GetConfig(CONFIG_DB);
......@@ -79,7 +79,7 @@ ServerError DBWrapper::StartService() {
opt.meta.archive_conf.SetCriterias(criterial);
//create db root folder
ServerError err = CommonUtil::CreateDirectory(opt.meta.path);
ErrorCode err = CommonUtil::CreateDirectory(opt.meta.path);
if(err != SERVER_SUCCESS) {
std::cout << "ERROR! Failed to create database root path: " << opt.meta.path << std::endl;
kill(0, SIGUSR1);
......@@ -112,7 +112,7 @@ ServerError DBWrapper::StartService() {
return SERVER_SUCCESS;
}
ServerError DBWrapper::StopService() {
ErrorCode DBWrapper::StopService() {
if(db_) {
db_->Stop();
}
......
......@@ -29,8 +29,8 @@ public:
return GetInstance().EngineDB();
}
ServerError StartService();
ServerError StopService();
ErrorCode StartService();
ErrorCode StopService();
std::shared_ptr<engine::DB> EngineDB() {
return db_;
......
......@@ -209,10 +209,10 @@ Server::Stop() {
}
ServerError
ErrorCode
Server::LoadConfig() {
ServerConfig::GetInstance().LoadConfigFile(config_filename_);
ServerError err = ServerConfig::GetInstance().ValidateConfig();
ErrorCode err = ServerConfig::GetInstance().ValidateConfig();
if(err != SERVER_SUCCESS){
exit(0);
}
......
......@@ -29,7 +29,7 @@ class Server {
void Daemonize();
static void HandleSignal(int signal);
ServerError LoadConfig();
ErrorCode LoadConfig();
void StartService();
void StopService();
......
......@@ -28,7 +28,7 @@ ServerConfig::GetInstance() {
return config;
}
ServerError
ErrorCode
ServerConfig::LoadConfigFile(const std::string& config_filename) {
std::string filename = config_filename;
if(filename.empty()){
......@@ -44,7 +44,7 @@ ServerConfig::LoadConfigFile(const std::string& config_filename) {
try {
ConfigMgr* mgr = const_cast<ConfigMgr*>(ConfigMgr::GetInstance());
ServerError err = mgr->LoadConfigFile(filename);
ErrorCode err = mgr->LoadConfigFile(filename);
if(err != 0) {
std::cout << "Server failed to load config file" << std::endl;
exit(1);//directly exit program if the config file is illegal
......@@ -58,7 +58,7 @@ ServerConfig::LoadConfigFile(const std::string& config_filename) {
return SERVER_SUCCESS;
}
ServerError ServerConfig::ValidateConfig() const {
ErrorCode ServerConfig::ValidateConfig() const {
//server config validation
ConfigNode server_config = GetConfig(CONFIG_SERVER);
uint32_t gpu_index = (uint32_t)server_config.GetInt32Value(CONFIG_GPU_INDEX, 0);
......
......@@ -68,8 +68,8 @@ class ServerConfig {
public:
static ServerConfig &GetInstance();
ServerError LoadConfigFile(const std::string& config_filename);
ServerError ValidateConfig() const;
ErrorCode LoadConfigFile(const std::string& config_filename);
ErrorCode ValidateConfig() const;
void PrintAll() const;
ConfigNode GetConfig(const std::string& name) const;
......
......@@ -16,8 +16,8 @@ namespace grpc {
using namespace ::milvus;
namespace {
const std::map<ServerError, ::milvus::grpc::ErrorCode> &ErrorMap() {
static const std::map<ServerError, ::milvus::grpc::ErrorCode> code_map = {
const std::map<ErrorCode, ::milvus::grpc::ErrorCode> &ErrorMap() {
static const std::map<ErrorCode, ::milvus::grpc::ErrorCode> code_map = {
{SERVER_UNEXPECTED_ERROR, ::milvus::grpc::ErrorCode::UNEXPECTED_ERROR},
{SERVER_UNSUPPORTED_ERROR, ::milvus::grpc::ErrorCode::UNEXPECTED_ERROR},
{SERVER_NULL_POINTER, ::milvus::grpc::ErrorCode::UNEXPECTED_ERROR},
......@@ -66,7 +66,7 @@ GrpcBaseTask::~GrpcBaseTask() {
WaitToFinish();
}
ServerError GrpcBaseTask::Execute() {
ErrorCode GrpcBaseTask::Execute() {
error_code_ = OnExecute();
Done();
return error_code_;
......@@ -77,7 +77,7 @@ void GrpcBaseTask::Done() {
finish_cond_.notify_all();
}
ServerError GrpcBaseTask::SetError(ServerError error_code, const std::string &error_msg) {
ErrorCode GrpcBaseTask::SetError(ErrorCode error_code, const std::string &error_msg) {
error_code_ = error_code;
error_msg_ = error_msg;
......@@ -85,7 +85,7 @@ ServerError GrpcBaseTask::SetError(ServerError error_code, const std::string &er
return error_code_;
}
ServerError GrpcBaseTask::WaitToFinish() {
ErrorCode GrpcBaseTask::WaitToFinish() {
std::unique_lock<std::mutex> lock(finish_mtx_);
finish_cond_.wait(lock, [this] { return done_; });
......@@ -112,7 +112,7 @@ void GrpcRequestScheduler::ExecTask(BaseTaskPtr &task_ptr, ::milvus::grpc::Statu
if (!task_ptr->IsAsync()) {
task_ptr->WaitToFinish();
ServerError err = task_ptr->ErrorCode();
ErrorCode err = task_ptr->ErrorID();
if (err != SERVER_SUCCESS) {
grpc_status->set_reason(task_ptr->ErrorMsg());
grpc_status->set_error_code(ErrorMap().at(err));
......@@ -153,12 +153,12 @@ void GrpcRequestScheduler::Stop() {
SERVER_LOG_INFO << "Scheduler stopped";
}
ServerError GrpcRequestScheduler::ExecuteTask(const BaseTaskPtr &task_ptr) {
ErrorCode GrpcRequestScheduler::ExecuteTask(const BaseTaskPtr &task_ptr) {
if (task_ptr == nullptr) {
return SERVER_NULL_POINTER;
}
ServerError err = PutTaskToQueue(task_ptr);
ErrorCode err = PutTaskToQueue(task_ptr);
if (err != SERVER_SUCCESS) {
SERVER_LOG_ERROR << "Put task to queue failed with code: " << err;
return err;
......@@ -185,7 +185,7 @@ void GrpcRequestScheduler::TakeTaskToExecute(TaskQueuePtr task_queue) {
}
try {
ServerError err = task->Execute();
ErrorCode err = task->Execute();
if (err != SERVER_SUCCESS) {
SERVER_LOG_ERROR << "Task failed with code: " << err;
}
......@@ -195,7 +195,7 @@ void GrpcRequestScheduler::TakeTaskToExecute(TaskQueuePtr task_queue) {
}
}
ServerError GrpcRequestScheduler::PutTaskToQueue(const BaseTaskPtr &task_ptr) {
ErrorCode GrpcRequestScheduler::PutTaskToQueue(const BaseTaskPtr &task_ptr) {
std::lock_guard<std::mutex> lock(queue_mtx_);
std::string group_name = task_ptr->TaskGroup();
......
......@@ -25,24 +25,24 @@ protected:
virtual ~GrpcBaseTask();
public:
ServerError Execute();
ErrorCode Execute();
void Done();
ServerError WaitToFinish();
ErrorCode WaitToFinish();
std::string TaskGroup() const { return task_group_; }
ServerError ErrorCode() const { return error_code_; }
ErrorCode ErrorID() const { return error_code_; }
std::string ErrorMsg() const { return error_msg_; }
bool IsAsync() const { return async_; }
protected:
virtual ServerError OnExecute() = 0;
virtual ErrorCode OnExecute() = 0;
ServerError SetError(ServerError error_code, const std::string &msg);
ErrorCode SetError(ErrorCode error_code, const std::string &msg);
protected:
mutable std::mutex finish_mtx_;
......@@ -51,7 +51,7 @@ protected:
std::string task_group_;
bool async_;
bool done_;
ServerError error_code_;
ErrorCode error_code_;
std::string error_msg_;
};
......@@ -71,7 +71,7 @@ public:
void Stop();
ServerError ExecuteTask(const BaseTaskPtr &task_ptr);
ErrorCode ExecuteTask(const BaseTaskPtr &task_ptr);
static void ExecTask(BaseTaskPtr &task_ptr, ::milvus::grpc::Status *grpc_status);
......@@ -82,7 +82,7 @@ protected:
void TakeTaskToExecute(TaskQueuePtr task_queue);
ServerError PutTaskToQueue(const BaseTaskPtr &task_ptr);
ErrorCode PutTaskToQueue(const BaseTaskPtr &task_ptr);
private:
mutable std::mutex queue_mtx_;
......
......@@ -67,7 +67,7 @@ namespace {
void
ConvertTimeRangeToDBDates(const std::vector<::milvus::grpc::Range> &range_array,
std::vector<DB_DATE> &dates,
ServerError &error_code,
ErrorCode &error_code,
std::string &error_msg) {
dates.clear();
for (auto &range : range_array) {
......@@ -123,13 +123,13 @@ CreateTableTask::Create(const ::milvus::grpc::TableSchema *schema) {
return std::shared_ptr<GrpcBaseTask>(new CreateTableTask(schema));
}
ServerError
ErrorCode
CreateTableTask::OnExecute() {
TimeRecorder rc("CreateTableTask");
try {
//step 1: check arguments
ServerError res = ValidationUtil::ValidateTableName(schema_->table_name().table_name());
ErrorCode res = ValidationUtil::ValidateTableName(schema_->table_name().table_name());
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + schema_->table_name().table_name());
}
......@@ -187,13 +187,13 @@ DescribeTableTask::Create(const std::string &table_name, ::milvus::grpc::TableSc
return std::shared_ptr<GrpcBaseTask>(new DescribeTableTask(table_name, schema));
}
ServerError
ErrorCode
DescribeTableTask::OnExecute() {
TimeRecorder rc("DescribeTableTask");
try {
//step 1: check arguments
ServerError res = ValidationUtil::ValidateTableName(table_name_);
ErrorCode res = ValidationUtil::ValidateTableName(table_name_);
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + table_name_);
}
......@@ -235,14 +235,14 @@ CreateIndexTask::Create(const ::milvus::grpc::IndexParam *index_param) {
return std::shared_ptr<GrpcBaseTask>(new CreateIndexTask(index_param));
}
ServerError
ErrorCode
CreateIndexTask::OnExecute() {
try {
TimeRecorder rc("CreateIndexTask");
//step 1: check arguments
std::string table_name_ = index_param_->table_name().table_name();
ServerError res = ValidationUtil::ValidateTableName(table_name_);
ErrorCode res = ValidationUtil::ValidateTableName(table_name_);
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + table_name_);
}
......@@ -298,13 +298,13 @@ HasTableTask::Create(const std::string &table_name, bool &has_table) {
return std::shared_ptr<GrpcBaseTask>(new HasTableTask(table_name, has_table));
}
ServerError
ErrorCode
HasTableTask::OnExecute() {
try {
TimeRecorder rc("HasTableTask");
//step 1: check arguments
ServerError res = ValidationUtil::ValidateTableName(table_name_);
ErrorCode res = ValidationUtil::ValidateTableName(table_name_);
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + table_name_);
}
......@@ -335,13 +335,13 @@ DropTableTask::Create(const std::string &table_name) {
return std::shared_ptr<GrpcBaseTask>(new DropTableTask(table_name));
}
ServerError
ErrorCode
DropTableTask::OnExecute() {
try {
TimeRecorder rc("DropTableTask");
//step 1: check arguments
ServerError res = ValidationUtil::ValidateTableName(table_name_);
ErrorCode res = ValidationUtil::ValidateTableName(table_name_);
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + table_name_);
}
......@@ -387,7 +387,7 @@ ShowTablesTask::Create(::grpc::ServerWriter<::milvus::grpc::TableName> *writer)
return std::shared_ptr<GrpcBaseTask>(new ShowTablesTask(writer));
}
ServerError
ErrorCode
ShowTablesTask::OnExecute() {
std::vector<engine::meta::TableSchema> schema_array;
engine::Status stat = DBWrapper::DB()->AllTables(schema_array);
......@@ -424,13 +424,13 @@ InsertTask::Create(const ::milvus::grpc::InsertParam *insert_param,
return std::shared_ptr<GrpcBaseTask>(new InsertTask(insert_param, record_ids));
}
ServerError
ErrorCode
InsertTask::OnExecute() {
try {
TimeRecorder rc("InsertVectorTask");
//step 1: check arguments
ServerError res = ValidationUtil::ValidateTableName(insert_param_->table_name());
ErrorCode res = ValidationUtil::ValidateTableName(insert_param_->table_name());
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + insert_param_->table_name());
}
......@@ -489,7 +489,7 @@ InsertTask::OnExecute() {
}
uint64_t vec_dim = insert_param_->row_record_array(i).vector_data().size();
if (vec_dim != table_info.dimension_) {
ServerError error_code = SERVER_INVALID_VECTOR_DIMENSION;
ErrorCode error_code = SERVER_INVALID_VECTOR_DIMENSION;
std::string error_msg = "Invalid rowrecord dimension: " + std::to_string(vec_dim)
+ " vs. table dimension:" +
std::to_string(table_info.dimension_);
......@@ -569,14 +569,14 @@ SearchTask::Create(const ::milvus::grpc::SearchParam *search_vector_infos,
response));
}
ServerError
ErrorCode
SearchTask::OnExecute() {
try {
TimeRecorder rc("SearchTask");
//step 1: check table name
std::string table_name_ = search_param_->table_name();
ServerError res = ValidationUtil::ValidateTableName(table_name_);
ErrorCode res = ValidationUtil::ValidateTableName(table_name_);
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + table_name_);
}
......@@ -612,7 +612,7 @@ SearchTask::OnExecute() {
//step 4: check date range, and convert to db dates
std::vector<DB_DATE> dates;
ServerError error_code = SERVER_SUCCESS;
ErrorCode error_code = SERVER_SUCCESS;
std::string error_msg;
std::vector<::milvus::grpc::Range> range_array;
......@@ -642,7 +642,7 @@ SearchTask::OnExecute() {
}
uint64_t query_vec_dim = search_param_->query_record_array(i).vector_data().size();
if (query_vec_dim != table_info.dimension_) {
ServerError error_code = SERVER_INVALID_VECTOR_DIMENSION;
ErrorCode error_code = SERVER_INVALID_VECTOR_DIMENSION;
std::string error_msg = "Invalid rowrecord dimension: " + std::to_string(query_vec_dim)
+ " vs. table dimension:" + std::to_string(table_info.dimension_);
return SetError(error_code, error_msg);
......@@ -722,13 +722,13 @@ CountTableTask::Create(const std::string &table_name, int64_t &row_count) {
return std::shared_ptr<GrpcBaseTask>(new CountTableTask(table_name, row_count));
}
ServerError
ErrorCode
CountTableTask::OnExecute() {
try {
TimeRecorder rc("GetTableRowCountTask");
//step 1: check arguments
ServerError res = SERVER_SUCCESS;
ErrorCode res = SERVER_SUCCESS;
res = ValidationUtil::ValidateTableName(table_name_);
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + table_name_);
......@@ -765,7 +765,7 @@ CmdTask::Create(const std::string &cmd, std::string &result) {
return std::shared_ptr<GrpcBaseTask>(new CmdTask(cmd, result));
}
ServerError
ErrorCode
CmdTask::OnExecute() {
if (cmd_ == "version") {
result_ = MILVUS_VERSION;
......@@ -794,14 +794,14 @@ DeleteByRangeTask::Create(const ::milvus::grpc::DeleteByRangeParam *delete_by_ra
return std::shared_ptr<GrpcBaseTask>(new DeleteByRangeTask(delete_by_range_param));
}
ServerError
ErrorCode
DeleteByRangeTask::OnExecute() {
try {
TimeRecorder rc("DeleteByRangeTask");
//step 1: check arguments
std::string table_name = delete_by_range_param_->table_name();
ServerError res = ValidationUtil::ValidateTableName(table_name);
ErrorCode res = ValidationUtil::ValidateTableName(table_name);
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + table_name);
}
......@@ -822,7 +822,7 @@ DeleteByRangeTask::OnExecute() {
//step 3: check date range, and convert to db dates
std::vector<DB_DATE> dates;
ServerError error_code = SERVER_SUCCESS;
ErrorCode error_code = SERVER_SUCCESS;
std::string error_msg;
std::vector<::milvus::grpc::Range> range_array;
......@@ -862,13 +862,13 @@ PreloadTableTask::Create(const std::string &table_name){
return std::shared_ptr<GrpcBaseTask>(new PreloadTableTask(table_name));
}
ServerError
ErrorCode
PreloadTableTask::OnExecute() {
try {
TimeRecorder rc("PreloadTableTask");
//step 1: check arguments
ServerError res = ValidationUtil::ValidateTableName(table_name_);
ErrorCode res = ValidationUtil::ValidateTableName(table_name_);
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + table_name_);
}
......@@ -902,13 +902,13 @@ DescribeIndexTask::Create(const std::string &table_name,
return std::shared_ptr<GrpcBaseTask>(new DescribeIndexTask(table_name, index_param));
}
ServerError
ErrorCode
DescribeIndexTask::OnExecute() {
try {
TimeRecorder rc("DescribeIndexTask");
//step 1: check arguments
ServerError res = ValidationUtil::ValidateTableName(table_name_);
ErrorCode res = ValidationUtil::ValidateTableName(table_name_);
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + table_name_);
}
......@@ -944,13 +944,13 @@ DropIndexTask::Create(const std::string &table_name){
return std::shared_ptr<GrpcBaseTask>(new DropIndexTask(table_name));
}
ServerError
ErrorCode
DropIndexTask::OnExecute() {
try {
TimeRecorder rc("DropIndexTask");
//step 1: check arguments
ServerError res = ValidationUtil::ValidateTableName(table_name_);
ErrorCode res = ValidationUtil::ValidateTableName(table_name_);
if (res != SERVER_SUCCESS) {
return SetError(res, "Invalid table name: " + table_name_);
}
......
......@@ -29,7 +29,7 @@ protected:
explicit
CreateTableTask(const ::milvus::grpc::TableSchema *request);
ServerError
ErrorCode
OnExecute() override;
private:
......@@ -45,7 +45,7 @@ public:
protected:
HasTableTask(const std::string &request, bool &has_table);
ServerError
ErrorCode
OnExecute() override;
......@@ -63,7 +63,7 @@ public:
protected:
DescribeTableTask(const std::string &table_name, ::milvus::grpc::TableSchema *schema);
ServerError
ErrorCode
OnExecute() override;
......@@ -82,7 +82,7 @@ protected:
explicit
DropTableTask(const std::string &table_name);
ServerError
ErrorCode
OnExecute() override;
......@@ -100,7 +100,7 @@ protected:
explicit
CreateIndexTask(const ::milvus::grpc::IndexParam *index_Param);
ServerError
ErrorCode
OnExecute() override;
......@@ -118,7 +118,7 @@ protected:
explicit
ShowTablesTask(::grpc::ServerWriter<::milvus::grpc::TableName> *writer);
ServerError
ErrorCode
OnExecute() override;
private:
......@@ -136,7 +136,7 @@ protected:
InsertTask(const ::milvus::grpc::InsertParam *insert_Param,
::milvus::grpc::VectorIds *record_ids_);
ServerError
ErrorCode
OnExecute() override;
private:
......@@ -157,7 +157,7 @@ protected:
const std::vector<std::string> &file_id_array,
::milvus::grpc::TopKQueryResultList *response);
ServerError
ErrorCode
OnExecute() override;
private:
......@@ -175,7 +175,7 @@ public:
protected:
CountTableTask(const std::string &table_name, int64_t &row_count);
ServerError
ErrorCode
OnExecute() override;
private:
......@@ -192,7 +192,7 @@ public:
protected:
CmdTask(const std::string &cmd, std::string &result);
ServerError
ErrorCode
OnExecute() override;
private:
......@@ -209,7 +209,7 @@ public:
protected:
DeleteByRangeTask(const ::milvus::grpc::DeleteByRangeParam *delete_by_range_param);
ServerError
ErrorCode
OnExecute() override;
private:
......@@ -225,7 +225,7 @@ public:
protected:
PreloadTableTask(const std::string &table_name);
ServerError
ErrorCode
OnExecute() override;
private:
......@@ -243,7 +243,7 @@ protected:
DescribeIndexTask(const std::string &table_name,
::milvus::grpc::IndexParam *index_param);
ServerError
ErrorCode
OnExecute() override;
private:
......@@ -260,7 +260,7 @@ public:
protected:
DropIndexTask(const std::string &table_name);
ServerError
ErrorCode
OnExecute() override;
private:
......
......@@ -61,7 +61,7 @@ bool CommonUtil::IsDirectoryExist(const std::string &path) {
return true;
}
ServerError CommonUtil::CreateDirectory(const std::string &path) {
ErrorCode CommonUtil::CreateDirectory(const std::string &path) {
if(path.empty()) {
return SERVER_SUCCESS;
}
......@@ -74,7 +74,7 @@ ServerError CommonUtil::CreateDirectory(const std::string &path) {
fs::path fs_path(path);
fs::path parent_path = fs_path.parent_path();
ServerError err = CreateDirectory(parent_path.string());
ErrorCode err = CreateDirectory(parent_path.string());
if(err != SERVER_SUCCESS){
return err;
}
......@@ -122,7 +122,7 @@ namespace {
}
}
ServerError CommonUtil::DeleteDirectory(const std::string &path) {
ErrorCode CommonUtil::DeleteDirectory(const std::string &path) {
if(path.empty()) {
return SERVER_SUCCESS;
}
......
......@@ -22,8 +22,8 @@ class CommonUtil {
static bool IsFileExist(const std::string &path);
static uint64_t GetFileSize(const std::string &path);
static bool IsDirectoryExist(const std::string &path);
static ServerError CreateDirectory(const std::string &path);
static ServerError DeleteDirectory(const std::string &path);
static ErrorCode CreateDirectory(const std::string &path);
static ErrorCode DeleteDirectory(const std::string &path);
static std::string GetExePath();
......
......@@ -11,82 +11,92 @@
namespace zilliz {
namespace milvus {
namespace server {
using ServerError = int32_t;
constexpr ServerError SERVER_SUCCESS = 0;
constexpr ServerError SERVER_ERROR_CODE_BASE = 0x30000;
using ErrorCode = int32_t;
constexpr ServerError
ToGlobalServerErrorCode(const ServerError error_code) {
constexpr ErrorCode SERVER_SUCCESS = 0;
constexpr ErrorCode SERVER_ERROR_CODE_BASE = 0x30000;
constexpr ErrorCode
ToServerErrorCode(const ErrorCode error_code) {
return SERVER_ERROR_CODE_BASE + error_code;
}
constexpr ServerError SERVER_UNEXPECTED_ERROR = ToGlobalServerErrorCode(1);
constexpr ServerError SERVER_UNSUPPORTED_ERROR = ToGlobalServerErrorCode(2);
constexpr ServerError SERVER_NULL_POINTER = ToGlobalServerErrorCode(3);
constexpr ServerError SERVER_INVALID_ARGUMENT = ToGlobalServerErrorCode(4);
constexpr ServerError SERVER_FILE_NOT_FOUND = ToGlobalServerErrorCode(5);
constexpr ServerError SERVER_NOT_IMPLEMENT = ToGlobalServerErrorCode(6);
constexpr ServerError SERVER_BLOCKING_QUEUE_EMPTY = ToGlobalServerErrorCode(7);
constexpr ServerError SERVER_CANNOT_CREATE_FOLDER = ToGlobalServerErrorCode(8);
constexpr ServerError SERVER_CANNOT_CREATE_FILE = ToGlobalServerErrorCode(9);
constexpr ServerError SERVER_CANNOT_DELETE_FOLDER = ToGlobalServerErrorCode(10);
constexpr ServerError SERVER_CANNOT_DELETE_FILE = ToGlobalServerErrorCode(11);
constexpr ServerError SERVER_BUILD_INDEX_ERROR = ToGlobalServerErrorCode(12);
constexpr ServerError SERVER_TABLE_NOT_EXIST = ToGlobalServerErrorCode(100);
constexpr ServerError SERVER_INVALID_TABLE_NAME = ToGlobalServerErrorCode(101);
constexpr ServerError SERVER_INVALID_TABLE_DIMENSION = ToGlobalServerErrorCode(102);
constexpr ServerError SERVER_INVALID_TIME_RANGE = ToGlobalServerErrorCode(103);
constexpr ServerError SERVER_INVALID_VECTOR_DIMENSION = ToGlobalServerErrorCode(104);
constexpr ServerError SERVER_INVALID_INDEX_TYPE = ToGlobalServerErrorCode(105);
constexpr ServerError SERVER_INVALID_ROWRECORD = ToGlobalServerErrorCode(106);
constexpr ServerError SERVER_INVALID_ROWRECORD_ARRAY = ToGlobalServerErrorCode(107);
constexpr ServerError SERVER_INVALID_TOPK = ToGlobalServerErrorCode(108);
constexpr ServerError SERVER_ILLEGAL_VECTOR_ID = ToGlobalServerErrorCode(109);
constexpr ServerError SERVER_ILLEGAL_SEARCH_RESULT = ToGlobalServerErrorCode(110);
constexpr ServerError SERVER_CACHE_ERROR = ToGlobalServerErrorCode(111);
constexpr ServerError SERVER_WRITE_ERROR = ToGlobalServerErrorCode(112);
constexpr ServerError SERVER_INVALID_NPROBE = ToGlobalServerErrorCode(113);
constexpr ServerError SERVER_INVALID_INDEX_NLIST = ToGlobalServerErrorCode(114);
constexpr ServerError SERVER_INVALID_INDEX_METRIC_TYPE = ToGlobalServerErrorCode(115);
constexpr ServerError SERVER_INVALID_INDEX_FILE_SIZE = ToGlobalServerErrorCode(116);
constexpr ServerError SERVER_LICENSE_FILE_NOT_EXIST = ToGlobalServerErrorCode(500);
constexpr ServerError SERVER_LICENSE_VALIDATION_FAIL = ToGlobalServerErrorCode(501);
constexpr ErrorCode ENGINE_SUCCESS = 0;
constexpr ErrorCode ENGINE_ERROR_CODE_BASE = 0x40000;
constexpr ErrorCode
ToEngineErrorCode(const ErrorCode error_code) {
return ENGINE_ERROR_CODE_BASE + error_code;
}
constexpr ServerError DB_META_TRANSACTION_FAILED = ToGlobalServerErrorCode(1000);
constexpr ErrorCode KNOWHERE_SUCCESS = 0;
constexpr ErrorCode KNOWHERE_ERROR_CODE_BASE = 0x50000;
constexpr ErrorCode
ToKnowhereErrorCode(const ErrorCode error_code) {
return KNOWHERE_ERROR_CODE_BASE + error_code;
}
using KnowhereError = int32_t;
constexpr KnowhereError KNOWHERE_SUCCESS = 0;
constexpr KnowhereError KNOWHERE_ERROR = ToGlobalServerErrorCode(1);
constexpr KnowhereError KNOWHERE_INVALID_ARGUMENT = ToGlobalServerErrorCode(2);
constexpr KnowhereError KNOWHERE_UNEXPECTED_ERROR = ToGlobalServerErrorCode(3);
//server error code
constexpr ErrorCode SERVER_UNEXPECTED_ERROR = ToServerErrorCode(1);
constexpr ErrorCode SERVER_UNSUPPORTED_ERROR = ToServerErrorCode(2);
constexpr ErrorCode SERVER_NULL_POINTER = ToServerErrorCode(3);
constexpr ErrorCode SERVER_INVALID_ARGUMENT = ToServerErrorCode(4);
constexpr ErrorCode SERVER_FILE_NOT_FOUND = ToServerErrorCode(5);
constexpr ErrorCode SERVER_NOT_IMPLEMENT = ToServerErrorCode(6);
constexpr ErrorCode SERVER_BLOCKING_QUEUE_EMPTY = ToServerErrorCode(7);
constexpr ErrorCode SERVER_CANNOT_CREATE_FOLDER = ToServerErrorCode(8);
constexpr ErrorCode SERVER_CANNOT_CREATE_FILE = ToServerErrorCode(9);
constexpr ErrorCode SERVER_CANNOT_DELETE_FOLDER = ToServerErrorCode(10);
constexpr ErrorCode SERVER_CANNOT_DELETE_FILE = ToServerErrorCode(11);
constexpr ErrorCode SERVER_BUILD_INDEX_ERROR = ToServerErrorCode(12);
constexpr ErrorCode SERVER_TABLE_NOT_EXIST = ToServerErrorCode(100);
constexpr ErrorCode SERVER_INVALID_TABLE_NAME = ToServerErrorCode(101);
constexpr ErrorCode SERVER_INVALID_TABLE_DIMENSION = ToServerErrorCode(102);
constexpr ErrorCode SERVER_INVALID_TIME_RANGE = ToServerErrorCode(103);
constexpr ErrorCode SERVER_INVALID_VECTOR_DIMENSION = ToServerErrorCode(104);
constexpr ErrorCode SERVER_INVALID_INDEX_TYPE = ToServerErrorCode(105);
constexpr ErrorCode SERVER_INVALID_ROWRECORD = ToServerErrorCode(106);
constexpr ErrorCode SERVER_INVALID_ROWRECORD_ARRAY = ToServerErrorCode(107);
constexpr ErrorCode SERVER_INVALID_TOPK = ToServerErrorCode(108);
constexpr ErrorCode SERVER_ILLEGAL_VECTOR_ID = ToServerErrorCode(109);
constexpr ErrorCode SERVER_ILLEGAL_SEARCH_RESULT = ToServerErrorCode(110);
constexpr ErrorCode SERVER_CACHE_ERROR = ToServerErrorCode(111);
constexpr ErrorCode SERVER_WRITE_ERROR = ToServerErrorCode(112);
constexpr ErrorCode SERVER_INVALID_NPROBE = ToServerErrorCode(113);
constexpr ErrorCode SERVER_INVALID_INDEX_NLIST = ToServerErrorCode(114);
constexpr ErrorCode SERVER_INVALID_INDEX_METRIC_TYPE = ToServerErrorCode(115);
constexpr ErrorCode SERVER_INVALID_INDEX_FILE_SIZE = ToServerErrorCode(116);
//engine error code
constexpr ErrorCode DB_META_TRANSACTION_FAILED = ToEngineErrorCode(1);
constexpr ErrorCode DB_TABLE_NOT_FOUND = ToEngineErrorCode(2);
//knowhere error code
constexpr ErrorCode KNOWHERE_ERROR = ToKnowhereErrorCode(1);
constexpr ErrorCode KNOWHERE_INVALID_ARGUMENT = ToKnowhereErrorCode(2);
constexpr ErrorCode KNOWHERE_UNEXPECTED_ERROR = ToKnowhereErrorCode(3);
class ServerException : public std::exception {
public:
ServerException(ServerError error_code,
const std::string &message = std::string())
namespace server {
class ServerException : public std::exception {
public:
ServerException(ErrorCode error_code,
const std::string &message = std::string())
: error_code_(error_code), message_(message) {}
public:
ServerError error_code() const {
return error_code_;
}
public:
ErrorCode error_code() const {
return error_code_;
}
virtual const char *what() const noexcept {
return message_.c_str();
}
virtual const char *what() const noexcept {
return message_.c_str();
}
private:
ServerError error_code_;
std::string message_;
};
private:
ErrorCode error_code_;
std::string message_;
};
}
} // namespace server
} // namespace milvus
} // namespace zilliz
......@@ -24,9 +24,9 @@ void StringHelpFunctions::TrimStringQuote(std::string &string, const std::string
}
}
ServerError StringHelpFunctions::SplitStringByDelimeter(const std::string &str,
const std::string &delimeter,
std::vector<std::string> &result) {
ErrorCode StringHelpFunctions::SplitStringByDelimeter(const std::string &str,
const std::string &delimeter,
std::vector<std::string> &result) {
if(str.empty()) {
return SERVER_SUCCESS;
}
......@@ -46,10 +46,10 @@ ServerError StringHelpFunctions::SplitStringByDelimeter(const std::string &str,
return SERVER_SUCCESS;
}
ServerError StringHelpFunctions::SplitStringByQuote(const std::string &str,
const std::string &delimeter,
const std::string &quote,
std::vector<std::string> &result) {
ErrorCode StringHelpFunctions::SplitStringByQuote(const std::string &str,
const std::string &delimeter,
const std::string &quote,
std::vector<std::string> &result) {
if (quote.empty()) {
return SplitStringByDelimeter(str, delimeter, result);
}
......
......@@ -29,9 +29,9 @@ public:
// ,b, | b |
// ,, | |
// a a
static ServerError SplitStringByDelimeter(const std::string &str,
const std::string &delimeter,
std::vector<std::string> &result);
static ErrorCode SplitStringByDelimeter(const std::string &str,
const std::string &delimeter,
std::vector<std::string> &result);
//assume the table has two columns, quote='\"', delimeter=','
// a,b a | b
......@@ -40,10 +40,10 @@ public:
// "aa,bb" aa,bb
// 55,1122\"aa,bb\",yyy,\"kkk\" 55 | 1122aa,bb | yyy | kkk
// "55,1122"aa,bb",yyy,"kkk" illegal
static ServerError SplitStringByQuote(const std::string &str,
const std::string &delimeter,
const std::string &quote,
std::vector<std::string> &result);
static ErrorCode SplitStringByQuote(const std::string &str,
const std::string &delimeter,
const std::string &quote,
std::vector<std::string> &result);
};
......
......@@ -12,7 +12,7 @@ constexpr size_t table_name_size_limit = 255;
constexpr int64_t table_dimension_limit = 16384;
constexpr int32_t index_file_size_limit = 4096; //index trigger size max = 4096 MB
ServerError
ErrorCode
ValidationUtil::ValidateTableName(const std::string &table_name) {
// Table name shouldn't be empty.
......@@ -46,7 +46,7 @@ ValidationUtil::ValidateTableName(const std::string &table_name) {
return SERVER_SUCCESS;
}
ServerError
ErrorCode
ValidationUtil::ValidateTableDimension(int64_t dimension) {
if (dimension <= 0 || dimension > table_dimension_limit) {
SERVER_LOG_ERROR << "Table dimension excceed the limitation: " << table_dimension_limit;
......@@ -56,7 +56,7 @@ ValidationUtil::ValidateTableDimension(int64_t dimension) {
}
}
ServerError
ErrorCode
ValidationUtil::ValidateTableIndexType(int32_t index_type) {
int engine_type = (int)engine::EngineType(index_type);
if(engine_type <= 0 || engine_type > (int)engine::EngineType::MAX_VALUE) {
......@@ -66,7 +66,7 @@ ValidationUtil::ValidateTableIndexType(int32_t index_type) {
return SERVER_SUCCESS;
}
ServerError
ErrorCode
ValidationUtil::ValidateTableIndexNlist(int32_t nlist) {
if(nlist <= 0) {
return SERVER_INVALID_INDEX_NLIST;
......@@ -75,7 +75,7 @@ ValidationUtil::ValidateTableIndexNlist(int32_t nlist) {
return SERVER_SUCCESS;
}
ServerError
ErrorCode
ValidationUtil::ValidateTableIndexFileSize(int64_t index_file_size) {
if(index_file_size <= 0 || index_file_size > index_file_size_limit) {
return SERVER_INVALID_INDEX_FILE_SIZE;
......@@ -84,7 +84,7 @@ ValidationUtil::ValidateTableIndexFileSize(int64_t index_file_size) {
return SERVER_SUCCESS;
}
ServerError
ErrorCode
ValidationUtil::ValidateTableIndexMetricType(int32_t metric_type) {
if(metric_type != (int32_t)engine::MetricType::L2 && metric_type != (int32_t)engine::MetricType::IP) {
return SERVER_INVALID_INDEX_METRIC_TYPE;
......@@ -92,7 +92,7 @@ ValidationUtil::ValidateTableIndexMetricType(int32_t metric_type) {
return SERVER_SUCCESS;
}
ServerError
ErrorCode
ValidationUtil::ValidateSearchTopk(int64_t top_k, const engine::meta::TableSchema& table_schema) {
if (top_k <= 0 || top_k > 1024) {
return SERVER_INVALID_TOPK;
......@@ -101,7 +101,7 @@ ValidationUtil::ValidateSearchTopk(int64_t top_k, const engine::meta::TableSchem
return SERVER_SUCCESS;
}
ServerError
ErrorCode
ValidationUtil::ValidateSearchNprobe(int64_t nprobe, const engine::meta::TableSchema& table_schema) {
if (nprobe <= 0 || nprobe > table_schema.nlist_) {
return SERVER_INVALID_NPROBE;
......@@ -110,7 +110,7 @@ ValidationUtil::ValidateSearchNprobe(int64_t nprobe, const engine::meta::TableSc
return SERVER_SUCCESS;
}
ServerError
ErrorCode
ValidationUtil::ValidateGpuIndex(uint32_t gpu_index) {
int num_devices = 0;
auto cuda_err = cudaGetDeviceCount(&num_devices);
......@@ -126,7 +126,7 @@ ValidationUtil::ValidateGpuIndex(uint32_t gpu_index) {
return SERVER_SUCCESS;
}
ServerError
ErrorCode
ValidationUtil::GetGpuMemory(uint32_t gpu_index, size_t& memory) {
cudaDeviceProp deviceProp;
auto cuda_err = cudaGetDeviceProperties(&deviceProp, gpu_index);
......
......@@ -9,34 +9,34 @@ namespace server {
class ValidationUtil {
public:
static ServerError
static ErrorCode
ValidateTableName(const std::string &table_name);
static ServerError
static ErrorCode
ValidateTableDimension(int64_t dimension);
static ServerError
static ErrorCode
ValidateTableIndexType(int32_t index_type);
static ServerError
static ErrorCode
ValidateTableIndexNlist(int32_t nlist);
static ServerError
static ErrorCode
ValidateTableIndexFileSize(int64_t index_file_size);
static ServerError
static ErrorCode
ValidateTableIndexMetricType(int32_t metric_type);
static ServerError
static ErrorCode
ValidateSearchTopk(int64_t top_k, const engine::meta::TableSchema& table_schema);
static ServerError
static ErrorCode
ValidateSearchNprobe(int64_t nprobe, const engine::meta::TableSchema& table_schema);
static ServerError
static ErrorCode
ValidateGpuIndex(uint32_t gpu_index);
static ServerError
static ErrorCode
GetGpuMemory(uint32_t gpu_index, size_t &memory);
};
......
......@@ -21,7 +21,7 @@ namespace engine {
using namespace zilliz::knowhere;
server::KnowhereError VecIndexImpl::BuildAll(const long &nb,
ErrorCode VecIndexImpl::BuildAll(const long &nb,
const float *xb,
const long *ids,
const Config &cfg,
......@@ -38,36 +38,36 @@ server::KnowhereError VecIndexImpl::BuildAll(const long &nb,
index_->Add(dataset, cfg);
} catch (KnowhereException &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_UNEXPECTED_ERROR;
return KNOWHERE_UNEXPECTED_ERROR;
} catch (jsoncons::json_exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_INVALID_ARGUMENT;
return KNOWHERE_INVALID_ARGUMENT;
} catch (std::exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_ERROR;
return KNOWHERE_ERROR;
}
return server::KNOWHERE_SUCCESS;
return KNOWHERE_SUCCESS;
}
server::KnowhereError VecIndexImpl::Add(const long &nb, const float *xb, const long *ids, const Config &cfg) {
ErrorCode VecIndexImpl::Add(const long &nb, const float *xb, const long *ids, const Config &cfg) {
try {
auto dataset = GenDatasetWithIds(nb, dim, xb, ids);
index_->Add(dataset, cfg);
} catch (KnowhereException &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_UNEXPECTED_ERROR;
return KNOWHERE_UNEXPECTED_ERROR;
} catch (jsoncons::json_exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_INVALID_ARGUMENT;
return KNOWHERE_INVALID_ARGUMENT;
} catch (std::exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_ERROR;
return KNOWHERE_ERROR;
}
return server::KNOWHERE_SUCCESS;
return KNOWHERE_SUCCESS;
}
server::KnowhereError VecIndexImpl::Search(const long &nq, const float *xq, float *dist, long *ids, const Config &cfg) {
ErrorCode VecIndexImpl::Search(const long &nq, const float *xq, float *dist, long *ids, const Config &cfg) {
try {
auto k = cfg["k"].as<int>();
auto dataset = GenDataset(nq, dim, xq);
......@@ -102,15 +102,15 @@ server::KnowhereError VecIndexImpl::Search(const long &nq, const float *xq, floa
memcpy(dist, p_dist, sizeof(float) * nq * k);
} catch (KnowhereException &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_UNEXPECTED_ERROR;
return KNOWHERE_UNEXPECTED_ERROR;
} catch (jsoncons::json_exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_INVALID_ARGUMENT;
return KNOWHERE_INVALID_ARGUMENT;
} catch (std::exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_ERROR;
return KNOWHERE_ERROR;
}
return server::KNOWHERE_SUCCESS;
return KNOWHERE_SUCCESS;
}
zilliz::knowhere::BinarySet VecIndexImpl::Serialize() {
......@@ -118,10 +118,10 @@ zilliz::knowhere::BinarySet VecIndexImpl::Serialize() {
return index_->Serialize();
}
server::KnowhereError VecIndexImpl::Load(const zilliz::knowhere::BinarySet &index_binary) {
ErrorCode VecIndexImpl::Load(const zilliz::knowhere::BinarySet &index_binary) {
index_->Load(index_binary);
dim = Dimension();
return server::KNOWHERE_SUCCESS;
return KNOWHERE_SUCCESS;
}
int64_t VecIndexImpl::Dimension() {
......@@ -177,24 +177,24 @@ int64_t *BFIndex::GetRawIds() {
return std::static_pointer_cast<IDMAP>(index_)->GetRawIds();
}
server::KnowhereError BFIndex::Build(const Config &cfg) {
ErrorCode BFIndex::Build(const Config &cfg) {
try {
dim = cfg["dim"].as<int>();
std::static_pointer_cast<IDMAP>(index_)->Train(cfg);
} catch (KnowhereException &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_UNEXPECTED_ERROR;
return KNOWHERE_UNEXPECTED_ERROR;
} catch (jsoncons::json_exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_INVALID_ARGUMENT;
return KNOWHERE_INVALID_ARGUMENT;
} catch (std::exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_ERROR;
return KNOWHERE_ERROR;
}
return server::KNOWHERE_SUCCESS;
return KNOWHERE_SUCCESS;
}
server::KnowhereError BFIndex::BuildAll(const long &nb,
ErrorCode BFIndex::BuildAll(const long &nb,
const float *xb,
const long *ids,
const Config &cfg,
......@@ -208,19 +208,19 @@ server::KnowhereError BFIndex::BuildAll(const long &nb,
index_->Add(dataset, cfg);
} catch (KnowhereException &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_UNEXPECTED_ERROR;
return KNOWHERE_UNEXPECTED_ERROR;
} catch (jsoncons::json_exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_INVALID_ARGUMENT;
return KNOWHERE_INVALID_ARGUMENT;
} catch (std::exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_ERROR;
return KNOWHERE_ERROR;
}
return server::KNOWHERE_SUCCESS;
return KNOWHERE_SUCCESS;
}
// TODO(linxj): add lock here.
server::KnowhereError IVFMixIndex::BuildAll(const long &nb,
ErrorCode IVFMixIndex::BuildAll(const long &nb,
const float *xb,
const long *ids,
const Config &cfg,
......@@ -242,26 +242,26 @@ server::KnowhereError IVFMixIndex::BuildAll(const long &nb,
type = ConvertToCpuIndexType(type);
} else {
WRAPPER_LOG_ERROR << "Build IVFMIXIndex Failed";
return server::KNOWHERE_ERROR;
return KNOWHERE_ERROR;
}
} catch (KnowhereException &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_UNEXPECTED_ERROR;
return KNOWHERE_UNEXPECTED_ERROR;
} catch (jsoncons::json_exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_INVALID_ARGUMENT;
return KNOWHERE_INVALID_ARGUMENT;
} catch (std::exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_ERROR;
return KNOWHERE_ERROR;
}
return server::KNOWHERE_SUCCESS;
return KNOWHERE_SUCCESS;
}
server::KnowhereError IVFMixIndex::Load(const zilliz::knowhere::BinarySet &index_binary) {
ErrorCode IVFMixIndex::Load(const zilliz::knowhere::BinarySet &index_binary) {
//index_ = std::make_shared<IVF>();
index_->Load(index_binary);
dim = Dimension();
return server::KNOWHERE_SUCCESS;
return KNOWHERE_SUCCESS;
}
}
......
......@@ -19,7 +19,7 @@ class VecIndexImpl : public VecIndex {
public:
explicit VecIndexImpl(std::shared_ptr<zilliz::knowhere::VectorIndex> index, const IndexType &type)
: index_(std::move(index)), type(type) {};
server::KnowhereError BuildAll(const long &nb,
ErrorCode BuildAll(const long &nb,
const float *xb,
const long *ids,
const Config &cfg,
......@@ -30,12 +30,12 @@ class VecIndexImpl : public VecIndex {
IndexType GetType() override;
int64_t Dimension() override;
int64_t Count() override;
server::KnowhereError Add(const long &nb, const float *xb, const long *ids, const Config &cfg) override;
ErrorCode Add(const long &nb, const float *xb, const long *ids, const Config &cfg) override;
zilliz::knowhere::BinarySet Serialize() override;
server::KnowhereError Load(const zilliz::knowhere::BinarySet &index_binary) override;
ErrorCode Load(const zilliz::knowhere::BinarySet &index_binary) override;
VecIndexPtr Clone() override;
int64_t GetDeviceId() override;
server::KnowhereError Search(const long &nq, const float *xq, float *dist, long *ids, const Config &cfg) override;
ErrorCode Search(const long &nq, const float *xq, float *dist, long *ids, const Config &cfg) override;
protected:
int64_t dim = 0;
......@@ -48,22 +48,22 @@ class IVFMixIndex : public VecIndexImpl {
explicit IVFMixIndex(std::shared_ptr<zilliz::knowhere::VectorIndex> index, const IndexType &type)
: VecIndexImpl(std::move(index), type) {};
server::KnowhereError BuildAll(const long &nb,
ErrorCode BuildAll(const long &nb,
const float *xb,
const long *ids,
const Config &cfg,
const long &nt,
const float *xt) override;
server::KnowhereError Load(const zilliz::knowhere::BinarySet &index_binary) override;
ErrorCode Load(const zilliz::knowhere::BinarySet &index_binary) override;
};
class BFIndex : public VecIndexImpl {
public:
explicit BFIndex(std::shared_ptr<zilliz::knowhere::VectorIndex> index) : VecIndexImpl(std::move(index),
IndexType::FAISS_IDMAP) {};
server::KnowhereError Build(const Config& cfg);
ErrorCode Build(const Config& cfg);
float *GetRawVectors();
server::KnowhereError BuildAll(const long &nb,
ErrorCode BuildAll(const long &nb,
const float *xb,
const long *ids,
const Config &cfg,
......
......@@ -172,7 +172,7 @@ VecIndexPtr read_index(const std::string &location) {
return LoadVecIndex(current_type, load_data_list);
}
server::KnowhereError write_index(VecIndexPtr index, const std::string &location) {
ErrorCode write_index(VecIndexPtr index, const std::string &location) {
try {
auto binaryset = index->Serialize();
auto index_type = index->GetType();
......@@ -192,12 +192,12 @@ server::KnowhereError write_index(VecIndexPtr index, const std::string &location
}
} catch (knowhere::KnowhereException &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_UNEXPECTED_ERROR;
return KNOWHERE_UNEXPECTED_ERROR;
} catch (std::exception &e) {
WRAPPER_LOG_ERROR << e.what();
return server::KNOWHERE_ERROR;
return KNOWHERE_ERROR;
}
return server::KNOWHERE_SUCCESS;
return KNOWHERE_SUCCESS;
}
......
......@@ -42,23 +42,23 @@ using VecIndexPtr = std::shared_ptr<VecIndex>;
class VecIndex {
public:
virtual server::KnowhereError BuildAll(const long &nb,
const float *xb,
const long *ids,
const Config &cfg,
const long &nt = 0,
const float *xt = nullptr) = 0;
virtual server::KnowhereError Add(const long &nb,
const float *xb,
const long *ids,
const Config &cfg = Config()) = 0;
virtual server::KnowhereError Search(const long &nq,
const float *xq,
float *dist,
long *ids,
const Config &cfg = Config()) = 0;
virtual ErrorCode BuildAll(const long &nb,
const float *xb,
const long *ids,
const Config &cfg,
const long &nt = 0,
const float *xt = nullptr) = 0;
virtual ErrorCode Add(const long &nb,
const float *xb,
const long *ids,
const Config &cfg = Config()) = 0;
virtual ErrorCode Search(const long &nq,
const float *xq,
float *dist,
long *ids,
const Config &cfg = Config()) = 0;
virtual VecIndexPtr CopyToGpu(const int64_t& device_id,
const Config &cfg = Config()) = 0;
......@@ -77,10 +77,10 @@ class VecIndex {
virtual zilliz::knowhere::BinarySet Serialize() = 0;
virtual server::KnowhereError Load(const zilliz::knowhere::BinarySet &index_binary) = 0;
virtual ErrorCode Load(const zilliz::knowhere::BinarySet &index_binary) = 0;
};
extern server::KnowhereError write_index(VecIndexPtr index, const std::string &location);
extern ErrorCode write_index(VecIndexPtr index, const std::string &location);
extern VecIndexPtr read_index(const std::string &location);
......
......@@ -30,12 +30,12 @@ public:
class MockVecIndex : public engine::VecIndex {
public:
virtual server::KnowhereError BuildAll(const long &nb,
const float *xb,
const long *ids,
const engine::Config &cfg,
const long &nt = 0,
const float *xt = nullptr) {
virtual ErrorCode BuildAll(const long &nb,
const float *xb,
const long *ids,
const engine::Config &cfg,
const long &nt = 0,
const float *xt = nullptr) {
}
......@@ -51,18 +51,18 @@ public:
return engine::IndexType::INVALID;
}
virtual server::KnowhereError Add(const long &nb,
const float *xb,
const long *ids,
const engine::Config &cfg = engine::Config()) {
virtual ErrorCode Add(const long &nb,
const float *xb,
const long *ids,
const engine::Config &cfg = engine::Config()) {
}
virtual server::KnowhereError Search(const long &nq,
const float *xq,
float *dist,
long *ids,
const engine::Config &cfg = engine::Config()) {
virtual ErrorCode Search(const long &nq,
const float *xq,
float *dist,
long *ids,
const engine::Config &cfg = engine::Config()) {
}
......@@ -87,7 +87,7 @@ public:
return binset;
}
virtual server::KnowhereError Load(const zilliz::knowhere::BinarySet &index_binary) {
virtual ErrorCode Load(const zilliz::knowhere::BinarySet &index_binary) {
}
......
......@@ -27,14 +27,14 @@ static constexpr uint64_t GB = MB*1024;
TEST(ConfigTest, CONFIG_TEST) {
server::ConfigMgr* config_mgr = server::ConfigMgr::GetInstance();
server::ServerError err = config_mgr->LoadConfigFile("");
ASSERT_EQ(err, server::SERVER_UNEXPECTED_ERROR);
ErrorCode err = config_mgr->LoadConfigFile("");
ASSERT_EQ(err, SERVER_UNEXPECTED_ERROR);
err = config_mgr->LoadConfigFile(LOG_FILE_PATH);
ASSERT_EQ(err, server::SERVER_UNEXPECTED_ERROR);
ASSERT_EQ(err, SERVER_UNEXPECTED_ERROR);
err = config_mgr->LoadConfigFile(CONFIG_FILE_PATH);
ASSERT_EQ(err, server::SERVER_SUCCESS);
ASSERT_EQ(err, SERVER_SUCCESS);
config_mgr->Print();
......@@ -95,11 +95,11 @@ TEST(ConfigTest, CONFIG_TEST) {
TEST(ConfigTest, SERVER_CONFIG_TEST) {
server::ServerConfig& config = server::ServerConfig::GetInstance();
server::ServerError err = config.LoadConfigFile(CONFIG_FILE_PATH);
ASSERT_EQ(err, server::SERVER_SUCCESS);
ErrorCode err = config.LoadConfigFile(CONFIG_FILE_PATH);
ASSERT_EQ(err, SERVER_SUCCESS);
err = server::ServerConfig::GetInstance().ValidateConfig();
ASSERT_EQ(err, server::SERVER_SUCCESS);
ASSERT_EQ(err, SERVER_SUCCESS);
server::ConfigNode node1 = config.GetConfig("server_config");
server::ConfigNode& node2 = config.GetConfig("cache_config");
......@@ -125,26 +125,26 @@ TEST(ConfigTest, SERVER_CONFIG_TEST) {
server::ConfigNode& cache_config = config.GetConfig(server::CONFIG_CACHE);
cache_config.SetValue(server::CACHE_FREE_PERCENT, "2.0");
err = config.ValidateConfig();
ASSERT_NE(err, server::SERVER_SUCCESS);
ASSERT_NE(err, SERVER_SUCCESS);
size_t cache_cap = 16;
size_t insert_buffer_size = (total_mem - cache_cap*GB + 1*GB)/GB;
db_config.SetValue(server::CONFIG_DB_INSERT_BUFFER_SIZE, std::to_string(insert_buffer_size));
cache_config.SetValue(server::CONFIG_CPU_CACHE_CAPACITY, std::to_string(cache_cap));
err = config.ValidateConfig();
ASSERT_NE(err, server::SERVER_SUCCESS);
ASSERT_NE(err, SERVER_SUCCESS);
cache_cap = total_mem/GB + 2;
cache_config.SetValue(server::CONFIG_CPU_CACHE_CAPACITY, std::to_string(cache_cap));
err = config.ValidateConfig();
ASSERT_NE(err, server::SERVER_SUCCESS);
ASSERT_NE(err, SERVER_SUCCESS);
insert_buffer_size = total_mem/GB + 2;
db_config.SetValue(server::CONFIG_DB_INSERT_BUFFER_SIZE, std::to_string(insert_buffer_size));
err = config.ValidateConfig();
ASSERT_NE(err, server::SERVER_SUCCESS);
ASSERT_NE(err, SERVER_SUCCESS);
server_config.SetValue(server::CONFIG_GPU_INDEX, "9999");
err = config.ValidateConfig();
ASSERT_NE(err, server::SERVER_SUCCESS);
ASSERT_NE(err, SERVER_SUCCESS);
}
\ No newline at end of file
......@@ -25,8 +25,8 @@ static const char* LOG_FILE_PATH = "./milvus/conf/log_config.conf";
TEST(UtilTest, EXCEPTION_TEST) {
std::string err_msg = "failed";
server::ServerException ex(server::SERVER_UNEXPECTED_ERROR, err_msg);
ASSERT_EQ(ex.error_code(), server::SERVER_UNEXPECTED_ERROR);
server::ServerException ex(SERVER_UNEXPECTED_ERROR, err_msg);
ASSERT_EQ(ex.error_code(), SERVER_UNEXPECTED_ERROR);
std::string msg = ex.what();
ASSERT_EQ(msg, err_msg);
}
......@@ -44,19 +44,19 @@ TEST(UtilTest, COMMON_TEST) {
std::string path1 = "/tmp/milvus_test/";
std::string path2 = path1 + "common_test_12345/";
std::string path3 = path2 + "abcdef";
server::ServerError err = server::CommonUtil::CreateDirectory(path3);
ASSERT_EQ(err, server::SERVER_SUCCESS);
ErrorCode err = server::CommonUtil::CreateDirectory(path3);
ASSERT_EQ(err, SERVER_SUCCESS);
//test again
err = server::CommonUtil::CreateDirectory(path3);
ASSERT_EQ(err, server::SERVER_SUCCESS);
ASSERT_EQ(err, SERVER_SUCCESS);
ASSERT_TRUE(server::CommonUtil::IsDirectoryExist(path3));
err = server::CommonUtil::DeleteDirectory(path1);
ASSERT_EQ(err, server::SERVER_SUCCESS);
ASSERT_EQ(err, SERVER_SUCCESS);
//test again
err = server::CommonUtil::DeleteDirectory(path1);
ASSERT_EQ(err, server::SERVER_SUCCESS);
ASSERT_EQ(err, SERVER_SUCCESS);
ASSERT_FALSE(server::CommonUtil::IsDirectoryExist(path1));
ASSERT_FALSE(server::CommonUtil::IsFileExist(path1));
......@@ -94,24 +94,24 @@ TEST(UtilTest, STRINGFUNCTIONS_TEST) {
str = "a,b,c";
std::vector<std::string> result;
server::ServerError err = server::StringHelpFunctions::SplitStringByDelimeter(str , ",", result);
ASSERT_EQ(err, server::SERVER_SUCCESS);
ErrorCode err = server::StringHelpFunctions::SplitStringByDelimeter(str , ",", result);
ASSERT_EQ(err, SERVER_SUCCESS);
ASSERT_EQ(result.size(), 3UL);
result.clear();
err = server::StringHelpFunctions::SplitStringByQuote(str , ",", "\"", result);
ASSERT_EQ(err, server::SERVER_SUCCESS);
ASSERT_EQ(err, SERVER_SUCCESS);
ASSERT_EQ(result.size(), 3UL);
result.clear();
err = server::StringHelpFunctions::SplitStringByQuote(str , ",", "", result);
ASSERT_EQ(err, server::SERVER_SUCCESS);
ASSERT_EQ(err, SERVER_SUCCESS);
ASSERT_EQ(result.size(), 3UL);
str = "55,\"aa,gg,yy\",b";
result.clear();
err = server::StringHelpFunctions::SplitStringByQuote(str , ",", "\"", result);
ASSERT_EQ(err, server::SERVER_SUCCESS);
ASSERT_EQ(err, SERVER_SUCCESS);
ASSERT_EQ(result.size(), 3UL);
......@@ -159,86 +159,86 @@ TEST(UtilTest, LOG_TEST) {
TEST(UtilTest, VALIDATE_TABLENAME_TEST) {
std::string table_name = "Normal123_";
server:: ServerError res = server::ValidationUtil::ValidateTableName(table_name);
ASSERT_EQ(res, server::SERVER_SUCCESS);
ErrorCode res = server::ValidationUtil::ValidateTableName(table_name);
ASSERT_EQ(res, SERVER_SUCCESS);
table_name = "12sds";
res = server::ValidationUtil::ValidateTableName(table_name);
ASSERT_EQ(res, server::SERVER_INVALID_TABLE_NAME);
ASSERT_EQ(res, SERVER_INVALID_TABLE_NAME);
table_name = "";
res = server::ValidationUtil::ValidateTableName(table_name);
ASSERT_EQ(res, server::SERVER_INVALID_TABLE_NAME);
ASSERT_EQ(res, SERVER_INVALID_TABLE_NAME);
table_name = "_asdasd";
res = server::ValidationUtil::ValidateTableName(table_name);
ASSERT_EQ(res, server::SERVER_SUCCESS);
ASSERT_EQ(res, SERVER_SUCCESS);
table_name = "!@#!@";
res = server::ValidationUtil::ValidateTableName(table_name);
ASSERT_EQ(res, server::SERVER_INVALID_TABLE_NAME);
ASSERT_EQ(res, SERVER_INVALID_TABLE_NAME);
table_name = "_!@#!@";
res = server::ValidationUtil::ValidateTableName(table_name);
ASSERT_EQ(res, server::SERVER_INVALID_TABLE_NAME);
ASSERT_EQ(res, SERVER_INVALID_TABLE_NAME);
table_name = "中文";
res = server::ValidationUtil::ValidateTableName(table_name);
ASSERT_EQ(res, server::SERVER_INVALID_TABLE_NAME);
ASSERT_EQ(res, SERVER_INVALID_TABLE_NAME);
table_name = std::string(10000, 'a');
res = server::ValidationUtil::ValidateTableName(table_name);
ASSERT_EQ(res, server::SERVER_INVALID_TABLE_NAME);
ASSERT_EQ(res, SERVER_INVALID_TABLE_NAME);
}
TEST(UtilTest, VALIDATE_DIMENSIONTEST) {
ASSERT_EQ(server::ValidationUtil::ValidateTableDimension(-1), server::SERVER_INVALID_VECTOR_DIMENSION);
ASSERT_EQ(server::ValidationUtil::ValidateTableDimension(0), server::SERVER_INVALID_VECTOR_DIMENSION);
ASSERT_EQ(server::ValidationUtil::ValidateTableDimension(16385), server::SERVER_INVALID_VECTOR_DIMENSION);
ASSERT_EQ(server::ValidationUtil::ValidateTableDimension(16384), server::SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateTableDimension(1), server::SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateTableDimension(-1), SERVER_INVALID_VECTOR_DIMENSION);
ASSERT_EQ(server::ValidationUtil::ValidateTableDimension(0), SERVER_INVALID_VECTOR_DIMENSION);
ASSERT_EQ(server::ValidationUtil::ValidateTableDimension(16385), SERVER_INVALID_VECTOR_DIMENSION);
ASSERT_EQ(server::ValidationUtil::ValidateTableDimension(16384), SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateTableDimension(1), SERVER_SUCCESS);
}
TEST(UtilTest, VALIDATE_INDEX_TEST) {
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexType((int)engine::EngineType::INVALID), server::SERVER_INVALID_INDEX_TYPE);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexType((int)engine::EngineType::INVALID), SERVER_INVALID_INDEX_TYPE);
for(int i = 1; i <= (int)engine::EngineType::MAX_VALUE; i++) {
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexType(i), server::SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexType(i), SERVER_SUCCESS);
}
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexType((int)engine::EngineType::MAX_VALUE + 1), server::SERVER_INVALID_INDEX_TYPE);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexType((int)engine::EngineType::MAX_VALUE + 1), SERVER_INVALID_INDEX_TYPE);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexNlist(0), server::SERVER_INVALID_INDEX_NLIST);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexNlist(100), server::SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexNlist(0), SERVER_INVALID_INDEX_NLIST);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexNlist(100), SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexFileSize(0), server::SERVER_INVALID_INDEX_FILE_SIZE);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexFileSize(100), server::SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexFileSize(0), SERVER_INVALID_INDEX_FILE_SIZE);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexFileSize(100), SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexMetricType(0), server::SERVER_INVALID_INDEX_METRIC_TYPE);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexMetricType(1), server::SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexMetricType(2), server::SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexMetricType(0), SERVER_INVALID_INDEX_METRIC_TYPE);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexMetricType(1), SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateTableIndexMetricType(2), SERVER_SUCCESS);
}
TEST(ValidationUtilTest, ValidateTopkTest) {
engine::meta::TableSchema schema;
ASSERT_EQ(server::ValidationUtil::ValidateSearchTopk(10, schema), server::SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::ValidateSearchTopk(65536, schema), server::SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::ValidateSearchTopk(0, schema), server::SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateSearchTopk(10, schema), SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::ValidateSearchTopk(65536, schema), SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::ValidateSearchTopk(0, schema), SERVER_SUCCESS);
}
TEST(ValidationUtilTest, ValidateNprobeTest) {
engine::meta::TableSchema schema;
schema.nlist_ = 100;
ASSERT_EQ(server::ValidationUtil::ValidateSearchNprobe(10, schema), server::SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::ValidateSearchNprobe(0, schema), server::SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::ValidateSearchNprobe(101, schema), server::SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateSearchNprobe(10, schema), SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::ValidateSearchNprobe(0, schema), SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::ValidateSearchNprobe(101, schema), SERVER_SUCCESS);
}
TEST(ValidationUtilTest, ValidateGpuTest) {
ASSERT_EQ(server::ValidationUtil::ValidateGpuIndex(0), server::SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::ValidateGpuIndex(100), server::SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::ValidateGpuIndex(0), SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::ValidateGpuIndex(100), SERVER_SUCCESS);
size_t memory = 0;
ASSERT_EQ(server::ValidationUtil::GetGpuMemory(0, memory), server::SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::GetGpuMemory(100, memory), server::SERVER_SUCCESS);
ASSERT_EQ(server::ValidationUtil::GetGpuMemory(0, memory), SERVER_SUCCESS);
ASSERT_NE(server::ValidationUtil::GetGpuMemory(100, memory), SERVER_SUCCESS);
}
TEST(UtilTest, TIMERECORDER_TEST) {
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册