mirror of https://github.com/milvus-io/milvus.git
Format Code and duplicate class Segment
Signed-off-by: FluorineDog <guilin.gou@zilliz.com>pull/4973/head^2
parent
1b31b85ef7
commit
a48ca80286
|
@ -0,0 +1,15 @@
|
|||
if [ -z $1 ]; then
|
||||
echo "usage: $0 <path_to_core>"
|
||||
exit -1
|
||||
else
|
||||
echo start formating
|
||||
fi
|
||||
CorePath=$1
|
||||
|
||||
formatThis() {
|
||||
find "$1" | grep -E "(*\.cpp|*\.h|*\.cc)$" | grep -v "/thirdparty" | grep -v "\.pb\." | xargs clang-format -i
|
||||
}
|
||||
|
||||
formatThis "${CorePath}/src"
|
||||
formatThis "${CorePath}/unittest"
|
||||
|
|
@ -3,6 +3,7 @@ aux_source_directory(${CMAKE_CURRENT_SOURCE_DIR}/../pb PB_SRC_FILES)
|
|||
# add_definitions(-DBOOST_STACKTRACE_USE_ADDR2LINE)
|
||||
set(DOG_SEGMENT_FILES
|
||||
SegmentNaive.cpp
|
||||
SegmentSmallIndex.cpp
|
||||
IndexMeta.cpp
|
||||
ConcurrentVector.cpp
|
||||
Collection.cpp
|
||||
|
@ -11,6 +12,7 @@ set(DOG_SEGMENT_FILES
|
|||
partition_c.cpp
|
||||
segment_c.cpp
|
||||
EasyAssert.cpp
|
||||
SegmentBase.cpp
|
||||
${PB_SRC_FILES}
|
||||
)
|
||||
add_library(milvus_dog_segment SHARED
|
||||
|
|
|
@ -1,5 +1,7 @@
|
|||
#include "Collection.h"
|
||||
#include "pb/master.pb.h"
|
||||
#include "pb/common.pb.h"
|
||||
#include "pb/schema.pb.h"
|
||||
#include "pb/etcd_meta.pb.h"
|
||||
#include "pb/message.pb.h"
|
||||
#include <google/protobuf/text_format.h>
|
||||
#include <knowhere/index/vector_index/adapter/VectorAdapter.h>
|
||||
|
@ -91,8 +93,8 @@ Collection::CreateIndex(std::string& index_config) {
|
|||
return;
|
||||
}
|
||||
|
||||
masterpb::Collection collection;
|
||||
auto suc = google::protobuf::TextFormat::ParseFromString(index_config, &collection);
|
||||
milvus::proto::etcd::CollectionMeta collection_meta;
|
||||
auto suc = google::protobuf::TextFormat::ParseFromString(index_config, &collection_meta);
|
||||
|
||||
if (!suc) {
|
||||
std::cerr << "unmarshal index string failed" << std::endl;
|
||||
|
@ -100,11 +102,11 @@ Collection::CreateIndex(std::string& index_config) {
|
|||
|
||||
index_ = std::make_shared<IndexMeta>(schema_);
|
||||
|
||||
for (const auto& index : collection.indexes()) {
|
||||
std::cout << "add index, index name =" << index.index_name() << ", field_name = " << index.field_name()
|
||||
<< std::endl;
|
||||
AddIndex(index);
|
||||
}
|
||||
// for (const auto& index : collection_meta.indexes()) {
|
||||
// std::cout << "add index, index name =" << index.index_name() << ", field_name = " << index.field_name()
|
||||
// << std::endl;
|
||||
// AddIndex(index);
|
||||
// }
|
||||
}
|
||||
|
||||
void
|
||||
|
@ -118,17 +120,24 @@ Collection::parse() {
|
|||
return;
|
||||
}
|
||||
|
||||
masterpb::Collection collection;
|
||||
auto suc = google::protobuf::TextFormat::ParseFromString(schema_json_, &collection);
|
||||
milvus::proto::etcd::CollectionMeta collection_meta;
|
||||
auto suc = google::protobuf::TextFormat::ParseFromString(schema_json_, &collection_meta);
|
||||
|
||||
if (!suc) {
|
||||
std::cerr << "unmarshal schema string failed" << std::endl;
|
||||
}
|
||||
auto schema = std::make_shared<Schema>();
|
||||
for (const milvus::grpc::FieldMeta& child : collection.schema().field_metas()) {
|
||||
std::cout << "add Field, name :" << child.field_name() << ", datatype :" << child.type()
|
||||
<< ", dim :" << int(child.dim()) << std::endl;
|
||||
schema->AddField(std::string_view(child.field_name()), DataType{child.type()}, int(child.dim()));
|
||||
for (const milvus::proto::schema::FieldSchema& child : collection_meta.schema().fields()) {
|
||||
const auto & type_params = child.type_params();
|
||||
int dim = 16;
|
||||
for (const auto & type_param: type_params){
|
||||
if(type_param.key() == "dim"){
|
||||
// dim = type_param.value();
|
||||
}
|
||||
}
|
||||
std::cout << "add Field, name :" << child.name() << ", datatype :" << child.data_type()
|
||||
<< ", dim :" << dim << std::endl;
|
||||
schema->AddField(std::string_view(child.name()), DataType(child.data_type()), dim);
|
||||
}
|
||||
/*
|
||||
schema->AddField("fakevec", DataType::VECTOR_FLOAT, 16);
|
||||
|
|
|
@ -50,7 +50,7 @@ struct DeletedRecord {
|
|||
std::shared_mutex shared_mutex_;
|
||||
};
|
||||
|
||||
auto
|
||||
inline auto
|
||||
DeletedRecord::TmpBitmap::clone(int64_t capacity) -> std::shared_ptr<TmpBitmap> {
|
||||
auto res = std::make_shared<TmpBitmap>();
|
||||
res->del_barrier = this->del_barrier;
|
||||
|
|
|
@ -0,0 +1,41 @@
|
|||
#include "SegmentNaive.h"
|
||||
#include "SegmentSmallIndex.h"
|
||||
|
||||
namespace milvus::dog_segment {
|
||||
|
||||
// seems to be deprecated
|
||||
struct ColumnBasedDataChunk {
|
||||
std::vector<std::vector<float>> entity_vecs;
|
||||
|
||||
static ColumnBasedDataChunk
|
||||
from(const DogDataChunk& source, const Schema& schema) {
|
||||
ColumnBasedDataChunk dest;
|
||||
auto count = source.count;
|
||||
auto raw_data = reinterpret_cast<const char*>(source.raw_data);
|
||||
auto align = source.sizeof_per_row;
|
||||
for (auto& field : schema) {
|
||||
auto len = field.get_sizeof();
|
||||
Assert(len % sizeof(float) == 0);
|
||||
std::vector<float> new_col(len * count / sizeof(float));
|
||||
for (int64_t i = 0; i < count; ++i) {
|
||||
memcpy(new_col.data() + i * len / sizeof(float), raw_data + i * align, len);
|
||||
}
|
||||
dest.entity_vecs.push_back(std::move(new_col));
|
||||
// offset the raw_data
|
||||
raw_data += len / sizeof(float);
|
||||
}
|
||||
return dest;
|
||||
}
|
||||
};
|
||||
|
||||
int
|
||||
TestABI() {
|
||||
return 42;
|
||||
}
|
||||
|
||||
std::unique_ptr<SegmentBase>
|
||||
CreateSegment(SchemaPtr schema) {
|
||||
auto segment = std::make_unique<SegmentSmallIndex>(schema);
|
||||
return segment;
|
||||
}
|
||||
} // namespace milvus::dog_segment
|
|
@ -10,17 +10,6 @@
|
|||
#include <faiss/utils/distances.h>
|
||||
|
||||
namespace milvus::dog_segment {
|
||||
int
|
||||
TestABI() {
|
||||
return 42;
|
||||
}
|
||||
|
||||
std::unique_ptr<SegmentBase>
|
||||
CreateSegment(SchemaPtr schema) {
|
||||
auto segment = std::make_unique<SegmentNaive>(schema);
|
||||
return segment;
|
||||
}
|
||||
|
||||
SegmentNaive::Record::Record(const Schema& schema) : uids_(1), timestamps_(1) {
|
||||
for (auto& field : schema) {
|
||||
if (field.is_vector()) {
|
||||
|
@ -317,7 +306,7 @@ SegmentNaive::QueryImpl(query::QueryPtr query_info, Timestamp timestamp, QueryRe
|
|||
return Status::OK();
|
||||
}
|
||||
|
||||
void
|
||||
static void
|
||||
merge_into(int64_t queries,
|
||||
int64_t topk,
|
||||
float* distances,
|
||||
|
|
|
@ -17,30 +17,6 @@
|
|||
#include "EasyAssert.h"
|
||||
|
||||
namespace milvus::dog_segment {
|
||||
struct ColumnBasedDataChunk {
|
||||
std::vector<std::vector<float>> entity_vecs;
|
||||
|
||||
static ColumnBasedDataChunk
|
||||
from(const DogDataChunk& source, const Schema& schema) {
|
||||
ColumnBasedDataChunk dest;
|
||||
auto count = source.count;
|
||||
auto raw_data = reinterpret_cast<const char*>(source.raw_data);
|
||||
auto align = source.sizeof_per_row;
|
||||
for (auto& field : schema) {
|
||||
auto len = field.get_sizeof();
|
||||
Assert(len % sizeof(float) == 0);
|
||||
std::vector<float> new_col(len * count / sizeof(float));
|
||||
for (int64_t i = 0; i < count; ++i) {
|
||||
memcpy(new_col.data() + i * len / sizeof(float), raw_data + i * align, len);
|
||||
}
|
||||
dest.entity_vecs.push_back(std::move(new_col));
|
||||
// offset the raw_data
|
||||
raw_data += len / sizeof(float);
|
||||
}
|
||||
return dest;
|
||||
}
|
||||
};
|
||||
|
||||
class SegmentNaive : public SegmentBase {
|
||||
public:
|
||||
virtual ~SegmentNaive() = default;
|
||||
|
|
|
@ -0,0 +1,622 @@
|
|||
#include <dog_segment/SegmentSmallIndex.h>
|
||||
#include <random>
|
||||
#include <algorithm>
|
||||
#include <numeric>
|
||||
#include <thread>
|
||||
#include <queue>
|
||||
|
||||
#include <knowhere/index/vector_index/adapter/VectorAdapter.h>
|
||||
#include <knowhere/index/vector_index/VecIndexFactory.h>
|
||||
#include <faiss/utils/distances.h>
|
||||
|
||||
namespace milvus::dog_segment {
|
||||
|
||||
SegmentSmallIndex::Record::Record(const Schema& schema) : uids_(1), timestamps_(1) {
|
||||
for (auto& field : schema) {
|
||||
if (field.is_vector()) {
|
||||
Assert(field.get_data_type() == DataType::VECTOR_FLOAT);
|
||||
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<float>>(field.get_dim()));
|
||||
} else {
|
||||
Assert(field.get_data_type() == DataType::INT32);
|
||||
entity_vec_.emplace_back(std::make_shared<ConcurrentVector<int32_t, true>>());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
int64_t
|
||||
SegmentSmallIndex::PreInsert(int64_t size) {
|
||||
auto reserved_begin = record_.reserved.fetch_add(size);
|
||||
return reserved_begin;
|
||||
}
|
||||
|
||||
int64_t
|
||||
SegmentSmallIndex::PreDelete(int64_t size) {
|
||||
auto reserved_begin = deleted_record_.reserved.fetch_add(size);
|
||||
return reserved_begin;
|
||||
}
|
||||
|
||||
auto
|
||||
SegmentSmallIndex::get_deleted_bitmap(int64_t del_barrier,
|
||||
Timestamp query_timestamp,
|
||||
int64_t insert_barrier,
|
||||
bool force) -> std::shared_ptr<DeletedRecord::TmpBitmap> {
|
||||
auto old = deleted_record_.get_lru_entry();
|
||||
|
||||
if (!force || old->bitmap_ptr->count() == insert_barrier) {
|
||||
if (old->del_barrier == del_barrier) {
|
||||
return old;
|
||||
}
|
||||
}
|
||||
|
||||
auto current = old->clone(insert_barrier);
|
||||
current->del_barrier = del_barrier;
|
||||
|
||||
auto bitmap = current->bitmap_ptr;
|
||||
if (del_barrier < old->del_barrier) {
|
||||
for (auto del_index = del_barrier; del_index < old->del_barrier; ++del_index) {
|
||||
// get uid in delete logs
|
||||
auto uid = deleted_record_.uids_[del_index];
|
||||
// map uid to corrensponding offsets, select the max one, which should be the target
|
||||
// the max one should be closest to query_timestamp, so the delete log should refer to it
|
||||
int64_t the_offset = -1;
|
||||
auto [iter_b, iter_e] = uid2offset_.equal_range(uid);
|
||||
for (auto iter = iter_b; iter != iter_e; ++iter) {
|
||||
auto offset = iter->second;
|
||||
if (record_.timestamps_[offset] < query_timestamp) {
|
||||
Assert(offset < insert_barrier);
|
||||
the_offset = std::max(the_offset, offset);
|
||||
}
|
||||
}
|
||||
// if not found, skip
|
||||
if (the_offset == -1) {
|
||||
continue;
|
||||
}
|
||||
// otherwise, clear the flag
|
||||
bitmap->clear(the_offset);
|
||||
}
|
||||
return current;
|
||||
} else {
|
||||
for (auto del_index = old->del_barrier; del_index < del_barrier; ++del_index) {
|
||||
// get uid in delete logs
|
||||
auto uid = deleted_record_.uids_[del_index];
|
||||
// map uid to corrensponding offsets, select the max one, which should be the target
|
||||
// the max one should be closest to query_timestamp, so the delete log should refer to it
|
||||
int64_t the_offset = -1;
|
||||
auto [iter_b, iter_e] = uid2offset_.equal_range(uid);
|
||||
for (auto iter = iter_b; iter != iter_e; ++iter) {
|
||||
auto offset = iter->second;
|
||||
if (offset >= insert_barrier) {
|
||||
continue;
|
||||
}
|
||||
if (record_.timestamps_[offset] < query_timestamp) {
|
||||
Assert(offset < insert_barrier);
|
||||
the_offset = std::max(the_offset, offset);
|
||||
}
|
||||
}
|
||||
|
||||
// if not found, skip
|
||||
if (the_offset == -1) {
|
||||
continue;
|
||||
}
|
||||
|
||||
// otherwise, set the flag
|
||||
bitmap->set(the_offset);
|
||||
}
|
||||
this->deleted_record_.insert_lru_entry(current);
|
||||
}
|
||||
return current;
|
||||
}
|
||||
|
||||
Status
|
||||
SegmentSmallIndex::Insert(int64_t reserved_begin,
|
||||
int64_t size,
|
||||
const int64_t* uids_raw,
|
||||
const Timestamp* timestamps_raw,
|
||||
const DogDataChunk& entities_raw) {
|
||||
Assert(entities_raw.count == size);
|
||||
if (entities_raw.sizeof_per_row != schema_->get_total_sizeof()) {
|
||||
std::string msg = "entity length = " + std::to_string(entities_raw.sizeof_per_row) +
|
||||
", schema length = " + std::to_string(schema_->get_total_sizeof());
|
||||
throw std::runtime_error(msg);
|
||||
}
|
||||
|
||||
auto raw_data = reinterpret_cast<const char*>(entities_raw.raw_data);
|
||||
// std::vector<char> entities(raw_data, raw_data + size * len_per_row);
|
||||
|
||||
auto len_per_row = entities_raw.sizeof_per_row;
|
||||
std::vector<std::tuple<Timestamp, idx_t, int64_t>> ordering;
|
||||
ordering.resize(size);
|
||||
// #pragma omp parallel for
|
||||
for (int i = 0; i < size; ++i) {
|
||||
ordering[i] = std::make_tuple(timestamps_raw[i], uids_raw[i], i);
|
||||
}
|
||||
std::sort(ordering.begin(), ordering.end());
|
||||
auto sizeof_infos = schema_->get_sizeof_infos();
|
||||
std::vector<int> offset_infos(schema_->size() + 1, 0);
|
||||
std::partial_sum(sizeof_infos.begin(), sizeof_infos.end(), offset_infos.begin() + 1);
|
||||
std::vector<std::vector<char>> entities(schema_->size());
|
||||
|
||||
for (int fid = 0; fid < schema_->size(); ++fid) {
|
||||
auto len = sizeof_infos[fid];
|
||||
entities[fid].resize(len * size);
|
||||
}
|
||||
|
||||
std::vector<idx_t> uids(size);
|
||||
std::vector<Timestamp> timestamps(size);
|
||||
// #pragma omp parallel for
|
||||
for (int index = 0; index < size; ++index) {
|
||||
auto [t, uid, order_index] = ordering[index];
|
||||
timestamps[index] = t;
|
||||
uids[index] = uid;
|
||||
for (int fid = 0; fid < schema_->size(); ++fid) {
|
||||
auto len = sizeof_infos[fid];
|
||||
auto offset = offset_infos[fid];
|
||||
auto src = raw_data + offset + order_index * len_per_row;
|
||||
auto dst = entities[fid].data() + index * len;
|
||||
memcpy(dst, src, len);
|
||||
}
|
||||
}
|
||||
|
||||
record_.timestamps_.set_data(reserved_begin, timestamps.data(), size);
|
||||
record_.uids_.set_data(reserved_begin, uids.data(), size);
|
||||
for (int fid = 0; fid < schema_->size(); ++fid) {
|
||||
record_.entity_vec_[fid]->set_data_raw(reserved_begin, entities[fid].data(), size);
|
||||
}
|
||||
|
||||
for (int i = 0; i < uids.size(); ++i) {
|
||||
auto uid = uids[i];
|
||||
// NOTE: this must be the last step, cannot be put above
|
||||
uid2offset_.insert(std::make_pair(uid, reserved_begin + i));
|
||||
}
|
||||
|
||||
record_.ack_responder_.AddSegment(reserved_begin, reserved_begin + size);
|
||||
return Status::OK();
|
||||
|
||||
// std::thread go(executor, std::move(uids), std::move(timestamps), std::move(entities));
|
||||
// go.detach();
|
||||
// const auto& schema = *schema_;
|
||||
// auto record_ptr = GetMutableRecord();
|
||||
// Assert(record_ptr);
|
||||
// auto& record = *record_ptr;
|
||||
// auto data_chunk = ColumnBasedDataChunk::from(row_values, schema);
|
||||
//
|
||||
// // TODO: use shared_lock for better concurrency
|
||||
// std::lock_guard lck(mutex_);
|
||||
// Assert(state_ == SegmentState::Open);
|
||||
// auto ack_id = ack_count_.load();
|
||||
// record.uids_.grow_by(primary_keys, primary_keys + size);
|
||||
// for (int64_t i = 0; i < size; ++i) {
|
||||
// auto key = primary_keys[i];
|
||||
// auto internal_index = i + ack_id;
|
||||
// internal_indexes_[key] = internal_index;
|
||||
// }
|
||||
// record.timestamps_.grow_by(timestamps, timestamps + size);
|
||||
// for (int fid = 0; fid < schema.size(); ++fid) {
|
||||
// auto field = schema[fid];
|
||||
// auto total_len = field.get_sizeof() * size / sizeof(float);
|
||||
// auto source_vec = data_chunk.entity_vecs[fid];
|
||||
// record.entity_vecs_[fid].grow_by(source_vec.data(), source_vec.data() + total_len);
|
||||
// }
|
||||
//
|
||||
// // finish insert
|
||||
// ack_count_ += size;
|
||||
// return Status::OK();
|
||||
}
|
||||
|
||||
Status
|
||||
SegmentSmallIndex::Delete(int64_t reserved_begin,
|
||||
int64_t size,
|
||||
const int64_t* uids_raw,
|
||||
const Timestamp* timestamps_raw) {
|
||||
std::vector<std::tuple<Timestamp, idx_t>> ordering;
|
||||
ordering.resize(size);
|
||||
// #pragma omp parallel for
|
||||
for (int i = 0; i < size; ++i) {
|
||||
ordering[i] = std::make_tuple(timestamps_raw[i], uids_raw[i]);
|
||||
}
|
||||
std::sort(ordering.begin(), ordering.end());
|
||||
std::vector<idx_t> uids(size);
|
||||
std::vector<Timestamp> timestamps(size);
|
||||
// #pragma omp parallel for
|
||||
for (int index = 0; index < size; ++index) {
|
||||
auto [t, uid] = ordering[index];
|
||||
timestamps[index] = t;
|
||||
uids[index] = uid;
|
||||
}
|
||||
deleted_record_.timestamps_.set_data(reserved_begin, timestamps.data(), size);
|
||||
deleted_record_.uids_.set_data(reserved_begin, uids.data(), size);
|
||||
deleted_record_.ack_responder_.AddSegment(reserved_begin, reserved_begin + size);
|
||||
return Status::OK();
|
||||
// for (int i = 0; i < size; ++i) {
|
||||
// auto key = primary_keys[i];
|
||||
// auto time = timestamps[i];
|
||||
// delete_logs_.insert(std::make_pair(key, time));
|
||||
// }
|
||||
// return Status::OK();
|
||||
}
|
||||
|
||||
template <typename RecordType>
|
||||
int64_t
|
||||
get_barrier(const RecordType& record, Timestamp timestamp) {
|
||||
auto& vec = record.timestamps_;
|
||||
int64_t beg = 0;
|
||||
int64_t end = record.ack_responder_.GetAck();
|
||||
while (beg < end) {
|
||||
auto mid = (beg + end) / 2;
|
||||
if (vec[mid] < timestamp) {
|
||||
beg = mid + 1;
|
||||
} else {
|
||||
end = mid;
|
||||
}
|
||||
}
|
||||
return beg;
|
||||
}
|
||||
|
||||
Status
|
||||
SegmentSmallIndex::QueryImpl(query::QueryPtr query_info, Timestamp timestamp, QueryResult& result) {
|
||||
auto ins_barrier = get_barrier(record_, timestamp);
|
||||
auto del_barrier = get_barrier(deleted_record_, timestamp);
|
||||
auto bitmap_holder = get_deleted_bitmap(del_barrier, timestamp, ins_barrier, true);
|
||||
Assert(bitmap_holder);
|
||||
Assert(bitmap_holder->bitmap_ptr->count() == ins_barrier);
|
||||
|
||||
auto field_offset = schema_->get_offset(query_info->field_name);
|
||||
auto& field = schema_->operator[](query_info->field_name);
|
||||
|
||||
Assert(field.get_data_type() == DataType::VECTOR_FLOAT);
|
||||
auto dim = field.get_dim();
|
||||
auto bitmap = bitmap_holder->bitmap_ptr;
|
||||
auto topK = query_info->topK;
|
||||
auto num_queries = query_info->num_queries;
|
||||
auto the_offset_opt = schema_->get_offset(query_info->field_name);
|
||||
Assert(the_offset_opt.has_value());
|
||||
Assert(the_offset_opt.value() < record_.entity_vec_.size());
|
||||
auto vec_ptr = std::static_pointer_cast<ConcurrentVector<float>>(record_.entity_vec_.at(the_offset_opt.value()));
|
||||
auto index_entry = index_meta_->lookup_by_field(query_info->field_name);
|
||||
auto conf = index_entry.config;
|
||||
|
||||
conf[milvus::knowhere::meta::TOPK] = query_info->topK;
|
||||
{
|
||||
auto count = 0;
|
||||
for (int i = 0; i < bitmap->count(); ++i) {
|
||||
if (bitmap->test(i)) {
|
||||
++count;
|
||||
}
|
||||
}
|
||||
std::cout << "fuck " << count << std::endl;
|
||||
}
|
||||
|
||||
auto indexing = std::static_pointer_cast<knowhere::VecIndex>(indexings_[index_entry.index_name]);
|
||||
indexing->SetBlacklist(bitmap);
|
||||
auto ds = knowhere::GenDataset(query_info->num_queries, dim, query_info->query_raw_data.data());
|
||||
auto final = indexing->Query(ds, conf, bitmap);
|
||||
|
||||
auto ids = final->Get<idx_t*>(knowhere::meta::IDS);
|
||||
auto distances = final->Get<float*>(knowhere::meta::DISTANCE);
|
||||
|
||||
auto total_num = num_queries * topK;
|
||||
result.result_ids_.resize(total_num);
|
||||
result.result_distances_.resize(total_num);
|
||||
|
||||
result.row_num_ = total_num;
|
||||
result.num_queries_ = num_queries;
|
||||
result.topK_ = topK;
|
||||
|
||||
std::copy_n(ids, total_num, result.result_ids_.data());
|
||||
std::copy_n(distances, total_num, result.result_distances_.data());
|
||||
|
||||
for (auto& id : result.result_ids_) {
|
||||
id = record_.uids_[id];
|
||||
}
|
||||
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
static void
|
||||
merge_into(int64_t queries,
|
||||
int64_t topk,
|
||||
float* distances,
|
||||
int64_t* uids,
|
||||
const float* new_distances,
|
||||
const int64_t* new_uids) {
|
||||
for (int64_t qn = 0; qn < queries; ++qn) {
|
||||
auto base = qn * topk;
|
||||
auto src2_dis = distances + base;
|
||||
auto src2_uids = uids + base;
|
||||
|
||||
auto src1_dis = new_distances + base;
|
||||
auto src1_uids = new_uids + base;
|
||||
|
||||
std::vector<float> buf_dis(topk);
|
||||
std::vector<int64_t> buf_uids(topk);
|
||||
|
||||
auto it1 = 0;
|
||||
auto it2 = 0;
|
||||
|
||||
for (auto buf = 0; buf < topk; ++buf) {
|
||||
if (src1_dis[it1] <= src2_dis[it2]) {
|
||||
buf_dis[buf] = src1_dis[it1];
|
||||
buf_uids[buf] = src1_uids[it1];
|
||||
++it1;
|
||||
} else {
|
||||
buf_dis[buf] = src2_dis[it2];
|
||||
buf_uids[buf] = src2_uids[it2];
|
||||
++it2;
|
||||
}
|
||||
}
|
||||
std::copy_n(buf_dis.data(), topk, src2_dis);
|
||||
std::copy_n(buf_uids.data(), topk, src2_uids);
|
||||
}
|
||||
}
|
||||
|
||||
Status
|
||||
SegmentSmallIndex::QueryBruteForceImpl(query::QueryPtr query_info, Timestamp timestamp, QueryResult& results) {
|
||||
auto ins_barrier = get_barrier(record_, timestamp);
|
||||
auto del_barrier = get_barrier(deleted_record_, timestamp);
|
||||
auto bitmap_holder = get_deleted_bitmap(del_barrier, timestamp, ins_barrier);
|
||||
Assert(bitmap_holder);
|
||||
|
||||
auto& field = schema_->operator[](query_info->field_name);
|
||||
Assert(field.get_data_type() == DataType::VECTOR_FLOAT);
|
||||
auto dim = field.get_dim();
|
||||
auto bitmap = bitmap_holder->bitmap_ptr;
|
||||
auto topK = query_info->topK;
|
||||
auto num_queries = query_info->num_queries;
|
||||
auto total_count = topK * num_queries;
|
||||
// TODO: optimize
|
||||
|
||||
auto the_offset_opt = schema_->get_offset(query_info->field_name);
|
||||
Assert(the_offset_opt.has_value());
|
||||
Assert(the_offset_opt.value() < record_.entity_vec_.size());
|
||||
auto vec_ptr = std::static_pointer_cast<ConcurrentVector<float>>(record_.entity_vec_.at(the_offset_opt.value()));
|
||||
|
||||
std::vector<int64_t> final_uids(total_count);
|
||||
std::vector<float> final_dis(total_count, std::numeric_limits<float>::max());
|
||||
|
||||
auto max_chunk = (ins_barrier + DefaultElementPerChunk - 1) / DefaultElementPerChunk;
|
||||
for (int chunk_id = 0; chunk_id < max_chunk; ++chunk_id) {
|
||||
std::vector<int64_t> buf_uids(total_count, -1);
|
||||
std::vector<float> buf_dis(total_count, std::numeric_limits<float>::max());
|
||||
|
||||
faiss::float_maxheap_array_t buf = {(size_t)num_queries, (size_t)topK, buf_uids.data(), buf_dis.data()};
|
||||
|
||||
auto src_data = vec_ptr->get_chunk(chunk_id).data();
|
||||
auto nsize =
|
||||
chunk_id != max_chunk - 1 ? DefaultElementPerChunk : ins_barrier - chunk_id * DefaultElementPerChunk;
|
||||
auto offset = chunk_id * DefaultElementPerChunk;
|
||||
|
||||
faiss::knn_L2sqr(query_info->query_raw_data.data(), src_data, dim, num_queries, nsize, &buf, bitmap, offset);
|
||||
if (chunk_id == 0) {
|
||||
final_uids = buf_uids;
|
||||
final_dis = buf_dis;
|
||||
} else {
|
||||
merge_into(num_queries, topK, final_dis.data(), final_uids.data(), buf_dis.data(), buf_uids.data());
|
||||
}
|
||||
}
|
||||
|
||||
for (auto& id : final_uids) {
|
||||
id = record_.uids_[id];
|
||||
}
|
||||
|
||||
results.result_ids_ = std::move(final_uids);
|
||||
results.result_distances_ = std::move(final_dis);
|
||||
results.topK_ = topK;
|
||||
results.num_queries_ = num_queries;
|
||||
results.row_num_ = total_count;
|
||||
|
||||
// throw std::runtime_error("unimplemented");
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status
|
||||
SegmentSmallIndex::QuerySlowImpl(query::QueryPtr query_info, Timestamp timestamp, QueryResult& result) {
|
||||
auto ins_barrier = get_barrier(record_, timestamp);
|
||||
auto del_barrier = get_barrier(deleted_record_, timestamp);
|
||||
auto bitmap_holder = get_deleted_bitmap(del_barrier, timestamp, ins_barrier);
|
||||
Assert(bitmap_holder);
|
||||
|
||||
auto& field = schema_->operator[](query_info->field_name);
|
||||
Assert(field.get_data_type() == DataType::VECTOR_FLOAT);
|
||||
auto dim = field.get_dim();
|
||||
auto bitmap = bitmap_holder->bitmap_ptr;
|
||||
auto topK = query_info->topK;
|
||||
auto num_queries = query_info->num_queries;
|
||||
// TODO: optimize
|
||||
auto the_offset_opt = schema_->get_offset(query_info->field_name);
|
||||
Assert(the_offset_opt.has_value());
|
||||
Assert(the_offset_opt.value() < record_.entity_vec_.size());
|
||||
auto vec_ptr = std::static_pointer_cast<ConcurrentVector<float>>(record_.entity_vec_.at(the_offset_opt.value()));
|
||||
std::vector<std::priority_queue<std::pair<float, int>>> records(num_queries);
|
||||
|
||||
auto get_L2_distance = [dim](const float* a, const float* b) {
|
||||
float L2_distance = 0;
|
||||
for (auto i = 0; i < dim; ++i) {
|
||||
auto d = a[i] - b[i];
|
||||
L2_distance += d * d;
|
||||
}
|
||||
return L2_distance;
|
||||
};
|
||||
|
||||
for (int64_t i = 0; i < ins_barrier; ++i) {
|
||||
if (i < bitmap->count() && bitmap->test(i)) {
|
||||
continue;
|
||||
}
|
||||
auto element = vec_ptr->get_element(i);
|
||||
for (auto query_id = 0; query_id < num_queries; ++query_id) {
|
||||
auto query_blob = query_info->query_raw_data.data() + query_id * dim;
|
||||
auto dis = get_L2_distance(query_blob, element);
|
||||
auto& record = records[query_id];
|
||||
if (record.size() < topK) {
|
||||
record.emplace(dis, i);
|
||||
} else if (record.top().first > dis) {
|
||||
record.emplace(dis, i);
|
||||
record.pop();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
result.num_queries_ = num_queries;
|
||||
result.topK_ = topK;
|
||||
auto row_num = topK * num_queries;
|
||||
result.row_num_ = topK * num_queries;
|
||||
|
||||
result.result_ids_.resize(row_num);
|
||||
result.result_distances_.resize(row_num);
|
||||
|
||||
for (int q_id = 0; q_id < num_queries; ++q_id) {
|
||||
// reverse
|
||||
for (int i = 0; i < topK; ++i) {
|
||||
auto dst_id = topK - 1 - i + q_id * topK;
|
||||
auto [dis, offset] = records[q_id].top();
|
||||
records[q_id].pop();
|
||||
result.result_ids_[dst_id] = record_.uids_[offset];
|
||||
result.result_distances_[dst_id] = dis;
|
||||
}
|
||||
}
|
||||
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status
|
||||
SegmentSmallIndex::Query(query::QueryPtr query_info, Timestamp timestamp, QueryResult& result) {
|
||||
// TODO: enable delete
|
||||
// TODO: enable index
|
||||
// TODO: remove mock
|
||||
if (query_info == nullptr) {
|
||||
query_info = std::make_shared<query::Query>();
|
||||
query_info->field_name = "fakevec";
|
||||
query_info->topK = 10;
|
||||
query_info->num_queries = 1;
|
||||
|
||||
auto dim = schema_->operator[]("fakevec").get_dim();
|
||||
std::default_random_engine e(42);
|
||||
std::uniform_real_distribution<> dis(0.0, 1.0);
|
||||
query_info->query_raw_data.resize(query_info->num_queries * dim);
|
||||
for (auto& x : query_info->query_raw_data) {
|
||||
x = dis(e);
|
||||
}
|
||||
}
|
||||
|
||||
if (index_ready_) {
|
||||
return QueryImpl(query_info, timestamp, result);
|
||||
} else {
|
||||
return QueryBruteForceImpl(query_info, timestamp, result);
|
||||
}
|
||||
}
|
||||
|
||||
Status
|
||||
SegmentSmallIndex::Close() {
|
||||
if (this->record_.reserved != this->record_.ack_responder_.GetAck()) {
|
||||
std::runtime_error("insert not ready");
|
||||
}
|
||||
if (this->deleted_record_.reserved != this->record_.ack_responder_.GetAck()) {
|
||||
std::runtime_error("delete not ready");
|
||||
}
|
||||
state_ = SegmentState::Closed;
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
knowhere::IndexPtr
|
||||
SegmentSmallIndex::BuildVecIndexImpl(const IndexMeta::Entry& entry) {
|
||||
auto offset_opt = schema_->get_offset(entry.field_name);
|
||||
Assert(offset_opt.has_value());
|
||||
auto offset = offset_opt.value();
|
||||
auto field = (*schema_)[offset];
|
||||
auto dim = field.get_dim();
|
||||
|
||||
auto indexing = knowhere::VecIndexFactory::GetInstance().CreateVecIndex(entry.type, entry.mode);
|
||||
auto chunk_size = record_.uids_.chunk_size();
|
||||
|
||||
auto& uids = record_.uids_;
|
||||
auto entities = record_.get_vec_entity<float>(offset);
|
||||
|
||||
std::vector<knowhere::DatasetPtr> datasets;
|
||||
for (int chunk_id = 0; chunk_id < uids.chunk_size(); ++chunk_id) {
|
||||
auto entities_chunk = entities->get_chunk(chunk_id).data();
|
||||
int64_t count = chunk_id == uids.chunk_size() - 1 ? record_.reserved - chunk_id * DefaultElementPerChunk
|
||||
: DefaultElementPerChunk;
|
||||
datasets.push_back(knowhere::GenDataset(count, dim, entities_chunk));
|
||||
}
|
||||
for (auto& ds : datasets) {
|
||||
indexing->Train(ds, entry.config);
|
||||
}
|
||||
for (auto& ds : datasets) {
|
||||
indexing->AddWithoutIds(ds, entry.config);
|
||||
}
|
||||
return indexing;
|
||||
}
|
||||
|
||||
Status
|
||||
SegmentSmallIndex::BuildIndex(IndexMetaPtr remote_index_meta) {
|
||||
if (remote_index_meta == nullptr) {
|
||||
std::cout << "WARN: Null index ptr is detected, use default index" << std::endl;
|
||||
|
||||
int dim = 0;
|
||||
std::string index_field_name;
|
||||
|
||||
for (auto& field : schema_->get_fields()) {
|
||||
if (field.get_data_type() == DataType::VECTOR_FLOAT) {
|
||||
dim = field.get_dim();
|
||||
index_field_name = field.get_name();
|
||||
}
|
||||
}
|
||||
|
||||
Assert(dim != 0);
|
||||
Assert(!index_field_name.empty());
|
||||
|
||||
auto index_meta = std::make_shared<IndexMeta>(schema_);
|
||||
// TODO: this is merge of query conf and insert conf
|
||||
// TODO: should be splitted into multiple configs
|
||||
auto conf = milvus::knowhere::Config{
|
||||
{milvus::knowhere::meta::DIM, dim}, {milvus::knowhere::IndexParams::nlist, 100},
|
||||
{milvus::knowhere::IndexParams::nprobe, 4}, {milvus::knowhere::IndexParams::m, 4},
|
||||
{milvus::knowhere::IndexParams::nbits, 8}, {milvus::knowhere::Metric::TYPE, milvus::knowhere::Metric::L2},
|
||||
{milvus::knowhere::meta::DEVICEID, 0},
|
||||
};
|
||||
index_meta->AddEntry("fakeindex", index_field_name, knowhere::IndexEnum::INDEX_FAISS_IVFPQ,
|
||||
knowhere::IndexMode::MODE_CPU, conf);
|
||||
remote_index_meta = index_meta;
|
||||
}
|
||||
|
||||
if (record_.ack_responder_.GetAck() < 1024 * 4) {
|
||||
return Status(SERVER_BUILD_INDEX_ERROR, "too few elements");
|
||||
}
|
||||
|
||||
index_meta_ = remote_index_meta;
|
||||
for (auto& [index_name, entry] : index_meta_->get_entries()) {
|
||||
Assert(entry.index_name == index_name);
|
||||
const auto& field = (*schema_)[entry.field_name];
|
||||
|
||||
if (field.is_vector()) {
|
||||
Assert(field.get_data_type() == engine::DataType::VECTOR_FLOAT);
|
||||
auto index_ptr = BuildVecIndexImpl<float>(entry);
|
||||
indexings_[index_name] = index_ptr;
|
||||
} else {
|
||||
throw std::runtime_error("unimplemented");
|
||||
}
|
||||
}
|
||||
|
||||
index_ready_ = true;
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
int64_t
|
||||
SegmentSmallIndex::GetMemoryUsageInBytes() {
|
||||
int64_t total_bytes = 0;
|
||||
if (index_ready_) {
|
||||
auto& index_entries = index_meta_->get_entries();
|
||||
for (auto [index_name, entry] : index_entries) {
|
||||
Assert(schema_->operator[](entry.field_name).is_vector());
|
||||
auto vec_ptr = std::static_pointer_cast<knowhere::VecIndex>(indexings_[index_name]);
|
||||
total_bytes += vec_ptr->IndexSize();
|
||||
}
|
||||
}
|
||||
int64_t ins_n = (record_.reserved + DefaultElementPerChunk - 1) & ~(DefaultElementPerChunk - 1);
|
||||
total_bytes += ins_n * (schema_->get_total_sizeof() + 16 + 1);
|
||||
int64_t del_n = (deleted_record_.reserved + DefaultElementPerChunk - 1) & ~(DefaultElementPerChunk - 1);
|
||||
total_bytes += del_n * (16 * 2);
|
||||
return total_bytes;
|
||||
}
|
||||
|
||||
} // namespace milvus::dog_segment
|
|
@ -0,0 +1,179 @@
|
|||
#pragma once
|
||||
|
||||
#include <tbb/concurrent_priority_queue.h>
|
||||
#include <tbb/concurrent_unordered_map.h>
|
||||
#include <tbb/concurrent_vector.h>
|
||||
|
||||
#include <shared_mutex>
|
||||
#include <knowhere/index/vector_index/VecIndex.h>
|
||||
|
||||
#include "AckResponder.h"
|
||||
#include "ConcurrentVector.h"
|
||||
#include "dog_segment/SegmentBase.h"
|
||||
// #include "knowhere/index/structured_index/StructuredIndex.h"
|
||||
#include "query/GeneralQuery.h"
|
||||
#include "utils/Status.h"
|
||||
#include "dog_segment/DeletedRecord.h"
|
||||
#include "EasyAssert.h"
|
||||
|
||||
namespace milvus::dog_segment {
|
||||
// struct ColumnBasedDataChunk {
|
||||
// std::vector<std::vector<float>> entity_vecs;
|
||||
//
|
||||
// static ColumnBasedDataChunk
|
||||
// from(const DogDataChunk& source, const Schema& schema) {
|
||||
// ColumnBasedDataChunk dest;
|
||||
// auto count = source.count;
|
||||
// auto raw_data = reinterpret_cast<const char*>(source.raw_data);
|
||||
// auto align = source.sizeof_per_row;
|
||||
// for (auto& field : schema) {
|
||||
// auto len = field.get_sizeof();
|
||||
// Assert(len % sizeof(float) == 0);
|
||||
// std::vector<float> new_col(len * count / sizeof(float));
|
||||
// for (int64_t i = 0; i < count; ++i) {
|
||||
// memcpy(new_col.data() + i * len / sizeof(float), raw_data + i * align, len);
|
||||
// }
|
||||
// dest.entity_vecs.push_back(std::move(new_col));
|
||||
// // offset the raw_data
|
||||
// raw_data += len / sizeof(float);
|
||||
// }
|
||||
// return dest;
|
||||
// }
|
||||
//};
|
||||
|
||||
class SegmentSmallIndex : public SegmentBase {
|
||||
public:
|
||||
virtual ~SegmentSmallIndex() = default;
|
||||
|
||||
// SegmentBase(std::shared_ptr<FieldsInfo> collection);
|
||||
|
||||
int64_t
|
||||
PreInsert(int64_t size) override;
|
||||
|
||||
// TODO: originally, id should be put into data_chunk
|
||||
// TODO: Is it ok to put them the other side?
|
||||
Status
|
||||
Insert(int64_t reserverd_offset,
|
||||
int64_t size,
|
||||
const int64_t* primary_keys,
|
||||
const Timestamp* timestamps,
|
||||
const DogDataChunk& values) override;
|
||||
|
||||
int64_t
|
||||
PreDelete(int64_t size) override;
|
||||
|
||||
// TODO: add id into delete log, possibly bitmap
|
||||
Status
|
||||
Delete(int64_t reserverd_offset, int64_t size, const int64_t* primary_keys, const Timestamp* timestamps) override;
|
||||
|
||||
// query contains metadata of
|
||||
Status
|
||||
Query(query::QueryPtr query_info, Timestamp timestamp, QueryResult& results) override;
|
||||
|
||||
// stop receive insert requests
|
||||
// will move data to immutable vector or something
|
||||
Status
|
||||
Close() override;
|
||||
|
||||
// using IndexType = knowhere::IndexType;
|
||||
// using IndexMode = knowhere::IndexMode;
|
||||
// using IndexConfig = knowhere::Config;
|
||||
// BuildIndex With Paramaters, must with Frozen State
|
||||
// NOTE: index_params contains serveral policies for several index
|
||||
// TODO: currently, index has to be set at startup, and can't be modified
|
||||
// AddIndex and DropIndex will be added later
|
||||
Status
|
||||
BuildIndex(IndexMetaPtr index_meta) override;
|
||||
|
||||
Status
|
||||
DropRawData(std::string_view field_name) override {
|
||||
// TODO: NO-OP
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
Status
|
||||
LoadRawData(std::string_view field_name, const char* blob, int64_t blob_size) override {
|
||||
// TODO: NO-OP
|
||||
return Status::OK();
|
||||
}
|
||||
|
||||
int64_t
|
||||
GetMemoryUsageInBytes() override;
|
||||
|
||||
public:
|
||||
ssize_t
|
||||
get_row_count() const override {
|
||||
return record_.ack_responder_.GetAck();
|
||||
}
|
||||
|
||||
SegmentState
|
||||
get_state() const override {
|
||||
return state_.load(std::memory_order_relaxed);
|
||||
}
|
||||
|
||||
ssize_t
|
||||
get_deleted_count() const override {
|
||||
return 0;
|
||||
}
|
||||
|
||||
public:
|
||||
friend std::unique_ptr<SegmentBase>
|
||||
CreateSegment(SchemaPtr schema);
|
||||
|
||||
explicit SegmentSmallIndex(SchemaPtr schema) : schema_(schema), record_(*schema) {
|
||||
}
|
||||
|
||||
private:
|
||||
// struct MutableRecord {
|
||||
// ConcurrentVector<uint64_t> uids_;
|
||||
// tbb::concurrent_vector<Timestamp> timestamps_;
|
||||
// std::vector<tbb::concurrent_vector<float>> entity_vecs_;
|
||||
//
|
||||
// MutableRecord(int entity_size) : entity_vecs_(entity_size) {
|
||||
// }
|
||||
// };
|
||||
|
||||
struct Record {
|
||||
std::atomic<int64_t> reserved = 0;
|
||||
AckResponder ack_responder_;
|
||||
ConcurrentVector<Timestamp, true> timestamps_;
|
||||
ConcurrentVector<idx_t, true> uids_;
|
||||
std::vector<std::shared_ptr<VectorBase>> entity_vec_;
|
||||
|
||||
Record(const Schema& schema);
|
||||
|
||||
template <typename Type>
|
||||
auto
|
||||
get_vec_entity(int offset) {
|
||||
return std::static_pointer_cast<ConcurrentVector<Type>>(entity_vec_[offset]);
|
||||
}
|
||||
};
|
||||
|
||||
std::shared_ptr<DeletedRecord::TmpBitmap>
|
||||
get_deleted_bitmap(int64_t del_barrier, Timestamp query_timestamp, int64_t insert_barrier, bool force = false);
|
||||
|
||||
Status
|
||||
QueryImpl(query::QueryPtr query, Timestamp timestamp, QueryResult& results);
|
||||
|
||||
Status
|
||||
QuerySlowImpl(query::QueryPtr query, Timestamp timestamp, QueryResult& results);
|
||||
|
||||
Status
|
||||
QueryBruteForceImpl(query::QueryPtr query, Timestamp timestamp, QueryResult& results);
|
||||
|
||||
template <typename Type>
|
||||
knowhere::IndexPtr
|
||||
BuildVecIndexImpl(const IndexMeta::Entry& entry);
|
||||
|
||||
private:
|
||||
SchemaPtr schema_;
|
||||
std::atomic<SegmentState> state_ = SegmentState::Open;
|
||||
Record record_;
|
||||
DeletedRecord deleted_record_;
|
||||
|
||||
std::atomic<bool> index_ready_ = false;
|
||||
IndexMetaPtr index_meta_;
|
||||
std::unordered_map<std::string, knowhere::IndexPtr> indexings_; // index_name => indexing
|
||||
tbb::concurrent_unordered_multimap<idx_t, int64_t> uid2offset_;
|
||||
};
|
||||
} // namespace milvus::dog_segment
|
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,822 @@
|
|||
// Generated by the protocol buffer compiler. DO NOT EDIT!
|
||||
// source: schema.proto
|
||||
|
||||
#ifndef GOOGLE_PROTOBUF_INCLUDED_schema_2eproto
|
||||
#define GOOGLE_PROTOBUF_INCLUDED_schema_2eproto
|
||||
|
||||
#include <limits>
|
||||
#include <string>
|
||||
|
||||
#include <google/protobuf/port_def.inc>
|
||||
#if PROTOBUF_VERSION < 3009000
|
||||
#error This file was generated by a newer version of protoc which is
|
||||
#error incompatible with your Protocol Buffer headers. Please update
|
||||
#error your headers.
|
||||
#endif
|
||||
#if 3009000 < PROTOBUF_MIN_PROTOC_VERSION
|
||||
#error This file was generated by an older version of protoc which is
|
||||
#error incompatible with your Protocol Buffer headers. Please
|
||||
#error regenerate this file with a newer version of protoc.
|
||||
#endif
|
||||
|
||||
#include <google/protobuf/port_undef.inc>
|
||||
#include <google/protobuf/io/coded_stream.h>
|
||||
#include <google/protobuf/arena.h>
|
||||
#include <google/protobuf/arenastring.h>
|
||||
#include <google/protobuf/generated_message_table_driven.h>
|
||||
#include <google/protobuf/generated_message_util.h>
|
||||
#include <google/protobuf/inlined_string_field.h>
|
||||
#include <google/protobuf/metadata.h>
|
||||
#include <google/protobuf/generated_message_reflection.h>
|
||||
#include <google/protobuf/message.h>
|
||||
#include <google/protobuf/repeated_field.h> // IWYU pragma: export
|
||||
#include <google/protobuf/extension_set.h> // IWYU pragma: export
|
||||
#include <google/protobuf/generated_enum_reflection.h>
|
||||
#include <google/protobuf/unknown_field_set.h>
|
||||
#include "common.pb.h"
|
||||
// @@protoc_insertion_point(includes)
|
||||
#include <google/protobuf/port_def.inc>
|
||||
#define PROTOBUF_INTERNAL_EXPORT_schema_2eproto
|
||||
PROTOBUF_NAMESPACE_OPEN
|
||||
namespace internal {
|
||||
class AnyMetadata;
|
||||
} // namespace internal
|
||||
PROTOBUF_NAMESPACE_CLOSE
|
||||
|
||||
// Internal implementation detail -- do not use these members.
|
||||
struct TableStruct_schema_2eproto {
|
||||
static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTableField entries[]
|
||||
PROTOBUF_SECTION_VARIABLE(protodesc_cold);
|
||||
static const ::PROTOBUF_NAMESPACE_ID::internal::AuxillaryParseTableField aux[]
|
||||
PROTOBUF_SECTION_VARIABLE(protodesc_cold);
|
||||
static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[2]
|
||||
PROTOBUF_SECTION_VARIABLE(protodesc_cold);
|
||||
static const ::PROTOBUF_NAMESPACE_ID::internal::FieldMetadata field_metadata[];
|
||||
static const ::PROTOBUF_NAMESPACE_ID::internal::SerializationTable serialization_table[];
|
||||
static const ::PROTOBUF_NAMESPACE_ID::uint32 offsets[];
|
||||
};
|
||||
extern const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable descriptor_table_schema_2eproto;
|
||||
namespace milvus {
|
||||
namespace proto {
|
||||
namespace schema {
|
||||
class CollectionSchema;
|
||||
class CollectionSchemaDefaultTypeInternal;
|
||||
extern CollectionSchemaDefaultTypeInternal _CollectionSchema_default_instance_;
|
||||
class FieldSchema;
|
||||
class FieldSchemaDefaultTypeInternal;
|
||||
extern FieldSchemaDefaultTypeInternal _FieldSchema_default_instance_;
|
||||
} // namespace schema
|
||||
} // namespace proto
|
||||
} // namespace milvus
|
||||
PROTOBUF_NAMESPACE_OPEN
|
||||
template<> ::milvus::proto::schema::CollectionSchema* Arena::CreateMaybeMessage<::milvus::proto::schema::CollectionSchema>(Arena*);
|
||||
template<> ::milvus::proto::schema::FieldSchema* Arena::CreateMaybeMessage<::milvus::proto::schema::FieldSchema>(Arena*);
|
||||
PROTOBUF_NAMESPACE_CLOSE
|
||||
namespace milvus {
|
||||
namespace proto {
|
||||
namespace schema {
|
||||
|
||||
enum DataType : int {
|
||||
NONE = 0,
|
||||
BOOL = 1,
|
||||
INT8 = 2,
|
||||
INT16 = 3,
|
||||
INT32 = 4,
|
||||
INT64 = 5,
|
||||
FLOAT = 10,
|
||||
DOUBLE = 11,
|
||||
STRING = 20,
|
||||
VECTOR_BINARY = 100,
|
||||
VECTOR_FLOAT = 101,
|
||||
DataType_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits<::PROTOBUF_NAMESPACE_ID::int32>::min(),
|
||||
DataType_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits<::PROTOBUF_NAMESPACE_ID::int32>::max()
|
||||
};
|
||||
bool DataType_IsValid(int value);
|
||||
constexpr DataType DataType_MIN = NONE;
|
||||
constexpr DataType DataType_MAX = VECTOR_FLOAT;
|
||||
constexpr int DataType_ARRAYSIZE = DataType_MAX + 1;
|
||||
|
||||
const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* DataType_descriptor();
|
||||
template<typename T>
|
||||
inline const std::string& DataType_Name(T enum_t_value) {
|
||||
static_assert(::std::is_same<T, DataType>::value ||
|
||||
::std::is_integral<T>::value,
|
||||
"Incorrect type passed to function DataType_Name.");
|
||||
return ::PROTOBUF_NAMESPACE_ID::internal::NameOfEnum(
|
||||
DataType_descriptor(), enum_t_value);
|
||||
}
|
||||
inline bool DataType_Parse(
|
||||
const std::string& name, DataType* value) {
|
||||
return ::PROTOBUF_NAMESPACE_ID::internal::ParseNamedEnum<DataType>(
|
||||
DataType_descriptor(), name, value);
|
||||
}
|
||||
// ===================================================================
|
||||
|
||||
class FieldSchema :
|
||||
public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.schema.FieldSchema) */ {
|
||||
public:
|
||||
FieldSchema();
|
||||
virtual ~FieldSchema();
|
||||
|
||||
FieldSchema(const FieldSchema& from);
|
||||
FieldSchema(FieldSchema&& from) noexcept
|
||||
: FieldSchema() {
|
||||
*this = ::std::move(from);
|
||||
}
|
||||
|
||||
inline FieldSchema& operator=(const FieldSchema& from) {
|
||||
CopyFrom(from);
|
||||
return *this;
|
||||
}
|
||||
inline FieldSchema& operator=(FieldSchema&& from) noexcept {
|
||||
if (GetArenaNoVirtual() == from.GetArenaNoVirtual()) {
|
||||
if (this != &from) InternalSwap(&from);
|
||||
} else {
|
||||
CopyFrom(from);
|
||||
}
|
||||
return *this;
|
||||
}
|
||||
|
||||
static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() {
|
||||
return GetDescriptor();
|
||||
}
|
||||
static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() {
|
||||
return GetMetadataStatic().descriptor;
|
||||
}
|
||||
static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() {
|
||||
return GetMetadataStatic().reflection;
|
||||
}
|
||||
static const FieldSchema& default_instance();
|
||||
|
||||
static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY
|
||||
static inline const FieldSchema* internal_default_instance() {
|
||||
return reinterpret_cast<const FieldSchema*>(
|
||||
&_FieldSchema_default_instance_);
|
||||
}
|
||||
static constexpr int kIndexInFileMessages =
|
||||
0;
|
||||
|
||||
friend void swap(FieldSchema& a, FieldSchema& b) {
|
||||
a.Swap(&b);
|
||||
}
|
||||
inline void Swap(FieldSchema* other) {
|
||||
if (other == this) return;
|
||||
InternalSwap(other);
|
||||
}
|
||||
|
||||
// implements Message ----------------------------------------------
|
||||
|
||||
inline FieldSchema* New() const final {
|
||||
return CreateMaybeMessage<FieldSchema>(nullptr);
|
||||
}
|
||||
|
||||
FieldSchema* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final {
|
||||
return CreateMaybeMessage<FieldSchema>(arena);
|
||||
}
|
||||
void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final;
|
||||
void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final;
|
||||
void CopyFrom(const FieldSchema& from);
|
||||
void MergeFrom(const FieldSchema& from);
|
||||
PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final;
|
||||
bool IsInitialized() const final;
|
||||
|
||||
size_t ByteSizeLong() const final;
|
||||
#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER
|
||||
const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final;
|
||||
#else
|
||||
bool MergePartialFromCodedStream(
|
||||
::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) final;
|
||||
#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER
|
||||
void SerializeWithCachedSizes(
|
||||
::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const final;
|
||||
::PROTOBUF_NAMESPACE_ID::uint8* InternalSerializeWithCachedSizesToArray(
|
||||
::PROTOBUF_NAMESPACE_ID::uint8* target) const final;
|
||||
int GetCachedSize() const final { return _cached_size_.Get(); }
|
||||
|
||||
private:
|
||||
inline void SharedCtor();
|
||||
inline void SharedDtor();
|
||||
void SetCachedSize(int size) const final;
|
||||
void InternalSwap(FieldSchema* other);
|
||||
friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata;
|
||||
static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() {
|
||||
return "milvus.proto.schema.FieldSchema";
|
||||
}
|
||||
private:
|
||||
inline ::PROTOBUF_NAMESPACE_ID::Arena* GetArenaNoVirtual() const {
|
||||
return nullptr;
|
||||
}
|
||||
inline void* MaybeArenaPtr() const {
|
||||
return nullptr;
|
||||
}
|
||||
public:
|
||||
|
||||
::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final;
|
||||
private:
|
||||
static ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadataStatic() {
|
||||
::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&::descriptor_table_schema_2eproto);
|
||||
return ::descriptor_table_schema_2eproto.file_level_metadata[kIndexInFileMessages];
|
||||
}
|
||||
|
||||
public:
|
||||
|
||||
// nested types ----------------------------------------------------
|
||||
|
||||
// accessors -------------------------------------------------------
|
||||
|
||||
enum : int {
|
||||
kTypeParamsFieldNumber = 4,
|
||||
kIndexParamsFieldNumber = 5,
|
||||
kNameFieldNumber = 1,
|
||||
kDescriptionFieldNumber = 2,
|
||||
kDataTypeFieldNumber = 3,
|
||||
};
|
||||
// repeated .milvus.proto.common.KeyValuePair type_params = 4;
|
||||
int type_params_size() const;
|
||||
void clear_type_params();
|
||||
::milvus::proto::common::KeyValuePair* mutable_type_params(int index);
|
||||
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::KeyValuePair >*
|
||||
mutable_type_params();
|
||||
const ::milvus::proto::common::KeyValuePair& type_params(int index) const;
|
||||
::milvus::proto::common::KeyValuePair* add_type_params();
|
||||
const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::KeyValuePair >&
|
||||
type_params() const;
|
||||
|
||||
// repeated .milvus.proto.common.KeyValuePair index_params = 5;
|
||||
int index_params_size() const;
|
||||
void clear_index_params();
|
||||
::milvus::proto::common::KeyValuePair* mutable_index_params(int index);
|
||||
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::KeyValuePair >*
|
||||
mutable_index_params();
|
||||
const ::milvus::proto::common::KeyValuePair& index_params(int index) const;
|
||||
::milvus::proto::common::KeyValuePair* add_index_params();
|
||||
const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::KeyValuePair >&
|
||||
index_params() const;
|
||||
|
||||
// string name = 1;
|
||||
void clear_name();
|
||||
const std::string& name() const;
|
||||
void set_name(const std::string& value);
|
||||
void set_name(std::string&& value);
|
||||
void set_name(const char* value);
|
||||
void set_name(const char* value, size_t size);
|
||||
std::string* mutable_name();
|
||||
std::string* release_name();
|
||||
void set_allocated_name(std::string* name);
|
||||
|
||||
// string description = 2;
|
||||
void clear_description();
|
||||
const std::string& description() const;
|
||||
void set_description(const std::string& value);
|
||||
void set_description(std::string&& value);
|
||||
void set_description(const char* value);
|
||||
void set_description(const char* value, size_t size);
|
||||
std::string* mutable_description();
|
||||
std::string* release_description();
|
||||
void set_allocated_description(std::string* description);
|
||||
|
||||
// .milvus.proto.schema.DataType data_type = 3;
|
||||
void clear_data_type();
|
||||
::milvus::proto::schema::DataType data_type() const;
|
||||
void set_data_type(::milvus::proto::schema::DataType value);
|
||||
|
||||
// @@protoc_insertion_point(class_scope:milvus.proto.schema.FieldSchema)
|
||||
private:
|
||||
class _Internal;
|
||||
|
||||
::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_;
|
||||
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::KeyValuePair > type_params_;
|
||||
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::KeyValuePair > index_params_;
|
||||
::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr name_;
|
||||
::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr description_;
|
||||
int data_type_;
|
||||
mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_;
|
||||
friend struct ::TableStruct_schema_2eproto;
|
||||
};
|
||||
// -------------------------------------------------------------------
|
||||
|
||||
class CollectionSchema :
|
||||
public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.schema.CollectionSchema) */ {
|
||||
public:
|
||||
CollectionSchema();
|
||||
virtual ~CollectionSchema();
|
||||
|
||||
CollectionSchema(const CollectionSchema& from);
|
||||
CollectionSchema(CollectionSchema&& from) noexcept
|
||||
: CollectionSchema() {
|
||||
*this = ::std::move(from);
|
||||
}
|
||||
|
||||
inline CollectionSchema& operator=(const CollectionSchema& from) {
|
||||
CopyFrom(from);
|
||||
return *this;
|
||||
}
|
||||
inline CollectionSchema& operator=(CollectionSchema&& from) noexcept {
|
||||
if (GetArenaNoVirtual() == from.GetArenaNoVirtual()) {
|
||||
if (this != &from) InternalSwap(&from);
|
||||
} else {
|
||||
CopyFrom(from);
|
||||
}
|
||||
return *this;
|
||||
}
|
||||
|
||||
static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() {
|
||||
return GetDescriptor();
|
||||
}
|
||||
static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() {
|
||||
return GetMetadataStatic().descriptor;
|
||||
}
|
||||
static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() {
|
||||
return GetMetadataStatic().reflection;
|
||||
}
|
||||
static const CollectionSchema& default_instance();
|
||||
|
||||
static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY
|
||||
static inline const CollectionSchema* internal_default_instance() {
|
||||
return reinterpret_cast<const CollectionSchema*>(
|
||||
&_CollectionSchema_default_instance_);
|
||||
}
|
||||
static constexpr int kIndexInFileMessages =
|
||||
1;
|
||||
|
||||
friend void swap(CollectionSchema& a, CollectionSchema& b) {
|
||||
a.Swap(&b);
|
||||
}
|
||||
inline void Swap(CollectionSchema* other) {
|
||||
if (other == this) return;
|
||||
InternalSwap(other);
|
||||
}
|
||||
|
||||
// implements Message ----------------------------------------------
|
||||
|
||||
inline CollectionSchema* New() const final {
|
||||
return CreateMaybeMessage<CollectionSchema>(nullptr);
|
||||
}
|
||||
|
||||
CollectionSchema* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final {
|
||||
return CreateMaybeMessage<CollectionSchema>(arena);
|
||||
}
|
||||
void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final;
|
||||
void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final;
|
||||
void CopyFrom(const CollectionSchema& from);
|
||||
void MergeFrom(const CollectionSchema& from);
|
||||
PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final;
|
||||
bool IsInitialized() const final;
|
||||
|
||||
size_t ByteSizeLong() const final;
|
||||
#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER
|
||||
const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final;
|
||||
#else
|
||||
bool MergePartialFromCodedStream(
|
||||
::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) final;
|
||||
#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER
|
||||
void SerializeWithCachedSizes(
|
||||
::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const final;
|
||||
::PROTOBUF_NAMESPACE_ID::uint8* InternalSerializeWithCachedSizesToArray(
|
||||
::PROTOBUF_NAMESPACE_ID::uint8* target) const final;
|
||||
int GetCachedSize() const final { return _cached_size_.Get(); }
|
||||
|
||||
private:
|
||||
inline void SharedCtor();
|
||||
inline void SharedDtor();
|
||||
void SetCachedSize(int size) const final;
|
||||
void InternalSwap(CollectionSchema* other);
|
||||
friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata;
|
||||
static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() {
|
||||
return "milvus.proto.schema.CollectionSchema";
|
||||
}
|
||||
private:
|
||||
inline ::PROTOBUF_NAMESPACE_ID::Arena* GetArenaNoVirtual() const {
|
||||
return nullptr;
|
||||
}
|
||||
inline void* MaybeArenaPtr() const {
|
||||
return nullptr;
|
||||
}
|
||||
public:
|
||||
|
||||
::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final;
|
||||
private:
|
||||
static ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadataStatic() {
|
||||
::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&::descriptor_table_schema_2eproto);
|
||||
return ::descriptor_table_schema_2eproto.file_level_metadata[kIndexInFileMessages];
|
||||
}
|
||||
|
||||
public:
|
||||
|
||||
// nested types ----------------------------------------------------
|
||||
|
||||
// accessors -------------------------------------------------------
|
||||
|
||||
enum : int {
|
||||
kFieldsFieldNumber = 4,
|
||||
kNameFieldNumber = 1,
|
||||
kDescriptionFieldNumber = 2,
|
||||
kAutoIdFieldNumber = 3,
|
||||
};
|
||||
// repeated .milvus.proto.schema.FieldSchema fields = 4;
|
||||
int fields_size() const;
|
||||
void clear_fields();
|
||||
::milvus::proto::schema::FieldSchema* mutable_fields(int index);
|
||||
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::schema::FieldSchema >*
|
||||
mutable_fields();
|
||||
const ::milvus::proto::schema::FieldSchema& fields(int index) const;
|
||||
::milvus::proto::schema::FieldSchema* add_fields();
|
||||
const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::schema::FieldSchema >&
|
||||
fields() const;
|
||||
|
||||
// string name = 1;
|
||||
void clear_name();
|
||||
const std::string& name() const;
|
||||
void set_name(const std::string& value);
|
||||
void set_name(std::string&& value);
|
||||
void set_name(const char* value);
|
||||
void set_name(const char* value, size_t size);
|
||||
std::string* mutable_name();
|
||||
std::string* release_name();
|
||||
void set_allocated_name(std::string* name);
|
||||
|
||||
// string description = 2;
|
||||
void clear_description();
|
||||
const std::string& description() const;
|
||||
void set_description(const std::string& value);
|
||||
void set_description(std::string&& value);
|
||||
void set_description(const char* value);
|
||||
void set_description(const char* value, size_t size);
|
||||
std::string* mutable_description();
|
||||
std::string* release_description();
|
||||
void set_allocated_description(std::string* description);
|
||||
|
||||
// bool auto_id = 3;
|
||||
void clear_auto_id();
|
||||
bool auto_id() const;
|
||||
void set_auto_id(bool value);
|
||||
|
||||
// @@protoc_insertion_point(class_scope:milvus.proto.schema.CollectionSchema)
|
||||
private:
|
||||
class _Internal;
|
||||
|
||||
::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_;
|
||||
::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::schema::FieldSchema > fields_;
|
||||
::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr name_;
|
||||
::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr description_;
|
||||
bool auto_id_;
|
||||
mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_;
|
||||
friend struct ::TableStruct_schema_2eproto;
|
||||
};
|
||||
// ===================================================================
|
||||
|
||||
|
||||
// ===================================================================
|
||||
|
||||
#ifdef __GNUC__
|
||||
#pragma GCC diagnostic push
|
||||
#pragma GCC diagnostic ignored "-Wstrict-aliasing"
|
||||
#endif // __GNUC__
|
||||
// FieldSchema
|
||||
|
||||
// string name = 1;
|
||||
inline void FieldSchema::clear_name() {
|
||||
name_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline const std::string& FieldSchema::name() const {
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.schema.FieldSchema.name)
|
||||
return name_.GetNoArena();
|
||||
}
|
||||
inline void FieldSchema::set_name(const std::string& value) {
|
||||
|
||||
name_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value);
|
||||
// @@protoc_insertion_point(field_set:milvus.proto.schema.FieldSchema.name)
|
||||
}
|
||||
inline void FieldSchema::set_name(std::string&& value) {
|
||||
|
||||
name_.SetNoArena(
|
||||
&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value));
|
||||
// @@protoc_insertion_point(field_set_rvalue:milvus.proto.schema.FieldSchema.name)
|
||||
}
|
||||
inline void FieldSchema::set_name(const char* value) {
|
||||
GOOGLE_DCHECK(value != nullptr);
|
||||
|
||||
name_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value));
|
||||
// @@protoc_insertion_point(field_set_char:milvus.proto.schema.FieldSchema.name)
|
||||
}
|
||||
inline void FieldSchema::set_name(const char* value, size_t size) {
|
||||
|
||||
name_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(),
|
||||
::std::string(reinterpret_cast<const char*>(value), size));
|
||||
// @@protoc_insertion_point(field_set_pointer:milvus.proto.schema.FieldSchema.name)
|
||||
}
|
||||
inline std::string* FieldSchema::mutable_name() {
|
||||
|
||||
// @@protoc_insertion_point(field_mutable:milvus.proto.schema.FieldSchema.name)
|
||||
return name_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline std::string* FieldSchema::release_name() {
|
||||
// @@protoc_insertion_point(field_release:milvus.proto.schema.FieldSchema.name)
|
||||
|
||||
return name_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline void FieldSchema::set_allocated_name(std::string* name) {
|
||||
if (name != nullptr) {
|
||||
|
||||
} else {
|
||||
|
||||
}
|
||||
name_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), name);
|
||||
// @@protoc_insertion_point(field_set_allocated:milvus.proto.schema.FieldSchema.name)
|
||||
}
|
||||
|
||||
// string description = 2;
|
||||
inline void FieldSchema::clear_description() {
|
||||
description_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline const std::string& FieldSchema::description() const {
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.schema.FieldSchema.description)
|
||||
return description_.GetNoArena();
|
||||
}
|
||||
inline void FieldSchema::set_description(const std::string& value) {
|
||||
|
||||
description_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value);
|
||||
// @@protoc_insertion_point(field_set:milvus.proto.schema.FieldSchema.description)
|
||||
}
|
||||
inline void FieldSchema::set_description(std::string&& value) {
|
||||
|
||||
description_.SetNoArena(
|
||||
&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value));
|
||||
// @@protoc_insertion_point(field_set_rvalue:milvus.proto.schema.FieldSchema.description)
|
||||
}
|
||||
inline void FieldSchema::set_description(const char* value) {
|
||||
GOOGLE_DCHECK(value != nullptr);
|
||||
|
||||
description_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value));
|
||||
// @@protoc_insertion_point(field_set_char:milvus.proto.schema.FieldSchema.description)
|
||||
}
|
||||
inline void FieldSchema::set_description(const char* value, size_t size) {
|
||||
|
||||
description_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(),
|
||||
::std::string(reinterpret_cast<const char*>(value), size));
|
||||
// @@protoc_insertion_point(field_set_pointer:milvus.proto.schema.FieldSchema.description)
|
||||
}
|
||||
inline std::string* FieldSchema::mutable_description() {
|
||||
|
||||
// @@protoc_insertion_point(field_mutable:milvus.proto.schema.FieldSchema.description)
|
||||
return description_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline std::string* FieldSchema::release_description() {
|
||||
// @@protoc_insertion_point(field_release:milvus.proto.schema.FieldSchema.description)
|
||||
|
||||
return description_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline void FieldSchema::set_allocated_description(std::string* description) {
|
||||
if (description != nullptr) {
|
||||
|
||||
} else {
|
||||
|
||||
}
|
||||
description_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), description);
|
||||
// @@protoc_insertion_point(field_set_allocated:milvus.proto.schema.FieldSchema.description)
|
||||
}
|
||||
|
||||
// .milvus.proto.schema.DataType data_type = 3;
|
||||
inline void FieldSchema::clear_data_type() {
|
||||
data_type_ = 0;
|
||||
}
|
||||
inline ::milvus::proto::schema::DataType FieldSchema::data_type() const {
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.schema.FieldSchema.data_type)
|
||||
return static_cast< ::milvus::proto::schema::DataType >(data_type_);
|
||||
}
|
||||
inline void FieldSchema::set_data_type(::milvus::proto::schema::DataType value) {
|
||||
|
||||
data_type_ = value;
|
||||
// @@protoc_insertion_point(field_set:milvus.proto.schema.FieldSchema.data_type)
|
||||
}
|
||||
|
||||
// repeated .milvus.proto.common.KeyValuePair type_params = 4;
|
||||
inline int FieldSchema::type_params_size() const {
|
||||
return type_params_.size();
|
||||
}
|
||||
inline ::milvus::proto::common::KeyValuePair* FieldSchema::mutable_type_params(int index) {
|
||||
// @@protoc_insertion_point(field_mutable:milvus.proto.schema.FieldSchema.type_params)
|
||||
return type_params_.Mutable(index);
|
||||
}
|
||||
inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::KeyValuePair >*
|
||||
FieldSchema::mutable_type_params() {
|
||||
// @@protoc_insertion_point(field_mutable_list:milvus.proto.schema.FieldSchema.type_params)
|
||||
return &type_params_;
|
||||
}
|
||||
inline const ::milvus::proto::common::KeyValuePair& FieldSchema::type_params(int index) const {
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.schema.FieldSchema.type_params)
|
||||
return type_params_.Get(index);
|
||||
}
|
||||
inline ::milvus::proto::common::KeyValuePair* FieldSchema::add_type_params() {
|
||||
// @@protoc_insertion_point(field_add:milvus.proto.schema.FieldSchema.type_params)
|
||||
return type_params_.Add();
|
||||
}
|
||||
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::KeyValuePair >&
|
||||
FieldSchema::type_params() const {
|
||||
// @@protoc_insertion_point(field_list:milvus.proto.schema.FieldSchema.type_params)
|
||||
return type_params_;
|
||||
}
|
||||
|
||||
// repeated .milvus.proto.common.KeyValuePair index_params = 5;
|
||||
inline int FieldSchema::index_params_size() const {
|
||||
return index_params_.size();
|
||||
}
|
||||
inline ::milvus::proto::common::KeyValuePair* FieldSchema::mutable_index_params(int index) {
|
||||
// @@protoc_insertion_point(field_mutable:milvus.proto.schema.FieldSchema.index_params)
|
||||
return index_params_.Mutable(index);
|
||||
}
|
||||
inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::KeyValuePair >*
|
||||
FieldSchema::mutable_index_params() {
|
||||
// @@protoc_insertion_point(field_mutable_list:milvus.proto.schema.FieldSchema.index_params)
|
||||
return &index_params_;
|
||||
}
|
||||
inline const ::milvus::proto::common::KeyValuePair& FieldSchema::index_params(int index) const {
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.schema.FieldSchema.index_params)
|
||||
return index_params_.Get(index);
|
||||
}
|
||||
inline ::milvus::proto::common::KeyValuePair* FieldSchema::add_index_params() {
|
||||
// @@protoc_insertion_point(field_add:milvus.proto.schema.FieldSchema.index_params)
|
||||
return index_params_.Add();
|
||||
}
|
||||
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::common::KeyValuePair >&
|
||||
FieldSchema::index_params() const {
|
||||
// @@protoc_insertion_point(field_list:milvus.proto.schema.FieldSchema.index_params)
|
||||
return index_params_;
|
||||
}
|
||||
|
||||
// -------------------------------------------------------------------
|
||||
|
||||
// CollectionSchema
|
||||
|
||||
// string name = 1;
|
||||
inline void CollectionSchema::clear_name() {
|
||||
name_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline const std::string& CollectionSchema::name() const {
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.schema.CollectionSchema.name)
|
||||
return name_.GetNoArena();
|
||||
}
|
||||
inline void CollectionSchema::set_name(const std::string& value) {
|
||||
|
||||
name_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value);
|
||||
// @@protoc_insertion_point(field_set:milvus.proto.schema.CollectionSchema.name)
|
||||
}
|
||||
inline void CollectionSchema::set_name(std::string&& value) {
|
||||
|
||||
name_.SetNoArena(
|
||||
&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value));
|
||||
// @@protoc_insertion_point(field_set_rvalue:milvus.proto.schema.CollectionSchema.name)
|
||||
}
|
||||
inline void CollectionSchema::set_name(const char* value) {
|
||||
GOOGLE_DCHECK(value != nullptr);
|
||||
|
||||
name_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value));
|
||||
// @@protoc_insertion_point(field_set_char:milvus.proto.schema.CollectionSchema.name)
|
||||
}
|
||||
inline void CollectionSchema::set_name(const char* value, size_t size) {
|
||||
|
||||
name_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(),
|
||||
::std::string(reinterpret_cast<const char*>(value), size));
|
||||
// @@protoc_insertion_point(field_set_pointer:milvus.proto.schema.CollectionSchema.name)
|
||||
}
|
||||
inline std::string* CollectionSchema::mutable_name() {
|
||||
|
||||
// @@protoc_insertion_point(field_mutable:milvus.proto.schema.CollectionSchema.name)
|
||||
return name_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline std::string* CollectionSchema::release_name() {
|
||||
// @@protoc_insertion_point(field_release:milvus.proto.schema.CollectionSchema.name)
|
||||
|
||||
return name_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline void CollectionSchema::set_allocated_name(std::string* name) {
|
||||
if (name != nullptr) {
|
||||
|
||||
} else {
|
||||
|
||||
}
|
||||
name_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), name);
|
||||
// @@protoc_insertion_point(field_set_allocated:milvus.proto.schema.CollectionSchema.name)
|
||||
}
|
||||
|
||||
// string description = 2;
|
||||
inline void CollectionSchema::clear_description() {
|
||||
description_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline const std::string& CollectionSchema::description() const {
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.schema.CollectionSchema.description)
|
||||
return description_.GetNoArena();
|
||||
}
|
||||
inline void CollectionSchema::set_description(const std::string& value) {
|
||||
|
||||
description_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value);
|
||||
// @@protoc_insertion_point(field_set:milvus.proto.schema.CollectionSchema.description)
|
||||
}
|
||||
inline void CollectionSchema::set_description(std::string&& value) {
|
||||
|
||||
description_.SetNoArena(
|
||||
&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value));
|
||||
// @@protoc_insertion_point(field_set_rvalue:milvus.proto.schema.CollectionSchema.description)
|
||||
}
|
||||
inline void CollectionSchema::set_description(const char* value) {
|
||||
GOOGLE_DCHECK(value != nullptr);
|
||||
|
||||
description_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value));
|
||||
// @@protoc_insertion_point(field_set_char:milvus.proto.schema.CollectionSchema.description)
|
||||
}
|
||||
inline void CollectionSchema::set_description(const char* value, size_t size) {
|
||||
|
||||
description_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(),
|
||||
::std::string(reinterpret_cast<const char*>(value), size));
|
||||
// @@protoc_insertion_point(field_set_pointer:milvus.proto.schema.CollectionSchema.description)
|
||||
}
|
||||
inline std::string* CollectionSchema::mutable_description() {
|
||||
|
||||
// @@protoc_insertion_point(field_mutable:milvus.proto.schema.CollectionSchema.description)
|
||||
return description_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline std::string* CollectionSchema::release_description() {
|
||||
// @@protoc_insertion_point(field_release:milvus.proto.schema.CollectionSchema.description)
|
||||
|
||||
return description_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited());
|
||||
}
|
||||
inline void CollectionSchema::set_allocated_description(std::string* description) {
|
||||
if (description != nullptr) {
|
||||
|
||||
} else {
|
||||
|
||||
}
|
||||
description_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), description);
|
||||
// @@protoc_insertion_point(field_set_allocated:milvus.proto.schema.CollectionSchema.description)
|
||||
}
|
||||
|
||||
// bool auto_id = 3;
|
||||
inline void CollectionSchema::clear_auto_id() {
|
||||
auto_id_ = false;
|
||||
}
|
||||
inline bool CollectionSchema::auto_id() const {
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.schema.CollectionSchema.auto_id)
|
||||
return auto_id_;
|
||||
}
|
||||
inline void CollectionSchema::set_auto_id(bool value) {
|
||||
|
||||
auto_id_ = value;
|
||||
// @@protoc_insertion_point(field_set:milvus.proto.schema.CollectionSchema.auto_id)
|
||||
}
|
||||
|
||||
// repeated .milvus.proto.schema.FieldSchema fields = 4;
|
||||
inline int CollectionSchema::fields_size() const {
|
||||
return fields_.size();
|
||||
}
|
||||
inline void CollectionSchema::clear_fields() {
|
||||
fields_.Clear();
|
||||
}
|
||||
inline ::milvus::proto::schema::FieldSchema* CollectionSchema::mutable_fields(int index) {
|
||||
// @@protoc_insertion_point(field_mutable:milvus.proto.schema.CollectionSchema.fields)
|
||||
return fields_.Mutable(index);
|
||||
}
|
||||
inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::schema::FieldSchema >*
|
||||
CollectionSchema::mutable_fields() {
|
||||
// @@protoc_insertion_point(field_mutable_list:milvus.proto.schema.CollectionSchema.fields)
|
||||
return &fields_;
|
||||
}
|
||||
inline const ::milvus::proto::schema::FieldSchema& CollectionSchema::fields(int index) const {
|
||||
// @@protoc_insertion_point(field_get:milvus.proto.schema.CollectionSchema.fields)
|
||||
return fields_.Get(index);
|
||||
}
|
||||
inline ::milvus::proto::schema::FieldSchema* CollectionSchema::add_fields() {
|
||||
// @@protoc_insertion_point(field_add:milvus.proto.schema.CollectionSchema.fields)
|
||||
return fields_.Add();
|
||||
}
|
||||
inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::milvus::proto::schema::FieldSchema >&
|
||||
CollectionSchema::fields() const {
|
||||
// @@protoc_insertion_point(field_list:milvus.proto.schema.CollectionSchema.fields)
|
||||
return fields_;
|
||||
}
|
||||
|
||||
#ifdef __GNUC__
|
||||
#pragma GCC diagnostic pop
|
||||
#endif // __GNUC__
|
||||
// -------------------------------------------------------------------
|
||||
|
||||
|
||||
// @@protoc_insertion_point(namespace_scope)
|
||||
|
||||
} // namespace schema
|
||||
} // namespace proto
|
||||
} // namespace milvus
|
||||
|
||||
PROTOBUF_NAMESPACE_OPEN
|
||||
|
||||
template <> struct is_proto_enum< ::milvus::proto::schema::DataType> : ::std::true_type {};
|
||||
template <>
|
||||
inline const EnumDescriptor* GetEnumDescriptor< ::milvus::proto::schema::DataType>() {
|
||||
return ::milvus::proto::schema::DataType_descriptor();
|
||||
}
|
||||
|
||||
PROTOBUF_NAMESPACE_CLOSE
|
||||
|
||||
// @@protoc_insertion_point(global_scope)
|
||||
|
||||
#include <google/protobuf/port_undef.inc>
|
||||
#endif // GOOGLE_PROTOBUF_INCLUDED_GOOGLE_PROTOBUF_INCLUDED_schema_2eproto
|
|
@ -1,7 +1,7 @@
|
|||
# TODO
|
||||
set(MILVUS_QUERY_SRCS
|
||||
BinaryQuery.cpp
|
||||
Parser.cpp
|
||||
|
||||
)
|
||||
add_library(milvus_query ${MILVUS_QUERY_SRCS})
|
||||
target_link_libraries(milvus_query libprotobuf)
|
||||
|
|
|
@ -86,7 +86,7 @@ add_custom_command(TARGET generate_milvus_pb_grpc
|
|||
POST_BUILD
|
||||
|
||||
COMMAND ${PROTOC_EXCUTABLE} -I "${PROTO_PATH}" --cpp_out "${PROTO_OUTPUT_PATH}"
|
||||
"message.proto" "master.proto"
|
||||
"common.proto" "schema.proto" "etcd_meta.proto"
|
||||
DEPENDS "${PROTO_PATH}/*.proto"
|
||||
)
|
||||
|
||||
|
|
|
@ -3,7 +3,8 @@ package collection
|
|||
import (
|
||||
"time"
|
||||
|
||||
masterpb "github.com/zilliztech/milvus-distributed/internal/proto/master"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
messagepb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
"github.com/gogo/protobuf/proto"
|
||||
jsoniter "github.com/json-iterator/go"
|
||||
|
@ -33,27 +34,24 @@ func GrpcMarshal(c *Collection) *Collection {
|
|||
if c.GrpcMarshalString != "" {
|
||||
c.GrpcMarshalString = ""
|
||||
}
|
||||
pbSchema := &messagepb.Schema{
|
||||
FieldMetas: []*messagepb.FieldMeta{},
|
||||
pbSchema := &schemapb.CollectionSchema{
|
||||
Fields: []*schemapb.FieldSchema{},
|
||||
}
|
||||
schemaSlice := []*messagepb.FieldMeta{}
|
||||
schemaSlice := []*schemapb.FieldSchema{}
|
||||
for _, v := range c.Schema {
|
||||
newpbMeta := &messagepb.FieldMeta{
|
||||
FieldName: v.FieldName,
|
||||
Type: v.Type,
|
||||
Dim: v.DIM,
|
||||
newpbMeta := &schemapb.FieldSchema{
|
||||
Name: v.FieldName,
|
||||
DataType: schemapb.DataType(v.Type), //czs_tag
|
||||
}
|
||||
schemaSlice = append(schemaSlice, newpbMeta)
|
||||
}
|
||||
pbSchema.FieldMetas = schemaSlice
|
||||
grpcCollection := &masterpb.Collection{
|
||||
pbSchema.Fields = schemaSlice
|
||||
grpcCollection := &etcdpb.CollectionMeta{
|
||||
Id: c.ID,
|
||||
Name: c.Name,
|
||||
Schema: pbSchema,
|
||||
CreateTime: c.CreateTime,
|
||||
SegmentIds: c.SegmentIDs,
|
||||
PartitionTags: c.PartitionTags,
|
||||
Indexes: c.IndexParam,
|
||||
}
|
||||
out := proto.MarshalTextString(grpcCollection)
|
||||
c.GrpcMarshalString = out
|
||||
|
|
|
@ -6,6 +6,7 @@ import (
|
|||
"time"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/master/collection"
|
||||
"github.com/zilliztech/milvus-distributed/internal/master/id"
|
||||
"github.com/zilliztech/milvus-distributed/internal/master/informer"
|
||||
|
@ -14,14 +15,13 @@ import (
|
|||
)
|
||||
|
||||
func SegmentStatsController(kvbase kv.Base, errch chan error) {
|
||||
ssChan := make(chan segment.SegmentStats, 10)
|
||||
ssChan := make(chan internalpb.SegmentStatistics, 10)
|
||||
ssClient := informer.NewPulsarClient()
|
||||
go segment.Listener(ssChan, ssClient)
|
||||
for {
|
||||
select {
|
||||
case ss := <-ssChan:
|
||||
errch <- ComputeCloseTime(ss, kvbase)
|
||||
errch <- UpdateSegmentStatus(ss, kvbase)
|
||||
case <-time.After(5 * time.Second):
|
||||
fmt.Println("wait for new request")
|
||||
return
|
||||
|
@ -30,15 +30,16 @@ func SegmentStatsController(kvbase kv.Base, errch chan error) {
|
|||
|
||||
}
|
||||
|
||||
func ComputeCloseTime(ss segment.SegmentStats, kvbase kv.Base) error {
|
||||
func ComputeCloseTime(ss internalpb.SegmentStatistics, kvbase kv.Base) error {
|
||||
if int(ss.MemorySize) > int(conf.Config.Master.SegmentThreshole*0.8) {
|
||||
currentTime := time.Now()
|
||||
memRate := int(ss.MemoryRate)
|
||||
//memRate := int(ss.MemoryRate)
|
||||
memRate := 1 // to do
|
||||
if memRate == 0 {
|
||||
memRate = 1
|
||||
}
|
||||
sec := int(conf.Config.Master.SegmentThreshole*0.2) / memRate
|
||||
data, err := kvbase.Load("segment/" + strconv.Itoa(int(ss.SegementID)))
|
||||
data, err := kvbase.Load("segment/" + strconv.Itoa(int(ss.SegmentId)))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -52,7 +53,7 @@ func ComputeCloseTime(ss segment.SegmentStats, kvbase kv.Base) error {
|
|||
if err != nil {
|
||||
return err
|
||||
}
|
||||
kvbase.Save("segment/"+strconv.Itoa(int(ss.SegementID)), updateData)
|
||||
kvbase.Save("segment/"+strconv.Itoa(int(ss.SegmentId)), updateData)
|
||||
//create new segment
|
||||
newSegID := id.New().Uint64()
|
||||
newSeg := segment.NewSegment(newSegID, seg.CollectionID, seg.CollectionName, "default", seg.ChannelStart, seg.ChannelEnd, currentTime, time.Unix(1<<36-1, 0))
|
||||
|
@ -80,35 +81,3 @@ func ComputeCloseTime(ss segment.SegmentStats, kvbase kv.Base) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func UpdateSegmentStatus(ss segment.SegmentStats, kvbase kv.Base) error {
|
||||
segmentData, err := kvbase.Load("segment/" + strconv.Itoa(int(ss.SegementID)))
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
seg, err := segment.JSON2Segment(segmentData)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
var changed bool
|
||||
changed = false
|
||||
if seg.Status != ss.Status {
|
||||
changed = true
|
||||
seg.Status = ss.Status
|
||||
}
|
||||
if seg.Rows != ss.Rows {
|
||||
changed = true
|
||||
seg.Rows = ss.Rows
|
||||
}
|
||||
|
||||
if changed {
|
||||
segData, err := segment.Segment2JSON(*seg)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
err = kvbase.Save("segment/"+strconv.Itoa(int(seg.CollectionID)), segData)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
|
|
@ -2,15 +2,16 @@ package grpc
|
|||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
||||
"net"
|
||||
"strconv"
|
||||
|
||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||
"github.com/zilliztech/milvus-distributed/internal/master/controller"
|
||||
masterpb "github.com/zilliztech/milvus-distributed/internal/proto/master"
|
||||
messagepb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
"github.com/zilliztech/milvus-distributed/internal/master/kv"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
||||
messagepb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
"google.golang.org/grpc"
|
||||
)
|
||||
|
||||
|
@ -37,33 +38,119 @@ type GRPCMasterServer struct {
|
|||
kvbase kv.Base
|
||||
}
|
||||
|
||||
func (ms GRPCMasterServer) CreateCollection(ctx context.Context, in *messagepb.Mapping) (*messagepb.Status, error) {
|
||||
// ms.CreateRequest <- in2
|
||||
fmt.Println("Handle a new create collection request")
|
||||
err := controller.WriteCollection2Datastore(in, ms.kvbase)
|
||||
if err != nil {
|
||||
return &messagepb.Status{
|
||||
ErrorCode: 100,
|
||||
Reason: "",
|
||||
}, err
|
||||
}
|
||||
return &messagepb.Status{
|
||||
func (ms GRPCMasterServer) CreateCollection(ctx context.Context, in *internalpb.CreateCollectionRequest) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (ms GRPCMasterServer) CreateIndex(ctx context.Context, in *messagepb.IndexParam) (*messagepb.Status, error) {
|
||||
fmt.Println("Handle a new create index request")
|
||||
err := controller.UpdateCollectionIndex(in, ms.kvbase)
|
||||
if err != nil {
|
||||
return &messagepb.Status{
|
||||
ErrorCode: 100,
|
||||
Reason: "",
|
||||
}, err
|
||||
}
|
||||
return &messagepb.Status{
|
||||
func (ms GRPCMasterServer) DropCollection(ctx context.Context, in *internalpb.DropCollectionRequest) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (ms GRPCMasterServer) HasCollection(ctx context.Context, in *internalpb.HasCollectionRequest) (*servicepb.BoolResponse, error) {
|
||||
return &servicepb.BoolResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
Value: true,
|
||||
},nil
|
||||
}
|
||||
|
||||
func (ms GRPCMasterServer) DescribeCollection(ctx context.Context, in *internalpb.DescribeCollectionRequest) (*servicepb.CollectionDescription, error) {
|
||||
return &servicepb.CollectionDescription{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
},nil
|
||||
}
|
||||
|
||||
func (ms GRPCMasterServer) ShowCollections(ctx context.Context, in *internalpb.ShowCollectionRequest) (*servicepb.StringListResponse, error) {
|
||||
return &servicepb.StringListResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
},nil
|
||||
}
|
||||
|
||||
|
||||
func (ms GRPCMasterServer) CreatePartition(ctx context.Context, in *internalpb.CreatePartitionRequest) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
|
||||
func (ms GRPCMasterServer) DropPartition(ctx context.Context, in *internalpb.DropPartitionRequest) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (ms GRPCMasterServer) HasPartition(ctx context.Context, in *internalpb.HasPartitionRequest) (*servicepb.BoolResponse, error) {
|
||||
return &servicepb.BoolResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
Value: true,
|
||||
},nil
|
||||
}
|
||||
|
||||
func (ms GRPCMasterServer) DescribePartition(ctx context.Context, in *internalpb.DescribePartitionRequest) (*servicepb.PartitionDescription, error) {
|
||||
return &servicepb.PartitionDescription{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
},nil
|
||||
}
|
||||
|
||||
func (ms GRPCMasterServer) ShowPartitions(ctx context.Context, in *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error) {
|
||||
return &servicepb.StringListResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
},nil
|
||||
}
|
||||
|
||||
//func (ms GRPCMasterServer) CreateCollection(ctx context.Context, in *messagepb.Mapping) (*messagepb.Status, error) {
|
||||
// // ms.CreateRequest <- in2
|
||||
// fmt.Println("Handle a new create collection request")
|
||||
// err := controller.WriteCollection2Datastore(in, ms.kvbase)
|
||||
// if err != nil {
|
||||
// return &messagepb.Status{
|
||||
// ErrorCode: 100,
|
||||
// Reason: "",
|
||||
// }, err
|
||||
// }
|
||||
// return &messagepb.Status{
|
||||
// ErrorCode: 0,
|
||||
// Reason: "",
|
||||
// }, nil
|
||||
//}
|
||||
|
||||
//func (ms GRPCMasterServer) CreateIndex(ctx context.Context, in *messagepb.IndexParam) (*messagepb.Status, error) {
|
||||
// fmt.Println("Handle a new create index request")
|
||||
// err := controller.UpdateCollectionIndex(in, ms.kvbase)
|
||||
// if err != nil {
|
||||
// return &messagepb.Status{
|
||||
// ErrorCode: 100,
|
||||
// Reason: "",
|
||||
// }, err
|
||||
// }
|
||||
// return &messagepb.Status{
|
||||
// ErrorCode: 0,
|
||||
// Reason: "",
|
||||
// }, nil
|
||||
//}
|
||||
|
|
|
@ -3,7 +3,6 @@ package segment
|
|||
import (
|
||||
"time"
|
||||
|
||||
masterpb "github.com/zilliztech/milvus-distributed/internal/proto/master"
|
||||
jsoniter "github.com/json-iterator/go"
|
||||
)
|
||||
|
||||
|
@ -18,7 +17,6 @@ type Segment struct {
|
|||
OpenTimeStamp uint64 `json:"open_timestamp"`
|
||||
CloseTimeStamp uint64 `json:"close_timestamp"`
|
||||
CollectionName string `json:"collection_name"`
|
||||
Status masterpb.SegmentStatus `json:"segment_status"`
|
||||
Rows int64 `json:"rows"`
|
||||
}
|
||||
|
||||
|
|
|
@ -1,15 +1,13 @@
|
|||
package segment
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/gob"
|
||||
"fmt"
|
||||
"log"
|
||||
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||
masterpb "github.com/zilliztech/milvus-distributed/internal/proto/master"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/master/informer"
|
||||
)
|
||||
|
||||
|
@ -17,31 +15,11 @@ type SegmentStats struct {
|
|||
SegementID uint64
|
||||
MemorySize uint64
|
||||
MemoryRate float64
|
||||
Status masterpb.SegmentStatus
|
||||
Rows int64
|
||||
}
|
||||
|
||||
func SegmentMarshal(s SegmentStats) ([]byte, error) {
|
||||
var nb bytes.Buffer
|
||||
enc := gob.NewEncoder(&nb)
|
||||
err := enc.Encode(s)
|
||||
if err != nil {
|
||||
return []byte{}, err
|
||||
}
|
||||
return nb.Bytes(), nil
|
||||
}
|
||||
|
||||
func SegmentUnMarshal(data []byte) (SegmentStats, error) {
|
||||
var ss SegmentStats
|
||||
dec := gob.NewDecoder(bytes.NewBuffer(data))
|
||||
err := dec.Decode(&ss)
|
||||
if err != nil {
|
||||
return SegmentStats{}, err
|
||||
}
|
||||
return ss, nil
|
||||
}
|
||||
|
||||
func Listener(ssChan chan SegmentStats, pc informer.PulsarClient) error {
|
||||
func Listener(ssChan chan internalpb.SegmentStatistics, pc informer.PulsarClient) error {
|
||||
consumer, err := pc.Client.Subscribe(pulsar.ConsumerOptions{
|
||||
Topic: conf.Config.Master.PulsarTopic,
|
||||
SubscriptionName: "my-sub",
|
||||
|
@ -55,9 +33,11 @@ func Listener(ssChan chan SegmentStats, pc informer.PulsarClient) error {
|
|||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
m, _ := SegmentUnMarshal(msg.Payload())
|
||||
|
||||
var m internalpb.SegmentStatistics
|
||||
proto.Unmarshal(msg.Payload(), &m)
|
||||
fmt.Printf("Received message msgId: %#v -- content: '%s'\n",
|
||||
msg.ID(), m.SegementID)
|
||||
msg.ID(), m.SegmentId)
|
||||
ssChan <- m
|
||||
consumer.Ack(msg)
|
||||
}
|
||||
|
|
|
@ -8,7 +8,7 @@ import (
|
|||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||
masterPb "github.com/zilliztech/milvus-distributed/internal/proto/master"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
"github.com/zilliztech/milvus-distributed/internal/timesync"
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
@ -77,7 +77,7 @@ func (mc *ReaderMessageClient) SendResult(ctx context.Context, msg msgpb.QueryRe
|
|||
}
|
||||
}
|
||||
|
||||
func (mc *ReaderMessageClient) SendSegmentsStatistic(ctx context.Context, statisticData *[]masterPb.SegmentStat) {
|
||||
func (mc *ReaderMessageClient) SendSegmentsStatistic(ctx context.Context, statisticData *[]internalpb.SegmentStatistics) {
|
||||
for _, data := range *statisticData {
|
||||
var stat, _ = proto.Marshal(&data)
|
||||
if _, err := mc.segmentsStatisticProducer.Send(ctx, &pulsar.ProducerMessage{
|
||||
|
|
|
@ -1,3 +1,7 @@
|
|||
syntax = "proto3";
|
||||
|
||||
package milvus.proto.common;
|
||||
option go_package="github.com/zilliztech/milvus-distributed/internal/proto/commonpb";
|
||||
|
||||
enum ErrorCode {
|
||||
SUCCESS = 0;
|
||||
|
@ -26,6 +30,8 @@ enum ErrorCode {
|
|||
OUT_OF_MEMORY = 24;
|
||||
}
|
||||
|
||||
message Empty{}
|
||||
|
||||
|
||||
message Status {
|
||||
ErrorCode error_code = 1;
|
||||
|
@ -44,3 +50,8 @@ message Blob {
|
|||
}
|
||||
|
||||
|
||||
message Address {
|
||||
string ip = 1;
|
||||
int64 port = 2;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,375 @@
|
|||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// source: common.proto
|
||||
|
||||
package commonpb
|
||||
|
||||
import (
|
||||
fmt "fmt"
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
math "math"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
var _ = fmt.Errorf
|
||||
var _ = math.Inf
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the proto package it is being compiled against.
|
||||
// A compilation error at this line likely means your copy of the
|
||||
// proto package needs to be updated.
|
||||
const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
|
||||
|
||||
type ErrorCode int32
|
||||
|
||||
const (
|
||||
ErrorCode_SUCCESS ErrorCode = 0
|
||||
ErrorCode_UNEXPECTED_ERROR ErrorCode = 1
|
||||
ErrorCode_CONNECT_FAILED ErrorCode = 2
|
||||
ErrorCode_PERMISSION_DENIED ErrorCode = 3
|
||||
ErrorCode_COLLECTION_NOT_EXISTS ErrorCode = 4
|
||||
ErrorCode_ILLEGAL_ARGUMENT ErrorCode = 5
|
||||
ErrorCode_ILLEGAL_DIMENSION ErrorCode = 7
|
||||
ErrorCode_ILLEGAL_INDEX_TYPE ErrorCode = 8
|
||||
ErrorCode_ILLEGAL_COLLECTION_NAME ErrorCode = 9
|
||||
ErrorCode_ILLEGAL_TOPK ErrorCode = 10
|
||||
ErrorCode_ILLEGAL_ROWRECORD ErrorCode = 11
|
||||
ErrorCode_ILLEGAL_VECTOR_ID ErrorCode = 12
|
||||
ErrorCode_ILLEGAL_SEARCH_RESULT ErrorCode = 13
|
||||
ErrorCode_FILE_NOT_FOUND ErrorCode = 14
|
||||
ErrorCode_META_FAILED ErrorCode = 15
|
||||
ErrorCode_CACHE_FAILED ErrorCode = 16
|
||||
ErrorCode_CANNOT_CREATE_FOLDER ErrorCode = 17
|
||||
ErrorCode_CANNOT_CREATE_FILE ErrorCode = 18
|
||||
ErrorCode_CANNOT_DELETE_FOLDER ErrorCode = 19
|
||||
ErrorCode_CANNOT_DELETE_FILE ErrorCode = 20
|
||||
ErrorCode_BUILD_INDEX_ERROR ErrorCode = 21
|
||||
ErrorCode_ILLEGAL_NLIST ErrorCode = 22
|
||||
ErrorCode_ILLEGAL_METRIC_TYPE ErrorCode = 23
|
||||
ErrorCode_OUT_OF_MEMORY ErrorCode = 24
|
||||
)
|
||||
|
||||
var ErrorCode_name = map[int32]string{
|
||||
0: "SUCCESS",
|
||||
1: "UNEXPECTED_ERROR",
|
||||
2: "CONNECT_FAILED",
|
||||
3: "PERMISSION_DENIED",
|
||||
4: "COLLECTION_NOT_EXISTS",
|
||||
5: "ILLEGAL_ARGUMENT",
|
||||
7: "ILLEGAL_DIMENSION",
|
||||
8: "ILLEGAL_INDEX_TYPE",
|
||||
9: "ILLEGAL_COLLECTION_NAME",
|
||||
10: "ILLEGAL_TOPK",
|
||||
11: "ILLEGAL_ROWRECORD",
|
||||
12: "ILLEGAL_VECTOR_ID",
|
||||
13: "ILLEGAL_SEARCH_RESULT",
|
||||
14: "FILE_NOT_FOUND",
|
||||
15: "META_FAILED",
|
||||
16: "CACHE_FAILED",
|
||||
17: "CANNOT_CREATE_FOLDER",
|
||||
18: "CANNOT_CREATE_FILE",
|
||||
19: "CANNOT_DELETE_FOLDER",
|
||||
20: "CANNOT_DELETE_FILE",
|
||||
21: "BUILD_INDEX_ERROR",
|
||||
22: "ILLEGAL_NLIST",
|
||||
23: "ILLEGAL_METRIC_TYPE",
|
||||
24: "OUT_OF_MEMORY",
|
||||
}
|
||||
|
||||
var ErrorCode_value = map[string]int32{
|
||||
"SUCCESS": 0,
|
||||
"UNEXPECTED_ERROR": 1,
|
||||
"CONNECT_FAILED": 2,
|
||||
"PERMISSION_DENIED": 3,
|
||||
"COLLECTION_NOT_EXISTS": 4,
|
||||
"ILLEGAL_ARGUMENT": 5,
|
||||
"ILLEGAL_DIMENSION": 7,
|
||||
"ILLEGAL_INDEX_TYPE": 8,
|
||||
"ILLEGAL_COLLECTION_NAME": 9,
|
||||
"ILLEGAL_TOPK": 10,
|
||||
"ILLEGAL_ROWRECORD": 11,
|
||||
"ILLEGAL_VECTOR_ID": 12,
|
||||
"ILLEGAL_SEARCH_RESULT": 13,
|
||||
"FILE_NOT_FOUND": 14,
|
||||
"META_FAILED": 15,
|
||||
"CACHE_FAILED": 16,
|
||||
"CANNOT_CREATE_FOLDER": 17,
|
||||
"CANNOT_CREATE_FILE": 18,
|
||||
"CANNOT_DELETE_FOLDER": 19,
|
||||
"CANNOT_DELETE_FILE": 20,
|
||||
"BUILD_INDEX_ERROR": 21,
|
||||
"ILLEGAL_NLIST": 22,
|
||||
"ILLEGAL_METRIC_TYPE": 23,
|
||||
"OUT_OF_MEMORY": 24,
|
||||
}
|
||||
|
||||
func (x ErrorCode) String() string {
|
||||
return proto.EnumName(ErrorCode_name, int32(x))
|
||||
}
|
||||
|
||||
func (ErrorCode) EnumDescriptor() ([]byte, []int) {
|
||||
return fileDescriptor_555bd8c177793206, []int{0}
|
||||
}
|
||||
|
||||
type Empty struct {
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Empty) Reset() { *m = Empty{} }
|
||||
func (m *Empty) String() string { return proto.CompactTextString(m) }
|
||||
func (*Empty) ProtoMessage() {}
|
||||
func (*Empty) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_555bd8c177793206, []int{0}
|
||||
}
|
||||
|
||||
func (m *Empty) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Empty.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Empty) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Empty.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *Empty) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Empty.Merge(m, src)
|
||||
}
|
||||
func (m *Empty) XXX_Size() int {
|
||||
return xxx_messageInfo_Empty.Size(m)
|
||||
}
|
||||
func (m *Empty) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Empty.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Empty proto.InternalMessageInfo
|
||||
|
||||
type Status struct {
|
||||
ErrorCode ErrorCode `protobuf:"varint,1,opt,name=error_code,json=errorCode,proto3,enum=milvus.proto.common.ErrorCode" json:"error_code,omitempty"`
|
||||
Reason string `protobuf:"bytes,2,opt,name=reason,proto3" json:"reason,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Status) Reset() { *m = Status{} }
|
||||
func (m *Status) String() string { return proto.CompactTextString(m) }
|
||||
func (*Status) ProtoMessage() {}
|
||||
func (*Status) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_555bd8c177793206, []int{1}
|
||||
}
|
||||
|
||||
func (m *Status) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Status.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Status) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Status.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *Status) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Status.Merge(m, src)
|
||||
}
|
||||
func (m *Status) XXX_Size() int {
|
||||
return xxx_messageInfo_Status.Size(m)
|
||||
}
|
||||
func (m *Status) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Status.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Status proto.InternalMessageInfo
|
||||
|
||||
func (m *Status) GetErrorCode() ErrorCode {
|
||||
if m != nil {
|
||||
return m.ErrorCode
|
||||
}
|
||||
return ErrorCode_SUCCESS
|
||||
}
|
||||
|
||||
func (m *Status) GetReason() string {
|
||||
if m != nil {
|
||||
return m.Reason
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type KeyValuePair struct {
|
||||
Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
|
||||
Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *KeyValuePair) Reset() { *m = KeyValuePair{} }
|
||||
func (m *KeyValuePair) String() string { return proto.CompactTextString(m) }
|
||||
func (*KeyValuePair) ProtoMessage() {}
|
||||
func (*KeyValuePair) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_555bd8c177793206, []int{2}
|
||||
}
|
||||
|
||||
func (m *KeyValuePair) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_KeyValuePair.Unmarshal(m, b)
|
||||
}
|
||||
func (m *KeyValuePair) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_KeyValuePair.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *KeyValuePair) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_KeyValuePair.Merge(m, src)
|
||||
}
|
||||
func (m *KeyValuePair) XXX_Size() int {
|
||||
return xxx_messageInfo_KeyValuePair.Size(m)
|
||||
}
|
||||
func (m *KeyValuePair) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_KeyValuePair.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_KeyValuePair proto.InternalMessageInfo
|
||||
|
||||
func (m *KeyValuePair) GetKey() string {
|
||||
if m != nil {
|
||||
return m.Key
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *KeyValuePair) GetValue() string {
|
||||
if m != nil {
|
||||
return m.Value
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type Blob struct {
|
||||
Value []byte `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Blob) Reset() { *m = Blob{} }
|
||||
func (m *Blob) String() string { return proto.CompactTextString(m) }
|
||||
func (*Blob) ProtoMessage() {}
|
||||
func (*Blob) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_555bd8c177793206, []int{3}
|
||||
}
|
||||
|
||||
func (m *Blob) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Blob.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Blob) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Blob.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *Blob) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Blob.Merge(m, src)
|
||||
}
|
||||
func (m *Blob) XXX_Size() int {
|
||||
return xxx_messageInfo_Blob.Size(m)
|
||||
}
|
||||
func (m *Blob) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Blob.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Blob proto.InternalMessageInfo
|
||||
|
||||
func (m *Blob) GetValue() []byte {
|
||||
if m != nil {
|
||||
return m.Value
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type Address struct {
|
||||
Ip string `protobuf:"bytes,1,opt,name=ip,proto3" json:"ip,omitempty"`
|
||||
Port int64 `protobuf:"varint,2,opt,name=port,proto3" json:"port,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Address) Reset() { *m = Address{} }
|
||||
func (m *Address) String() string { return proto.CompactTextString(m) }
|
||||
func (*Address) ProtoMessage() {}
|
||||
func (*Address) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_555bd8c177793206, []int{4}
|
||||
}
|
||||
|
||||
func (m *Address) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Address.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Address) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Address.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *Address) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Address.Merge(m, src)
|
||||
}
|
||||
func (m *Address) XXX_Size() int {
|
||||
return xxx_messageInfo_Address.Size(m)
|
||||
}
|
||||
func (m *Address) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Address.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Address proto.InternalMessageInfo
|
||||
|
||||
func (m *Address) GetIp() string {
|
||||
if m != nil {
|
||||
return m.Ip
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *Address) GetPort() int64 {
|
||||
if m != nil {
|
||||
return m.Port
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterEnum("milvus.proto.common.ErrorCode", ErrorCode_name, ErrorCode_value)
|
||||
proto.RegisterType((*Empty)(nil), "milvus.proto.common.Empty")
|
||||
proto.RegisterType((*Status)(nil), "milvus.proto.common.Status")
|
||||
proto.RegisterType((*KeyValuePair)(nil), "milvus.proto.common.KeyValuePair")
|
||||
proto.RegisterType((*Blob)(nil), "milvus.proto.common.Blob")
|
||||
proto.RegisterType((*Address)(nil), "milvus.proto.common.Address")
|
||||
}
|
||||
|
||||
func init() { proto.RegisterFile("common.proto", fileDescriptor_555bd8c177793206) }
|
||||
|
||||
var fileDescriptor_555bd8c177793206 = []byte{
|
||||
// 577 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x53, 0x51, 0x4f, 0xdb, 0x3c,
|
||||
0x14, 0xfd, 0x5a, 0x0a, 0xfd, 0x7a, 0x29, 0xc5, 0x98, 0x02, 0x9d, 0x36, 0x4d, 0xa8, 0x4f, 0x68,
|
||||
0x12, 0xad, 0xb4, 0x49, 0x7b, 0x9b, 0xb4, 0xd4, 0xb9, 0x05, 0x0b, 0x27, 0xae, 0x1c, 0x87, 0xc1,
|
||||
0x5e, 0xac, 0x96, 0x46, 0x23, 0x5a, 0xdb, 0x54, 0x69, 0x8a, 0x04, 0x3f, 0x67, 0xbf, 0x74, 0x72,
|
||||
0xd2, 0x8c, 0x6a, 0xda, 0xdb, 0xbd, 0xe7, 0xf8, 0x1c, 0xfb, 0x9e, 0xe4, 0x42, 0xf3, 0x21, 0x99,
|
||||
0xcf, 0x93, 0x45, 0x6f, 0x99, 0x26, 0x59, 0x42, 0x8f, 0xe7, 0xf1, 0xec, 0x69, 0xbd, 0x2a, 0xba,
|
||||
0x5e, 0x41, 0x75, 0xeb, 0xb0, 0x8b, 0xf3, 0x65, 0xf6, 0xdc, 0x35, 0xb0, 0x17, 0x64, 0xe3, 0x6c,
|
||||
0xbd, 0xa2, 0x5f, 0x00, 0xa2, 0x34, 0x4d, 0x52, 0xf3, 0x90, 0x4c, 0xa3, 0x4e, 0xe5, 0xbc, 0x72,
|
||||
0xd1, 0xfa, 0xf8, 0xbe, 0xf7, 0x0f, 0x71, 0x0f, 0xed, 0x31, 0x96, 0x4c, 0x23, 0xd5, 0x88, 0xca,
|
||||
0x92, 0x9e, 0xc2, 0x5e, 0x1a, 0x8d, 0x57, 0xc9, 0xa2, 0x53, 0x3d, 0xaf, 0x5c, 0x34, 0xd4, 0xa6,
|
||||
0xeb, 0x7e, 0x86, 0xe6, 0x4d, 0xf4, 0x7c, 0x3b, 0x9e, 0xad, 0xa3, 0xd1, 0x38, 0x4e, 0x29, 0x81,
|
||||
0x9d, 0x9f, 0xd1, 0x73, 0xee, 0xdf, 0x50, 0xb6, 0xa4, 0x6d, 0xd8, 0x7d, 0xb2, 0xf4, 0x46, 0x58,
|
||||
0x34, 0xdd, 0x77, 0x50, 0x1b, 0xcc, 0x92, 0xc9, 0x2b, 0x6b, 0x15, 0xcd, 0x92, 0xbd, 0x84, 0xba,
|
||||
0x33, 0x9d, 0xa6, 0xd1, 0x6a, 0x45, 0x5b, 0x50, 0x8d, 0x97, 0x1b, 0xbf, 0x6a, 0xbc, 0xa4, 0x14,
|
||||
0x6a, 0xcb, 0x24, 0xcd, 0x72, 0xb7, 0x1d, 0x95, 0xd7, 0x1f, 0x7e, 0xd5, 0xa0, 0xf1, 0xe7, 0xd5,
|
||||
0x74, 0x1f, 0xea, 0x41, 0xc8, 0x18, 0x06, 0x01, 0xf9, 0x8f, 0xb6, 0x81, 0x84, 0x3e, 0xde, 0x8d,
|
||||
0x90, 0x69, 0x74, 0x0d, 0x2a, 0x25, 0x15, 0xa9, 0x50, 0x0a, 0x2d, 0x26, 0x7d, 0x1f, 0x99, 0x36,
|
||||
0x43, 0x87, 0x0b, 0x74, 0x49, 0x95, 0x9e, 0xc0, 0xd1, 0x08, 0x95, 0xc7, 0x83, 0x80, 0x4b, 0xdf,
|
||||
0xb8, 0xe8, 0x73, 0x74, 0xc9, 0x0e, 0x7d, 0x03, 0x27, 0x4c, 0x0a, 0x81, 0x4c, 0x5b, 0xd8, 0x97,
|
||||
0xda, 0xe0, 0x1d, 0x0f, 0x74, 0x40, 0x6a, 0xd6, 0x9b, 0x0b, 0x81, 0x57, 0x8e, 0x30, 0x8e, 0xba,
|
||||
0x0a, 0x3d, 0xf4, 0x35, 0xd9, 0xb5, 0x3e, 0x25, 0xea, 0x72, 0x0f, 0x7d, 0x6b, 0x47, 0xea, 0xf4,
|
||||
0x14, 0x68, 0x09, 0x73, 0xdf, 0xc5, 0x3b, 0xa3, 0xef, 0x47, 0x48, 0xfe, 0xa7, 0x6f, 0xe1, 0xac,
|
||||
0xc4, 0xb7, 0xef, 0x71, 0x3c, 0x24, 0x0d, 0x4a, 0xa0, 0x59, 0x92, 0x5a, 0x8e, 0x6e, 0x08, 0x6c,
|
||||
0xbb, 0x2b, 0xf9, 0x4d, 0x21, 0x93, 0xca, 0x25, 0xfb, 0xdb, 0xf0, 0x2d, 0x32, 0x2d, 0x95, 0xe1,
|
||||
0x2e, 0x69, 0xda, 0xc7, 0x97, 0x70, 0x80, 0x8e, 0x62, 0xd7, 0x46, 0x61, 0x10, 0x0a, 0x4d, 0x0e,
|
||||
0x6c, 0x04, 0x43, 0x2e, 0x30, 0x9f, 0x68, 0x28, 0x43, 0xdf, 0x25, 0x2d, 0x7a, 0x08, 0xfb, 0x1e,
|
||||
0x6a, 0xa7, 0xcc, 0xe4, 0xd0, 0xde, 0xcf, 0x1c, 0x76, 0x8d, 0x25, 0x42, 0x68, 0x07, 0xda, 0xcc,
|
||||
0xf1, 0xad, 0x88, 0x29, 0x74, 0x34, 0x9a, 0xa1, 0x14, 0x2e, 0x2a, 0x72, 0x64, 0x07, 0xfc, 0x8b,
|
||||
0xe1, 0x02, 0x09, 0xdd, 0x52, 0xb8, 0x28, 0xf0, 0x55, 0x71, 0xbc, 0xa5, 0x28, 0x19, 0xab, 0x68,
|
||||
0xdb, 0x61, 0x06, 0x21, 0x17, 0xee, 0x26, 0xa8, 0xe2, 0xa3, 0x9d, 0xd0, 0x23, 0x38, 0x28, 0x87,
|
||||
0xf1, 0x05, 0x0f, 0x34, 0x39, 0xa5, 0x67, 0x70, 0x5c, 0x42, 0x1e, 0x6a, 0xc5, 0x59, 0x91, 0xea,
|
||||
0x99, 0x3d, 0x2b, 0x43, 0x6d, 0xe4, 0xd0, 0x78, 0xe8, 0x49, 0x75, 0x4f, 0x3a, 0x83, 0xc1, 0xf7,
|
||||
0xaf, 0x3f, 0xe2, 0xec, 0x71, 0x3d, 0xb1, 0xff, 0x79, 0xff, 0x25, 0x9e, 0xcd, 0xe2, 0x97, 0x2c,
|
||||
0x7a, 0x78, 0xec, 0x17, 0x3b, 0x70, 0x39, 0x8d, 0x57, 0x59, 0x1a, 0x4f, 0xd6, 0x59, 0x34, 0xed,
|
||||
0xc7, 0x8b, 0x2c, 0x4a, 0x17, 0xe3, 0x59, 0x3f, 0x5f, 0x8c, 0x7e, 0xb1, 0x18, 0xcb, 0xc9, 0x64,
|
||||
0x2f, 0xef, 0x3f, 0xfd, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x4f, 0x8c, 0xe4, 0x07, 0x83, 0x03, 0x00,
|
||||
0x00,
|
||||
}
|
|
@ -0,0 +1,40 @@
|
|||
syntax = "proto3";
|
||||
package milvus.proto.etcd;
|
||||
option go_package="github.com/zilliztech/milvus-distributed/internal/proto/etcdpb";
|
||||
|
||||
import "common.proto";
|
||||
import "schema.proto";
|
||||
|
||||
message TenantMeta {
|
||||
uint64 id = 1;
|
||||
uint64 num_query_nodes = 2;
|
||||
repeated string insert_channel_ids = 3;
|
||||
string query_channel_id = 4;
|
||||
}
|
||||
|
||||
message ProxyMeta {
|
||||
uint64 id = 1;
|
||||
common.Address address = 2;
|
||||
repeated string result_channel_ids = 3;
|
||||
}
|
||||
|
||||
|
||||
message CollectionMeta {
|
||||
uint64 id=1;
|
||||
schema.CollectionSchema schema=2;
|
||||
uint64 create_time=3;
|
||||
repeated uint64 segment_ids=4;
|
||||
repeated string partition_tags=5;
|
||||
}
|
||||
|
||||
|
||||
message SegmentMeta {
|
||||
uint64 segment_id=1;
|
||||
uint64 collection_id =2;
|
||||
string partition_tag=3;
|
||||
int32 channel_start=4;
|
||||
int32 channel_end=5;
|
||||
uint64 open_time=6;
|
||||
uint64 close_time=7;
|
||||
int64 num_rows=8;
|
||||
}
|
|
@ -0,0 +1,352 @@
|
|||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// source: etcd_meta.proto
|
||||
|
||||
package etcdpb
|
||||
|
||||
import (
|
||||
fmt "fmt"
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
schemapb "github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
math "math"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
var _ = fmt.Errorf
|
||||
var _ = math.Inf
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the proto package it is being compiled against.
|
||||
// A compilation error at this line likely means your copy of the
|
||||
// proto package needs to be updated.
|
||||
const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
|
||||
|
||||
type TenantMeta struct {
|
||||
Id uint64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
|
||||
NumQueryNodes uint64 `protobuf:"varint,2,opt,name=num_query_nodes,json=numQueryNodes,proto3" json:"num_query_nodes,omitempty"`
|
||||
InsertChannelIds []string `protobuf:"bytes,3,rep,name=insert_channel_ids,json=insertChannelIds,proto3" json:"insert_channel_ids,omitempty"`
|
||||
QueryChannelId string `protobuf:"bytes,4,opt,name=query_channel_id,json=queryChannelId,proto3" json:"query_channel_id,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *TenantMeta) Reset() { *m = TenantMeta{} }
|
||||
func (m *TenantMeta) String() string { return proto.CompactTextString(m) }
|
||||
func (*TenantMeta) ProtoMessage() {}
|
||||
func (*TenantMeta) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_975d306d62b73e88, []int{0}
|
||||
}
|
||||
|
||||
func (m *TenantMeta) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_TenantMeta.Unmarshal(m, b)
|
||||
}
|
||||
func (m *TenantMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_TenantMeta.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *TenantMeta) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_TenantMeta.Merge(m, src)
|
||||
}
|
||||
func (m *TenantMeta) XXX_Size() int {
|
||||
return xxx_messageInfo_TenantMeta.Size(m)
|
||||
}
|
||||
func (m *TenantMeta) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_TenantMeta.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_TenantMeta proto.InternalMessageInfo
|
||||
|
||||
func (m *TenantMeta) GetId() uint64 {
|
||||
if m != nil {
|
||||
return m.Id
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *TenantMeta) GetNumQueryNodes() uint64 {
|
||||
if m != nil {
|
||||
return m.NumQueryNodes
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *TenantMeta) GetInsertChannelIds() []string {
|
||||
if m != nil {
|
||||
return m.InsertChannelIds
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *TenantMeta) GetQueryChannelId() string {
|
||||
if m != nil {
|
||||
return m.QueryChannelId
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
type ProxyMeta struct {
|
||||
Id uint64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
|
||||
Address *commonpb.Address `protobuf:"bytes,2,opt,name=address,proto3" json:"address,omitempty"`
|
||||
ResultChannelIds []string `protobuf:"bytes,3,rep,name=result_channel_ids,json=resultChannelIds,proto3" json:"result_channel_ids,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *ProxyMeta) Reset() { *m = ProxyMeta{} }
|
||||
func (m *ProxyMeta) String() string { return proto.CompactTextString(m) }
|
||||
func (*ProxyMeta) ProtoMessage() {}
|
||||
func (*ProxyMeta) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_975d306d62b73e88, []int{1}
|
||||
}
|
||||
|
||||
func (m *ProxyMeta) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_ProxyMeta.Unmarshal(m, b)
|
||||
}
|
||||
func (m *ProxyMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_ProxyMeta.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *ProxyMeta) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_ProxyMeta.Merge(m, src)
|
||||
}
|
||||
func (m *ProxyMeta) XXX_Size() int {
|
||||
return xxx_messageInfo_ProxyMeta.Size(m)
|
||||
}
|
||||
func (m *ProxyMeta) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_ProxyMeta.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_ProxyMeta proto.InternalMessageInfo
|
||||
|
||||
func (m *ProxyMeta) GetId() uint64 {
|
||||
if m != nil {
|
||||
return m.Id
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *ProxyMeta) GetAddress() *commonpb.Address {
|
||||
if m != nil {
|
||||
return m.Address
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *ProxyMeta) GetResultChannelIds() []string {
|
||||
if m != nil {
|
||||
return m.ResultChannelIds
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type CollectionMeta struct {
|
||||
Id uint64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
|
||||
Schema *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"`
|
||||
CreateTime uint64 `protobuf:"varint,3,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"`
|
||||
SegmentIds []uint64 `protobuf:"varint,4,rep,packed,name=segment_ids,json=segmentIds,proto3" json:"segment_ids,omitempty"`
|
||||
PartitionTags []string `protobuf:"bytes,5,rep,name=partition_tags,json=partitionTags,proto3" json:"partition_tags,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *CollectionMeta) Reset() { *m = CollectionMeta{} }
|
||||
func (m *CollectionMeta) String() string { return proto.CompactTextString(m) }
|
||||
func (*CollectionMeta) ProtoMessage() {}
|
||||
func (*CollectionMeta) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_975d306d62b73e88, []int{2}
|
||||
}
|
||||
|
||||
func (m *CollectionMeta) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_CollectionMeta.Unmarshal(m, b)
|
||||
}
|
||||
func (m *CollectionMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_CollectionMeta.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *CollectionMeta) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_CollectionMeta.Merge(m, src)
|
||||
}
|
||||
func (m *CollectionMeta) XXX_Size() int {
|
||||
return xxx_messageInfo_CollectionMeta.Size(m)
|
||||
}
|
||||
func (m *CollectionMeta) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_CollectionMeta.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_CollectionMeta proto.InternalMessageInfo
|
||||
|
||||
func (m *CollectionMeta) GetId() uint64 {
|
||||
if m != nil {
|
||||
return m.Id
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *CollectionMeta) GetSchema() *schemapb.CollectionSchema {
|
||||
if m != nil {
|
||||
return m.Schema
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *CollectionMeta) GetCreateTime() uint64 {
|
||||
if m != nil {
|
||||
return m.CreateTime
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *CollectionMeta) GetSegmentIds() []uint64 {
|
||||
if m != nil {
|
||||
return m.SegmentIds
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *CollectionMeta) GetPartitionTags() []string {
|
||||
if m != nil {
|
||||
return m.PartitionTags
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type SegmentMeta struct {
|
||||
SegmentId uint64 `protobuf:"varint,1,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"`
|
||||
CollectionId uint64 `protobuf:"varint,2,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
|
||||
PartitionTag string `protobuf:"bytes,3,opt,name=partition_tag,json=partitionTag,proto3" json:"partition_tag,omitempty"`
|
||||
ChannelStart int32 `protobuf:"varint,4,opt,name=channel_start,json=channelStart,proto3" json:"channel_start,omitempty"`
|
||||
ChannelEnd int32 `protobuf:"varint,5,opt,name=channel_end,json=channelEnd,proto3" json:"channel_end,omitempty"`
|
||||
OpenTime uint64 `protobuf:"varint,6,opt,name=open_time,json=openTime,proto3" json:"open_time,omitempty"`
|
||||
CloseTime uint64 `protobuf:"varint,7,opt,name=close_time,json=closeTime,proto3" json:"close_time,omitempty"`
|
||||
NumRows int64 `protobuf:"varint,8,opt,name=num_rows,json=numRows,proto3" json:"num_rows,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *SegmentMeta) Reset() { *m = SegmentMeta{} }
|
||||
func (m *SegmentMeta) String() string { return proto.CompactTextString(m) }
|
||||
func (*SegmentMeta) ProtoMessage() {}
|
||||
func (*SegmentMeta) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_975d306d62b73e88, []int{3}
|
||||
}
|
||||
|
||||
func (m *SegmentMeta) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_SegmentMeta.Unmarshal(m, b)
|
||||
}
|
||||
func (m *SegmentMeta) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_SegmentMeta.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *SegmentMeta) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_SegmentMeta.Merge(m, src)
|
||||
}
|
||||
func (m *SegmentMeta) XXX_Size() int {
|
||||
return xxx_messageInfo_SegmentMeta.Size(m)
|
||||
}
|
||||
func (m *SegmentMeta) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_SegmentMeta.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_SegmentMeta proto.InternalMessageInfo
|
||||
|
||||
func (m *SegmentMeta) GetSegmentId() uint64 {
|
||||
if m != nil {
|
||||
return m.SegmentId
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SegmentMeta) GetCollectionId() uint64 {
|
||||
if m != nil {
|
||||
return m.CollectionId
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SegmentMeta) GetPartitionTag() string {
|
||||
if m != nil {
|
||||
return m.PartitionTag
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *SegmentMeta) GetChannelStart() int32 {
|
||||
if m != nil {
|
||||
return m.ChannelStart
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SegmentMeta) GetChannelEnd() int32 {
|
||||
if m != nil {
|
||||
return m.ChannelEnd
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SegmentMeta) GetOpenTime() uint64 {
|
||||
if m != nil {
|
||||
return m.OpenTime
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SegmentMeta) GetCloseTime() uint64 {
|
||||
if m != nil {
|
||||
return m.CloseTime
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SegmentMeta) GetNumRows() int64 {
|
||||
if m != nil {
|
||||
return m.NumRows
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterType((*TenantMeta)(nil), "milvus.proto.etcd.TenantMeta")
|
||||
proto.RegisterType((*ProxyMeta)(nil), "milvus.proto.etcd.ProxyMeta")
|
||||
proto.RegisterType((*CollectionMeta)(nil), "milvus.proto.etcd.CollectionMeta")
|
||||
proto.RegisterType((*SegmentMeta)(nil), "milvus.proto.etcd.SegmentMeta")
|
||||
}
|
||||
|
||||
func init() { proto.RegisterFile("etcd_meta.proto", fileDescriptor_975d306d62b73e88) }
|
||||
|
||||
var fileDescriptor_975d306d62b73e88 = []byte{
|
||||
// 503 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x93, 0x4d, 0x8f, 0xd3, 0x30,
|
||||
0x10, 0x86, 0x95, 0x7e, 0x67, 0xfa, 0xb1, 0x4b, 0x4e, 0x61, 0x61, 0x45, 0x55, 0xb4, 0x28, 0x07,
|
||||
0x68, 0x24, 0x90, 0xb8, 0x81, 0x80, 0x15, 0x07, 0x0e, 0x20, 0x48, 0x7b, 0xe2, 0x12, 0xb9, 0xf1,
|
||||
0xa8, 0xb5, 0x14, 0xdb, 0xc5, 0x76, 0x58, 0x76, 0x6f, 0xfc, 0x06, 0xae, 0xfc, 0x20, 0x7e, 0x16,
|
||||
0xf2, 0x07, 0x0d, 0x95, 0x7a, 0xcc, 0x33, 0xaf, 0xc7, 0xef, 0xbc, 0xe3, 0xc0, 0x19, 0x9a, 0x8a,
|
||||
0x96, 0x1c, 0x0d, 0x59, 0xee, 0x95, 0x34, 0x32, 0xb9, 0xc7, 0x59, 0xfd, 0xbd, 0xd1, 0xfe, 0x6b,
|
||||
0x69, 0xab, 0x17, 0x93, 0x4a, 0x72, 0x2e, 0x85, 0x47, 0x17, 0x13, 0x5d, 0xed, 0x90, 0x07, 0xf9,
|
||||
0xe2, 0x77, 0x04, 0xb0, 0x46, 0x41, 0x84, 0xf9, 0x88, 0x86, 0x24, 0x33, 0xe8, 0x30, 0x9a, 0x46,
|
||||
0xf3, 0x28, 0xeb, 0x15, 0x1d, 0x46, 0x93, 0x27, 0x70, 0x26, 0x1a, 0x5e, 0x7e, 0x6b, 0x50, 0xdd,
|
||||
0x96, 0x42, 0x52, 0xd4, 0x69, 0xc7, 0x15, 0xa7, 0xa2, 0xe1, 0x5f, 0x2c, 0xfd, 0x64, 0x61, 0xf2,
|
||||
0x14, 0x12, 0x26, 0x34, 0x2a, 0x53, 0x56, 0x3b, 0x22, 0x04, 0xd6, 0x25, 0xa3, 0x3a, 0xed, 0xce,
|
||||
0xbb, 0x59, 0x5c, 0x9c, 0xfb, 0xca, 0xb5, 0x2f, 0x7c, 0xa0, 0x3a, 0xc9, 0xe0, 0xdc, 0x77, 0x6c,
|
||||
0xc5, 0x69, 0x6f, 0x1e, 0x65, 0x71, 0x31, 0x73, 0xfc, 0x20, 0x5d, 0xfc, 0x8c, 0x20, 0xfe, 0xac,
|
||||
0xe4, 0x8f, 0xdb, 0x93, 0xee, 0x5e, 0xc2, 0x90, 0x50, 0xaa, 0x50, 0x7b, 0x57, 0xe3, 0xe7, 0x0f,
|
||||
0x97, 0x47, 0xd3, 0x87, 0xb9, 0xdf, 0x7a, 0x4d, 0xf1, 0x4f, 0x6c, 0xdd, 0x2a, 0xd4, 0x4d, 0x7d,
|
||||
0xd2, 0xad, 0xaf, 0xb4, 0x6e, 0x17, 0x7f, 0x22, 0x98, 0x5d, 0xcb, 0xba, 0xc6, 0xca, 0x30, 0x29,
|
||||
0x4e, 0x1a, 0x79, 0x05, 0x03, 0x9f, 0x6a, 0xf0, 0x71, 0x75, 0xec, 0x23, 0x24, 0xde, 0x36, 0x59,
|
||||
0x39, 0x50, 0x84, 0x43, 0xc9, 0x23, 0x18, 0x57, 0x0a, 0x89, 0xc1, 0xd2, 0x30, 0x8e, 0x69, 0xd7,
|
||||
0xf5, 0x05, 0x8f, 0xd6, 0x8c, 0xa3, 0x15, 0x68, 0xdc, 0x72, 0x14, 0xc6, 0x39, 0xed, 0xcd, 0xbb,
|
||||
0x56, 0x10, 0x90, 0x4d, 0xf4, 0x0a, 0x66, 0x7b, 0xa2, 0x0c, 0xb3, 0xcd, 0x4b, 0x43, 0xb6, 0x3a,
|
||||
0xed, 0xbb, 0x69, 0xa6, 0x07, 0xba, 0x26, 0x5b, 0xbd, 0xf8, 0xd5, 0x81, 0xf1, 0xca, 0x9f, 0x72,
|
||||
0x73, 0x5c, 0x02, 0xb4, 0x7d, 0xc3, 0x3c, 0xf1, 0xa1, 0x6d, 0xf2, 0x18, 0xa6, 0xd5, 0xc1, 0xb3,
|
||||
0x55, 0xf8, 0xdd, 0x4f, 0x5a, 0xe8, 0x45, 0x47, 0x57, 0x3b, 0xfb, 0x71, 0x31, 0xf9, 0xff, 0x66,
|
||||
0xd7, 0x29, 0x44, 0xad, 0x0d, 0x51, 0xc6, 0xad, 0xbb, 0x5f, 0x4c, 0x02, 0x5c, 0x59, 0xe6, 0x62,
|
||||
0x08, 0x22, 0x14, 0x34, 0xed, 0x3b, 0x09, 0x04, 0xf4, 0x5e, 0xd0, 0xe4, 0x01, 0xc4, 0x72, 0x8f,
|
||||
0xc2, 0xa7, 0x34, 0x70, 0x5e, 0x46, 0x16, 0xb8, 0x8c, 0x2e, 0x01, 0xaa, 0x5a, 0xea, 0x90, 0xe1,
|
||||
0xd0, 0xcf, 0xe2, 0x88, 0x2b, 0xdf, 0x87, 0x91, 0x7d, 0xc9, 0x4a, 0xde, 0xe8, 0x74, 0x34, 0x8f,
|
||||
0xb2, 0x6e, 0x31, 0x14, 0x0d, 0x2f, 0xe4, 0x8d, 0x7e, 0xf7, 0xe6, 0xeb, 0xeb, 0x2d, 0x33, 0xbb,
|
||||
0x66, 0x63, 0x1f, 0x4c, 0x7e, 0xc7, 0xea, 0x9a, 0xdd, 0x19, 0xac, 0x76, 0xb9, 0x5f, 0xe2, 0x33,
|
||||
0xca, 0xb4, 0x51, 0x6c, 0xd3, 0x18, 0xa4, 0x39, 0x13, 0x06, 0x95, 0x20, 0x75, 0xee, 0x36, 0x9b,
|
||||
0xdb, 0xff, 0x6b, 0xbf, 0xd9, 0x0c, 0xdc, 0xd7, 0x8b, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0x6e,
|
||||
0x15, 0xf5, 0xbd, 0x8e, 0x03, 0x00, 0x00,
|
||||
}
|
|
@ -0,0 +1,193 @@
|
|||
syntax = "proto3";
|
||||
package milvus.proto.internal;
|
||||
option go_package="github.com/zilliztech/milvus-distributed/internal/proto/internalpb";
|
||||
|
||||
import "common.proto";
|
||||
import "service_msg.proto";
|
||||
|
||||
|
||||
enum ReqType {
|
||||
kNone = 0;
|
||||
/* Definition Requests: collection */
|
||||
kCreateCollection = 100;
|
||||
kDropCollection = 101;
|
||||
kHasCollection = 102;
|
||||
kDescribeCollection = 103;
|
||||
kShowCollections = 104;
|
||||
|
||||
/* Definition Requests: partition */
|
||||
kCreatePartition = 200;
|
||||
kDropPartition = 201;
|
||||
kHasPartition = 202;
|
||||
kDescribePartition = 203;
|
||||
kShowPartitions = 204;
|
||||
|
||||
/* Manipulation Requests */
|
||||
kInsert = 400;
|
||||
|
||||
/* Query */
|
||||
kSearch = 500;
|
||||
}
|
||||
|
||||
|
||||
message CreateCollectionRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
uint64 timestamp = 3;
|
||||
int64 proxy_id = 4;
|
||||
common.Blob schema = 5;
|
||||
}
|
||||
|
||||
|
||||
message DropCollectionRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
uint64 timestamp = 3;
|
||||
int64 proxy_id = 4;
|
||||
service.CollectionName collection_name = 5;
|
||||
}
|
||||
|
||||
|
||||
message HasCollectionRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
uint64 timestamp = 3;
|
||||
int64 proxy_id = 4;
|
||||
service.CollectionName collection_name = 5;
|
||||
}
|
||||
|
||||
|
||||
message DescribeCollectionRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
uint64 timestamp = 3;
|
||||
int64 proxy_id = 4;
|
||||
service.CollectionName collection_name = 5;
|
||||
}
|
||||
|
||||
|
||||
message ShowCollectionRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
uint64 timestamp = 3;
|
||||
int64 proxy_id = 4;
|
||||
}
|
||||
|
||||
|
||||
message CreatePartitionRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
uint64 timestamp = 3;
|
||||
int64 proxy_id = 4;
|
||||
service.PartitionName partition_name = 5;
|
||||
}
|
||||
|
||||
|
||||
message DropPartitionRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
uint64 timestamp = 3;
|
||||
int64 proxy_id = 4;
|
||||
service.PartitionName partition_name = 5;
|
||||
}
|
||||
|
||||
|
||||
message HasPartitionRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
uint64 timestamp = 3;
|
||||
int64 proxy_id = 4;
|
||||
service.PartitionName partition_name = 5;
|
||||
}
|
||||
|
||||
|
||||
message DescribePartitionRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
uint64 timestamp = 3;
|
||||
int64 proxy_id = 4;
|
||||
service.PartitionName partition_name = 5;
|
||||
}
|
||||
|
||||
|
||||
message ShowPartitionRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
uint64 timestamp = 3;
|
||||
int64 proxy_id = 4;
|
||||
service.CollectionName collection_name = 5;
|
||||
}
|
||||
|
||||
|
||||
message InsertRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
string collection_name = 3;
|
||||
string partition_tag = 4;
|
||||
uint64 segment_id = 5;
|
||||
uint64 channel_id = 6;
|
||||
int64 proxy_id = 7;
|
||||
uint64 timestamp = 8;
|
||||
repeated int64 row_ids = 9;
|
||||
repeated common.Blob row_data = 10;
|
||||
}
|
||||
|
||||
|
||||
message DeleteRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
string collection_name = 3;
|
||||
uint64 channel_id = 4;
|
||||
int64 proxy_id = 5;
|
||||
uint64 timestamp = 6;
|
||||
repeated int64 primary_keys = 7;
|
||||
}
|
||||
|
||||
|
||||
message SearchRequest {
|
||||
ReqType req_type = 1;
|
||||
uint64 req_id = 2;
|
||||
int64 proxy_id = 3;
|
||||
uint64 timestamp = 4;
|
||||
uint64 result_channel_id = 5;
|
||||
common.Blob query = 6;
|
||||
|
||||
}
|
||||
|
||||
message SearchResult {
|
||||
common.Status status = 1;
|
||||
uint64 req_id = 2;
|
||||
int64 proxy_id = 3;
|
||||
int64 query_node_id = 4;
|
||||
uint64 timestamp = 5;
|
||||
uint64 result_channel_id = 6;
|
||||
repeated service.Hits hits = 7;
|
||||
}
|
||||
|
||||
|
||||
message TimeSyncMsg {
|
||||
int64 peer_id = 1;
|
||||
uint64 timestamp = 2;
|
||||
}
|
||||
|
||||
|
||||
message Key2Seg {
|
||||
int64 row_id = 1;
|
||||
uint64 primary_key = 2;
|
||||
uint64 timestamp = 3;
|
||||
bool is_valid = 4;
|
||||
repeated uint64 segment_ids = 5;
|
||||
}
|
||||
|
||||
|
||||
message Key2SegMsg {
|
||||
uint64 req_id = 1;
|
||||
repeated Key2Seg key2seg = 2;
|
||||
}
|
||||
|
||||
|
||||
message SegmentStatistics {
|
||||
uint64 segment_id = 1;
|
||||
uint64 memory_size = 2;
|
||||
int64 num_rows = 3;
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -1,50 +1,90 @@
|
|||
syntax = "proto3";
|
||||
package milvus.proto.master;
|
||||
|
||||
package masterpb;
|
||||
option go_package="github.com/zilliztech/milvus-distributed/internal/proto/masterpb";
|
||||
|
||||
option go_package="github.com/zilliztech/milvus-distributed/internal/proto/master";
|
||||
|
||||
import "message.proto";
|
||||
|
||||
message Collection {
|
||||
uint64 id=1;
|
||||
string name=2;
|
||||
milvus.grpc.Schema schema=3;
|
||||
uint64 create_time=4;
|
||||
repeated uint64 segment_ids=5;
|
||||
repeated string partition_tags=6;
|
||||
repeated milvus.grpc.IndexParam indexes=7;
|
||||
}
|
||||
|
||||
enum SegmentStatus {
|
||||
OPENED = 0;
|
||||
CLOSED = 1;
|
||||
INDEXING = 2;
|
||||
INDEXED = 3;
|
||||
}
|
||||
|
||||
message Segment {
|
||||
uint64 segment_id=1;
|
||||
uint64 collection_id =2;
|
||||
string partition_tag=3;
|
||||
int32 channel_start=4;
|
||||
int32 channel_end=5;
|
||||
uint64 open_timestamp=6;
|
||||
uint64 close_timestamp=7;
|
||||
string collection_name=8;
|
||||
SegmentStatus status=9;
|
||||
int64 rows=10;
|
||||
}
|
||||
|
||||
message SegmentStat {
|
||||
uint64 segment_id=1;
|
||||
uint64 memory_size=2;
|
||||
float memory_rate=3;
|
||||
SegmentStatus status=4;
|
||||
int64 rows=5;
|
||||
}
|
||||
import "common.proto";
|
||||
import "internal_msg.proto";
|
||||
import "service_msg.proto";
|
||||
|
||||
service Master {
|
||||
rpc CreateCollection(milvus.grpc.Mapping) returns (milvus.grpc.Status){}
|
||||
rpc CreateIndex(milvus.grpc.IndexParam) returns (milvus.grpc.Status) {}
|
||||
}
|
||||
/**
|
||||
* @brief This method is used to create collection
|
||||
*
|
||||
* @param CreateCollectionRequest, use to provide collection information to be created.
|
||||
*
|
||||
* @return Status
|
||||
*/
|
||||
rpc CreateCollection(internal.CreateCollectionRequest) returns (common.Status){}
|
||||
|
||||
/**
|
||||
* @brief This method is used to delete collection.
|
||||
*
|
||||
* @param DropCollectionRequest, collection name is going to be deleted.
|
||||
*
|
||||
* @return Status
|
||||
*/
|
||||
rpc DropCollection(internal.DropCollectionRequest) returns (common.Status) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to test collection existence.
|
||||
*
|
||||
* @param HasCollectionRequest, collection name is going to be tested.
|
||||
*
|
||||
* @return BoolResponse
|
||||
*/
|
||||
rpc HasCollection(internal.HasCollectionRequest) returns (service.BoolResponse) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to get collection schema.
|
||||
*
|
||||
* @param DescribeCollectionRequest, target collection name.
|
||||
*
|
||||
* @return CollectionSchema
|
||||
*/
|
||||
rpc DescribeCollection(internal.DescribeCollectionRequest) returns (service.CollectionDescription) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to list all collections.
|
||||
*
|
||||
* @return StringListResponse, collection name list
|
||||
*/
|
||||
rpc ShowCollections(internal.ShowCollectionRequest) returns (service.StringListResponse) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to create partition
|
||||
*
|
||||
* @return Status
|
||||
*/
|
||||
rpc CreatePartition(internal.CreatePartitionRequest) returns (common.Status) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to drop partition
|
||||
*
|
||||
* @return Status
|
||||
*/
|
||||
rpc DropPartition(internal.DropPartitionRequest) returns (common.Status) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to test partition existence.
|
||||
*
|
||||
* @return BoolResponse
|
||||
*/
|
||||
rpc HasPartition(internal.HasPartitionRequest) returns (service.BoolResponse) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to get basic partition infomation.
|
||||
*
|
||||
* @return PartitionDescription
|
||||
*/
|
||||
rpc DescribePartition(internal.DescribePartitionRequest) returns (service.PartitionDescription) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to show partition information
|
||||
*
|
||||
* @param ShowPartitionRequest, target collection name.
|
||||
*
|
||||
* @return StringListResponse
|
||||
*/
|
||||
rpc ShowPartitions(internal.ShowPartitionRequest) returns (service.StringListResponse) {}
|
||||
}
|
|
@ -1,492 +0,0 @@
|
|||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// source: master.proto
|
||||
|
||||
package master
|
||||
|
||||
import (
|
||||
context "context"
|
||||
fmt "fmt"
|
||||
message "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
grpc "google.golang.org/grpc"
|
||||
codes "google.golang.org/grpc/codes"
|
||||
status "google.golang.org/grpc/status"
|
||||
math "math"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
var _ = fmt.Errorf
|
||||
var _ = math.Inf
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the proto package it is being compiled against.
|
||||
// A compilation error at this line likely means your copy of the
|
||||
// proto package needs to be updated.
|
||||
const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
|
||||
|
||||
type SegmentStatus int32
|
||||
|
||||
const (
|
||||
SegmentStatus_OPENED SegmentStatus = 0
|
||||
SegmentStatus_CLOSED SegmentStatus = 1
|
||||
SegmentStatus_INDEXING SegmentStatus = 2
|
||||
SegmentStatus_INDEXED SegmentStatus = 3
|
||||
)
|
||||
|
||||
var SegmentStatus_name = map[int32]string{
|
||||
0: "OPENED",
|
||||
1: "CLOSED",
|
||||
2: "INDEXING",
|
||||
3: "INDEXED",
|
||||
}
|
||||
|
||||
var SegmentStatus_value = map[string]int32{
|
||||
"OPENED": 0,
|
||||
"CLOSED": 1,
|
||||
"INDEXING": 2,
|
||||
"INDEXED": 3,
|
||||
}
|
||||
|
||||
func (x SegmentStatus) String() string {
|
||||
return proto.EnumName(SegmentStatus_name, int32(x))
|
||||
}
|
||||
|
||||
func (SegmentStatus) EnumDescriptor() ([]byte, []int) {
|
||||
return fileDescriptor_f9c348dec43a6705, []int{0}
|
||||
}
|
||||
|
||||
type Collection struct {
|
||||
Id uint64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"`
|
||||
Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
|
||||
Schema *message.Schema `protobuf:"bytes,3,opt,name=schema,proto3" json:"schema,omitempty"`
|
||||
CreateTime uint64 `protobuf:"varint,4,opt,name=create_time,json=createTime,proto3" json:"create_time,omitempty"`
|
||||
SegmentIds []uint64 `protobuf:"varint,5,rep,packed,name=segment_ids,json=segmentIds,proto3" json:"segment_ids,omitempty"`
|
||||
PartitionTags []string `protobuf:"bytes,6,rep,name=partition_tags,json=partitionTags,proto3" json:"partition_tags,omitempty"`
|
||||
Indexes []*message.IndexParam `protobuf:"bytes,7,rep,name=indexes,proto3" json:"indexes,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Collection) Reset() { *m = Collection{} }
|
||||
func (m *Collection) String() string { return proto.CompactTextString(m) }
|
||||
func (*Collection) ProtoMessage() {}
|
||||
func (*Collection) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_f9c348dec43a6705, []int{0}
|
||||
}
|
||||
|
||||
func (m *Collection) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Collection.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Collection) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Collection.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *Collection) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Collection.Merge(m, src)
|
||||
}
|
||||
func (m *Collection) XXX_Size() int {
|
||||
return xxx_messageInfo_Collection.Size(m)
|
||||
}
|
||||
func (m *Collection) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Collection.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Collection proto.InternalMessageInfo
|
||||
|
||||
func (m *Collection) GetId() uint64 {
|
||||
if m != nil {
|
||||
return m.Id
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *Collection) GetName() string {
|
||||
if m != nil {
|
||||
return m.Name
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *Collection) GetSchema() *message.Schema {
|
||||
if m != nil {
|
||||
return m.Schema
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *Collection) GetCreateTime() uint64 {
|
||||
if m != nil {
|
||||
return m.CreateTime
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *Collection) GetSegmentIds() []uint64 {
|
||||
if m != nil {
|
||||
return m.SegmentIds
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *Collection) GetPartitionTags() []string {
|
||||
if m != nil {
|
||||
return m.PartitionTags
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *Collection) GetIndexes() []*message.IndexParam {
|
||||
if m != nil {
|
||||
return m.Indexes
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type Segment struct {
|
||||
SegmentId uint64 `protobuf:"varint,1,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"`
|
||||
CollectionId uint64 `protobuf:"varint,2,opt,name=collection_id,json=collectionId,proto3" json:"collection_id,omitempty"`
|
||||
PartitionTag string `protobuf:"bytes,3,opt,name=partition_tag,json=partitionTag,proto3" json:"partition_tag,omitempty"`
|
||||
ChannelStart int32 `protobuf:"varint,4,opt,name=channel_start,json=channelStart,proto3" json:"channel_start,omitempty"`
|
||||
ChannelEnd int32 `protobuf:"varint,5,opt,name=channel_end,json=channelEnd,proto3" json:"channel_end,omitempty"`
|
||||
OpenTimestamp uint64 `protobuf:"varint,6,opt,name=open_timestamp,json=openTimestamp,proto3" json:"open_timestamp,omitempty"`
|
||||
CloseTimestamp uint64 `protobuf:"varint,7,opt,name=close_timestamp,json=closeTimestamp,proto3" json:"close_timestamp,omitempty"`
|
||||
CollectionName string `protobuf:"bytes,8,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
|
||||
Status SegmentStatus `protobuf:"varint,9,opt,name=status,proto3,enum=masterpb.SegmentStatus" json:"status,omitempty"`
|
||||
Rows int64 `protobuf:"varint,10,opt,name=rows,proto3" json:"rows,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Segment) Reset() { *m = Segment{} }
|
||||
func (m *Segment) String() string { return proto.CompactTextString(m) }
|
||||
func (*Segment) ProtoMessage() {}
|
||||
func (*Segment) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_f9c348dec43a6705, []int{1}
|
||||
}
|
||||
|
||||
func (m *Segment) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Segment.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Segment) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Segment.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *Segment) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Segment.Merge(m, src)
|
||||
}
|
||||
func (m *Segment) XXX_Size() int {
|
||||
return xxx_messageInfo_Segment.Size(m)
|
||||
}
|
||||
func (m *Segment) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Segment.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Segment proto.InternalMessageInfo
|
||||
|
||||
func (m *Segment) GetSegmentId() uint64 {
|
||||
if m != nil {
|
||||
return m.SegmentId
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *Segment) GetCollectionId() uint64 {
|
||||
if m != nil {
|
||||
return m.CollectionId
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *Segment) GetPartitionTag() string {
|
||||
if m != nil {
|
||||
return m.PartitionTag
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *Segment) GetChannelStart() int32 {
|
||||
if m != nil {
|
||||
return m.ChannelStart
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *Segment) GetChannelEnd() int32 {
|
||||
if m != nil {
|
||||
return m.ChannelEnd
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *Segment) GetOpenTimestamp() uint64 {
|
||||
if m != nil {
|
||||
return m.OpenTimestamp
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *Segment) GetCloseTimestamp() uint64 {
|
||||
if m != nil {
|
||||
return m.CloseTimestamp
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *Segment) GetCollectionName() string {
|
||||
if m != nil {
|
||||
return m.CollectionName
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *Segment) GetStatus() SegmentStatus {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return SegmentStatus_OPENED
|
||||
}
|
||||
|
||||
func (m *Segment) GetRows() int64 {
|
||||
if m != nil {
|
||||
return m.Rows
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
type SegmentStat struct {
|
||||
SegmentId uint64 `protobuf:"varint,1,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"`
|
||||
MemorySize uint64 `protobuf:"varint,2,opt,name=memory_size,json=memorySize,proto3" json:"memory_size,omitempty"`
|
||||
MemoryRate float32 `protobuf:"fixed32,3,opt,name=memory_rate,json=memoryRate,proto3" json:"memory_rate,omitempty"`
|
||||
Status SegmentStatus `protobuf:"varint,4,opt,name=status,proto3,enum=masterpb.SegmentStatus" json:"status,omitempty"`
|
||||
Rows int64 `protobuf:"varint,5,opt,name=rows,proto3" json:"rows,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *SegmentStat) Reset() { *m = SegmentStat{} }
|
||||
func (m *SegmentStat) String() string { return proto.CompactTextString(m) }
|
||||
func (*SegmentStat) ProtoMessage() {}
|
||||
func (*SegmentStat) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_f9c348dec43a6705, []int{2}
|
||||
}
|
||||
|
||||
func (m *SegmentStat) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_SegmentStat.Unmarshal(m, b)
|
||||
}
|
||||
func (m *SegmentStat) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_SegmentStat.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *SegmentStat) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_SegmentStat.Merge(m, src)
|
||||
}
|
||||
func (m *SegmentStat) XXX_Size() int {
|
||||
return xxx_messageInfo_SegmentStat.Size(m)
|
||||
}
|
||||
func (m *SegmentStat) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_SegmentStat.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_SegmentStat proto.InternalMessageInfo
|
||||
|
||||
func (m *SegmentStat) GetSegmentId() uint64 {
|
||||
if m != nil {
|
||||
return m.SegmentId
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SegmentStat) GetMemorySize() uint64 {
|
||||
if m != nil {
|
||||
return m.MemorySize
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SegmentStat) GetMemoryRate() float32 {
|
||||
if m != nil {
|
||||
return m.MemoryRate
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func (m *SegmentStat) GetStatus() SegmentStatus {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return SegmentStatus_OPENED
|
||||
}
|
||||
|
||||
func (m *SegmentStat) GetRows() int64 {
|
||||
if m != nil {
|
||||
return m.Rows
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterEnum("masterpb.SegmentStatus", SegmentStatus_name, SegmentStatus_value)
|
||||
proto.RegisterType((*Collection)(nil), "masterpb.Collection")
|
||||
proto.RegisterType((*Segment)(nil), "masterpb.Segment")
|
||||
proto.RegisterType((*SegmentStat)(nil), "masterpb.SegmentStat")
|
||||
}
|
||||
|
||||
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
|
||||
|
||||
var fileDescriptor_f9c348dec43a6705 = []byte{
|
||||
// 586 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x94, 0x94, 0xd1, 0x6a, 0xdb, 0x30,
|
||||
0x14, 0x86, 0x6b, 0x27, 0x71, 0x9a, 0xe3, 0x24, 0x0b, 0xda, 0x60, 0xa6, 0x30, 0x6a, 0x32, 0xc6,
|
||||
0xcc, 0x06, 0x4e, 0xd7, 0x5d, 0xec, 0x6a, 0x0c, 0xda, 0x84, 0x11, 0x58, 0xd3, 0x62, 0xf7, 0x62,
|
||||
0xec, 0x26, 0xa8, 0xb6, 0x70, 0x05, 0x96, 0x6c, 0x2c, 0xa5, 0xdb, 0xfa, 0x04, 0x7b, 0x9f, 0xbd,
|
||||
0xdb, 0x60, 0x77, 0x43, 0xc7, 0x6e, 0xe3, 0x0c, 0xca, 0xd8, 0x9d, 0xf4, 0x9f, 0x2f, 0xd2, 0x9f,
|
||||
0x5f, 0xe7, 0x18, 0x86, 0x82, 0x2a, 0xcd, 0xaa, 0xb0, 0xac, 0x0a, 0x5d, 0x90, 0xfd, 0x7a, 0x57,
|
||||
0x5e, 0x1d, 0x8c, 0x04, 0x53, 0x8a, 0x66, 0xac, 0x2e, 0x4c, 0x7f, 0x5b, 0x00, 0xa7, 0x45, 0x9e,
|
||||
0xb3, 0x44, 0xf3, 0x42, 0x92, 0x31, 0xd8, 0x3c, 0xf5, 0x2c, 0xdf, 0x0a, 0xba, 0x91, 0xcd, 0x53,
|
||||
0x42, 0xa0, 0x2b, 0xa9, 0x60, 0x9e, 0xed, 0x5b, 0xc1, 0x20, 0xc2, 0x35, 0x79, 0x0d, 0x8e, 0x4a,
|
||||
0xae, 0x99, 0xa0, 0x5e, 0xc7, 0xb7, 0x02, 0xf7, 0xf8, 0x71, 0x28, 0x78, 0x7e, 0xb3, 0x51, 0x61,
|
||||
0x56, 0x95, 0x49, 0x18, 0x63, 0x29, 0x6a, 0x10, 0x72, 0x08, 0x6e, 0x52, 0x31, 0xaa, 0xd9, 0x5a,
|
||||
0x73, 0xc1, 0xbc, 0x2e, 0x9e, 0x0c, 0xb5, 0x74, 0xc9, 0x05, 0x33, 0x80, 0x62, 0x99, 0x60, 0x52,
|
||||
0xaf, 0x79, 0xaa, 0xbc, 0x9e, 0xdf, 0x31, 0x40, 0x23, 0x2d, 0x53, 0x45, 0x5e, 0xc0, 0xb8, 0xa4,
|
||||
0x95, 0xe6, 0xc6, 0xdf, 0x5a, 0xd3, 0x4c, 0x79, 0x8e, 0xdf, 0x09, 0x06, 0xd1, 0xe8, 0x5e, 0xbd,
|
||||
0xa4, 0x99, 0x22, 0x6f, 0xa0, 0xcf, 0x65, 0xca, 0xbe, 0x31, 0xe5, 0xf5, 0xfd, 0x4e, 0xe0, 0x1e,
|
||||
0x3f, 0xdd, 0xb1, 0xb5, 0x34, 0xb5, 0x0b, 0x5a, 0x51, 0x11, 0xdd, 0x71, 0xd3, 0x5f, 0x36, 0xf4,
|
||||
0xe3, 0xfa, 0x22, 0xf2, 0x0c, 0x60, 0x6b, 0xa3, 0x09, 0x60, 0x70, 0xef, 0x82, 0x3c, 0x87, 0x51,
|
||||
0x72, 0x9f, 0x92, 0x21, 0x6c, 0x24, 0x86, 0x5b, 0xb1, 0x86, 0x76, 0x9c, 0x62, 0x3e, 0x83, 0x68,
|
||||
0xd8, 0x36, 0x8a, 0x27, 0x5d, 0x53, 0x29, 0x59, 0xbe, 0x56, 0x9a, 0x56, 0x1a, 0x23, 0xe9, 0x45,
|
||||
0xc3, 0x46, 0x8c, 0x8d, 0x86, 0xa9, 0x35, 0x10, 0x93, 0xa9, 0xd7, 0x43, 0x04, 0x1a, 0x69, 0x21,
|
||||
0x53, 0x13, 0x4a, 0x51, 0x32, 0x89, 0xa1, 0x2a, 0x4d, 0x45, 0xe9, 0x39, 0x68, 0x68, 0x64, 0xd4,
|
||||
0xcb, 0x3b, 0x91, 0xbc, 0x84, 0x47, 0x49, 0x5e, 0x28, 0xd6, 0xe2, 0xfa, 0xc8, 0x8d, 0x51, 0xde,
|
||||
0x05, 0xb7, 0xff, 0x0f, 0x9f, 0x7c, 0x1f, 0xcd, 0x8f, 0xb7, 0xf2, 0xca, 0x3c, 0xfe, 0x0c, 0x1c,
|
||||
0xa5, 0xa9, 0xde, 0x28, 0x6f, 0xe0, 0x5b, 0xc1, 0xd8, 0xa4, 0xdc, 0x74, 0x56, 0xd8, 0x44, 0x19,
|
||||
0x63, 0x39, 0x6a, 0x30, 0xd3, 0x41, 0x55, 0xf1, 0x55, 0x79, 0xe0, 0x5b, 0x41, 0x27, 0xc2, 0xf5,
|
||||
0xf4, 0xa7, 0x05, 0x6e, 0x8b, 0xfe, 0x57, 0xf8, 0x87, 0xe0, 0x0a, 0x26, 0x8a, 0xea, 0xfb, 0x5a,
|
||||
0xf1, 0x5b, 0xd6, 0x44, 0x0f, 0xb5, 0x14, 0xf3, 0x5b, 0xd6, 0x02, 0x2a, 0xaa, 0x19, 0xc6, 0x6e,
|
||||
0xdf, 0x01, 0x11, 0xd5, 0x6d, 0xd7, 0xdd, 0xff, 0x73, 0xdd, 0xdb, 0xba, 0x7e, 0x75, 0x02, 0xa3,
|
||||
0x1d, 0x98, 0x00, 0x38, 0xe7, 0x17, 0x8b, 0xd5, 0x62, 0x3e, 0xd9, 0x33, 0xeb, 0xd3, 0x4f, 0xe7,
|
||||
0xf1, 0x62, 0x3e, 0xb1, 0xc8, 0x10, 0xf6, 0x97, 0xab, 0xf9, 0xe2, 0xf3, 0x72, 0xf5, 0x71, 0x62,
|
||||
0x13, 0x17, 0xfa, 0xb8, 0x5b, 0xcc, 0x27, 0x9d, 0xe3, 0x1f, 0x16, 0x38, 0x67, 0x78, 0x35, 0xf9,
|
||||
0x00, 0x93, 0x53, 0x1c, 0x83, 0xd6, 0xf8, 0x3d, 0xd9, 0xe9, 0xd9, 0x33, 0x5a, 0x96, 0x5c, 0x66,
|
||||
0x07, 0x7f, 0x0d, 0x18, 0x5e, 0x3e, 0xdd, 0x23, 0xef, 0xc1, 0xad, 0x0f, 0xc0, 0xde, 0x26, 0x0f,
|
||||
0xf5, 0xfb, 0x03, 0x3f, 0x3f, 0x39, 0xfa, 0x12, 0x66, 0x5c, 0x5f, 0x6f, 0xae, 0xc2, 0xa4, 0x10,
|
||||
0xb3, 0xe4, 0x56, 0x1d, 0x1d, 0xbd, 0x9b, 0xa9, 0xcd, 0x4d, 0xce, 0xc5, 0x8c, 0x4b, 0xcd, 0x2a,
|
||||
0x49, 0xf3, 0x19, 0x7e, 0x24, 0x66, 0x75, 0x58, 0x57, 0x0e, 0xee, 0xde, 0xfe, 0x09, 0x00, 0x00,
|
||||
0xff, 0xff, 0x6c, 0xc2, 0x7d, 0xd6, 0x5b, 0x04, 0x00, 0x00,
|
||||
}
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ context.Context
|
||||
var _ grpc.ClientConn
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the grpc package it is being compiled against.
|
||||
const _ = grpc.SupportPackageIsVersion4
|
||||
|
||||
// MasterClient is the client API for Master service.
|
||||
//
|
||||
// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
|
||||
type MasterClient interface {
|
||||
CreateCollection(ctx context.Context, in *message.Mapping, opts ...grpc.CallOption) (*message.Status, error)
|
||||
CreateIndex(ctx context.Context, in *message.IndexParam, opts ...grpc.CallOption) (*message.Status, error)
|
||||
}
|
||||
|
||||
type masterClient struct {
|
||||
cc *grpc.ClientConn
|
||||
}
|
||||
|
||||
func NewMasterClient(cc *grpc.ClientConn) MasterClient {
|
||||
return &masterClient{cc}
|
||||
}
|
||||
|
||||
func (c *masterClient) CreateCollection(ctx context.Context, in *message.Mapping, opts ...grpc.CallOption) (*message.Status, error) {
|
||||
out := new(message.Status)
|
||||
err := c.cc.Invoke(ctx, "/masterpb.Master/CreateCollection", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) CreateIndex(ctx context.Context, in *message.IndexParam, opts ...grpc.CallOption) (*message.Status, error) {
|
||||
out := new(message.Status)
|
||||
err := c.cc.Invoke(ctx, "/masterpb.Master/CreateIndex", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
// MasterServer is the server API for Master service.
|
||||
type MasterServer interface {
|
||||
CreateCollection(context.Context, *message.Mapping) (*message.Status, error)
|
||||
CreateIndex(context.Context, *message.IndexParam) (*message.Status, error)
|
||||
}
|
||||
|
||||
// UnimplementedMasterServer can be embedded to have forward compatible implementations.
|
||||
type UnimplementedMasterServer struct {
|
||||
}
|
||||
|
||||
func (*UnimplementedMasterServer) CreateCollection(ctx context.Context, req *message.Mapping) (*message.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method CreateCollection not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) CreateIndex(ctx context.Context, req *message.IndexParam) (*message.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method CreateIndex not implemented")
|
||||
}
|
||||
|
||||
func RegisterMasterServer(s *grpc.Server, srv MasterServer) {
|
||||
s.RegisterService(&_Master_serviceDesc, srv)
|
||||
}
|
||||
|
||||
func _Master_CreateCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(message.Mapping)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).CreateCollection(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/masterpb.Master/CreateCollection",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).CreateCollection(ctx, req.(*message.Mapping))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_CreateIndex_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(message.IndexParam)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).CreateIndex(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/masterpb.Master/CreateIndex",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).CreateIndex(ctx, req.(*message.IndexParam))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
var _Master_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "masterpb.Master",
|
||||
HandlerType: (*MasterServer)(nil),
|
||||
Methods: []grpc.MethodDesc{
|
||||
{
|
||||
MethodName: "CreateCollection",
|
||||
Handler: _Master_CreateCollection_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "CreateIndex",
|
||||
Handler: _Master_CreateIndex_Handler,
|
||||
},
|
||||
},
|
||||
Streams: []grpc.StreamDesc{},
|
||||
Metadata: "master.proto",
|
||||
}
|
|
@ -0,0 +1,562 @@
|
|||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// source: master.proto
|
||||
|
||||
package masterpb
|
||||
|
||||
import (
|
||||
context "context"
|
||||
fmt "fmt"
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
internalpb "github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
servicepb "github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
||||
grpc "google.golang.org/grpc"
|
||||
codes "google.golang.org/grpc/codes"
|
||||
status "google.golang.org/grpc/status"
|
||||
math "math"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
var _ = fmt.Errorf
|
||||
var _ = math.Inf
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the proto package it is being compiled against.
|
||||
// A compilation error at this line likely means your copy of the
|
||||
// proto package needs to be updated.
|
||||
const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
|
||||
|
||||
func init() { proto.RegisterFile("master.proto", fileDescriptor_f9c348dec43a6705) }
|
||||
|
||||
var fileDescriptor_f9c348dec43a6705 = []byte{
|
||||
// 384 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x54, 0x4d, 0x4f, 0xc2, 0x40,
|
||||
0x10, 0xe5, 0x44, 0xcc, 0x86, 0x0f, 0x59, 0x6f, 0x78, 0xeb, 0xc9, 0x80, 0xb4, 0x46, 0xff, 0x80,
|
||||
0x01, 0x0e, 0x1c, 0x34, 0x31, 0x70, 0xd3, 0x18, 0xdc, 0x96, 0x0d, 0x4c, 0x6c, 0xbb, 0x75, 0x67,
|
||||
0x8a, 0x09, 0xff, 0xc2, 0x7f, 0x6c, 0xda, 0x52, 0xda, 0x15, 0x8a, 0xe8, 0xad, 0x3b, 0xf3, 0xf6,
|
||||
0xbd, 0xce, 0xbc, 0x97, 0x65, 0x8d, 0x40, 0x20, 0x49, 0x6d, 0x47, 0x5a, 0x91, 0xe2, 0x17, 0x01,
|
||||
0xf8, 0xeb, 0x18, 0xb3, 0x93, 0x9d, 0xb5, 0xba, 0x0d, 0x4f, 0x05, 0x81, 0x0a, 0xb3, 0x62, 0x97,
|
||||
0x43, 0x48, 0x52, 0x87, 0xc2, 0x9f, 0x07, 0xb8, 0xdc, 0xd6, 0x3a, 0x28, 0xf5, 0x1a, 0x3c, 0x59,
|
||||
0x94, 0x6e, 0xbf, 0xce, 0x58, 0xfd, 0x31, 0xbd, 0xcf, 0x05, 0x3b, 0x1f, 0x69, 0x29, 0x48, 0x8e,
|
||||
0x94, 0xef, 0x4b, 0x8f, 0x40, 0x85, 0xdc, 0xb6, 0x0d, 0xa5, 0x9c, 0xd3, 0xfe, 0x09, 0x9c, 0xca,
|
||||
0x8f, 0x58, 0x22, 0x75, 0x2f, 0x4d, 0xfc, 0xf6, 0x8f, 0x66, 0x24, 0x28, 0x46, 0xab, 0xc6, 0x5f,
|
||||
0x59, 0x6b, 0xac, 0x55, 0x54, 0x12, 0xb8, 0xae, 0x10, 0x30, 0x61, 0x27, 0xd2, 0xbb, 0xac, 0x39,
|
||||
0x11, 0x58, 0x62, 0xef, 0x57, 0xb0, 0x1b, 0xa8, 0x9c, 0xdc, 0x32, 0xc1, 0xdb, 0x5d, 0xd9, 0x43,
|
||||
0xa5, 0xfc, 0xa9, 0xc4, 0x48, 0x85, 0x28, 0xad, 0x1a, 0x8f, 0x19, 0x1f, 0x4b, 0xf4, 0x34, 0xb8,
|
||||
0xe5, 0x3d, 0xdd, 0x54, 0x8d, 0xb1, 0x07, 0xcd, 0xd5, 0xfa, 0x87, 0xd5, 0x0a, 0x60, 0x76, 0x35,
|
||||
0x4a, 0x3e, 0xad, 0x1a, 0x7f, 0x67, 0xed, 0xd9, 0x4a, 0x7d, 0x16, 0x6d, 0xac, 0x5c, 0x9d, 0x89,
|
||||
0xcb, 0xf5, 0xae, 0x0e, 0xeb, 0xcd, 0x48, 0x43, 0xb8, 0x7c, 0x00, 0xa4, 0xd2, 0x8c, 0x73, 0xd6,
|
||||
0xce, 0x0c, 0x7e, 0x12, 0x9a, 0x20, 0x1d, 0x70, 0x70, 0x34, 0x08, 0x3b, 0xdc, 0x89, 0x46, 0xbd,
|
||||
0xb0, 0x66, 0x62, 0x70, 0x41, 0xdf, 0x3f, 0x12, 0x83, 0xbf, 0x92, 0xbf, 0xb1, 0xc6, 0x44, 0x60,
|
||||
0xc1, 0xdd, 0xab, 0x0e, 0xc1, 0x1e, 0xf5, 0x69, 0x19, 0xd0, 0xac, 0x93, 0x1b, 0x5b, 0xc8, 0x38,
|
||||
0xbf, 0x44, 0x60, 0x4f, 0xab, 0x77, 0x58, 0x6b, 0x87, 0x33, 0x03, 0x00, 0xac, 0x95, 0x18, 0xbb,
|
||||
0xeb, 0x62, 0xe5, 0xce, 0x0c, 0xd8, 0x3f, 0xec, 0x1f, 0x0e, 0x9f, 0xef, 0x97, 0x40, 0xab, 0xd8,
|
||||
0x4d, 0x56, 0xeb, 0x6c, 0xc0, 0xf7, 0x61, 0x43, 0xd2, 0x5b, 0x39, 0x19, 0xc5, 0x60, 0x01, 0x48,
|
||||
0x1a, 0xdc, 0x98, 0xe4, 0xc2, 0xc9, 0x55, 0x9d, 0x94, 0xd7, 0xc9, 0x9e, 0xa2, 0xc8, 0x75, 0xeb,
|
||||
0xe9, 0xf9, 0xee, 0x3b, 0x00, 0x00, 0xff, 0xff, 0x96, 0xb6, 0xf6, 0x5a, 0xb8, 0x04, 0x00, 0x00,
|
||||
}
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ context.Context
|
||||
var _ grpc.ClientConn
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the grpc package it is being compiled against.
|
||||
const _ = grpc.SupportPackageIsVersion4
|
||||
|
||||
// MasterClient is the client API for Master service.
|
||||
//
|
||||
// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
|
||||
type MasterClient interface {
|
||||
//*
|
||||
// @brief This method is used to create collection
|
||||
//
|
||||
// @param CreateCollectionRequest, use to provide collection information to be created.
|
||||
//
|
||||
// @return Status
|
||||
CreateCollection(ctx context.Context, in *internalpb.CreateCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to delete collection.
|
||||
//
|
||||
// @param DropCollectionRequest, collection name is going to be deleted.
|
||||
//
|
||||
// @return Status
|
||||
DropCollection(ctx context.Context, in *internalpb.DropCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to test collection existence.
|
||||
//
|
||||
// @param HasCollectionRequest, collection name is going to be tested.
|
||||
//
|
||||
// @return BoolResponse
|
||||
HasCollection(ctx context.Context, in *internalpb.HasCollectionRequest, opts ...grpc.CallOption) (*servicepb.BoolResponse, error)
|
||||
//*
|
||||
// @brief This method is used to get collection schema.
|
||||
//
|
||||
// @param DescribeCollectionRequest, target collection name.
|
||||
//
|
||||
// @return CollectionSchema
|
||||
DescribeCollection(ctx context.Context, in *internalpb.DescribeCollectionRequest, opts ...grpc.CallOption) (*servicepb.CollectionDescription, error)
|
||||
//*
|
||||
// @brief This method is used to list all collections.
|
||||
//
|
||||
// @return StringListResponse, collection name list
|
||||
ShowCollections(ctx context.Context, in *internalpb.ShowCollectionRequest, opts ...grpc.CallOption) (*servicepb.StringListResponse, error)
|
||||
//*
|
||||
// @brief This method is used to create partition
|
||||
//
|
||||
// @return Status
|
||||
CreatePartition(ctx context.Context, in *internalpb.CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to drop partition
|
||||
//
|
||||
// @return Status
|
||||
DropPartition(ctx context.Context, in *internalpb.DropPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to test partition existence.
|
||||
//
|
||||
// @return BoolResponse
|
||||
HasPartition(ctx context.Context, in *internalpb.HasPartitionRequest, opts ...grpc.CallOption) (*servicepb.BoolResponse, error)
|
||||
//*
|
||||
// @brief This method is used to get basic partition infomation.
|
||||
//
|
||||
// @return PartitionDescription
|
||||
DescribePartition(ctx context.Context, in *internalpb.DescribePartitionRequest, opts ...grpc.CallOption) (*servicepb.PartitionDescription, error)
|
||||
//*
|
||||
// @brief This method is used to show partition information
|
||||
//
|
||||
// @param ShowPartitionRequest, target collection name.
|
||||
//
|
||||
// @return StringListResponse
|
||||
ShowPartitions(ctx context.Context, in *internalpb.ShowPartitionRequest, opts ...grpc.CallOption) (*servicepb.StringListResponse, error)
|
||||
}
|
||||
|
||||
type masterClient struct {
|
||||
cc *grpc.ClientConn
|
||||
}
|
||||
|
||||
func NewMasterClient(cc *grpc.ClientConn) MasterClient {
|
||||
return &masterClient{cc}
|
||||
}
|
||||
|
||||
func (c *masterClient) CreateCollection(ctx context.Context, in *internalpb.CreateCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
out := new(commonpb.Status)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/CreateCollection", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) DropCollection(ctx context.Context, in *internalpb.DropCollectionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
out := new(commonpb.Status)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/DropCollection", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) HasCollection(ctx context.Context, in *internalpb.HasCollectionRequest, opts ...grpc.CallOption) (*servicepb.BoolResponse, error) {
|
||||
out := new(servicepb.BoolResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/HasCollection", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) DescribeCollection(ctx context.Context, in *internalpb.DescribeCollectionRequest, opts ...grpc.CallOption) (*servicepb.CollectionDescription, error) {
|
||||
out := new(servicepb.CollectionDescription)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/DescribeCollection", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) ShowCollections(ctx context.Context, in *internalpb.ShowCollectionRequest, opts ...grpc.CallOption) (*servicepb.StringListResponse, error) {
|
||||
out := new(servicepb.StringListResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/ShowCollections", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) CreatePartition(ctx context.Context, in *internalpb.CreatePartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
out := new(commonpb.Status)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/CreatePartition", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) DropPartition(ctx context.Context, in *internalpb.DropPartitionRequest, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
out := new(commonpb.Status)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/DropPartition", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) HasPartition(ctx context.Context, in *internalpb.HasPartitionRequest, opts ...grpc.CallOption) (*servicepb.BoolResponse, error) {
|
||||
out := new(servicepb.BoolResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/HasPartition", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) DescribePartition(ctx context.Context, in *internalpb.DescribePartitionRequest, opts ...grpc.CallOption) (*servicepb.PartitionDescription, error) {
|
||||
out := new(servicepb.PartitionDescription)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/DescribePartition", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *masterClient) ShowPartitions(ctx context.Context, in *internalpb.ShowPartitionRequest, opts ...grpc.CallOption) (*servicepb.StringListResponse, error) {
|
||||
out := new(servicepb.StringListResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.master.Master/ShowPartitions", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
// MasterServer is the server API for Master service.
|
||||
type MasterServer interface {
|
||||
//*
|
||||
// @brief This method is used to create collection
|
||||
//
|
||||
// @param CreateCollectionRequest, use to provide collection information to be created.
|
||||
//
|
||||
// @return Status
|
||||
CreateCollection(context.Context, *internalpb.CreateCollectionRequest) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to delete collection.
|
||||
//
|
||||
// @param DropCollectionRequest, collection name is going to be deleted.
|
||||
//
|
||||
// @return Status
|
||||
DropCollection(context.Context, *internalpb.DropCollectionRequest) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to test collection existence.
|
||||
//
|
||||
// @param HasCollectionRequest, collection name is going to be tested.
|
||||
//
|
||||
// @return BoolResponse
|
||||
HasCollection(context.Context, *internalpb.HasCollectionRequest) (*servicepb.BoolResponse, error)
|
||||
//*
|
||||
// @brief This method is used to get collection schema.
|
||||
//
|
||||
// @param DescribeCollectionRequest, target collection name.
|
||||
//
|
||||
// @return CollectionSchema
|
||||
DescribeCollection(context.Context, *internalpb.DescribeCollectionRequest) (*servicepb.CollectionDescription, error)
|
||||
//*
|
||||
// @brief This method is used to list all collections.
|
||||
//
|
||||
// @return StringListResponse, collection name list
|
||||
ShowCollections(context.Context, *internalpb.ShowCollectionRequest) (*servicepb.StringListResponse, error)
|
||||
//*
|
||||
// @brief This method is used to create partition
|
||||
//
|
||||
// @return Status
|
||||
CreatePartition(context.Context, *internalpb.CreatePartitionRequest) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to drop partition
|
||||
//
|
||||
// @return Status
|
||||
DropPartition(context.Context, *internalpb.DropPartitionRequest) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to test partition existence.
|
||||
//
|
||||
// @return BoolResponse
|
||||
HasPartition(context.Context, *internalpb.HasPartitionRequest) (*servicepb.BoolResponse, error)
|
||||
//*
|
||||
// @brief This method is used to get basic partition infomation.
|
||||
//
|
||||
// @return PartitionDescription
|
||||
DescribePartition(context.Context, *internalpb.DescribePartitionRequest) (*servicepb.PartitionDescription, error)
|
||||
//*
|
||||
// @brief This method is used to show partition information
|
||||
//
|
||||
// @param ShowPartitionRequest, target collection name.
|
||||
//
|
||||
// @return StringListResponse
|
||||
ShowPartitions(context.Context, *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error)
|
||||
}
|
||||
|
||||
// UnimplementedMasterServer can be embedded to have forward compatible implementations.
|
||||
type UnimplementedMasterServer struct {
|
||||
}
|
||||
|
||||
func (*UnimplementedMasterServer) CreateCollection(ctx context.Context, req *internalpb.CreateCollectionRequest) (*commonpb.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method CreateCollection not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) DropCollection(ctx context.Context, req *internalpb.DropCollectionRequest) (*commonpb.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method DropCollection not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) HasCollection(ctx context.Context, req *internalpb.HasCollectionRequest) (*servicepb.BoolResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method HasCollection not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) DescribeCollection(ctx context.Context, req *internalpb.DescribeCollectionRequest) (*servicepb.CollectionDescription, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method DescribeCollection not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) ShowCollections(ctx context.Context, req *internalpb.ShowCollectionRequest) (*servicepb.StringListResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method ShowCollections not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) CreatePartition(ctx context.Context, req *internalpb.CreatePartitionRequest) (*commonpb.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method CreatePartition not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) DropPartition(ctx context.Context, req *internalpb.DropPartitionRequest) (*commonpb.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method DropPartition not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) HasPartition(ctx context.Context, req *internalpb.HasPartitionRequest) (*servicepb.BoolResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method HasPartition not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) DescribePartition(ctx context.Context, req *internalpb.DescribePartitionRequest) (*servicepb.PartitionDescription, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method DescribePartition not implemented")
|
||||
}
|
||||
func (*UnimplementedMasterServer) ShowPartitions(ctx context.Context, req *internalpb.ShowPartitionRequest) (*servicepb.StringListResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented")
|
||||
}
|
||||
|
||||
func RegisterMasterServer(s *grpc.Server, srv MasterServer) {
|
||||
s.RegisterService(&_Master_serviceDesc, srv)
|
||||
}
|
||||
|
||||
func _Master_CreateCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.CreateCollectionRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).CreateCollection(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.master.Master/CreateCollection",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).CreateCollection(ctx, req.(*internalpb.CreateCollectionRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_DropCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.DropCollectionRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).DropCollection(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.master.Master/DropCollection",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).DropCollection(ctx, req.(*internalpb.DropCollectionRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_HasCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.HasCollectionRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).HasCollection(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.master.Master/HasCollection",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).HasCollection(ctx, req.(*internalpb.HasCollectionRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_DescribeCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.DescribeCollectionRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).DescribeCollection(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.master.Master/DescribeCollection",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).DescribeCollection(ctx, req.(*internalpb.DescribeCollectionRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_ShowCollections_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.ShowCollectionRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).ShowCollections(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.master.Master/ShowCollections",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).ShowCollections(ctx, req.(*internalpb.ShowCollectionRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_CreatePartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.CreatePartitionRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).CreatePartition(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.master.Master/CreatePartition",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).CreatePartition(ctx, req.(*internalpb.CreatePartitionRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_DropPartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.DropPartitionRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).DropPartition(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.master.Master/DropPartition",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).DropPartition(ctx, req.(*internalpb.DropPartitionRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_HasPartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.HasPartitionRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).HasPartition(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.master.Master/HasPartition",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).HasPartition(ctx, req.(*internalpb.HasPartitionRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_DescribePartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.DescribePartitionRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).DescribePartition(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.master.Master/DescribePartition",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).DescribePartition(ctx, req.(*internalpb.DescribePartitionRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _Master_ShowPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(internalpb.ShowPartitionRequest)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MasterServer).ShowPartitions(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.master.Master/ShowPartitions",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MasterServer).ShowPartitions(ctx, req.(*internalpb.ShowPartitionRequest))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
var _Master_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "milvus.proto.master.Master",
|
||||
HandlerType: (*MasterServer)(nil),
|
||||
Methods: []grpc.MethodDesc{
|
||||
{
|
||||
MethodName: "CreateCollection",
|
||||
Handler: _Master_CreateCollection_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "DropCollection",
|
||||
Handler: _Master_DropCollection_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "HasCollection",
|
||||
Handler: _Master_HasCollection_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "DescribeCollection",
|
||||
Handler: _Master_DescribeCollection_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "ShowCollections",
|
||||
Handler: _Master_ShowCollections_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "CreatePartition",
|
||||
Handler: _Master_CreatePartition_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "DropPartition",
|
||||
Handler: _Master_DropPartition_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "HasPartition",
|
||||
Handler: _Master_HasPartition_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "DescribePartition",
|
||||
Handler: _Master_DescribePartition_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "ShowPartitions",
|
||||
Handler: _Master_ShowPartitions_Handler,
|
||||
},
|
||||
},
|
||||
Streams: []grpc.StreamDesc{},
|
||||
Metadata: "master.proto",
|
||||
}
|
|
@ -1,3 +1,9 @@
|
|||
syntax = "proto3";
|
||||
|
||||
package milvus.proto.schema;
|
||||
option go_package="github.com/zilliztech/milvus-distributed/internal/proto/schemapb";
|
||||
|
||||
import "common.proto";
|
||||
|
||||
/**
|
||||
* @brief Field data type
|
||||
|
@ -19,7 +25,6 @@ enum DataType {
|
|||
VECTOR_FLOAT = 101;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Field schema
|
||||
*/
|
||||
|
@ -27,17 +32,16 @@ message FieldSchema {
|
|||
string name = 1;
|
||||
string description = 2;
|
||||
DataType data_type = 3;
|
||||
repeated KeyValuePair type_params = 4;
|
||||
repeated KeyValuePair index_params = 5;
|
||||
repeated common.KeyValuePair type_params = 4;
|
||||
repeated common.KeyValuePair index_params = 5;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Collection schema
|
||||
*/
|
||||
message CollectionSchema {
|
||||
string name = 1;
|
||||
bool auto_id = 2;
|
||||
repeated FieldSchema fields = 3;
|
||||
}
|
||||
|
||||
string description = 2;
|
||||
bool auto_id = 3;
|
||||
repeated FieldSchema fields = 4;
|
||||
}
|
|
@ -0,0 +1,253 @@
|
|||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// source: schema.proto
|
||||
|
||||
package schemapb
|
||||
|
||||
import (
|
||||
fmt "fmt"
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
math "math"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
var _ = fmt.Errorf
|
||||
var _ = math.Inf
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the proto package it is being compiled against.
|
||||
// A compilation error at this line likely means your copy of the
|
||||
// proto package needs to be updated.
|
||||
const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
|
||||
|
||||
//*
|
||||
// @brief Field data type
|
||||
type DataType int32
|
||||
|
||||
const (
|
||||
DataType_NONE DataType = 0
|
||||
DataType_BOOL DataType = 1
|
||||
DataType_INT8 DataType = 2
|
||||
DataType_INT16 DataType = 3
|
||||
DataType_INT32 DataType = 4
|
||||
DataType_INT64 DataType = 5
|
||||
DataType_FLOAT DataType = 10
|
||||
DataType_DOUBLE DataType = 11
|
||||
DataType_STRING DataType = 20
|
||||
DataType_VECTOR_BINARY DataType = 100
|
||||
DataType_VECTOR_FLOAT DataType = 101
|
||||
)
|
||||
|
||||
var DataType_name = map[int32]string{
|
||||
0: "NONE",
|
||||
1: "BOOL",
|
||||
2: "INT8",
|
||||
3: "INT16",
|
||||
4: "INT32",
|
||||
5: "INT64",
|
||||
10: "FLOAT",
|
||||
11: "DOUBLE",
|
||||
20: "STRING",
|
||||
100: "VECTOR_BINARY",
|
||||
101: "VECTOR_FLOAT",
|
||||
}
|
||||
|
||||
var DataType_value = map[string]int32{
|
||||
"NONE": 0,
|
||||
"BOOL": 1,
|
||||
"INT8": 2,
|
||||
"INT16": 3,
|
||||
"INT32": 4,
|
||||
"INT64": 5,
|
||||
"FLOAT": 10,
|
||||
"DOUBLE": 11,
|
||||
"STRING": 20,
|
||||
"VECTOR_BINARY": 100,
|
||||
"VECTOR_FLOAT": 101,
|
||||
}
|
||||
|
||||
func (x DataType) String() string {
|
||||
return proto.EnumName(DataType_name, int32(x))
|
||||
}
|
||||
|
||||
func (DataType) EnumDescriptor() ([]byte, []int) {
|
||||
return fileDescriptor_1c5fb4d8cc22d66a, []int{0}
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Field schema
|
||||
type FieldSchema struct {
|
||||
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
|
||||
Description string `protobuf:"bytes,2,opt,name=description,proto3" json:"description,omitempty"`
|
||||
DataType DataType `protobuf:"varint,3,opt,name=data_type,json=dataType,proto3,enum=milvus.proto.schema.DataType" json:"data_type,omitempty"`
|
||||
TypeParams []*commonpb.KeyValuePair `protobuf:"bytes,4,rep,name=type_params,json=typeParams,proto3" json:"type_params,omitempty"`
|
||||
IndexParams []*commonpb.KeyValuePair `protobuf:"bytes,5,rep,name=index_params,json=indexParams,proto3" json:"index_params,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *FieldSchema) Reset() { *m = FieldSchema{} }
|
||||
func (m *FieldSchema) String() string { return proto.CompactTextString(m) }
|
||||
func (*FieldSchema) ProtoMessage() {}
|
||||
func (*FieldSchema) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_1c5fb4d8cc22d66a, []int{0}
|
||||
}
|
||||
|
||||
func (m *FieldSchema) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_FieldSchema.Unmarshal(m, b)
|
||||
}
|
||||
func (m *FieldSchema) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_FieldSchema.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *FieldSchema) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_FieldSchema.Merge(m, src)
|
||||
}
|
||||
func (m *FieldSchema) XXX_Size() int {
|
||||
return xxx_messageInfo_FieldSchema.Size(m)
|
||||
}
|
||||
func (m *FieldSchema) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_FieldSchema.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_FieldSchema proto.InternalMessageInfo
|
||||
|
||||
func (m *FieldSchema) GetName() string {
|
||||
if m != nil {
|
||||
return m.Name
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *FieldSchema) GetDescription() string {
|
||||
if m != nil {
|
||||
return m.Description
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *FieldSchema) GetDataType() DataType {
|
||||
if m != nil {
|
||||
return m.DataType
|
||||
}
|
||||
return DataType_NONE
|
||||
}
|
||||
|
||||
func (m *FieldSchema) GetTypeParams() []*commonpb.KeyValuePair {
|
||||
if m != nil {
|
||||
return m.TypeParams
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *FieldSchema) GetIndexParams() []*commonpb.KeyValuePair {
|
||||
if m != nil {
|
||||
return m.IndexParams
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Collection schema
|
||||
type CollectionSchema struct {
|
||||
Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
|
||||
Description string `protobuf:"bytes,2,opt,name=description,proto3" json:"description,omitempty"`
|
||||
AutoId bool `protobuf:"varint,3,opt,name=auto_id,json=autoId,proto3" json:"auto_id,omitempty"`
|
||||
Fields []*FieldSchema `protobuf:"bytes,4,rep,name=fields,proto3" json:"fields,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *CollectionSchema) Reset() { *m = CollectionSchema{} }
|
||||
func (m *CollectionSchema) String() string { return proto.CompactTextString(m) }
|
||||
func (*CollectionSchema) ProtoMessage() {}
|
||||
func (*CollectionSchema) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_1c5fb4d8cc22d66a, []int{1}
|
||||
}
|
||||
|
||||
func (m *CollectionSchema) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_CollectionSchema.Unmarshal(m, b)
|
||||
}
|
||||
func (m *CollectionSchema) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_CollectionSchema.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *CollectionSchema) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_CollectionSchema.Merge(m, src)
|
||||
}
|
||||
func (m *CollectionSchema) XXX_Size() int {
|
||||
return xxx_messageInfo_CollectionSchema.Size(m)
|
||||
}
|
||||
func (m *CollectionSchema) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_CollectionSchema.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_CollectionSchema proto.InternalMessageInfo
|
||||
|
||||
func (m *CollectionSchema) GetName() string {
|
||||
if m != nil {
|
||||
return m.Name
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *CollectionSchema) GetDescription() string {
|
||||
if m != nil {
|
||||
return m.Description
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *CollectionSchema) GetAutoId() bool {
|
||||
if m != nil {
|
||||
return m.AutoId
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func (m *CollectionSchema) GetFields() []*FieldSchema {
|
||||
if m != nil {
|
||||
return m.Fields
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterEnum("milvus.proto.schema.DataType", DataType_name, DataType_value)
|
||||
proto.RegisterType((*FieldSchema)(nil), "milvus.proto.schema.FieldSchema")
|
||||
proto.RegisterType((*CollectionSchema)(nil), "milvus.proto.schema.CollectionSchema")
|
||||
}
|
||||
|
||||
func init() { proto.RegisterFile("schema.proto", fileDescriptor_1c5fb4d8cc22d66a) }
|
||||
|
||||
var fileDescriptor_1c5fb4d8cc22d66a = []byte{
|
||||
// 419 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x91, 0x5f, 0x8b, 0xd4, 0x30,
|
||||
0x14, 0xc5, 0xed, 0xfc, 0x73, 0xe6, 0x76, 0x94, 0x18, 0x05, 0x07, 0x41, 0xa8, 0xfb, 0x34, 0x08,
|
||||
0xb6, 0x38, 0x2b, 0xcb, 0xe2, 0x93, 0xdb, 0x9d, 0x59, 0x29, 0x0e, 0xed, 0x92, 0xad, 0x0b, 0xfa,
|
||||
0x52, 0xd2, 0x26, 0x3a, 0x81, 0xf4, 0x0f, 0x6d, 0x2a, 0xce, 0x7e, 0x03, 0x1f, 0x7d, 0xf6, 0xcb,
|
||||
0x4a, 0xd2, 0x0e, 0xac, 0xb0, 0x0f, 0xc2, 0xbe, 0xfd, 0xee, 0xbd, 0x39, 0x37, 0x39, 0x27, 0x30,
|
||||
0x6f, 0xb2, 0x1d, 0xcf, 0xa9, 0x5b, 0xd5, 0xa5, 0x2a, 0xf1, 0xd3, 0x5c, 0xc8, 0x1f, 0x6d, 0xd3,
|
||||
0x55, 0x6e, 0x37, 0x7a, 0x31, 0xcf, 0xca, 0x3c, 0x2f, 0x8b, 0xae, 0x79, 0xf4, 0x6b, 0x00, 0xf6,
|
||||
0x85, 0xe0, 0x92, 0x5d, 0x99, 0x29, 0xc6, 0x30, 0x2a, 0x68, 0xce, 0x17, 0x96, 0x63, 0x2d, 0x67,
|
||||
0xc4, 0x30, 0x76, 0xc0, 0x66, 0xbc, 0xc9, 0x6a, 0x51, 0x29, 0x51, 0x16, 0x8b, 0x81, 0x19, 0xdd,
|
||||
0x6e, 0xe1, 0xf7, 0x30, 0x63, 0x54, 0xd1, 0x44, 0xed, 0x2b, 0xbe, 0x18, 0x3a, 0xd6, 0xf2, 0xf1,
|
||||
0xea, 0xa5, 0x7b, 0xc7, 0xe5, 0xee, 0x9a, 0x2a, 0x1a, 0xef, 0x2b, 0x4e, 0xa6, 0xac, 0x27, 0xec,
|
||||
0x83, 0xad, 0x65, 0x49, 0x45, 0x6b, 0x9a, 0x37, 0x8b, 0x91, 0x33, 0x5c, 0xda, 0xab, 0x57, 0xff,
|
||||
0xaa, 0xfb, 0x27, 0x7f, 0xe2, 0xfb, 0x6b, 0x2a, 0x5b, 0x7e, 0x49, 0x45, 0x4d, 0x40, 0xab, 0x2e,
|
||||
0x8d, 0x08, 0xaf, 0x61, 0x2e, 0x0a, 0xc6, 0x7f, 0x1e, 0x96, 0x8c, 0xff, 0x77, 0x89, 0x6d, 0x64,
|
||||
0xdd, 0x96, 0xa3, 0x3f, 0x16, 0xa0, 0xf3, 0x52, 0x4a, 0x9e, 0x69, 0x53, 0xf7, 0x0a, 0xe4, 0x39,
|
||||
0x3c, 0xa4, 0xad, 0x2a, 0x13, 0xc1, 0x4c, 0x1c, 0x53, 0x32, 0xd1, 0x65, 0xc0, 0xf0, 0x29, 0x4c,
|
||||
0xbe, 0xe9, 0xb8, 0x0f, 0x46, 0x9d, 0x3b, 0x63, 0xba, 0xf5, 0x23, 0xa4, 0x3f, 0xff, 0xfa, 0xb7,
|
||||
0x05, 0xd3, 0x43, 0x7c, 0x78, 0x0a, 0xa3, 0x30, 0x0a, 0x37, 0xe8, 0x81, 0x26, 0x3f, 0x8a, 0xb6,
|
||||
0xc8, 0xd2, 0x14, 0x84, 0xf1, 0x29, 0x1a, 0xe0, 0x19, 0x8c, 0x83, 0x30, 0x7e, 0x7b, 0x82, 0x86,
|
||||
0x3d, 0x1e, 0xaf, 0xd0, 0xa8, 0xc7, 0x93, 0x77, 0x68, 0xac, 0xf1, 0x62, 0x1b, 0x9d, 0xc5, 0x08,
|
||||
0x30, 0xc0, 0x64, 0x1d, 0x7d, 0xf6, 0xb7, 0x1b, 0x64, 0x6b, 0xbe, 0x8a, 0x49, 0x10, 0x7e, 0x44,
|
||||
0xcf, 0xf0, 0x13, 0x78, 0x74, 0xbd, 0x39, 0x8f, 0x23, 0x92, 0xf8, 0x41, 0x78, 0x46, 0xbe, 0x20,
|
||||
0x86, 0x11, 0xcc, 0xfb, 0x56, 0x27, 0xe6, 0xbe, 0xff, 0xf5, 0xc3, 0x77, 0xa1, 0x76, 0x6d, 0xaa,
|
||||
0xc3, 0xf5, 0x6e, 0x84, 0x94, 0xe2, 0x46, 0xf1, 0x6c, 0xe7, 0x75, 0xa6, 0xde, 0x30, 0xd1, 0xa8,
|
||||
0x5a, 0xa4, 0xad, 0xe2, 0xcc, 0x13, 0x85, 0xe2, 0x75, 0x41, 0xa5, 0x67, 0x9c, 0x7a, 0x9d, 0xd3,
|
||||
0x2a, 0x4d, 0x27, 0xa6, 0x3e, 0xfe, 0x1b, 0x00, 0x00, 0xff, 0xff, 0x6e, 0x3c, 0x4e, 0x3c, 0xbb,
|
||||
0x02, 0x00, 0x00,
|
||||
}
|
|
@ -1,150 +1,11 @@
|
|||
import "google/protobuf/empty.proto";
|
||||
syntax = "proto3";
|
||||
|
||||
package milvus.proto.service;
|
||||
option go_package="github.com/zilliztech/milvus-distributed/internal/proto/servicepb";
|
||||
|
||||
/**
|
||||
* @brief Collection name
|
||||
*/
|
||||
message CollectionName {
|
||||
string collection_name = 1;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Partition name
|
||||
*/
|
||||
message PartitionName {
|
||||
string collection_name = 1;
|
||||
string tag = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Row batch for Insert call
|
||||
*/
|
||||
message RowBatch {
|
||||
string collection_name = 1;
|
||||
string partition_tag = 2;
|
||||
repeated Blob row_data = 3;
|
||||
repeated uint64 row_id = 4;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Placeholder value in DSL
|
||||
*/
|
||||
message PlaceholderValue {
|
||||
string tag = 1;
|
||||
Blob value = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Query for Search call
|
||||
*/
|
||||
message Query {
|
||||
string collection_name = 1;
|
||||
repeated string partition_tags = 2;
|
||||
string dsl = 3;
|
||||
repeated PlaceholderValue placeholders = 4;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief String response
|
||||
*/
|
||||
message StringResponse {
|
||||
Status status = 1;
|
||||
string value = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Bool response
|
||||
*/
|
||||
message BoolResponse {
|
||||
Status status = 1;
|
||||
bool value = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief String list response
|
||||
*/
|
||||
message StringListResponse {
|
||||
Status status = 1;
|
||||
repeated string values = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Integer list response
|
||||
*/
|
||||
message IntegerListResponse {
|
||||
Status status = 1;
|
||||
repeated int64 values = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Range response, [begin, end)
|
||||
*/
|
||||
message IntegerRangeResponse {
|
||||
Status status = 1;
|
||||
repeated int64 begin = 2;
|
||||
repeated int64 end = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Response of DescribeCollection
|
||||
*/
|
||||
message CollectionDescription {
|
||||
Status status = 1;
|
||||
CollectionSchema schema = 2;
|
||||
repeated KeyValuePair statistics = 3;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Response of DescribePartition
|
||||
*/
|
||||
message PartitionDescription {
|
||||
Status status = 1;
|
||||
PartitionName name = 2;
|
||||
repeated KeyValuePair statistics = 3;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Scores of a query.
|
||||
* The default value of tag is "root".
|
||||
* It corresponds to the final score of each hit.
|
||||
*/
|
||||
message Score {
|
||||
string tag = 1;
|
||||
repeated float values = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Entities hit by query
|
||||
*/
|
||||
message Hits {
|
||||
Status status = 1;
|
||||
repeated int64 ids = 2;
|
||||
repeated Blob row_data = 4;
|
||||
repeated Score scores = 5;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Query result
|
||||
*/
|
||||
message QueryResult {
|
||||
Status status = 1;
|
||||
repeated Hits hits = 2;
|
||||
}
|
||||
|
||||
import "common.proto";
|
||||
import "service_msg.proto";
|
||||
import "schema.proto";
|
||||
|
||||
service MilvusService {
|
||||
/**
|
||||
|
@ -154,7 +15,7 @@ service MilvusService {
|
|||
*
|
||||
* @return Status
|
||||
*/
|
||||
rpc CreateCollection(CollectionSchema) returns (Status){}
|
||||
rpc CreateCollection(schema.CollectionSchema) returns (common.Status){}
|
||||
|
||||
/**
|
||||
* @brief This method is used to delete collection.
|
||||
|
@ -163,7 +24,7 @@ service MilvusService {
|
|||
*
|
||||
* @return Status
|
||||
*/
|
||||
rpc DropCollection(CollectionName) returns (Status) {}
|
||||
rpc DropCollection(CollectionName) returns (common.Status) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to test collection existence.
|
||||
|
@ -179,7 +40,7 @@ service MilvusService {
|
|||
*
|
||||
* @param CollectionName, target collection name.
|
||||
*
|
||||
* @return CollectionSchema
|
||||
* @return CollectionDescription
|
||||
*/
|
||||
rpc DescribeCollection(CollectionName) returns (CollectionDescription) {}
|
||||
|
||||
|
@ -188,21 +49,21 @@ service MilvusService {
|
|||
*
|
||||
* @return CollectionNameList
|
||||
*/
|
||||
rpc ShowCollections(google.protobuf.Empty) returns (StringListResponse) {}
|
||||
rpc ShowCollections(common.Empty) returns (StringListResponse) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to create partition
|
||||
*
|
||||
* @return Status
|
||||
*/
|
||||
rpc CreatePartition(PartitionName) returns (Status) {}
|
||||
rpc CreatePartition(PartitionName) returns (common.Status) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to drop partition
|
||||
*
|
||||
* @return Status
|
||||
*/
|
||||
rpc DropPartition(PartitionName) returns (Status) {}
|
||||
rpc DropPartition(PartitionName) returns (common.Status) {}
|
||||
|
||||
/**
|
||||
* @brief This method is used to test partition existence.
|
||||
|
|
|
@ -0,0 +1,151 @@
|
|||
syntax = "proto3";
|
||||
|
||||
package milvus.proto.service;
|
||||
option go_package="github.com/zilliztech/milvus-distributed/internal/proto/servicepb";
|
||||
|
||||
import "common.proto";
|
||||
import "schema.proto";
|
||||
|
||||
/**
|
||||
* @brief Collection name
|
||||
*/
|
||||
message CollectionName {
|
||||
string collection_name = 1;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Partition name
|
||||
*/
|
||||
message PartitionName {
|
||||
string collection_name = 1;
|
||||
string tag = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Row batch for Insert call
|
||||
*/
|
||||
message RowBatch {
|
||||
string collection_name = 1;
|
||||
string partition_tag = 2;
|
||||
repeated common.Blob row_data = 3;
|
||||
repeated int32 hash_values = 4;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Placeholder value in DSL
|
||||
*/
|
||||
message PlaceholderValue {
|
||||
string tag = 1;
|
||||
common.Blob value = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Query for Search call
|
||||
*/
|
||||
message Query {
|
||||
string collection_name = 1;
|
||||
repeated string partition_tags = 2;
|
||||
string dsl = 3;
|
||||
repeated PlaceholderValue placeholders = 4;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief String response
|
||||
*/
|
||||
message StringResponse {
|
||||
common.Status status = 1;
|
||||
string value = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Bool response
|
||||
*/
|
||||
message BoolResponse {
|
||||
common.Status status = 1;
|
||||
bool value = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief String list response
|
||||
*/
|
||||
message StringListResponse {
|
||||
common.Status status = 1;
|
||||
repeated string values = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Integer list response
|
||||
*/
|
||||
message IntegerListResponse {
|
||||
common.Status status = 1;
|
||||
repeated int64 values = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Range response, [begin, end)
|
||||
*/
|
||||
message IntegerRangeResponse {
|
||||
common.Status status = 1;
|
||||
repeated int64 begin = 2;
|
||||
repeated int64 end = 3;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Response of DescribeCollection
|
||||
*/
|
||||
message CollectionDescription {
|
||||
common.Status status = 1;
|
||||
schema.CollectionSchema schema = 2;
|
||||
repeated common.KeyValuePair statistics = 3;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Response of DescribePartition
|
||||
*/
|
||||
message PartitionDescription {
|
||||
common.Status status = 1;
|
||||
PartitionName name = 2;
|
||||
repeated common.KeyValuePair statistics = 3;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Scores of a query.
|
||||
* The default value of tag is "root".
|
||||
* It corresponds to the final score of each hit.
|
||||
*/
|
||||
message Score {
|
||||
string tag = 1;
|
||||
repeated float values = 2;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Entities hit by query
|
||||
*/
|
||||
message Hits {
|
||||
repeated int64 ids = 1;
|
||||
repeated common.Blob row_data = 2;
|
||||
repeated Score scores = 3;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @brief Query result
|
||||
*/
|
||||
message QueryResult {
|
||||
common.Status status = 1;
|
||||
repeated Hits hits = 2;
|
||||
}
|
||||
|
|
@ -0,0 +1,659 @@
|
|||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// source: service.proto
|
||||
|
||||
package servicepb
|
||||
|
||||
import (
|
||||
context "context"
|
||||
fmt "fmt"
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
schemapb "github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
grpc "google.golang.org/grpc"
|
||||
codes "google.golang.org/grpc/codes"
|
||||
status "google.golang.org/grpc/status"
|
||||
math "math"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
var _ = fmt.Errorf
|
||||
var _ = math.Inf
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the proto package it is being compiled against.
|
||||
// A compilation error at this line likely means your copy of the
|
||||
// proto package needs to be updated.
|
||||
const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
|
||||
|
||||
func init() { proto.RegisterFile("service.proto", fileDescriptor_a0b84a42fa06f626) }
|
||||
|
||||
var fileDescriptor_a0b84a42fa06f626 = []byte{
|
||||
// 412 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x92, 0x51, 0x6b, 0xd4, 0x40,
|
||||
0x10, 0xc7, 0xf3, 0x74, 0xc8, 0x72, 0x69, 0xed, 0xe2, 0x53, 0x0a, 0x82, 0x51, 0x41, 0x14, 0x13,
|
||||
0xd0, 0x4f, 0xe0, 0x5d, 0x85, 0x56, 0x54, 0x34, 0xa1, 0x62, 0xf5, 0x41, 0x36, 0xdb, 0x31, 0x59,
|
||||
0xd8, 0xec, 0x86, 0x9d, 0x49, 0x4b, 0xfb, 0x51, 0xfd, 0x34, 0x92, 0x6c, 0xec, 0xe5, 0x20, 0x7a,
|
||||
0x17, 0xf0, 0x2d, 0xfb, 0x9f, 0xff, 0xfc, 0x76, 0x26, 0xff, 0x65, 0x21, 0x82, 0xbb, 0x52, 0x12,
|
||||
0x92, 0xc6, 0x59, 0xb2, 0xfc, 0x41, 0xad, 0xf4, 0x55, 0x8b, 0xfe, 0x94, 0x0c, 0xb5, 0x68, 0x29,
|
||||
0x6d, 0x5d, 0x5b, 0xe3, 0xd5, 0xe8, 0x68, 0x90, 0x7f, 0xd4, 0x58, 0x0e, 0xd2, 0x12, 0x65, 0x05,
|
||||
0xb5, 0xf0, 0xa7, 0x57, 0xbf, 0xee, 0xb1, 0xf0, 0x43, 0xcf, 0xc9, 0xbd, 0x93, 0x7f, 0x65, 0xf7,
|
||||
0xd7, 0x0e, 0x04, 0xc1, 0xda, 0x6a, 0x0d, 0x92, 0x94, 0x35, 0xfc, 0x69, 0xb2, 0x7d, 0x97, 0x27,
|
||||
0x6c, 0x0c, 0x79, 0x2f, 0x44, 0xc7, 0xdb, 0xb6, 0x61, 0x92, 0x9c, 0x04, 0xb5, 0x18, 0x07, 0xfc,
|
||||
0x9c, 0x1d, 0x9c, 0x38, 0xdb, 0x8c, 0xb8, 0x4f, 0x92, 0xa9, 0x1d, 0x46, 0xe0, 0x8f, 0xa2, 0x86,
|
||||
0x5d, 0xd8, 0xef, 0x2c, 0x3c, 0x15, 0x38, 0x9b, 0x1a, 0x4f, 0xbb, 0x56, 0xd6, 0xea, 0x0c, 0xb0,
|
||||
0xb1, 0x06, 0x21, 0x0e, 0xb8, 0x62, 0xfc, 0x04, 0x50, 0x3a, 0x55, 0xc0, 0xec, 0x1b, 0x5e, 0xec,
|
||||
0x72, 0x79, 0x72, 0xd3, 0x7d, 0xc6, 0x01, 0xbf, 0x60, 0x87, 0x79, 0x65, 0xaf, 0x37, 0x65, 0xe4,
|
||||
0xd1, 0xe4, 0xe6, 0x6f, 0xeb, 0x86, 0x6e, 0xa2, 0x67, 0xd3, 0xf4, 0x9c, 0x9c, 0x32, 0xe5, 0x7b,
|
||||
0x85, 0x34, 0xda, 0xe2, 0x9c, 0x1d, 0xfa, 0x4c, 0x3f, 0x09, 0x47, 0xaa, 0x5f, 0xe1, 0xf1, 0x74,
|
||||
0xfb, 0x9d, 0x61, 0x9f, 0x3f, 0x9f, 0xb3, 0xb0, 0x0b, 0xf4, 0xff, 0x42, 0x2f, 0xd8, 0xf2, 0x54,
|
||||
0xe0, 0x4c, 0xe6, 0x7e, 0x61, 0xfe, 0x64, 0x47, 0x7f, 0xc2, 0x9c, 0xc9, 0x7f, 0xbe, 0xc3, 0xb4,
|
||||
0x9d, 0x64, 0xc1, 0x0e, 0xba, 0x24, 0xef, 0xaa, 0xb8, 0xe7, 0x83, 0x99, 0x13, 0xe9, 0x17, 0xb6,
|
||||
0x38, 0x33, 0x08, 0x8e, 0xf8, 0xc3, 0xe9, 0xae, 0xcc, 0x5e, 0xaf, 0x04, 0xc9, 0xea, 0x6f, 0xb3,
|
||||
0x9f, 0x19, 0x82, 0x12, 0x5c, 0x26, 0x4c, 0x09, 0x23, 0xee, 0x3b, 0xb6, 0xc8, 0x41, 0x38, 0x59,
|
||||
0xf1, 0xe3, 0xe9, 0xbe, 0xcf, 0x2d, 0xb8, 0x9b, 0xe8, 0xd1, 0x3f, 0x8a, 0x19, 0x60, 0xab, 0x29,
|
||||
0x0e, 0x56, 0xeb, 0x6f, 0x6f, 0x4a, 0x45, 0x55, 0x5b, 0x74, 0x21, 0xa7, 0xb7, 0x4a, 0x6b, 0x75,
|
||||
0x4b, 0x20, 0xab, 0xd4, 0xf7, 0xbe, 0xbc, 0x54, 0x48, 0x4e, 0x15, 0x2d, 0xc1, 0x65, 0xaa, 0x0c,
|
||||
0x81, 0x33, 0x42, 0xa7, 0x3d, 0x30, 0x1d, 0x80, 0x4d, 0x51, 0x2c, 0x7a, 0xe1, 0xf5, 0xef, 0x00,
|
||||
0x00, 0x00, 0xff, 0xff, 0x36, 0x07, 0xbc, 0xa2, 0xfe, 0x04, 0x00, 0x00,
|
||||
}
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ context.Context
|
||||
var _ grpc.ClientConn
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the grpc package it is being compiled against.
|
||||
const _ = grpc.SupportPackageIsVersion4
|
||||
|
||||
// MilvusServiceClient is the client API for MilvusService service.
|
||||
//
|
||||
// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
|
||||
type MilvusServiceClient interface {
|
||||
//*
|
||||
// @brief This method is used to create collection
|
||||
//
|
||||
// @param CollectionSchema, use to provide collection information to be created.
|
||||
//
|
||||
// @return Status
|
||||
CreateCollection(ctx context.Context, in *schemapb.CollectionSchema, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to delete collection.
|
||||
//
|
||||
// @param CollectionName, collection name is going to be deleted.
|
||||
//
|
||||
// @return Status
|
||||
DropCollection(ctx context.Context, in *CollectionName, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to test collection existence.
|
||||
//
|
||||
// @param CollectionName, collection name is going to be tested.
|
||||
//
|
||||
// @return BoolResponse
|
||||
HasCollection(ctx context.Context, in *CollectionName, opts ...grpc.CallOption) (*BoolResponse, error)
|
||||
//*
|
||||
// @brief This method is used to get collection schema.
|
||||
//
|
||||
// @param CollectionName, target collection name.
|
||||
//
|
||||
// @return CollectionDescription
|
||||
DescribeCollection(ctx context.Context, in *CollectionName, opts ...grpc.CallOption) (*CollectionDescription, error)
|
||||
//*
|
||||
// @brief This method is used to list all collections.
|
||||
//
|
||||
// @return CollectionNameList
|
||||
ShowCollections(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*StringListResponse, error)
|
||||
//*
|
||||
// @brief This method is used to create partition
|
||||
//
|
||||
// @return Status
|
||||
CreatePartition(ctx context.Context, in *PartitionName, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to drop partition
|
||||
//
|
||||
// @return Status
|
||||
DropPartition(ctx context.Context, in *PartitionName, opts ...grpc.CallOption) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to test partition existence.
|
||||
//
|
||||
// @return BoolResponse
|
||||
HasPartition(ctx context.Context, in *PartitionName, opts ...grpc.CallOption) (*BoolResponse, error)
|
||||
//*
|
||||
// @brief This method is used to get basic partition infomation.
|
||||
//
|
||||
// @return PartitionDescription
|
||||
DescribePartition(ctx context.Context, in *PartitionName, opts ...grpc.CallOption) (*PartitionDescription, error)
|
||||
//*
|
||||
// @brief This method is used to show partition information
|
||||
//
|
||||
// @param CollectionName, target collection name.
|
||||
//
|
||||
// @return StringListResponse
|
||||
ShowPartitions(ctx context.Context, in *CollectionName, opts ...grpc.CallOption) (*StringListResponse, error)
|
||||
//*
|
||||
// @brief This method is used to add vector array to collection.
|
||||
//
|
||||
// @param RowBatch, insert rows.
|
||||
//
|
||||
// @return IntegerRangeResponse contains id of the inserted rows.
|
||||
Insert(ctx context.Context, in *RowBatch, opts ...grpc.CallOption) (*IntegerRangeResponse, error)
|
||||
//*
|
||||
// @brief This method is used to query vector in collection.
|
||||
//
|
||||
// @param Query.
|
||||
//
|
||||
// @return QueryResult
|
||||
Search(ctx context.Context, in *Query, opts ...grpc.CallOption) (*QueryResult, error)
|
||||
}
|
||||
|
||||
type milvusServiceClient struct {
|
||||
cc *grpc.ClientConn
|
||||
}
|
||||
|
||||
func NewMilvusServiceClient(cc *grpc.ClientConn) MilvusServiceClient {
|
||||
return &milvusServiceClient{cc}
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) CreateCollection(ctx context.Context, in *schemapb.CollectionSchema, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
out := new(commonpb.Status)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/CreateCollection", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) DropCollection(ctx context.Context, in *CollectionName, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
out := new(commonpb.Status)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/DropCollection", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) HasCollection(ctx context.Context, in *CollectionName, opts ...grpc.CallOption) (*BoolResponse, error) {
|
||||
out := new(BoolResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/HasCollection", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) DescribeCollection(ctx context.Context, in *CollectionName, opts ...grpc.CallOption) (*CollectionDescription, error) {
|
||||
out := new(CollectionDescription)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/DescribeCollection", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) ShowCollections(ctx context.Context, in *commonpb.Empty, opts ...grpc.CallOption) (*StringListResponse, error) {
|
||||
out := new(StringListResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/ShowCollections", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) CreatePartition(ctx context.Context, in *PartitionName, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
out := new(commonpb.Status)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/CreatePartition", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) DropPartition(ctx context.Context, in *PartitionName, opts ...grpc.CallOption) (*commonpb.Status, error) {
|
||||
out := new(commonpb.Status)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/DropPartition", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) HasPartition(ctx context.Context, in *PartitionName, opts ...grpc.CallOption) (*BoolResponse, error) {
|
||||
out := new(BoolResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/HasPartition", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) DescribePartition(ctx context.Context, in *PartitionName, opts ...grpc.CallOption) (*PartitionDescription, error) {
|
||||
out := new(PartitionDescription)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/DescribePartition", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) ShowPartitions(ctx context.Context, in *CollectionName, opts ...grpc.CallOption) (*StringListResponse, error) {
|
||||
out := new(StringListResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/ShowPartitions", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) Insert(ctx context.Context, in *RowBatch, opts ...grpc.CallOption) (*IntegerRangeResponse, error) {
|
||||
out := new(IntegerRangeResponse)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/Insert", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
func (c *milvusServiceClient) Search(ctx context.Context, in *Query, opts ...grpc.CallOption) (*QueryResult, error) {
|
||||
out := new(QueryResult)
|
||||
err := c.cc.Invoke(ctx, "/milvus.proto.service.MilvusService/Search", in, out, opts...)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return out, nil
|
||||
}
|
||||
|
||||
// MilvusServiceServer is the server API for MilvusService service.
|
||||
type MilvusServiceServer interface {
|
||||
//*
|
||||
// @brief This method is used to create collection
|
||||
//
|
||||
// @param CollectionSchema, use to provide collection information to be created.
|
||||
//
|
||||
// @return Status
|
||||
CreateCollection(context.Context, *schemapb.CollectionSchema) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to delete collection.
|
||||
//
|
||||
// @param CollectionName, collection name is going to be deleted.
|
||||
//
|
||||
// @return Status
|
||||
DropCollection(context.Context, *CollectionName) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to test collection existence.
|
||||
//
|
||||
// @param CollectionName, collection name is going to be tested.
|
||||
//
|
||||
// @return BoolResponse
|
||||
HasCollection(context.Context, *CollectionName) (*BoolResponse, error)
|
||||
//*
|
||||
// @brief This method is used to get collection schema.
|
||||
//
|
||||
// @param CollectionName, target collection name.
|
||||
//
|
||||
// @return CollectionDescription
|
||||
DescribeCollection(context.Context, *CollectionName) (*CollectionDescription, error)
|
||||
//*
|
||||
// @brief This method is used to list all collections.
|
||||
//
|
||||
// @return CollectionNameList
|
||||
ShowCollections(context.Context, *commonpb.Empty) (*StringListResponse, error)
|
||||
//*
|
||||
// @brief This method is used to create partition
|
||||
//
|
||||
// @return Status
|
||||
CreatePartition(context.Context, *PartitionName) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to drop partition
|
||||
//
|
||||
// @return Status
|
||||
DropPartition(context.Context, *PartitionName) (*commonpb.Status, error)
|
||||
//*
|
||||
// @brief This method is used to test partition existence.
|
||||
//
|
||||
// @return BoolResponse
|
||||
HasPartition(context.Context, *PartitionName) (*BoolResponse, error)
|
||||
//*
|
||||
// @brief This method is used to get basic partition infomation.
|
||||
//
|
||||
// @return PartitionDescription
|
||||
DescribePartition(context.Context, *PartitionName) (*PartitionDescription, error)
|
||||
//*
|
||||
// @brief This method is used to show partition information
|
||||
//
|
||||
// @param CollectionName, target collection name.
|
||||
//
|
||||
// @return StringListResponse
|
||||
ShowPartitions(context.Context, *CollectionName) (*StringListResponse, error)
|
||||
//*
|
||||
// @brief This method is used to add vector array to collection.
|
||||
//
|
||||
// @param RowBatch, insert rows.
|
||||
//
|
||||
// @return IntegerRangeResponse contains id of the inserted rows.
|
||||
Insert(context.Context, *RowBatch) (*IntegerRangeResponse, error)
|
||||
//*
|
||||
// @brief This method is used to query vector in collection.
|
||||
//
|
||||
// @param Query.
|
||||
//
|
||||
// @return QueryResult
|
||||
Search(context.Context, *Query) (*QueryResult, error)
|
||||
}
|
||||
|
||||
// UnimplementedMilvusServiceServer can be embedded to have forward compatible implementations.
|
||||
type UnimplementedMilvusServiceServer struct {
|
||||
}
|
||||
|
||||
func (*UnimplementedMilvusServiceServer) CreateCollection(ctx context.Context, req *schemapb.CollectionSchema) (*commonpb.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method CreateCollection not implemented")
|
||||
}
|
||||
func (*UnimplementedMilvusServiceServer) DropCollection(ctx context.Context, req *CollectionName) (*commonpb.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method DropCollection not implemented")
|
||||
}
|
||||
func (*UnimplementedMilvusServiceServer) HasCollection(ctx context.Context, req *CollectionName) (*BoolResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method HasCollection not implemented")
|
||||
}
|
||||
func (*UnimplementedMilvusServiceServer) DescribeCollection(ctx context.Context, req *CollectionName) (*CollectionDescription, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method DescribeCollection not implemented")
|
||||
}
|
||||
func (*UnimplementedMilvusServiceServer) ShowCollections(ctx context.Context, req *commonpb.Empty) (*StringListResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method ShowCollections not implemented")
|
||||
}
|
||||
func (*UnimplementedMilvusServiceServer) CreatePartition(ctx context.Context, req *PartitionName) (*commonpb.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method CreatePartition not implemented")
|
||||
}
|
||||
func (*UnimplementedMilvusServiceServer) DropPartition(ctx context.Context, req *PartitionName) (*commonpb.Status, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method DropPartition not implemented")
|
||||
}
|
||||
func (*UnimplementedMilvusServiceServer) HasPartition(ctx context.Context, req *PartitionName) (*BoolResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method HasPartition not implemented")
|
||||
}
|
||||
func (*UnimplementedMilvusServiceServer) DescribePartition(ctx context.Context, req *PartitionName) (*PartitionDescription, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method DescribePartition not implemented")
|
||||
}
|
||||
func (*UnimplementedMilvusServiceServer) ShowPartitions(ctx context.Context, req *CollectionName) (*StringListResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method ShowPartitions not implemented")
|
||||
}
|
||||
func (*UnimplementedMilvusServiceServer) Insert(ctx context.Context, req *RowBatch) (*IntegerRangeResponse, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method Insert not implemented")
|
||||
}
|
||||
func (*UnimplementedMilvusServiceServer) Search(ctx context.Context, req *Query) (*QueryResult, error) {
|
||||
return nil, status.Errorf(codes.Unimplemented, "method Search not implemented")
|
||||
}
|
||||
|
||||
func RegisterMilvusServiceServer(s *grpc.Server, srv MilvusServiceServer) {
|
||||
s.RegisterService(&_MilvusService_serviceDesc, srv)
|
||||
}
|
||||
|
||||
func _MilvusService_CreateCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(schemapb.CollectionSchema)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).CreateCollection(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/CreateCollection",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).CreateCollection(ctx, req.(*schemapb.CollectionSchema))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _MilvusService_DropCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(CollectionName)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).DropCollection(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/DropCollection",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).DropCollection(ctx, req.(*CollectionName))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _MilvusService_HasCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(CollectionName)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).HasCollection(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/HasCollection",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).HasCollection(ctx, req.(*CollectionName))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _MilvusService_DescribeCollection_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(CollectionName)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).DescribeCollection(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/DescribeCollection",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).DescribeCollection(ctx, req.(*CollectionName))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _MilvusService_ShowCollections_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(commonpb.Empty)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).ShowCollections(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/ShowCollections",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).ShowCollections(ctx, req.(*commonpb.Empty))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _MilvusService_CreatePartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(PartitionName)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).CreatePartition(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/CreatePartition",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).CreatePartition(ctx, req.(*PartitionName))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _MilvusService_DropPartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(PartitionName)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).DropPartition(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/DropPartition",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).DropPartition(ctx, req.(*PartitionName))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _MilvusService_HasPartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(PartitionName)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).HasPartition(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/HasPartition",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).HasPartition(ctx, req.(*PartitionName))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _MilvusService_DescribePartition_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(PartitionName)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).DescribePartition(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/DescribePartition",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).DescribePartition(ctx, req.(*PartitionName))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _MilvusService_ShowPartitions_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(CollectionName)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).ShowPartitions(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/ShowPartitions",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).ShowPartitions(ctx, req.(*CollectionName))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _MilvusService_Insert_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(RowBatch)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).Insert(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/Insert",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).Insert(ctx, req.(*RowBatch))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
func _MilvusService_Search_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
|
||||
in := new(Query)
|
||||
if err := dec(in); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if interceptor == nil {
|
||||
return srv.(MilvusServiceServer).Search(ctx, in)
|
||||
}
|
||||
info := &grpc.UnaryServerInfo{
|
||||
Server: srv,
|
||||
FullMethod: "/milvus.proto.service.MilvusService/Search",
|
||||
}
|
||||
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
|
||||
return srv.(MilvusServiceServer).Search(ctx, req.(*Query))
|
||||
}
|
||||
return interceptor(ctx, in, info, handler)
|
||||
}
|
||||
|
||||
var _MilvusService_serviceDesc = grpc.ServiceDesc{
|
||||
ServiceName: "milvus.proto.service.MilvusService",
|
||||
HandlerType: (*MilvusServiceServer)(nil),
|
||||
Methods: []grpc.MethodDesc{
|
||||
{
|
||||
MethodName: "CreateCollection",
|
||||
Handler: _MilvusService_CreateCollection_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "DropCollection",
|
||||
Handler: _MilvusService_DropCollection_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "HasCollection",
|
||||
Handler: _MilvusService_HasCollection_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "DescribeCollection",
|
||||
Handler: _MilvusService_DescribeCollection_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "ShowCollections",
|
||||
Handler: _MilvusService_ShowCollections_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "CreatePartition",
|
||||
Handler: _MilvusService_CreatePartition_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "DropPartition",
|
||||
Handler: _MilvusService_DropPartition_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "HasPartition",
|
||||
Handler: _MilvusService_HasPartition_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "DescribePartition",
|
||||
Handler: _MilvusService_DescribePartition_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "ShowPartitions",
|
||||
Handler: _MilvusService_ShowPartitions_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "Insert",
|
||||
Handler: _MilvusService_Insert_Handler,
|
||||
},
|
||||
{
|
||||
MethodName: "Search",
|
||||
Handler: _MilvusService_Search_Handler,
|
||||
},
|
||||
},
|
||||
Streams: []grpc.StreamDesc{},
|
||||
Metadata: "service.proto",
|
||||
}
|
|
@ -0,0 +1,882 @@
|
|||
// Code generated by protoc-gen-go. DO NOT EDIT.
|
||||
// source: service_msg.proto
|
||||
|
||||
package servicepb
|
||||
|
||||
import (
|
||||
fmt "fmt"
|
||||
proto "github.com/golang/protobuf/proto"
|
||||
commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
schemapb "github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
math "math"
|
||||
)
|
||||
|
||||
// Reference imports to suppress errors if they are not otherwise used.
|
||||
var _ = proto.Marshal
|
||||
var _ = fmt.Errorf
|
||||
var _ = math.Inf
|
||||
|
||||
// This is a compile-time assertion to ensure that this generated file
|
||||
// is compatible with the proto package it is being compiled against.
|
||||
// A compilation error at this line likely means your copy of the
|
||||
// proto package needs to be updated.
|
||||
const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
|
||||
|
||||
//*
|
||||
// @brief Collection name
|
||||
type CollectionName struct {
|
||||
CollectionName string `protobuf:"bytes,1,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *CollectionName) Reset() { *m = CollectionName{} }
|
||||
func (m *CollectionName) String() string { return proto.CompactTextString(m) }
|
||||
func (*CollectionName) ProtoMessage() {}
|
||||
func (*CollectionName) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{0}
|
||||
}
|
||||
|
||||
func (m *CollectionName) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_CollectionName.Unmarshal(m, b)
|
||||
}
|
||||
func (m *CollectionName) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_CollectionName.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *CollectionName) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_CollectionName.Merge(m, src)
|
||||
}
|
||||
func (m *CollectionName) XXX_Size() int {
|
||||
return xxx_messageInfo_CollectionName.Size(m)
|
||||
}
|
||||
func (m *CollectionName) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_CollectionName.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_CollectionName proto.InternalMessageInfo
|
||||
|
||||
func (m *CollectionName) GetCollectionName() string {
|
||||
if m != nil {
|
||||
return m.CollectionName
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Partition name
|
||||
type PartitionName struct {
|
||||
CollectionName string `protobuf:"bytes,1,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
|
||||
Tag string `protobuf:"bytes,2,opt,name=tag,proto3" json:"tag,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *PartitionName) Reset() { *m = PartitionName{} }
|
||||
func (m *PartitionName) String() string { return proto.CompactTextString(m) }
|
||||
func (*PartitionName) ProtoMessage() {}
|
||||
func (*PartitionName) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{1}
|
||||
}
|
||||
|
||||
func (m *PartitionName) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_PartitionName.Unmarshal(m, b)
|
||||
}
|
||||
func (m *PartitionName) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_PartitionName.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *PartitionName) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_PartitionName.Merge(m, src)
|
||||
}
|
||||
func (m *PartitionName) XXX_Size() int {
|
||||
return xxx_messageInfo_PartitionName.Size(m)
|
||||
}
|
||||
func (m *PartitionName) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_PartitionName.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_PartitionName proto.InternalMessageInfo
|
||||
|
||||
func (m *PartitionName) GetCollectionName() string {
|
||||
if m != nil {
|
||||
return m.CollectionName
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *PartitionName) GetTag() string {
|
||||
if m != nil {
|
||||
return m.Tag
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Row batch for Insert call
|
||||
type RowBatch struct {
|
||||
CollectionName string `protobuf:"bytes,1,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
|
||||
PartitionTag string `protobuf:"bytes,2,opt,name=partition_tag,json=partitionTag,proto3" json:"partition_tag,omitempty"`
|
||||
RowData []*commonpb.Blob `protobuf:"bytes,3,rep,name=row_data,json=rowData,proto3" json:"row_data,omitempty"`
|
||||
HashValues []int32 `protobuf:"varint,4,rep,packed,name=hash_values,json=hashValues,proto3" json:"hash_values,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *RowBatch) Reset() { *m = RowBatch{} }
|
||||
func (m *RowBatch) String() string { return proto.CompactTextString(m) }
|
||||
func (*RowBatch) ProtoMessage() {}
|
||||
func (*RowBatch) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{2}
|
||||
}
|
||||
|
||||
func (m *RowBatch) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_RowBatch.Unmarshal(m, b)
|
||||
}
|
||||
func (m *RowBatch) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_RowBatch.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *RowBatch) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_RowBatch.Merge(m, src)
|
||||
}
|
||||
func (m *RowBatch) XXX_Size() int {
|
||||
return xxx_messageInfo_RowBatch.Size(m)
|
||||
}
|
||||
func (m *RowBatch) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_RowBatch.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_RowBatch proto.InternalMessageInfo
|
||||
|
||||
func (m *RowBatch) GetCollectionName() string {
|
||||
if m != nil {
|
||||
return m.CollectionName
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *RowBatch) GetPartitionTag() string {
|
||||
if m != nil {
|
||||
return m.PartitionTag
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *RowBatch) GetRowData() []*commonpb.Blob {
|
||||
if m != nil {
|
||||
return m.RowData
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *RowBatch) GetHashValues() []int32 {
|
||||
if m != nil {
|
||||
return m.HashValues
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Placeholder value in DSL
|
||||
type PlaceholderValue struct {
|
||||
Tag string `protobuf:"bytes,1,opt,name=tag,proto3" json:"tag,omitempty"`
|
||||
Value *commonpb.Blob `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *PlaceholderValue) Reset() { *m = PlaceholderValue{} }
|
||||
func (m *PlaceholderValue) String() string { return proto.CompactTextString(m) }
|
||||
func (*PlaceholderValue) ProtoMessage() {}
|
||||
func (*PlaceholderValue) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{3}
|
||||
}
|
||||
|
||||
func (m *PlaceholderValue) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_PlaceholderValue.Unmarshal(m, b)
|
||||
}
|
||||
func (m *PlaceholderValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_PlaceholderValue.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *PlaceholderValue) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_PlaceholderValue.Merge(m, src)
|
||||
}
|
||||
func (m *PlaceholderValue) XXX_Size() int {
|
||||
return xxx_messageInfo_PlaceholderValue.Size(m)
|
||||
}
|
||||
func (m *PlaceholderValue) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_PlaceholderValue.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_PlaceholderValue proto.InternalMessageInfo
|
||||
|
||||
func (m *PlaceholderValue) GetTag() string {
|
||||
if m != nil {
|
||||
return m.Tag
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *PlaceholderValue) GetValue() *commonpb.Blob {
|
||||
if m != nil {
|
||||
return m.Value
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Query for Search call
|
||||
type Query struct {
|
||||
CollectionName string `protobuf:"bytes,1,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"`
|
||||
PartitionTags []string `protobuf:"bytes,2,rep,name=partition_tags,json=partitionTags,proto3" json:"partition_tags,omitempty"`
|
||||
Dsl string `protobuf:"bytes,3,opt,name=dsl,proto3" json:"dsl,omitempty"`
|
||||
Placeholders []*PlaceholderValue `protobuf:"bytes,4,rep,name=placeholders,proto3" json:"placeholders,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Query) Reset() { *m = Query{} }
|
||||
func (m *Query) String() string { return proto.CompactTextString(m) }
|
||||
func (*Query) ProtoMessage() {}
|
||||
func (*Query) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{4}
|
||||
}
|
||||
|
||||
func (m *Query) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Query.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Query) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Query.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *Query) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Query.Merge(m, src)
|
||||
}
|
||||
func (m *Query) XXX_Size() int {
|
||||
return xxx_messageInfo_Query.Size(m)
|
||||
}
|
||||
func (m *Query) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Query.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Query proto.InternalMessageInfo
|
||||
|
||||
func (m *Query) GetCollectionName() string {
|
||||
if m != nil {
|
||||
return m.CollectionName
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *Query) GetPartitionTags() []string {
|
||||
if m != nil {
|
||||
return m.PartitionTags
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *Query) GetDsl() string {
|
||||
if m != nil {
|
||||
return m.Dsl
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *Query) GetPlaceholders() []*PlaceholderValue {
|
||||
if m != nil {
|
||||
return m.Placeholders
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief String response
|
||||
type StringResponse struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *StringResponse) Reset() { *m = StringResponse{} }
|
||||
func (m *StringResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*StringResponse) ProtoMessage() {}
|
||||
func (*StringResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{5}
|
||||
}
|
||||
|
||||
func (m *StringResponse) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_StringResponse.Unmarshal(m, b)
|
||||
}
|
||||
func (m *StringResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_StringResponse.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *StringResponse) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_StringResponse.Merge(m, src)
|
||||
}
|
||||
func (m *StringResponse) XXX_Size() int {
|
||||
return xxx_messageInfo_StringResponse.Size(m)
|
||||
}
|
||||
func (m *StringResponse) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_StringResponse.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_StringResponse proto.InternalMessageInfo
|
||||
|
||||
func (m *StringResponse) GetStatus() *commonpb.Status {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *StringResponse) GetValue() string {
|
||||
if m != nil {
|
||||
return m.Value
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Bool response
|
||||
type BoolResponse struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
Value bool `protobuf:"varint,2,opt,name=value,proto3" json:"value,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *BoolResponse) Reset() { *m = BoolResponse{} }
|
||||
func (m *BoolResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*BoolResponse) ProtoMessage() {}
|
||||
func (*BoolResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{6}
|
||||
}
|
||||
|
||||
func (m *BoolResponse) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_BoolResponse.Unmarshal(m, b)
|
||||
}
|
||||
func (m *BoolResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_BoolResponse.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *BoolResponse) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_BoolResponse.Merge(m, src)
|
||||
}
|
||||
func (m *BoolResponse) XXX_Size() int {
|
||||
return xxx_messageInfo_BoolResponse.Size(m)
|
||||
}
|
||||
func (m *BoolResponse) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_BoolResponse.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_BoolResponse proto.InternalMessageInfo
|
||||
|
||||
func (m *BoolResponse) GetStatus() *commonpb.Status {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *BoolResponse) GetValue() bool {
|
||||
if m != nil {
|
||||
return m.Value
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief String list response
|
||||
type StringListResponse struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
Values []string `protobuf:"bytes,2,rep,name=values,proto3" json:"values,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *StringListResponse) Reset() { *m = StringListResponse{} }
|
||||
func (m *StringListResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*StringListResponse) ProtoMessage() {}
|
||||
func (*StringListResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{7}
|
||||
}
|
||||
|
||||
func (m *StringListResponse) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_StringListResponse.Unmarshal(m, b)
|
||||
}
|
||||
func (m *StringListResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_StringListResponse.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *StringListResponse) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_StringListResponse.Merge(m, src)
|
||||
}
|
||||
func (m *StringListResponse) XXX_Size() int {
|
||||
return xxx_messageInfo_StringListResponse.Size(m)
|
||||
}
|
||||
func (m *StringListResponse) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_StringListResponse.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_StringListResponse proto.InternalMessageInfo
|
||||
|
||||
func (m *StringListResponse) GetStatus() *commonpb.Status {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *StringListResponse) GetValues() []string {
|
||||
if m != nil {
|
||||
return m.Values
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Integer list response
|
||||
type IntegerListResponse struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
Values []int64 `protobuf:"varint,2,rep,packed,name=values,proto3" json:"values,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *IntegerListResponse) Reset() { *m = IntegerListResponse{} }
|
||||
func (m *IntegerListResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*IntegerListResponse) ProtoMessage() {}
|
||||
func (*IntegerListResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{8}
|
||||
}
|
||||
|
||||
func (m *IntegerListResponse) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_IntegerListResponse.Unmarshal(m, b)
|
||||
}
|
||||
func (m *IntegerListResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_IntegerListResponse.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *IntegerListResponse) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_IntegerListResponse.Merge(m, src)
|
||||
}
|
||||
func (m *IntegerListResponse) XXX_Size() int {
|
||||
return xxx_messageInfo_IntegerListResponse.Size(m)
|
||||
}
|
||||
func (m *IntegerListResponse) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_IntegerListResponse.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_IntegerListResponse proto.InternalMessageInfo
|
||||
|
||||
func (m *IntegerListResponse) GetStatus() *commonpb.Status {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *IntegerListResponse) GetValues() []int64 {
|
||||
if m != nil {
|
||||
return m.Values
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Range response, [begin, end)
|
||||
type IntegerRangeResponse struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
Begin []int64 `protobuf:"varint,2,rep,packed,name=begin,proto3" json:"begin,omitempty"`
|
||||
End []int64 `protobuf:"varint,3,rep,packed,name=end,proto3" json:"end,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *IntegerRangeResponse) Reset() { *m = IntegerRangeResponse{} }
|
||||
func (m *IntegerRangeResponse) String() string { return proto.CompactTextString(m) }
|
||||
func (*IntegerRangeResponse) ProtoMessage() {}
|
||||
func (*IntegerRangeResponse) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{9}
|
||||
}
|
||||
|
||||
func (m *IntegerRangeResponse) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_IntegerRangeResponse.Unmarshal(m, b)
|
||||
}
|
||||
func (m *IntegerRangeResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_IntegerRangeResponse.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *IntegerRangeResponse) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_IntegerRangeResponse.Merge(m, src)
|
||||
}
|
||||
func (m *IntegerRangeResponse) XXX_Size() int {
|
||||
return xxx_messageInfo_IntegerRangeResponse.Size(m)
|
||||
}
|
||||
func (m *IntegerRangeResponse) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_IntegerRangeResponse.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_IntegerRangeResponse proto.InternalMessageInfo
|
||||
|
||||
func (m *IntegerRangeResponse) GetStatus() *commonpb.Status {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *IntegerRangeResponse) GetBegin() []int64 {
|
||||
if m != nil {
|
||||
return m.Begin
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *IntegerRangeResponse) GetEnd() []int64 {
|
||||
if m != nil {
|
||||
return m.End
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Response of DescribeCollection
|
||||
type CollectionDescription struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
Schema *schemapb.CollectionSchema `protobuf:"bytes,2,opt,name=schema,proto3" json:"schema,omitempty"`
|
||||
Statistics []*commonpb.KeyValuePair `protobuf:"bytes,3,rep,name=statistics,proto3" json:"statistics,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *CollectionDescription) Reset() { *m = CollectionDescription{} }
|
||||
func (m *CollectionDescription) String() string { return proto.CompactTextString(m) }
|
||||
func (*CollectionDescription) ProtoMessage() {}
|
||||
func (*CollectionDescription) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{10}
|
||||
}
|
||||
|
||||
func (m *CollectionDescription) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_CollectionDescription.Unmarshal(m, b)
|
||||
}
|
||||
func (m *CollectionDescription) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_CollectionDescription.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *CollectionDescription) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_CollectionDescription.Merge(m, src)
|
||||
}
|
||||
func (m *CollectionDescription) XXX_Size() int {
|
||||
return xxx_messageInfo_CollectionDescription.Size(m)
|
||||
}
|
||||
func (m *CollectionDescription) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_CollectionDescription.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_CollectionDescription proto.InternalMessageInfo
|
||||
|
||||
func (m *CollectionDescription) GetStatus() *commonpb.Status {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *CollectionDescription) GetSchema() *schemapb.CollectionSchema {
|
||||
if m != nil {
|
||||
return m.Schema
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *CollectionDescription) GetStatistics() []*commonpb.KeyValuePair {
|
||||
if m != nil {
|
||||
return m.Statistics
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Response of DescribePartition
|
||||
type PartitionDescription struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
Name *PartitionName `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"`
|
||||
Statistics []*commonpb.KeyValuePair `protobuf:"bytes,3,rep,name=statistics,proto3" json:"statistics,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *PartitionDescription) Reset() { *m = PartitionDescription{} }
|
||||
func (m *PartitionDescription) String() string { return proto.CompactTextString(m) }
|
||||
func (*PartitionDescription) ProtoMessage() {}
|
||||
func (*PartitionDescription) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{11}
|
||||
}
|
||||
|
||||
func (m *PartitionDescription) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_PartitionDescription.Unmarshal(m, b)
|
||||
}
|
||||
func (m *PartitionDescription) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_PartitionDescription.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *PartitionDescription) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_PartitionDescription.Merge(m, src)
|
||||
}
|
||||
func (m *PartitionDescription) XXX_Size() int {
|
||||
return xxx_messageInfo_PartitionDescription.Size(m)
|
||||
}
|
||||
func (m *PartitionDescription) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_PartitionDescription.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_PartitionDescription proto.InternalMessageInfo
|
||||
|
||||
func (m *PartitionDescription) GetStatus() *commonpb.Status {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *PartitionDescription) GetName() *PartitionName {
|
||||
if m != nil {
|
||||
return m.Name
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *PartitionDescription) GetStatistics() []*commonpb.KeyValuePair {
|
||||
if m != nil {
|
||||
return m.Statistics
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Scores of a query.
|
||||
// The default value of tag is "root".
|
||||
// It corresponds to the final score of each hit.
|
||||
type Score struct {
|
||||
Tag string `protobuf:"bytes,1,opt,name=tag,proto3" json:"tag,omitempty"`
|
||||
Values []float32 `protobuf:"fixed32,2,rep,packed,name=values,proto3" json:"values,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Score) Reset() { *m = Score{} }
|
||||
func (m *Score) String() string { return proto.CompactTextString(m) }
|
||||
func (*Score) ProtoMessage() {}
|
||||
func (*Score) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{12}
|
||||
}
|
||||
|
||||
func (m *Score) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Score.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Score) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Score.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *Score) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Score.Merge(m, src)
|
||||
}
|
||||
func (m *Score) XXX_Size() int {
|
||||
return xxx_messageInfo_Score.Size(m)
|
||||
}
|
||||
func (m *Score) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Score.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Score proto.InternalMessageInfo
|
||||
|
||||
func (m *Score) GetTag() string {
|
||||
if m != nil {
|
||||
return m.Tag
|
||||
}
|
||||
return ""
|
||||
}
|
||||
|
||||
func (m *Score) GetValues() []float32 {
|
||||
if m != nil {
|
||||
return m.Values
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Entities hit by query
|
||||
type Hits struct {
|
||||
Ids []int64 `protobuf:"varint,1,rep,packed,name=ids,proto3" json:"ids,omitempty"`
|
||||
RowData []*commonpb.Blob `protobuf:"bytes,2,rep,name=row_data,json=rowData,proto3" json:"row_data,omitempty"`
|
||||
Scores []*Score `protobuf:"bytes,3,rep,name=scores,proto3" json:"scores,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *Hits) Reset() { *m = Hits{} }
|
||||
func (m *Hits) String() string { return proto.CompactTextString(m) }
|
||||
func (*Hits) ProtoMessage() {}
|
||||
func (*Hits) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{13}
|
||||
}
|
||||
|
||||
func (m *Hits) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_Hits.Unmarshal(m, b)
|
||||
}
|
||||
func (m *Hits) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_Hits.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *Hits) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_Hits.Merge(m, src)
|
||||
}
|
||||
func (m *Hits) XXX_Size() int {
|
||||
return xxx_messageInfo_Hits.Size(m)
|
||||
}
|
||||
func (m *Hits) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_Hits.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_Hits proto.InternalMessageInfo
|
||||
|
||||
func (m *Hits) GetIds() []int64 {
|
||||
if m != nil {
|
||||
return m.Ids
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *Hits) GetRowData() []*commonpb.Blob {
|
||||
if m != nil {
|
||||
return m.RowData
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *Hits) GetScores() []*Score {
|
||||
if m != nil {
|
||||
return m.Scores
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
//*
|
||||
// @brief Query result
|
||||
type QueryResult struct {
|
||||
Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"`
|
||||
Hits []*Hits `protobuf:"bytes,2,rep,name=hits,proto3" json:"hits,omitempty"`
|
||||
XXX_NoUnkeyedLiteral struct{} `json:"-"`
|
||||
XXX_unrecognized []byte `json:"-"`
|
||||
XXX_sizecache int32 `json:"-"`
|
||||
}
|
||||
|
||||
func (m *QueryResult) Reset() { *m = QueryResult{} }
|
||||
func (m *QueryResult) String() string { return proto.CompactTextString(m) }
|
||||
func (*QueryResult) ProtoMessage() {}
|
||||
func (*QueryResult) Descriptor() ([]byte, []int) {
|
||||
return fileDescriptor_b4b40b84dd2f74cb, []int{14}
|
||||
}
|
||||
|
||||
func (m *QueryResult) XXX_Unmarshal(b []byte) error {
|
||||
return xxx_messageInfo_QueryResult.Unmarshal(m, b)
|
||||
}
|
||||
func (m *QueryResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
|
||||
return xxx_messageInfo_QueryResult.Marshal(b, m, deterministic)
|
||||
}
|
||||
func (m *QueryResult) XXX_Merge(src proto.Message) {
|
||||
xxx_messageInfo_QueryResult.Merge(m, src)
|
||||
}
|
||||
func (m *QueryResult) XXX_Size() int {
|
||||
return xxx_messageInfo_QueryResult.Size(m)
|
||||
}
|
||||
func (m *QueryResult) XXX_DiscardUnknown() {
|
||||
xxx_messageInfo_QueryResult.DiscardUnknown(m)
|
||||
}
|
||||
|
||||
var xxx_messageInfo_QueryResult proto.InternalMessageInfo
|
||||
|
||||
func (m *QueryResult) GetStatus() *commonpb.Status {
|
||||
if m != nil {
|
||||
return m.Status
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (m *QueryResult) GetHits() []*Hits {
|
||||
if m != nil {
|
||||
return m.Hits
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func init() {
|
||||
proto.RegisterType((*CollectionName)(nil), "milvus.proto.service.CollectionName")
|
||||
proto.RegisterType((*PartitionName)(nil), "milvus.proto.service.PartitionName")
|
||||
proto.RegisterType((*RowBatch)(nil), "milvus.proto.service.RowBatch")
|
||||
proto.RegisterType((*PlaceholderValue)(nil), "milvus.proto.service.PlaceholderValue")
|
||||
proto.RegisterType((*Query)(nil), "milvus.proto.service.Query")
|
||||
proto.RegisterType((*StringResponse)(nil), "milvus.proto.service.StringResponse")
|
||||
proto.RegisterType((*BoolResponse)(nil), "milvus.proto.service.BoolResponse")
|
||||
proto.RegisterType((*StringListResponse)(nil), "milvus.proto.service.StringListResponse")
|
||||
proto.RegisterType((*IntegerListResponse)(nil), "milvus.proto.service.IntegerListResponse")
|
||||
proto.RegisterType((*IntegerRangeResponse)(nil), "milvus.proto.service.IntegerRangeResponse")
|
||||
proto.RegisterType((*CollectionDescription)(nil), "milvus.proto.service.CollectionDescription")
|
||||
proto.RegisterType((*PartitionDescription)(nil), "milvus.proto.service.PartitionDescription")
|
||||
proto.RegisterType((*Score)(nil), "milvus.proto.service.Score")
|
||||
proto.RegisterType((*Hits)(nil), "milvus.proto.service.Hits")
|
||||
proto.RegisterType((*QueryResult)(nil), "milvus.proto.service.QueryResult")
|
||||
}
|
||||
|
||||
func init() { proto.RegisterFile("service_msg.proto", fileDescriptor_b4b40b84dd2f74cb) }
|
||||
|
||||
var fileDescriptor_b4b40b84dd2f74cb = []byte{
|
||||
// 662 bytes of a gzipped FileDescriptorProto
|
||||
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x54, 0xcf, 0x6f, 0xd3, 0x30,
|
||||
0x14, 0x56, 0x9a, 0xb6, 0x6c, 0xaf, 0x5d, 0x19, 0xa1, 0xa0, 0xb2, 0x1d, 0x28, 0x99, 0x06, 0xbd,
|
||||
0xd0, 0x8a, 0x0d, 0x09, 0x71, 0xe0, 0xb0, 0x6e, 0x07, 0x18, 0x08, 0x0d, 0x17, 0x90, 0x80, 0x43,
|
||||
0xe5, 0x24, 0x56, 0x62, 0xc9, 0x89, 0x2b, 0xdb, 0xd9, 0xb4, 0x5d, 0xf9, 0x83, 0x38, 0xf1, 0x37,
|
||||
0x70, 0xe3, 0x6f, 0x42, 0xb1, 0xdd, 0x5f, 0x53, 0x40, 0x63, 0xeb, 0xcd, 0x7e, 0x79, 0xef, 0x7d,
|
||||
0xef, 0xfb, 0xf2, 0xfc, 0xc1, 0x1d, 0x49, 0xc4, 0x29, 0x0d, 0xc9, 0x38, 0x95, 0x71, 0x7f, 0x22,
|
||||
0xb8, 0xe2, 0x5e, 0x3b, 0xa5, 0xec, 0x34, 0x97, 0xe6, 0xd6, 0xb7, 0xdf, 0xb7, 0x9a, 0x21, 0x4f,
|
||||
0x53, 0x9e, 0x99, 0xe8, 0x56, 0x53, 0x86, 0x09, 0x49, 0xb1, 0xb9, 0xf9, 0x2f, 0xa1, 0x75, 0xc8,
|
||||
0x19, 0x23, 0xa1, 0xa2, 0x3c, 0x7b, 0x8f, 0x53, 0xe2, 0x3d, 0x81, 0xdb, 0xe1, 0x2c, 0x32, 0xce,
|
||||
0x70, 0x4a, 0x3a, 0x4e, 0xd7, 0xe9, 0xad, 0xa3, 0x56, 0xb8, 0x94, 0xe8, 0x1f, 0xc3, 0xc6, 0x09,
|
||||
0x16, 0x8a, 0xfe, 0x77, 0xa5, 0xb7, 0x09, 0xae, 0xc2, 0x71, 0xa7, 0xa2, 0x3f, 0x16, 0x47, 0xff,
|
||||
0x87, 0x03, 0x6b, 0x88, 0x9f, 0x0d, 0xb1, 0x0a, 0x93, 0xab, 0xf7, 0xd9, 0x81, 0x8d, 0xc9, 0x74,
|
||||
0x82, 0xf1, 0xbc, 0x63, 0x73, 0x16, 0xfc, 0x88, 0x63, 0xef, 0x39, 0xac, 0x09, 0x7e, 0x36, 0x8e,
|
||||
0xb0, 0xc2, 0x1d, 0xb7, 0xeb, 0xf6, 0x1a, 0x7b, 0x0f, 0xfa, 0x4b, 0x32, 0x59, 0x75, 0x86, 0x8c,
|
||||
0x07, 0xe8, 0x96, 0xe0, 0x67, 0x47, 0x58, 0x61, 0xef, 0x21, 0x34, 0x12, 0x2c, 0x93, 0xf1, 0x29,
|
||||
0x66, 0x39, 0x91, 0x9d, 0x6a, 0xd7, 0xed, 0xd5, 0x10, 0x14, 0xa1, 0xcf, 0x3a, 0xe2, 0x7f, 0x82,
|
||||
0xcd, 0x13, 0x86, 0x43, 0x92, 0x70, 0x16, 0x11, 0xa1, 0x83, 0x53, 0x5e, 0xce, 0x8c, 0x97, 0x37,
|
||||
0x80, 0x9a, 0xee, 0xa0, 0x27, 0xfb, 0x27, 0xb2, 0xc9, 0xf3, 0x7f, 0x3a, 0x50, 0xfb, 0x90, 0x13,
|
||||
0x71, 0x7e, 0x75, 0x15, 0x76, 0xa1, 0xb5, 0xa4, 0x82, 0xec, 0x54, 0xba, 0x6e, 0x6f, 0x1d, 0x6d,
|
||||
0x2c, 0xca, 0x20, 0x8b, 0xe1, 0x22, 0xc9, 0x3a, 0xae, 0x19, 0x2e, 0x92, 0xcc, 0x3b, 0x86, 0xe6,
|
||||
0x64, 0x4e, 0xc1, 0x90, 0x6c, 0xec, 0x3d, 0xee, 0x97, 0x2d, 0x51, 0xff, 0x32, 0x59, 0xb4, 0x54,
|
||||
0xeb, 0x7f, 0x83, 0xd6, 0x48, 0x09, 0x9a, 0xc5, 0x88, 0xc8, 0x09, 0xcf, 0x24, 0xf1, 0xf6, 0xa1,
|
||||
0x2e, 0x15, 0x56, 0xb9, 0xd4, 0x63, 0x37, 0xf6, 0xb6, 0x4b, 0xb9, 0x8f, 0x74, 0x0a, 0xb2, 0xa9,
|
||||
0x5e, 0x7b, 0x51, 0xaf, 0xf5, 0xa9, 0x28, 0x5f, 0xa0, 0x39, 0xe4, 0x9c, 0xad, 0xb0, 0xf5, 0xda,
|
||||
0xb4, 0x35, 0x06, 0xcf, 0xcc, 0xfd, 0x8e, 0x4a, 0x75, 0x33, 0x80, 0xfb, 0x50, 0xb7, 0xdb, 0x62,
|
||||
0xf4, 0xb7, 0x37, 0x3f, 0x80, 0xbb, 0x6f, 0x32, 0x45, 0x62, 0x22, 0x56, 0x8d, 0xe1, 0xce, 0x30,
|
||||
0x24, 0xb4, 0x2d, 0x06, 0xc2, 0x59, 0x4c, 0x6e, 0xac, 0x54, 0x40, 0x62, 0x9a, 0x59, 0x0c, 0x73,
|
||||
0x29, 0xf6, 0x87, 0x64, 0x91, 0x7e, 0x42, 0x2e, 0x2a, 0x8e, 0xfe, 0x6f, 0x07, 0xee, 0xcd, 0xcd,
|
||||
0xe3, 0x88, 0xc8, 0x50, 0xd0, 0x49, 0x71, 0xbc, 0x1e, 0xec, 0x2b, 0xa8, 0x1b, 0x6b, 0xb2, 0x8f,
|
||||
0x65, 0xf7, 0xd2, 0x22, 0x1a, 0xdb, 0x9a, 0x03, 0x8e, 0x74, 0x00, 0xd9, 0x22, 0xef, 0x00, 0xa0,
|
||||
0x68, 0x44, 0xa5, 0xa2, 0xa1, 0xb4, 0x2f, 0xfd, 0x51, 0x29, 0xee, 0x5b, 0x72, 0xae, 0x57, 0xf8,
|
||||
0x04, 0x53, 0x81, 0x16, 0x8a, 0xfc, 0x5f, 0x0e, 0xb4, 0x67, 0x96, 0x76, 0x63, 0x3e, 0x2f, 0xa0,
|
||||
0xaa, 0x5f, 0xad, 0x61, 0xb3, 0xf3, 0x97, 0x67, 0xb5, 0xe8, 0xa0, 0x48, 0x17, 0xac, 0x82, 0xc9,
|
||||
0x33, 0xa8, 0x8d, 0x42, 0x2e, 0xca, 0x2c, 0x69, 0x79, 0x85, 0x2a, 0xb3, 0x15, 0xfa, 0xee, 0x40,
|
||||
0xf5, 0x35, 0x55, 0xda, 0x28, 0x68, 0x54, 0x30, 0xd5, 0x3f, 0x9a, 0x46, 0x72, 0xc9, 0x42, 0x2b,
|
||||
0x57, 0xb6, 0xd0, 0x42, 0xb4, 0x62, 0x86, 0x29, 0x85, 0xed, 0x72, 0x05, 0xf4, 0x9c, 0xc8, 0xa6,
|
||||
0xfa, 0x02, 0x1a, 0xda, 0xfe, 0x10, 0x91, 0x39, 0x53, 0xd7, 0x13, 0xbe, 0x0f, 0xd5, 0x84, 0x2a,
|
||||
0x69, 0x47, 0xdd, 0x2a, 0x87, 0x2d, 0xa8, 0x22, 0x9d, 0x37, 0x3c, 0xfc, 0x7a, 0x10, 0x53, 0x95,
|
||||
0xe4, 0x41, 0xd1, 0x6f, 0x70, 0x41, 0x19, 0xa3, 0x17, 0x8a, 0x84, 0xc9, 0xc0, 0x14, 0x3e, 0x8d,
|
||||
0xa8, 0x54, 0x82, 0x06, 0xb9, 0x22, 0xd1, 0x80, 0x66, 0x8a, 0x88, 0x0c, 0xb3, 0x81, 0xee, 0x36,
|
||||
0xb0, 0xdd, 0x26, 0x41, 0x50, 0xd7, 0x81, 0xfd, 0x3f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xce, 0x67,
|
||||
0x71, 0xe7, 0x96, 0x07, 0x00, 0x00,
|
||||
}
|
|
@ -1,44 +1,45 @@
|
|||
package proxy
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
"github.com/golang/protobuf/proto"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
||||
"log"
|
||||
"sort"
|
||||
"sync"
|
||||
)
|
||||
|
||||
type queryReq struct {
|
||||
pb.QueryReqMsg
|
||||
result []*pb.QueryResult
|
||||
internalpb.SearchRequest
|
||||
result []*internalpb.SearchResult
|
||||
wg sync.WaitGroup
|
||||
proxy *proxyServer
|
||||
}
|
||||
|
||||
// BaseRequest interfaces
|
||||
func (req *queryReq) Type() pb.ReqType {
|
||||
func (req *queryReq) Type() internalpb.ReqType {
|
||||
return req.ReqType
|
||||
}
|
||||
|
||||
func (req *queryReq) PreExecute() pb.Status {
|
||||
return pb.Status{ErrorCode: pb.ErrorCode_SUCCESS}
|
||||
func (req *queryReq) PreExecute() commonpb.Status {
|
||||
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
|
||||
}
|
||||
|
||||
func (req *queryReq) Execute() pb.Status {
|
||||
func (req *queryReq) Execute() commonpb.Status {
|
||||
req.proxy.reqSch.queryChan <- req
|
||||
return pb.Status{ErrorCode: pb.ErrorCode_SUCCESS}
|
||||
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
|
||||
}
|
||||
|
||||
func (req *queryReq) PostExecute() pb.Status { // send into pulsar
|
||||
func (req *queryReq) PostExecute() commonpb.Status { // send into pulsar
|
||||
req.wg.Add(1)
|
||||
return pb.Status{ErrorCode: pb.ErrorCode_SUCCESS}
|
||||
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
|
||||
}
|
||||
|
||||
func (req *queryReq) WaitToFinish() pb.Status { // wait unitl send into pulsar
|
||||
func (req *queryReq) WaitToFinish() commonpb.Status { // wait unitl send into pulsar
|
||||
req.wg.Wait()
|
||||
return pb.Status{ErrorCode: pb.ErrorCode_SUCCESS}
|
||||
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
|
||||
}
|
||||
|
||||
func (s *proxyServer) restartQueryRoutine(buf_size int) error {
|
||||
|
@ -78,20 +79,10 @@ func (s *proxyServer) restartQueryRoutine(buf_size int) error {
|
|||
log.Printf("get time stamp failed, error code = %d, msg = %s", st.ErrorCode, st.Reason)
|
||||
break
|
||||
}
|
||||
qm.Timestamp = uint64(ts[0])
|
||||
|
||||
q := pb.QueryReqMsg{
|
||||
CollectionName: qm.CollectionName,
|
||||
VectorParam: qm.VectorParam,
|
||||
PartitionTags: qm.PartitionTags,
|
||||
Dsl: qm.Dsl,
|
||||
ExtraParams: qm.ExtraParams,
|
||||
Timestamp: uint64(ts[0]),
|
||||
ProxyId: qm.ProxyId,
|
||||
QueryId: qm.QueryId,
|
||||
ReqType: qm.ReqType,
|
||||
}
|
||||
|
||||
qb, err := proto.Marshal(&q)
|
||||
qb, err := proto.Marshal(qm)
|
||||
if err != nil {
|
||||
log.Printf("Marshal QueryReqMsg failed, error = %v", err)
|
||||
continue
|
||||
|
@ -106,14 +97,14 @@ func (s *proxyServer) restartQueryRoutine(buf_size int) error {
|
|||
log.Printf("there is some wrong with q_timestamp, it goes back, current = %d, previous = %d", ts[0], s.reqSch.q_timestamp)
|
||||
}
|
||||
s.reqSch.q_timestamp_mux.Unlock()
|
||||
resultMap[qm.QueryId] = qm
|
||||
resultMap[qm.ReqId] = qm
|
||||
//log.Printf("start search, query id = %d", qm.QueryId)
|
||||
case cm, ok := <-result.Chan():
|
||||
if !ok {
|
||||
log.Printf("consumer of result topic has closed")
|
||||
return
|
||||
}
|
||||
var rm pb.QueryResult
|
||||
var rm internalpb.SearchResult
|
||||
if err := proto.Unmarshal(cm.Message.Payload(), &rm); err != nil {
|
||||
log.Printf("Unmarshal QueryReqMsg failed, error = %v", err)
|
||||
break
|
||||
|
@ -121,15 +112,15 @@ func (s *proxyServer) restartQueryRoutine(buf_size int) error {
|
|||
if rm.ProxyId != s.proxyId {
|
||||
break
|
||||
}
|
||||
qm, ok := resultMap[rm.QueryId]
|
||||
qm, ok := resultMap[rm.ReqId]
|
||||
if !ok {
|
||||
log.Printf("unknown query id = %d", rm.QueryId)
|
||||
log.Printf("unknown query id = %d", rm.ReqId)
|
||||
break
|
||||
}
|
||||
qm.result = append(qm.result, &rm)
|
||||
if len(qm.result) == s.numReaderNode {
|
||||
qm.wg.Done()
|
||||
delete(resultMap, rm.QueryId)
|
||||
delete(resultMap, rm.ReqId)
|
||||
}
|
||||
result.AckID(cm.ID())
|
||||
}
|
||||
|
@ -139,114 +130,102 @@ func (s *proxyServer) restartQueryRoutine(buf_size int) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) reduceResult(query *queryReq) *pb.QueryResult {
|
||||
if s.numReaderNode == 1 {
|
||||
return query.result[0]
|
||||
}
|
||||
var result []*pb.QueryResult
|
||||
//func (s *proxyServer) reduceResult(query *queryReq) *servicepb.QueryResult {
|
||||
//}
|
||||
|
||||
func (s *proxyServer) reduceResults(query *queryReq) *servicepb.QueryResult {
|
||||
|
||||
var results []*internalpb.SearchResult
|
||||
var status commonpb.Status
|
||||
status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR
|
||||
for _, r := range query.result {
|
||||
if r.Status.ErrorCode == pb.ErrorCode_SUCCESS {
|
||||
result = append(result, r)
|
||||
status = *r.Status
|
||||
if status.ErrorCode == commonpb.ErrorCode_SUCCESS {
|
||||
results = append(results, r)
|
||||
}else{
|
||||
break
|
||||
}
|
||||
}
|
||||
if len(result) == 0 {
|
||||
return query.result[0]
|
||||
if len(results) != s.numReaderNode{
|
||||
status.ErrorCode = commonpb.ErrorCode_UNEXPECTED_ERROR
|
||||
}
|
||||
if len(result) == 1 {
|
||||
return result[0]
|
||||
if status.ErrorCode != commonpb.ErrorCode_SUCCESS{
|
||||
result:= servicepb.QueryResult{
|
||||
Status: &status,
|
||||
}
|
||||
return &result
|
||||
}
|
||||
|
||||
var entities []*struct {
|
||||
Ids int64
|
||||
ValidRow bool
|
||||
RowsData *pb.RowData
|
||||
Scores float32
|
||||
Distances float32
|
||||
}
|
||||
var rows int
|
||||
|
||||
result_err := func(msg string) *pb.QueryResult {
|
||||
return &pb.QueryResult{
|
||||
Status: &pb.Status{
|
||||
ErrorCode: pb.ErrorCode_UNEXPECTED_ERROR,
|
||||
Reason: msg,
|
||||
if s.numReaderNode == 1 {
|
||||
result:= servicepb.QueryResult{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
||||
},
|
||||
Hits: results[0].Hits,
|
||||
}
|
||||
return &result
|
||||
}
|
||||
|
||||
for _, r := range result {
|
||||
if len(r.Entities.Ids) > rows {
|
||||
rows = len(r.Entities.Ids)
|
||||
}
|
||||
if len(r.Entities.Ids) != len(r.Entities.ValidRow) {
|
||||
return result_err(fmt.Sprintf("len(Entities.Ids)=%d, len(Entities.ValidRow)=%d", len(r.Entities.Ids), len(r.Entities.ValidRow)))
|
||||
}
|
||||
if len(r.Entities.Ids) != len(r.Entities.RowsData) {
|
||||
return result_err(fmt.Sprintf("len(Entities.Ids)=%d, len(Entities.RowsData)=%d", len(r.Entities.Ids), len(r.Entities.RowsData)))
|
||||
}
|
||||
if len(r.Entities.Ids) != len(r.Scores) {
|
||||
return result_err(fmt.Sprintf("len(Entities.Ids)=%d, len(Scores)=%d", len(r.Entities.Ids), len(r.Scores)))
|
||||
}
|
||||
if len(r.Entities.Ids) != len(r.Distances) {
|
||||
return result_err(fmt.Sprintf("len(Entities.Ids)=%d, len(Distances)=%d", len(r.Entities.Ids), len(r.Distances)))
|
||||
}
|
||||
for i := 0; i < len(r.Entities.Ids); i++ {
|
||||
entity := struct {
|
||||
Ids int64
|
||||
ValidRow bool
|
||||
RowsData *pb.RowData
|
||||
Scores float32
|
||||
Distances float32
|
||||
}{
|
||||
Ids: r.Entities.Ids[i],
|
||||
ValidRow: r.Entities.ValidRow[i],
|
||||
RowsData: r.Entities.RowsData[i],
|
||||
Scores: r.Scores[i],
|
||||
Distances: r.Distances[i],
|
||||
}
|
||||
entities = append(entities, &entity)
|
||||
}
|
||||
}
|
||||
sort.Slice(entities, func(i, j int) bool {
|
||||
if entities[i].ValidRow == true {
|
||||
if entities[j].ValidRow == false {
|
||||
return true
|
||||
}
|
||||
return entities[i].Scores > entities[j].Scores
|
||||
} else {
|
||||
return false
|
||||
}
|
||||
})
|
||||
rIds := make([]int64, 0, rows)
|
||||
rValidRow := make([]bool, 0, rows)
|
||||
rRowsData := make([]*pb.RowData, 0, rows)
|
||||
rScores := make([]float32, 0, rows)
|
||||
rDistances := make([]float32, 0, rows)
|
||||
for i := 0; i < rows; i++ {
|
||||
rIds = append(rIds, entities[i].Ids)
|
||||
rValidRow = append(rValidRow, entities[i].ValidRow)
|
||||
rRowsData = append(rRowsData, entities[i].RowsData)
|
||||
rScores = append(rScores, entities[i].Scores)
|
||||
rDistances = append(rDistances, entities[i].Distances)
|
||||
}
|
||||
//var entities []*struct {
|
||||
// Idx int64
|
||||
// Score float32
|
||||
// Hit *servicepb.Hits
|
||||
//}
|
||||
//var rows int
|
||||
//
|
||||
//result_err := func(msg string) *pb.QueryResult {
|
||||
// return &pb.QueryResult{
|
||||
// Status: &pb.Status{
|
||||
// ErrorCode: pb.ErrorCode_UNEXPECTED_ERROR,
|
||||
// Reason: msg,
|
||||
// },
|
||||
// }
|
||||
//}
|
||||
|
||||
return &pb.QueryResult{
|
||||
Status: &pb.Status{
|
||||
ErrorCode: pb.ErrorCode_SUCCESS,
|
||||
//for _, r := range results {
|
||||
// for i := 0; i < len(r.Hits); i++ {
|
||||
// entity := struct {
|
||||
// Ids int64
|
||||
// ValidRow bool
|
||||
// RowsData *pb.RowData
|
||||
// Scores float32
|
||||
// Distances float32
|
||||
// }{
|
||||
// Ids: r.Entities.Ids[i],
|
||||
// ValidRow: r.Entities.ValidRow[i],
|
||||
// RowsData: r.Entities.RowsData[i],
|
||||
// Scores: r.Scores[i],
|
||||
// Distances: r.Distances[i],
|
||||
// }
|
||||
// entities = append(entities, &entity)
|
||||
// }
|
||||
//}
|
||||
//sort.Slice(entities, func(i, j int) bool {
|
||||
// if entities[i].ValidRow == true {
|
||||
// if entities[j].ValidRow == false {
|
||||
// return true
|
||||
// }
|
||||
// return entities[i].Scores > entities[j].Scores
|
||||
// } else {
|
||||
// return false
|
||||
// }
|
||||
//})
|
||||
//rIds := make([]int64, 0, rows)
|
||||
//rValidRow := make([]bool, 0, rows)
|
||||
//rRowsData := make([]*pb.RowData, 0, rows)
|
||||
//rScores := make([]float32, 0, rows)
|
||||
//rDistances := make([]float32, 0, rows)
|
||||
//for i := 0; i < rows; i++ {
|
||||
// rIds = append(rIds, entities[i].Ids)
|
||||
// rValidRow = append(rValidRow, entities[i].ValidRow)
|
||||
// rRowsData = append(rRowsData, entities[i].RowsData)
|
||||
// rScores = append(rScores, entities[i].Scores)
|
||||
// rDistances = append(rDistances, entities[i].Distances)
|
||||
//}
|
||||
|
||||
return &servicepb.QueryResult{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
||||
},
|
||||
Entities: &pb.Entities{
|
||||
Status: &pb.Status{
|
||||
ErrorCode: pb.ErrorCode_SUCCESS,
|
||||
},
|
||||
Ids: rIds,
|
||||
ValidRow: rValidRow,
|
||||
RowsData: rRowsData,
|
||||
},
|
||||
RowNum: int64(rows),
|
||||
Scores: rScores,
|
||||
Distances: rDistances,
|
||||
ExtraParams: result[0].ExtraParams,
|
||||
QueryId: query.QueryId,
|
||||
ProxyId: query.ProxyId,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -4,7 +4,12 @@ import (
|
|||
"context"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
mpb "github.com/zilliztech/milvus-distributed/internal/proto/master"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
mpb "github.com/zilliztech/milvus-distributed/internal/proto/masterpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/etcdpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/schemapb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/servicepb"
|
||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
"github.com/zilliztech/milvus-distributed/internal/master/collection"
|
||||
"github.com/golang/protobuf/proto"
|
||||
|
@ -25,7 +30,7 @@ const (
|
|||
)
|
||||
|
||||
type proxyServer struct {
|
||||
pb.UnimplementedMilvusServiceServer
|
||||
servicepb.UnimplementedMilvusServiceServer
|
||||
address string
|
||||
masterAddress string
|
||||
rootPath string // etcd root path
|
||||
|
@ -47,58 +52,99 @@ type proxyServer struct {
|
|||
grpcServer *grpc.Server
|
||||
reqSch *requestScheduler
|
||||
///////////////////////////////////////////////////////////////
|
||||
collectionList map[uint64]*mpb.Collection
|
||||
collectionList map[uint64]*etcdpb.CollectionMeta
|
||||
nameCollectionId map[string]uint64
|
||||
segmentList map[uint64]*mpb.Segment
|
||||
segmentList map[uint64]*etcdpb.SegmentMeta
|
||||
collectionMux sync.Mutex
|
||||
queryId atomic.Uint64
|
||||
}
|
||||
|
||||
func (s *proxyServer) CreateCollection(ctx context.Context, req *pb.Mapping) (*pb.Status, error) {
|
||||
log.Printf("create collection %s", req.CollectionName)
|
||||
return s.masterClient.CreateCollection(ctx, req)
|
||||
}
|
||||
|
||||
func (s *proxyServer) CountCollection(ctx context.Context, req *pb.CollectionName) (*pb.CollectionRowCount, error) {
|
||||
s.collectionMux.Lock()
|
||||
defer s.collectionMux.Unlock()
|
||||
|
||||
collection_id, ok := s.nameCollectionId[req.CollectionName]
|
||||
if !ok {
|
||||
return &pb.CollectionRowCount{
|
||||
CollectionRowCount: 0,
|
||||
Status: &pb.Status{
|
||||
ErrorCode: pb.ErrorCode_UNEXPECTED_ERROR,
|
||||
Reason: fmt.Sprintf("unable to get collection %s", req.CollectionName),
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
if info, ok := s.collectionList[collection_id]; ok {
|
||||
count := int64(0)
|
||||
for _, seg_id := range info.SegmentIds {
|
||||
if seg, ok := s.segmentList[seg_id]; ok {
|
||||
count += seg.Rows
|
||||
}
|
||||
}
|
||||
return &pb.CollectionRowCount{
|
||||
CollectionRowCount: count,
|
||||
Status: &pb.Status{
|
||||
ErrorCode: pb.ErrorCode_SUCCESS,
|
||||
},
|
||||
}, nil
|
||||
}
|
||||
return &pb.CollectionRowCount{
|
||||
CollectionRowCount: 0,
|
||||
Status: &pb.Status{
|
||||
ErrorCode: pb.ErrorCode_UNEXPECTED_ERROR,
|
||||
Reason: fmt.Sprintf("unable to get collection %s", req.CollectionName),
|
||||
},
|
||||
func (s *proxyServer) CreateCollection(ctx context.Context, req *schemapb.CollectionSchema) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) CreateIndex(ctx context.Context, req *pb.IndexParam) (*pb.Status, error) {
|
||||
log.Printf("create index, collection name = %s, index name = %s, filed_name = %s", req.CollectionName, req.IndexName, req.FieldName)
|
||||
return s.masterClient.CreateIndex(ctx, req)
|
||||
|
||||
func (s *proxyServer) DropCollection(ctx context.Context, req *servicepb.CollectionName) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) HasCollection(ctx context.Context, req *servicepb.CollectionName) (*servicepb.BoolResponse, error) {
|
||||
return &servicepb.BoolResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
Value: true,
|
||||
},nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) DescribeCollection(ctx context.Context, req *servicepb.CollectionName) (*servicepb.CollectionDescription, error) {
|
||||
return &servicepb.CollectionDescription{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
},nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) ShowCollections(ctx context.Context, req * commonpb.Empty) (*servicepb.StringListResponse, error) {
|
||||
return &servicepb.StringListResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
},nil
|
||||
}
|
||||
|
||||
|
||||
func (s *proxyServer) CreatePartition(ctx context.Context, in *servicepb.PartitionName) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) DropPartition(ctx context.Context, in *servicepb.PartitionName) (*commonpb.Status, error) {
|
||||
return &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) HasPartition(ctx context.Context, in *servicepb.PartitionName) (*servicepb.BoolResponse, error) {
|
||||
return &servicepb.BoolResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
Value: true,
|
||||
},nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) DescribePartition(ctx context.Context, in *servicepb.PartitionName) (*servicepb.PartitionDescription, error) {
|
||||
return &servicepb.PartitionDescription{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
},nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) ShowPartitions(ctx context.Context, req *servicepb.CollectionName) (*servicepb.StringListResponse, error) {
|
||||
return &servicepb.StringListResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: 0,
|
||||
Reason: "",
|
||||
},
|
||||
},nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) DeleteByID(ctx context.Context, req *pb.DeleteByIDParam) (*pb.Status, error) {
|
||||
|
@ -131,26 +177,26 @@ func (s *proxyServer) DeleteByID(ctx context.Context, req *pb.DeleteByIDParam) (
|
|||
return &pb.Status{ErrorCode: pb.ErrorCode_SUCCESS}, nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) Insert(ctx context.Context, req *pb.InsertParam) (*pb.EntityIds, error) {
|
||||
log.Printf("Insert Entities, total = %d", len(req.RowsData))
|
||||
|
||||
func (s *proxyServer) Insert(ctx context.Context, req *servicepb.RowBatch) (*servicepb.IntegerRangeResponse, error) {
|
||||
log.Printf("Insert Entities, total = %d", len(req.RowData))
|
||||
ipm := make(map[uint32]*manipulationReq)
|
||||
|
||||
//TODO
|
||||
if len(req.EntityIdArray) == 0 { //primary key is empty, set primary key by server
|
||||
//TODO check collection schema's auto_id
|
||||
if len(req.RowData) == 0 { //primary key is empty, set primary key by server
|
||||
log.Printf("Set primary key")
|
||||
}
|
||||
if len(req.EntityIdArray) != len(req.RowsData) {
|
||||
return &pb.EntityIds{
|
||||
Status: &pb.Status{
|
||||
ErrorCode: pb.ErrorCode_UNEXPECTED_ERROR,
|
||||
if len(req.HashValues) != len(req.RowData) {
|
||||
return &servicepb.IntegerRangeResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR,
|
||||
Reason: fmt.Sprintf("length of EntityIdArray not equal to lenght of RowsData"),
|
||||
},
|
||||
EntityIdArray: req.EntityIdArray,
|
||||
}, nil
|
||||
}
|
||||
|
||||
for i := 0; i < len(req.EntityIdArray); i++ {
|
||||
key := uint64(req.EntityIdArray[i])
|
||||
for i := 0; i < len(req.HashValues); i++ {
|
||||
key := uint64(req.HashValues[i])
|
||||
hash, err := Hash32_Uint64(key)
|
||||
if err != nil {
|
||||
return nil, status.Errorf(codes.Unknown, "hash failed on %d", key)
|
||||
|
@ -170,32 +216,29 @@ func (s *proxyServer) Insert(ctx context.Context, req *pb.InsertParam) (*pb.Enti
|
|||
ChannelId: uint64(hash),
|
||||
ReqType: pb.ReqType_kInsert,
|
||||
ProxyId: s.proxyId,
|
||||
ExtraParams: req.ExtraParams,
|
||||
//ExtraParams: req.ExtraParams,
|
||||
},
|
||||
proxy: s,
|
||||
}
|
||||
ip = ipm[hash]
|
||||
}
|
||||
ip.PrimaryKeys = append(ip.PrimaryKeys, key)
|
||||
ip.RowsData = append(ip.RowsData, req.RowsData[i])
|
||||
ip.RowsData = append(ip.RowsData, &pb.RowData{Blob:req.RowData[i].Value}) // czs_tag
|
||||
}
|
||||
for _, ip := range ipm {
|
||||
if st := ip.PreExecute(); st.ErrorCode != pb.ErrorCode_SUCCESS { //do nothing
|
||||
return &pb.EntityIds{
|
||||
Status: &st,
|
||||
EntityIdArray: req.EntityIdArray,
|
||||
return &servicepb.IntegerRangeResponse{
|
||||
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
|
||||
}, nil
|
||||
}
|
||||
if st := ip.Execute(); st.ErrorCode != pb.ErrorCode_SUCCESS { // push into chan
|
||||
return &pb.EntityIds{
|
||||
Status: &st,
|
||||
EntityIdArray: req.EntityIdArray,
|
||||
return &servicepb.IntegerRangeResponse{
|
||||
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
|
||||
}, nil
|
||||
}
|
||||
if st := ip.PostExecute(); st.ErrorCode != pb.ErrorCode_SUCCESS { //post to pulsar
|
||||
return &pb.EntityIds{
|
||||
Status: &st,
|
||||
EntityIdArray: req.EntityIdArray,
|
||||
return &servicepb.IntegerRangeResponse{
|
||||
Status: &commonpb.Status{ErrorCode: commonpb.ErrorCode_UNEXPECTED_ERROR},
|
||||
}, nil
|
||||
}
|
||||
}
|
||||
|
@ -205,58 +248,46 @@ func (s *proxyServer) Insert(ctx context.Context, req *pb.InsertParam) (*pb.Enti
|
|||
}
|
||||
}
|
||||
|
||||
return &pb.EntityIds{
|
||||
Status: &pb.Status{
|
||||
ErrorCode: pb.ErrorCode_SUCCESS,
|
||||
return &servicepb.IntegerRangeResponse{
|
||||
Status: &commonpb.Status{
|
||||
ErrorCode: commonpb.ErrorCode_SUCCESS,
|
||||
},
|
||||
EntityIdArray: req.EntityIdArray,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (s *proxyServer) Search(ctx context.Context, req *pb.SearchParam) (*pb.QueryResult, error) {
|
||||
func (s *proxyServer) Search(ctx context.Context, req *servicepb.Query) (*servicepb.QueryResult, error) {
|
||||
qm := &queryReq{
|
||||
QueryReqMsg: pb.QueryReqMsg{
|
||||
CollectionName: req.CollectionName,
|
||||
VectorParam: req.VectorParam,
|
||||
PartitionTags: req.PartitionTag,
|
||||
Dsl: req.Dsl,
|
||||
ExtraParams: req.ExtraParams,
|
||||
SearchRequest: internalpb.SearchRequest{
|
||||
ReqType: internalpb.ReqType_kSearch,
|
||||
ProxyId: s.proxyId,
|
||||
QueryId: s.queryId.Add(1),
|
||||
ReqType: pb.ReqType_kSearch,
|
||||
ReqId: s.queryId.Add(1),
|
||||
Timestamp: 0,
|
||||
ResultChannelId: 0,
|
||||
},
|
||||
proxy: s,
|
||||
}
|
||||
log.Printf("search on collection %s, proxy id = %d, query id = %d", req.CollectionName, qm.ProxyId, qm.QueryId)
|
||||
if st := qm.PreExecute(); st.ErrorCode != pb.ErrorCode_SUCCESS {
|
||||
return &pb.QueryResult{
|
||||
log.Printf("search on collection %s, proxy id = %d, query id = %d", req.CollectionName, qm.ProxyId, qm.ReqId)
|
||||
if st := qm.PreExecute(); st.ErrorCode != commonpb.ErrorCode_SUCCESS {
|
||||
return &servicepb.QueryResult{
|
||||
Status: &st,
|
||||
QueryId: qm.QueryId,
|
||||
ProxyId: qm.ProxyId,
|
||||
}, nil
|
||||
}
|
||||
if st := qm.Execute(); st.ErrorCode != pb.ErrorCode_SUCCESS {
|
||||
return &pb.QueryResult{
|
||||
if st := qm.Execute(); st.ErrorCode != commonpb.ErrorCode_SUCCESS {
|
||||
return &servicepb.QueryResult{
|
||||
Status: &st,
|
||||
QueryId: qm.QueryId,
|
||||
ProxyId: qm.ProxyId,
|
||||
}, nil
|
||||
}
|
||||
if st := qm.PostExecute(); st.ErrorCode != pb.ErrorCode_SUCCESS {
|
||||
return &pb.QueryResult{
|
||||
if st := qm.PostExecute(); st.ErrorCode != commonpb.ErrorCode_SUCCESS {
|
||||
return &servicepb.QueryResult{
|
||||
Status: &st,
|
||||
QueryId: qm.QueryId,
|
||||
ProxyId: qm.ProxyId,
|
||||
}, nil
|
||||
}
|
||||
if st := qm.WaitToFinish(); st.ErrorCode != pb.ErrorCode_SUCCESS {
|
||||
return &pb.QueryResult{
|
||||
if st := qm.WaitToFinish(); st.ErrorCode != commonpb.ErrorCode_SUCCESS {
|
||||
return &servicepb.QueryResult{
|
||||
Status: &st,
|
||||
QueryId: qm.QueryId,
|
||||
ProxyId: qm.ProxyId,
|
||||
}, nil
|
||||
}
|
||||
return s.reduceResult(qm), nil
|
||||
return s.reduceResults(qm), nil
|
||||
}
|
||||
|
||||
//check if proxySerer is set correct
|
||||
|
@ -319,15 +350,11 @@ func (s *proxyServer) getSegmentId(channelId int32, colName string) (uint64, err
|
|||
return 0, status.Errorf(codes.Unknown, "can't get collection, name = %s, id = %d", colName, colId)
|
||||
}
|
||||
for _, segId := range colInfo.SegmentIds {
|
||||
seg, ok := s.segmentList[segId]
|
||||
_, ok := s.segmentList[segId]
|
||||
if !ok {
|
||||
return 0, status.Errorf(codes.Unknown, "can't get segment of %d", segId)
|
||||
}
|
||||
if seg.Status == mpb.SegmentStatus_OPENED {
|
||||
if seg.ChannelStart <= channelId && channelId < seg.ChannelEnd {
|
||||
return segId, nil
|
||||
}
|
||||
}
|
||||
return segId, nil
|
||||
}
|
||||
return 0, status.Errorf(codes.Unknown, "can't get segment id, channel id = %d", channelId)
|
||||
}
|
||||
|
@ -360,7 +387,7 @@ func (s *proxyServer) StartGrpcServer() error {
|
|||
s.wg.Add(1)
|
||||
defer s.wg.Done()
|
||||
server := grpc.NewServer()
|
||||
pb.RegisterMilvusServiceServer(server, s)
|
||||
servicepb.RegisterMilvusServiceServer(server, s)
|
||||
err := server.Serve(lis)
|
||||
if err != nil {
|
||||
log.Fatalf("Proxy grpc server fatal error=%v", err)
|
||||
|
@ -379,22 +406,22 @@ func (s *proxyServer) WatchEtcd() error {
|
|||
}
|
||||
for _, cob := range cos.Kvs {
|
||||
// TODO: simplify collection struct
|
||||
var co mpb.Collection
|
||||
var co etcdpb.CollectionMeta
|
||||
var mco collection.Collection
|
||||
if err := json.Unmarshal(cob.Value, &mco); err != nil {
|
||||
return err
|
||||
}
|
||||
proto.UnmarshalText(mco.GrpcMarshalString, &co)
|
||||
s.nameCollectionId[co.Name] = co.Id
|
||||
s.nameCollectionId[co.Schema.Name] = co.Id
|
||||
s.collectionList[co.Id] = &co
|
||||
log.Printf("watch collection, name = %s, id = %d", co.Name, co.Id)
|
||||
log.Printf("watch collection, name = %s, id = %d", co.Schema.Name, co.Id)
|
||||
}
|
||||
segs, err := s.client.Get(s.ctx, s.rootPath+"/"+keySegmentPath, etcd.WithPrefix())
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
for _, segb := range segs.Kvs {
|
||||
var seg mpb.Segment
|
||||
var seg etcdpb.SegmentMeta
|
||||
if err := json.Unmarshal(segb.Value, &seg); err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -416,15 +443,15 @@ func (s *proxyServer) WatchEtcd() error {
|
|||
s.collectionMux.Lock()
|
||||
defer s.collectionMux.Unlock()
|
||||
for _, e := range coe.Events {
|
||||
var co mpb.Collection
|
||||
var co etcdpb.CollectionMeta
|
||||
var mco collection.Collection
|
||||
if err := json.Unmarshal(e.Kv.Value, &mco); err != nil {
|
||||
log.Printf("unmarshal Collection failed, error = %v", err)
|
||||
} else {
|
||||
proto.UnmarshalText(mco.GrpcMarshalString, &co)
|
||||
s.nameCollectionId[co.Name] = co.Id
|
||||
s.nameCollectionId[co.Schema.Name] = co.Id
|
||||
s.collectionList[co.Id] = &co
|
||||
log.Printf("watch collection, name = %s, id = %d", co.Name, co.Id)
|
||||
log.Printf("watch collection, name = %s, id = %d", co.Schema.Name, co.Id)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
@ -433,7 +460,7 @@ func (s *proxyServer) WatchEtcd() error {
|
|||
s.collectionMux.Lock()
|
||||
defer s.collectionMux.Unlock()
|
||||
for _, e := range sege.Events {
|
||||
var seg mpb.Segment
|
||||
var seg etcdpb.SegmentMeta
|
||||
if err := json.Unmarshal(e.Kv.Value, &seg); err != nil {
|
||||
log.Printf("unmarshal Segment failed, error = %v", err)
|
||||
} else {
|
||||
|
@ -462,8 +489,8 @@ func startProxyServer(srv *proxyServer) error {
|
|||
}
|
||||
|
||||
srv.nameCollectionId = make(map[string]uint64)
|
||||
srv.collectionList = make(map[uint64]*mpb.Collection)
|
||||
srv.segmentList = make(map[uint64]*mpb.Segment)
|
||||
srv.collectionList = make(map[uint64]*etcdpb.CollectionMeta)
|
||||
srv.segmentList = make(map[uint64]*etcdpb.SegmentMeta)
|
||||
|
||||
if err := srv.connectMaster(); err != nil {
|
||||
return err
|
||||
|
|
|
@ -13,29 +13,30 @@ package reader
|
|||
*/
|
||||
import "C"
|
||||
import (
|
||||
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
)
|
||||
|
||||
type IndexConfig struct{}
|
||||
|
||||
func (s *Segment) BuildIndex(collection *Collection) msgPb.Status {
|
||||
func (s *Segment) buildIndex(collection* Collection) commonpb.Status {
|
||||
/*
|
||||
int
|
||||
BuildIndex(CCollection c_collection, CSegmentBase c_segment);
|
||||
int
|
||||
BuildIndex(CCollection c_collection, CSegmentBase c_segment);
|
||||
*/
|
||||
var status = C.BuildIndex(collection.CollectionPtr, s.SegmentPtr)
|
||||
if status != 0 {
|
||||
return msgPb.Status{ErrorCode: msgPb.ErrorCode_BUILD_INDEX_ERROR}
|
||||
return commonpb.Status{ErrorCode: commonpb.ErrorCode_BUILD_INDEX_ERROR}
|
||||
}
|
||||
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
|
||||
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
|
||||
}
|
||||
|
||||
func (s *Segment) DropIndex(fieldName string) msgPb.Status {
|
||||
func (s *Segment) dropIndex(fieldName string) commonpb.Status {
|
||||
// WARN: Not support yet
|
||||
|
||||
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
|
||||
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
|
||||
}
|
||||
|
||||
|
||||
func (node *QueryNode) UpdateIndexes(collection *Collection, indexConfig *string) {
|
||||
/*
|
||||
void
|
||||
|
|
|
@ -2,8 +2,9 @@ package reader
|
|||
|
||||
import (
|
||||
"context"
|
||||
masterPb "github.com/zilliztech/milvus-distributed/internal/proto/master"
|
||||
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
msgpb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
)
|
||||
|
||||
type ResultEntityIds []int64
|
||||
|
@ -13,32 +14,28 @@ type SearchResult struct {
|
|||
ResultDistances []float32
|
||||
}
|
||||
|
||||
func (node *QueryNode) PublishSearchResult(results *msgPb.QueryResult) msgPb.Status {
|
||||
func (node *QueryNode) PublishSearchResult(results *msgpb.QueryResult) commonpb.Status {
|
||||
var ctx = context.Background()
|
||||
|
||||
node.messageClient.SendResult(ctx, *results, results.ProxyId)
|
||||
|
||||
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
|
||||
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
|
||||
}
|
||||
|
||||
func (node *QueryNode) PublishFailedSearchResult() msgPb.Status {
|
||||
var results = msgPb.QueryResult{
|
||||
Status: &msgPb.Status{
|
||||
ErrorCode: 1,
|
||||
Reason: "Search Failed",
|
||||
},
|
||||
func (node *QueryNode) PublishFailedSearchResult() commonpb.Status {
|
||||
var results = msgpb.QueryResult{
|
||||
}
|
||||
|
||||
var ctx = context.Background()
|
||||
|
||||
node.messageClient.SendResult(ctx, results, results.ProxyId)
|
||||
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
|
||||
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
|
||||
}
|
||||
|
||||
func (node *QueryNode) PublicStatistic(statisticData *[]masterPb.SegmentStat) msgPb.Status {
|
||||
func (node *QueryNode) PublicStatistic(statisticData *[]internalpb.SegmentStatistics) commonpb.Status {
|
||||
var ctx = context.Background()
|
||||
|
||||
node.messageClient.SendSegmentsStatistic(ctx, statisticData)
|
||||
|
||||
return msgPb.Status{ErrorCode: msgPb.ErrorCode_SUCCESS}
|
||||
return commonpb.Status{ErrorCode: commonpb.ErrorCode_SUCCESS}
|
||||
}
|
||||
|
|
|
@ -2,8 +2,8 @@ package reader
|
|||
|
||||
import (
|
||||
"fmt"
|
||||
masterPb "github.com/zilliztech/milvus-distributed/internal/proto/master"
|
||||
msgPb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/commonpb"
|
||||
"log"
|
||||
"strconv"
|
||||
"time"
|
||||
|
@ -46,23 +46,19 @@ import (
|
|||
//}
|
||||
|
||||
func (node *QueryNode) SegmentStatistic(sleepMillisecondTime int) {
|
||||
var statisticData = make([]masterPb.SegmentStat, 0)
|
||||
var statisticData = make([]internalpb.SegmentStatistics, 0)
|
||||
|
||||
for segmentID, segment := range node.SegmentsMap {
|
||||
currentMemSize := segment.GetMemSize()
|
||||
memIncreaseRate := float32((int64(currentMemSize))-(int64(segment.LastMemSize))) / (float32(sleepMillisecondTime) / 1000)
|
||||
segment.LastMemSize = currentMemSize
|
||||
|
||||
segmentStatus := segment.SegmentStatus
|
||||
segmentNumOfRows := segment.GetRowCount()
|
||||
|
||||
stat := masterPb.SegmentStat{
|
||||
stat := internalpb.SegmentStatistics{
|
||||
// TODO: set master pb's segment id type from uint64 to int64
|
||||
SegmentId: uint64(segmentID),
|
||||
MemorySize: currentMemSize,
|
||||
MemoryRate: memIncreaseRate,
|
||||
Status: masterPb.SegmentStatus(segmentStatus),
|
||||
Rows: segmentNumOfRows,
|
||||
NumRows: segmentNumOfRows,
|
||||
}
|
||||
|
||||
statisticData = append(statisticData, stat)
|
||||
|
@ -71,7 +67,7 @@ func (node *QueryNode) SegmentStatistic(sleepMillisecondTime int) {
|
|||
// fmt.Println("Publish segment statistic")
|
||||
// fmt.Println(statisticData)
|
||||
var status = node.PublicStatistic(&statisticData)
|
||||
if status.ErrorCode != msgPb.ErrorCode_SUCCESS {
|
||||
if status.ErrorCode != commonpb.ErrorCode_SUCCESS {
|
||||
log.Printf("Publish segments statistic failed")
|
||||
}
|
||||
}
|
||||
|
|
|
@ -13,6 +13,7 @@ import (
|
|||
|
||||
"github.com/apache/pulsar-client-go/pulsar"
|
||||
"github.com/zilliztech/milvus-distributed/internal/conf"
|
||||
"github.com/zilliztech/milvus-distributed/internal/proto/internalpb"
|
||||
pb "github.com/zilliztech/milvus-distributed/internal/proto/message"
|
||||
"github.com/golang/protobuf/proto"
|
||||
)
|
||||
|
@ -77,7 +78,7 @@ layout of timestamp
|
|||
/-------46 bit-----------\/------18bit-----\
|
||||
+-------------------------+================+
|
||||
*/
|
||||
func toMillisecond(ts *pb.TimeSyncMsg) int {
|
||||
func toMillisecond(ts *internalpb.TimeSyncMsg) int {
|
||||
// get Millisecond in second
|
||||
return int(ts.GetTimestamp() >> 18)
|
||||
}
|
||||
|
@ -225,7 +226,7 @@ func (r *TimeSyncCfg) IsInsertDeleteChanFull() bool {
|
|||
return len(r.insertOrDeleteChan) == len(r.readerProducer)*r.readerQueueSize
|
||||
}
|
||||
|
||||
func (r *TimeSyncCfg) alignTimeSync(ts []*pb.TimeSyncMsg) []*pb.TimeSyncMsg {
|
||||
func (r *TimeSyncCfg) alignTimeSync(ts []*internalpb.TimeSyncMsg) []*internalpb.TimeSyncMsg {
|
||||
if len(r.proxyIdList) > 1 {
|
||||
if len(ts) > 1 {
|
||||
for i := 1; i < len(r.proxyIdList); i++ {
|
||||
|
@ -238,9 +239,9 @@ func (r *TimeSyncCfg) alignTimeSync(ts []*pb.TimeSyncMsg) []*pb.TimeSyncMsg {
|
|||
}
|
||||
}
|
||||
ts = ts[len(ts)-len(r.proxyIdList):]
|
||||
sort.Slice(ts, func(i int, j int) bool { return ts[i].Peer_Id < ts[j].Peer_Id })
|
||||
sort.Slice(ts, func(i int, j int) bool { return ts[i].PeerId < ts[j].PeerId })
|
||||
for i := 0; i < len(r.proxyIdList); i++ {
|
||||
if ts[i].Peer_Id != r.proxyIdList[i] {
|
||||
if ts[i].PeerId != r.proxyIdList[i] {
|
||||
ts = ts[:0]
|
||||
return ts
|
||||
}
|
||||
|
@ -254,7 +255,7 @@ func (r *TimeSyncCfg) alignTimeSync(ts []*pb.TimeSyncMsg) []*pb.TimeSyncMsg {
|
|||
return ts
|
||||
}
|
||||
|
||||
func (r *TimeSyncCfg) readTimeSync(ctx context.Context, ts []*pb.TimeSyncMsg, n int) ([]*pb.TimeSyncMsg, error) {
|
||||
func (r *TimeSyncCfg) readTimeSync(ctx context.Context, ts []*internalpb.TimeSyncMsg, n int) ([]*internalpb.TimeSyncMsg, error) {
|
||||
for i := 0; i < n; i++ {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
|
@ -265,7 +266,7 @@ func (r *TimeSyncCfg) readTimeSync(ctx context.Context, ts []*pb.TimeSyncMsg, n
|
|||
}
|
||||
|
||||
msg := cm.Message
|
||||
var tsm pb.TimeSyncMsg
|
||||
var tsm internalpb.TimeSyncMsg
|
||||
if err := proto.Unmarshal(msg.Payload(), &tsm); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -287,7 +288,7 @@ func (r *TimeSyncCfg) sendEOFMsg(ctx context.Context, msg *pulsar.ProducerMessag
|
|||
|
||||
func (r *TimeSyncCfg) startTimeSync() {
|
||||
ctx := r.ctx
|
||||
tsm := make([]*pb.TimeSyncMsg, 0, len(r.proxyIdList)*2)
|
||||
tsm := make([]*internalpb.TimeSyncMsg, 0, len(r.proxyIdList)*2)
|
||||
var err error
|
||||
for {
|
||||
select {
|
||||
|
|
|
@ -1,31 +1,6 @@
|
|||
#!/usr/bin/env bash
|
||||
SCRIPTS_DIR=$(dirname "$0")
|
||||
|
||||
while getopts "p:h" arg; do
|
||||
case $arg in
|
||||
p)
|
||||
protoc=$(readlink -f "${OPTARG}")
|
||||
;;
|
||||
h) # help
|
||||
echo "
|
||||
|
||||
parameter:
|
||||
-p: protoc path default("protoc")
|
||||
-h: help
|
||||
|
||||
usage:
|
||||
./build.sh -p protoc [-h]
|
||||
"
|
||||
exit 0
|
||||
;;
|
||||
?)
|
||||
echo "ERROR! unknown argument"
|
||||
exit 1
|
||||
;;
|
||||
esac
|
||||
done
|
||||
|
||||
|
||||
PROTO_DIR=$SCRIPTS_DIR/../internal/proto/
|
||||
|
||||
PROGRAM=$(basename "$0")
|
||||
|
@ -39,27 +14,22 @@ fi
|
|||
export PATH=${GOPATH}/bin:$PATH
|
||||
echo `which protoc-gen-go`
|
||||
|
||||
|
||||
# Although eraftpb.proto is copying from raft-rs, however there is no
|
||||
# official go code ship with the crate, so we need to generate it manually.
|
||||
pushd ${PROTO_DIR}
|
||||
|
||||
PB_FILES=("message.proto" "master.proto")
|
||||
mkdir -p commonpb
|
||||
mkdir -p schemapb
|
||||
mkdir -p etcdpb
|
||||
mkdir -p internalpb
|
||||
mkdir -p servicepb
|
||||
mkdir -p masterpb
|
||||
|
||||
ret=0
|
||||
|
||||
function gen_pb() {
|
||||
base_name=$(basename $1 ".proto")
|
||||
mkdir -p ./$base_name
|
||||
${protoc} --go_out=plugins=grpc,paths=source_relative:./$base_name $1 || ret=$?
|
||||
}
|
||||
|
||||
for file in ${PB_FILES[@]}
|
||||
do
|
||||
echo $file
|
||||
gen_pb $file
|
||||
done
|
||||
${protoc} --go_out=plugins=grpc,paths=source_relative:./commonpb common.proto
|
||||
${protoc} --go_out=plugins=grpc,paths=source_relative:./schemapb schema.proto
|
||||
${protoc} --go_out=plugins=grpc,paths=source_relative:./etcdpb etcd_meta.proto
|
||||
${protoc} --go_out=plugins=grpc,paths=source_relative:./internalpb internal_msg.proto
|
||||
${protoc} --go_out=plugins=grpc,paths=source_relative:./servicepb service_msg.proto
|
||||
${protoc} --go_out=plugins=grpc,paths=source_relative:./servicepb service.proto
|
||||
${protoc} --go_out=plugins=grpc,paths=source_relative:./masterpb master.proto
|
||||
|
||||
popd
|
||||
|
||||
exit $ret
|
||||
|
|
Loading…
Reference in New Issue