diff --git a/internal/core/run_clang_format.sh b/internal/core/run_clang_format.sh new file mode 100755 index 0000000000..76bdf880ba --- /dev/null +++ b/internal/core/run_clang_format.sh @@ -0,0 +1,15 @@ +if [ -z $1 ]; then + echo "usage: $0 " + 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" + diff --git a/internal/core/src/dog_segment/CMakeLists.txt b/internal/core/src/dog_segment/CMakeLists.txt index 632fc54e4c..6ed48a73cd 100644 --- a/internal/core/src/dog_segment/CMakeLists.txt +++ b/internal/core/src/dog_segment/CMakeLists.txt @@ -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 diff --git a/internal/core/src/dog_segment/Collection.cpp b/internal/core/src/dog_segment/Collection.cpp index c5b96a69d2..8aa3b2148f 100644 --- a/internal/core/src/dog_segment/Collection.cpp +++ b/internal/core/src/dog_segment/Collection.cpp @@ -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 #include @@ -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(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(); - 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); diff --git a/internal/core/src/dog_segment/DeletedRecord.h b/internal/core/src/dog_segment/DeletedRecord.h index e35098449d..e802cdf31b 100644 --- a/internal/core/src/dog_segment/DeletedRecord.h +++ b/internal/core/src/dog_segment/DeletedRecord.h @@ -50,7 +50,7 @@ struct DeletedRecord { std::shared_mutex shared_mutex_; }; -auto +inline auto DeletedRecord::TmpBitmap::clone(int64_t capacity) -> std::shared_ptr { auto res = std::make_shared(); res->del_barrier = this->del_barrier; diff --git a/internal/core/src/dog_segment/SegmentBase.cpp b/internal/core/src/dog_segment/SegmentBase.cpp new file mode 100644 index 0000000000..8fc0eedd34 --- /dev/null +++ b/internal/core/src/dog_segment/SegmentBase.cpp @@ -0,0 +1,41 @@ +#include "SegmentNaive.h" +#include "SegmentSmallIndex.h" + +namespace milvus::dog_segment { + +// seems to be deprecated +struct ColumnBasedDataChunk { + std::vector> entity_vecs; + + static ColumnBasedDataChunk + from(const DogDataChunk& source, const Schema& schema) { + ColumnBasedDataChunk dest; + auto count = source.count; + auto raw_data = reinterpret_cast(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 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 +CreateSegment(SchemaPtr schema) { + auto segment = std::make_unique(schema); + return segment; +} +} // namespace milvus::dog_segment diff --git a/internal/core/src/dog_segment/SegmentNaive.cpp b/internal/core/src/dog_segment/SegmentNaive.cpp index a30b83de3d..698650a519 100644 --- a/internal/core/src/dog_segment/SegmentNaive.cpp +++ b/internal/core/src/dog_segment/SegmentNaive.cpp @@ -10,17 +10,6 @@ #include namespace milvus::dog_segment { -int -TestABI() { - return 42; -} - -std::unique_ptr -CreateSegment(SchemaPtr schema) { - auto segment = std::make_unique(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, diff --git a/internal/core/src/dog_segment/SegmentNaive.h b/internal/core/src/dog_segment/SegmentNaive.h index bce984b3fa..93e3ad9c87 100644 --- a/internal/core/src/dog_segment/SegmentNaive.h +++ b/internal/core/src/dog_segment/SegmentNaive.h @@ -17,30 +17,6 @@ #include "EasyAssert.h" namespace milvus::dog_segment { -struct ColumnBasedDataChunk { - std::vector> entity_vecs; - - static ColumnBasedDataChunk - from(const DogDataChunk& source, const Schema& schema) { - ColumnBasedDataChunk dest; - auto count = source.count; - auto raw_data = reinterpret_cast(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 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; diff --git a/internal/core/src/dog_segment/SegmentSmallIndex.cpp b/internal/core/src/dog_segment/SegmentSmallIndex.cpp new file mode 100644 index 0000000000..04c7774ee0 --- /dev/null +++ b/internal/core/src/dog_segment/SegmentSmallIndex.cpp @@ -0,0 +1,622 @@ +#include +#include +#include +#include +#include +#include + +#include +#include +#include + +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>(field.get_dim())); + } else { + Assert(field.get_data_type() == DataType::INT32); + entity_vec_.emplace_back(std::make_shared>()); + } + } +} + +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 { + 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(entities_raw.raw_data); + // std::vector entities(raw_data, raw_data + size * len_per_row); + + auto len_per_row = entities_raw.sizeof_per_row; + std::vector> 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 offset_infos(schema_->size() + 1, 0); + std::partial_sum(sizeof_infos.begin(), sizeof_infos.end(), offset_infos.begin() + 1); + std::vector> entities(schema_->size()); + + for (int fid = 0; fid < schema_->size(); ++fid) { + auto len = sizeof_infos[fid]; + entities[fid].resize(len * size); + } + + std::vector uids(size); + std::vector 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> 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 uids(size); + std::vector 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 +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>(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(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(knowhere::meta::IDS); + auto distances = final->Get(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 buf_dis(topk); + std::vector 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>(record_.entity_vec_.at(the_offset_opt.value())); + + std::vector final_uids(total_count); + std::vector final_dis(total_count, std::numeric_limits::max()); + + auto max_chunk = (ins_barrier + DefaultElementPerChunk - 1) / DefaultElementPerChunk; + for (int chunk_id = 0; chunk_id < max_chunk; ++chunk_id) { + std::vector buf_uids(total_count, -1); + std::vector buf_dis(total_count, std::numeric_limits::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>(record_.entity_vec_.at(the_offset_opt.value())); + std::vector>> 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_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 +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(offset); + + std::vector 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(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(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(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 diff --git a/internal/core/src/dog_segment/SegmentSmallIndex.h b/internal/core/src/dog_segment/SegmentSmallIndex.h new file mode 100644 index 0000000000..cfa0a022e8 --- /dev/null +++ b/internal/core/src/dog_segment/SegmentSmallIndex.h @@ -0,0 +1,179 @@ +#pragma once + +#include +#include +#include + +#include +#include + +#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> entity_vecs; +// +// static ColumnBasedDataChunk +// from(const DogDataChunk& source, const Schema& schema) { +// ColumnBasedDataChunk dest; +// auto count = source.count; +// auto raw_data = reinterpret_cast(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 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 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 + CreateSegment(SchemaPtr schema); + + explicit SegmentSmallIndex(SchemaPtr schema) : schema_(schema), record_(*schema) { + } + + private: + // struct MutableRecord { + // ConcurrentVector uids_; + // tbb::concurrent_vector timestamps_; + // std::vector> entity_vecs_; + // + // MutableRecord(int entity_size) : entity_vecs_(entity_size) { + // } + // }; + + struct Record { + std::atomic reserved = 0; + AckResponder ack_responder_; + ConcurrentVector timestamps_; + ConcurrentVector uids_; + std::vector> entity_vec_; + + Record(const Schema& schema); + + template + auto + get_vec_entity(int offset) { + return std::static_pointer_cast>(entity_vec_[offset]); + } + }; + + std::shared_ptr + 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 + knowhere::IndexPtr + BuildVecIndexImpl(const IndexMeta::Entry& entry); + + private: + SchemaPtr schema_; + std::atomic state_ = SegmentState::Open; + Record record_; + DeletedRecord deleted_record_; + + std::atomic index_ready_ = false; + IndexMetaPtr index_meta_; + std::unordered_map indexings_; // index_name => indexing + tbb::concurrent_unordered_multimap uid2offset_; +}; +} // namespace milvus::dog_segment diff --git a/internal/core/src/pb/common.pb.cc b/internal/core/src/pb/common.pb.cc new file mode 100644 index 0000000000..5c53a82975 --- /dev/null +++ b/internal/core/src/pb/common.pb.cc @@ -0,0 +1,1692 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: common.proto + +#include "common.pb.h" + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +// @@protoc_insertion_point(includes) +#include +namespace milvus { +namespace proto { +namespace common { +class EmptyDefaultTypeInternal { + public: + ::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed _instance; +} _Empty_default_instance_; +class StatusDefaultTypeInternal { + public: + ::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed _instance; +} _Status_default_instance_; +class KeyValuePairDefaultTypeInternal { + public: + ::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed _instance; +} _KeyValuePair_default_instance_; +class BlobDefaultTypeInternal { + public: + ::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed _instance; +} _Blob_default_instance_; +class AddressDefaultTypeInternal { + public: + ::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed
_instance; +} _Address_default_instance_; +} // namespace common +} // namespace proto +} // namespace milvus +static void InitDefaultsscc_info_Address_common_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::milvus::proto::common::_Address_default_instance_; + new (ptr) ::milvus::proto::common::Address(); + ::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr); + } + ::milvus::proto::common::Address::InitAsDefaultInstance(); +} + +::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<0> scc_info_Address_common_2eproto = + {{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsscc_info_Address_common_2eproto}, {}}; + +static void InitDefaultsscc_info_Blob_common_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::milvus::proto::common::_Blob_default_instance_; + new (ptr) ::milvus::proto::common::Blob(); + ::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr); + } + ::milvus::proto::common::Blob::InitAsDefaultInstance(); +} + +::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<0> scc_info_Blob_common_2eproto = + {{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsscc_info_Blob_common_2eproto}, {}}; + +static void InitDefaultsscc_info_Empty_common_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::milvus::proto::common::_Empty_default_instance_; + new (ptr) ::milvus::proto::common::Empty(); + ::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr); + } + ::milvus::proto::common::Empty::InitAsDefaultInstance(); +} + +::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<0> scc_info_Empty_common_2eproto = + {{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsscc_info_Empty_common_2eproto}, {}}; + +static void InitDefaultsscc_info_KeyValuePair_common_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::milvus::proto::common::_KeyValuePair_default_instance_; + new (ptr) ::milvus::proto::common::KeyValuePair(); + ::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr); + } + ::milvus::proto::common::KeyValuePair::InitAsDefaultInstance(); +} + +::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<0> scc_info_KeyValuePair_common_2eproto = + {{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsscc_info_KeyValuePair_common_2eproto}, {}}; + +static void InitDefaultsscc_info_Status_common_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::milvus::proto::common::_Status_default_instance_; + new (ptr) ::milvus::proto::common::Status(); + ::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr); + } + ::milvus::proto::common::Status::InitAsDefaultInstance(); +} + +::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<0> scc_info_Status_common_2eproto = + {{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsscc_info_Status_common_2eproto}, {}}; + +static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_common_2eproto[5]; +static const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* file_level_enum_descriptors_common_2eproto[1]; +static constexpr ::PROTOBUF_NAMESPACE_ID::ServiceDescriptor const** file_level_service_descriptors_common_2eproto = nullptr; + +const ::PROTOBUF_NAMESPACE_ID::uint32 TableStruct_common_2eproto::offsets[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::Empty, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::Status, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::Status, error_code_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::Status, reason_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::KeyValuePair, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::KeyValuePair, key_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::KeyValuePair, value_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::Blob, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::Blob, value_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::Address, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::Address, ip_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::common::Address, port_), +}; +static const ::PROTOBUF_NAMESPACE_ID::internal::MigrationSchema schemas[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { + { 0, -1, sizeof(::milvus::proto::common::Empty)}, + { 5, -1, sizeof(::milvus::proto::common::Status)}, + { 12, -1, sizeof(::milvus::proto::common::KeyValuePair)}, + { 19, -1, sizeof(::milvus::proto::common::Blob)}, + { 25, -1, sizeof(::milvus::proto::common::Address)}, +}; + +static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = { + reinterpret_cast(&::milvus::proto::common::_Empty_default_instance_), + reinterpret_cast(&::milvus::proto::common::_Status_default_instance_), + reinterpret_cast(&::milvus::proto::common::_KeyValuePair_default_instance_), + reinterpret_cast(&::milvus::proto::common::_Blob_default_instance_), + reinterpret_cast(&::milvus::proto::common::_Address_default_instance_), +}; + +const char descriptor_table_protodef_common_2eproto[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = + "\n\014common.proto\022\023milvus.proto.common\"\007\n\005E" + "mpty\"L\n\006Status\0222\n\nerror_code\030\001 \001(\0162\036.mil" + "vus.proto.common.ErrorCode\022\016\n\006reason\030\002 \001" + "(\t\"*\n\014KeyValuePair\022\013\n\003key\030\001 \001(\t\022\r\n\005value" + "\030\002 \001(\t\"\025\n\004Blob\022\r\n\005value\030\001 \001(\014\"#\n\007Address" + "\022\n\n\002ip\030\001 \001(\t\022\014\n\004port\030\002 \001(\003*\242\004\n\tErrorCode" + "\022\013\n\007SUCCESS\020\000\022\024\n\020UNEXPECTED_ERROR\020\001\022\022\n\016C" + "ONNECT_FAILED\020\002\022\025\n\021PERMISSION_DENIED\020\003\022\031" + "\n\025COLLECTION_NOT_EXISTS\020\004\022\024\n\020ILLEGAL_ARG" + "UMENT\020\005\022\025\n\021ILLEGAL_DIMENSION\020\007\022\026\n\022ILLEGA" + "L_INDEX_TYPE\020\010\022\033\n\027ILLEGAL_COLLECTION_NAM" + "E\020\t\022\020\n\014ILLEGAL_TOPK\020\n\022\025\n\021ILLEGAL_ROWRECO" + "RD\020\013\022\025\n\021ILLEGAL_VECTOR_ID\020\014\022\031\n\025ILLEGAL_S" + "EARCH_RESULT\020\r\022\022\n\016FILE_NOT_FOUND\020\016\022\017\n\013ME" + "TA_FAILED\020\017\022\020\n\014CACHE_FAILED\020\020\022\030\n\024CANNOT_" + "CREATE_FOLDER\020\021\022\026\n\022CANNOT_CREATE_FILE\020\022\022" + "\030\n\024CANNOT_DELETE_FOLDER\020\023\022\026\n\022CANNOT_DELE" + "TE_FILE\020\024\022\025\n\021BUILD_INDEX_ERROR\020\025\022\021\n\rILLE" + "GAL_NLIST\020\026\022\027\n\023ILLEGAL_METRIC_TYPE\020\027\022\021\n\r" + "OUT_OF_MEMORY\020\030BBZ@github.com/zilliztech" + "/milvus-distributed/internal/proto/commo" + "npbb\006proto3" + ; +static const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable*const descriptor_table_common_2eproto_deps[1] = { +}; +static ::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase*const descriptor_table_common_2eproto_sccs[5] = { + &scc_info_Address_common_2eproto.base, + &scc_info_Blob_common_2eproto.base, + &scc_info_Empty_common_2eproto.base, + &scc_info_KeyValuePair_common_2eproto.base, + &scc_info_Status_common_2eproto.base, +}; +static ::PROTOBUF_NAMESPACE_ID::internal::once_flag descriptor_table_common_2eproto_once; +static bool descriptor_table_common_2eproto_initialized = false; +const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable descriptor_table_common_2eproto = { + &descriptor_table_common_2eproto_initialized, descriptor_table_protodef_common_2eproto, "common.proto", 851, + &descriptor_table_common_2eproto_once, descriptor_table_common_2eproto_sccs, descriptor_table_common_2eproto_deps, 5, 0, + schemas, file_default_instances, TableStruct_common_2eproto::offsets, + file_level_metadata_common_2eproto, 5, file_level_enum_descriptors_common_2eproto, file_level_service_descriptors_common_2eproto, +}; + +// Force running AddDescriptors() at dynamic initialization time. +static bool dynamic_init_dummy_common_2eproto = ( ::PROTOBUF_NAMESPACE_ID::internal::AddDescriptors(&descriptor_table_common_2eproto), true); +namespace milvus { +namespace proto { +namespace common { +const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* ErrorCode_descriptor() { + ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_common_2eproto); + return file_level_enum_descriptors_common_2eproto[0]; +} +bool ErrorCode_IsValid(int value) { + switch (value) { + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 7: + case 8: + case 9: + case 10: + case 11: + case 12: + case 13: + case 14: + case 15: + case 16: + case 17: + case 18: + case 19: + case 20: + case 21: + case 22: + case 23: + case 24: + return true; + default: + return false; + } +} + + +// =================================================================== + +void Empty::InitAsDefaultInstance() { +} +class Empty::_Internal { + public: +}; + +Empty::Empty() + : ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:milvus.proto.common.Empty) +} +Empty::Empty(const Empty& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _internal_metadata_(nullptr) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + // @@protoc_insertion_point(copy_constructor:milvus.proto.common.Empty) +} + +void Empty::SharedCtor() { +} + +Empty::~Empty() { + // @@protoc_insertion_point(destructor:milvus.proto.common.Empty) + SharedDtor(); +} + +void Empty::SharedDtor() { +} + +void Empty::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const Empty& Empty::default_instance() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_Empty_common_2eproto.base); + return *internal_default_instance(); +} + + +void Empty::Clear() { +// @@protoc_insertion_point(message_clear_start:milvus.proto.common.Empty) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* Empty::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + CHK_(ptr); + switch (tag >> 3) { + default: { + if ((tag & 7) == 4 || tag == 0) { + ctx->SetLastTag(tag); + goto success; + } + ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx); + CHK_(ptr != nullptr); + continue; + } + } // switch + } // while +success: + return ptr; +failure: + ptr = nullptr; + goto success; +#undef CHK_ +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool Empty::MergePartialFromCodedStream( + ::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + // @@protoc_insertion_point(parse_start:milvus.proto.common.Empty) + for (;;) { + ::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + } +success: + // @@protoc_insertion_point(parse_success:milvus.proto.common.Empty) + return true; +failure: + // @@protoc_insertion_point(parse_failure:milvus.proto.common.Empty) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void Empty::SerializeWithCachedSizes( + ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:milvus.proto.common.Empty) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + if (_internal_metadata_.have_unknown_fields()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:milvus.proto.common.Empty) +} + +::PROTOBUF_NAMESPACE_ID::uint8* Empty::InternalSerializeWithCachedSizesToArray( + ::PROTOBUF_NAMESPACE_ID::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:milvus.proto.common.Empty) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + if (_internal_metadata_.have_unknown_fields()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:milvus.proto.common.Empty) + return target; +} + +size_t Empty::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:milvus.proto.common.Empty) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void Empty::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.common.Empty) + GOOGLE_DCHECK_NE(&from, this); + const Empty* source = + ::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.common.Empty) + ::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.common.Empty) + MergeFrom(*source); + } +} + +void Empty::MergeFrom(const Empty& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.common.Empty) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + +} + +void Empty::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.common.Empty) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void Empty::CopyFrom(const Empty& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.common.Empty) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool Empty::IsInitialized() const { + return true; +} + +void Empty::InternalSwap(Empty* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); +} + +::PROTOBUF_NAMESPACE_ID::Metadata Empty::GetMetadata() const { + return GetMetadataStatic(); +} + + +// =================================================================== + +void Status::InitAsDefaultInstance() { +} +class Status::_Internal { + public: +}; + +Status::Status() + : ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:milvus.proto.common.Status) +} +Status::Status(const Status& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _internal_metadata_(nullptr) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + reason_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + if (!from.reason().empty()) { + reason_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.reason_); + } + error_code_ = from.error_code_; + // @@protoc_insertion_point(copy_constructor:milvus.proto.common.Status) +} + +void Status::SharedCtor() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&scc_info_Status_common_2eproto.base); + reason_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + error_code_ = 0; +} + +Status::~Status() { + // @@protoc_insertion_point(destructor:milvus.proto.common.Status) + SharedDtor(); +} + +void Status::SharedDtor() { + reason_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +void Status::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const Status& Status::default_instance() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_Status_common_2eproto.base); + return *internal_default_instance(); +} + + +void Status::Clear() { +// @@protoc_insertion_point(message_clear_start:milvus.proto.common.Status) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + reason_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + error_code_ = 0; + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* Status::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + CHK_(ptr); + switch (tag >> 3) { + // .milvus.proto.common.ErrorCode error_code = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 8)) { + ::PROTOBUF_NAMESPACE_ID::uint64 val = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + set_error_code(static_cast<::milvus::proto::common::ErrorCode>(val)); + } else goto handle_unusual; + continue; + // string reason = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 18)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(mutable_reason(), ptr, ctx, "milvus.proto.common.Status.reason"); + CHK_(ptr); + } else goto handle_unusual; + continue; + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->SetLastTag(tag); + goto success; + } + ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx); + CHK_(ptr != nullptr); + continue; + } + } // switch + } // while +success: + return ptr; +failure: + ptr = nullptr; + goto success; +#undef CHK_ +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool Status::MergePartialFromCodedStream( + ::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + // @@protoc_insertion_point(parse_start:milvus.proto.common.Status) + for (;;) { + ::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // .milvus.proto.common.ErrorCode error_code = 1; + case 1: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (8 & 0xFF)) { + int value = 0; + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + int, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_ENUM>( + input, &value))); + set_error_code(static_cast< ::milvus::proto::common::ErrorCode >(value)); + } else { + goto handle_unusual; + } + break; + } + + // string reason = 2; + case 2: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (18 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->mutable_reason())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->reason().data(), static_cast(this->reason().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.common.Status.reason")); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:milvus.proto.common.Status) + return true; +failure: + // @@protoc_insertion_point(parse_failure:milvus.proto.common.Status) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void Status::SerializeWithCachedSizes( + ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:milvus.proto.common.Status) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // .milvus.proto.common.ErrorCode error_code = 1; + if (this->error_code() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnum( + 1, this->error_code(), output); + } + + // string reason = 2; + if (this->reason().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->reason().data(), static_cast(this->reason().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.common.Status.reason"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringMaybeAliased( + 2, this->reason(), output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:milvus.proto.common.Status) +} + +::PROTOBUF_NAMESPACE_ID::uint8* Status::InternalSerializeWithCachedSizesToArray( + ::PROTOBUF_NAMESPACE_ID::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:milvus.proto.common.Status) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // .milvus.proto.common.ErrorCode error_code = 1; + if (this->error_code() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnumToArray( + 1, this->error_code(), target); + } + + // string reason = 2; + if (this->reason().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->reason().data(), static_cast(this->reason().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.common.Status.reason"); + target = + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringToArray( + 2, this->reason(), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:milvus.proto.common.Status) + return target; +} + +size_t Status::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:milvus.proto.common.Status) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // string reason = 2; + if (this->reason().size() > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->reason()); + } + + // .milvus.proto.common.ErrorCode error_code = 1; + if (this->error_code() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->error_code()); + } + + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void Status::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.common.Status) + GOOGLE_DCHECK_NE(&from, this); + const Status* source = + ::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.common.Status) + ::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.common.Status) + MergeFrom(*source); + } +} + +void Status::MergeFrom(const Status& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.common.Status) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + if (from.reason().size() > 0) { + + reason_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.reason_); + } + if (from.error_code() != 0) { + set_error_code(from.error_code()); + } +} + +void Status::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.common.Status) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void Status::CopyFrom(const Status& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.common.Status) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool Status::IsInitialized() const { + return true; +} + +void Status::InternalSwap(Status* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + reason_.Swap(&other->reason_, &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); + swap(error_code_, other->error_code_); +} + +::PROTOBUF_NAMESPACE_ID::Metadata Status::GetMetadata() const { + return GetMetadataStatic(); +} + + +// =================================================================== + +void KeyValuePair::InitAsDefaultInstance() { +} +class KeyValuePair::_Internal { + public: +}; + +KeyValuePair::KeyValuePair() + : ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:milvus.proto.common.KeyValuePair) +} +KeyValuePair::KeyValuePair(const KeyValuePair& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _internal_metadata_(nullptr) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + key_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + if (!from.key().empty()) { + key_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.key_); + } + value_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + if (!from.value().empty()) { + value_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.value_); + } + // @@protoc_insertion_point(copy_constructor:milvus.proto.common.KeyValuePair) +} + +void KeyValuePair::SharedCtor() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&scc_info_KeyValuePair_common_2eproto.base); + key_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + value_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +KeyValuePair::~KeyValuePair() { + // @@protoc_insertion_point(destructor:milvus.proto.common.KeyValuePair) + SharedDtor(); +} + +void KeyValuePair::SharedDtor() { + key_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + value_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +void KeyValuePair::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const KeyValuePair& KeyValuePair::default_instance() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_KeyValuePair_common_2eproto.base); + return *internal_default_instance(); +} + + +void KeyValuePair::Clear() { +// @@protoc_insertion_point(message_clear_start:milvus.proto.common.KeyValuePair) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + key_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + value_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* KeyValuePair::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + CHK_(ptr); + switch (tag >> 3) { + // string key = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 10)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(mutable_key(), ptr, ctx, "milvus.proto.common.KeyValuePair.key"); + CHK_(ptr); + } else goto handle_unusual; + continue; + // string value = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 18)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(mutable_value(), ptr, ctx, "milvus.proto.common.KeyValuePair.value"); + CHK_(ptr); + } else goto handle_unusual; + continue; + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->SetLastTag(tag); + goto success; + } + ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx); + CHK_(ptr != nullptr); + continue; + } + } // switch + } // while +success: + return ptr; +failure: + ptr = nullptr; + goto success; +#undef CHK_ +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool KeyValuePair::MergePartialFromCodedStream( + ::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + // @@protoc_insertion_point(parse_start:milvus.proto.common.KeyValuePair) + for (;;) { + ::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // string key = 1; + case 1: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (10 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->mutable_key())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->key().data(), static_cast(this->key().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.common.KeyValuePair.key")); + } else { + goto handle_unusual; + } + break; + } + + // string value = 2; + case 2: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (18 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->mutable_value())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->value().data(), static_cast(this->value().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.common.KeyValuePair.value")); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:milvus.proto.common.KeyValuePair) + return true; +failure: + // @@protoc_insertion_point(parse_failure:milvus.proto.common.KeyValuePair) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void KeyValuePair::SerializeWithCachedSizes( + ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:milvus.proto.common.KeyValuePair) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // string key = 1; + if (this->key().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->key().data(), static_cast(this->key().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.common.KeyValuePair.key"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringMaybeAliased( + 1, this->key(), output); + } + + // string value = 2; + if (this->value().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->value().data(), static_cast(this->value().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.common.KeyValuePair.value"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringMaybeAliased( + 2, this->value(), output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:milvus.proto.common.KeyValuePair) +} + +::PROTOBUF_NAMESPACE_ID::uint8* KeyValuePair::InternalSerializeWithCachedSizesToArray( + ::PROTOBUF_NAMESPACE_ID::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:milvus.proto.common.KeyValuePair) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // string key = 1; + if (this->key().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->key().data(), static_cast(this->key().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.common.KeyValuePair.key"); + target = + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringToArray( + 1, this->key(), target); + } + + // string value = 2; + if (this->value().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->value().data(), static_cast(this->value().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.common.KeyValuePair.value"); + target = + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringToArray( + 2, this->value(), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:milvus.proto.common.KeyValuePair) + return target; +} + +size_t KeyValuePair::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:milvus.proto.common.KeyValuePair) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // string key = 1; + if (this->key().size() > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->key()); + } + + // string value = 2; + if (this->value().size() > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->value()); + } + + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void KeyValuePair::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.common.KeyValuePair) + GOOGLE_DCHECK_NE(&from, this); + const KeyValuePair* source = + ::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.common.KeyValuePair) + ::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.common.KeyValuePair) + MergeFrom(*source); + } +} + +void KeyValuePair::MergeFrom(const KeyValuePair& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.common.KeyValuePair) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + if (from.key().size() > 0) { + + key_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.key_); + } + if (from.value().size() > 0) { + + value_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.value_); + } +} + +void KeyValuePair::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.common.KeyValuePair) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void KeyValuePair::CopyFrom(const KeyValuePair& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.common.KeyValuePair) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool KeyValuePair::IsInitialized() const { + return true; +} + +void KeyValuePair::InternalSwap(KeyValuePair* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + key_.Swap(&other->key_, &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); + value_.Swap(&other->value_, &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); +} + +::PROTOBUF_NAMESPACE_ID::Metadata KeyValuePair::GetMetadata() const { + return GetMetadataStatic(); +} + + +// =================================================================== + +void Blob::InitAsDefaultInstance() { +} +class Blob::_Internal { + public: +}; + +Blob::Blob() + : ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:milvus.proto.common.Blob) +} +Blob::Blob(const Blob& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _internal_metadata_(nullptr) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + value_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + if (!from.value().empty()) { + value_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.value_); + } + // @@protoc_insertion_point(copy_constructor:milvus.proto.common.Blob) +} + +void Blob::SharedCtor() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&scc_info_Blob_common_2eproto.base); + value_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +Blob::~Blob() { + // @@protoc_insertion_point(destructor:milvus.proto.common.Blob) + SharedDtor(); +} + +void Blob::SharedDtor() { + value_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +void Blob::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const Blob& Blob::default_instance() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_Blob_common_2eproto.base); + return *internal_default_instance(); +} + + +void Blob::Clear() { +// @@protoc_insertion_point(message_clear_start:milvus.proto.common.Blob) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + value_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* Blob::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + CHK_(ptr); + switch (tag >> 3) { + // bytes value = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 10)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParser(mutable_value(), ptr, ctx); + CHK_(ptr); + } else goto handle_unusual; + continue; + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->SetLastTag(tag); + goto success; + } + ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx); + CHK_(ptr != nullptr); + continue; + } + } // switch + } // while +success: + return ptr; +failure: + ptr = nullptr; + goto success; +#undef CHK_ +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool Blob::MergePartialFromCodedStream( + ::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + // @@protoc_insertion_point(parse_start:milvus.proto.common.Blob) + for (;;) { + ::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // bytes value = 1; + case 1: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (10 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadBytes( + input, this->mutable_value())); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:milvus.proto.common.Blob) + return true; +failure: + // @@protoc_insertion_point(parse_failure:milvus.proto.common.Blob) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void Blob::SerializeWithCachedSizes( + ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:milvus.proto.common.Blob) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // bytes value = 1; + if (this->value().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteBytesMaybeAliased( + 1, this->value(), output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:milvus.proto.common.Blob) +} + +::PROTOBUF_NAMESPACE_ID::uint8* Blob::InternalSerializeWithCachedSizesToArray( + ::PROTOBUF_NAMESPACE_ID::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:milvus.proto.common.Blob) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // bytes value = 1; + if (this->value().size() > 0) { + target = + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteBytesToArray( + 1, this->value(), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:milvus.proto.common.Blob) + return target; +} + +size_t Blob::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:milvus.proto.common.Blob) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // bytes value = 1; + if (this->value().size() > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::BytesSize( + this->value()); + } + + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void Blob::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.common.Blob) + GOOGLE_DCHECK_NE(&from, this); + const Blob* source = + ::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.common.Blob) + ::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.common.Blob) + MergeFrom(*source); + } +} + +void Blob::MergeFrom(const Blob& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.common.Blob) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + if (from.value().size() > 0) { + + value_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.value_); + } +} + +void Blob::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.common.Blob) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void Blob::CopyFrom(const Blob& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.common.Blob) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool Blob::IsInitialized() const { + return true; +} + +void Blob::InternalSwap(Blob* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + value_.Swap(&other->value_, &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); +} + +::PROTOBUF_NAMESPACE_ID::Metadata Blob::GetMetadata() const { + return GetMetadataStatic(); +} + + +// =================================================================== + +void Address::InitAsDefaultInstance() { +} +class Address::_Internal { + public: +}; + +Address::Address() + : ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:milvus.proto.common.Address) +} +Address::Address(const Address& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _internal_metadata_(nullptr) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + ip_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + if (!from.ip().empty()) { + ip_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.ip_); + } + port_ = from.port_; + // @@protoc_insertion_point(copy_constructor:milvus.proto.common.Address) +} + +void Address::SharedCtor() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&scc_info_Address_common_2eproto.base); + ip_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + port_ = PROTOBUF_LONGLONG(0); +} + +Address::~Address() { + // @@protoc_insertion_point(destructor:milvus.proto.common.Address) + SharedDtor(); +} + +void Address::SharedDtor() { + ip_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +void Address::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const Address& Address::default_instance() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_Address_common_2eproto.base); + return *internal_default_instance(); +} + + +void Address::Clear() { +// @@protoc_insertion_point(message_clear_start:milvus.proto.common.Address) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + ip_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + port_ = PROTOBUF_LONGLONG(0); + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* Address::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + CHK_(ptr); + switch (tag >> 3) { + // string ip = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 10)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(mutable_ip(), ptr, ctx, "milvus.proto.common.Address.ip"); + CHK_(ptr); + } else goto handle_unusual; + continue; + // int64 port = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 16)) { + port_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->SetLastTag(tag); + goto success; + } + ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx); + CHK_(ptr != nullptr); + continue; + } + } // switch + } // while +success: + return ptr; +failure: + ptr = nullptr; + goto success; +#undef CHK_ +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool Address::MergePartialFromCodedStream( + ::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + // @@protoc_insertion_point(parse_start:milvus.proto.common.Address) + for (;;) { + ::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // string ip = 1; + case 1: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (10 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->mutable_ip())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->ip().data(), static_cast(this->ip().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.common.Address.ip")); + } else { + goto handle_unusual; + } + break; + } + + // int64 port = 2; + case 2: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (16 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::int64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_INT64>( + input, &port_))); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:milvus.proto.common.Address) + return true; +failure: + // @@protoc_insertion_point(parse_failure:milvus.proto.common.Address) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void Address::SerializeWithCachedSizes( + ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:milvus.proto.common.Address) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // string ip = 1; + if (this->ip().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->ip().data(), static_cast(this->ip().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.common.Address.ip"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringMaybeAliased( + 1, this->ip(), output); + } + + // int64 port = 2; + if (this->port() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt64(2, this->port(), output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:milvus.proto.common.Address) +} + +::PROTOBUF_NAMESPACE_ID::uint8* Address::InternalSerializeWithCachedSizesToArray( + ::PROTOBUF_NAMESPACE_ID::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:milvus.proto.common.Address) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // string ip = 1; + if (this->ip().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->ip().data(), static_cast(this->ip().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.common.Address.ip"); + target = + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringToArray( + 1, this->ip(), target); + } + + // int64 port = 2; + if (this->port() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt64ToArray(2, this->port(), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:milvus.proto.common.Address) + return target; +} + +size_t Address::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:milvus.proto.common.Address) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // string ip = 1; + if (this->ip().size() > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->ip()); + } + + // int64 port = 2; + if (this->port() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int64Size( + this->port()); + } + + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void Address::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.common.Address) + GOOGLE_DCHECK_NE(&from, this); + const Address* source = + ::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated
( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.common.Address) + ::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.common.Address) + MergeFrom(*source); + } +} + +void Address::MergeFrom(const Address& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.common.Address) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + if (from.ip().size() > 0) { + + ip_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.ip_); + } + if (from.port() != 0) { + set_port(from.port()); + } +} + +void Address::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.common.Address) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void Address::CopyFrom(const Address& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.common.Address) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool Address::IsInitialized() const { + return true; +} + +void Address::InternalSwap(Address* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + ip_.Swap(&other->ip_, &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); + swap(port_, other->port_); +} + +::PROTOBUF_NAMESPACE_ID::Metadata Address::GetMetadata() const { + return GetMetadataStatic(); +} + + +// @@protoc_insertion_point(namespace_scope) +} // namespace common +} // namespace proto +} // namespace milvus +PROTOBUF_NAMESPACE_OPEN +template<> PROTOBUF_NOINLINE ::milvus::proto::common::Empty* Arena::CreateMaybeMessage< ::milvus::proto::common::Empty >(Arena* arena) { + return Arena::CreateInternal< ::milvus::proto::common::Empty >(arena); +} +template<> PROTOBUF_NOINLINE ::milvus::proto::common::Status* Arena::CreateMaybeMessage< ::milvus::proto::common::Status >(Arena* arena) { + return Arena::CreateInternal< ::milvus::proto::common::Status >(arena); +} +template<> PROTOBUF_NOINLINE ::milvus::proto::common::KeyValuePair* Arena::CreateMaybeMessage< ::milvus::proto::common::KeyValuePair >(Arena* arena) { + return Arena::CreateInternal< ::milvus::proto::common::KeyValuePair >(arena); +} +template<> PROTOBUF_NOINLINE ::milvus::proto::common::Blob* Arena::CreateMaybeMessage< ::milvus::proto::common::Blob >(Arena* arena) { + return Arena::CreateInternal< ::milvus::proto::common::Blob >(arena); +} +template<> PROTOBUF_NOINLINE ::milvus::proto::common::Address* Arena::CreateMaybeMessage< ::milvus::proto::common::Address >(Arena* arena) { + return Arena::CreateInternal< ::milvus::proto::common::Address >(arena); +} +PROTOBUF_NAMESPACE_CLOSE + +// @@protoc_insertion_point(global_scope) +#include diff --git a/internal/core/src/pb/common.pb.h b/internal/core/src/pb/common.pb.h new file mode 100644 index 0000000000..5a0c230a75 --- /dev/null +++ b/internal/core/src/pb/common.pb.h @@ -0,0 +1,1175 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: common.proto + +#ifndef GOOGLE_PROTOBUF_INCLUDED_common_2eproto +#define GOOGLE_PROTOBUF_INCLUDED_common_2eproto + +#include +#include + +#include +#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 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include // IWYU pragma: export +#include // IWYU pragma: export +#include +#include +// @@protoc_insertion_point(includes) +#include +#define PROTOBUF_INTERNAL_EXPORT_common_2eproto +PROTOBUF_NAMESPACE_OPEN +namespace internal { +class AnyMetadata; +} // namespace internal +PROTOBUF_NAMESPACE_CLOSE + +// Internal implementation detail -- do not use these members. +struct TableStruct_common_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[5] + 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_common_2eproto; +namespace milvus { +namespace proto { +namespace common { +class Address; +class AddressDefaultTypeInternal; +extern AddressDefaultTypeInternal _Address_default_instance_; +class Blob; +class BlobDefaultTypeInternal; +extern BlobDefaultTypeInternal _Blob_default_instance_; +class Empty; +class EmptyDefaultTypeInternal; +extern EmptyDefaultTypeInternal _Empty_default_instance_; +class KeyValuePair; +class KeyValuePairDefaultTypeInternal; +extern KeyValuePairDefaultTypeInternal _KeyValuePair_default_instance_; +class Status; +class StatusDefaultTypeInternal; +extern StatusDefaultTypeInternal _Status_default_instance_; +} // namespace common +} // namespace proto +} // namespace milvus +PROTOBUF_NAMESPACE_OPEN +template<> ::milvus::proto::common::Address* Arena::CreateMaybeMessage<::milvus::proto::common::Address>(Arena*); +template<> ::milvus::proto::common::Blob* Arena::CreateMaybeMessage<::milvus::proto::common::Blob>(Arena*); +template<> ::milvus::proto::common::Empty* Arena::CreateMaybeMessage<::milvus::proto::common::Empty>(Arena*); +template<> ::milvus::proto::common::KeyValuePair* Arena::CreateMaybeMessage<::milvus::proto::common::KeyValuePair>(Arena*); +template<> ::milvus::proto::common::Status* Arena::CreateMaybeMessage<::milvus::proto::common::Status>(Arena*); +PROTOBUF_NAMESPACE_CLOSE +namespace milvus { +namespace proto { +namespace common { + +enum ErrorCode : int { + 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, + ErrorCode_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits<::PROTOBUF_NAMESPACE_ID::int32>::min(), + ErrorCode_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits<::PROTOBUF_NAMESPACE_ID::int32>::max() +}; +bool ErrorCode_IsValid(int value); +constexpr ErrorCode ErrorCode_MIN = SUCCESS; +constexpr ErrorCode ErrorCode_MAX = OUT_OF_MEMORY; +constexpr int ErrorCode_ARRAYSIZE = ErrorCode_MAX + 1; + +const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* ErrorCode_descriptor(); +template +inline const std::string& ErrorCode_Name(T enum_t_value) { + static_assert(::std::is_same::value || + ::std::is_integral::value, + "Incorrect type passed to function ErrorCode_Name."); + return ::PROTOBUF_NAMESPACE_ID::internal::NameOfEnum( + ErrorCode_descriptor(), enum_t_value); +} +inline bool ErrorCode_Parse( + const std::string& name, ErrorCode* value) { + return ::PROTOBUF_NAMESPACE_ID::internal::ParseNamedEnum( + ErrorCode_descriptor(), name, value); +} +// =================================================================== + +class Empty : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.common.Empty) */ { + public: + Empty(); + virtual ~Empty(); + + Empty(const Empty& from); + Empty(Empty&& from) noexcept + : Empty() { + *this = ::std::move(from); + } + + inline Empty& operator=(const Empty& from) { + CopyFrom(from); + return *this; + } + inline Empty& operator=(Empty&& 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 Empty& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const Empty* internal_default_instance() { + return reinterpret_cast( + &_Empty_default_instance_); + } + static constexpr int kIndexInFileMessages = + 0; + + friend void swap(Empty& a, Empty& b) { + a.Swap(&b); + } + inline void Swap(Empty* other) { + if (other == this) return; + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + inline Empty* New() const final { + return CreateMaybeMessage(nullptr); + } + + Empty* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void CopyFrom(const Empty& from); + void MergeFrom(const Empty& 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(Empty* other); + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "milvus.proto.common.Empty"; + } + 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_common_2eproto); + return ::descriptor_table_common_2eproto.file_level_metadata[kIndexInFileMessages]; + } + + public: + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + // @@protoc_insertion_point(class_scope:milvus.proto.common.Empty) + private: + class _Internal; + + ::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_common_2eproto; +}; +// ------------------------------------------------------------------- + +class Status : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.common.Status) */ { + public: + Status(); + virtual ~Status(); + + Status(const Status& from); + Status(Status&& from) noexcept + : Status() { + *this = ::std::move(from); + } + + inline Status& operator=(const Status& from) { + CopyFrom(from); + return *this; + } + inline Status& operator=(Status&& 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 Status& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const Status* internal_default_instance() { + return reinterpret_cast( + &_Status_default_instance_); + } + static constexpr int kIndexInFileMessages = + 1; + + friend void swap(Status& a, Status& b) { + a.Swap(&b); + } + inline void Swap(Status* other) { + if (other == this) return; + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + inline Status* New() const final { + return CreateMaybeMessage(nullptr); + } + + Status* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void CopyFrom(const Status& from); + void MergeFrom(const Status& 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(Status* other); + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "milvus.proto.common.Status"; + } + 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_common_2eproto); + return ::descriptor_table_common_2eproto.file_level_metadata[kIndexInFileMessages]; + } + + public: + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kReasonFieldNumber = 2, + kErrorCodeFieldNumber = 1, + }; + // string reason = 2; + void clear_reason(); + const std::string& reason() const; + void set_reason(const std::string& value); + void set_reason(std::string&& value); + void set_reason(const char* value); + void set_reason(const char* value, size_t size); + std::string* mutable_reason(); + std::string* release_reason(); + void set_allocated_reason(std::string* reason); + + // .milvus.proto.common.ErrorCode error_code = 1; + void clear_error_code(); + ::milvus::proto::common::ErrorCode error_code() const; + void set_error_code(::milvus::proto::common::ErrorCode value); + + // @@protoc_insertion_point(class_scope:milvus.proto.common.Status) + private: + class _Internal; + + ::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr reason_; + int error_code_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_common_2eproto; +}; +// ------------------------------------------------------------------- + +class KeyValuePair : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.common.KeyValuePair) */ { + public: + KeyValuePair(); + virtual ~KeyValuePair(); + + KeyValuePair(const KeyValuePair& from); + KeyValuePair(KeyValuePair&& from) noexcept + : KeyValuePair() { + *this = ::std::move(from); + } + + inline KeyValuePair& operator=(const KeyValuePair& from) { + CopyFrom(from); + return *this; + } + inline KeyValuePair& operator=(KeyValuePair&& 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 KeyValuePair& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const KeyValuePair* internal_default_instance() { + return reinterpret_cast( + &_KeyValuePair_default_instance_); + } + static constexpr int kIndexInFileMessages = + 2; + + friend void swap(KeyValuePair& a, KeyValuePair& b) { + a.Swap(&b); + } + inline void Swap(KeyValuePair* other) { + if (other == this) return; + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + inline KeyValuePair* New() const final { + return CreateMaybeMessage(nullptr); + } + + KeyValuePair* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void CopyFrom(const KeyValuePair& from); + void MergeFrom(const KeyValuePair& 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(KeyValuePair* other); + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "milvus.proto.common.KeyValuePair"; + } + 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_common_2eproto); + return ::descriptor_table_common_2eproto.file_level_metadata[kIndexInFileMessages]; + } + + public: + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kKeyFieldNumber = 1, + kValueFieldNumber = 2, + }; + // string key = 1; + void clear_key(); + const std::string& key() const; + void set_key(const std::string& value); + void set_key(std::string&& value); + void set_key(const char* value); + void set_key(const char* value, size_t size); + std::string* mutable_key(); + std::string* release_key(); + void set_allocated_key(std::string* key); + + // string value = 2; + void clear_value(); + const std::string& value() const; + void set_value(const std::string& value); + void set_value(std::string&& value); + void set_value(const char* value); + void set_value(const char* value, size_t size); + std::string* mutable_value(); + std::string* release_value(); + void set_allocated_value(std::string* value); + + // @@protoc_insertion_point(class_scope:milvus.proto.common.KeyValuePair) + private: + class _Internal; + + ::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr key_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr value_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_common_2eproto; +}; +// ------------------------------------------------------------------- + +class Blob : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.common.Blob) */ { + public: + Blob(); + virtual ~Blob(); + + Blob(const Blob& from); + Blob(Blob&& from) noexcept + : Blob() { + *this = ::std::move(from); + } + + inline Blob& operator=(const Blob& from) { + CopyFrom(from); + return *this; + } + inline Blob& operator=(Blob&& 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 Blob& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const Blob* internal_default_instance() { + return reinterpret_cast( + &_Blob_default_instance_); + } + static constexpr int kIndexInFileMessages = + 3; + + friend void swap(Blob& a, Blob& b) { + a.Swap(&b); + } + inline void Swap(Blob* other) { + if (other == this) return; + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + inline Blob* New() const final { + return CreateMaybeMessage(nullptr); + } + + Blob* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void CopyFrom(const Blob& from); + void MergeFrom(const Blob& 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(Blob* other); + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "milvus.proto.common.Blob"; + } + 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_common_2eproto); + return ::descriptor_table_common_2eproto.file_level_metadata[kIndexInFileMessages]; + } + + public: + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kValueFieldNumber = 1, + }; + // bytes value = 1; + void clear_value(); + const std::string& value() const; + void set_value(const std::string& value); + void set_value(std::string&& value); + void set_value(const char* value); + void set_value(const void* value, size_t size); + std::string* mutable_value(); + std::string* release_value(); + void set_allocated_value(std::string* value); + + // @@protoc_insertion_point(class_scope:milvus.proto.common.Blob) + private: + class _Internal; + + ::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr value_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_common_2eproto; +}; +// ------------------------------------------------------------------- + +class Address : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.common.Address) */ { + public: + Address(); + virtual ~Address(); + + Address(const Address& from); + Address(Address&& from) noexcept + : Address() { + *this = ::std::move(from); + } + + inline Address& operator=(const Address& from) { + CopyFrom(from); + return *this; + } + inline Address& operator=(Address&& 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 Address& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const Address* internal_default_instance() { + return reinterpret_cast( + &_Address_default_instance_); + } + static constexpr int kIndexInFileMessages = + 4; + + friend void swap(Address& a, Address& b) { + a.Swap(&b); + } + inline void Swap(Address* other) { + if (other == this) return; + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + inline Address* New() const final { + return CreateMaybeMessage
(nullptr); + } + + Address* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final { + return CreateMaybeMessage
(arena); + } + void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void CopyFrom(const Address& from); + void MergeFrom(const Address& 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(Address* other); + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "milvus.proto.common.Address"; + } + 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_common_2eproto); + return ::descriptor_table_common_2eproto.file_level_metadata[kIndexInFileMessages]; + } + + public: + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kIpFieldNumber = 1, + kPortFieldNumber = 2, + }; + // string ip = 1; + void clear_ip(); + const std::string& ip() const; + void set_ip(const std::string& value); + void set_ip(std::string&& value); + void set_ip(const char* value); + void set_ip(const char* value, size_t size); + std::string* mutable_ip(); + std::string* release_ip(); + void set_allocated_ip(std::string* ip); + + // int64 port = 2; + void clear_port(); + ::PROTOBUF_NAMESPACE_ID::int64 port() const; + void set_port(::PROTOBUF_NAMESPACE_ID::int64 value); + + // @@protoc_insertion_point(class_scope:milvus.proto.common.Address) + private: + class _Internal; + + ::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr ip_; + ::PROTOBUF_NAMESPACE_ID::int64 port_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_common_2eproto; +}; +// =================================================================== + + +// =================================================================== + +#ifdef __GNUC__ + #pragma GCC diagnostic push + #pragma GCC diagnostic ignored "-Wstrict-aliasing" +#endif // __GNUC__ +// Empty + +// ------------------------------------------------------------------- + +// Status + +// .milvus.proto.common.ErrorCode error_code = 1; +inline void Status::clear_error_code() { + error_code_ = 0; +} +inline ::milvus::proto::common::ErrorCode Status::error_code() const { + // @@protoc_insertion_point(field_get:milvus.proto.common.Status.error_code) + return static_cast< ::milvus::proto::common::ErrorCode >(error_code_); +} +inline void Status::set_error_code(::milvus::proto::common::ErrorCode value) { + + error_code_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.common.Status.error_code) +} + +// string reason = 2; +inline void Status::clear_reason() { + reason_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline const std::string& Status::reason() const { + // @@protoc_insertion_point(field_get:milvus.proto.common.Status.reason) + return reason_.GetNoArena(); +} +inline void Status::set_reason(const std::string& value) { + + reason_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set:milvus.proto.common.Status.reason) +} +inline void Status::set_reason(std::string&& value) { + + reason_.SetNoArena( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); + // @@protoc_insertion_point(field_set_rvalue:milvus.proto.common.Status.reason) +} +inline void Status::set_reason(const char* value) { + GOOGLE_DCHECK(value != nullptr); + + reason_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); + // @@protoc_insertion_point(field_set_char:milvus.proto.common.Status.reason) +} +inline void Status::set_reason(const char* value, size_t size) { + + reason_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + ::std::string(reinterpret_cast(value), size)); + // @@protoc_insertion_point(field_set_pointer:milvus.proto.common.Status.reason) +} +inline std::string* Status::mutable_reason() { + + // @@protoc_insertion_point(field_mutable:milvus.proto.common.Status.reason) + return reason_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline std::string* Status::release_reason() { + // @@protoc_insertion_point(field_release:milvus.proto.common.Status.reason) + + return reason_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline void Status::set_allocated_reason(std::string* reason) { + if (reason != nullptr) { + + } else { + + } + reason_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), reason); + // @@protoc_insertion_point(field_set_allocated:milvus.proto.common.Status.reason) +} + +// ------------------------------------------------------------------- + +// KeyValuePair + +// string key = 1; +inline void KeyValuePair::clear_key() { + key_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline const std::string& KeyValuePair::key() const { + // @@protoc_insertion_point(field_get:milvus.proto.common.KeyValuePair.key) + return key_.GetNoArena(); +} +inline void KeyValuePair::set_key(const std::string& value) { + + key_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set:milvus.proto.common.KeyValuePair.key) +} +inline void KeyValuePair::set_key(std::string&& value) { + + key_.SetNoArena( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); + // @@protoc_insertion_point(field_set_rvalue:milvus.proto.common.KeyValuePair.key) +} +inline void KeyValuePair::set_key(const char* value) { + GOOGLE_DCHECK(value != nullptr); + + key_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); + // @@protoc_insertion_point(field_set_char:milvus.proto.common.KeyValuePair.key) +} +inline void KeyValuePair::set_key(const char* value, size_t size) { + + key_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + ::std::string(reinterpret_cast(value), size)); + // @@protoc_insertion_point(field_set_pointer:milvus.proto.common.KeyValuePair.key) +} +inline std::string* KeyValuePair::mutable_key() { + + // @@protoc_insertion_point(field_mutable:milvus.proto.common.KeyValuePair.key) + return key_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline std::string* KeyValuePair::release_key() { + // @@protoc_insertion_point(field_release:milvus.proto.common.KeyValuePair.key) + + return key_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline void KeyValuePair::set_allocated_key(std::string* key) { + if (key != nullptr) { + + } else { + + } + key_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), key); + // @@protoc_insertion_point(field_set_allocated:milvus.proto.common.KeyValuePair.key) +} + +// string value = 2; +inline void KeyValuePair::clear_value() { + value_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline const std::string& KeyValuePair::value() const { + // @@protoc_insertion_point(field_get:milvus.proto.common.KeyValuePair.value) + return value_.GetNoArena(); +} +inline void KeyValuePair::set_value(const std::string& value) { + + value_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set:milvus.proto.common.KeyValuePair.value) +} +inline void KeyValuePair::set_value(std::string&& value) { + + value_.SetNoArena( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); + // @@protoc_insertion_point(field_set_rvalue:milvus.proto.common.KeyValuePair.value) +} +inline void KeyValuePair::set_value(const char* value) { + GOOGLE_DCHECK(value != nullptr); + + value_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); + // @@protoc_insertion_point(field_set_char:milvus.proto.common.KeyValuePair.value) +} +inline void KeyValuePair::set_value(const char* value, size_t size) { + + value_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + ::std::string(reinterpret_cast(value), size)); + // @@protoc_insertion_point(field_set_pointer:milvus.proto.common.KeyValuePair.value) +} +inline std::string* KeyValuePair::mutable_value() { + + // @@protoc_insertion_point(field_mutable:milvus.proto.common.KeyValuePair.value) + return value_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline std::string* KeyValuePair::release_value() { + // @@protoc_insertion_point(field_release:milvus.proto.common.KeyValuePair.value) + + return value_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline void KeyValuePair::set_allocated_value(std::string* value) { + if (value != nullptr) { + + } else { + + } + value_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set_allocated:milvus.proto.common.KeyValuePair.value) +} + +// ------------------------------------------------------------------- + +// Blob + +// bytes value = 1; +inline void Blob::clear_value() { + value_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline const std::string& Blob::value() const { + // @@protoc_insertion_point(field_get:milvus.proto.common.Blob.value) + return value_.GetNoArena(); +} +inline void Blob::set_value(const std::string& value) { + + value_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set:milvus.proto.common.Blob.value) +} +inline void Blob::set_value(std::string&& value) { + + value_.SetNoArena( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); + // @@protoc_insertion_point(field_set_rvalue:milvus.proto.common.Blob.value) +} +inline void Blob::set_value(const char* value) { + GOOGLE_DCHECK(value != nullptr); + + value_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); + // @@protoc_insertion_point(field_set_char:milvus.proto.common.Blob.value) +} +inline void Blob::set_value(const void* value, size_t size) { + + value_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + ::std::string(reinterpret_cast(value), size)); + // @@protoc_insertion_point(field_set_pointer:milvus.proto.common.Blob.value) +} +inline std::string* Blob::mutable_value() { + + // @@protoc_insertion_point(field_mutable:milvus.proto.common.Blob.value) + return value_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline std::string* Blob::release_value() { + // @@protoc_insertion_point(field_release:milvus.proto.common.Blob.value) + + return value_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline void Blob::set_allocated_value(std::string* value) { + if (value != nullptr) { + + } else { + + } + value_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set_allocated:milvus.proto.common.Blob.value) +} + +// ------------------------------------------------------------------- + +// Address + +// string ip = 1; +inline void Address::clear_ip() { + ip_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline const std::string& Address::ip() const { + // @@protoc_insertion_point(field_get:milvus.proto.common.Address.ip) + return ip_.GetNoArena(); +} +inline void Address::set_ip(const std::string& value) { + + ip_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set:milvus.proto.common.Address.ip) +} +inline void Address::set_ip(std::string&& value) { + + ip_.SetNoArena( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); + // @@protoc_insertion_point(field_set_rvalue:milvus.proto.common.Address.ip) +} +inline void Address::set_ip(const char* value) { + GOOGLE_DCHECK(value != nullptr); + + ip_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); + // @@protoc_insertion_point(field_set_char:milvus.proto.common.Address.ip) +} +inline void Address::set_ip(const char* value, size_t size) { + + ip_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + ::std::string(reinterpret_cast(value), size)); + // @@protoc_insertion_point(field_set_pointer:milvus.proto.common.Address.ip) +} +inline std::string* Address::mutable_ip() { + + // @@protoc_insertion_point(field_mutable:milvus.proto.common.Address.ip) + return ip_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline std::string* Address::release_ip() { + // @@protoc_insertion_point(field_release:milvus.proto.common.Address.ip) + + return ip_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline void Address::set_allocated_ip(std::string* ip) { + if (ip != nullptr) { + + } else { + + } + ip_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ip); + // @@protoc_insertion_point(field_set_allocated:milvus.proto.common.Address.ip) +} + +// int64 port = 2; +inline void Address::clear_port() { + port_ = PROTOBUF_LONGLONG(0); +} +inline ::PROTOBUF_NAMESPACE_ID::int64 Address::port() const { + // @@protoc_insertion_point(field_get:milvus.proto.common.Address.port) + return port_; +} +inline void Address::set_port(::PROTOBUF_NAMESPACE_ID::int64 value) { + + port_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.common.Address.port) +} + +#ifdef __GNUC__ + #pragma GCC diagnostic pop +#endif // __GNUC__ +// ------------------------------------------------------------------- + +// ------------------------------------------------------------------- + +// ------------------------------------------------------------------- + +// ------------------------------------------------------------------- + + +// @@protoc_insertion_point(namespace_scope) + +} // namespace common +} // namespace proto +} // namespace milvus + +PROTOBUF_NAMESPACE_OPEN + +template <> struct is_proto_enum< ::milvus::proto::common::ErrorCode> : ::std::true_type {}; +template <> +inline const EnumDescriptor* GetEnumDescriptor< ::milvus::proto::common::ErrorCode>() { + return ::milvus::proto::common::ErrorCode_descriptor(); +} + +PROTOBUF_NAMESPACE_CLOSE + +// @@protoc_insertion_point(global_scope) + +#include +#endif // GOOGLE_PROTOBUF_INCLUDED_GOOGLE_PROTOBUF_INCLUDED_common_2eproto diff --git a/internal/core/src/pb/etcd_meta.pb.cc b/internal/core/src/pb/etcd_meta.pb.cc new file mode 100644 index 0000000000..8b8ec55122 --- /dev/null +++ b/internal/core/src/pb/etcd_meta.pb.cc @@ -0,0 +1,2077 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: etcd_meta.proto + +#include "etcd_meta.pb.h" + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +// @@protoc_insertion_point(includes) +#include +extern PROTOBUF_INTERNAL_EXPORT_common_2eproto ::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<0> scc_info_Address_common_2eproto; +extern PROTOBUF_INTERNAL_EXPORT_schema_2eproto ::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<1> scc_info_CollectionSchema_schema_2eproto; +namespace milvus { +namespace proto { +namespace etcd { +class TenantMetaDefaultTypeInternal { + public: + ::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed _instance; +} _TenantMeta_default_instance_; +class ProxyMetaDefaultTypeInternal { + public: + ::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed _instance; +} _ProxyMeta_default_instance_; +class CollectionMetaDefaultTypeInternal { + public: + ::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed _instance; +} _CollectionMeta_default_instance_; +class SegmentMetaDefaultTypeInternal { + public: + ::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed _instance; +} _SegmentMeta_default_instance_; +} // namespace etcd +} // namespace proto +} // namespace milvus +static void InitDefaultsscc_info_CollectionMeta_etcd_5fmeta_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::milvus::proto::etcd::_CollectionMeta_default_instance_; + new (ptr) ::milvus::proto::etcd::CollectionMeta(); + ::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr); + } + ::milvus::proto::etcd::CollectionMeta::InitAsDefaultInstance(); +} + +::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<1> scc_info_CollectionMeta_etcd_5fmeta_2eproto = + {{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsscc_info_CollectionMeta_etcd_5fmeta_2eproto}, { + &scc_info_CollectionSchema_schema_2eproto.base,}}; + +static void InitDefaultsscc_info_ProxyMeta_etcd_5fmeta_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::milvus::proto::etcd::_ProxyMeta_default_instance_; + new (ptr) ::milvus::proto::etcd::ProxyMeta(); + ::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr); + } + ::milvus::proto::etcd::ProxyMeta::InitAsDefaultInstance(); +} + +::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<1> scc_info_ProxyMeta_etcd_5fmeta_2eproto = + {{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsscc_info_ProxyMeta_etcd_5fmeta_2eproto}, { + &scc_info_Address_common_2eproto.base,}}; + +static void InitDefaultsscc_info_SegmentMeta_etcd_5fmeta_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::milvus::proto::etcd::_SegmentMeta_default_instance_; + new (ptr) ::milvus::proto::etcd::SegmentMeta(); + ::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr); + } + ::milvus::proto::etcd::SegmentMeta::InitAsDefaultInstance(); +} + +::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<0> scc_info_SegmentMeta_etcd_5fmeta_2eproto = + {{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsscc_info_SegmentMeta_etcd_5fmeta_2eproto}, {}}; + +static void InitDefaultsscc_info_TenantMeta_etcd_5fmeta_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::milvus::proto::etcd::_TenantMeta_default_instance_; + new (ptr) ::milvus::proto::etcd::TenantMeta(); + ::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr); + } + ::milvus::proto::etcd::TenantMeta::InitAsDefaultInstance(); +} + +::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<0> scc_info_TenantMeta_etcd_5fmeta_2eproto = + {{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 0, InitDefaultsscc_info_TenantMeta_etcd_5fmeta_2eproto}, {}}; + +static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_etcd_5fmeta_2eproto[4]; +static constexpr ::PROTOBUF_NAMESPACE_ID::EnumDescriptor const** file_level_enum_descriptors_etcd_5fmeta_2eproto = nullptr; +static constexpr ::PROTOBUF_NAMESPACE_ID::ServiceDescriptor const** file_level_service_descriptors_etcd_5fmeta_2eproto = nullptr; + +const ::PROTOBUF_NAMESPACE_ID::uint32 TableStruct_etcd_5fmeta_2eproto::offsets[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::TenantMeta, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::TenantMeta, id_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::TenantMeta, num_query_nodes_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::TenantMeta, insert_channel_ids_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::TenantMeta, query_channel_id_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::ProxyMeta, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::ProxyMeta, id_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::ProxyMeta, address_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::ProxyMeta, result_channel_ids_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::CollectionMeta, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::CollectionMeta, id_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::CollectionMeta, schema_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::CollectionMeta, create_time_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::CollectionMeta, segment_ids_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::CollectionMeta, partition_tags_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::SegmentMeta, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::SegmentMeta, segment_id_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::SegmentMeta, collection_id_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::SegmentMeta, partition_tag_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::SegmentMeta, channel_start_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::SegmentMeta, channel_end_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::SegmentMeta, open_time_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::SegmentMeta, close_time_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::etcd::SegmentMeta, num_rows_), +}; +static const ::PROTOBUF_NAMESPACE_ID::internal::MigrationSchema schemas[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { + { 0, -1, sizeof(::milvus::proto::etcd::TenantMeta)}, + { 9, -1, sizeof(::milvus::proto::etcd::ProxyMeta)}, + { 17, -1, sizeof(::milvus::proto::etcd::CollectionMeta)}, + { 27, -1, sizeof(::milvus::proto::etcd::SegmentMeta)}, +}; + +static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = { + reinterpret_cast(&::milvus::proto::etcd::_TenantMeta_default_instance_), + reinterpret_cast(&::milvus::proto::etcd::_ProxyMeta_default_instance_), + reinterpret_cast(&::milvus::proto::etcd::_CollectionMeta_default_instance_), + reinterpret_cast(&::milvus::proto::etcd::_SegmentMeta_default_instance_), +}; + +const char descriptor_table_protodef_etcd_5fmeta_2eproto[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = + "\n\017etcd_meta.proto\022\021milvus.proto.etcd\032\014co" + "mmon.proto\032\014schema.proto\"g\n\nTenantMeta\022\n" + "\n\002id\030\001 \001(\004\022\027\n\017num_query_nodes\030\002 \001(\004\022\032\n\022i" + "nsert_channel_ids\030\003 \003(\t\022\030\n\020query_channel" + "_id\030\004 \001(\t\"b\n\tProxyMeta\022\n\n\002id\030\001 \001(\004\022-\n\007ad" + "dress\030\002 \001(\0132\034.milvus.proto.common.Addres" + "s\022\032\n\022result_channel_ids\030\003 \003(\t\"\225\001\n\016Collec" + "tionMeta\022\n\n\002id\030\001 \001(\004\0225\n\006schema\030\002 \001(\0132%.m" + "ilvus.proto.schema.CollectionSchema\022\023\n\013c" + "reate_time\030\003 \001(\004\022\023\n\013segment_ids\030\004 \003(\004\022\026\n" + "\016partition_tags\030\005 \003(\t\"\264\001\n\013SegmentMeta\022\022\n" + "\nsegment_id\030\001 \001(\004\022\025\n\rcollection_id\030\002 \001(\004" + "\022\025\n\rpartition_tag\030\003 \001(\t\022\025\n\rchannel_start" + "\030\004 \001(\005\022\023\n\013channel_end\030\005 \001(\005\022\021\n\topen_time" + "\030\006 \001(\004\022\022\n\nclose_time\030\007 \001(\004\022\020\n\010num_rows\030\010" + " \001(\003B@Z>github.com/zilliztech/milvus-dis" + "tributed/internal/proto/etcdpbb\006proto3" + ; +static const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable*const descriptor_table_etcd_5fmeta_2eproto_deps[2] = { + &::descriptor_table_common_2eproto, + &::descriptor_table_schema_2eproto, +}; +static ::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase*const descriptor_table_etcd_5fmeta_2eproto_sccs[4] = { + &scc_info_CollectionMeta_etcd_5fmeta_2eproto.base, + &scc_info_ProxyMeta_etcd_5fmeta_2eproto.base, + &scc_info_SegmentMeta_etcd_5fmeta_2eproto.base, + &scc_info_TenantMeta_etcd_5fmeta_2eproto.base, +}; +static ::PROTOBUF_NAMESPACE_ID::internal::once_flag descriptor_table_etcd_5fmeta_2eproto_once; +static bool descriptor_table_etcd_5fmeta_2eproto_initialized = false; +const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable descriptor_table_etcd_5fmeta_2eproto = { + &descriptor_table_etcd_5fmeta_2eproto_initialized, descriptor_table_protodef_etcd_5fmeta_2eproto, "etcd_meta.proto", 678, + &descriptor_table_etcd_5fmeta_2eproto_once, descriptor_table_etcd_5fmeta_2eproto_sccs, descriptor_table_etcd_5fmeta_2eproto_deps, 4, 2, + schemas, file_default_instances, TableStruct_etcd_5fmeta_2eproto::offsets, + file_level_metadata_etcd_5fmeta_2eproto, 4, file_level_enum_descriptors_etcd_5fmeta_2eproto, file_level_service_descriptors_etcd_5fmeta_2eproto, +}; + +// Force running AddDescriptors() at dynamic initialization time. +static bool dynamic_init_dummy_etcd_5fmeta_2eproto = ( ::PROTOBUF_NAMESPACE_ID::internal::AddDescriptors(&descriptor_table_etcd_5fmeta_2eproto), true); +namespace milvus { +namespace proto { +namespace etcd { + +// =================================================================== + +void TenantMeta::InitAsDefaultInstance() { +} +class TenantMeta::_Internal { + public: +}; + +TenantMeta::TenantMeta() + : ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:milvus.proto.etcd.TenantMeta) +} +TenantMeta::TenantMeta(const TenantMeta& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _internal_metadata_(nullptr), + insert_channel_ids_(from.insert_channel_ids_) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + query_channel_id_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + if (!from.query_channel_id().empty()) { + query_channel_id_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.query_channel_id_); + } + ::memcpy(&id_, &from.id_, + static_cast(reinterpret_cast(&num_query_nodes_) - + reinterpret_cast(&id_)) + sizeof(num_query_nodes_)); + // @@protoc_insertion_point(copy_constructor:milvus.proto.etcd.TenantMeta) +} + +void TenantMeta::SharedCtor() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&scc_info_TenantMeta_etcd_5fmeta_2eproto.base); + query_channel_id_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + ::memset(&id_, 0, static_cast( + reinterpret_cast(&num_query_nodes_) - + reinterpret_cast(&id_)) + sizeof(num_query_nodes_)); +} + +TenantMeta::~TenantMeta() { + // @@protoc_insertion_point(destructor:milvus.proto.etcd.TenantMeta) + SharedDtor(); +} + +void TenantMeta::SharedDtor() { + query_channel_id_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +void TenantMeta::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const TenantMeta& TenantMeta::default_instance() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_TenantMeta_etcd_5fmeta_2eproto.base); + return *internal_default_instance(); +} + + +void TenantMeta::Clear() { +// @@protoc_insertion_point(message_clear_start:milvus.proto.etcd.TenantMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + insert_channel_ids_.Clear(); + query_channel_id_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + ::memset(&id_, 0, static_cast( + reinterpret_cast(&num_query_nodes_) - + reinterpret_cast(&id_)) + sizeof(num_query_nodes_)); + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* TenantMeta::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + CHK_(ptr); + switch (tag >> 3) { + // uint64 id = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 8)) { + id_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // uint64 num_query_nodes = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 16)) { + num_query_nodes_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // repeated string insert_channel_ids = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 26)) { + ptr -= 1; + do { + ptr += 1; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(add_insert_channel_ids(), ptr, ctx, "milvus.proto.etcd.TenantMeta.insert_channel_ids"); + CHK_(ptr); + if (!ctx->DataAvailable(ptr)) break; + } while (::PROTOBUF_NAMESPACE_ID::internal::UnalignedLoad<::PROTOBUF_NAMESPACE_ID::uint8>(ptr) == 26); + } else goto handle_unusual; + continue; + // string query_channel_id = 4; + case 4: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 34)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(mutable_query_channel_id(), ptr, ctx, "milvus.proto.etcd.TenantMeta.query_channel_id"); + CHK_(ptr); + } else goto handle_unusual; + continue; + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->SetLastTag(tag); + goto success; + } + ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx); + CHK_(ptr != nullptr); + continue; + } + } // switch + } // while +success: + return ptr; +failure: + ptr = nullptr; + goto success; +#undef CHK_ +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool TenantMeta::MergePartialFromCodedStream( + ::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + // @@protoc_insertion_point(parse_start:milvus.proto.etcd.TenantMeta) + for (;;) { + ::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // uint64 id = 1; + case 1: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (8 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::uint64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_UINT64>( + input, &id_))); + } else { + goto handle_unusual; + } + break; + } + + // uint64 num_query_nodes = 2; + case 2: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (16 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::uint64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_UINT64>( + input, &num_query_nodes_))); + } else { + goto handle_unusual; + } + break; + } + + // repeated string insert_channel_ids = 3; + case 3: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (26 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->add_insert_channel_ids())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->insert_channel_ids(this->insert_channel_ids_size() - 1).data(), + static_cast(this->insert_channel_ids(this->insert_channel_ids_size() - 1).length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.etcd.TenantMeta.insert_channel_ids")); + } else { + goto handle_unusual; + } + break; + } + + // string query_channel_id = 4; + case 4: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (34 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->mutable_query_channel_id())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->query_channel_id().data(), static_cast(this->query_channel_id().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.etcd.TenantMeta.query_channel_id")); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:milvus.proto.etcd.TenantMeta) + return true; +failure: + // @@protoc_insertion_point(parse_failure:milvus.proto.etcd.TenantMeta) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void TenantMeta::SerializeWithCachedSizes( + ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:milvus.proto.etcd.TenantMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // uint64 id = 1; + if (this->id() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64(1, this->id(), output); + } + + // uint64 num_query_nodes = 2; + if (this->num_query_nodes() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64(2, this->num_query_nodes(), output); + } + + // repeated string insert_channel_ids = 3; + for (int i = 0, n = this->insert_channel_ids_size(); i < n; i++) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->insert_channel_ids(i).data(), static_cast(this->insert_channel_ids(i).length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.etcd.TenantMeta.insert_channel_ids"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteString( + 3, this->insert_channel_ids(i), output); + } + + // string query_channel_id = 4; + if (this->query_channel_id().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->query_channel_id().data(), static_cast(this->query_channel_id().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.etcd.TenantMeta.query_channel_id"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringMaybeAliased( + 4, this->query_channel_id(), output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:milvus.proto.etcd.TenantMeta) +} + +::PROTOBUF_NAMESPACE_ID::uint8* TenantMeta::InternalSerializeWithCachedSizesToArray( + ::PROTOBUF_NAMESPACE_ID::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:milvus.proto.etcd.TenantMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // uint64 id = 1; + if (this->id() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(1, this->id(), target); + } + + // uint64 num_query_nodes = 2; + if (this->num_query_nodes() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(2, this->num_query_nodes(), target); + } + + // repeated string insert_channel_ids = 3; + for (int i = 0, n = this->insert_channel_ids_size(); i < n; i++) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->insert_channel_ids(i).data(), static_cast(this->insert_channel_ids(i).length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.etcd.TenantMeta.insert_channel_ids"); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + WriteStringToArray(3, this->insert_channel_ids(i), target); + } + + // string query_channel_id = 4; + if (this->query_channel_id().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->query_channel_id().data(), static_cast(this->query_channel_id().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.etcd.TenantMeta.query_channel_id"); + target = + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringToArray( + 4, this->query_channel_id(), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:milvus.proto.etcd.TenantMeta) + return target; +} + +size_t TenantMeta::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:milvus.proto.etcd.TenantMeta) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // repeated string insert_channel_ids = 3; + total_size += 1 * + ::PROTOBUF_NAMESPACE_ID::internal::FromIntSize(this->insert_channel_ids_size()); + for (int i = 0, n = this->insert_channel_ids_size(); i < n; i++) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->insert_channel_ids(i)); + } + + // string query_channel_id = 4; + if (this->query_channel_id().size() > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->query_channel_id()); + } + + // uint64 id = 1; + if (this->id() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64Size( + this->id()); + } + + // uint64 num_query_nodes = 2; + if (this->num_query_nodes() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64Size( + this->num_query_nodes()); + } + + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void TenantMeta::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.etcd.TenantMeta) + GOOGLE_DCHECK_NE(&from, this); + const TenantMeta* source = + ::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.etcd.TenantMeta) + ::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.etcd.TenantMeta) + MergeFrom(*source); + } +} + +void TenantMeta::MergeFrom(const TenantMeta& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.etcd.TenantMeta) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + insert_channel_ids_.MergeFrom(from.insert_channel_ids_); + if (from.query_channel_id().size() > 0) { + + query_channel_id_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.query_channel_id_); + } + if (from.id() != 0) { + set_id(from.id()); + } + if (from.num_query_nodes() != 0) { + set_num_query_nodes(from.num_query_nodes()); + } +} + +void TenantMeta::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.etcd.TenantMeta) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void TenantMeta::CopyFrom(const TenantMeta& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.etcd.TenantMeta) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool TenantMeta::IsInitialized() const { + return true; +} + +void TenantMeta::InternalSwap(TenantMeta* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + insert_channel_ids_.InternalSwap(CastToBase(&other->insert_channel_ids_)); + query_channel_id_.Swap(&other->query_channel_id_, &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); + swap(id_, other->id_); + swap(num_query_nodes_, other->num_query_nodes_); +} + +::PROTOBUF_NAMESPACE_ID::Metadata TenantMeta::GetMetadata() const { + return GetMetadataStatic(); +} + + +// =================================================================== + +void ProxyMeta::InitAsDefaultInstance() { + ::milvus::proto::etcd::_ProxyMeta_default_instance_._instance.get_mutable()->address_ = const_cast< ::milvus::proto::common::Address*>( + ::milvus::proto::common::Address::internal_default_instance()); +} +class ProxyMeta::_Internal { + public: + static const ::milvus::proto::common::Address& address(const ProxyMeta* msg); +}; + +const ::milvus::proto::common::Address& +ProxyMeta::_Internal::address(const ProxyMeta* msg) { + return *msg->address_; +} +void ProxyMeta::clear_address() { + if (GetArenaNoVirtual() == nullptr && address_ != nullptr) { + delete address_; + } + address_ = nullptr; +} +ProxyMeta::ProxyMeta() + : ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:milvus.proto.etcd.ProxyMeta) +} +ProxyMeta::ProxyMeta(const ProxyMeta& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _internal_metadata_(nullptr), + result_channel_ids_(from.result_channel_ids_) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + if (from.has_address()) { + address_ = new ::milvus::proto::common::Address(*from.address_); + } else { + address_ = nullptr; + } + id_ = from.id_; + // @@protoc_insertion_point(copy_constructor:milvus.proto.etcd.ProxyMeta) +} + +void ProxyMeta::SharedCtor() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&scc_info_ProxyMeta_etcd_5fmeta_2eproto.base); + ::memset(&address_, 0, static_cast( + reinterpret_cast(&id_) - + reinterpret_cast(&address_)) + sizeof(id_)); +} + +ProxyMeta::~ProxyMeta() { + // @@protoc_insertion_point(destructor:milvus.proto.etcd.ProxyMeta) + SharedDtor(); +} + +void ProxyMeta::SharedDtor() { + if (this != internal_default_instance()) delete address_; +} + +void ProxyMeta::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const ProxyMeta& ProxyMeta::default_instance() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_ProxyMeta_etcd_5fmeta_2eproto.base); + return *internal_default_instance(); +} + + +void ProxyMeta::Clear() { +// @@protoc_insertion_point(message_clear_start:milvus.proto.etcd.ProxyMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + result_channel_ids_.Clear(); + if (GetArenaNoVirtual() == nullptr && address_ != nullptr) { + delete address_; + } + address_ = nullptr; + id_ = PROTOBUF_ULONGLONG(0); + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* ProxyMeta::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + CHK_(ptr); + switch (tag >> 3) { + // uint64 id = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 8)) { + id_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // .milvus.proto.common.Address address = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 18)) { + ptr = ctx->ParseMessage(mutable_address(), ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // repeated string result_channel_ids = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 26)) { + ptr -= 1; + do { + ptr += 1; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(add_result_channel_ids(), ptr, ctx, "milvus.proto.etcd.ProxyMeta.result_channel_ids"); + CHK_(ptr); + if (!ctx->DataAvailable(ptr)) break; + } while (::PROTOBUF_NAMESPACE_ID::internal::UnalignedLoad<::PROTOBUF_NAMESPACE_ID::uint8>(ptr) == 26); + } else goto handle_unusual; + continue; + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->SetLastTag(tag); + goto success; + } + ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx); + CHK_(ptr != nullptr); + continue; + } + } // switch + } // while +success: + return ptr; +failure: + ptr = nullptr; + goto success; +#undef CHK_ +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool ProxyMeta::MergePartialFromCodedStream( + ::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + // @@protoc_insertion_point(parse_start:milvus.proto.etcd.ProxyMeta) + for (;;) { + ::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // uint64 id = 1; + case 1: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (8 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::uint64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_UINT64>( + input, &id_))); + } else { + goto handle_unusual; + } + break; + } + + // .milvus.proto.common.Address address = 2; + case 2: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (18 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadMessage( + input, mutable_address())); + } else { + goto handle_unusual; + } + break; + } + + // repeated string result_channel_ids = 3; + case 3: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (26 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->add_result_channel_ids())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->result_channel_ids(this->result_channel_ids_size() - 1).data(), + static_cast(this->result_channel_ids(this->result_channel_ids_size() - 1).length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.etcd.ProxyMeta.result_channel_ids")); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:milvus.proto.etcd.ProxyMeta) + return true; +failure: + // @@protoc_insertion_point(parse_failure:milvus.proto.etcd.ProxyMeta) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void ProxyMeta::SerializeWithCachedSizes( + ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:milvus.proto.etcd.ProxyMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // uint64 id = 1; + if (this->id() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64(1, this->id(), output); + } + + // .milvus.proto.common.Address address = 2; + if (this->has_address()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteMessageMaybeToArray( + 2, _Internal::address(this), output); + } + + // repeated string result_channel_ids = 3; + for (int i = 0, n = this->result_channel_ids_size(); i < n; i++) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->result_channel_ids(i).data(), static_cast(this->result_channel_ids(i).length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.etcd.ProxyMeta.result_channel_ids"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteString( + 3, this->result_channel_ids(i), output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:milvus.proto.etcd.ProxyMeta) +} + +::PROTOBUF_NAMESPACE_ID::uint8* ProxyMeta::InternalSerializeWithCachedSizesToArray( + ::PROTOBUF_NAMESPACE_ID::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:milvus.proto.etcd.ProxyMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // uint64 id = 1; + if (this->id() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(1, this->id(), target); + } + + // .milvus.proto.common.Address address = 2; + if (this->has_address()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessageToArray( + 2, _Internal::address(this), target); + } + + // repeated string result_channel_ids = 3; + for (int i = 0, n = this->result_channel_ids_size(); i < n; i++) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->result_channel_ids(i).data(), static_cast(this->result_channel_ids(i).length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.etcd.ProxyMeta.result_channel_ids"); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + WriteStringToArray(3, this->result_channel_ids(i), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:milvus.proto.etcd.ProxyMeta) + return target; +} + +size_t ProxyMeta::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:milvus.proto.etcd.ProxyMeta) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // repeated string result_channel_ids = 3; + total_size += 1 * + ::PROTOBUF_NAMESPACE_ID::internal::FromIntSize(this->result_channel_ids_size()); + for (int i = 0, n = this->result_channel_ids_size(); i < n; i++) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->result_channel_ids(i)); + } + + // .milvus.proto.common.Address address = 2; + if (this->has_address()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *address_); + } + + // uint64 id = 1; + if (this->id() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64Size( + this->id()); + } + + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void ProxyMeta::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.etcd.ProxyMeta) + GOOGLE_DCHECK_NE(&from, this); + const ProxyMeta* source = + ::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.etcd.ProxyMeta) + ::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.etcd.ProxyMeta) + MergeFrom(*source); + } +} + +void ProxyMeta::MergeFrom(const ProxyMeta& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.etcd.ProxyMeta) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + result_channel_ids_.MergeFrom(from.result_channel_ids_); + if (from.has_address()) { + mutable_address()->::milvus::proto::common::Address::MergeFrom(from.address()); + } + if (from.id() != 0) { + set_id(from.id()); + } +} + +void ProxyMeta::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.etcd.ProxyMeta) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void ProxyMeta::CopyFrom(const ProxyMeta& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.etcd.ProxyMeta) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool ProxyMeta::IsInitialized() const { + return true; +} + +void ProxyMeta::InternalSwap(ProxyMeta* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + result_channel_ids_.InternalSwap(CastToBase(&other->result_channel_ids_)); + swap(address_, other->address_); + swap(id_, other->id_); +} + +::PROTOBUF_NAMESPACE_ID::Metadata ProxyMeta::GetMetadata() const { + return GetMetadataStatic(); +} + + +// =================================================================== + +void CollectionMeta::InitAsDefaultInstance() { + ::milvus::proto::etcd::_CollectionMeta_default_instance_._instance.get_mutable()->schema_ = const_cast< ::milvus::proto::schema::CollectionSchema*>( + ::milvus::proto::schema::CollectionSchema::internal_default_instance()); +} +class CollectionMeta::_Internal { + public: + static const ::milvus::proto::schema::CollectionSchema& schema(const CollectionMeta* msg); +}; + +const ::milvus::proto::schema::CollectionSchema& +CollectionMeta::_Internal::schema(const CollectionMeta* msg) { + return *msg->schema_; +} +void CollectionMeta::clear_schema() { + if (GetArenaNoVirtual() == nullptr && schema_ != nullptr) { + delete schema_; + } + schema_ = nullptr; +} +CollectionMeta::CollectionMeta() + : ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:milvus.proto.etcd.CollectionMeta) +} +CollectionMeta::CollectionMeta(const CollectionMeta& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _internal_metadata_(nullptr), + segment_ids_(from.segment_ids_), + partition_tags_(from.partition_tags_) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + if (from.has_schema()) { + schema_ = new ::milvus::proto::schema::CollectionSchema(*from.schema_); + } else { + schema_ = nullptr; + } + ::memcpy(&id_, &from.id_, + static_cast(reinterpret_cast(&create_time_) - + reinterpret_cast(&id_)) + sizeof(create_time_)); + // @@protoc_insertion_point(copy_constructor:milvus.proto.etcd.CollectionMeta) +} + +void CollectionMeta::SharedCtor() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&scc_info_CollectionMeta_etcd_5fmeta_2eproto.base); + ::memset(&schema_, 0, static_cast( + reinterpret_cast(&create_time_) - + reinterpret_cast(&schema_)) + sizeof(create_time_)); +} + +CollectionMeta::~CollectionMeta() { + // @@protoc_insertion_point(destructor:milvus.proto.etcd.CollectionMeta) + SharedDtor(); +} + +void CollectionMeta::SharedDtor() { + if (this != internal_default_instance()) delete schema_; +} + +void CollectionMeta::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const CollectionMeta& CollectionMeta::default_instance() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_CollectionMeta_etcd_5fmeta_2eproto.base); + return *internal_default_instance(); +} + + +void CollectionMeta::Clear() { +// @@protoc_insertion_point(message_clear_start:milvus.proto.etcd.CollectionMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + segment_ids_.Clear(); + partition_tags_.Clear(); + if (GetArenaNoVirtual() == nullptr && schema_ != nullptr) { + delete schema_; + } + schema_ = nullptr; + ::memset(&id_, 0, static_cast( + reinterpret_cast(&create_time_) - + reinterpret_cast(&id_)) + sizeof(create_time_)); + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* CollectionMeta::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + CHK_(ptr); + switch (tag >> 3) { + // uint64 id = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 8)) { + id_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // .milvus.proto.schema.CollectionSchema schema = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 18)) { + ptr = ctx->ParseMessage(mutable_schema(), ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // uint64 create_time = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 24)) { + create_time_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // repeated uint64 segment_ids = 4; + case 4: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 34)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::PackedUInt64Parser(mutable_segment_ids(), ptr, ctx); + CHK_(ptr); + } else if (static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 32) { + add_segment_ids(::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr)); + CHK_(ptr); + } else goto handle_unusual; + continue; + // repeated string partition_tags = 5; + case 5: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 42)) { + ptr -= 1; + do { + ptr += 1; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(add_partition_tags(), ptr, ctx, "milvus.proto.etcd.CollectionMeta.partition_tags"); + CHK_(ptr); + if (!ctx->DataAvailable(ptr)) break; + } while (::PROTOBUF_NAMESPACE_ID::internal::UnalignedLoad<::PROTOBUF_NAMESPACE_ID::uint8>(ptr) == 42); + } else goto handle_unusual; + continue; + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->SetLastTag(tag); + goto success; + } + ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx); + CHK_(ptr != nullptr); + continue; + } + } // switch + } // while +success: + return ptr; +failure: + ptr = nullptr; + goto success; +#undef CHK_ +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool CollectionMeta::MergePartialFromCodedStream( + ::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + // @@protoc_insertion_point(parse_start:milvus.proto.etcd.CollectionMeta) + for (;;) { + ::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // uint64 id = 1; + case 1: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (8 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::uint64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_UINT64>( + input, &id_))); + } else { + goto handle_unusual; + } + break; + } + + // .milvus.proto.schema.CollectionSchema schema = 2; + case 2: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (18 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadMessage( + input, mutable_schema())); + } else { + goto handle_unusual; + } + break; + } + + // uint64 create_time = 3; + case 3: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (24 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::uint64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_UINT64>( + input, &create_time_))); + } else { + goto handle_unusual; + } + break; + } + + // repeated uint64 segment_ids = 4; + case 4: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (34 & 0xFF)) { + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPackedPrimitive< + ::PROTOBUF_NAMESPACE_ID::uint64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_UINT64>( + input, this->mutable_segment_ids()))); + } else if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (32 & 0xFF)) { + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadRepeatedPrimitiveNoInline< + ::PROTOBUF_NAMESPACE_ID::uint64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_UINT64>( + 1, 34u, input, this->mutable_segment_ids()))); + } else { + goto handle_unusual; + } + break; + } + + // repeated string partition_tags = 5; + case 5: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (42 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->add_partition_tags())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->partition_tags(this->partition_tags_size() - 1).data(), + static_cast(this->partition_tags(this->partition_tags_size() - 1).length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.etcd.CollectionMeta.partition_tags")); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:milvus.proto.etcd.CollectionMeta) + return true; +failure: + // @@protoc_insertion_point(parse_failure:milvus.proto.etcd.CollectionMeta) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void CollectionMeta::SerializeWithCachedSizes( + ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:milvus.proto.etcd.CollectionMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // uint64 id = 1; + if (this->id() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64(1, this->id(), output); + } + + // .milvus.proto.schema.CollectionSchema schema = 2; + if (this->has_schema()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteMessageMaybeToArray( + 2, _Internal::schema(this), output); + } + + // uint64 create_time = 3; + if (this->create_time() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64(3, this->create_time(), output); + } + + // repeated uint64 segment_ids = 4; + if (this->segment_ids_size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteTag(4, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED, output); + output->WriteVarint32(_segment_ids_cached_byte_size_.load( + std::memory_order_relaxed)); + } + for (int i = 0, n = this->segment_ids_size(); i < n; i++) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64NoTag( + this->segment_ids(i), output); + } + + // repeated string partition_tags = 5; + for (int i = 0, n = this->partition_tags_size(); i < n; i++) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->partition_tags(i).data(), static_cast(this->partition_tags(i).length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.etcd.CollectionMeta.partition_tags"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteString( + 5, this->partition_tags(i), output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:milvus.proto.etcd.CollectionMeta) +} + +::PROTOBUF_NAMESPACE_ID::uint8* CollectionMeta::InternalSerializeWithCachedSizesToArray( + ::PROTOBUF_NAMESPACE_ID::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:milvus.proto.etcd.CollectionMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // uint64 id = 1; + if (this->id() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(1, this->id(), target); + } + + // .milvus.proto.schema.CollectionSchema schema = 2; + if (this->has_schema()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessageToArray( + 2, _Internal::schema(this), target); + } + + // uint64 create_time = 3; + if (this->create_time() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(3, this->create_time(), target); + } + + // repeated uint64 segment_ids = 4; + if (this->segment_ids_size() > 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteTagToArray( + 4, + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WIRETYPE_LENGTH_DELIMITED, + target); + target = ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream::WriteVarint32ToArray( + _segment_ids_cached_byte_size_.load(std::memory_order_relaxed), + target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + WriteUInt64NoTagToArray(this->segment_ids_, target); + } + + // repeated string partition_tags = 5; + for (int i = 0, n = this->partition_tags_size(); i < n; i++) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->partition_tags(i).data(), static_cast(this->partition_tags(i).length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.etcd.CollectionMeta.partition_tags"); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + WriteStringToArray(5, this->partition_tags(i), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:milvus.proto.etcd.CollectionMeta) + return target; +} + +size_t CollectionMeta::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:milvus.proto.etcd.CollectionMeta) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // repeated uint64 segment_ids = 4; + { + size_t data_size = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + UInt64Size(this->segment_ids_); + if (data_size > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32Size( + static_cast<::PROTOBUF_NAMESPACE_ID::int32>(data_size)); + } + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(data_size); + _segment_ids_cached_byte_size_.store(cached_size, + std::memory_order_relaxed); + total_size += data_size; + } + + // repeated string partition_tags = 5; + total_size += 1 * + ::PROTOBUF_NAMESPACE_ID::internal::FromIntSize(this->partition_tags_size()); + for (int i = 0, n = this->partition_tags_size(); i < n; i++) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->partition_tags(i)); + } + + // .milvus.proto.schema.CollectionSchema schema = 2; + if (this->has_schema()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *schema_); + } + + // uint64 id = 1; + if (this->id() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64Size( + this->id()); + } + + // uint64 create_time = 3; + if (this->create_time() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64Size( + this->create_time()); + } + + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void CollectionMeta::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.etcd.CollectionMeta) + GOOGLE_DCHECK_NE(&from, this); + const CollectionMeta* source = + ::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.etcd.CollectionMeta) + ::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.etcd.CollectionMeta) + MergeFrom(*source); + } +} + +void CollectionMeta::MergeFrom(const CollectionMeta& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.etcd.CollectionMeta) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + segment_ids_.MergeFrom(from.segment_ids_); + partition_tags_.MergeFrom(from.partition_tags_); + if (from.has_schema()) { + mutable_schema()->::milvus::proto::schema::CollectionSchema::MergeFrom(from.schema()); + } + if (from.id() != 0) { + set_id(from.id()); + } + if (from.create_time() != 0) { + set_create_time(from.create_time()); + } +} + +void CollectionMeta::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.etcd.CollectionMeta) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void CollectionMeta::CopyFrom(const CollectionMeta& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.etcd.CollectionMeta) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool CollectionMeta::IsInitialized() const { + return true; +} + +void CollectionMeta::InternalSwap(CollectionMeta* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + segment_ids_.InternalSwap(&other->segment_ids_); + partition_tags_.InternalSwap(CastToBase(&other->partition_tags_)); + swap(schema_, other->schema_); + swap(id_, other->id_); + swap(create_time_, other->create_time_); +} + +::PROTOBUF_NAMESPACE_ID::Metadata CollectionMeta::GetMetadata() const { + return GetMetadataStatic(); +} + + +// =================================================================== + +void SegmentMeta::InitAsDefaultInstance() { +} +class SegmentMeta::_Internal { + public: +}; + +SegmentMeta::SegmentMeta() + : ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:milvus.proto.etcd.SegmentMeta) +} +SegmentMeta::SegmentMeta(const SegmentMeta& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _internal_metadata_(nullptr) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + partition_tag_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + if (!from.partition_tag().empty()) { + partition_tag_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.partition_tag_); + } + ::memcpy(&segment_id_, &from.segment_id_, + static_cast(reinterpret_cast(&num_rows_) - + reinterpret_cast(&segment_id_)) + sizeof(num_rows_)); + // @@protoc_insertion_point(copy_constructor:milvus.proto.etcd.SegmentMeta) +} + +void SegmentMeta::SharedCtor() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&scc_info_SegmentMeta_etcd_5fmeta_2eproto.base); + partition_tag_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + ::memset(&segment_id_, 0, static_cast( + reinterpret_cast(&num_rows_) - + reinterpret_cast(&segment_id_)) + sizeof(num_rows_)); +} + +SegmentMeta::~SegmentMeta() { + // @@protoc_insertion_point(destructor:milvus.proto.etcd.SegmentMeta) + SharedDtor(); +} + +void SegmentMeta::SharedDtor() { + partition_tag_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +void SegmentMeta::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const SegmentMeta& SegmentMeta::default_instance() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_SegmentMeta_etcd_5fmeta_2eproto.base); + return *internal_default_instance(); +} + + +void SegmentMeta::Clear() { +// @@protoc_insertion_point(message_clear_start:milvus.proto.etcd.SegmentMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + partition_tag_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + ::memset(&segment_id_, 0, static_cast( + reinterpret_cast(&num_rows_) - + reinterpret_cast(&segment_id_)) + sizeof(num_rows_)); + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* SegmentMeta::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + CHK_(ptr); + switch (tag >> 3) { + // uint64 segment_id = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 8)) { + segment_id_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // uint64 collection_id = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 16)) { + collection_id_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // string partition_tag = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 26)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(mutable_partition_tag(), ptr, ctx, "milvus.proto.etcd.SegmentMeta.partition_tag"); + CHK_(ptr); + } else goto handle_unusual; + continue; + // int32 channel_start = 4; + case 4: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 32)) { + channel_start_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // int32 channel_end = 5; + case 5: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 40)) { + channel_end_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // uint64 open_time = 6; + case 6: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 48)) { + open_time_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // uint64 close_time = 7; + case 7: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 56)) { + close_time_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // int64 num_rows = 8; + case 8: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 64)) { + num_rows_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->SetLastTag(tag); + goto success; + } + ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx); + CHK_(ptr != nullptr); + continue; + } + } // switch + } // while +success: + return ptr; +failure: + ptr = nullptr; + goto success; +#undef CHK_ +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool SegmentMeta::MergePartialFromCodedStream( + ::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + // @@protoc_insertion_point(parse_start:milvus.proto.etcd.SegmentMeta) + for (;;) { + ::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // uint64 segment_id = 1; + case 1: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (8 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::uint64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_UINT64>( + input, &segment_id_))); + } else { + goto handle_unusual; + } + break; + } + + // uint64 collection_id = 2; + case 2: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (16 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::uint64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_UINT64>( + input, &collection_id_))); + } else { + goto handle_unusual; + } + break; + } + + // string partition_tag = 3; + case 3: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (26 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->mutable_partition_tag())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->partition_tag().data(), static_cast(this->partition_tag().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.etcd.SegmentMeta.partition_tag")); + } else { + goto handle_unusual; + } + break; + } + + // int32 channel_start = 4; + case 4: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (32 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::int32, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_INT32>( + input, &channel_start_))); + } else { + goto handle_unusual; + } + break; + } + + // int32 channel_end = 5; + case 5: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (40 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::int32, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_INT32>( + input, &channel_end_))); + } else { + goto handle_unusual; + } + break; + } + + // uint64 open_time = 6; + case 6: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (48 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::uint64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_UINT64>( + input, &open_time_))); + } else { + goto handle_unusual; + } + break; + } + + // uint64 close_time = 7; + case 7: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (56 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::uint64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_UINT64>( + input, &close_time_))); + } else { + goto handle_unusual; + } + break; + } + + // int64 num_rows = 8; + case 8: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (64 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + ::PROTOBUF_NAMESPACE_ID::int64, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_INT64>( + input, &num_rows_))); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:milvus.proto.etcd.SegmentMeta) + return true; +failure: + // @@protoc_insertion_point(parse_failure:milvus.proto.etcd.SegmentMeta) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void SegmentMeta::SerializeWithCachedSizes( + ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:milvus.proto.etcd.SegmentMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // uint64 segment_id = 1; + if (this->segment_id() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64(1, this->segment_id(), output); + } + + // uint64 collection_id = 2; + if (this->collection_id() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64(2, this->collection_id(), output); + } + + // string partition_tag = 3; + if (this->partition_tag().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->partition_tag().data(), static_cast(this->partition_tag().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.etcd.SegmentMeta.partition_tag"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringMaybeAliased( + 3, this->partition_tag(), output); + } + + // int32 channel_start = 4; + if (this->channel_start() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32(4, this->channel_start(), output); + } + + // int32 channel_end = 5; + if (this->channel_end() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32(5, this->channel_end(), output); + } + + // uint64 open_time = 6; + if (this->open_time() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64(6, this->open_time(), output); + } + + // uint64 close_time = 7; + if (this->close_time() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64(7, this->close_time(), output); + } + + // int64 num_rows = 8; + if (this->num_rows() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt64(8, this->num_rows(), output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:milvus.proto.etcd.SegmentMeta) +} + +::PROTOBUF_NAMESPACE_ID::uint8* SegmentMeta::InternalSerializeWithCachedSizesToArray( + ::PROTOBUF_NAMESPACE_ID::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:milvus.proto.etcd.SegmentMeta) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // uint64 segment_id = 1; + if (this->segment_id() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(1, this->segment_id(), target); + } + + // uint64 collection_id = 2; + if (this->collection_id() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(2, this->collection_id(), target); + } + + // string partition_tag = 3; + if (this->partition_tag().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->partition_tag().data(), static_cast(this->partition_tag().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.etcd.SegmentMeta.partition_tag"); + target = + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringToArray( + 3, this->partition_tag(), target); + } + + // int32 channel_start = 4; + if (this->channel_start() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(4, this->channel_start(), target); + } + + // int32 channel_end = 5; + if (this->channel_end() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(5, this->channel_end(), target); + } + + // uint64 open_time = 6; + if (this->open_time() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(6, this->open_time(), target); + } + + // uint64 close_time = 7; + if (this->close_time() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(7, this->close_time(), target); + } + + // int64 num_rows = 8; + if (this->num_rows() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt64ToArray(8, this->num_rows(), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:milvus.proto.etcd.SegmentMeta) + return target; +} + +size_t SegmentMeta::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:milvus.proto.etcd.SegmentMeta) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // string partition_tag = 3; + if (this->partition_tag().size() > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->partition_tag()); + } + + // uint64 segment_id = 1; + if (this->segment_id() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64Size( + this->segment_id()); + } + + // uint64 collection_id = 2; + if (this->collection_id() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64Size( + this->collection_id()); + } + + // int32 channel_start = 4; + if (this->channel_start() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32Size( + this->channel_start()); + } + + // int32 channel_end = 5; + if (this->channel_end() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32Size( + this->channel_end()); + } + + // uint64 open_time = 6; + if (this->open_time() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64Size( + this->open_time()); + } + + // uint64 close_time = 7; + if (this->close_time() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64Size( + this->close_time()); + } + + // int64 num_rows = 8; + if (this->num_rows() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int64Size( + this->num_rows()); + } + + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void SegmentMeta::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.etcd.SegmentMeta) + GOOGLE_DCHECK_NE(&from, this); + const SegmentMeta* source = + ::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.etcd.SegmentMeta) + ::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.etcd.SegmentMeta) + MergeFrom(*source); + } +} + +void SegmentMeta::MergeFrom(const SegmentMeta& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.etcd.SegmentMeta) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + if (from.partition_tag().size() > 0) { + + partition_tag_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.partition_tag_); + } + if (from.segment_id() != 0) { + set_segment_id(from.segment_id()); + } + if (from.collection_id() != 0) { + set_collection_id(from.collection_id()); + } + if (from.channel_start() != 0) { + set_channel_start(from.channel_start()); + } + if (from.channel_end() != 0) { + set_channel_end(from.channel_end()); + } + if (from.open_time() != 0) { + set_open_time(from.open_time()); + } + if (from.close_time() != 0) { + set_close_time(from.close_time()); + } + if (from.num_rows() != 0) { + set_num_rows(from.num_rows()); + } +} + +void SegmentMeta::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.etcd.SegmentMeta) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void SegmentMeta::CopyFrom(const SegmentMeta& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.etcd.SegmentMeta) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool SegmentMeta::IsInitialized() const { + return true; +} + +void SegmentMeta::InternalSwap(SegmentMeta* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + partition_tag_.Swap(&other->partition_tag_, &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); + swap(segment_id_, other->segment_id_); + swap(collection_id_, other->collection_id_); + swap(channel_start_, other->channel_start_); + swap(channel_end_, other->channel_end_); + swap(open_time_, other->open_time_); + swap(close_time_, other->close_time_); + swap(num_rows_, other->num_rows_); +} + +::PROTOBUF_NAMESPACE_ID::Metadata SegmentMeta::GetMetadata() const { + return GetMetadataStatic(); +} + + +// @@protoc_insertion_point(namespace_scope) +} // namespace etcd +} // namespace proto +} // namespace milvus +PROTOBUF_NAMESPACE_OPEN +template<> PROTOBUF_NOINLINE ::milvus::proto::etcd::TenantMeta* Arena::CreateMaybeMessage< ::milvus::proto::etcd::TenantMeta >(Arena* arena) { + return Arena::CreateInternal< ::milvus::proto::etcd::TenantMeta >(arena); +} +template<> PROTOBUF_NOINLINE ::milvus::proto::etcd::ProxyMeta* Arena::CreateMaybeMessage< ::milvus::proto::etcd::ProxyMeta >(Arena* arena) { + return Arena::CreateInternal< ::milvus::proto::etcd::ProxyMeta >(arena); +} +template<> PROTOBUF_NOINLINE ::milvus::proto::etcd::CollectionMeta* Arena::CreateMaybeMessage< ::milvus::proto::etcd::CollectionMeta >(Arena* arena) { + return Arena::CreateInternal< ::milvus::proto::etcd::CollectionMeta >(arena); +} +template<> PROTOBUF_NOINLINE ::milvus::proto::etcd::SegmentMeta* Arena::CreateMaybeMessage< ::milvus::proto::etcd::SegmentMeta >(Arena* arena) { + return Arena::CreateInternal< ::milvus::proto::etcd::SegmentMeta >(arena); +} +PROTOBUF_NAMESPACE_CLOSE + +// @@protoc_insertion_point(global_scope) +#include diff --git a/internal/core/src/pb/etcd_meta.pb.h b/internal/core/src/pb/etcd_meta.pb.h new file mode 100644 index 0000000000..49644be7c0 --- /dev/null +++ b/internal/core/src/pb/etcd_meta.pb.h @@ -0,0 +1,1411 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: etcd_meta.proto + +#ifndef GOOGLE_PROTOBUF_INCLUDED_etcd_5fmeta_2eproto +#define GOOGLE_PROTOBUF_INCLUDED_etcd_5fmeta_2eproto + +#include +#include + +#include +#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 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include // IWYU pragma: export +#include // IWYU pragma: export +#include +#include "common.pb.h" +#include "schema.pb.h" +// @@protoc_insertion_point(includes) +#include +#define PROTOBUF_INTERNAL_EXPORT_etcd_5fmeta_2eproto +PROTOBUF_NAMESPACE_OPEN +namespace internal { +class AnyMetadata; +} // namespace internal +PROTOBUF_NAMESPACE_CLOSE + +// Internal implementation detail -- do not use these members. +struct TableStruct_etcd_5fmeta_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[4] + 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_etcd_5fmeta_2eproto; +namespace milvus { +namespace proto { +namespace etcd { +class CollectionMeta; +class CollectionMetaDefaultTypeInternal; +extern CollectionMetaDefaultTypeInternal _CollectionMeta_default_instance_; +class ProxyMeta; +class ProxyMetaDefaultTypeInternal; +extern ProxyMetaDefaultTypeInternal _ProxyMeta_default_instance_; +class SegmentMeta; +class SegmentMetaDefaultTypeInternal; +extern SegmentMetaDefaultTypeInternal _SegmentMeta_default_instance_; +class TenantMeta; +class TenantMetaDefaultTypeInternal; +extern TenantMetaDefaultTypeInternal _TenantMeta_default_instance_; +} // namespace etcd +} // namespace proto +} // namespace milvus +PROTOBUF_NAMESPACE_OPEN +template<> ::milvus::proto::etcd::CollectionMeta* Arena::CreateMaybeMessage<::milvus::proto::etcd::CollectionMeta>(Arena*); +template<> ::milvus::proto::etcd::ProxyMeta* Arena::CreateMaybeMessage<::milvus::proto::etcd::ProxyMeta>(Arena*); +template<> ::milvus::proto::etcd::SegmentMeta* Arena::CreateMaybeMessage<::milvus::proto::etcd::SegmentMeta>(Arena*); +template<> ::milvus::proto::etcd::TenantMeta* Arena::CreateMaybeMessage<::milvus::proto::etcd::TenantMeta>(Arena*); +PROTOBUF_NAMESPACE_CLOSE +namespace milvus { +namespace proto { +namespace etcd { + +// =================================================================== + +class TenantMeta : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.etcd.TenantMeta) */ { + public: + TenantMeta(); + virtual ~TenantMeta(); + + TenantMeta(const TenantMeta& from); + TenantMeta(TenantMeta&& from) noexcept + : TenantMeta() { + *this = ::std::move(from); + } + + inline TenantMeta& operator=(const TenantMeta& from) { + CopyFrom(from); + return *this; + } + inline TenantMeta& operator=(TenantMeta&& 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 TenantMeta& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const TenantMeta* internal_default_instance() { + return reinterpret_cast( + &_TenantMeta_default_instance_); + } + static constexpr int kIndexInFileMessages = + 0; + + friend void swap(TenantMeta& a, TenantMeta& b) { + a.Swap(&b); + } + inline void Swap(TenantMeta* other) { + if (other == this) return; + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + inline TenantMeta* New() const final { + return CreateMaybeMessage(nullptr); + } + + TenantMeta* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void CopyFrom(const TenantMeta& from); + void MergeFrom(const TenantMeta& 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(TenantMeta* other); + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "milvus.proto.etcd.TenantMeta"; + } + 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_etcd_5fmeta_2eproto); + return ::descriptor_table_etcd_5fmeta_2eproto.file_level_metadata[kIndexInFileMessages]; + } + + public: + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kInsertChannelIdsFieldNumber = 3, + kQueryChannelIdFieldNumber = 4, + kIdFieldNumber = 1, + kNumQueryNodesFieldNumber = 2, + }; + // repeated string insert_channel_ids = 3; + int insert_channel_ids_size() const; + void clear_insert_channel_ids(); + const std::string& insert_channel_ids(int index) const; + std::string* mutable_insert_channel_ids(int index); + void set_insert_channel_ids(int index, const std::string& value); + void set_insert_channel_ids(int index, std::string&& value); + void set_insert_channel_ids(int index, const char* value); + void set_insert_channel_ids(int index, const char* value, size_t size); + std::string* add_insert_channel_ids(); + void add_insert_channel_ids(const std::string& value); + void add_insert_channel_ids(std::string&& value); + void add_insert_channel_ids(const char* value); + void add_insert_channel_ids(const char* value, size_t size); + const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField& insert_channel_ids() const; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField* mutable_insert_channel_ids(); + + // string query_channel_id = 4; + void clear_query_channel_id(); + const std::string& query_channel_id() const; + void set_query_channel_id(const std::string& value); + void set_query_channel_id(std::string&& value); + void set_query_channel_id(const char* value); + void set_query_channel_id(const char* value, size_t size); + std::string* mutable_query_channel_id(); + std::string* release_query_channel_id(); + void set_allocated_query_channel_id(std::string* query_channel_id); + + // uint64 id = 1; + void clear_id(); + ::PROTOBUF_NAMESPACE_ID::uint64 id() const; + void set_id(::PROTOBUF_NAMESPACE_ID::uint64 value); + + // uint64 num_query_nodes = 2; + void clear_num_query_nodes(); + ::PROTOBUF_NAMESPACE_ID::uint64 num_query_nodes() const; + void set_num_query_nodes(::PROTOBUF_NAMESPACE_ID::uint64 value); + + // @@protoc_insertion_point(class_scope:milvus.proto.etcd.TenantMeta) + private: + class _Internal; + + ::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField insert_channel_ids_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr query_channel_id_; + ::PROTOBUF_NAMESPACE_ID::uint64 id_; + ::PROTOBUF_NAMESPACE_ID::uint64 num_query_nodes_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_etcd_5fmeta_2eproto; +}; +// ------------------------------------------------------------------- + +class ProxyMeta : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.etcd.ProxyMeta) */ { + public: + ProxyMeta(); + virtual ~ProxyMeta(); + + ProxyMeta(const ProxyMeta& from); + ProxyMeta(ProxyMeta&& from) noexcept + : ProxyMeta() { + *this = ::std::move(from); + } + + inline ProxyMeta& operator=(const ProxyMeta& from) { + CopyFrom(from); + return *this; + } + inline ProxyMeta& operator=(ProxyMeta&& 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 ProxyMeta& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const ProxyMeta* internal_default_instance() { + return reinterpret_cast( + &_ProxyMeta_default_instance_); + } + static constexpr int kIndexInFileMessages = + 1; + + friend void swap(ProxyMeta& a, ProxyMeta& b) { + a.Swap(&b); + } + inline void Swap(ProxyMeta* other) { + if (other == this) return; + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + inline ProxyMeta* New() const final { + return CreateMaybeMessage(nullptr); + } + + ProxyMeta* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void CopyFrom(const ProxyMeta& from); + void MergeFrom(const ProxyMeta& 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(ProxyMeta* other); + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "milvus.proto.etcd.ProxyMeta"; + } + 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_etcd_5fmeta_2eproto); + return ::descriptor_table_etcd_5fmeta_2eproto.file_level_metadata[kIndexInFileMessages]; + } + + public: + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kResultChannelIdsFieldNumber = 3, + kAddressFieldNumber = 2, + kIdFieldNumber = 1, + }; + // repeated string result_channel_ids = 3; + int result_channel_ids_size() const; + void clear_result_channel_ids(); + const std::string& result_channel_ids(int index) const; + std::string* mutable_result_channel_ids(int index); + void set_result_channel_ids(int index, const std::string& value); + void set_result_channel_ids(int index, std::string&& value); + void set_result_channel_ids(int index, const char* value); + void set_result_channel_ids(int index, const char* value, size_t size); + std::string* add_result_channel_ids(); + void add_result_channel_ids(const std::string& value); + void add_result_channel_ids(std::string&& value); + void add_result_channel_ids(const char* value); + void add_result_channel_ids(const char* value, size_t size); + const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField& result_channel_ids() const; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField* mutable_result_channel_ids(); + + // .milvus.proto.common.Address address = 2; + bool has_address() const; + void clear_address(); + const ::milvus::proto::common::Address& address() const; + ::milvus::proto::common::Address* release_address(); + ::milvus::proto::common::Address* mutable_address(); + void set_allocated_address(::milvus::proto::common::Address* address); + + // uint64 id = 1; + void clear_id(); + ::PROTOBUF_NAMESPACE_ID::uint64 id() const; + void set_id(::PROTOBUF_NAMESPACE_ID::uint64 value); + + // @@protoc_insertion_point(class_scope:milvus.proto.etcd.ProxyMeta) + private: + class _Internal; + + ::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField result_channel_ids_; + ::milvus::proto::common::Address* address_; + ::PROTOBUF_NAMESPACE_ID::uint64 id_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_etcd_5fmeta_2eproto; +}; +// ------------------------------------------------------------------- + +class CollectionMeta : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.etcd.CollectionMeta) */ { + public: + CollectionMeta(); + virtual ~CollectionMeta(); + + CollectionMeta(const CollectionMeta& from); + CollectionMeta(CollectionMeta&& from) noexcept + : CollectionMeta() { + *this = ::std::move(from); + } + + inline CollectionMeta& operator=(const CollectionMeta& from) { + CopyFrom(from); + return *this; + } + inline CollectionMeta& operator=(CollectionMeta&& 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 CollectionMeta& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const CollectionMeta* internal_default_instance() { + return reinterpret_cast( + &_CollectionMeta_default_instance_); + } + static constexpr int kIndexInFileMessages = + 2; + + friend void swap(CollectionMeta& a, CollectionMeta& b) { + a.Swap(&b); + } + inline void Swap(CollectionMeta* other) { + if (other == this) return; + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + inline CollectionMeta* New() const final { + return CreateMaybeMessage(nullptr); + } + + CollectionMeta* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void CopyFrom(const CollectionMeta& from); + void MergeFrom(const CollectionMeta& 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(CollectionMeta* other); + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "milvus.proto.etcd.CollectionMeta"; + } + 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_etcd_5fmeta_2eproto); + return ::descriptor_table_etcd_5fmeta_2eproto.file_level_metadata[kIndexInFileMessages]; + } + + public: + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kSegmentIdsFieldNumber = 4, + kPartitionTagsFieldNumber = 5, + kSchemaFieldNumber = 2, + kIdFieldNumber = 1, + kCreateTimeFieldNumber = 3, + }; + // repeated uint64 segment_ids = 4; + int segment_ids_size() const; + void clear_segment_ids(); + ::PROTOBUF_NAMESPACE_ID::uint64 segment_ids(int index) const; + void set_segment_ids(int index, ::PROTOBUF_NAMESPACE_ID::uint64 value); + void add_segment_ids(::PROTOBUF_NAMESPACE_ID::uint64 value); + const ::PROTOBUF_NAMESPACE_ID::RepeatedField< ::PROTOBUF_NAMESPACE_ID::uint64 >& + segment_ids() const; + ::PROTOBUF_NAMESPACE_ID::RepeatedField< ::PROTOBUF_NAMESPACE_ID::uint64 >* + mutable_segment_ids(); + + // repeated string partition_tags = 5; + int partition_tags_size() const; + void clear_partition_tags(); + const std::string& partition_tags(int index) const; + std::string* mutable_partition_tags(int index); + void set_partition_tags(int index, const std::string& value); + void set_partition_tags(int index, std::string&& value); + void set_partition_tags(int index, const char* value); + void set_partition_tags(int index, const char* value, size_t size); + std::string* add_partition_tags(); + void add_partition_tags(const std::string& value); + void add_partition_tags(std::string&& value); + void add_partition_tags(const char* value); + void add_partition_tags(const char* value, size_t size); + const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField& partition_tags() const; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField* mutable_partition_tags(); + + // .milvus.proto.schema.CollectionSchema schema = 2; + bool has_schema() const; + void clear_schema(); + const ::milvus::proto::schema::CollectionSchema& schema() const; + ::milvus::proto::schema::CollectionSchema* release_schema(); + ::milvus::proto::schema::CollectionSchema* mutable_schema(); + void set_allocated_schema(::milvus::proto::schema::CollectionSchema* schema); + + // uint64 id = 1; + void clear_id(); + ::PROTOBUF_NAMESPACE_ID::uint64 id() const; + void set_id(::PROTOBUF_NAMESPACE_ID::uint64 value); + + // uint64 create_time = 3; + void clear_create_time(); + ::PROTOBUF_NAMESPACE_ID::uint64 create_time() const; + void set_create_time(::PROTOBUF_NAMESPACE_ID::uint64 value); + + // @@protoc_insertion_point(class_scope:milvus.proto.etcd.CollectionMeta) + private: + class _Internal; + + ::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_; + ::PROTOBUF_NAMESPACE_ID::RepeatedField< ::PROTOBUF_NAMESPACE_ID::uint64 > segment_ids_; + mutable std::atomic _segment_ids_cached_byte_size_; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField partition_tags_; + ::milvus::proto::schema::CollectionSchema* schema_; + ::PROTOBUF_NAMESPACE_ID::uint64 id_; + ::PROTOBUF_NAMESPACE_ID::uint64 create_time_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_etcd_5fmeta_2eproto; +}; +// ------------------------------------------------------------------- + +class SegmentMeta : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:milvus.proto.etcd.SegmentMeta) */ { + public: + SegmentMeta(); + virtual ~SegmentMeta(); + + SegmentMeta(const SegmentMeta& from); + SegmentMeta(SegmentMeta&& from) noexcept + : SegmentMeta() { + *this = ::std::move(from); + } + + inline SegmentMeta& operator=(const SegmentMeta& from) { + CopyFrom(from); + return *this; + } + inline SegmentMeta& operator=(SegmentMeta&& 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 SegmentMeta& default_instance(); + + static void InitAsDefaultInstance(); // FOR INTERNAL USE ONLY + static inline const SegmentMeta* internal_default_instance() { + return reinterpret_cast( + &_SegmentMeta_default_instance_); + } + static constexpr int kIndexInFileMessages = + 3; + + friend void swap(SegmentMeta& a, SegmentMeta& b) { + a.Swap(&b); + } + inline void Swap(SegmentMeta* other) { + if (other == this) return; + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + inline SegmentMeta* New() const final { + return CreateMaybeMessage(nullptr); + } + + SegmentMeta* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final { + return CreateMaybeMessage(arena); + } + void CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) final; + void CopyFrom(const SegmentMeta& from); + void MergeFrom(const SegmentMeta& 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(SegmentMeta* other); + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "milvus.proto.etcd.SegmentMeta"; + } + 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_etcd_5fmeta_2eproto); + return ::descriptor_table_etcd_5fmeta_2eproto.file_level_metadata[kIndexInFileMessages]; + } + + public: + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kPartitionTagFieldNumber = 3, + kSegmentIdFieldNumber = 1, + kCollectionIdFieldNumber = 2, + kChannelStartFieldNumber = 4, + kChannelEndFieldNumber = 5, + kOpenTimeFieldNumber = 6, + kCloseTimeFieldNumber = 7, + kNumRowsFieldNumber = 8, + }; + // string partition_tag = 3; + void clear_partition_tag(); + const std::string& partition_tag() const; + void set_partition_tag(const std::string& value); + void set_partition_tag(std::string&& value); + void set_partition_tag(const char* value); + void set_partition_tag(const char* value, size_t size); + std::string* mutable_partition_tag(); + std::string* release_partition_tag(); + void set_allocated_partition_tag(std::string* partition_tag); + + // uint64 segment_id = 1; + void clear_segment_id(); + ::PROTOBUF_NAMESPACE_ID::uint64 segment_id() const; + void set_segment_id(::PROTOBUF_NAMESPACE_ID::uint64 value); + + // uint64 collection_id = 2; + void clear_collection_id(); + ::PROTOBUF_NAMESPACE_ID::uint64 collection_id() const; + void set_collection_id(::PROTOBUF_NAMESPACE_ID::uint64 value); + + // int32 channel_start = 4; + void clear_channel_start(); + ::PROTOBUF_NAMESPACE_ID::int32 channel_start() const; + void set_channel_start(::PROTOBUF_NAMESPACE_ID::int32 value); + + // int32 channel_end = 5; + void clear_channel_end(); + ::PROTOBUF_NAMESPACE_ID::int32 channel_end() const; + void set_channel_end(::PROTOBUF_NAMESPACE_ID::int32 value); + + // uint64 open_time = 6; + void clear_open_time(); + ::PROTOBUF_NAMESPACE_ID::uint64 open_time() const; + void set_open_time(::PROTOBUF_NAMESPACE_ID::uint64 value); + + // uint64 close_time = 7; + void clear_close_time(); + ::PROTOBUF_NAMESPACE_ID::uint64 close_time() const; + void set_close_time(::PROTOBUF_NAMESPACE_ID::uint64 value); + + // int64 num_rows = 8; + void clear_num_rows(); + ::PROTOBUF_NAMESPACE_ID::int64 num_rows() const; + void set_num_rows(::PROTOBUF_NAMESPACE_ID::int64 value); + + // @@protoc_insertion_point(class_scope:milvus.proto.etcd.SegmentMeta) + private: + class _Internal; + + ::PROTOBUF_NAMESPACE_ID::internal::InternalMetadataWithArena _internal_metadata_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr partition_tag_; + ::PROTOBUF_NAMESPACE_ID::uint64 segment_id_; + ::PROTOBUF_NAMESPACE_ID::uint64 collection_id_; + ::PROTOBUF_NAMESPACE_ID::int32 channel_start_; + ::PROTOBUF_NAMESPACE_ID::int32 channel_end_; + ::PROTOBUF_NAMESPACE_ID::uint64 open_time_; + ::PROTOBUF_NAMESPACE_ID::uint64 close_time_; + ::PROTOBUF_NAMESPACE_ID::int64 num_rows_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_etcd_5fmeta_2eproto; +}; +// =================================================================== + + +// =================================================================== + +#ifdef __GNUC__ + #pragma GCC diagnostic push + #pragma GCC diagnostic ignored "-Wstrict-aliasing" +#endif // __GNUC__ +// TenantMeta + +// uint64 id = 1; +inline void TenantMeta::clear_id() { + id_ = PROTOBUF_ULONGLONG(0); +} +inline ::PROTOBUF_NAMESPACE_ID::uint64 TenantMeta::id() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.TenantMeta.id) + return id_; +} +inline void TenantMeta::set_id(::PROTOBUF_NAMESPACE_ID::uint64 value) { + + id_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.TenantMeta.id) +} + +// uint64 num_query_nodes = 2; +inline void TenantMeta::clear_num_query_nodes() { + num_query_nodes_ = PROTOBUF_ULONGLONG(0); +} +inline ::PROTOBUF_NAMESPACE_ID::uint64 TenantMeta::num_query_nodes() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.TenantMeta.num_query_nodes) + return num_query_nodes_; +} +inline void TenantMeta::set_num_query_nodes(::PROTOBUF_NAMESPACE_ID::uint64 value) { + + num_query_nodes_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.TenantMeta.num_query_nodes) +} + +// repeated string insert_channel_ids = 3; +inline int TenantMeta::insert_channel_ids_size() const { + return insert_channel_ids_.size(); +} +inline void TenantMeta::clear_insert_channel_ids() { + insert_channel_ids_.Clear(); +} +inline const std::string& TenantMeta::insert_channel_ids(int index) const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.TenantMeta.insert_channel_ids) + return insert_channel_ids_.Get(index); +} +inline std::string* TenantMeta::mutable_insert_channel_ids(int index) { + // @@protoc_insertion_point(field_mutable:milvus.proto.etcd.TenantMeta.insert_channel_ids) + return insert_channel_ids_.Mutable(index); +} +inline void TenantMeta::set_insert_channel_ids(int index, const std::string& value) { + // @@protoc_insertion_point(field_set:milvus.proto.etcd.TenantMeta.insert_channel_ids) + insert_channel_ids_.Mutable(index)->assign(value); +} +inline void TenantMeta::set_insert_channel_ids(int index, std::string&& value) { + // @@protoc_insertion_point(field_set:milvus.proto.etcd.TenantMeta.insert_channel_ids) + insert_channel_ids_.Mutable(index)->assign(std::move(value)); +} +inline void TenantMeta::set_insert_channel_ids(int index, const char* value) { + GOOGLE_DCHECK(value != nullptr); + insert_channel_ids_.Mutable(index)->assign(value); + // @@protoc_insertion_point(field_set_char:milvus.proto.etcd.TenantMeta.insert_channel_ids) +} +inline void TenantMeta::set_insert_channel_ids(int index, const char* value, size_t size) { + insert_channel_ids_.Mutable(index)->assign( + reinterpret_cast(value), size); + // @@protoc_insertion_point(field_set_pointer:milvus.proto.etcd.TenantMeta.insert_channel_ids) +} +inline std::string* TenantMeta::add_insert_channel_ids() { + // @@protoc_insertion_point(field_add_mutable:milvus.proto.etcd.TenantMeta.insert_channel_ids) + return insert_channel_ids_.Add(); +} +inline void TenantMeta::add_insert_channel_ids(const std::string& value) { + insert_channel_ids_.Add()->assign(value); + // @@protoc_insertion_point(field_add:milvus.proto.etcd.TenantMeta.insert_channel_ids) +} +inline void TenantMeta::add_insert_channel_ids(std::string&& value) { + insert_channel_ids_.Add(std::move(value)); + // @@protoc_insertion_point(field_add:milvus.proto.etcd.TenantMeta.insert_channel_ids) +} +inline void TenantMeta::add_insert_channel_ids(const char* value) { + GOOGLE_DCHECK(value != nullptr); + insert_channel_ids_.Add()->assign(value); + // @@protoc_insertion_point(field_add_char:milvus.proto.etcd.TenantMeta.insert_channel_ids) +} +inline void TenantMeta::add_insert_channel_ids(const char* value, size_t size) { + insert_channel_ids_.Add()->assign(reinterpret_cast(value), size); + // @@protoc_insertion_point(field_add_pointer:milvus.proto.etcd.TenantMeta.insert_channel_ids) +} +inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField& +TenantMeta::insert_channel_ids() const { + // @@protoc_insertion_point(field_list:milvus.proto.etcd.TenantMeta.insert_channel_ids) + return insert_channel_ids_; +} +inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField* +TenantMeta::mutable_insert_channel_ids() { + // @@protoc_insertion_point(field_mutable_list:milvus.proto.etcd.TenantMeta.insert_channel_ids) + return &insert_channel_ids_; +} + +// string query_channel_id = 4; +inline void TenantMeta::clear_query_channel_id() { + query_channel_id_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline const std::string& TenantMeta::query_channel_id() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.TenantMeta.query_channel_id) + return query_channel_id_.GetNoArena(); +} +inline void TenantMeta::set_query_channel_id(const std::string& value) { + + query_channel_id_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set:milvus.proto.etcd.TenantMeta.query_channel_id) +} +inline void TenantMeta::set_query_channel_id(std::string&& value) { + + query_channel_id_.SetNoArena( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); + // @@protoc_insertion_point(field_set_rvalue:milvus.proto.etcd.TenantMeta.query_channel_id) +} +inline void TenantMeta::set_query_channel_id(const char* value) { + GOOGLE_DCHECK(value != nullptr); + + query_channel_id_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); + // @@protoc_insertion_point(field_set_char:milvus.proto.etcd.TenantMeta.query_channel_id) +} +inline void TenantMeta::set_query_channel_id(const char* value, size_t size) { + + query_channel_id_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + ::std::string(reinterpret_cast(value), size)); + // @@protoc_insertion_point(field_set_pointer:milvus.proto.etcd.TenantMeta.query_channel_id) +} +inline std::string* TenantMeta::mutable_query_channel_id() { + + // @@protoc_insertion_point(field_mutable:milvus.proto.etcd.TenantMeta.query_channel_id) + return query_channel_id_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline std::string* TenantMeta::release_query_channel_id() { + // @@protoc_insertion_point(field_release:milvus.proto.etcd.TenantMeta.query_channel_id) + + return query_channel_id_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline void TenantMeta::set_allocated_query_channel_id(std::string* query_channel_id) { + if (query_channel_id != nullptr) { + + } else { + + } + query_channel_id_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), query_channel_id); + // @@protoc_insertion_point(field_set_allocated:milvus.proto.etcd.TenantMeta.query_channel_id) +} + +// ------------------------------------------------------------------- + +// ProxyMeta + +// uint64 id = 1; +inline void ProxyMeta::clear_id() { + id_ = PROTOBUF_ULONGLONG(0); +} +inline ::PROTOBUF_NAMESPACE_ID::uint64 ProxyMeta::id() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.ProxyMeta.id) + return id_; +} +inline void ProxyMeta::set_id(::PROTOBUF_NAMESPACE_ID::uint64 value) { + + id_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.ProxyMeta.id) +} + +// .milvus.proto.common.Address address = 2; +inline bool ProxyMeta::has_address() const { + return this != internal_default_instance() && address_ != nullptr; +} +inline const ::milvus::proto::common::Address& ProxyMeta::address() const { + const ::milvus::proto::common::Address* p = address_; + // @@protoc_insertion_point(field_get:milvus.proto.etcd.ProxyMeta.address) + return p != nullptr ? *p : *reinterpret_cast( + &::milvus::proto::common::_Address_default_instance_); +} +inline ::milvus::proto::common::Address* ProxyMeta::release_address() { + // @@protoc_insertion_point(field_release:milvus.proto.etcd.ProxyMeta.address) + + ::milvus::proto::common::Address* temp = address_; + address_ = nullptr; + return temp; +} +inline ::milvus::proto::common::Address* ProxyMeta::mutable_address() { + + if (address_ == nullptr) { + auto* p = CreateMaybeMessage<::milvus::proto::common::Address>(GetArenaNoVirtual()); + address_ = p; + } + // @@protoc_insertion_point(field_mutable:milvus.proto.etcd.ProxyMeta.address) + return address_; +} +inline void ProxyMeta::set_allocated_address(::milvus::proto::common::Address* address) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaNoVirtual(); + if (message_arena == nullptr) { + delete reinterpret_cast< ::PROTOBUF_NAMESPACE_ID::MessageLite*>(address_); + } + if (address) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = nullptr; + if (message_arena != submessage_arena) { + address = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, address, submessage_arena); + } + + } else { + + } + address_ = address; + // @@protoc_insertion_point(field_set_allocated:milvus.proto.etcd.ProxyMeta.address) +} + +// repeated string result_channel_ids = 3; +inline int ProxyMeta::result_channel_ids_size() const { + return result_channel_ids_.size(); +} +inline void ProxyMeta::clear_result_channel_ids() { + result_channel_ids_.Clear(); +} +inline const std::string& ProxyMeta::result_channel_ids(int index) const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.ProxyMeta.result_channel_ids) + return result_channel_ids_.Get(index); +} +inline std::string* ProxyMeta::mutable_result_channel_ids(int index) { + // @@protoc_insertion_point(field_mutable:milvus.proto.etcd.ProxyMeta.result_channel_ids) + return result_channel_ids_.Mutable(index); +} +inline void ProxyMeta::set_result_channel_ids(int index, const std::string& value) { + // @@protoc_insertion_point(field_set:milvus.proto.etcd.ProxyMeta.result_channel_ids) + result_channel_ids_.Mutable(index)->assign(value); +} +inline void ProxyMeta::set_result_channel_ids(int index, std::string&& value) { + // @@protoc_insertion_point(field_set:milvus.proto.etcd.ProxyMeta.result_channel_ids) + result_channel_ids_.Mutable(index)->assign(std::move(value)); +} +inline void ProxyMeta::set_result_channel_ids(int index, const char* value) { + GOOGLE_DCHECK(value != nullptr); + result_channel_ids_.Mutable(index)->assign(value); + // @@protoc_insertion_point(field_set_char:milvus.proto.etcd.ProxyMeta.result_channel_ids) +} +inline void ProxyMeta::set_result_channel_ids(int index, const char* value, size_t size) { + result_channel_ids_.Mutable(index)->assign( + reinterpret_cast(value), size); + // @@protoc_insertion_point(field_set_pointer:milvus.proto.etcd.ProxyMeta.result_channel_ids) +} +inline std::string* ProxyMeta::add_result_channel_ids() { + // @@protoc_insertion_point(field_add_mutable:milvus.proto.etcd.ProxyMeta.result_channel_ids) + return result_channel_ids_.Add(); +} +inline void ProxyMeta::add_result_channel_ids(const std::string& value) { + result_channel_ids_.Add()->assign(value); + // @@protoc_insertion_point(field_add:milvus.proto.etcd.ProxyMeta.result_channel_ids) +} +inline void ProxyMeta::add_result_channel_ids(std::string&& value) { + result_channel_ids_.Add(std::move(value)); + // @@protoc_insertion_point(field_add:milvus.proto.etcd.ProxyMeta.result_channel_ids) +} +inline void ProxyMeta::add_result_channel_ids(const char* value) { + GOOGLE_DCHECK(value != nullptr); + result_channel_ids_.Add()->assign(value); + // @@protoc_insertion_point(field_add_char:milvus.proto.etcd.ProxyMeta.result_channel_ids) +} +inline void ProxyMeta::add_result_channel_ids(const char* value, size_t size) { + result_channel_ids_.Add()->assign(reinterpret_cast(value), size); + // @@protoc_insertion_point(field_add_pointer:milvus.proto.etcd.ProxyMeta.result_channel_ids) +} +inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField& +ProxyMeta::result_channel_ids() const { + // @@protoc_insertion_point(field_list:milvus.proto.etcd.ProxyMeta.result_channel_ids) + return result_channel_ids_; +} +inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField* +ProxyMeta::mutable_result_channel_ids() { + // @@protoc_insertion_point(field_mutable_list:milvus.proto.etcd.ProxyMeta.result_channel_ids) + return &result_channel_ids_; +} + +// ------------------------------------------------------------------- + +// CollectionMeta + +// uint64 id = 1; +inline void CollectionMeta::clear_id() { + id_ = PROTOBUF_ULONGLONG(0); +} +inline ::PROTOBUF_NAMESPACE_ID::uint64 CollectionMeta::id() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.CollectionMeta.id) + return id_; +} +inline void CollectionMeta::set_id(::PROTOBUF_NAMESPACE_ID::uint64 value) { + + id_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.CollectionMeta.id) +} + +// .milvus.proto.schema.CollectionSchema schema = 2; +inline bool CollectionMeta::has_schema() const { + return this != internal_default_instance() && schema_ != nullptr; +} +inline const ::milvus::proto::schema::CollectionSchema& CollectionMeta::schema() const { + const ::milvus::proto::schema::CollectionSchema* p = schema_; + // @@protoc_insertion_point(field_get:milvus.proto.etcd.CollectionMeta.schema) + return p != nullptr ? *p : *reinterpret_cast( + &::milvus::proto::schema::_CollectionSchema_default_instance_); +} +inline ::milvus::proto::schema::CollectionSchema* CollectionMeta::release_schema() { + // @@protoc_insertion_point(field_release:milvus.proto.etcd.CollectionMeta.schema) + + ::milvus::proto::schema::CollectionSchema* temp = schema_; + schema_ = nullptr; + return temp; +} +inline ::milvus::proto::schema::CollectionSchema* CollectionMeta::mutable_schema() { + + if (schema_ == nullptr) { + auto* p = CreateMaybeMessage<::milvus::proto::schema::CollectionSchema>(GetArenaNoVirtual()); + schema_ = p; + } + // @@protoc_insertion_point(field_mutable:milvus.proto.etcd.CollectionMeta.schema) + return schema_; +} +inline void CollectionMeta::set_allocated_schema(::milvus::proto::schema::CollectionSchema* schema) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaNoVirtual(); + if (message_arena == nullptr) { + delete reinterpret_cast< ::PROTOBUF_NAMESPACE_ID::MessageLite*>(schema_); + } + if (schema) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = nullptr; + if (message_arena != submessage_arena) { + schema = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, schema, submessage_arena); + } + + } else { + + } + schema_ = schema; + // @@protoc_insertion_point(field_set_allocated:milvus.proto.etcd.CollectionMeta.schema) +} + +// uint64 create_time = 3; +inline void CollectionMeta::clear_create_time() { + create_time_ = PROTOBUF_ULONGLONG(0); +} +inline ::PROTOBUF_NAMESPACE_ID::uint64 CollectionMeta::create_time() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.CollectionMeta.create_time) + return create_time_; +} +inline void CollectionMeta::set_create_time(::PROTOBUF_NAMESPACE_ID::uint64 value) { + + create_time_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.CollectionMeta.create_time) +} + +// repeated uint64 segment_ids = 4; +inline int CollectionMeta::segment_ids_size() const { + return segment_ids_.size(); +} +inline void CollectionMeta::clear_segment_ids() { + segment_ids_.Clear(); +} +inline ::PROTOBUF_NAMESPACE_ID::uint64 CollectionMeta::segment_ids(int index) const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.CollectionMeta.segment_ids) + return segment_ids_.Get(index); +} +inline void CollectionMeta::set_segment_ids(int index, ::PROTOBUF_NAMESPACE_ID::uint64 value) { + segment_ids_.Set(index, value); + // @@protoc_insertion_point(field_set:milvus.proto.etcd.CollectionMeta.segment_ids) +} +inline void CollectionMeta::add_segment_ids(::PROTOBUF_NAMESPACE_ID::uint64 value) { + segment_ids_.Add(value); + // @@protoc_insertion_point(field_add:milvus.proto.etcd.CollectionMeta.segment_ids) +} +inline const ::PROTOBUF_NAMESPACE_ID::RepeatedField< ::PROTOBUF_NAMESPACE_ID::uint64 >& +CollectionMeta::segment_ids() const { + // @@protoc_insertion_point(field_list:milvus.proto.etcd.CollectionMeta.segment_ids) + return segment_ids_; +} +inline ::PROTOBUF_NAMESPACE_ID::RepeatedField< ::PROTOBUF_NAMESPACE_ID::uint64 >* +CollectionMeta::mutable_segment_ids() { + // @@protoc_insertion_point(field_mutable_list:milvus.proto.etcd.CollectionMeta.segment_ids) + return &segment_ids_; +} + +// repeated string partition_tags = 5; +inline int CollectionMeta::partition_tags_size() const { + return partition_tags_.size(); +} +inline void CollectionMeta::clear_partition_tags() { + partition_tags_.Clear(); +} +inline const std::string& CollectionMeta::partition_tags(int index) const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.CollectionMeta.partition_tags) + return partition_tags_.Get(index); +} +inline std::string* CollectionMeta::mutable_partition_tags(int index) { + // @@protoc_insertion_point(field_mutable:milvus.proto.etcd.CollectionMeta.partition_tags) + return partition_tags_.Mutable(index); +} +inline void CollectionMeta::set_partition_tags(int index, const std::string& value) { + // @@protoc_insertion_point(field_set:milvus.proto.etcd.CollectionMeta.partition_tags) + partition_tags_.Mutable(index)->assign(value); +} +inline void CollectionMeta::set_partition_tags(int index, std::string&& value) { + // @@protoc_insertion_point(field_set:milvus.proto.etcd.CollectionMeta.partition_tags) + partition_tags_.Mutable(index)->assign(std::move(value)); +} +inline void CollectionMeta::set_partition_tags(int index, const char* value) { + GOOGLE_DCHECK(value != nullptr); + partition_tags_.Mutable(index)->assign(value); + // @@protoc_insertion_point(field_set_char:milvus.proto.etcd.CollectionMeta.partition_tags) +} +inline void CollectionMeta::set_partition_tags(int index, const char* value, size_t size) { + partition_tags_.Mutable(index)->assign( + reinterpret_cast(value), size); + // @@protoc_insertion_point(field_set_pointer:milvus.proto.etcd.CollectionMeta.partition_tags) +} +inline std::string* CollectionMeta::add_partition_tags() { + // @@protoc_insertion_point(field_add_mutable:milvus.proto.etcd.CollectionMeta.partition_tags) + return partition_tags_.Add(); +} +inline void CollectionMeta::add_partition_tags(const std::string& value) { + partition_tags_.Add()->assign(value); + // @@protoc_insertion_point(field_add:milvus.proto.etcd.CollectionMeta.partition_tags) +} +inline void CollectionMeta::add_partition_tags(std::string&& value) { + partition_tags_.Add(std::move(value)); + // @@protoc_insertion_point(field_add:milvus.proto.etcd.CollectionMeta.partition_tags) +} +inline void CollectionMeta::add_partition_tags(const char* value) { + GOOGLE_DCHECK(value != nullptr); + partition_tags_.Add()->assign(value); + // @@protoc_insertion_point(field_add_char:milvus.proto.etcd.CollectionMeta.partition_tags) +} +inline void CollectionMeta::add_partition_tags(const char* value, size_t size) { + partition_tags_.Add()->assign(reinterpret_cast(value), size); + // @@protoc_insertion_point(field_add_pointer:milvus.proto.etcd.CollectionMeta.partition_tags) +} +inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField& +CollectionMeta::partition_tags() const { + // @@protoc_insertion_point(field_list:milvus.proto.etcd.CollectionMeta.partition_tags) + return partition_tags_; +} +inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField* +CollectionMeta::mutable_partition_tags() { + // @@protoc_insertion_point(field_mutable_list:milvus.proto.etcd.CollectionMeta.partition_tags) + return &partition_tags_; +} + +// ------------------------------------------------------------------- + +// SegmentMeta + +// uint64 segment_id = 1; +inline void SegmentMeta::clear_segment_id() { + segment_id_ = PROTOBUF_ULONGLONG(0); +} +inline ::PROTOBUF_NAMESPACE_ID::uint64 SegmentMeta::segment_id() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.SegmentMeta.segment_id) + return segment_id_; +} +inline void SegmentMeta::set_segment_id(::PROTOBUF_NAMESPACE_ID::uint64 value) { + + segment_id_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.SegmentMeta.segment_id) +} + +// uint64 collection_id = 2; +inline void SegmentMeta::clear_collection_id() { + collection_id_ = PROTOBUF_ULONGLONG(0); +} +inline ::PROTOBUF_NAMESPACE_ID::uint64 SegmentMeta::collection_id() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.SegmentMeta.collection_id) + return collection_id_; +} +inline void SegmentMeta::set_collection_id(::PROTOBUF_NAMESPACE_ID::uint64 value) { + + collection_id_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.SegmentMeta.collection_id) +} + +// string partition_tag = 3; +inline void SegmentMeta::clear_partition_tag() { + partition_tag_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline const std::string& SegmentMeta::partition_tag() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.SegmentMeta.partition_tag) + return partition_tag_.GetNoArena(); +} +inline void SegmentMeta::set_partition_tag(const std::string& value) { + + partition_tag_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value); + // @@protoc_insertion_point(field_set:milvus.proto.etcd.SegmentMeta.partition_tag) +} +inline void SegmentMeta::set_partition_tag(std::string&& value) { + + partition_tag_.SetNoArena( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::move(value)); + // @@protoc_insertion_point(field_set_rvalue:milvus.proto.etcd.SegmentMeta.partition_tag) +} +inline void SegmentMeta::set_partition_tag(const char* value) { + GOOGLE_DCHECK(value != nullptr); + + partition_tag_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), ::std::string(value)); + // @@protoc_insertion_point(field_set_char:milvus.proto.etcd.SegmentMeta.partition_tag) +} +inline void SegmentMeta::set_partition_tag(const char* value, size_t size) { + + partition_tag_.SetNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + ::std::string(reinterpret_cast(value), size)); + // @@protoc_insertion_point(field_set_pointer:milvus.proto.etcd.SegmentMeta.partition_tag) +} +inline std::string* SegmentMeta::mutable_partition_tag() { + + // @@protoc_insertion_point(field_mutable:milvus.proto.etcd.SegmentMeta.partition_tag) + return partition_tag_.MutableNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline std::string* SegmentMeta::release_partition_tag() { + // @@protoc_insertion_point(field_release:milvus.proto.etcd.SegmentMeta.partition_tag) + + return partition_tag_.ReleaseNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} +inline void SegmentMeta::set_allocated_partition_tag(std::string* partition_tag) { + if (partition_tag != nullptr) { + + } else { + + } + partition_tag_.SetAllocatedNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), partition_tag); + // @@protoc_insertion_point(field_set_allocated:milvus.proto.etcd.SegmentMeta.partition_tag) +} + +// int32 channel_start = 4; +inline void SegmentMeta::clear_channel_start() { + channel_start_ = 0; +} +inline ::PROTOBUF_NAMESPACE_ID::int32 SegmentMeta::channel_start() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.SegmentMeta.channel_start) + return channel_start_; +} +inline void SegmentMeta::set_channel_start(::PROTOBUF_NAMESPACE_ID::int32 value) { + + channel_start_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.SegmentMeta.channel_start) +} + +// int32 channel_end = 5; +inline void SegmentMeta::clear_channel_end() { + channel_end_ = 0; +} +inline ::PROTOBUF_NAMESPACE_ID::int32 SegmentMeta::channel_end() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.SegmentMeta.channel_end) + return channel_end_; +} +inline void SegmentMeta::set_channel_end(::PROTOBUF_NAMESPACE_ID::int32 value) { + + channel_end_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.SegmentMeta.channel_end) +} + +// uint64 open_time = 6; +inline void SegmentMeta::clear_open_time() { + open_time_ = PROTOBUF_ULONGLONG(0); +} +inline ::PROTOBUF_NAMESPACE_ID::uint64 SegmentMeta::open_time() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.SegmentMeta.open_time) + return open_time_; +} +inline void SegmentMeta::set_open_time(::PROTOBUF_NAMESPACE_ID::uint64 value) { + + open_time_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.SegmentMeta.open_time) +} + +// uint64 close_time = 7; +inline void SegmentMeta::clear_close_time() { + close_time_ = PROTOBUF_ULONGLONG(0); +} +inline ::PROTOBUF_NAMESPACE_ID::uint64 SegmentMeta::close_time() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.SegmentMeta.close_time) + return close_time_; +} +inline void SegmentMeta::set_close_time(::PROTOBUF_NAMESPACE_ID::uint64 value) { + + close_time_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.SegmentMeta.close_time) +} + +// int64 num_rows = 8; +inline void SegmentMeta::clear_num_rows() { + num_rows_ = PROTOBUF_LONGLONG(0); +} +inline ::PROTOBUF_NAMESPACE_ID::int64 SegmentMeta::num_rows() const { + // @@protoc_insertion_point(field_get:milvus.proto.etcd.SegmentMeta.num_rows) + return num_rows_; +} +inline void SegmentMeta::set_num_rows(::PROTOBUF_NAMESPACE_ID::int64 value) { + + num_rows_ = value; + // @@protoc_insertion_point(field_set:milvus.proto.etcd.SegmentMeta.num_rows) +} + +#ifdef __GNUC__ + #pragma GCC diagnostic pop +#endif // __GNUC__ +// ------------------------------------------------------------------- + +// ------------------------------------------------------------------- + +// ------------------------------------------------------------------- + + +// @@protoc_insertion_point(namespace_scope) + +} // namespace etcd +} // namespace proto +} // namespace milvus + +// @@protoc_insertion_point(global_scope) + +#include +#endif // GOOGLE_PROTOBUF_INCLUDED_GOOGLE_PROTOBUF_INCLUDED_etcd_5fmeta_2eproto diff --git a/internal/core/src/pb/schema.pb.cc b/internal/core/src/pb/schema.pb.cc new file mode 100644 index 0000000000..e78e42bae9 --- /dev/null +++ b/internal/core/src/pb/schema.pb.cc @@ -0,0 +1,1100 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: schema.proto + +#include "schema.pb.h" + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +// @@protoc_insertion_point(includes) +#include +extern PROTOBUF_INTERNAL_EXPORT_common_2eproto ::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<0> scc_info_KeyValuePair_common_2eproto; +extern PROTOBUF_INTERNAL_EXPORT_schema_2eproto ::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<1> scc_info_FieldSchema_schema_2eproto; +namespace milvus { +namespace proto { +namespace schema { +class FieldSchemaDefaultTypeInternal { + public: + ::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed _instance; +} _FieldSchema_default_instance_; +class CollectionSchemaDefaultTypeInternal { + public: + ::PROTOBUF_NAMESPACE_ID::internal::ExplicitlyConstructed _instance; +} _CollectionSchema_default_instance_; +} // namespace schema +} // namespace proto +} // namespace milvus +static void InitDefaultsscc_info_CollectionSchema_schema_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::milvus::proto::schema::_CollectionSchema_default_instance_; + new (ptr) ::milvus::proto::schema::CollectionSchema(); + ::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr); + } + ::milvus::proto::schema::CollectionSchema::InitAsDefaultInstance(); +} + +::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<1> scc_info_CollectionSchema_schema_2eproto = + {{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsscc_info_CollectionSchema_schema_2eproto}, { + &scc_info_FieldSchema_schema_2eproto.base,}}; + +static void InitDefaultsscc_info_FieldSchema_schema_2eproto() { + GOOGLE_PROTOBUF_VERIFY_VERSION; + + { + void* ptr = &::milvus::proto::schema::_FieldSchema_default_instance_; + new (ptr) ::milvus::proto::schema::FieldSchema(); + ::PROTOBUF_NAMESPACE_ID::internal::OnShutdownDestroyMessage(ptr); + } + ::milvus::proto::schema::FieldSchema::InitAsDefaultInstance(); +} + +::PROTOBUF_NAMESPACE_ID::internal::SCCInfo<1> scc_info_FieldSchema_schema_2eproto = + {{ATOMIC_VAR_INIT(::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase::kUninitialized), 1, InitDefaultsscc_info_FieldSchema_schema_2eproto}, { + &scc_info_KeyValuePair_common_2eproto.base,}}; + +static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_schema_2eproto[2]; +static const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* file_level_enum_descriptors_schema_2eproto[1]; +static constexpr ::PROTOBUF_NAMESPACE_ID::ServiceDescriptor const** file_level_service_descriptors_schema_2eproto = nullptr; + +const ::PROTOBUF_NAMESPACE_ID::uint32 TableStruct_schema_2eproto::offsets[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::schema::FieldSchema, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::schema::FieldSchema, name_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::schema::FieldSchema, description_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::schema::FieldSchema, data_type_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::schema::FieldSchema, type_params_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::schema::FieldSchema, index_params_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::schema::CollectionSchema, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + PROTOBUF_FIELD_OFFSET(::milvus::proto::schema::CollectionSchema, name_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::schema::CollectionSchema, description_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::schema::CollectionSchema, auto_id_), + PROTOBUF_FIELD_OFFSET(::milvus::proto::schema::CollectionSchema, fields_), +}; +static const ::PROTOBUF_NAMESPACE_ID::internal::MigrationSchema schemas[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { + { 0, -1, sizeof(::milvus::proto::schema::FieldSchema)}, + { 10, -1, sizeof(::milvus::proto::schema::CollectionSchema)}, +}; + +static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = { + reinterpret_cast(&::milvus::proto::schema::_FieldSchema_default_instance_), + reinterpret_cast(&::milvus::proto::schema::_CollectionSchema_default_instance_), +}; + +const char descriptor_table_protodef_schema_2eproto[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = + "\n\014schema.proto\022\023milvus.proto.schema\032\014com" + "mon.proto\"\323\001\n\013FieldSchema\022\014\n\004name\030\001 \001(\t\022" + "\023\n\013description\030\002 \001(\t\0220\n\tdata_type\030\003 \001(\0162" + "\035.milvus.proto.schema.DataType\0226\n\013type_p" + "arams\030\004 \003(\0132!.milvus.proto.common.KeyVal" + "uePair\0227\n\014index_params\030\005 \003(\0132!.milvus.pr" + "oto.common.KeyValuePair\"x\n\020CollectionSch" + "ema\022\014\n\004name\030\001 \001(\t\022\023\n\013description\030\002 \001(\t\022\017" + "\n\007auto_id\030\003 \001(\010\0220\n\006fields\030\004 \003(\0132 .milvus" + ".proto.schema.FieldSchema*\221\001\n\010DataType\022\010" + "\n\004NONE\020\000\022\010\n\004BOOL\020\001\022\010\n\004INT8\020\002\022\t\n\005INT16\020\003\022" + "\t\n\005INT32\020\004\022\t\n\005INT64\020\005\022\t\n\005FLOAT\020\n\022\n\n\006DOUB" + "LE\020\013\022\n\n\006STRING\020\024\022\021\n\rVECTOR_BINARY\020d\022\020\n\014V" + "ECTOR_FLOAT\020eBBZ@github.com/zilliztech/m" + "ilvus-distributed/internal/proto/schemap" + "bb\006proto3" + ; +static const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable*const descriptor_table_schema_2eproto_deps[1] = { + &::descriptor_table_common_2eproto, +}; +static ::PROTOBUF_NAMESPACE_ID::internal::SCCInfoBase*const descriptor_table_schema_2eproto_sccs[2] = { + &scc_info_CollectionSchema_schema_2eproto.base, + &scc_info_FieldSchema_schema_2eproto.base, +}; +static ::PROTOBUF_NAMESPACE_ID::internal::once_flag descriptor_table_schema_2eproto_once; +static bool descriptor_table_schema_2eproto_initialized = false; +const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable descriptor_table_schema_2eproto = { + &descriptor_table_schema_2eproto_initialized, descriptor_table_protodef_schema_2eproto, "schema.proto", 609, + &descriptor_table_schema_2eproto_once, descriptor_table_schema_2eproto_sccs, descriptor_table_schema_2eproto_deps, 2, 1, + schemas, file_default_instances, TableStruct_schema_2eproto::offsets, + file_level_metadata_schema_2eproto, 2, file_level_enum_descriptors_schema_2eproto, file_level_service_descriptors_schema_2eproto, +}; + +// Force running AddDescriptors() at dynamic initialization time. +static bool dynamic_init_dummy_schema_2eproto = ( ::PROTOBUF_NAMESPACE_ID::internal::AddDescriptors(&descriptor_table_schema_2eproto), true); +namespace milvus { +namespace proto { +namespace schema { +const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* DataType_descriptor() { + ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_schema_2eproto); + return file_level_enum_descriptors_schema_2eproto[0]; +} +bool DataType_IsValid(int value) { + switch (value) { + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 10: + case 11: + case 20: + case 100: + case 101: + return true; + default: + return false; + } +} + + +// =================================================================== + +void FieldSchema::InitAsDefaultInstance() { +} +class FieldSchema::_Internal { + public: +}; + +void FieldSchema::clear_type_params() { + type_params_.Clear(); +} +void FieldSchema::clear_index_params() { + index_params_.Clear(); +} +FieldSchema::FieldSchema() + : ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:milvus.proto.schema.FieldSchema) +} +FieldSchema::FieldSchema(const FieldSchema& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _internal_metadata_(nullptr), + type_params_(from.type_params_), + index_params_(from.index_params_) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + name_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + if (!from.name().empty()) { + name_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.name_); + } + description_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + if (!from.description().empty()) { + description_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.description_); + } + data_type_ = from.data_type_; + // @@protoc_insertion_point(copy_constructor:milvus.proto.schema.FieldSchema) +} + +void FieldSchema::SharedCtor() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&scc_info_FieldSchema_schema_2eproto.base); + name_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + description_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + data_type_ = 0; +} + +FieldSchema::~FieldSchema() { + // @@protoc_insertion_point(destructor:milvus.proto.schema.FieldSchema) + SharedDtor(); +} + +void FieldSchema::SharedDtor() { + name_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + description_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +void FieldSchema::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const FieldSchema& FieldSchema::default_instance() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_FieldSchema_schema_2eproto.base); + return *internal_default_instance(); +} + + +void FieldSchema::Clear() { +// @@protoc_insertion_point(message_clear_start:milvus.proto.schema.FieldSchema) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + type_params_.Clear(); + index_params_.Clear(); + name_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + description_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + data_type_ = 0; + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* FieldSchema::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + CHK_(ptr); + switch (tag >> 3) { + // string name = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 10)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(mutable_name(), ptr, ctx, "milvus.proto.schema.FieldSchema.name"); + CHK_(ptr); + } else goto handle_unusual; + continue; + // string description = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 18)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(mutable_description(), ptr, ctx, "milvus.proto.schema.FieldSchema.description"); + CHK_(ptr); + } else goto handle_unusual; + continue; + // .milvus.proto.schema.DataType data_type = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 24)) { + ::PROTOBUF_NAMESPACE_ID::uint64 val = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + set_data_type(static_cast<::milvus::proto::schema::DataType>(val)); + } else goto handle_unusual; + continue; + // repeated .milvus.proto.common.KeyValuePair type_params = 4; + case 4: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 34)) { + ptr -= 1; + do { + ptr += 1; + ptr = ctx->ParseMessage(add_type_params(), ptr); + CHK_(ptr); + if (!ctx->DataAvailable(ptr)) break; + } while (::PROTOBUF_NAMESPACE_ID::internal::UnalignedLoad<::PROTOBUF_NAMESPACE_ID::uint8>(ptr) == 34); + } else goto handle_unusual; + continue; + // repeated .milvus.proto.common.KeyValuePair index_params = 5; + case 5: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 42)) { + ptr -= 1; + do { + ptr += 1; + ptr = ctx->ParseMessage(add_index_params(), ptr); + CHK_(ptr); + if (!ctx->DataAvailable(ptr)) break; + } while (::PROTOBUF_NAMESPACE_ID::internal::UnalignedLoad<::PROTOBUF_NAMESPACE_ID::uint8>(ptr) == 42); + } else goto handle_unusual; + continue; + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->SetLastTag(tag); + goto success; + } + ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx); + CHK_(ptr != nullptr); + continue; + } + } // switch + } // while +success: + return ptr; +failure: + ptr = nullptr; + goto success; +#undef CHK_ +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool FieldSchema::MergePartialFromCodedStream( + ::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + // @@protoc_insertion_point(parse_start:milvus.proto.schema.FieldSchema) + for (;;) { + ::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // string name = 1; + case 1: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (10 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->mutable_name())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->name().data(), static_cast(this->name().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.schema.FieldSchema.name")); + } else { + goto handle_unusual; + } + break; + } + + // string description = 2; + case 2: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (18 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->mutable_description())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->description().data(), static_cast(this->description().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.schema.FieldSchema.description")); + } else { + goto handle_unusual; + } + break; + } + + // .milvus.proto.schema.DataType data_type = 3; + case 3: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (24 & 0xFF)) { + int value = 0; + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + int, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_ENUM>( + input, &value))); + set_data_type(static_cast< ::milvus::proto::schema::DataType >(value)); + } else { + goto handle_unusual; + } + break; + } + + // repeated .milvus.proto.common.KeyValuePair type_params = 4; + case 4: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (34 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadMessage( + input, add_type_params())); + } else { + goto handle_unusual; + } + break; + } + + // repeated .milvus.proto.common.KeyValuePair index_params = 5; + case 5: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (42 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadMessage( + input, add_index_params())); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:milvus.proto.schema.FieldSchema) + return true; +failure: + // @@protoc_insertion_point(parse_failure:milvus.proto.schema.FieldSchema) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void FieldSchema::SerializeWithCachedSizes( + ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:milvus.proto.schema.FieldSchema) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // string name = 1; + if (this->name().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->name().data(), static_cast(this->name().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.schema.FieldSchema.name"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringMaybeAliased( + 1, this->name(), output); + } + + // string description = 2; + if (this->description().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->description().data(), static_cast(this->description().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.schema.FieldSchema.description"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringMaybeAliased( + 2, this->description(), output); + } + + // .milvus.proto.schema.DataType data_type = 3; + if (this->data_type() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnum( + 3, this->data_type(), output); + } + + // repeated .milvus.proto.common.KeyValuePair type_params = 4; + for (unsigned int i = 0, + n = static_cast(this->type_params_size()); i < n; i++) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteMessageMaybeToArray( + 4, + this->type_params(static_cast(i)), + output); + } + + // repeated .milvus.proto.common.KeyValuePair index_params = 5; + for (unsigned int i = 0, + n = static_cast(this->index_params_size()); i < n; i++) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteMessageMaybeToArray( + 5, + this->index_params(static_cast(i)), + output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:milvus.proto.schema.FieldSchema) +} + +::PROTOBUF_NAMESPACE_ID::uint8* FieldSchema::InternalSerializeWithCachedSizesToArray( + ::PROTOBUF_NAMESPACE_ID::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:milvus.proto.schema.FieldSchema) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // string name = 1; + if (this->name().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->name().data(), static_cast(this->name().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.schema.FieldSchema.name"); + target = + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringToArray( + 1, this->name(), target); + } + + // string description = 2; + if (this->description().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->description().data(), static_cast(this->description().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.schema.FieldSchema.description"); + target = + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringToArray( + 2, this->description(), target); + } + + // .milvus.proto.schema.DataType data_type = 3; + if (this->data_type() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnumToArray( + 3, this->data_type(), target); + } + + // repeated .milvus.proto.common.KeyValuePair type_params = 4; + for (unsigned int i = 0, + n = static_cast(this->type_params_size()); i < n; i++) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessageToArray( + 4, this->type_params(static_cast(i)), target); + } + + // repeated .milvus.proto.common.KeyValuePair index_params = 5; + for (unsigned int i = 0, + n = static_cast(this->index_params_size()); i < n; i++) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessageToArray( + 5, this->index_params(static_cast(i)), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:milvus.proto.schema.FieldSchema) + return target; +} + +size_t FieldSchema::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:milvus.proto.schema.FieldSchema) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // repeated .milvus.proto.common.KeyValuePair type_params = 4; + { + unsigned int count = static_cast(this->type_params_size()); + total_size += 1UL * count; + for (unsigned int i = 0; i < count; i++) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + this->type_params(static_cast(i))); + } + } + + // repeated .milvus.proto.common.KeyValuePair index_params = 5; + { + unsigned int count = static_cast(this->index_params_size()); + total_size += 1UL * count; + for (unsigned int i = 0; i < count; i++) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + this->index_params(static_cast(i))); + } + } + + // string name = 1; + if (this->name().size() > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->name()); + } + + // string description = 2; + if (this->description().size() > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->description()); + } + + // .milvus.proto.schema.DataType data_type = 3; + if (this->data_type() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->data_type()); + } + + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void FieldSchema::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.schema.FieldSchema) + GOOGLE_DCHECK_NE(&from, this); + const FieldSchema* source = + ::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.schema.FieldSchema) + ::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.schema.FieldSchema) + MergeFrom(*source); + } +} + +void FieldSchema::MergeFrom(const FieldSchema& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.schema.FieldSchema) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + type_params_.MergeFrom(from.type_params_); + index_params_.MergeFrom(from.index_params_); + if (from.name().size() > 0) { + + name_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.name_); + } + if (from.description().size() > 0) { + + description_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.description_); + } + if (from.data_type() != 0) { + set_data_type(from.data_type()); + } +} + +void FieldSchema::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.schema.FieldSchema) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void FieldSchema::CopyFrom(const FieldSchema& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.schema.FieldSchema) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool FieldSchema::IsInitialized() const { + return true; +} + +void FieldSchema::InternalSwap(FieldSchema* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + CastToBase(&type_params_)->InternalSwap(CastToBase(&other->type_params_)); + CastToBase(&index_params_)->InternalSwap(CastToBase(&other->index_params_)); + name_.Swap(&other->name_, &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); + description_.Swap(&other->description_, &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); + swap(data_type_, other->data_type_); +} + +::PROTOBUF_NAMESPACE_ID::Metadata FieldSchema::GetMetadata() const { + return GetMetadataStatic(); +} + + +// =================================================================== + +void CollectionSchema::InitAsDefaultInstance() { +} +class CollectionSchema::_Internal { + public: +}; + +CollectionSchema::CollectionSchema() + : ::PROTOBUF_NAMESPACE_ID::Message(), _internal_metadata_(nullptr) { + SharedCtor(); + // @@protoc_insertion_point(constructor:milvus.proto.schema.CollectionSchema) +} +CollectionSchema::CollectionSchema(const CollectionSchema& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _internal_metadata_(nullptr), + fields_(from.fields_) { + _internal_metadata_.MergeFrom(from._internal_metadata_); + name_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + if (!from.name().empty()) { + name_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.name_); + } + description_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + if (!from.description().empty()) { + description_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.description_); + } + auto_id_ = from.auto_id_; + // @@protoc_insertion_point(copy_constructor:milvus.proto.schema.CollectionSchema) +} + +void CollectionSchema::SharedCtor() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&scc_info_CollectionSchema_schema_2eproto.base); + name_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + description_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + auto_id_ = false; +} + +CollectionSchema::~CollectionSchema() { + // @@protoc_insertion_point(destructor:milvus.proto.schema.CollectionSchema) + SharedDtor(); +} + +void CollectionSchema::SharedDtor() { + name_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + description_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +void CollectionSchema::SetCachedSize(int size) const { + _cached_size_.Set(size); +} +const CollectionSchema& CollectionSchema::default_instance() { + ::PROTOBUF_NAMESPACE_ID::internal::InitSCC(&::scc_info_CollectionSchema_schema_2eproto.base); + return *internal_default_instance(); +} + + +void CollectionSchema::Clear() { +// @@protoc_insertion_point(message_clear_start:milvus.proto.schema.CollectionSchema) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + fields_.Clear(); + name_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + description_.ClearToEmptyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + auto_id_ = false; + _internal_metadata_.Clear(); +} + +#if GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +const char* CollectionSchema::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + CHK_(ptr); + switch (tag >> 3) { + // string name = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 10)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(mutable_name(), ptr, ctx, "milvus.proto.schema.CollectionSchema.name"); + CHK_(ptr); + } else goto handle_unusual; + continue; + // string description = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 18)) { + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParserUTF8(mutable_description(), ptr, ctx, "milvus.proto.schema.CollectionSchema.description"); + CHK_(ptr); + } else goto handle_unusual; + continue; + // bool auto_id = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 24)) { + auto_id_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint(&ptr); + CHK_(ptr); + } else goto handle_unusual; + continue; + // repeated .milvus.proto.schema.FieldSchema fields = 4; + case 4: + if (PROTOBUF_PREDICT_TRUE(static_cast<::PROTOBUF_NAMESPACE_ID::uint8>(tag) == 34)) { + ptr -= 1; + do { + ptr += 1; + ptr = ctx->ParseMessage(add_fields(), ptr); + CHK_(ptr); + if (!ctx->DataAvailable(ptr)) break; + } while (::PROTOBUF_NAMESPACE_ID::internal::UnalignedLoad<::PROTOBUF_NAMESPACE_ID::uint8>(ptr) == 34); + } else goto handle_unusual; + continue; + default: { + handle_unusual: + if ((tag & 7) == 4 || tag == 0) { + ctx->SetLastTag(tag); + goto success; + } + ptr = UnknownFieldParse(tag, &_internal_metadata_, ptr, ctx); + CHK_(ptr != nullptr); + continue; + } + } // switch + } // while +success: + return ptr; +failure: + ptr = nullptr; + goto success; +#undef CHK_ +} +#else // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER +bool CollectionSchema::MergePartialFromCodedStream( + ::PROTOBUF_NAMESPACE_ID::io::CodedInputStream* input) { +#define DO_(EXPRESSION) if (!PROTOBUF_PREDICT_TRUE(EXPRESSION)) goto failure + ::PROTOBUF_NAMESPACE_ID::uint32 tag; + // @@protoc_insertion_point(parse_start:milvus.proto.schema.CollectionSchema) + for (;;) { + ::std::pair<::PROTOBUF_NAMESPACE_ID::uint32, bool> p = input->ReadTagWithCutoffNoLastTag(127u); + tag = p.first; + if (!p.second) goto handle_unusual; + switch (::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::GetTagFieldNumber(tag)) { + // string name = 1; + case 1: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (10 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->mutable_name())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->name().data(), static_cast(this->name().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.schema.CollectionSchema.name")); + } else { + goto handle_unusual; + } + break; + } + + // string description = 2; + case 2: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (18 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadString( + input, this->mutable_description())); + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->description().data(), static_cast(this->description().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::PARSE, + "milvus.proto.schema.CollectionSchema.description")); + } else { + goto handle_unusual; + } + break; + } + + // bool auto_id = 3; + case 3: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (24 & 0xFF)) { + + DO_((::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadPrimitive< + bool, ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::TYPE_BOOL>( + input, &auto_id_))); + } else { + goto handle_unusual; + } + break; + } + + // repeated .milvus.proto.schema.FieldSchema fields = 4; + case 4: { + if (static_cast< ::PROTOBUF_NAMESPACE_ID::uint8>(tag) == (34 & 0xFF)) { + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::ReadMessage( + input, add_fields())); + } else { + goto handle_unusual; + } + break; + } + + default: { + handle_unusual: + if (tag == 0) { + goto success; + } + DO_(::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SkipField( + input, tag, _internal_metadata_.mutable_unknown_fields())); + break; + } + } + } +success: + // @@protoc_insertion_point(parse_success:milvus.proto.schema.CollectionSchema) + return true; +failure: + // @@protoc_insertion_point(parse_failure:milvus.proto.schema.CollectionSchema) + return false; +#undef DO_ +} +#endif // GOOGLE_PROTOBUF_ENABLE_EXPERIMENTAL_PARSER + +void CollectionSchema::SerializeWithCachedSizes( + ::PROTOBUF_NAMESPACE_ID::io::CodedOutputStream* output) const { + // @@protoc_insertion_point(serialize_start:milvus.proto.schema.CollectionSchema) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // string name = 1; + if (this->name().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->name().data(), static_cast(this->name().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.schema.CollectionSchema.name"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringMaybeAliased( + 1, this->name(), output); + } + + // string description = 2; + if (this->description().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->description().data(), static_cast(this->description().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.schema.CollectionSchema.description"); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringMaybeAliased( + 2, this->description(), output); + } + + // bool auto_id = 3; + if (this->auto_id() != 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteBool(3, this->auto_id(), output); + } + + // repeated .milvus.proto.schema.FieldSchema fields = 4; + for (unsigned int i = 0, + n = static_cast(this->fields_size()); i < n; i++) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteMessageMaybeToArray( + 4, + this->fields(static_cast(i)), + output); + } + + if (_internal_metadata_.have_unknown_fields()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFields( + _internal_metadata_.unknown_fields(), output); + } + // @@protoc_insertion_point(serialize_end:milvus.proto.schema.CollectionSchema) +} + +::PROTOBUF_NAMESPACE_ID::uint8* CollectionSchema::InternalSerializeWithCachedSizesToArray( + ::PROTOBUF_NAMESPACE_ID::uint8* target) const { + // @@protoc_insertion_point(serialize_to_array_start:milvus.proto.schema.CollectionSchema) + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + // string name = 1; + if (this->name().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->name().data(), static_cast(this->name().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.schema.CollectionSchema.name"); + target = + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringToArray( + 1, this->name(), target); + } + + // string description = 2; + if (this->description().size() > 0) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->description().data(), static_cast(this->description().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "milvus.proto.schema.CollectionSchema.description"); + target = + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteStringToArray( + 2, this->description(), target); + } + + // bool auto_id = 3; + if (this->auto_id() != 0) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteBoolToArray(3, this->auto_id(), target); + } + + // repeated .milvus.proto.schema.FieldSchema fields = 4; + for (unsigned int i = 0, + n = static_cast(this->fields_size()); i < n; i++) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessageToArray( + 4, this->fields(static_cast(i)), target); + } + + if (_internal_metadata_.have_unknown_fields()) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::SerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields(), target); + } + // @@protoc_insertion_point(serialize_to_array_end:milvus.proto.schema.CollectionSchema) + return target; +} + +size_t CollectionSchema::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:milvus.proto.schema.CollectionSchema) + size_t total_size = 0; + + if (_internal_metadata_.have_unknown_fields()) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::ComputeUnknownFieldsSize( + _internal_metadata_.unknown_fields()); + } + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // repeated .milvus.proto.schema.FieldSchema fields = 4; + { + unsigned int count = static_cast(this->fields_size()); + total_size += 1UL * count; + for (unsigned int i = 0; i < count; i++) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + this->fields(static_cast(i))); + } + } + + // string name = 1; + if (this->name().size() > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->name()); + } + + // string description = 2; + if (this->description().size() > 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->description()); + } + + // bool auto_id = 3; + if (this->auto_id() != 0) { + total_size += 1 + 1; + } + + int cached_size = ::PROTOBUF_NAMESPACE_ID::internal::ToCachedSize(total_size); + SetCachedSize(cached_size); + return total_size; +} + +void CollectionSchema::MergeFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_merge_from_start:milvus.proto.schema.CollectionSchema) + GOOGLE_DCHECK_NE(&from, this); + const CollectionSchema* source = + ::PROTOBUF_NAMESPACE_ID::DynamicCastToGenerated( + &from); + if (source == nullptr) { + // @@protoc_insertion_point(generalized_merge_from_cast_fail:milvus.proto.schema.CollectionSchema) + ::PROTOBUF_NAMESPACE_ID::internal::ReflectionOps::Merge(from, this); + } else { + // @@protoc_insertion_point(generalized_merge_from_cast_success:milvus.proto.schema.CollectionSchema) + MergeFrom(*source); + } +} + +void CollectionSchema::MergeFrom(const CollectionSchema& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:milvus.proto.schema.CollectionSchema) + GOOGLE_DCHECK_NE(&from, this); + _internal_metadata_.MergeFrom(from._internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::uint32 cached_has_bits = 0; + (void) cached_has_bits; + + fields_.MergeFrom(from.fields_); + if (from.name().size() > 0) { + + name_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.name_); + } + if (from.description().size() > 0) { + + description_.AssignWithDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), from.description_); + } + if (from.auto_id() != 0) { + set_auto_id(from.auto_id()); + } +} + +void CollectionSchema::CopyFrom(const ::PROTOBUF_NAMESPACE_ID::Message& from) { +// @@protoc_insertion_point(generalized_copy_from_start:milvus.proto.schema.CollectionSchema) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +void CollectionSchema::CopyFrom(const CollectionSchema& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:milvus.proto.schema.CollectionSchema) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool CollectionSchema::IsInitialized() const { + return true; +} + +void CollectionSchema::InternalSwap(CollectionSchema* other) { + using std::swap; + _internal_metadata_.Swap(&other->_internal_metadata_); + CastToBase(&fields_)->InternalSwap(CastToBase(&other->fields_)); + name_.Swap(&other->name_, &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); + description_.Swap(&other->description_, &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + GetArenaNoVirtual()); + swap(auto_id_, other->auto_id_); +} + +::PROTOBUF_NAMESPACE_ID::Metadata CollectionSchema::GetMetadata() const { + return GetMetadataStatic(); +} + + +// @@protoc_insertion_point(namespace_scope) +} // namespace schema +} // namespace proto +} // namespace milvus +PROTOBUF_NAMESPACE_OPEN +template<> PROTOBUF_NOINLINE ::milvus::proto::schema::FieldSchema* Arena::CreateMaybeMessage< ::milvus::proto::schema::FieldSchema >(Arena* arena) { + return Arena::CreateInternal< ::milvus::proto::schema::FieldSchema >(arena); +} +template<> PROTOBUF_NOINLINE ::milvus::proto::schema::CollectionSchema* Arena::CreateMaybeMessage< ::milvus::proto::schema::CollectionSchema >(Arena* arena) { + return Arena::CreateInternal< ::milvus::proto::schema::CollectionSchema >(arena); +} +PROTOBUF_NAMESPACE_CLOSE + +// @@protoc_insertion_point(global_scope) +#include diff --git a/internal/core/src/pb/schema.pb.h b/internal/core/src/pb/schema.pb.h new file mode 100644 index 0000000000..6a99d022f0 --- /dev/null +++ b/internal/core/src/pb/schema.pb.h @@ -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 +#include + +#include +#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 +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include // IWYU pragma: export +#include // IWYU pragma: export +#include +#include +#include "common.pb.h" +// @@protoc_insertion_point(includes) +#include +#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 +inline const std::string& DataType_Name(T enum_t_value) { + static_assert(::std::is_same::value || + ::std::is_integral::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_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( + &_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(nullptr); + } + + FieldSchema* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final { + return CreateMaybeMessage(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( + &_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(nullptr); + } + + CollectionSchema* New(::PROTOBUF_NAMESPACE_ID::Arena* arena) const final { + return CreateMaybeMessage(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(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(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(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(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 +#endif // GOOGLE_PROTOBUF_INCLUDED_GOOGLE_PROTOBUF_INCLUDED_schema_2eproto diff --git a/internal/core/src/query/CMakeLists.txt b/internal/core/src/query/CMakeLists.txt index c6ab38448a..7e3f31720f 100644 --- a/internal/core/src/query/CMakeLists.txt +++ b/internal/core/src/query/CMakeLists.txt @@ -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) diff --git a/internal/core/thirdparty/protobuf/CMakeLists.txt b/internal/core/thirdparty/protobuf/CMakeLists.txt index 664c84f71f..93b1aac765 100644 --- a/internal/core/thirdparty/protobuf/CMakeLists.txt +++ b/internal/core/thirdparty/protobuf/CMakeLists.txt @@ -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" ) diff --git a/internal/master/collection/collection.go b/internal/master/collection/collection.go index 8f4b96c81f..286c171646 100644 --- a/internal/master/collection/collection.go +++ b/internal/master/collection/collection.go @@ -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 diff --git a/internal/master/controller/segment.go b/internal/master/controller/segment.go index f46b5e7856..590162cc64 100644 --- a/internal/master/controller/segment.go +++ b/internal/master/controller/segment.go @@ -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 -} diff --git a/internal/master/grpc/server.go b/internal/master/grpc/server.go index 1bb8544480..72850bec18 100644 --- a/internal/master/grpc/server.go +++ b/internal/master/grpc/server.go @@ -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 +//} diff --git a/internal/master/segment/segment.go b/internal/master/segment/segment.go index ad42ac3c1b..fcc396e255 100644 --- a/internal/master/segment/segment.go +++ b/internal/master/segment/segment.go @@ -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"` } diff --git a/internal/master/segment/stats.go b/internal/master/segment/stats.go index 6842855d53..9d97f57d82 100644 --- a/internal/master/segment/stats.go +++ b/internal/master/segment/stats.go @@ -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) } diff --git a/internal/msgclient/msgclient.go b/internal/msgclient/msgclient.go index 8825f60157..006db608d8 100644 --- a/internal/msgclient/msgclient.go +++ b/internal/msgclient/msgclient.go @@ -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{ diff --git a/internal/proto/common.proto b/internal/proto/common.proto index 52942e4b7b..b9b272dace 100644 --- a/internal/proto/common.proto +++ b/internal/proto/common.proto @@ -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; +} + diff --git a/internal/proto/commonpb/common.pb.go b/internal/proto/commonpb/common.pb.go new file mode 100644 index 0000000000..48a09b4707 --- /dev/null +++ b/internal/proto/commonpb/common.pb.go @@ -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, +} diff --git a/internal/proto/etcd_meta.proto b/internal/proto/etcd_meta.proto new file mode 100644 index 0000000000..24f2a25045 --- /dev/null +++ b/internal/proto/etcd_meta.proto @@ -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; +} \ No newline at end of file diff --git a/internal/proto/etcdpb/etcd_meta.pb.go b/internal/proto/etcdpb/etcd_meta.pb.go new file mode 100644 index 0000000000..4db3ce36d3 --- /dev/null +++ b/internal/proto/etcdpb/etcd_meta.pb.go @@ -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, +} diff --git a/internal/proto/internal_msg.proto b/internal/proto/internal_msg.proto new file mode 100644 index 0000000000..2c626a8df7 --- /dev/null +++ b/internal/proto/internal_msg.proto @@ -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; +} \ No newline at end of file diff --git a/internal/proto/internalpb/internal_msg.pb.go b/internal/proto/internalpb/internal_msg.pb.go new file mode 100644 index 0000000000..01cdd8ff1a --- /dev/null +++ b/internal/proto/internalpb/internal_msg.pb.go @@ -0,0 +1,1463 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// source: internal_msg.proto + +package internalpb + +import ( + fmt "fmt" + proto "github.com/golang/protobuf/proto" + commonpb "github.com/zilliztech/milvus-distributed/internal/proto/commonpb" + servicepb "github.com/zilliztech/milvus-distributed/internal/proto/servicepb" + 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 ReqType int32 + +const ( + ReqType_kNone ReqType = 0 + // Definition Requests: collection + ReqType_kCreateCollection ReqType = 100 + ReqType_kDropCollection ReqType = 101 + ReqType_kHasCollection ReqType = 102 + ReqType_kDescribeCollection ReqType = 103 + ReqType_kShowCollections ReqType = 104 + // Definition Requests: partition + ReqType_kCreatePartition ReqType = 200 + ReqType_kDropPartition ReqType = 201 + ReqType_kHasPartition ReqType = 202 + ReqType_kDescribePartition ReqType = 203 + ReqType_kShowPartitions ReqType = 204 + // Manipulation Requests + ReqType_kInsert ReqType = 400 + // Query + ReqType_kSearch ReqType = 500 +) + +var ReqType_name = map[int32]string{ + 0: "kNone", + 100: "kCreateCollection", + 101: "kDropCollection", + 102: "kHasCollection", + 103: "kDescribeCollection", + 104: "kShowCollections", + 200: "kCreatePartition", + 201: "kDropPartition", + 202: "kHasPartition", + 203: "kDescribePartition", + 204: "kShowPartitions", + 400: "kInsert", + 500: "kSearch", +} + +var ReqType_value = map[string]int32{ + "kNone": 0, + "kCreateCollection": 100, + "kDropCollection": 101, + "kHasCollection": 102, + "kDescribeCollection": 103, + "kShowCollections": 104, + "kCreatePartition": 200, + "kDropPartition": 201, + "kHasPartition": 202, + "kDescribePartition": 203, + "kShowPartitions": 204, + "kInsert": 400, + "kSearch": 500, +} + +func (x ReqType) String() string { + return proto.EnumName(ReqType_name, int32(x)) +} + +func (ReqType) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{0} +} + +type CreateCollectionRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + Schema *commonpb.Blob `protobuf:"bytes,5,opt,name=schema,proto3" json:"schema,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CreateCollectionRequest) Reset() { *m = CreateCollectionRequest{} } +func (m *CreateCollectionRequest) String() string { return proto.CompactTextString(m) } +func (*CreateCollectionRequest) ProtoMessage() {} +func (*CreateCollectionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{0} +} + +func (m *CreateCollectionRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_CreateCollectionRequest.Unmarshal(m, b) +} +func (m *CreateCollectionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_CreateCollectionRequest.Marshal(b, m, deterministic) +} +func (m *CreateCollectionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_CreateCollectionRequest.Merge(m, src) +} +func (m *CreateCollectionRequest) XXX_Size() int { + return xxx_messageInfo_CreateCollectionRequest.Size(m) +} +func (m *CreateCollectionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_CreateCollectionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_CreateCollectionRequest proto.InternalMessageInfo + +func (m *CreateCollectionRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *CreateCollectionRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *CreateCollectionRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *CreateCollectionRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *CreateCollectionRequest) GetSchema() *commonpb.Blob { + if m != nil { + return m.Schema + } + return nil +} + +type DropCollectionRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + CollectionName *servicepb.CollectionName `protobuf:"bytes,5,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 *DropCollectionRequest) Reset() { *m = DropCollectionRequest{} } +func (m *DropCollectionRequest) String() string { return proto.CompactTextString(m) } +func (*DropCollectionRequest) ProtoMessage() {} +func (*DropCollectionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{1} +} + +func (m *DropCollectionRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DropCollectionRequest.Unmarshal(m, b) +} +func (m *DropCollectionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DropCollectionRequest.Marshal(b, m, deterministic) +} +func (m *DropCollectionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DropCollectionRequest.Merge(m, src) +} +func (m *DropCollectionRequest) XXX_Size() int { + return xxx_messageInfo_DropCollectionRequest.Size(m) +} +func (m *DropCollectionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DropCollectionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_DropCollectionRequest proto.InternalMessageInfo + +func (m *DropCollectionRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *DropCollectionRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *DropCollectionRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *DropCollectionRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *DropCollectionRequest) GetCollectionName() *servicepb.CollectionName { + if m != nil { + return m.CollectionName + } + return nil +} + +type HasCollectionRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + CollectionName *servicepb.CollectionName `protobuf:"bytes,5,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 *HasCollectionRequest) Reset() { *m = HasCollectionRequest{} } +func (m *HasCollectionRequest) String() string { return proto.CompactTextString(m) } +func (*HasCollectionRequest) ProtoMessage() {} +func (*HasCollectionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{2} +} + +func (m *HasCollectionRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_HasCollectionRequest.Unmarshal(m, b) +} +func (m *HasCollectionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_HasCollectionRequest.Marshal(b, m, deterministic) +} +func (m *HasCollectionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_HasCollectionRequest.Merge(m, src) +} +func (m *HasCollectionRequest) XXX_Size() int { + return xxx_messageInfo_HasCollectionRequest.Size(m) +} +func (m *HasCollectionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_HasCollectionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_HasCollectionRequest proto.InternalMessageInfo + +func (m *HasCollectionRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *HasCollectionRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *HasCollectionRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *HasCollectionRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *HasCollectionRequest) GetCollectionName() *servicepb.CollectionName { + if m != nil { + return m.CollectionName + } + return nil +} + +type DescribeCollectionRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + CollectionName *servicepb.CollectionName `protobuf:"bytes,5,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 *DescribeCollectionRequest) Reset() { *m = DescribeCollectionRequest{} } +func (m *DescribeCollectionRequest) String() string { return proto.CompactTextString(m) } +func (*DescribeCollectionRequest) ProtoMessage() {} +func (*DescribeCollectionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{3} +} + +func (m *DescribeCollectionRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DescribeCollectionRequest.Unmarshal(m, b) +} +func (m *DescribeCollectionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DescribeCollectionRequest.Marshal(b, m, deterministic) +} +func (m *DescribeCollectionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DescribeCollectionRequest.Merge(m, src) +} +func (m *DescribeCollectionRequest) XXX_Size() int { + return xxx_messageInfo_DescribeCollectionRequest.Size(m) +} +func (m *DescribeCollectionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DescribeCollectionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_DescribeCollectionRequest proto.InternalMessageInfo + +func (m *DescribeCollectionRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *DescribeCollectionRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *DescribeCollectionRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *DescribeCollectionRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *DescribeCollectionRequest) GetCollectionName() *servicepb.CollectionName { + if m != nil { + return m.CollectionName + } + return nil +} + +type ShowCollectionRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *ShowCollectionRequest) Reset() { *m = ShowCollectionRequest{} } +func (m *ShowCollectionRequest) String() string { return proto.CompactTextString(m) } +func (*ShowCollectionRequest) ProtoMessage() {} +func (*ShowCollectionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{4} +} + +func (m *ShowCollectionRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ShowCollectionRequest.Unmarshal(m, b) +} +func (m *ShowCollectionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ShowCollectionRequest.Marshal(b, m, deterministic) +} +func (m *ShowCollectionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShowCollectionRequest.Merge(m, src) +} +func (m *ShowCollectionRequest) XXX_Size() int { + return xxx_messageInfo_ShowCollectionRequest.Size(m) +} +func (m *ShowCollectionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ShowCollectionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ShowCollectionRequest proto.InternalMessageInfo + +func (m *ShowCollectionRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *ShowCollectionRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *ShowCollectionRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *ShowCollectionRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +type CreatePartitionRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *CreatePartitionRequest) Reset() { *m = CreatePartitionRequest{} } +func (m *CreatePartitionRequest) String() string { return proto.CompactTextString(m) } +func (*CreatePartitionRequest) ProtoMessage() {} +func (*CreatePartitionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{5} +} + +func (m *CreatePartitionRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_CreatePartitionRequest.Unmarshal(m, b) +} +func (m *CreatePartitionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_CreatePartitionRequest.Marshal(b, m, deterministic) +} +func (m *CreatePartitionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_CreatePartitionRequest.Merge(m, src) +} +func (m *CreatePartitionRequest) XXX_Size() int { + return xxx_messageInfo_CreatePartitionRequest.Size(m) +} +func (m *CreatePartitionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_CreatePartitionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_CreatePartitionRequest proto.InternalMessageInfo + +func (m *CreatePartitionRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *CreatePartitionRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *CreatePartitionRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *CreatePartitionRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *CreatePartitionRequest) GetPartitionName() *servicepb.PartitionName { + if m != nil { + return m.PartitionName + } + return nil +} + +type DropPartitionRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DropPartitionRequest) Reset() { *m = DropPartitionRequest{} } +func (m *DropPartitionRequest) String() string { return proto.CompactTextString(m) } +func (*DropPartitionRequest) ProtoMessage() {} +func (*DropPartitionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{6} +} + +func (m *DropPartitionRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DropPartitionRequest.Unmarshal(m, b) +} +func (m *DropPartitionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DropPartitionRequest.Marshal(b, m, deterministic) +} +func (m *DropPartitionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DropPartitionRequest.Merge(m, src) +} +func (m *DropPartitionRequest) XXX_Size() int { + return xxx_messageInfo_DropPartitionRequest.Size(m) +} +func (m *DropPartitionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DropPartitionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_DropPartitionRequest proto.InternalMessageInfo + +func (m *DropPartitionRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *DropPartitionRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *DropPartitionRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *DropPartitionRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *DropPartitionRequest) GetPartitionName() *servicepb.PartitionName { + if m != nil { + return m.PartitionName + } + return nil +} + +type HasPartitionRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *HasPartitionRequest) Reset() { *m = HasPartitionRequest{} } +func (m *HasPartitionRequest) String() string { return proto.CompactTextString(m) } +func (*HasPartitionRequest) ProtoMessage() {} +func (*HasPartitionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{7} +} + +func (m *HasPartitionRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_HasPartitionRequest.Unmarshal(m, b) +} +func (m *HasPartitionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_HasPartitionRequest.Marshal(b, m, deterministic) +} +func (m *HasPartitionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_HasPartitionRequest.Merge(m, src) +} +func (m *HasPartitionRequest) XXX_Size() int { + return xxx_messageInfo_HasPartitionRequest.Size(m) +} +func (m *HasPartitionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_HasPartitionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_HasPartitionRequest proto.InternalMessageInfo + +func (m *HasPartitionRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *HasPartitionRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *HasPartitionRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *HasPartitionRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *HasPartitionRequest) GetPartitionName() *servicepb.PartitionName { + if m != nil { + return m.PartitionName + } + return nil +} + +type DescribePartitionRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + PartitionName *servicepb.PartitionName `protobuf:"bytes,5,opt,name=partition_name,json=partitionName,proto3" json:"partition_name,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DescribePartitionRequest) Reset() { *m = DescribePartitionRequest{} } +func (m *DescribePartitionRequest) String() string { return proto.CompactTextString(m) } +func (*DescribePartitionRequest) ProtoMessage() {} +func (*DescribePartitionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{8} +} + +func (m *DescribePartitionRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DescribePartitionRequest.Unmarshal(m, b) +} +func (m *DescribePartitionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DescribePartitionRequest.Marshal(b, m, deterministic) +} +func (m *DescribePartitionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DescribePartitionRequest.Merge(m, src) +} +func (m *DescribePartitionRequest) XXX_Size() int { + return xxx_messageInfo_DescribePartitionRequest.Size(m) +} +func (m *DescribePartitionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DescribePartitionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_DescribePartitionRequest proto.InternalMessageInfo + +func (m *DescribePartitionRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *DescribePartitionRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *DescribePartitionRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *DescribePartitionRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *DescribePartitionRequest) GetPartitionName() *servicepb.PartitionName { + if m != nil { + return m.PartitionName + } + return nil +} + +type ShowPartitionRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ProxyId int64 `protobuf:"varint,4,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + CollectionName *servicepb.CollectionName `protobuf:"bytes,5,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 *ShowPartitionRequest) Reset() { *m = ShowPartitionRequest{} } +func (m *ShowPartitionRequest) String() string { return proto.CompactTextString(m) } +func (*ShowPartitionRequest) ProtoMessage() {} +func (*ShowPartitionRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{9} +} + +func (m *ShowPartitionRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_ShowPartitionRequest.Unmarshal(m, b) +} +func (m *ShowPartitionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_ShowPartitionRequest.Marshal(b, m, deterministic) +} +func (m *ShowPartitionRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_ShowPartitionRequest.Merge(m, src) +} +func (m *ShowPartitionRequest) XXX_Size() int { + return xxx_messageInfo_ShowPartitionRequest.Size(m) +} +func (m *ShowPartitionRequest) XXX_DiscardUnknown() { + xxx_messageInfo_ShowPartitionRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_ShowPartitionRequest proto.InternalMessageInfo + +func (m *ShowPartitionRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *ShowPartitionRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *ShowPartitionRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *ShowPartitionRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *ShowPartitionRequest) GetCollectionName() *servicepb.CollectionName { + if m != nil { + return m.CollectionName + } + return nil +} + +type InsertRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + PartitionTag string `protobuf:"bytes,4,opt,name=partition_tag,json=partitionTag,proto3" json:"partition_tag,omitempty"` + SegmentId uint64 `protobuf:"varint,5,opt,name=segment_id,json=segmentId,proto3" json:"segment_id,omitempty"` + ChannelId uint64 `protobuf:"varint,6,opt,name=channel_id,json=channelId,proto3" json:"channel_id,omitempty"` + ProxyId int64 `protobuf:"varint,7,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + Timestamp uint64 `protobuf:"varint,8,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + RowIds []int64 `protobuf:"varint,9,rep,packed,name=row_ids,json=rowIds,proto3" json:"row_ids,omitempty"` + RowData []*commonpb.Blob `protobuf:"bytes,10,rep,name=row_data,json=rowData,proto3" json:"row_data,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *InsertRequest) Reset() { *m = InsertRequest{} } +func (m *InsertRequest) String() string { return proto.CompactTextString(m) } +func (*InsertRequest) ProtoMessage() {} +func (*InsertRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{10} +} + +func (m *InsertRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_InsertRequest.Unmarshal(m, b) +} +func (m *InsertRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_InsertRequest.Marshal(b, m, deterministic) +} +func (m *InsertRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_InsertRequest.Merge(m, src) +} +func (m *InsertRequest) XXX_Size() int { + return xxx_messageInfo_InsertRequest.Size(m) +} +func (m *InsertRequest) XXX_DiscardUnknown() { + xxx_messageInfo_InsertRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_InsertRequest proto.InternalMessageInfo + +func (m *InsertRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *InsertRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *InsertRequest) GetCollectionName() string { + if m != nil { + return m.CollectionName + } + return "" +} + +func (m *InsertRequest) GetPartitionTag() string { + if m != nil { + return m.PartitionTag + } + return "" +} + +func (m *InsertRequest) GetSegmentId() uint64 { + if m != nil { + return m.SegmentId + } + return 0 +} + +func (m *InsertRequest) GetChannelId() uint64 { + if m != nil { + return m.ChannelId + } + return 0 +} + +func (m *InsertRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *InsertRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *InsertRequest) GetRowIds() []int64 { + if m != nil { + return m.RowIds + } + return nil +} + +func (m *InsertRequest) GetRowData() []*commonpb.Blob { + if m != nil { + return m.RowData + } + return nil +} + +type DeleteRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + CollectionName string `protobuf:"bytes,3,opt,name=collection_name,json=collectionName,proto3" json:"collection_name,omitempty"` + ChannelId uint64 `protobuf:"varint,4,opt,name=channel_id,json=channelId,proto3" json:"channel_id,omitempty"` + ProxyId int64 `protobuf:"varint,5,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + Timestamp uint64 `protobuf:"varint,6,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + PrimaryKeys []int64 `protobuf:"varint,7,rep,packed,name=primary_keys,json=primaryKeys,proto3" json:"primary_keys,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *DeleteRequest) Reset() { *m = DeleteRequest{} } +func (m *DeleteRequest) String() string { return proto.CompactTextString(m) } +func (*DeleteRequest) ProtoMessage() {} +func (*DeleteRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{11} +} + +func (m *DeleteRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_DeleteRequest.Unmarshal(m, b) +} +func (m *DeleteRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_DeleteRequest.Marshal(b, m, deterministic) +} +func (m *DeleteRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_DeleteRequest.Merge(m, src) +} +func (m *DeleteRequest) XXX_Size() int { + return xxx_messageInfo_DeleteRequest.Size(m) +} +func (m *DeleteRequest) XXX_DiscardUnknown() { + xxx_messageInfo_DeleteRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_DeleteRequest proto.InternalMessageInfo + +func (m *DeleteRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *DeleteRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *DeleteRequest) GetCollectionName() string { + if m != nil { + return m.CollectionName + } + return "" +} + +func (m *DeleteRequest) GetChannelId() uint64 { + if m != nil { + return m.ChannelId + } + return 0 +} + +func (m *DeleteRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *DeleteRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *DeleteRequest) GetPrimaryKeys() []int64 { + if m != nil { + return m.PrimaryKeys + } + return nil +} + +type SearchRequest struct { + ReqType ReqType `protobuf:"varint,1,opt,name=req_type,json=reqType,proto3,enum=milvus.proto.internal.ReqType" json:"req_type,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + ProxyId int64 `protobuf:"varint,3,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + Timestamp uint64 `protobuf:"varint,4,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ResultChannelId uint64 `protobuf:"varint,5,opt,name=result_channel_id,json=resultChannelId,proto3" json:"result_channel_id,omitempty"` + Query *commonpb.Blob `protobuf:"bytes,6,opt,name=query,proto3" json:"query,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SearchRequest) Reset() { *m = SearchRequest{} } +func (m *SearchRequest) String() string { return proto.CompactTextString(m) } +func (*SearchRequest) ProtoMessage() {} +func (*SearchRequest) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{12} +} + +func (m *SearchRequest) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SearchRequest.Unmarshal(m, b) +} +func (m *SearchRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SearchRequest.Marshal(b, m, deterministic) +} +func (m *SearchRequest) XXX_Merge(src proto.Message) { + xxx_messageInfo_SearchRequest.Merge(m, src) +} +func (m *SearchRequest) XXX_Size() int { + return xxx_messageInfo_SearchRequest.Size(m) +} +func (m *SearchRequest) XXX_DiscardUnknown() { + xxx_messageInfo_SearchRequest.DiscardUnknown(m) +} + +var xxx_messageInfo_SearchRequest proto.InternalMessageInfo + +func (m *SearchRequest) GetReqType() ReqType { + if m != nil { + return m.ReqType + } + return ReqType_kNone +} + +func (m *SearchRequest) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *SearchRequest) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *SearchRequest) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *SearchRequest) GetResultChannelId() uint64 { + if m != nil { + return m.ResultChannelId + } + return 0 +} + +func (m *SearchRequest) GetQuery() *commonpb.Blob { + if m != nil { + return m.Query + } + return nil +} + +type SearchResult struct { + Status *commonpb.Status `protobuf:"bytes,1,opt,name=status,proto3" json:"status,omitempty"` + ReqId uint64 `protobuf:"varint,2,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + ProxyId int64 `protobuf:"varint,3,opt,name=proxy_id,json=proxyId,proto3" json:"proxy_id,omitempty"` + QueryNodeId int64 `protobuf:"varint,4,opt,name=query_node_id,json=queryNodeId,proto3" json:"query_node_id,omitempty"` + Timestamp uint64 `protobuf:"varint,5,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + ResultChannelId uint64 `protobuf:"varint,6,opt,name=result_channel_id,json=resultChannelId,proto3" json:"result_channel_id,omitempty"` + Hits []*servicepb.Hits `protobuf:"bytes,7,rep,name=hits,proto3" json:"hits,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *SearchResult) Reset() { *m = SearchResult{} } +func (m *SearchResult) String() string { return proto.CompactTextString(m) } +func (*SearchResult) ProtoMessage() {} +func (*SearchResult) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{13} +} + +func (m *SearchResult) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SearchResult.Unmarshal(m, b) +} +func (m *SearchResult) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SearchResult.Marshal(b, m, deterministic) +} +func (m *SearchResult) XXX_Merge(src proto.Message) { + xxx_messageInfo_SearchResult.Merge(m, src) +} +func (m *SearchResult) XXX_Size() int { + return xxx_messageInfo_SearchResult.Size(m) +} +func (m *SearchResult) XXX_DiscardUnknown() { + xxx_messageInfo_SearchResult.DiscardUnknown(m) +} + +var xxx_messageInfo_SearchResult proto.InternalMessageInfo + +func (m *SearchResult) GetStatus() *commonpb.Status { + if m != nil { + return m.Status + } + return nil +} + +func (m *SearchResult) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *SearchResult) GetProxyId() int64 { + if m != nil { + return m.ProxyId + } + return 0 +} + +func (m *SearchResult) GetQueryNodeId() int64 { + if m != nil { + return m.QueryNodeId + } + return 0 +} + +func (m *SearchResult) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *SearchResult) GetResultChannelId() uint64 { + if m != nil { + return m.ResultChannelId + } + return 0 +} + +func (m *SearchResult) GetHits() []*servicepb.Hits { + if m != nil { + return m.Hits + } + return nil +} + +type TimeSyncMsg struct { + PeerId int64 `protobuf:"varint,1,opt,name=peer_id,json=peerId,proto3" json:"peer_id,omitempty"` + Timestamp uint64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *TimeSyncMsg) Reset() { *m = TimeSyncMsg{} } +func (m *TimeSyncMsg) String() string { return proto.CompactTextString(m) } +func (*TimeSyncMsg) ProtoMessage() {} +func (*TimeSyncMsg) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{14} +} + +func (m *TimeSyncMsg) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_TimeSyncMsg.Unmarshal(m, b) +} +func (m *TimeSyncMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_TimeSyncMsg.Marshal(b, m, deterministic) +} +func (m *TimeSyncMsg) XXX_Merge(src proto.Message) { + xxx_messageInfo_TimeSyncMsg.Merge(m, src) +} +func (m *TimeSyncMsg) XXX_Size() int { + return xxx_messageInfo_TimeSyncMsg.Size(m) +} +func (m *TimeSyncMsg) XXX_DiscardUnknown() { + xxx_messageInfo_TimeSyncMsg.DiscardUnknown(m) +} + +var xxx_messageInfo_TimeSyncMsg proto.InternalMessageInfo + +func (m *TimeSyncMsg) GetPeerId() int64 { + if m != nil { + return m.PeerId + } + return 0 +} + +func (m *TimeSyncMsg) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +type Key2Seg struct { + RowId int64 `protobuf:"varint,1,opt,name=row_id,json=rowId,proto3" json:"row_id,omitempty"` + PrimaryKey uint64 `protobuf:"varint,2,opt,name=primary_key,json=primaryKey,proto3" json:"primary_key,omitempty"` + Timestamp uint64 `protobuf:"varint,3,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + IsValid bool `protobuf:"varint,4,opt,name=is_valid,json=isValid,proto3" json:"is_valid,omitempty"` + SegmentIds []uint64 `protobuf:"varint,5,rep,packed,name=segment_ids,json=segmentIds,proto3" json:"segment_ids,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Key2Seg) Reset() { *m = Key2Seg{} } +func (m *Key2Seg) String() string { return proto.CompactTextString(m) } +func (*Key2Seg) ProtoMessage() {} +func (*Key2Seg) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{15} +} + +func (m *Key2Seg) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_Key2Seg.Unmarshal(m, b) +} +func (m *Key2Seg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_Key2Seg.Marshal(b, m, deterministic) +} +func (m *Key2Seg) XXX_Merge(src proto.Message) { + xxx_messageInfo_Key2Seg.Merge(m, src) +} +func (m *Key2Seg) XXX_Size() int { + return xxx_messageInfo_Key2Seg.Size(m) +} +func (m *Key2Seg) XXX_DiscardUnknown() { + xxx_messageInfo_Key2Seg.DiscardUnknown(m) +} + +var xxx_messageInfo_Key2Seg proto.InternalMessageInfo + +func (m *Key2Seg) GetRowId() int64 { + if m != nil { + return m.RowId + } + return 0 +} + +func (m *Key2Seg) GetPrimaryKey() uint64 { + if m != nil { + return m.PrimaryKey + } + return 0 +} + +func (m *Key2Seg) GetTimestamp() uint64 { + if m != nil { + return m.Timestamp + } + return 0 +} + +func (m *Key2Seg) GetIsValid() bool { + if m != nil { + return m.IsValid + } + return false +} + +func (m *Key2Seg) GetSegmentIds() []uint64 { + if m != nil { + return m.SegmentIds + } + return nil +} + +type Key2SegMsg struct { + ReqId uint64 `protobuf:"varint,1,opt,name=req_id,json=reqId,proto3" json:"req_id,omitempty"` + Key2Seg []*Key2Seg `protobuf:"bytes,2,rep,name=key2seg,proto3" json:"key2seg,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` +} + +func (m *Key2SegMsg) Reset() { *m = Key2SegMsg{} } +func (m *Key2SegMsg) String() string { return proto.CompactTextString(m) } +func (*Key2SegMsg) ProtoMessage() {} +func (*Key2SegMsg) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{16} +} + +func (m *Key2SegMsg) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_Key2SegMsg.Unmarshal(m, b) +} +func (m *Key2SegMsg) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_Key2SegMsg.Marshal(b, m, deterministic) +} +func (m *Key2SegMsg) XXX_Merge(src proto.Message) { + xxx_messageInfo_Key2SegMsg.Merge(m, src) +} +func (m *Key2SegMsg) XXX_Size() int { + return xxx_messageInfo_Key2SegMsg.Size(m) +} +func (m *Key2SegMsg) XXX_DiscardUnknown() { + xxx_messageInfo_Key2SegMsg.DiscardUnknown(m) +} + +var xxx_messageInfo_Key2SegMsg proto.InternalMessageInfo + +func (m *Key2SegMsg) GetReqId() uint64 { + if m != nil { + return m.ReqId + } + return 0 +} + +func (m *Key2SegMsg) GetKey2Seg() []*Key2Seg { + if m != nil { + return m.Key2Seg + } + return nil +} + +type SegmentStatistics 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"` + NumRows int64 `protobuf:"varint,3,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 *SegmentStatistics) Reset() { *m = SegmentStatistics{} } +func (m *SegmentStatistics) String() string { return proto.CompactTextString(m) } +func (*SegmentStatistics) ProtoMessage() {} +func (*SegmentStatistics) Descriptor() ([]byte, []int) { + return fileDescriptor_7eb37f6b80b23116, []int{17} +} + +func (m *SegmentStatistics) XXX_Unmarshal(b []byte) error { + return xxx_messageInfo_SegmentStatistics.Unmarshal(m, b) +} +func (m *SegmentStatistics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + return xxx_messageInfo_SegmentStatistics.Marshal(b, m, deterministic) +} +func (m *SegmentStatistics) XXX_Merge(src proto.Message) { + xxx_messageInfo_SegmentStatistics.Merge(m, src) +} +func (m *SegmentStatistics) XXX_Size() int { + return xxx_messageInfo_SegmentStatistics.Size(m) +} +func (m *SegmentStatistics) XXX_DiscardUnknown() { + xxx_messageInfo_SegmentStatistics.DiscardUnknown(m) +} + +var xxx_messageInfo_SegmentStatistics proto.InternalMessageInfo + +func (m *SegmentStatistics) GetSegmentId() uint64 { + if m != nil { + return m.SegmentId + } + return 0 +} + +func (m *SegmentStatistics) GetMemorySize() uint64 { + if m != nil { + return m.MemorySize + } + return 0 +} + +func (m *SegmentStatistics) GetNumRows() int64 { + if m != nil { + return m.NumRows + } + return 0 +} + +func init() { + proto.RegisterEnum("milvus.proto.internal.ReqType", ReqType_name, ReqType_value) + proto.RegisterType((*CreateCollectionRequest)(nil), "milvus.proto.internal.CreateCollectionRequest") + proto.RegisterType((*DropCollectionRequest)(nil), "milvus.proto.internal.DropCollectionRequest") + proto.RegisterType((*HasCollectionRequest)(nil), "milvus.proto.internal.HasCollectionRequest") + proto.RegisterType((*DescribeCollectionRequest)(nil), "milvus.proto.internal.DescribeCollectionRequest") + proto.RegisterType((*ShowCollectionRequest)(nil), "milvus.proto.internal.ShowCollectionRequest") + proto.RegisterType((*CreatePartitionRequest)(nil), "milvus.proto.internal.CreatePartitionRequest") + proto.RegisterType((*DropPartitionRequest)(nil), "milvus.proto.internal.DropPartitionRequest") + proto.RegisterType((*HasPartitionRequest)(nil), "milvus.proto.internal.HasPartitionRequest") + proto.RegisterType((*DescribePartitionRequest)(nil), "milvus.proto.internal.DescribePartitionRequest") + proto.RegisterType((*ShowPartitionRequest)(nil), "milvus.proto.internal.ShowPartitionRequest") + proto.RegisterType((*InsertRequest)(nil), "milvus.proto.internal.InsertRequest") + proto.RegisterType((*DeleteRequest)(nil), "milvus.proto.internal.DeleteRequest") + proto.RegisterType((*SearchRequest)(nil), "milvus.proto.internal.SearchRequest") + proto.RegisterType((*SearchResult)(nil), "milvus.proto.internal.SearchResult") + proto.RegisterType((*TimeSyncMsg)(nil), "milvus.proto.internal.TimeSyncMsg") + proto.RegisterType((*Key2Seg)(nil), "milvus.proto.internal.Key2Seg") + proto.RegisterType((*Key2SegMsg)(nil), "milvus.proto.internal.Key2SegMsg") + proto.RegisterType((*SegmentStatistics)(nil), "milvus.proto.internal.SegmentStatistics") +} + +func init() { proto.RegisterFile("internal_msg.proto", fileDescriptor_7eb37f6b80b23116) } + +var fileDescriptor_7eb37f6b80b23116 = []byte{ + // 1020 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x57, 0xcd, 0x6f, 0xe3, 0x44, + 0x14, 0x67, 0x92, 0x26, 0x69, 0x5f, 0x9a, 0xd6, 0x9d, 0x36, 0xd4, 0x5b, 0x3e, 0x36, 0x78, 0x91, + 0xa8, 0x56, 0x22, 0x15, 0x59, 0x0e, 0x70, 0x6d, 0x73, 0x68, 0x58, 0x6d, 0x85, 0x9c, 0x8a, 0x03, + 0x12, 0xb2, 0x1c, 0xfb, 0x91, 0x8c, 0xfc, 0xd9, 0x99, 0x49, 0x8b, 0x7b, 0xe7, 0x8a, 0x90, 0x38, + 0x72, 0xe3, 0xaf, 0xe1, 0xeb, 0xce, 0x3f, 0x01, 0x82, 0x95, 0x40, 0x5c, 0xd1, 0xd8, 0xce, 0x87, + 0xdd, 0x52, 0x10, 0x62, 0xa5, 0x4a, 0xb9, 0xe5, 0xbd, 0x99, 0xbc, 0xf9, 0x7d, 0xcc, 0x3c, 0xcf, + 0x00, 0x65, 0xa1, 0x44, 0x1e, 0xda, 0xbe, 0x15, 0x88, 0x71, 0x37, 0xe6, 0x91, 0x8c, 0x68, 0x3b, + 0x60, 0xfe, 0xe5, 0x54, 0x64, 0x51, 0x77, 0x36, 0xe1, 0x60, 0xd3, 0x89, 0x82, 0x20, 0x0a, 0xb3, + 0xf4, 0xc1, 0x8e, 0x40, 0x7e, 0xc9, 0x1c, 0x5c, 0xfc, 0xcf, 0xf8, 0x89, 0xc0, 0xfe, 0x09, 0x47, + 0x5b, 0xe2, 0x49, 0xe4, 0xfb, 0xe8, 0x48, 0x16, 0x85, 0x26, 0x5e, 0x4c, 0x51, 0x48, 0xfa, 0x3e, + 0xac, 0x73, 0xbc, 0xb0, 0x64, 0x12, 0xa3, 0x4e, 0x3a, 0xe4, 0x70, 0xab, 0xf7, 0x7a, 0xf7, 0xd6, + 0x65, 0xba, 0x26, 0x5e, 0x9c, 0x27, 0x31, 0x9a, 0x0d, 0x9e, 0xfd, 0xa0, 0x6d, 0xa8, 0xab, 0xbf, + 0x32, 0x57, 0xaf, 0x74, 0xc8, 0xe1, 0x9a, 0x59, 0xe3, 0x78, 0x31, 0x70, 0xe9, 0xab, 0xb0, 0x21, + 0x59, 0x80, 0x42, 0xda, 0x41, 0xac, 0x57, 0xd3, 0x91, 0x45, 0x82, 0x3e, 0x80, 0xf5, 0x98, 0x47, + 0x9f, 0x25, 0xea, 0x6f, 0x6b, 0x1d, 0x72, 0x58, 0x35, 0x1b, 0x69, 0x3c, 0x70, 0xe9, 0x3b, 0x50, + 0x17, 0xce, 0x04, 0x03, 0x5b, 0xaf, 0x75, 0xc8, 0x61, 0xb3, 0xf7, 0xa0, 0x08, 0x24, 0x67, 0x79, + 0xec, 0x47, 0x23, 0x33, 0x9f, 0x68, 0x3c, 0x27, 0xd0, 0xee, 0xf3, 0x28, 0xbe, 0xd7, 0xbc, 0x9e, + 0xc1, 0xb6, 0x33, 0xc7, 0x67, 0x85, 0x76, 0x80, 0x39, 0xc1, 0x37, 0x8b, 0x88, 0x72, 0xe3, 0xba, + 0x0b, 0x32, 0x67, 0x76, 0x80, 0xe6, 0x96, 0x53, 0x88, 0x8d, 0xdf, 0x08, 0xec, 0x9d, 0xda, 0x62, + 0x95, 0x28, 0xff, 0x41, 0xe0, 0x41, 0x1f, 0x85, 0xc3, 0xd9, 0x08, 0x57, 0x89, 0xf7, 0x37, 0x04, + 0xda, 0xc3, 0x49, 0x74, 0x75, 0x9f, 0x39, 0x1b, 0xbf, 0x12, 0x78, 0x39, 0xeb, 0x2e, 0x1f, 0xda, + 0x5c, 0xb2, 0x7b, 0xea, 0xcc, 0x07, 0xb0, 0x15, 0xcf, 0xe0, 0x2d, 0x1b, 0xf3, 0xe8, 0x76, 0x63, + 0xe6, 0x54, 0x52, 0x5f, 0x5a, 0xf1, 0x72, 0x68, 0xfc, 0x42, 0x60, 0x4f, 0x75, 0x9d, 0x55, 0xe1, + 0xfb, 0x33, 0x81, 0xdd, 0x53, 0x5b, 0xac, 0x0a, 0xdd, 0xe7, 0x04, 0xf4, 0x59, 0xb7, 0x59, 0x15, + 0xce, 0xea, 0xa3, 0xa2, 0x3a, 0xcd, 0x7d, 0xe6, 0xfb, 0x7f, 0x7f, 0x54, 0x2a, 0xd0, 0x1a, 0x84, + 0x02, 0xb9, 0x7c, 0x71, 0x5c, 0xdf, 0xba, 0x09, 0x59, 0x31, 0xde, 0x28, 0x83, 0xa1, 0x8f, 0x60, + 0x61, 0x88, 0x25, 0xed, 0x71, 0xca, 0x7d, 0xc3, 0xdc, 0x9c, 0x27, 0xcf, 0xed, 0x31, 0x7d, 0x0d, + 0x40, 0xe0, 0x38, 0xc0, 0x50, 0xaa, 0x85, 0x6a, 0x99, 0x74, 0x79, 0x66, 0xe0, 0xaa, 0x61, 0x67, + 0x62, 0x87, 0x21, 0xfa, 0x6a, 0xb8, 0x9e, 0x0d, 0xe7, 0x99, 0x81, 0x5b, 0x50, 0xb6, 0x51, 0x54, + 0xb6, 0x60, 0xc9, 0x7a, 0xd9, 0x92, 0x7d, 0x68, 0xf0, 0xe8, 0xca, 0x62, 0xae, 0xd0, 0x37, 0x3a, + 0xd5, 0xc3, 0xaa, 0x59, 0xe7, 0xd1, 0xd5, 0xc0, 0x15, 0xf4, 0x5d, 0x58, 0x57, 0x03, 0xae, 0x2d, + 0x6d, 0x1d, 0x3a, 0xd5, 0xbb, 0xaf, 0x6c, 0xaa, 0x46, 0xdf, 0x96, 0xb6, 0xf1, 0x79, 0x05, 0x5a, + 0x7d, 0xf4, 0x51, 0xe2, 0x3d, 0xd0, 0xbd, 0xa8, 0xd9, 0xda, 0x5d, 0x9a, 0xd5, 0xee, 0xd0, 0xac, + 0x5e, 0xd6, 0xec, 0x0d, 0xd8, 0x8c, 0x39, 0x0b, 0x6c, 0x9e, 0x58, 0x1e, 0x26, 0x42, 0x6f, 0xa4, + 0xc2, 0x35, 0xf3, 0xdc, 0x53, 0x4c, 0x84, 0xf1, 0x27, 0x81, 0xd6, 0x10, 0x6d, 0xee, 0x4c, 0x5e, + 0x9c, 0x0e, 0xcb, 0xf8, 0xab, 0x77, 0xe0, 0x5f, 0x2b, 0xe3, 0x7f, 0x0c, 0x3b, 0x1c, 0xc5, 0xd4, + 0x97, 0xd6, 0x92, 0x3c, 0xd9, 0x8e, 0xdb, 0xce, 0x06, 0x4e, 0xe6, 0x22, 0x1d, 0x41, 0xed, 0x62, + 0x8a, 0x3c, 0x49, 0x55, 0xb8, 0x73, 0x0f, 0x64, 0xf3, 0x8c, 0xaf, 0x2a, 0xb0, 0x39, 0x63, 0xae, + 0x4a, 0xd1, 0x27, 0x50, 0x17, 0xd2, 0x96, 0x53, 0x91, 0xd2, 0x6e, 0xf6, 0x5e, 0xb9, 0xb5, 0xc4, + 0x30, 0x9d, 0x62, 0xe6, 0x53, 0xff, 0x03, 0x65, 0x03, 0x5a, 0x29, 0x00, 0x2b, 0x8c, 0x5c, 0x5c, + 0x34, 0x98, 0x66, 0x9a, 0x3c, 0x8b, 0x5c, 0x2c, 0xcb, 0x52, 0xfb, 0x57, 0xb2, 0xd4, 0x6f, 0x97, + 0xa5, 0x0b, 0x6b, 0x13, 0x26, 0x33, 0xeb, 0x9b, 0xbd, 0x83, 0xdb, 0x7b, 0xd4, 0x29, 0x93, 0xc2, + 0x4c, 0xe7, 0x19, 0x7d, 0x68, 0x9e, 0xb3, 0x00, 0x87, 0x49, 0xe8, 0x3c, 0x13, 0x63, 0x75, 0xea, + 0x62, 0x44, 0xae, 0x16, 0x20, 0x29, 0xcc, 0xba, 0x0a, 0xcb, 0x08, 0x2b, 0x25, 0x84, 0xc6, 0xd7, + 0x04, 0x1a, 0x4f, 0x31, 0xe9, 0x0d, 0x71, 0x9c, 0x2a, 0x94, 0x1e, 0xdc, 0xbc, 0x42, 0x2d, 0x3d, + 0xb7, 0xf4, 0x21, 0x34, 0x97, 0xf6, 0x66, 0x5e, 0x02, 0x16, 0x5b, 0xf3, 0x9f, 0x3b, 0x34, 0x13, + 0xd6, 0xa5, 0xed, 0xe7, 0x02, 0xae, 0x9b, 0x0d, 0x26, 0x3e, 0x52, 0xa1, 0xaa, 0xbc, 0x68, 0x50, + 0x42, 0xaf, 0x75, 0xaa, 0xaa, 0xf2, 0xbc, 0x43, 0x09, 0xe3, 0x13, 0x80, 0x1c, 0x9c, 0xa2, 0xb8, + 0x70, 0x90, 0x2c, 0x3b, 0xf8, 0x1e, 0x34, 0x3c, 0x4c, 0x7a, 0x02, 0xc7, 0x7a, 0x25, 0xd5, 0xee, + 0xef, 0x4e, 0x41, 0x5e, 0xca, 0x9c, 0x4d, 0x37, 0x42, 0xd8, 0x19, 0x66, 0x8b, 0xa9, 0xbd, 0xc2, + 0x84, 0x64, 0x8e, 0x28, 0x75, 0x4d, 0x52, 0xee, 0x9a, 0x0f, 0xa1, 0x19, 0x60, 0x10, 0xf1, 0xc4, + 0x12, 0xec, 0x1a, 0x67, 0x6a, 0x64, 0xa9, 0x21, 0xbb, 0x46, 0xc5, 0x37, 0x9c, 0x06, 0x16, 0x8f, + 0xae, 0xc4, 0x6c, 0x43, 0x85, 0xd3, 0xc0, 0x8c, 0xae, 0xc4, 0xe3, 0x2f, 0x2a, 0xd0, 0xc8, 0x8f, + 0x22, 0xdd, 0x80, 0x9a, 0x77, 0x16, 0x85, 0xa8, 0xbd, 0x44, 0xdb, 0xb0, 0xe3, 0x95, 0xdf, 0xdb, + 0x9a, 0x4b, 0x77, 0x61, 0xdb, 0x2b, 0x3e, 0x56, 0x35, 0xa4, 0x14, 0xb6, 0xbc, 0xc2, 0x6b, 0x4e, + 0xfb, 0x94, 0xee, 0xc3, 0xae, 0x77, 0xf3, 0xb9, 0xa3, 0x8d, 0xe9, 0x1e, 0x68, 0x5e, 0xf1, 0x3d, + 0x20, 0xb4, 0x09, 0x6d, 0x83, 0xe6, 0x95, 0x2e, 0xe0, 0xda, 0xb7, 0x84, 0xee, 0xc2, 0x96, 0x57, + 0xb8, 0xa5, 0x6a, 0xdf, 0x11, 0x4a, 0xa1, 0xe5, 0x2d, 0x5f, 0xe5, 0xb4, 0xef, 0x09, 0xdd, 0x07, + 0xea, 0xdd, 0xb8, 0xef, 0x68, 0x3f, 0x10, 0xba, 0x07, 0xdb, 0x5e, 0xe1, 0x52, 0x20, 0xb4, 0x1f, + 0x09, 0xdd, 0x84, 0x86, 0x97, 0x7d, 0x37, 0xb5, 0x2f, 0xab, 0x69, 0x94, 0x9d, 0x65, 0xed, 0xf7, + 0xea, 0x71, 0xff, 0xe3, 0xe3, 0x31, 0x93, 0x93, 0xe9, 0x48, 0x9d, 0xd9, 0xa3, 0x6b, 0xe6, 0xfb, + 0xec, 0x5a, 0xa2, 0x33, 0x39, 0xca, 0x0c, 0x7c, 0xdb, 0x65, 0x42, 0x72, 0x36, 0x9a, 0x4a, 0x74, + 0x8f, 0x66, 0x36, 0x1e, 0xa5, 0xae, 0xce, 0xc3, 0x78, 0x34, 0xaa, 0xa7, 0x99, 0x27, 0x7f, 0x05, + 0x00, 0x00, 0xff, 0xff, 0xf3, 0x38, 0x9d, 0x4f, 0x04, 0x11, 0x00, 0x00, +} diff --git a/internal/proto/master.proto b/internal/proto/master.proto index 2bdd433fbb..81093d55e4 100644 --- a/internal/proto/master.proto +++ b/internal/proto/master.proto @@ -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) {} +} \ No newline at end of file diff --git a/internal/proto/master/master.pb.go b/internal/proto/master/master.pb.go deleted file mode 100644 index 7ec3de1505..0000000000 --- a/internal/proto/master/master.pb.go +++ /dev/null @@ -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", -} diff --git a/internal/proto/masterpb/master.pb.go b/internal/proto/masterpb/master.pb.go new file mode 100644 index 0000000000..c14e4de92a --- /dev/null +++ b/internal/proto/masterpb/master.pb.go @@ -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", +} diff --git a/internal/proto/schema.proto b/internal/proto/schema.proto index 07089c0680..7b04177b2e 100644 --- a/internal/proto/schema.proto +++ b/internal/proto/schema.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; +} \ No newline at end of file diff --git a/internal/proto/schemapb/schema.pb.go b/internal/proto/schemapb/schema.pb.go new file mode 100644 index 0000000000..d708a0dbb0 --- /dev/null +++ b/internal/proto/schemapb/schema.pb.go @@ -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, +} diff --git a/internal/proto/service.proto b/internal/proto/service.proto index b996a2f116..20b5f0cf29 100644 --- a/internal/proto/service.proto +++ b/internal/proto/service.proto @@ -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. diff --git a/internal/proto/service_msg.proto b/internal/proto/service_msg.proto new file mode 100644 index 0000000000..0d72f7de0a --- /dev/null +++ b/internal/proto/service_msg.proto @@ -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; +} + diff --git a/internal/proto/servicepb/service.pb.go b/internal/proto/servicepb/service.pb.go new file mode 100644 index 0000000000..7c471c891a --- /dev/null +++ b/internal/proto/servicepb/service.pb.go @@ -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", +} diff --git a/internal/proto/servicepb/service_msg.pb.go b/internal/proto/servicepb/service_msg.pb.go new file mode 100644 index 0000000000..1d5a3ede20 --- /dev/null +++ b/internal/proto/servicepb/service_msg.pb.go @@ -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, +} diff --git a/internal/proxy/query_req.go b/internal/proxy/query_req.go index a35de83edf..f2cdea3641 100644 --- a/internal/proxy/query_req.go +++ b/internal/proxy/query_req.go @@ -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, } } diff --git a/internal/proxy/server.go b/internal/proxy/server.go index 5c1293c680..905dec68c1 100644 --- a/internal/proxy/server.go +++ b/internal/proxy/server.go @@ -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 diff --git a/internal/reader/index.go b/internal/reader/index.go index 69ed57b30a..01e3846150 100644 --- a/internal/reader/index.go +++ b/internal/reader/index.go @@ -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 diff --git a/internal/reader/result.go b/internal/reader/result.go index 46db3c6363..17610e5dcb 100644 --- a/internal/reader/result.go +++ b/internal/reader/result.go @@ -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} } diff --git a/internal/reader/segment_service.go b/internal/reader/segment_service.go index 28edacd599..29c3942858 100644 --- a/internal/reader/segment_service.go +++ b/internal/reader/segment_service.go @@ -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") } } diff --git a/internal/timesync/timesync.go b/internal/timesync/timesync.go index 64dfd479aa..6b1a9b95e6 100644 --- a/internal/timesync/timesync.go +++ b/internal/timesync/timesync.go @@ -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 { diff --git a/scripts/proto_gen_go.sh b/scripts/proto_gen_go.sh index 4412ff9c5a..59ffd75a68 100755 --- a/scripts/proto_gen_go.sh +++ b/scripts/proto_gen_go.sh @@ -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